From cda3bd10c61514d4b489edb088b2be4c83cfd898 Mon Sep 17 00:00:00 2001 From: Mike Percy Date: Mon, 24 Jun 2013 17:18:55 -0700 Subject: [PATCH 001/341] FLUME-2108. Corrections to CHANGELOG for 1.4.0 release. (Mike Percy) --- CHANGELOG | 100 +++--------------------------------------------------- 1 file changed, 4 insertions(+), 96 deletions(-) diff --git a/CHANGELOG b/CHANGELOG index 00b95b4e4d..edbc4828d4 100644 --- a/CHANGELOG +++ b/CHANGELOG @@ -1,47 +1,23 @@ Release Notes - Flume - Version v1.4.0 ** New Feature - * [FLUME-917] - Thrift Support * [FLUME-924] - Implement a JMS source for Flume NG * [FLUME-997] - Support secure transport mechanism * [FLUME-1170] - Create a framework for transaction listeners - * [FLUME-1249] - Add start/stop script - * [FLUME-1485] - FileChannel should support checksum * [FLUME-1502] - Support for running simple configurations embedded in host process * [FLUME-1516] - FileChannel Write Dual Checkpoints to avoid replays * [FLUME-1632] - Persist progress on each file in file spooling client/source - * [FLUME-1669] - Add support for columnar event serializer in HDFS - * [FLUME-1715] - OpenTSDB Sink + * [FLUME-1735] - Add support for a plugins.d directory * [FLUME-1894] - Implement Thrift RPC * [FLUME-1917] - FileChannel group commit (coalesce fsync) - * [FLUME-1941] - Support defaults or inheritance in configs - * [FLUME-1988] - Add Support for Additional Deserializers for SpoolingDirectorySource - * [FLUME-1998] - FlumeRESTClient - A HTTP Client for Interacting with REST Servers - * [FLUME-1999] - GooglePlusSource - An EventDrivenSource for for Streaming In Data from Google+ - * [FLUME-2000] - EmailSource - An EventDrivenSource for Streaming In Events from POP/IMAP Mailboxes - * [FLUME-2001] - TwitterSource - An EventDrivenSource for Streaming In Data from Twitter * [FLUME-2004] - Need to capture metrics on the Flume exec source such as events received, rejected, etc. * [FLUME-2010] - Support Avro records in Log4jAppender and the HDFS Sink * [FLUME-2048] - Avro container file deserializer + * [FLUME-2070] - Add a Flume Morphline Solr Sink ** Improvement - * [FLUME-841] - Generate javadoc from NG build - * [FLUME-890] - Add unit tests for flume ng channels - * [FLUME-1033] - Create system tests for flume - * [FLUME-1071] - Evaluate performance of AVRO source with multiple clients - * [FLUME-1076] - SInk batch sizes vary wildy - * [FLUME-1081] - Load testing tool for Avro source + * [FLUME-1076] - Sink batch sizes vary wildy * [FLUME-1100] - HDFSWriterFactory and HDFSFormatterFactory should allow extension - * [FLUME-1141] - Avro compression unit test should fail if compression does not work - * [FLUME-1159] - Add reconfigurability support to components - * [FLUME-1178] - Improve performance of SyslogTcpSource - * [FLUME-1209] - Support file name extension for HDFS sink - * [FLUME-1237] - Refactor configuration and add more tests. - * [FLUME-1274] - Clarify semantics of uncaught runtime exceptions thrown from configure() method - * [FLUME-1278] - Flume hot-reconfiguration specification - * [FLUME-1413] - Document of Channel (especially FileChannel) and Transaction - * [FLUME-1497] - Make the TestFileChannel.testInOut() more deterministic - * [FLUME-1521] - Document the StressSource * [FLUME-1571] - Channels should check for positive capacity and transaction capacity values * [FLUME-1586] - File Channel should support verifying integrity of individual events. * [FLUME-1652] - Logutils.getLogs could NPE @@ -49,16 +25,11 @@ Release Notes - Flume - Version v1.4.0 * [FLUME-1677] - Add File-channel dependency to flume-ng-node's pom.xml * [FLUME-1699] - Make the rename of the meta file platform neutral * [FLUME-1702] - HDFSEventSink should write to a hidden file as opposed to a .tmp file - * [FLUME-1722] - License file corrections - * [FLUME-1735] - Add support for a plugins.d directory * [FLUME-1740] - Remove contrib/ directory from Flume NG * [FLUME-1745] - FlumeConfiguration Eats Exceptions * [FLUME-1756] - Avro client should be able to use load balancing RPC * [FLUME-1757] - Improve configuration of hbase serializers - * [FLUME-1759] - make file and dirname of flume log tweakable in log4j - * [FLUME-1760] - an extra hdfs. prefix feels redundant in HDFS sink configuration * [FLUME-1762] - File Channel should recover automatically if the checkpoint is incomplete or bad by deleting the contents of the checkpoint directory - * [FLUME-1764] - File_roll type Sink how to achieve according to multiple files dynamically generated directory, and the file on the corresponding directory * [FLUME-1768] - Multiplexing channel selector should allow optional-only channels * [FLUME-1769] - Replicating channel selector should support optional channels * [FLUME-1770] - Flume should have serializer which supports serializer the headers to a simple string @@ -66,7 +37,6 @@ Release Notes - Flume - Version v1.4.0 * [FLUME-1790] - Commands in EncryptionTestUtils comments require high encryption pack to be installed * [FLUME-1794] - FileChannel check for full disks in the background * [FLUME-1800] - Docs for spooling source durability changes - * [FLUME-1801] - Configurable batch size for spooling client * [FLUME-1808] - ElasticSearchSink is missing log4.properties * [FLUME-1821] - Support configuration of hbase instances to be used in AsyncHBaseSink from flume config * [FLUME-1847] - NPE in SourceConfiguration @@ -83,7 +53,6 @@ Release Notes - Flume - Version v1.4.0 * [FLUME-1926] - Optionally timeout Avro Sink Rpc Clients to avoid stickiness * [FLUME-1940] - Log a snapshot of Flume metrics on shutdown * [FLUME-1945] - HBase Serializer allow key from regular expression group - * [FLUME-1961] - Enhancements to the Channel interface * [FLUME-1976] - JMS Source document should provide instruction on JMS implementation jars * [FLUME-1977] - JMS Source connectionFactory property is not documented * [FLUME-1992] - ElasticSearch dependency is marked optional @@ -97,47 +66,24 @@ Release Notes - Flume - Version v1.4.0 * [FLUME-2049] - Compile ElasticSearchSink with elasticsearch 0.90 * [FLUME-2062] - make it possible for HBase sink to deposit event headers into corresponding column qualifiers * [FLUME-2063] - Add Configurable charset to RegexHbaseEventSerializer - * [FLUME-2070] - Add a Flume Morphline Solr Sink + * [FLUME-2076] - JMX metrics support for HTTP Source * [FLUME-2093] - binary tarball that is created by flume's assembly shouldn't contain sources * [FLUME-2100] - Increase default batchSize of Morphline Solr Sink * [FLUME-2105] - Add docs for MorphlineSolrSink ** Bug - * [FLUME-981] - Unify sink failure behavior, throwing EventDeliveryException - * [FLUME-1000] - Properties validator misses an invalid config - * [FLUME-1086] - duplicate lib in Flume's lib dir - * [FLUME-1093] - HDFSEventSink gives no indication of failure when Hadoop jars aren't in the classpath * [FLUME-1110] - HDFS Sink throws IllegalStateException when flume-daemon shuts down - * [FLUME-1115] - project "flume-ng-node" should depends on "flume-recoverable-memory-channel" - * [FLUME-1152] - flume-ng script has issue with hadoop 0.23 script * [FLUME-1153] - flume-ng script is missing some agent options in help output - * [FLUME-1161] - Logging output says name:null after initiating agent shutdown - * [FLUME-1165] - Separate the component factories into pools which hold state and factories which simply create the new components * [FLUME-1175] - RollingFileSink complains of Bad File Descriptor upon a reconfig event - * [FLUME-1203] - Hbase Sink needs to check for partial failures. - * [FLUME-1206] - Hbase sink should pull in dependencies based on profiles - * [FLUME-1233] - HDFS Sink has problem with %c escape sequence in bucket path * [FLUME-1262] - Move doc generation to a different profile - * [FLUME-1283] - Flume HDFS Sink can't generate file name containing space and : which mentioned by %c * [FLUME-1285] - FileChannel has a dependency on Hadoop IO classes - * [FLUME-1291] - FileChannel log close calls interrupt on current thread and does not kill worker thread immediately * [FLUME-1296] - Lifecycle supervisor should check if the monitor service is still running before supervising - * [FLUME-1308] - HDFS Sink throws DFSOutputStream when exception when maxOpenFiles=1 - * [FLUME-1321] - BasicTransactionSemantics should never throw from close() - * [FLUME-1378] - Sink falls into infinite lifecycle with EventDeliveryException - * [FLUME-1403] - File channel throws logFile is null exception in channel.take - * [FLUME-1411] - Add average events per second params to MBeans - * [FLUME-1481] - Custom component may not support dynamic configuration * [FLUME-1511] - Scribe-source doesn't handle zero message request correctly. - * [FLUME-1551] - Create a home for custom Flume components - * [FLUME-1623] - Add more info about failover sink processor * [FLUME-1676] - ExecSource should provide a configurable charset * [FLUME-1688] - Bump AsyncHBase version to 1.4.1 * [FLUME-1709] - HDFS CompressedDataStream doesn't support serializer parameter * [FLUME-1720] - LICENSE file contain entry for protobuf-.jar, however proper artifact name is protobuf-java-.jar - * [FLUME-1721] - SpoolDirectorySource stop should shutdown executor * [FLUME-1731] - SpoolableDirectorySource should have configurable support for deleting files it has already completed instead of renaming - * [FLUME-1732] - Build is failing due to netty problems * [FLUME-1741] - ElasticSearch tests leave directory data/elasticsearch/nodes/ lying around * [FLUME-1748] - HDFS Sink should check if the thread is interrupted before performing any HDFS operations * [FLUME-1755] - Load balancing RPC client has issues with downed hosts @@ -158,7 +104,6 @@ Release Notes - Flume - Version v1.4.0 * [FLUME-1803] - Generated dist tarball is missing flume-ng-embedded-agent * [FLUME-1804] - JMS source not included in binary dist * [FLUME-1805] - Embedded agent deps should be specified in dependencyManagement section of pom - * [FLUME-1815] - File Channel auto-recovery when checkpoint is bad should be optional * [FLUME-1818] - Support various layouts in log4jappender * [FLUME-1819] - ExecSource don't flush the cache if there is no input entries * [FLUME-1820] - Should not be possible for RPC client to block indefinitely on close() @@ -190,7 +135,6 @@ Release Notes - Flume - Version v1.4.0 * [FLUME-1911] - Add deprecation back to the legacy thrift code * [FLUME-1916] - HDFS sink should poll for # of active replicas. If less than required, roll the file. * [FLUME-1918] - File Channel cannot handle capacity of more than 500 Million events - * [FLUME-1919] - Consider adding areYouOK() RPC calls to Avro & Thrift RPC mechanisms * [FLUME-1922] - HDFS Sink should optionally insert the timestamp at the sink * [FLUME-1924] - Bug in serializer context parsing in RollingFileSink * [FLUME-1925] - HDFS timeouts should not starve other threads @@ -198,11 +142,9 @@ Release Notes - Flume - Version v1.4.0 * [FLUME-1930] - Inflights should clean up executors on close. * [FLUME-1931] - HDFS Sink has a commons-lang dependency which is missing in pom * [FLUME-1932] - no-reload-conf command line param does not work - * [FLUME-1936] - Configure max rolls for under-replicated blocks * [FLUME-1937] - Issue with maxUnderReplication in HDFS sink * [FLUME-1939] - FlumeEventQueue must check if file is open before setting the length of the file * [FLUME-1943] - ExecSource tests failing on Jenkins - * [FLUME-1944] - parameterising the dependency variables in pom file. * [FLUME-1948] - plugins.d directory(ies) should be separately overridable, independent of FLUME_HOME * [FLUME-1949] - Documentation for sink processor lists incorrect default * [FLUME-1955] - fileSuffix does not work with compressed streams @@ -233,7 +175,6 @@ Release Notes - Flume - Version v1.4.0 * [FLUME-2054] - Support Version Info on Windows and fix failure of TestVersionInfo * [FLUME-2057] - Failures in FileChannel's TestEventQueueBackingStoreFactory on Windows * [FLUME-2060] - Failure in TestLog.testReplaySucceedsWithUnusedEmptyLogMetaDataFastReplay test on Windows - * [FLUME-2068] - File Channel issue - recovering from BadCheckpoint exception on Windows * [FLUME-2072] - JMX metrics support for HBase Sink * [FLUME-2081] - JMX metrics support for SpoolDir * [FLUME-2082] - JMX support for Seq Generator Source @@ -241,14 +182,7 @@ Release Notes - Flume - Version v1.4.0 * [FLUME-2098] - Make Solr sink depend on the CDK version of morphlines ** Documentation - * [FLUME-1014] - Agent Config docs are out of date - * [FLUME-1022] - Update Flume NG Getting Started Guide to mention pulling from trunk instead of flume-728 - * [FLUME-1144] - Update Flume User Guide with a section on Flume properties such as flume.called.from.service - * [FLUME-1592] - Remove appendTimeout from HDFS sink docs - * [FLUME-1594] - Document JMX options * [FLUME-1621] - Document new MemoryChannel parameters in Flume User Guide - * [FLUME-1624] - Docs could be more consistent when giving examples of types - * [FLUME-1833] - Embedded Agent docs are missing from website * [FLUME-1910] - Add thrift RPC documentation * [FLUME-1953] - Fix dev guide error that says sink can read from multiple channels * [FLUME-1962] - Document proper specification of lzo codec as lzop in Flume User Guide @@ -256,42 +190,16 @@ Release Notes - Flume - Version v1.4.0 * [FLUME-2030] - Documentation of Configuration Changes JMSSource, HBaseSink, AsyncHBaseSink and ElasticSearchSink ** Task - * [FLUME-813] - Build Flume site from Maven - * [FLUME-1051] - Configuration System - * [FLUME-1618] - Make Flume NG build and tests work with Hadoop 2.0 * [FLUME-1686] - Exclude target directories & Eclipse files from rat checks - * [FLUME-2076] - JMX metrics support for HTTP Source * [FLUME-2094] - Remove the deprecated - Recoverable Memory Channel ** Sub-task - * [FLUME-731] - Create NG internal diagrams - * [FLUME-732] - Create NG developer code tour docs - * [FLUME-733] - Define NG master interfaces and classes - * [FLUME-738] - Create NG system configuration component - * [FLUME-749] - Add support for monitoring APIs - * [FLUME-1056] - AvroSource configuration - * [FLUME-1057] - AvroSink configuration - * [FLUME-1058] - Exec source configuration - * [FLUME-1059] - failover sink configuration - * [FLUME-1060] - Hdfs sink configuration - * [FLUME-1061] - jdbc channel configuration - * [FLUME-1062] - Memory Channel configuration - * [FLUME-1063] - Miscellanous changes related to imports and minor changes related to config changes - * [FLUME-1064] - Channel selector configuration - * [FLUME-1065] - netcat source configuration - * [FLUME-1066] - PseudoTxnMemoryChannel configuration - * [FLUME-1067] - RollingFileSink configuration - * [FLUME-1068] - syslog source configuration - * [FLUME-1069] - FileChannel configuration * [FLUME-1626] - Support Hbase security in Hbase sink * [FLUME-1630] - Flume configuration code could be improved * [FLUME-1674] - Documentation / Wiki * [FLUME-1896] - Implement Thrift RpcClient * [FLUME-1897] - Implement Thrift Sink * [FLUME-1898] - Implement Thrift Source - * [FLUME-1989] - Source Code and Unit Tests for org.apache.flume.serialization.RegexDelimiterDeSerializer - * [FLUME-1990] - Source Code and Unit Tests for org.apache.flume.serialization.CharSequenceDelimiterDeSerializer - * [FLUME-1991] - Documentation for New Deserializers for SpoolingDirectorySource * [FLUME-2102] - Update LICENSE file for Flume 1.4.0 ---- From 4cb3b6d8ee617138aba4e93ddc51aafe71209ada Mon Sep 17 00:00:00 2001 From: Mike Percy Date: Thu, 27 Jun 2013 22:32:04 -0700 Subject: [PATCH 002/341] FLUME-2088. Minor typo in Flume User Guide JSON Reporting section. (Ashish Paliwal via Mike Percy) --- flume-ng-doc/sphinx/FlumeUserGuide.rst | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/flume-ng-doc/sphinx/FlumeUserGuide.rst b/flume-ng-doc/sphinx/FlumeUserGuide.rst index e810d3d546..63cad2191c 100644 --- a/flume-ng-doc/sphinx/FlumeUserGuide.rst +++ b/flume-ng-doc/sphinx/FlumeUserGuide.rst @@ -2936,7 +2936,7 @@ Property Name Default Description port 41414 The port to start the server on. ======================= ======= ===================================================================================== -We can start Flume with Ganglia support as follows:: +We can start Flume with JSON Reporting support as follows:: $ bin/flume-ng agent --conf-file example.conf --name a1 -Dflume.monitoring.type=http -Dflume.monitoring.port=34545 From d66bf94b1dd059bc7e4b1ff332be59a280498077 Mon Sep 17 00:00:00 2001 From: Mike Percy Date: Fri, 28 Jun 2013 13:59:33 -0700 Subject: [PATCH 003/341] FLUME-2110. Scribe Source must check if category is null before inserting the headers. (Hari Shreedharan via Mike Percy) --- .../java/org/apache/flume/source/scribe/ScribeSource.java | 6 +++++- 1 file changed, 5 insertions(+), 1 deletion(-) diff --git a/flume-ng-sources/flume-scribe-source/src/main/java/org/apache/flume/source/scribe/ScribeSource.java b/flume-ng-sources/flume-scribe-source/src/main/java/org/apache/flume/source/scribe/ScribeSource.java index b3a8de30af..f9a14c1c4c 100644 --- a/flume-ng-sources/flume-scribe-source/src/main/java/org/apache/flume/source/scribe/ScribeSource.java +++ b/flume-ng-sources/flume-scribe-source/src/main/java/org/apache/flume/source/scribe/ScribeSource.java @@ -149,7 +149,11 @@ public ResultCode Log(List list) throws TException { for (LogEntry entry : list) { Map headers = new HashMap(1, 1); - headers.put(SCRIBE_CATEGORY, entry.getCategory()); + String category = entry.getCategory(); + + if (category != null) { + headers.put(SCRIBE_CATEGORY, category); + } Event event = EventBuilder.withBody(entry.getMessage().getBytes(), headers); events.add(event); From afc806f2eb01a78f9fa25dd38118df15ac6274d4 Mon Sep 17 00:00:00 2001 From: Hari Shreedharan Date: Wed, 10 Jul 2013 17:58:54 -0700 Subject: [PATCH 004/341] FLUME-2124. Upgrade Morphline Solr Sink to CDK 1.4.1 (Wolfgang Hoschek via Hari Shreedharan) --- flume-ng-doc/sphinx/FlumeUserGuide.rst | 4 ++-- flume-ng-sinks/flume-ng-morphline-solr-sink/pom.xml | 8 +++++++- 2 files changed, 9 insertions(+), 3 deletions(-) diff --git a/flume-ng-doc/sphinx/FlumeUserGuide.rst b/flume-ng-doc/sphinx/FlumeUserGuide.rst index 63cad2191c..fb425288ad 100644 --- a/flume-ng-doc/sphinx/FlumeUserGuide.rst +++ b/flume-ng-doc/sphinx/FlumeUserGuide.rst @@ -1835,7 +1835,7 @@ This sink extracts data from Flume events, transforms it, and loads it in near-r This sink is well suited for use cases that stream raw data into HDFS (via the HdfsSink) and simultaneously extract, transform and load the same data into Solr (via MorphlineSolrSink). In particular, this sink can process arbitrary heterogeneous raw data from disparate data sources and turn it into a data model that is useful to Search applications. -The ETL functionality is customizable using a `morphline configuration file `_ that defines a chain of transformation commands that pipe event records from one command to another. +The ETL functionality is customizable using a `morphline configuration file `_ that defines a chain of transformation commands that pipe event records from one command to another. Morphlines can be seen as an evolution of Unix pipelines where the data model is generalized to work with streams of generic records, including arbitrary binary payloads. A morphline command is a bit like a Flume Interceptor. Morphlines can be embedded into Hadoop components such as Flume. @@ -2595,7 +2595,7 @@ prefix "" The prefix string constant to prepend to each generat Morphline Interceptor ~~~~~~~~~~~~~~~~~~~~~~~~~~~ -This interceptor filters the events through a `morphline configuration file `_ that defines a chain of transformation commands that pipe records from one command to another. +This interceptor filters the events through a `morphline configuration file `_ that defines a chain of transformation commands that pipe records from one command to another. For example the morphline can ignore certain events or alter or insert certain event headers via regular expression based pattern matching, or it can auto-detect and set a MIME type via Apache Tika on events that are intercepted. For example, this kind of packet sniffing can be used for content based dynamic routing in a Flume topology. MorphlineInterceptor can also help to implement dynamic routing to multiple Apache Solr collections (e.g. for multi-tenancy). diff --git a/flume-ng-sinks/flume-ng-morphline-solr-sink/pom.xml b/flume-ng-sinks/flume-ng-morphline-solr-sink/pom.xml index 5115090d2a..5452aa9db3 100644 --- a/flume-ng-sinks/flume-ng-morphline-solr-sink/pom.xml +++ b/flume-ng-sinks/flume-ng-morphline-solr-sink/pom.xml @@ -34,7 +34,7 @@ limitations under the License. 4.3.0 4.3.0 1.3 - 0.4.0 + 0.4.1 1.6.1 2.12.4 @@ -100,6 +100,12 @@ limitations under the License. ${cdk.version} + + com.cloudera.cdk + cdk-morphlines-json + ${cdk.version} + + com.cloudera.cdk cdk-morphlines-twitter From a34cdb0eaaa9f37732e19cf932219ef090ad2281 Mon Sep 17 00:00:00 2001 From: Mike Percy Date: Mon, 15 Jul 2013 15:00:39 -0700 Subject: [PATCH 005/341] FLUME-2121. Upgrade Flume to log4j 1.2.17 to be in sync with HDFS/HBase. (Hari Shreedharan via Mike Percy) --- pom.xml | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/pom.xml b/pom.xml index 15e6d9b740..1405cdae9b 100644 --- a/pom.xml +++ b/pom.xml @@ -672,7 +672,7 @@ limitations under the License. log4j log4j - 1.2.16 + 1.2.17 com.sun.jdmk From 5b5470bd5d3e94842032009c36788d4ae346674b Mon Sep 17 00:00:00 2001 From: Hari Shreedharan Date: Tue, 16 Jul 2013 15:12:11 -0700 Subject: [PATCH 006/341] FLUME-2007. HDFS Sink should check if file is closed and retry if it is not. (Ted Malaska via Hari Shreedharan) --- .../flume/sink/hdfs/AbstractHDFSWriter.java | 64 ++++++++ .../sink/hdfs/HDFSCompressedDataStream.java | 2 +- .../flume/sink/hdfs/HDFSDataStream.java | 2 +- .../flume/sink/hdfs/HDFSSequenceFile.java | 8 +- .../hdfs/MockFileSystemCloseRetryWrapper.java | 142 ++++++++++++++++++ ...ckFsDataOutputStreamCloseRetryWrapper.java | 73 +++++++++ .../sink/hdfs/TestUseRawLocalFileSystem.java | 52 +++++++ 7 files changed, 340 insertions(+), 3 deletions(-) create mode 100644 flume-ng-sinks/flume-hdfs-sink/src/test/java/org/apache/flume/sink/hdfs/MockFileSystemCloseRetryWrapper.java create mode 100644 flume-ng-sinks/flume-hdfs-sink/src/test/java/org/apache/flume/sink/hdfs/MockFsDataOutputStreamCloseRetryWrapper.java diff --git a/flume-ng-sinks/flume-hdfs-sink/src/main/java/org/apache/flume/sink/hdfs/AbstractHDFSWriter.java b/flume-ng-sinks/flume-hdfs-sink/src/main/java/org/apache/flume/sink/hdfs/AbstractHDFSWriter.java index bc3b3837f4..da0466d4e6 100644 --- a/flume-ng-sinks/flume-hdfs-sink/src/main/java/org/apache/flume/sink/hdfs/AbstractHDFSWriter.java +++ b/flume-ng-sinks/flume-hdfs-sink/src/main/java/org/apache/flume/sink/hdfs/AbstractHDFSWriter.java @@ -27,6 +27,7 @@ import org.slf4j.Logger; import org.slf4j.LoggerFactory; +import java.io.IOException; import java.io.OutputStream; import java.lang.reflect.InvocationTargetException; import java.lang.reflect.Method; @@ -44,6 +45,8 @@ public abstract class AbstractHDFSWriter implements HDFSWriter { private Method refGetNumCurrentReplicas = null; private Method refGetDefaultReplication = null; private Integer configuredMinReplicas = null; + private Integer numberOfCloseRetries = null; + private long timeBetweenCloseRetries = Long.MAX_VALUE; final static Object [] NO_ARGS = new Object []{}; @@ -54,6 +57,17 @@ public void configure(Context context) { Preconditions.checkArgument(configuredMinReplicas >= 0, "hdfs.minBlockReplicas must be greater than or equal to 0"); } + numberOfCloseRetries = context.getInteger("hdfs.closeTries", 1) - 1; + + if (numberOfCloseRetries > 1) { + try { + timeBetweenCloseRetries = context.getLong("hdfs.callTimeout", 10000l); + } catch (NumberFormatException e) { + logger.warn("hdfs.callTimeout can not be parsed to a long: " + context.getLong("hdfs.callTimeout")); + } + timeBetweenCloseRetries = Math.max(timeBetweenCloseRetries/numberOfCloseRetries, 1000); + } + } /** @@ -97,6 +111,7 @@ protected void registerCurrentStream(FSDataOutputStream outputStream, this.destPath = destPath; this.refGetNumCurrentReplicas = reflectGetNumCurrentReplicas(outputStream); this.refGetDefaultReplication = reflectGetDefaultReplication(fs); + } protected void unregisterCurrentStream() { @@ -212,4 +227,53 @@ private Method reflectGetDefaultReplication(FileSystem fileSystem) { return m; } + /** + * This will + * @param outputStream + * @throws IOException + */ + protected void closeHDFSOutputStream(OutputStream outputStream) + throws IOException { + try { + outputStream.close(); + + if (numberOfCloseRetries > 0) { + try { + Method isFileClosedMethod = getIsFileClosedMethod(); + int closeAttemptsMade = 0; + if (isFileClosedMethod != null) { + while (closeAttemptsMade < numberOfCloseRetries.intValue() && + Boolean.FALSE.equals(isFileClosedMethod.invoke(fs, destPath))) { + closeAttemptsMade++; + logger.debug("Waiting: '" + timeBetweenCloseRetries + "' before retry close"); + Thread.sleep(timeBetweenCloseRetries); + try { + outputStream.close(); + } catch (IOException e) { + logger.error("Unable to close HDFS file: '" + destPath + "'"); + } + } + if (closeAttemptsMade == numberOfCloseRetries.intValue()) { + logger.warn("Failed to close '" + destPath + "' is " + + numberOfCloseRetries + " retries, over " + (timeBetweenCloseRetries * numberOfCloseRetries) + " millseconds"); + } + } + } catch (Exception e) { + logger.error("Failed to close '" + destPath + "' is " + + numberOfCloseRetries + " retries, over " + (timeBetweenCloseRetries * numberOfCloseRetries) + " millseconds", e); + } + } + } catch (IOException e) { + logger.error("Unable to close HDFS file: '" + destPath + "'"); + } + } + + private Method getIsFileClosedMethod() { + try { + return fs.getClass().getMethod("isFileClosed", Path.class); + } catch (Exception e) { + return null; + } + } + } diff --git a/flume-ng-sinks/flume-hdfs-sink/src/main/java/org/apache/flume/sink/hdfs/HDFSCompressedDataStream.java b/flume-ng-sinks/flume-hdfs-sink/src/main/java/org/apache/flume/sink/hdfs/HDFSCompressedDataStream.java index 2c2be6a3b5..551854779f 100644 --- a/flume-ng-sinks/flume-hdfs-sink/src/main/java/org/apache/flume/sink/hdfs/HDFSCompressedDataStream.java +++ b/flume-ng-sinks/flume-hdfs-sink/src/main/java/org/apache/flume/sink/hdfs/HDFSCompressedDataStream.java @@ -147,7 +147,7 @@ public void close() throws IOException { } fsOut.flush(); fsOut.sync(); - cmpOut.close(); + closeHDFSOutputStream(cmpOut); unregisterCurrentStream(); } diff --git a/flume-ng-sinks/flume-hdfs-sink/src/main/java/org/apache/flume/sink/hdfs/HDFSDataStream.java b/flume-ng-sinks/flume-hdfs-sink/src/main/java/org/apache/flume/sink/hdfs/HDFSDataStream.java index b8214be690..e20d1eeb47 100644 --- a/flume-ng-sinks/flume-hdfs-sink/src/main/java/org/apache/flume/sink/hdfs/HDFSDataStream.java +++ b/flume-ng-sinks/flume-hdfs-sink/src/main/java/org/apache/flume/sink/hdfs/HDFSDataStream.java @@ -123,7 +123,7 @@ public void close() throws IOException { serializer.beforeClose(); outStream.flush(); outStream.sync(); - outStream.close(); + closeHDFSOutputStream(outStream); unregisterCurrentStream(); } diff --git a/flume-ng-sinks/flume-hdfs-sink/src/main/java/org/apache/flume/sink/hdfs/HDFSSequenceFile.java b/flume-ng-sinks/flume-hdfs-sink/src/main/java/org/apache/flume/sink/hdfs/HDFSSequenceFile.java index 0383744098..5fe9f1b52a 100644 --- a/flume-ng-sinks/flume-hdfs-sink/src/main/java/org/apache/flume/sink/hdfs/HDFSSequenceFile.java +++ b/flume-ng-sinks/flume-hdfs-sink/src/main/java/org/apache/flume/sink/hdfs/HDFSSequenceFile.java @@ -75,6 +75,12 @@ public void open(String filePath, CompressionCodec codeC, Configuration conf = new Configuration(); Path dstPath = new Path(filePath); FileSystem hdfs = dstPath.getFileSystem(conf); + open(dstPath, codeC, compType, conf, hdfs); + } + + protected void open(Path dstPath, CompressionCodec codeC, + CompressionType compType, Configuration conf, FileSystem hdfs) + throws IOException { if(useRawLocalFileSystem) { if(hdfs instanceof LocalFileSystem) { hdfs = ((LocalFileSystem)hdfs).getRaw(); @@ -110,7 +116,7 @@ public void sync() throws IOException { @Override public void close() throws IOException { writer.close(); - outStream.close(); + closeHDFSOutputStream(outStream); unregisterCurrentStream(); } diff --git a/flume-ng-sinks/flume-hdfs-sink/src/test/java/org/apache/flume/sink/hdfs/MockFileSystemCloseRetryWrapper.java b/flume-ng-sinks/flume-hdfs-sink/src/test/java/org/apache/flume/sink/hdfs/MockFileSystemCloseRetryWrapper.java new file mode 100644 index 0000000000..b5d89e6bb2 --- /dev/null +++ b/flume-ng-sinks/flume-hdfs-sink/src/test/java/org/apache/flume/sink/hdfs/MockFileSystemCloseRetryWrapper.java @@ -0,0 +1,142 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, + * software distributed under the License is distributed on an + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + * KIND, either express or implied. See the License for the + * specific language governing permissions and limitations + * under the License. + */ +package org.apache.flume.sink.hdfs; + +import java.io.IOException; +import java.net.URI; + +import org.apache.hadoop.fs.FSDataInputStream; +import org.apache.hadoop.fs.FSDataOutputStream; +import org.apache.hadoop.fs.FileStatus; +import org.apache.hadoop.fs.FileSystem; +import org.apache.hadoop.fs.Path; +import org.apache.hadoop.fs.permission.FsPermission; +import org.apache.hadoop.util.Progressable; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +public class MockFileSystemCloseRetryWrapper extends FileSystem{ + + private static final Logger logger = + LoggerFactory.getLogger(MockFileSystemCloseRetryWrapper.class); + + FileSystem fs; + int numberOfClosesRequired; + boolean throwExceptionsOfFailedClose; + MockFsDataOutputStreamCloseRetryWrapper latestOutputStream; + + public MockFileSystemCloseRetryWrapper (FileSystem fs, + int numberOfClosesRequired, boolean throwExceptionsOfFailedClose) { + this.fs = fs; + this.throwExceptionsOfFailedClose = throwExceptionsOfFailedClose; + this.numberOfClosesRequired = numberOfClosesRequired; + } + + public MockFsDataOutputStreamCloseRetryWrapper getLastMockOutputStream() { + return latestOutputStream; + } + + @Override + public FSDataOutputStream append(Path arg0, int arg1, Progressable arg2) + throws IOException { + + latestOutputStream = new MockFsDataOutputStreamCloseRetryWrapper(fs.append(arg0, arg1, arg2), numberOfClosesRequired, throwExceptionsOfFailedClose); + + return latestOutputStream; + } + + @Override + public FSDataOutputStream create(Path arg0) throws IOException { + //throw new IOException ("HI there2"); + latestOutputStream = new MockFsDataOutputStreamCloseRetryWrapper(fs.create(arg0), numberOfClosesRequired, throwExceptionsOfFailedClose); + + return latestOutputStream; + } + + @Override + public FSDataOutputStream create(Path arg0, FsPermission arg1, boolean arg2, + int arg3, short arg4, long arg5, Progressable arg6) throws IOException { + throw new IOException ("Not a real file system"); + //return new MockFsDataOutputStreamCloseRetryWrapper(fs.create(arg0, arg1, arg2, arg3, arg4, arg5, arg6), numberOfClosesRequired, throwExceptionsOfFailedClose); + } + + @Override + @Deprecated + public boolean delete(Path arg0) throws IOException { + return fs.delete(arg0); + } + + @Override + public boolean delete(Path arg0, boolean arg1) throws IOException { + return fs.delete(arg0, arg1); + } + + @Override + public FileStatus getFileStatus(Path arg0) throws IOException { + return fs.getFileStatus(arg0); + } + + @Override + public URI getUri() { + return fs.getUri(); + } + + @Override + public Path getWorkingDirectory() { + return fs.getWorkingDirectory(); + } + + @Override + public FileStatus[] listStatus(Path arg0) throws IOException { + return fs.listStatus(arg0); + } + + @Override + public boolean mkdirs(Path arg0, FsPermission arg1) throws IOException { + // TODO Auto-generated method stub + return fs.mkdirs(arg0, arg1); + } + + @Override + public FSDataInputStream open(Path arg0, int arg1) throws IOException { + return fs.open(arg0, arg1); + } + + @Override + public boolean rename(Path arg0, Path arg1) throws IOException { + + return fs.rename(arg0, arg1); + } + + @Override + public void setWorkingDirectory(Path arg0) { + fs.setWorkingDirectory(arg0); + + } + + public boolean isFileClosed(Path path) { + + logger.info("isFileClosed: '" + latestOutputStream.getCurrentCloseAttempts() + "' , '" + numberOfClosesRequired + "'"); + + return latestOutputStream.getCurrentCloseAttempts() >= numberOfClosesRequired || numberOfClosesRequired == 0; + } + + + +} diff --git a/flume-ng-sinks/flume-hdfs-sink/src/test/java/org/apache/flume/sink/hdfs/MockFsDataOutputStreamCloseRetryWrapper.java b/flume-ng-sinks/flume-hdfs-sink/src/test/java/org/apache/flume/sink/hdfs/MockFsDataOutputStreamCloseRetryWrapper.java new file mode 100644 index 0000000000..1d8c140bb7 --- /dev/null +++ b/flume-ng-sinks/flume-hdfs-sink/src/test/java/org/apache/flume/sink/hdfs/MockFsDataOutputStreamCloseRetryWrapper.java @@ -0,0 +1,73 @@ +/** ++ * Licensed to the Apache Software Foundation (ASF) under one ++ * or more contributor license agreements. See the NOTICE file ++ * distributed with this work for additional information ++ * regarding copyright ownership. The ASF licenses this file ++ * to you under the Apache License, Version 2.0 (the ++ * "License"); you may not use this file except in compliance ++ * with the License. You may obtain a copy of the License at ++ * ++ * http://www.apache.org/licenses/LICENSE-2.0 ++ * ++ * Unless required by applicable law or agreed to in writing, software ++ * distributed under the License is distributed on an "AS IS" BASIS, ++ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. ++ * See the License for the specific language governing permissions and ++ * limitations under the License. ++ */ +package org.apache.flume.sink.hdfs; + +import java.io.IOException; +import java.io.OutputStream; + +import org.apache.hadoop.fs.FSDataOutputStream; +import org.apache.hadoop.fs.FileSystem.Statistics; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +public class MockFsDataOutputStreamCloseRetryWrapper extends FSDataOutputStream{ + + private static final Logger logger = + LoggerFactory.getLogger(MockFsDataOutputStreamCloseRetryWrapper.class); + + int currentCloseAttempts = 0; + int numberOfClosesRequired; + boolean throwExceptionsOfFailedClose; + + public MockFsDataOutputStreamCloseRetryWrapper(FSDataOutputStream wrapMe, + int numberOfClosesRequired, boolean throwExceptionsOfFailedClose) + throws IOException { + super(wrapMe.getWrappedStream(), null); + + this.numberOfClosesRequired = numberOfClosesRequired; + this.throwExceptionsOfFailedClose = throwExceptionsOfFailedClose; + + } + + public MockFsDataOutputStreamCloseRetryWrapper(OutputStream out, + Statistics stats) throws IOException { + super(out, stats); + + } + + @Override + public void close() throws IOException { + currentCloseAttempts++; + logger.info("Attempting to Close: '" + currentCloseAttempts + "' of '" + numberOfClosesRequired + "'"); + if (currentCloseAttempts > numberOfClosesRequired || numberOfClosesRequired == 0) { + logger.info("closing file"); + super.close(); + } else { + if (throwExceptionsOfFailedClose) { + logger.info("no closed and throwing exception"); + throw new IOException("MockIOException"); + } else { + logger.info("no closed and doing nothing"); + } + } + } + + public int getCurrentCloseAttempts() { + return currentCloseAttempts; + } +} diff --git a/flume-ng-sinks/flume-hdfs-sink/src/test/java/org/apache/flume/sink/hdfs/TestUseRawLocalFileSystem.java b/flume-ng-sinks/flume-hdfs-sink/src/test/java/org/apache/flume/sink/hdfs/TestUseRawLocalFileSystem.java index ffbdde006c..4476530133 100644 --- a/flume-ng-sinks/flume-hdfs-sink/src/test/java/org/apache/flume/sink/hdfs/TestUseRawLocalFileSystem.java +++ b/flume-ng-sinks/flume-hdfs-sink/src/test/java/org/apache/flume/sink/hdfs/TestUseRawLocalFileSystem.java @@ -30,6 +30,9 @@ import org.apache.flume.Event; import org.apache.flume.event.EventBuilder; import org.apache.flume.instrumentation.SinkCounter; +import org.apache.hadoop.conf.Configuration; +import org.apache.hadoop.fs.FileSystem; +import org.apache.hadoop.fs.Path; import org.apache.hadoop.io.SequenceFile; import org.apache.hadoop.io.SequenceFile.CompressionType; import org.apache.hadoop.io.compress.CompressionCodec; @@ -102,4 +105,53 @@ public void testSequenceFile() throws Exception { stream.sync(); Assert.assertTrue(testFile.length() > 0); } + + @Test + public void testSequenceFileCloseRetries() throws Exception { + SequenceFileCloseRetryCoreTest(3, 0, false); + SequenceFileCloseRetryCoreTest(3, 1, false); + SequenceFileCloseRetryCoreTest(3, 5, false); + + SequenceFileCloseRetryCoreTest(3, 0, true); + SequenceFileCloseRetryCoreTest(3, 1, true); + SequenceFileCloseRetryCoreTest(3, 5, true); + + SequenceFileCloseRetryCoreTest(3, 2, true); + SequenceFileCloseRetryCoreTest(3, 2, true); + + SequenceFileCloseRetryCoreTest(0, 0, true); + SequenceFileCloseRetryCoreTest(1, 0, true); + } + + + public void SequenceFileCloseRetryCoreTest(int numberOfCloseRetriesToAttempt, int numberOfClosesRequired, boolean throwExceptionsOfFailedClose) throws Exception { + String file = testFile.getCanonicalPath(); + HDFSSequenceFile stream = new HDFSSequenceFile(); + context.put("hdfs.useRawLocalFileSystem", "true"); + context.put("hdfs.closeTries", String.valueOf(numberOfCloseRetriesToAttempt)); + Configuration conf = new Configuration(); + Path dstPath = new Path(file); + MockFileSystemCloseRetryWrapper mockFs = new MockFileSystemCloseRetryWrapper(dstPath.getFileSystem(conf), numberOfClosesRequired, throwExceptionsOfFailedClose); + stream.configure(context); + stream.open(dstPath, null, CompressionType.NONE, conf, mockFs); + stream.append(event); + stream.sync(); + + stream.close(); + + if (throwExceptionsOfFailedClose) { + int expectedNumberOfCloses = 1; + Assert.assertTrue("Expected " + expectedNumberOfCloses + " but got " + mockFs.getLastMockOutputStream().getCurrentCloseAttempts() , mockFs.getLastMockOutputStream().currentCloseAttempts == expectedNumberOfCloses); + } else { + int expectedNumberOfCloses = Math.max(Math.min(numberOfClosesRequired, numberOfCloseRetriesToAttempt), 1); + Assert.assertTrue("Expected " + expectedNumberOfCloses + " but got " + mockFs.getLastMockOutputStream().getCurrentCloseAttempts() , mockFs.getLastMockOutputStream().currentCloseAttempts == expectedNumberOfCloses); + } + + + + + + } + + } \ No newline at end of file From e25661041c2d478bf27d64d39241a9fce9a0d263 Mon Sep 17 00:00:00 2001 From: Hari Shreedharan Date: Thu, 1 Aug 2013 22:46:38 -0700 Subject: [PATCH 007/341] FLUME-2109. HTTPS support in HTTP Source. (Ashish Paliwal via Hari Shreedharan) --- .../apache/flume/source/http/HTTPSource.java | 56 ++++++++- .../HTTPSourceConfigurationConstants.java | 5 + .../flume/source/http/TestHTTPSource.java | 110 +++++++++++++++++- .../src/test/resources/jettykeystore | Bin 0 -> 1355 bytes flume-ng-doc/sphinx/FlumeUserGuide.rst | 4 + 5 files changed, 165 insertions(+), 10 deletions(-) create mode 100644 flume-ng-core/src/test/resources/jettykeystore diff --git a/flume-ng-core/src/main/java/org/apache/flume/source/http/HTTPSource.java b/flume-ng-core/src/main/java/org/apache/flume/source/http/HTTPSource.java index c90f067c27..84ee33b2b5 100644 --- a/flume-ng-core/src/main/java/org/apache/flume/source/http/HTTPSource.java +++ b/flume-ng-core/src/main/java/org/apache/flume/source/http/HTTPSource.java @@ -29,6 +29,7 @@ import org.mortbay.jetty.Connector; import org.mortbay.jetty.Server; import org.mortbay.jetty.bio.SocketConnector; +import org.mortbay.jetty.security.SslSocketConnector; import org.mortbay.jetty.servlet.ServletHolder; import org.slf4j.Logger; import org.slf4j.LoggerFactory; @@ -88,16 +89,46 @@ public class HTTPSource extends AbstractSource implements private HTTPSourceHandler handler; private SourceCounter sourceCounter; + // SSL configuration variable + private volatile Integer sslPort; + private volatile String keyStorePath; + private volatile String keyStorePassword; + private volatile Boolean sslEnabled; + + @Override public void configure(Context context) { try { + // SSL related config + sslEnabled = context.getBoolean(HTTPSourceConfigurationConstants.SSL_ENABLED, false); + port = context.getInteger(HTTPSourceConfigurationConstants.CONFIG_PORT); host = context.getString(HTTPSourceConfigurationConstants.CONFIG_BIND, HTTPSourceConfigurationConstants.DEFAULT_BIND); - checkHostAndPort(); + + Preconditions.checkState(host != null && !host.isEmpty(), + "HTTPSource hostname specified is empty"); + // verify port only if its not ssl + if(!sslEnabled) { + Preconditions.checkNotNull(port, "HTTPSource requires a port number to be" + + " specified"); + } + String handlerClassName = context.getString( HTTPSourceConfigurationConstants.CONFIG_HANDLER, HTTPSourceConfigurationConstants.DEFAULT_HANDLER).trim(); + + if(sslEnabled) { + LOG.debug("SSL configuration enabled"); + sslPort = context.getInteger(HTTPSourceConfigurationConstants.SSL_PORT); + Preconditions.checkArgument(sslPort != null && sslPort > 0, "SSL Port cannot be null or less than 0" ); + keyStorePath = context.getString(HTTPSourceConfigurationConstants.SSL_KEYSTORE); + Preconditions.checkArgument(keyStorePath != null && !keyStorePath.isEmpty(), + "Keystore is required for SSL Conifguration" ); + keyStorePassword = context.getString(HTTPSourceConfigurationConstants.SSL_KEYSTORE_PASSWORD); + Preconditions.checkArgument(keyStorePassword != null, "Keystore password is required for SSL Configuration"); + } + @SuppressWarnings("unchecked") Class clazz = (Class) @@ -139,10 +170,25 @@ public void start() { + " before I started one." + "Will not attempt to start."); srv = new Server(); - SocketConnector connector = new SocketConnector(); - connector.setPort(port); - connector.setHost(host); - srv.setConnectors(new Connector[] { connector }); + + // Connector Array + Connector[] connectors = new Connector[1]; + + + if(sslEnabled) { + SslSocketConnector sslSocketConnector = new SslSocketConnector(); + sslSocketConnector.setKeystore(keyStorePath); + sslSocketConnector.setKeyPassword(keyStorePassword); + sslSocketConnector.setPort(sslPort); + connectors[0] = sslSocketConnector; + } else { + SocketConnector connector = new SocketConnector(); + connector.setPort(port); + connector.setHost(host); + connectors[0] = connector; + } + + srv.setConnectors(connectors); try { org.mortbay.jetty.servlet.Context root = new org.mortbay.jetty.servlet.Context( diff --git a/flume-ng-core/src/main/java/org/apache/flume/source/http/HTTPSourceConfigurationConstants.java b/flume-ng-core/src/main/java/org/apache/flume/source/http/HTTPSourceConfigurationConstants.java index f547e0f08b..205aeab3e4 100644 --- a/flume-ng-core/src/main/java/org/apache/flume/source/http/HTTPSourceConfigurationConstants.java +++ b/flume-ng-core/src/main/java/org/apache/flume/source/http/HTTPSourceConfigurationConstants.java @@ -34,4 +34,9 @@ public class HTTPSourceConfigurationConstants { public static final String DEFAULT_HANDLER = "org.apache.flume.source.http.JSONHandler"; + public static final String SSL_PORT = "sslPort"; + public static final String SSL_KEYSTORE = "keystore"; + public static final String SSL_KEYSTORE_PASSWORD = "keystorePassword"; + public static final String SSL_ENABLED = "enableSSL"; + } diff --git a/flume-ng-core/src/test/java/org/apache/flume/source/http/TestHTTPSource.java b/flume-ng-core/src/test/java/org/apache/flume/source/http/TestHTTPSource.java index 8952db30c6..6c9fd86d30 100644 --- a/flume-ng-core/src/test/java/org/apache/flume/source/http/TestHTTPSource.java +++ b/flume-ng-core/src/test/java/org/apache/flume/source/http/TestHTTPSource.java @@ -22,11 +22,7 @@ import com.google.gson.Gson; import com.google.gson.reflect.TypeToken; import junit.framework.Assert; -import org.apache.flume.Channel; -import org.apache.flume.ChannelSelector; -import org.apache.flume.Context; -import org.apache.flume.Event; -import org.apache.flume.Transaction; +import org.apache.flume.*; import org.apache.flume.channel.ChannelProcessor; import org.apache.flume.channel.MemoryChannel; import org.apache.flume.channel.ReplicatingChannelSelector; @@ -34,6 +30,7 @@ import org.apache.flume.event.JSONEvent; import org.apache.http.HttpResponse; import org.apache.http.client.methods.HttpPost; +import org.apache.http.conn.ssl.SSLSocketFactory; import org.apache.http.entity.StringEntity; import org.apache.http.impl.client.DefaultHttpClient; import org.junit.AfterClass; @@ -41,10 +38,14 @@ import org.junit.BeforeClass; import org.junit.Test; +import javax.net.ssl.*; import javax.servlet.http.HttpServletResponse; import java.io.IOException; import java.lang.reflect.Type; import java.net.ServerSocket; +import java.net.URL; +import java.security.SecureRandom; +import java.security.cert.CertificateException; import java.util.ArrayList; import java.util.List; import java.util.Map; @@ -58,8 +59,12 @@ public class TestHTTPSource { private static HTTPSource source; + private static HTTPSource httpsSource; +// private static Channel httpsChannel; + private static Channel channel; private static int selectedPort; + private static int sslPort; DefaultHttpClient httpClient; HttpPost postRequest; @@ -77,9 +82,13 @@ public static void setUpClass() throws Exception { source = new HTTPSource(); channel = new MemoryChannel(); + httpsSource = new HTTPSource(); +// httpsChannel = new MemoryChannel(); + Context ctx = new Context(); ctx.put("capacity", "100"); Configurables.configure(channel, ctx); +// Configurables.configure(httpsChannel, ctx); List channels = new ArrayList(1); channels.add(channel); @@ -90,19 +99,43 @@ public static void setUpClass() throws Exception { source.setChannelProcessor(new ChannelProcessor(rcs)); channel.start(); + + // Channel for HTTPS source +// List sslChannels = new ArrayList(1); +// channels.add(httpsChannel); +// +// ChannelSelector sslRcs = new ReplicatingChannelSelector(); +// rcs.setChannels(sslChannels); + + httpsSource.setChannelProcessor(new ChannelProcessor(rcs)); +// httpsChannel.start(); + + // HTTP context Context context = new Context(); context.put("port", String.valueOf(selectedPort)); context.put("host", "0.0.0.0"); + // SSL context props + Context sslContext = new Context(); + sslContext.put(HTTPSourceConfigurationConstants.SSL_ENABLED, "true"); + sslPort = findFreePort(); + sslContext.put(HTTPSourceConfigurationConstants.SSL_PORT, String.valueOf(sslPort)); + sslContext.put(HTTPSourceConfigurationConstants.SSL_KEYSTORE_PASSWORD, "password"); + sslContext.put(HTTPSourceConfigurationConstants.SSL_KEYSTORE, "src/test/resources/jettykeystore"); + Configurables.configure(source, context); + Configurables.configure(httpsSource, sslContext); source.start(); + httpsSource.start(); } @AfterClass public static void tearDownClass() throws Exception { source.stop(); channel.stop(); + httpsSource.stop(); +// httpsChannel.stop(); } @Before @@ -268,6 +301,73 @@ private ResultWrapper putWithEncoding(String encoding, int n) return new ResultWrapper(resp, events); } + @Test + public void testHttps() throws Exception { + Type listType = new TypeToken>() { + }.getType(); + List events = Lists.newArrayList(); + Random rand = new Random(); + for (int i = 0; i < 10; i++) { + Map input = Maps.newHashMap(); + for (int j = 0; j < 10; j++) { + input.put(String.valueOf(i) + String.valueOf(j), String.valueOf(i)); + } + JSONEvent e = new JSONEvent(); + e.setHeaders(input); + e.setBody(String.valueOf(rand.nextGaussian()).getBytes("UTF-8")); + events.add(e); + } + Gson gson = new Gson(); + String json = gson.toJson(events, listType); + HttpsURLConnection httpsURLConnection = null; + try { + TrustManager[] trustAllCerts = {new X509TrustManager() { + @Override + public void checkClientTrusted( + java.security.cert.X509Certificate[] x509Certificates, String s) + throws CertificateException { + // noop + } + + @Override + public void checkServerTrusted( + java.security.cert.X509Certificate[] x509Certificates, String s) + throws CertificateException { + // noop + } + + public java.security.cert.X509Certificate[] getAcceptedIssuers() { + return null; + } + }}; + SSLContext sc = SSLContext.getInstance("SSL"); + + HostnameVerifier hv = new HostnameVerifier() { + public boolean verify(String arg0, SSLSession arg1) { + return true; + } + }; + sc.init(null, trustAllCerts, new SecureRandom()); + HttpsURLConnection.setDefaultSSLSocketFactory(sc.getSocketFactory()); + HttpsURLConnection.setDefaultHostnameVerifier( + SSLSocketFactory.ALLOW_ALL_HOSTNAME_VERIFIER); + URL sslUrl = new URL("https://0.0.0.0:" + sslPort); + httpsURLConnection = (HttpsURLConnection) sslUrl.openConnection(); + httpsURLConnection.setDoInput(true); + httpsURLConnection.setDoOutput(true); + httpsURLConnection.setRequestMethod("POST"); + httpsURLConnection.getOutputStream().write(json.getBytes()); + + int statusCode = httpsURLConnection.getResponseCode(); + Assert.assertEquals(200, statusCode); + } catch (Exception exception) { + Assert.fail("Exception not expected"); + exception.printStackTrace(); + } finally { + httpsURLConnection.disconnect(); + } + } + private void takeWithEncoding(String encoding, int n, List events) throws Exception{ Transaction tx = channel.getTransaction(); diff --git a/flume-ng-core/src/test/resources/jettykeystore b/flume-ng-core/src/test/resources/jettykeystore new file mode 100644 index 0000000000000000000000000000000000000000..db76bcbc0ef81760371acef0edd177a830ab960b GIT binary patch literal 1355 zcmezO_TO6u1_mY|W&~r_tkjZ{N+8d^>gU-#K$%?zO-$Pi_}I9#*%(f`JM2){7TjW@rCPUUvXUaGRh2Hy+-n7jzPiGkK0oeSx=WUvI^f?(OdAhO?UCx zdu#iQOkJAeZW|c<3z%8)QL zP~b;Xgl6U5qWN=c4gA-(>?t}J@%;u%XhL}T#Y)A*Q)~H~PsSBEcOSm$abC6IR;_B5 zM${G6_GvDMy7HP#g%>!!T((O6!NPeTbr@NXMjZRE^49$FKEW+JGba2h-RStw@O@Ny zJC9Ek)mp)}r&r|r10K%xPeP6xe^rrI-7llzaJC~y+~&cYbBr>A6IBl~E&e6D z*>EYx6u~npJ6GjxIdwPikxalf+r-!P0^gQy7vFS~alOFPmofW|9zA=#uJLf0q>I<7 zJ)EK)cZ|ED1wyEG}8{bkOTkVM_?!raDN;WE_=tXLcRxL)fnKjYKBsch;~ z%@00jmohH?UD?9L|9%qluYYQLS*4CBZMyMkU1&+m;%QOESy!(#c-^#}SGT3KXIUSI z(%0{&CVbj5r-1GJx`#Vlrf*&2(r`!b!Q<2&GaJ^r{NPt*LHS~fd~L1fd1gPQXhpS@>E(HE8E@(?!ETYvW4vm=lM9ieAxUxtu~Lg@4UcN&B|bqVJHAh>dc`mY{KlJdD(gS<#{j> z4h#`?3=uAb2%iBD$Q*8ARxeP7GKBJ(p?m{5ab81X12ZF2LjzM&Q%OAGNm=t&VxfO zV$p_+({5a`UUxmh=#klamHdsH6^?&C!KOdUWVLm-+myQ8vwRjSKSXlJaHJS_UbJ20 zRJHbV+yOUvuJ3Frx^=T78TRb#XzLZ5=9H8&pKU(u-kb#GXNgUd3TdM0K@ z21aD(0V9bS=&tOoU#mA<{u`AP6j8oOA>`DTQ~58ya0YICGLP?(Xu`r*lif7$N2@HE zwEJRfm|4TgFx3P9=FY#-C+!{Coy011K;3#{OQb?-_M Date: Fri, 2 Aug 2013 18:17:12 -0700 Subject: [PATCH 008/341] FLUME-2139. Upgrade morphline CDK dependency to 0.5.0 release. (Roman Shaposhnik via Mike Percy) --- flume-ng-sinks/flume-ng-morphline-solr-sink/pom.xml | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/flume-ng-sinks/flume-ng-morphline-solr-sink/pom.xml b/flume-ng-sinks/flume-ng-morphline-solr-sink/pom.xml index 5452aa9db3..7cbc9b9cb3 100644 --- a/flume-ng-sinks/flume-ng-morphline-solr-sink/pom.xml +++ b/flume-ng-sinks/flume-ng-morphline-solr-sink/pom.xml @@ -34,7 +34,7 @@ limitations under the License. 4.3.0 4.3.0 1.3 - 0.4.1 + 0.5.0 1.6.1 2.12.4 From 41f1e8afbbc4854c516e1ffd0baf9d0de73d60af Mon Sep 17 00:00:00 2001 From: Hari Shreedharan Date: Mon, 5 Aug 2013 15:13:22 -0700 Subject: [PATCH 009/341] FLUME-2142. HTTPS tests for http source (Ashish Paliwal via Hari Shreedharan) --- .../flume/source/http/TestHTTPSource.java | 63 +++++++++++++++---- 1 file changed, 51 insertions(+), 12 deletions(-) diff --git a/flume-ng-core/src/test/java/org/apache/flume/source/http/TestHTTPSource.java b/flume-ng-core/src/test/java/org/apache/flume/source/http/TestHTTPSource.java index 6c9fd86d30..9e14648b13 100644 --- a/flume-ng-core/src/test/java/org/apache/flume/source/http/TestHTTPSource.java +++ b/flume-ng-core/src/test/java/org/apache/flume/source/http/TestHTTPSource.java @@ -42,6 +42,7 @@ import javax.servlet.http.HttpServletResponse; import java.io.IOException; import java.lang.reflect.Type; +import java.net.HttpURLConnection; import java.net.ServerSocket; import java.net.URL; import java.security.SecureRandom; @@ -83,12 +84,11 @@ public static void setUpClass() throws Exception { channel = new MemoryChannel(); httpsSource = new HTTPSource(); -// httpsChannel = new MemoryChannel(); + httpsSource.setName("HTTPS Source"); Context ctx = new Context(); ctx.put("capacity", "100"); Configurables.configure(channel, ctx); -// Configurables.configure(httpsChannel, ctx); List channels = new ArrayList(1); channels.add(channel); @@ -100,15 +100,7 @@ public static void setUpClass() throws Exception { channel.start(); - // Channel for HTTPS source -// List sslChannels = new ArrayList(1); -// channels.add(httpsChannel); -// -// ChannelSelector sslRcs = new ReplicatingChannelSelector(); -// rcs.setChannels(sslChannels); - httpsSource.setChannelProcessor(new ChannelProcessor(rcs)); -// httpsChannel.start(); // HTTP context Context context = new Context(); @@ -135,7 +127,6 @@ public static void tearDownClass() throws Exception { source.stop(); channel.stop(); httpsSource.stop(); -// httpsChannel.stop(); } @Before @@ -312,6 +303,7 @@ public void testHttps() throws Exception { for (int j = 0; j < 10; j++) { input.put(String.valueOf(i) + String.valueOf(j), String.valueOf(i)); } + input.put("MsgNum", String.valueOf(i)); JSONEvent e = new JSONEvent(); e.setHeaders(input); e.setBody(String.valueOf(rand.nextGaussian()).getBytes("UTF-8")); @@ -360,14 +352,61 @@ public boolean verify(String arg0, SSLSession arg1) { int statusCode = httpsURLConnection.getResponseCode(); Assert.assertEquals(200, statusCode); + + Transaction transaction = channel.getTransaction(); + transaction.begin(); + for(int i = 0; i < 10; i++) { + Event e = channel.take(); + Assert.assertNotNull(e); + Assert.assertEquals(String.valueOf(i), e.getHeaders().get("MsgNum")); + } + + transaction.commit(); + transaction.close(); } catch (Exception exception) { Assert.fail("Exception not expected"); - exception.printStackTrace(); } finally { httpsURLConnection.disconnect(); } } + @Test + public void testHttpsSourceNonHttpsClient() throws Exception { + Type listType = new TypeToken>() { + }.getType(); + List events = Lists.newArrayList(); + Random rand = new Random(); + for (int i = 0; i < 10; i++) { + Map input = Maps.newHashMap(); + for (int j = 0; j < 10; j++) { + input.put(String.valueOf(i) + String.valueOf(j), String.valueOf(i)); + } + input.put("MsgNum", String.valueOf(i)); + JSONEvent e = new JSONEvent(); + e.setHeaders(input); + e.setBody(String.valueOf(rand.nextGaussian()).getBytes("UTF-8")); + events.add(e); + } + Gson gson = new Gson(); + String json = gson.toJson(events, listType); + HttpURLConnection httpURLConnection = null; + try { + URL url = new URL("http://0.0.0.0:" + sslPort); + httpURLConnection = (HttpURLConnection) url.openConnection(); + httpURLConnection.setDoInput(true); + httpURLConnection.setDoOutput(true); + httpURLConnection.setRequestMethod("POST"); + httpURLConnection.getOutputStream().write(json.getBytes()); + httpURLConnection.getResponseCode(); + + Assert.fail("HTTP Client cannot connect to HTTPS source"); + } catch (Exception exception) { + Assert.assertTrue("Exception expected", true); + } finally { + httpURLConnection.disconnect(); + } + } + private void takeWithEncoding(String encoding, int n, List events) throws Exception{ Transaction tx = channel.getTransaction(); From 5d49eeb734d1c55aaf48219699706851d9e820b5 Mon Sep 17 00:00:00 2001 From: Hari Shreedharan Date: Mon, 5 Aug 2013 17:25:53 -0700 Subject: [PATCH 010/341] FLUME-2134. AsyncHbase Sink should use ZKConfig.getZKQuorumServersString plus test fixes on Windows (Roshan Naik via Hari Shreedharan) --- .../flume/sink/hbase/AsyncHBaseSink.java | 3 +- .../flume/sink/hbase/TestAsyncHBaseSink.java | 83 ++----------------- 2 files changed, 8 insertions(+), 78 deletions(-) diff --git a/flume-ng-sinks/flume-ng-hbase-sink/src/main/java/org/apache/flume/sink/hbase/AsyncHBaseSink.java b/flume-ng-sinks/flume-ng-hbase-sink/src/main/java/org/apache/flume/sink/hbase/AsyncHBaseSink.java index 7020fcddcc..5e297b1333 100644 --- a/flume-ng-sinks/flume-ng-hbase-sink/src/main/java/org/apache/flume/sink/hbase/AsyncHBaseSink.java +++ b/flume-ng-sinks/flume-ng-hbase-sink/src/main/java/org/apache/flume/sink/hbase/AsyncHBaseSink.java @@ -37,6 +37,7 @@ import org.apache.hadoop.conf.Configuration; import org.apache.hadoop.hbase.HBaseConfiguration; import org.apache.hadoop.hbase.HConstants; +import org.apache.hadoop.hbase.zookeeper.ZKConfig; import org.hbase.async.AtomicIncrementRequest; import org.hbase.async.HBaseClient; import org.hbase.async.PutRequest; @@ -318,7 +319,7 @@ public void configure(Context context) { if (conf == null) { //In tests, we pass the conf in. conf = HBaseConfiguration.create(); } - zkQuorum = conf.get(HConstants.ZOOKEEPER_QUORUM); + zkQuorum = ZKConfig.getZKQuorumServersString(conf); zkBaseDir = conf.get(HConstants.ZOOKEEPER_ZNODE_PARENT, HConstants.DEFAULT_ZOOKEEPER_ZNODE_PARENT); } diff --git a/flume-ng-sinks/flume-ng-hbase-sink/src/test/java/org/apache/flume/sink/hbase/TestAsyncHBaseSink.java b/flume-ng-sinks/flume-ng-hbase-sink/src/test/java/org/apache/flume/sink/hbase/TestAsyncHBaseSink.java index 7ddfdae9ef..a0c04eb6f9 100644 --- a/flume-ng-sinks/flume-ng-hbase-sink/src/test/java/org/apache/flume/sink/hbase/TestAsyncHBaseSink.java +++ b/flume-ng-sinks/flume-ng-hbase-sink/src/test/java/org/apache/flume/sink/hbase/TestAsyncHBaseSink.java @@ -19,14 +19,11 @@ package org.apache.flume.sink.hbase; - -import java.io.File; import java.io.IOException; import java.util.Arrays; import java.util.HashMap; import java.util.Map; -import org.apache.commons.io.FileUtils; import org.apache.flume.Channel; import org.apache.flume.Context; import org.apache.flume.Event; @@ -37,35 +34,26 @@ import org.apache.flume.channel.MemoryChannel; import org.apache.flume.conf.Configurables; import org.apache.flume.event.EventBuilder; -import org.apache.hadoop.conf.Configuration; -import org.apache.hadoop.hbase.HBaseConfiguration; import org.apache.hadoop.hbase.HBaseTestingUtility; import org.apache.hadoop.hbase.HConstants; -import org.apache.hadoop.hbase.MiniHBaseCluster; import org.apache.hadoop.hbase.client.HTable; import org.apache.hadoop.hbase.client.Result; import org.apache.hadoop.hbase.client.ResultScanner; import org.apache.hadoop.hbase.client.Scan; -import org.apache.hadoop.hbase.master.HMaster; import org.apache.hadoop.hbase.util.Bytes; -import org.apache.hadoop.hbase.zookeeper.MiniZooKeeperCluster; +import org.apache.hadoop.hbase.zookeeper.ZKConfig; import org.junit.AfterClass; import org.junit.Assert; import org.junit.BeforeClass; import org.junit.Ignore; import org.junit.Test; -import com.google.common.io.Files; import com.google.common.primitives.Longs; -import java.lang.reflect.Method; import org.junit.After; public class TestAsyncHBaseSink { - private static HBaseTestingUtility testUtility; - private static MiniZooKeeperCluster zookeeperCluster; - private static MiniHBaseCluster hbaseCluster; - private static String workDir = Files.createTempDir().getAbsolutePath(); + private static HBaseTestingUtility testUtility = new HBaseTestingUtility(); private static String tableName = "TestHbaseSink"; private static String columnFamily = "TestColumnFamily"; @@ -78,65 +66,8 @@ public class TestAsyncHBaseSink { @BeforeClass public static void setUp() throws Exception { + testUtility.startMiniCluster(); - /* - * Borrowed from HCatalog ManyMiniCluster.java - * https://svn.apache.org/repos/asf/incubator/hcatalog/trunk/ - * storage-handlers/hbase/src/test/org/apache/hcatalog/ - * hbase/ManyMiniCluster.java - * - */ - String hbaseDir = new File(workDir,"hbase").getAbsolutePath(); - String hbaseRoot = "file://" + hbaseDir; - Configuration hbaseConf = HBaseConfiguration.create(); - - hbaseConf.set(HConstants.HBASE_DIR, hbaseRoot); - hbaseConf.setInt(HConstants.ZOOKEEPER_CLIENT_PORT, 2181); - hbaseConf.set(HConstants.ZOOKEEPER_QUORUM, "0.0.0.0"); - hbaseConf.setInt("hbase.master.info.port", -1); - hbaseConf.setInt("hbase.zookeeper.property.maxClientCnxns",500); - String zookeeperDir = new File(workDir,"zk").getAbsolutePath(); - int zookeeperPort = 2181; - zookeeperCluster = new MiniZooKeeperCluster(); - Method m; - Class zkParam[] = {Integer.TYPE}; - try{ - m = MiniZooKeeperCluster.class.getDeclaredMethod("setDefaultClientPort", - zkParam); - } catch (NoSuchMethodException e) { - m = MiniZooKeeperCluster.class.getDeclaredMethod("setClientPort", - zkParam); - } - - m.invoke(zookeeperCluster, new Object[] {new Integer(zookeeperPort)}); - zookeeperCluster.startup(new File(zookeeperDir)); - hbaseCluster = new MiniHBaseCluster(hbaseConf, 1); - HMaster master = hbaseCluster.getMaster(); - Object serverName = master.getServerName(); - String hostAndPort; - if(serverName instanceof String) { - System.out.println("Server name is string, using HServerAddress."); - m = HMaster.class.getDeclaredMethod("getMasterAddress", - new Class[]{}); - Class clazz = Class.forName("org.apache.hadoop.hbase.HServerAddress"); - /* - * Call method to get server address - */ - Object serverAddr = clazz.cast(m.invoke(master, new Object[]{})); - //returns the address as hostname:port - hostAndPort = serverAddr.toString(); - } else { - System.out.println("ServerName is org.apache.hadoop.hbase.ServerName," + - "using getHostAndPort()"); - Class clazz = Class.forName("org.apache.hadoop.hbase.ServerName"); - m = clazz.getDeclaredMethod("getHostAndPort", new Class[] {}); - hostAndPort = m.invoke(serverName, new Object[]{}).toString(); - } - - hbaseConf.set("hbase.master", hostAndPort); - testUtility = new HBaseTestingUtility(hbaseConf); - testUtility.setZkCluster(zookeeperCluster); - hbaseCluster.startMaster(); Map ctxMap = new HashMap(); ctxMap.put("table", tableName); ctxMap.put("columnFamily", columnFamily); @@ -151,9 +82,7 @@ public static void setUp() throws Exception { @AfterClass public static void tearDown() throws Exception { - hbaseCluster.shutdown(); - zookeeperCluster.shutdown(); - FileUtils.deleteDirectory(new File(workDir)); + testUtility.shutdownMiniCluster(); } @After @@ -347,7 +276,7 @@ public void testWithoutConfigurationObject() throws Exception{ deleteTable = true; ctx.put("batchSize", "2"); ctx.put(HBaseSinkConfigurationConstants.ZK_QUORUM, - testUtility.getConfiguration().get(HConstants.ZOOKEEPER_QUORUM)); + ZKConfig.getZKQuorumServersString(testUtility.getConfiguration()) ); ctx.put(HBaseSinkConfigurationConstants.ZK_ZNODE_PARENT, testUtility.getConfiguration().get(HConstants.ZOOKEEPER_ZNODE_PARENT)); AsyncHBaseSink sink = new AsyncHBaseSink(); @@ -485,7 +414,7 @@ public void testHBaseFailure() throws Exception { Assert.assertEquals(2, found); out = results[2]; Assert.assertArrayEquals(Longs.toByteArray(2), out); - hbaseCluster.shutdown(); + testUtility.shutdownMiniCluster(); sink.process(); sink.stop(); } From 99db32ccd163daf9d7685f0e8485941701e1133d Mon Sep 17 00:00:00 2001 From: Hari Shreedharan Date: Thu, 8 Aug 2013 23:44:59 -0700 Subject: [PATCH 011/341] FLUME-2154. Reducing duplicate events caused by reset-connection-interval (Juhani Connolly via Hari Shreedharan) --- .../apache/flume/sink/AbstractRpcSink.java | 37 ++++++++++++------- .../org/apache/flume/sink/TestAvroSink.java | 6 +++ 2 files changed, 29 insertions(+), 14 deletions(-) diff --git a/flume-ng-core/src/main/java/org/apache/flume/sink/AbstractRpcSink.java b/flume-ng-core/src/main/java/org/apache/flume/sink/AbstractRpcSink.java index b3208fc549..5146834998 100644 --- a/flume-ng-core/src/main/java/org/apache/flume/sink/AbstractRpcSink.java +++ b/flume-ng-core/src/main/java/org/apache/flume/sink/AbstractRpcSink.java @@ -43,6 +43,7 @@ import java.util.concurrent.Executors; import java.util.concurrent.ScheduledExecutorService; import java.util.concurrent.TimeUnit; +import java.util.concurrent.atomic.AtomicBoolean; import java.util.concurrent.locks.Lock; import java.util.concurrent.locks.ReentrantLock; @@ -149,11 +150,11 @@ public abstract class AbstractRpcSink extends AbstractSink private Properties clientProps; private SinkCounter sinkCounter; private int cxnResetInterval; + private AtomicBoolean resetConnectionFlag; private final int DEFAULT_CXN_RESET_INTERVAL = 0; private final ScheduledExecutorService cxnResetExecutor = Executors .newSingleThreadScheduledExecutor(new ThreadFactoryBuilder() .setNameFormat("Rpc Sink Reset Thread").build()); - private final Lock resetLock = new ReentrantLock(); @Override public void configure(Context context) { @@ -206,6 +207,7 @@ private void createConnection() throws FlumeException { "port: {}", new Object[] { getName(), hostname, port }); try { + resetConnectionFlag = new AtomicBoolean(false); client = initializeRpcClient(clientProps); Preconditions.checkNotNull(client, "Rpc Client could not be " + "initialized. " + getName() + " could not be started"); @@ -214,17 +216,7 @@ private void createConnection() throws FlumeException { cxnResetExecutor.schedule(new Runnable() { @Override public void run() { - resetLock.lock(); - try { - destroyConnection(); - createConnection(); - } catch (Throwable throwable) { - //Don't rethrow, else this runnable won't get scheduled again. - logger.error("Error while trying to expire connection", - throwable); - } finally { - resetLock.unlock(); - } + resetConnectionFlag.set(true); } }, cxnResetInterval, TimeUnit.SECONDS); } @@ -241,6 +233,17 @@ public void run() { } + private void resetConnection() { + try { + destroyConnection(); + createConnection(); + } catch (Throwable throwable) { + //Don't rethrow, else this runnable won't get scheduled again. + logger.error("Error while trying to expire connection", + throwable); + } + } + private void destroyConnection() { if (client != null) { logger.debug("Rpc sink {} closing Rpc client: {}", getName(), client); @@ -332,7 +335,14 @@ public Status process() throws EventDeliveryException { Channel channel = getChannel(); Transaction transaction = channel.getTransaction(); - resetLock.lock(); + if(resetConnectionFlag.get()) { + resetConnection(); + // if the time to reset is long and the timeout is short + // this may cancel the next reset request + // this should however not be an issue + resetConnectionFlag.set(false); + } + try { transaction.begin(); @@ -382,7 +392,6 @@ public Status process() throws EventDeliveryException { throw new EventDeliveryException("Failed to send events", t); } } finally { - resetLock.unlock(); transaction.close(); } diff --git a/flume-ng-core/src/test/java/org/apache/flume/sink/TestAvroSink.java b/flume-ng-core/src/test/java/org/apache/flume/sink/TestAvroSink.java index 8760c256ed..757a53626d 100644 --- a/flume-ng-core/src/test/java/org/apache/flume/sink/TestAvroSink.java +++ b/flume-ng-core/src/test/java/org/apache/flume/sink/TestAvroSink.java @@ -300,6 +300,12 @@ public void testReset() throws Exception { sink.start(); RpcClient firstClient = sink.getUnderlyingClient(); Thread.sleep(6000); + Transaction t = channel.getTransaction(); + t.begin(); + channel.put(EventBuilder.withBody("This is a test", Charset.defaultCharset())); + t.commit(); + t.close(); + sink.process(); // Make sure they are not the same object, connection should be reset Assert.assertFalse(firstClient == sink.getUnderlyingClient()); sink.stop(); From f9da62be22101b00b907be3a66a6ce8d823c2f8f Mon Sep 17 00:00:00 2001 From: Hari Shreedharan Date: Fri, 9 Aug 2013 21:49:19 -0700 Subject: [PATCH 012/341] FLUME-2157. Spool directory source does not shut down correctly when Flume is reconfigured. (Mike Percy via Hari Shreedharan) --- .../avro/ReliableSpoolingFileEventReader.java | 13 +- .../flume/source/SpoolDirectorySource.java | 41 ++++- .../source/TestSpoolDirectorySource.java | 31 ++++ .../flume/test/agent/TestSpooldirSource.java | 174 ++++++++++++++++++ .../apache/flume/test/util/StagedInstall.java | 30 ++- 5 files changed, 271 insertions(+), 18 deletions(-) create mode 100644 flume-ng-tests/src/test/java/org/apache/flume/test/agent/TestSpooldirSource.java diff --git a/flume-ng-core/src/main/java/org/apache/flume/client/avro/ReliableSpoolingFileEventReader.java b/flume-ng-core/src/main/java/org/apache/flume/client/avro/ReliableSpoolingFileEventReader.java index f82fe1fa96..724ab38f1a 100644 --- a/flume-ng-core/src/main/java/org/apache/flume/client/avro/ReliableSpoolingFileEventReader.java +++ b/flume-ng-core/src/main/java/org/apache/flume/client/avro/ReliableSpoolingFileEventReader.java @@ -133,12 +133,15 @@ private ReliableSpoolingFileEventReader(File spoolDirectory, // Do a canary test to make sure we have access to spooling directory try { - File f1 = File.createTempFile("flume", "test", spoolDirectory); - Files.write("testing flume file permissions\n", f1, Charsets.UTF_8); - Files.readLines(f1, Charsets.UTF_8); - if (!f1.delete()) { - throw new IOException("Unable to delete canary file " + f1); + File canary = File.createTempFile("flume-spooldir-perm-check-", ".canary", + spoolDirectory); + Files.write("testing flume file permissions\n", canary, Charsets.UTF_8); + List lines = Files.readLines(canary, Charsets.UTF_8); + Preconditions.checkState(!lines.isEmpty(), "Empty canary file %s", canary); + if (!canary.delete()) { + throw new IOException("Unable to delete canary file " + canary); } + logger.debug("Successfully created and deleted canary file: {}", canary); } catch (IOException e) { throw new FlumeException("Unable to read and modify files" + " in the spooling directory: " + spoolDirectory, e); diff --git a/flume-ng-core/src/main/java/org/apache/flume/source/SpoolDirectorySource.java b/flume-ng-core/src/main/java/org/apache/flume/source/SpoolDirectorySource.java index 7145580cba..957eb8bc24 100644 --- a/flume-ng-core/src/main/java/org/apache/flume/source/SpoolDirectorySource.java +++ b/flume-ng-core/src/main/java/org/apache/flume/source/SpoolDirectorySource.java @@ -24,6 +24,8 @@ import java.util.concurrent.ScheduledExecutorService; import java.util.concurrent.TimeUnit; +import com.google.common.annotations.VisibleForTesting; +import com.google.common.base.Throwables; import org.apache.flume.*; import org.apache.flume.client.avro.ReliableSpoolingFileEventReader; import org.apache.flume.conf.Configurable; @@ -58,17 +60,18 @@ public class SpoolDirectorySource extends AbstractSource implements private Context deserializerContext; private String deletePolicy; private String inputCharset; + private volatile boolean hasFatalError = false; private SourceCounter sourceCounter; ReliableSpoolingFileEventReader reader; + private ScheduledExecutorService executor; @Override - public void start() { + public synchronized void start() { logger.info("SpoolDirectorySource source starting with directory: {}", spoolDirectory); - ScheduledExecutorService executor = - Executors.newSingleThreadScheduledExecutor(); + executor = Executors.newSingleThreadScheduledExecutor(); File directory = new File(spoolDirectory); try { @@ -99,7 +102,15 @@ public void start() { } @Override - public void stop() { + public synchronized void stop() { + executor.shutdown(); + try { + executor.awaitTermination(10L, TimeUnit.SECONDS); + } catch (InterruptedException ex) { + logger.info("Interrupted while awaiting termination", ex); + } + executor.shutdownNow(); + super.stop(); sourceCounter.stop(); logger.info("SpoolDir source {} stopped. Metrics: {}", getName(), @@ -107,7 +118,13 @@ public void stop() { } @Override - public void configure(Context context) { + public String toString() { + return "Spool Directory source " + getName() + + ": { spoolDir: " + spoolDirectory + " }"; + } + + @Override + public synchronized void configure(Context context) { spoolDirectory = context.getString(SPOOL_DIRECTORY); Preconditions.checkState(spoolDirectory != null, "Configuration must specify a spooling directory"); @@ -143,6 +160,11 @@ public void configure(Context context) { } } + @VisibleForTesting + protected boolean hasFatalError() { + return hasFatalError; + } + private class SpoolDirectoryRunnable implements Runnable { private ReliableSpoolingFileEventReader reader; private SourceCounter sourceCounter; @@ -170,10 +192,11 @@ public void run() { sourceCounter.incrementAppendBatchAcceptedCount(); } } catch (Throwable t) { - logger.error("Uncaught exception in Runnable", t); - if (t instanceof Error) { - throw (Error) t; - } + logger.error("FATAL: " + SpoolDirectorySource.this.toString() + ": " + + "Uncaught exception in SpoolDirectorySource thread. " + + "Restart or reconfigure Flume to continue processing.", t); + hasFatalError = true; + Throwables.propagate(t); } } } diff --git a/flume-ng-core/src/test/java/org/apache/flume/source/TestSpoolDirectorySource.java b/flume-ng-core/src/test/java/org/apache/flume/source/TestSpoolDirectorySource.java index 652d2a290a..837cf152ba 100644 --- a/flume-ng-core/src/test/java/org/apache/flume/source/TestSpoolDirectorySource.java +++ b/flume-ng-core/src/test/java/org/apache/flume/source/TestSpoolDirectorySource.java @@ -21,8 +21,10 @@ import java.io.IOException; import java.util.ArrayList; import java.util.List; +import java.util.concurrent.TimeUnit; import org.apache.flume.Channel; +import org.apache.flume.ChannelException; import org.apache.flume.ChannelSelector; import org.apache.flume.Context; import org.apache.flume.Event; @@ -132,4 +134,33 @@ public void testLifecycle() throws IOException, InterruptedException { source.getLifecycleState()); } } + + @Test + public void testReconfigure() throws InterruptedException, IOException { + final int NUM_RECONFIGS = 20; + for (int i = 0; i < NUM_RECONFIGS; i++) { + Context context = new Context(); + File file = new File(tmpDir.getAbsolutePath() + "/file-" + i); + Files.write("File " + i, file, Charsets.UTF_8); + context.put(SpoolDirectorySourceConfigurationConstants.SPOOL_DIRECTORY, + tmpDir.getAbsolutePath()); + Configurables.configure(source, context); + source.start(); + Thread.sleep(TimeUnit.SECONDS.toMillis(1)); + Transaction txn = channel.getTransaction(); + txn.begin(); + try { + Event event = channel.take(); + String content = new String(event.getBody(), Charsets.UTF_8); + Assert.assertEquals("File " + i, content); + txn.commit(); + } catch (Throwable t) { + txn.rollback(); + } finally { + txn.close(); + } + source.stop(); + Assert.assertFalse("Fatal error on iteration " + i, source.hasFatalError()); + } + } } diff --git a/flume-ng-tests/src/test/java/org/apache/flume/test/agent/TestSpooldirSource.java b/flume-ng-tests/src/test/java/org/apache/flume/test/agent/TestSpooldirSource.java new file mode 100644 index 0000000000..601838078e --- /dev/null +++ b/flume-ng-tests/src/test/java/org/apache/flume/test/agent/TestSpooldirSource.java @@ -0,0 +1,174 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, + * software distributed under the License is distributed on an + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + * KIND, either express or implied. See the License for the + * specific language governing permissions and limitations + * under the License. + */ +package org.apache.flume.test.agent; + +import com.google.common.base.Charsets; +import com.google.common.base.Joiner; +import com.google.common.collect.Lists; +import com.google.common.collect.Sets; +import com.google.common.io.Files; +import org.apache.commons.io.FileUtils; +import org.apache.flume.test.util.StagedInstall; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +import java.io.File; +import java.io.IOException; +import java.util.ArrayList; +import java.util.Collection; +import java.util.List; +import java.util.Properties; +import java.util.Set; +import java.util.UUID; +import java.util.concurrent.TimeUnit; + +import org.junit.After; +import org.junit.Before; +import org.junit.Test; + +import static org.junit.Assert.*; + +public class TestSpooldirSource { + + private static final Logger LOGGER = + LoggerFactory.getLogger(TestSpooldirSource.class); + + private Properties agentProps; + private File sinkOutputDir; + private List spoolDirs = Lists.newArrayList(); + + @Before + public void setup() throws Exception { + + File agentDir = StagedInstall.getInstance().getStageDir(); + LOGGER.debug("Using agent stage dir: {}", agentDir); + + File testDir = new File(agentDir, TestSpooldirSource.class.getName()); + assertTrue(testDir.mkdirs()); + + File spoolParentDir = new File(testDir, "spools"); + assertTrue("Unable to create sink output dir: " + spoolParentDir.getPath(), + spoolParentDir.mkdir()); + + final int NUM_SOURCES = 100; + + agentProps = new Properties(); + List spooldirSrcNames = Lists.newArrayList(); + String channelName = "mem-01"; + + // Create source dirs and property file chunks + for (int i = 0; i < NUM_SOURCES; i++) { + String srcName = String.format("spooldir-%03d", i); + File spoolDir = new File(spoolParentDir, srcName); + assertTrue(spoolDir.mkdir()); + spooldirSrcNames.add(srcName); + spoolDirs.add(spoolDir); + + agentProps.put(String.format("agent.sources.%s.type", srcName), + "SPOOLDIR"); + agentProps.put(String.format("agent.sources.%s.spoolDir", srcName), + spoolDir.getPath()); + agentProps.put(String.format("agent.sources.%s.channels", srcName), + channelName); + } + + // Create the rest of the properties file + agentProps.put("agent.channels.mem-01.type", "MEMORY"); + agentProps.put("agent.channels.mem-01.capacity", String.valueOf(100000)); + + sinkOutputDir = new File(testDir, "out"); + assertTrue("Unable to create sink output dir: " + sinkOutputDir.getPath(), + sinkOutputDir.mkdir()); + + agentProps.put("agent.sinks.roll-01.channel", channelName); + agentProps.put("agent.sinks.roll-01.type", "FILE_ROLL"); + agentProps.put("agent.sinks.roll-01.sink.directory", sinkOutputDir.getPath()); + agentProps.put("agent.sinks.roll-01.sink.rollInterval", "0"); + + agentProps.put("agent.sources", Joiner.on(" ").join(spooldirSrcNames)); + agentProps.put("agent.channels", channelName); + agentProps.put("agent.sinks", "roll-01"); + } + + @After + public void teardown() throws Exception { + StagedInstall.getInstance().stopAgent(); + } + + private String getTestString(int dirNum, int fileNum) { + return String.format("Test dir %03d, test file %03d.\n", dirNum, fileNum); + } + + /** Create a bunch of test files. */ + private void createInputTestFiles(List spoolDirs, int numFiles, int startNum) + throws IOException { + int numSpoolDirs = spoolDirs.size(); + for (int dirNum = 0; dirNum < numSpoolDirs; dirNum++) { + File spoolDir = spoolDirs.get(dirNum); + for (int fileNum = startNum; fileNum < numFiles; fileNum++) { + // Stage the files on what is almost certainly the same FS partition. + File tmp = new File(spoolDir.getParent(), UUID.randomUUID().toString()); + Files.append(getTestString(dirNum, fileNum), tmp, Charsets.UTF_8); + File dst = new File(spoolDir, String.format("test-file-%03d", fileNum)); + // Ensure we move them into the spool directory atomically, if possible. + assertTrue(String.format("Failed to rename %s to %s", tmp, dst), + tmp.renameTo(dst)); + } + } + } + + private void validateSeenEvents(File outDir, int outFiles, int dirs, int events) + throws IOException { + File[] sinkOutputDirChildren = outDir.listFiles(); + assertEquals("Unexpected number of files in output dir", + outFiles, sinkOutputDirChildren.length); + Set seenEvents = Sets.newHashSet(); + for (File outFile : sinkOutputDirChildren) { + List lines = Files.readLines(outFile, Charsets.UTF_8); + for (String line : lines) { + seenEvents.add(line); + } + } + for (int dirNum = 0; dirNum < dirs; dirNum++) { + for (int fileNum = 0; fileNum < events; fileNum++) { + String event = getTestString(dirNum, fileNum).trim(); + assertTrue("Missing event: {" + event + "}", seenEvents.contains(event)); + } + } + } + + @Test + public void testManySpooldirs() throws Exception { + LOGGER.debug("testManySpooldirs() started."); + + StagedInstall.getInstance().startAgent("agent", agentProps); + + final int NUM_FILES_PER_DIR = 10; + createInputTestFiles(spoolDirs, NUM_FILES_PER_DIR, 0); + + TimeUnit.SECONDS.sleep(10); // Wait for sources and sink to process files + + // Ensure we received all events. + validateSeenEvents(sinkOutputDir,1, spoolDirs.size(), NUM_FILES_PER_DIR); + LOGGER.debug("Processed all the events!"); + + LOGGER.debug("testManySpooldirs() ended."); + } + +} diff --git a/flume-ng-tests/src/test/java/org/apache/flume/test/util/StagedInstall.java b/flume-ng-tests/src/test/java/org/apache/flume/test/util/StagedInstall.java index bc58340322..336ffc4768 100644 --- a/flume-ng-tests/src/test/java/org/apache/flume/test/util/StagedInstall.java +++ b/flume-ng-tests/src/test/java/org/apache/flume/test/util/StagedInstall.java @@ -29,8 +29,11 @@ import java.util.List; import java.util.Map; import java.util.Properties; +import java.util.concurrent.TimeUnit; import java.util.zip.GZIPInputStream; +import com.google.common.base.Preconditions; +import com.google.common.io.Files; import org.apache.commons.compress.archivers.tar.TarArchiveEntry; import org.apache.commons.compress.archivers.tar.TarArchiveInputStream; import org.apache.log4j.Logger; @@ -62,6 +65,7 @@ public class StagedInstall { private final String logDirPath; // State per invocation - config file, process, shutdown hook + private String agentName; private String configFilePath; private Process process; private ProcessShutdownHook shutdownHook; @@ -113,17 +117,22 @@ public synchronized void startAgent(String name, String configResource) public synchronized void startAgent(String name, Properties properties) throws Exception { + Preconditions.checkArgument(!name.isEmpty(), "agent name must not be empty"); + Preconditions.checkNotNull(properties, "properties object must not be null"); + + agentName = name; + if (process != null) { throw new Exception("A process is already running"); } - LOGGER.info("Starting process for agent: " + name + " using config: " + LOGGER.info("Starting process for agent: " + agentName + " using config: " + properties); - File configFile = createConfigurationFile(name, properties); + File configFile = createConfigurationFile(agentName, properties); configFilePath = configFile.getCanonicalPath(); String configFileName = configFile.getName(); - String logFileName = "flume-" + name + "-" + String logFileName = "flume-" + agentName + "-" + configFileName.substring(0, configFileName.indexOf('.')) + ".log"; LOGGER.info("Created configuration file: " + configFilePath); @@ -136,7 +145,7 @@ public synchronized void startAgent(String name, Properties properties) builder.add("--classpath", agentClasspath); } builder.add("--conf-file", configFilePath); - builder.add("--name", name); + builder.add("--name", agentName); builder.add("-D" + ENV_FLUME_LOG_DIR + "=" + logDirPath); builder.add("-D" + ENV_FLUME_ROOT_LOGGER + "=" + ENV_FLUME_ROOT_LOGGER_VALUE); @@ -168,8 +177,21 @@ public synchronized void setAgentClasspath(String agentClasspath) { this.agentClasspath = agentClasspath; } + public synchronized void reconfigure(Properties properties) throws Exception { + File configFile = createConfigurationFile(agentName, properties); + Files.copy(configFile, new File(configFilePath)); + configFile.delete(); + LOGGER.info("Updated agent config file: " + configFilePath); + } + + public synchronized File getStageDir() { + return stageDir; + } + private File createConfigurationFile(String agentName, Properties properties) throws Exception { + Preconditions.checkNotNull(properties, "properties object must not be null"); + File file = File.createTempFile("agent", "config.properties", stageDir); OutputStream os = null; From 507ff137160021dda2c4a08199746228827c9432 Mon Sep 17 00:00:00 2001 From: Hari Shreedharan Date: Sun, 11 Aug 2013 00:11:48 -0700 Subject: [PATCH 013/341] FLUME-2156. Unregister then re-register MonitoredCounterGroup JMX MBeans on reconfigure (Mike Percy via Hari Shreedharan) --- .../MonitoredCounterGroup.java | 20 ++++++++++++----- .../TestMonitoredCounterGroup.java | 22 +++++++++++++++++++ 2 files changed, 37 insertions(+), 5 deletions(-) diff --git a/flume-ng-core/src/main/java/org/apache/flume/instrumentation/MonitoredCounterGroup.java b/flume-ng-core/src/main/java/org/apache/flume/instrumentation/MonitoredCounterGroup.java index c5c2956998..1d3f0f1a23 100644 --- a/flume-ng-core/src/main/java/org/apache/flume/instrumentation/MonitoredCounterGroup.java +++ b/flume-ng-core/src/main/java/org/apache/flume/instrumentation/MonitoredCounterGroup.java @@ -29,6 +29,7 @@ import javax.management.ObjectName; +import com.google.common.annotations.VisibleForTesting; import org.slf4j.Logger; import org.slf4j.LoggerFactory; @@ -95,20 +96,29 @@ public void start() { } /** - * Registers the counter. This method should be used only for testing, and - * there should be no need for any implementations to directly call this - * method. + * Registers the counter. + * This method is exposed only for testing, and there should be no need for + * any implementations to call this method directly. */ + @VisibleForTesting void register() { if (!registered) { try { ObjectName objName = new ObjectName("org.apache.flume." + type.name().toLowerCase() + ":type=" + this.name); + if (ManagementFactory.getPlatformMBeanServer().isRegistered(objName)) { + logger.debug("Monitored counter group for type: " + type + ", name: " + + name + ": Another MBean is already registered with this name. " + + "Unregistering that pre-existing MBean now..."); + ManagementFactory.getPlatformMBeanServer().unregisterMBean(objName); + logger.debug("Monitored counter group for type: " + type + ", name: " + + name + ": Successfully unregistered pre-existing MBean."); + } ManagementFactory.getPlatformMBeanServer().registerMBean(this, objName); + logger.info("Monitored counter group for type: " + type + ", name: " + + name + ": Successfully registered new MBean."); registered = true; - logger.info("Monitoried counter group for type: " + type + ", name: " + name - + ", registered successfully."); } catch (Exception ex) { logger.error("Failed to register monitored counter group for type: " + type + ", name: " + name, ex); diff --git a/flume-ng-core/src/test/java/org/apache/flume/instrumentation/TestMonitoredCounterGroup.java b/flume-ng-core/src/test/java/org/apache/flume/instrumentation/TestMonitoredCounterGroup.java index e417fb3cc7..b1f637f1d2 100644 --- a/flume-ng-core/src/test/java/org/apache/flume/instrumentation/TestMonitoredCounterGroup.java +++ b/flume-ng-core/src/test/java/org/apache/flume/instrumentation/TestMonitoredCounterGroup.java @@ -314,6 +314,28 @@ public void testSourceCounter() throws Exception { 0L, 0L, 0L, 0L); } + @Test + public void testRegisterTwice() throws Exception { + String name = "re-register-" + getRandomName(); + + SourceCounter c1 = new SourceCounter(name); + c1.register(); + ObjectName on = new ObjectName(SOURCE_OBJ_NAME_PREFIX + name); + + Assert.assertEquals("StartTime", 0L, getStartTime(on)); + Assert.assertEquals("StopTime", 0L, getStopTime(on)); + c1.start(); + c1.stop(); + Assert.assertTrue("StartTime", getStartTime(on) > 0L); + Assert.assertTrue("StopTime", getStopTime(on) > 0L); + + SourceCounter c2 = new SourceCounter(name); + c2.register(); + + Assert.assertEquals("StartTime", 0L, getStartTime(on)); + Assert.assertEquals("StopTime", 0L, getStopTime(on)); + } + private void assertSrcCounterState(ObjectName on, long eventReceivedCount, long eventAcceptedCount, long appendReceivedCount, long appendAcceptedCount, long appendBatchReceivedCount, From 37e56e85b08869e01dce98b95879a09f1bf57a92 Mon Sep 17 00:00:00 2001 From: Hari Shreedharan Date: Wed, 14 Aug 2013 18:23:51 -0700 Subject: [PATCH 014/341] FLUME-2161. Flume does not support spaces in -X java-opt command line args (Mike Percy via Hari Shreedharan) --- bin/flume-ng | 16 +++++++--------- 1 file changed, 7 insertions(+), 9 deletions(-) diff --git a/bin/flume-ng b/bin/flume-ng index 65cc985210..3a8a10da2c 100755 --- a/bin/flume-ng +++ b/bin/flume-ng @@ -224,7 +224,7 @@ run_flume() { if [ ${CLEAN_FLAG} -ne 0 ]; then set -x fi - $EXEC $JAVA_HOME/bin/java $JAVA_OPTS -cp "$FLUME_CLASSPATH" \ + $EXEC $JAVA_HOME/bin/java $JAVA_OPTS "${arr_java_props[@]}" -cp "$FLUME_CLASSPATH" \ -Djava.library.path=$FLUME_JAVA_LIBRARY_PATH "$FLUME_APPLICATION_CLASS" $* } @@ -241,7 +241,8 @@ LD_LIBRARY_PATH="" opt_conf="" opt_classpath="" opt_plugins_dirs="" -opt_java_props="" +arr_java_props=() +arr_java_props_ct=0 opt_dryrun="" mode=$1 @@ -301,10 +302,12 @@ while [ -n "$*" ] ; do shift ;; -D*) - opt_java_props="$opt_java_props $arg" + arr_java_props[arr_java_props_ct]=$arg + ((++arr_java_props_ct)) ;; -X*) - opt_java_props="$opt_java_props $arg" + arr_java_props[arr_java_props_ct]=$arg + ((++arr_java_props_ct)) ;; *) args="$args $arg" @@ -325,11 +328,6 @@ elif [ -f "$opt_conf/flume-env.sh" ]; then source "$opt_conf/flume-env.sh" fi -# append command-line java options to stock or env script JAVA_OPTS -if [ -n "${opt_java_props}" ]; then - JAVA_OPTS="${JAVA_OPTS} ${opt_java_props}" -fi - # prepend command-line classpath to env script classpath if [ -n "${opt_classpath}" ]; then if [ -n "${FLUME_CLASSPATH}" ]; then From 088067cf41403978b262470a87b5dc094a568dd7 Mon Sep 17 00:00:00 2001 From: Hari Shreedharan Date: Wed, 21 Aug 2013 23:52:15 -0700 Subject: [PATCH 015/341] FLUME-2174. Integration of morphline solr sink puts Lucene/Solr dependencies on the default classpath (Roman Shaposhnik via Hari Shreedharan) --- .../flume-ng-morphline-solr-sink/pom.xml | 65 +++++-------------- 1 file changed, 16 insertions(+), 49 deletions(-) diff --git a/flume-ng-sinks/flume-ng-morphline-solr-sink/pom.xml b/flume-ng-sinks/flume-ng-morphline-solr-sink/pom.xml index 7cbc9b9cb3..fd6b4ea455 100644 --- a/flume-ng-sinks/flume-ng-morphline-solr-sink/pom.xml +++ b/flume-ng-sinks/flume-ng-morphline-solr-sink/pom.xml @@ -34,7 +34,7 @@ limitations under the License. 4.3.0 4.3.0 1.3 - 0.5.0 + 0.6.0 1.6.1 2.12.4 @@ -74,6 +74,14 @@ limitations under the License. slf4j-api + + com.cloudera.cdk + cdk-morphlines-all + ${cdk.version} + pom + true + + org.slf4j jcl-over-slf4j @@ -86,48 +94,19 @@ limitations under the License. solr-test-framework ${solr.version} test - - - - com.cloudera.cdk - cdk-morphlines-solr-cell - ${cdk.version} - - - - com.cloudera.cdk - cdk-morphlines-avro - ${cdk.version} - - - - com.cloudera.cdk - cdk-morphlines-json - ${cdk.version} - - - - com.cloudera.cdk - cdk-morphlines-twitter - ${cdk.version} - - - - com.cloudera.cdk - cdk-morphlines-tika-core - ${cdk.version} - - - - com.cloudera.cdk - cdk-morphlines-tika-decompress - ${cdk.version} + + + org.slf4j + slf4j-jdk14 + + org.apache.tika tika-xmp ${tika.version} + test org.apache.geronimo.specs @@ -148,18 +127,6 @@ limitations under the License. test - - org.apache.solr - solr-core - ${solr.version} - - - org.slf4j - slf4j-jdk14 - - - - junit junit From 6e461f13c25a2a2e6c50c1dd06b38b600036ee9f Mon Sep 17 00:00:00 2001 From: Hari Shreedharan Date: Wed, 11 Sep 2013 17:18:22 -0700 Subject: [PATCH 016/341] FLUME-2184. flume-ng-morphline-solr-sink build failing due to incorrect hadoop-common dependency declaration (Jagat Singh via Hari Shreedharan) --- flume-ng-sinks/flume-ng-morphline-solr-sink/pom.xml | 6 ++++++ 1 file changed, 6 insertions(+) diff --git a/flume-ng-sinks/flume-ng-morphline-solr-sink/pom.xml b/flume-ng-sinks/flume-ng-morphline-solr-sink/pom.xml index fd6b4ea455..a2fb931e86 100644 --- a/flume-ng-sinks/flume-ng-morphline-solr-sink/pom.xml +++ b/flume-ng-sinks/flume-ng-morphline-solr-sink/pom.xml @@ -78,6 +78,12 @@ limitations under the License. com.cloudera.cdk cdk-morphlines-all ${cdk.version} + + + org.apache.hadoop + hadoop-common + + pom true From 39b6b5282938b9429cee3acaac0763cedd8fa9f0 Mon Sep 17 00:00:00 2001 From: Hari Shreedharan Date: Wed, 11 Sep 2013 17:45:05 -0700 Subject: [PATCH 017/341] FLUME-2172. Update protocol buffer from 2.4.1 to 2.5.0 (Roshan Naik via Hari Shreedharan) --- .../channel/file/proto/ProtosFactory.java | 4278 +++++++++++------ pom.xml | 2 +- 2 files changed, 2845 insertions(+), 1435 deletions(-) diff --git a/flume-ng-channels/flume-file-channel/src/main/java/org/apache/flume/channel/file/proto/ProtosFactory.java b/flume-ng-channels/flume-file-channel/src/main/java/org/apache/flume/channel/file/proto/ProtosFactory.java index 6885832750..34f93d9b93 100644 --- a/flume-ng-channels/flume-file-channel/src/main/java/org/apache/flume/channel/file/proto/ProtosFactory.java +++ b/flume-ng-channels/flume-file-channel/src/main/java/org/apache/flume/channel/file/proto/ProtosFactory.java @@ -28,123 +28,294 @@ public static void registerAllExtensions( } public interface CheckpointOrBuilder extends com.google.protobuf.MessageOrBuilder { - + // required sfixed32 version = 1; + /** + * required sfixed32 version = 1; + */ boolean hasVersion(); + /** + * required sfixed32 version = 1; + */ int getVersion(); - + // required sfixed64 writeOrderID = 2; + /** + * required sfixed64 writeOrderID = 2; + */ boolean hasWriteOrderID(); + /** + * required sfixed64 writeOrderID = 2; + */ long getWriteOrderID(); - + // required sfixed32 queueSize = 3; + /** + * required sfixed32 queueSize = 3; + */ boolean hasQueueSize(); + /** + * required sfixed32 queueSize = 3; + */ int getQueueSize(); - + // required sfixed32 queueHead = 4; + /** + * required sfixed32 queueHead = 4; + */ boolean hasQueueHead(); + /** + * required sfixed32 queueHead = 4; + */ int getQueueHead(); - + // repeated .ActiveLog activeLogs = 5; + /** + * repeated .ActiveLog activeLogs = 5; + */ java.util.List getActiveLogsList(); + /** + * repeated .ActiveLog activeLogs = 5; + */ org.apache.flume.channel.file.proto.ProtosFactory.ActiveLog getActiveLogs(int index); + /** + * repeated .ActiveLog activeLogs = 5; + */ int getActiveLogsCount(); + /** + * repeated .ActiveLog activeLogs = 5; + */ java.util.List getActiveLogsOrBuilderList(); + /** + * repeated .ActiveLog activeLogs = 5; + */ org.apache.flume.channel.file.proto.ProtosFactory.ActiveLogOrBuilder getActiveLogsOrBuilder( int index); } + /** + * Protobuf type {@code Checkpoint} + */ public static final class Checkpoint extends com.google.protobuf.GeneratedMessage implements CheckpointOrBuilder { // Use Checkpoint.newBuilder() to construct. - private Checkpoint(Builder builder) { + private Checkpoint(com.google.protobuf.GeneratedMessage.Builder builder) { super(builder); + this.unknownFields = builder.getUnknownFields(); } - private Checkpoint(boolean noInit) {} - + private Checkpoint(boolean noInit) { this.unknownFields = com.google.protobuf.UnknownFieldSet.getDefaultInstance(); } + private static final Checkpoint defaultInstance; public static Checkpoint getDefaultInstance() { return defaultInstance; } - + public Checkpoint getDefaultInstanceForType() { return defaultInstance; } - + + private final com.google.protobuf.UnknownFieldSet unknownFields; + @java.lang.Override + public final com.google.protobuf.UnknownFieldSet + getUnknownFields() { + return this.unknownFields; + } + private Checkpoint( + com.google.protobuf.CodedInputStream input, + com.google.protobuf.ExtensionRegistryLite extensionRegistry) + throws com.google.protobuf.InvalidProtocolBufferException { + initFields(); + int mutable_bitField0_ = 0; + com.google.protobuf.UnknownFieldSet.Builder unknownFields = + com.google.protobuf.UnknownFieldSet.newBuilder(); + try { + boolean done = false; + while (!done) { + int tag = input.readTag(); + switch (tag) { + case 0: + done = true; + break; + default: { + if (!parseUnknownField(input, unknownFields, + extensionRegistry, tag)) { + done = true; + } + break; + } + case 13: { + bitField0_ |= 0x00000001; + version_ = input.readSFixed32(); + break; + } + case 17: { + bitField0_ |= 0x00000002; + writeOrderID_ = input.readSFixed64(); + break; + } + case 29: { + bitField0_ |= 0x00000004; + queueSize_ = input.readSFixed32(); + break; + } + case 37: { + bitField0_ |= 0x00000008; + queueHead_ = input.readSFixed32(); + break; + } + case 42: { + if (!((mutable_bitField0_ & 0x00000010) == 0x00000010)) { + activeLogs_ = new java.util.ArrayList(); + mutable_bitField0_ |= 0x00000010; + } + activeLogs_.add(input.readMessage(org.apache.flume.channel.file.proto.ProtosFactory.ActiveLog.PARSER, extensionRegistry)); + break; + } + } + } + } catch (com.google.protobuf.InvalidProtocolBufferException e) { + throw e.setUnfinishedMessage(this); + } catch (java.io.IOException e) { + throw new com.google.protobuf.InvalidProtocolBufferException( + e.getMessage()).setUnfinishedMessage(this); + } finally { + if (((mutable_bitField0_ & 0x00000010) == 0x00000010)) { + activeLogs_ = java.util.Collections.unmodifiableList(activeLogs_); + } + this.unknownFields = unknownFields.build(); + makeExtensionsImmutable(); + } + } public static final com.google.protobuf.Descriptors.Descriptor getDescriptor() { return org.apache.flume.channel.file.proto.ProtosFactory.internal_static_Checkpoint_descriptor; } - + protected com.google.protobuf.GeneratedMessage.FieldAccessorTable internalGetFieldAccessorTable() { - return org.apache.flume.channel.file.proto.ProtosFactory.internal_static_Checkpoint_fieldAccessorTable; + return org.apache.flume.channel.file.proto.ProtosFactory.internal_static_Checkpoint_fieldAccessorTable + .ensureFieldAccessorsInitialized( + org.apache.flume.channel.file.proto.ProtosFactory.Checkpoint.class, org.apache.flume.channel.file.proto.ProtosFactory.Checkpoint.Builder.class); + } + + public static com.google.protobuf.Parser PARSER = + new com.google.protobuf.AbstractParser() { + public Checkpoint parsePartialFrom( + com.google.protobuf.CodedInputStream input, + com.google.protobuf.ExtensionRegistryLite extensionRegistry) + throws com.google.protobuf.InvalidProtocolBufferException { + return new Checkpoint(input, extensionRegistry); + } + }; + + @java.lang.Override + public com.google.protobuf.Parser getParserForType() { + return PARSER; } - + private int bitField0_; // required sfixed32 version = 1; public static final int VERSION_FIELD_NUMBER = 1; private int version_; + /** + * required sfixed32 version = 1; + */ public boolean hasVersion() { return ((bitField0_ & 0x00000001) == 0x00000001); } + /** + * required sfixed32 version = 1; + */ public int getVersion() { return version_; } - + // required sfixed64 writeOrderID = 2; public static final int WRITEORDERID_FIELD_NUMBER = 2; private long writeOrderID_; + /** + * required sfixed64 writeOrderID = 2; + */ public boolean hasWriteOrderID() { return ((bitField0_ & 0x00000002) == 0x00000002); } + /** + * required sfixed64 writeOrderID = 2; + */ public long getWriteOrderID() { return writeOrderID_; } - + // required sfixed32 queueSize = 3; public static final int QUEUESIZE_FIELD_NUMBER = 3; private int queueSize_; + /** + * required sfixed32 queueSize = 3; + */ public boolean hasQueueSize() { return ((bitField0_ & 0x00000004) == 0x00000004); } + /** + * required sfixed32 queueSize = 3; + */ public int getQueueSize() { return queueSize_; } - + // required sfixed32 queueHead = 4; public static final int QUEUEHEAD_FIELD_NUMBER = 4; private int queueHead_; + /** + * required sfixed32 queueHead = 4; + */ public boolean hasQueueHead() { return ((bitField0_ & 0x00000008) == 0x00000008); } + /** + * required sfixed32 queueHead = 4; + */ public int getQueueHead() { return queueHead_; } - + // repeated .ActiveLog activeLogs = 5; public static final int ACTIVELOGS_FIELD_NUMBER = 5; private java.util.List activeLogs_; + /** + * repeated .ActiveLog activeLogs = 5; + */ public java.util.List getActiveLogsList() { return activeLogs_; } + /** + * repeated .ActiveLog activeLogs = 5; + */ public java.util.List getActiveLogsOrBuilderList() { return activeLogs_; } + /** + * repeated .ActiveLog activeLogs = 5; + */ public int getActiveLogsCount() { return activeLogs_.size(); } + /** + * repeated .ActiveLog activeLogs = 5; + */ public org.apache.flume.channel.file.proto.ProtosFactory.ActiveLog getActiveLogs(int index) { return activeLogs_.get(index); } + /** + * repeated .ActiveLog activeLogs = 5; + */ public org.apache.flume.channel.file.proto.ProtosFactory.ActiveLogOrBuilder getActiveLogsOrBuilder( int index) { return activeLogs_.get(index); } - + private void initFields() { version_ = 0; writeOrderID_ = 0L; @@ -156,7 +327,7 @@ private void initFields() { public final boolean isInitialized() { byte isInitialized = memoizedIsInitialized; if (isInitialized != -1) return isInitialized == 1; - + if (!hasVersion()) { memoizedIsInitialized = 0; return false; @@ -182,7 +353,7 @@ public final boolean isInitialized() { memoizedIsInitialized = 1; return true; } - + public void writeTo(com.google.protobuf.CodedOutputStream output) throws java.io.IOException { getSerializedSize(); @@ -203,12 +374,12 @@ public void writeTo(com.google.protobuf.CodedOutputStream output) } getUnknownFields().writeTo(output); } - + private int memoizedSerializedSize = -1; public int getSerializedSize() { int size = memoizedSerializedSize; if (size != -1) return size; - + size = 0; if (((bitField0_ & 0x00000001) == 0x00000001)) { size += com.google.protobuf.CodedOutputStream @@ -234,94 +405,83 @@ public int getSerializedSize() { memoizedSerializedSize = size; return size; } - + private static final long serialVersionUID = 0L; @java.lang.Override protected java.lang.Object writeReplace() throws java.io.ObjectStreamException { return super.writeReplace(); } - + public static org.apache.flume.channel.file.proto.ProtosFactory.Checkpoint parseFrom( com.google.protobuf.ByteString data) throws com.google.protobuf.InvalidProtocolBufferException { - return newBuilder().mergeFrom(data).buildParsed(); + return PARSER.parseFrom(data); } public static org.apache.flume.channel.file.proto.ProtosFactory.Checkpoint parseFrom( com.google.protobuf.ByteString data, com.google.protobuf.ExtensionRegistryLite extensionRegistry) throws com.google.protobuf.InvalidProtocolBufferException { - return newBuilder().mergeFrom(data, extensionRegistry) - .buildParsed(); + return PARSER.parseFrom(data, extensionRegistry); } public static org.apache.flume.channel.file.proto.ProtosFactory.Checkpoint parseFrom(byte[] data) throws com.google.protobuf.InvalidProtocolBufferException { - return newBuilder().mergeFrom(data).buildParsed(); + return PARSER.parseFrom(data); } public static org.apache.flume.channel.file.proto.ProtosFactory.Checkpoint parseFrom( byte[] data, com.google.protobuf.ExtensionRegistryLite extensionRegistry) throws com.google.protobuf.InvalidProtocolBufferException { - return newBuilder().mergeFrom(data, extensionRegistry) - .buildParsed(); + return PARSER.parseFrom(data, extensionRegistry); } public static org.apache.flume.channel.file.proto.ProtosFactory.Checkpoint parseFrom(java.io.InputStream input) throws java.io.IOException { - return newBuilder().mergeFrom(input).buildParsed(); + return PARSER.parseFrom(input); } public static org.apache.flume.channel.file.proto.ProtosFactory.Checkpoint parseFrom( java.io.InputStream input, com.google.protobuf.ExtensionRegistryLite extensionRegistry) throws java.io.IOException { - return newBuilder().mergeFrom(input, extensionRegistry) - .buildParsed(); + return PARSER.parseFrom(input, extensionRegistry); } public static org.apache.flume.channel.file.proto.ProtosFactory.Checkpoint parseDelimitedFrom(java.io.InputStream input) throws java.io.IOException { - Builder builder = newBuilder(); - if (builder.mergeDelimitedFrom(input)) { - return builder.buildParsed(); - } else { - return null; - } + return PARSER.parseDelimitedFrom(input); } public static org.apache.flume.channel.file.proto.ProtosFactory.Checkpoint parseDelimitedFrom( java.io.InputStream input, com.google.protobuf.ExtensionRegistryLite extensionRegistry) throws java.io.IOException { - Builder builder = newBuilder(); - if (builder.mergeDelimitedFrom(input, extensionRegistry)) { - return builder.buildParsed(); - } else { - return null; - } + return PARSER.parseDelimitedFrom(input, extensionRegistry); } public static org.apache.flume.channel.file.proto.ProtosFactory.Checkpoint parseFrom( com.google.protobuf.CodedInputStream input) throws java.io.IOException { - return newBuilder().mergeFrom(input).buildParsed(); + return PARSER.parseFrom(input); } public static org.apache.flume.channel.file.proto.ProtosFactory.Checkpoint parseFrom( com.google.protobuf.CodedInputStream input, com.google.protobuf.ExtensionRegistryLite extensionRegistry) throws java.io.IOException { - return newBuilder().mergeFrom(input, extensionRegistry) - .buildParsed(); + return PARSER.parseFrom(input, extensionRegistry); } - + public static Builder newBuilder() { return Builder.create(); } public Builder newBuilderForType() { return newBuilder(); } public static Builder newBuilder(org.apache.flume.channel.file.proto.ProtosFactory.Checkpoint prototype) { return newBuilder().mergeFrom(prototype); } public Builder toBuilder() { return newBuilder(this); } - + @java.lang.Override protected Builder newBuilderForType( com.google.protobuf.GeneratedMessage.BuilderParent parent) { Builder builder = new Builder(parent); return builder; } + /** + * Protobuf type {@code Checkpoint} + */ public static final class Builder extends com.google.protobuf.GeneratedMessage.Builder implements org.apache.flume.channel.file.proto.ProtosFactory.CheckpointOrBuilder { @@ -329,18 +489,21 @@ public static final class Builder extends getDescriptor() { return org.apache.flume.channel.file.proto.ProtosFactory.internal_static_Checkpoint_descriptor; } - + protected com.google.protobuf.GeneratedMessage.FieldAccessorTable internalGetFieldAccessorTable() { - return org.apache.flume.channel.file.proto.ProtosFactory.internal_static_Checkpoint_fieldAccessorTable; + return org.apache.flume.channel.file.proto.ProtosFactory.internal_static_Checkpoint_fieldAccessorTable + .ensureFieldAccessorsInitialized( + org.apache.flume.channel.file.proto.ProtosFactory.Checkpoint.class, org.apache.flume.channel.file.proto.ProtosFactory.Checkpoint.Builder.class); } - + // Construct using org.apache.flume.channel.file.proto.ProtosFactory.Checkpoint.newBuilder() private Builder() { maybeForceBuilderInitialization(); } - - private Builder(BuilderParent parent) { + + private Builder( + com.google.protobuf.GeneratedMessage.BuilderParent parent) { super(parent); maybeForceBuilderInitialization(); } @@ -352,7 +515,7 @@ private void maybeForceBuilderInitialization() { private static Builder create() { return new Builder(); } - + public Builder clear() { super.clear(); version_ = 0; @@ -371,20 +534,20 @@ public Builder clear() { } return this; } - + public Builder clone() { return create().mergeFrom(buildPartial()); } - + public com.google.protobuf.Descriptors.Descriptor getDescriptorForType() { - return org.apache.flume.channel.file.proto.ProtosFactory.Checkpoint.getDescriptor(); + return org.apache.flume.channel.file.proto.ProtosFactory.internal_static_Checkpoint_descriptor; } - + public org.apache.flume.channel.file.proto.ProtosFactory.Checkpoint getDefaultInstanceForType() { return org.apache.flume.channel.file.proto.ProtosFactory.Checkpoint.getDefaultInstance(); } - + public org.apache.flume.channel.file.proto.ProtosFactory.Checkpoint build() { org.apache.flume.channel.file.proto.ProtosFactory.Checkpoint result = buildPartial(); if (!result.isInitialized()) { @@ -392,17 +555,7 @@ public org.apache.flume.channel.file.proto.ProtosFactory.Checkpoint build() { } return result; } - - private org.apache.flume.channel.file.proto.ProtosFactory.Checkpoint buildParsed() - throws com.google.protobuf.InvalidProtocolBufferException { - org.apache.flume.channel.file.proto.ProtosFactory.Checkpoint result = buildPartial(); - if (!result.isInitialized()) { - throw newUninitializedMessageException( - result).asInvalidProtocolBufferException(); - } - return result; - } - + public org.apache.flume.channel.file.proto.ProtosFactory.Checkpoint buildPartial() { org.apache.flume.channel.file.proto.ProtosFactory.Checkpoint result = new org.apache.flume.channel.file.proto.ProtosFactory.Checkpoint(this); int from_bitField0_ = bitField0_; @@ -436,7 +589,7 @@ public org.apache.flume.channel.file.proto.ProtosFactory.Checkpoint buildPartial onBuilt(); return result; } - + public Builder mergeFrom(com.google.protobuf.Message other) { if (other instanceof org.apache.flume.channel.file.proto.ProtosFactory.Checkpoint) { return mergeFrom((org.apache.flume.channel.file.proto.ProtosFactory.Checkpoint)other); @@ -445,7 +598,7 @@ public Builder mergeFrom(com.google.protobuf.Message other) { return this; } } - + public Builder mergeFrom(org.apache.flume.channel.file.proto.ProtosFactory.Checkpoint other) { if (other == org.apache.flume.channel.file.proto.ProtosFactory.Checkpoint.getDefaultInstance()) return this; if (other.hasVersion()) { @@ -489,7 +642,7 @@ public Builder mergeFrom(org.apache.flume.channel.file.proto.ProtosFactory.Check this.mergeUnknownFields(other.getUnknownFields()); return this; } - + public final boolean isInitialized() { if (!hasVersion()) { @@ -515,146 +668,158 @@ public final boolean isInitialized() { } return true; } - + public Builder mergeFrom( com.google.protobuf.CodedInputStream input, com.google.protobuf.ExtensionRegistryLite extensionRegistry) throws java.io.IOException { - com.google.protobuf.UnknownFieldSet.Builder unknownFields = - com.google.protobuf.UnknownFieldSet.newBuilder( - this.getUnknownFields()); - while (true) { - int tag = input.readTag(); - switch (tag) { - case 0: - this.setUnknownFields(unknownFields.build()); - onChanged(); - return this; - default: { - if (!parseUnknownField(input, unknownFields, - extensionRegistry, tag)) { - this.setUnknownFields(unknownFields.build()); - onChanged(); - return this; - } - break; - } - case 13: { - bitField0_ |= 0x00000001; - version_ = input.readSFixed32(); - break; - } - case 17: { - bitField0_ |= 0x00000002; - writeOrderID_ = input.readSFixed64(); - break; - } - case 29: { - bitField0_ |= 0x00000004; - queueSize_ = input.readSFixed32(); - break; - } - case 37: { - bitField0_ |= 0x00000008; - queueHead_ = input.readSFixed32(); - break; - } - case 42: { - org.apache.flume.channel.file.proto.ProtosFactory.ActiveLog.Builder subBuilder = org.apache.flume.channel.file.proto.ProtosFactory.ActiveLog.newBuilder(); - input.readMessage(subBuilder, extensionRegistry); - addActiveLogs(subBuilder.buildPartial()); - break; - } + org.apache.flume.channel.file.proto.ProtosFactory.Checkpoint parsedMessage = null; + try { + parsedMessage = PARSER.parsePartialFrom(input, extensionRegistry); + } catch (com.google.protobuf.InvalidProtocolBufferException e) { + parsedMessage = (org.apache.flume.channel.file.proto.ProtosFactory.Checkpoint) e.getUnfinishedMessage(); + throw e; + } finally { + if (parsedMessage != null) { + mergeFrom(parsedMessage); } } + return this; } - private int bitField0_; - + // required sfixed32 version = 1; private int version_ ; + /** + * required sfixed32 version = 1; + */ public boolean hasVersion() { return ((bitField0_ & 0x00000001) == 0x00000001); } + /** + * required sfixed32 version = 1; + */ public int getVersion() { return version_; } + /** + * required sfixed32 version = 1; + */ public Builder setVersion(int value) { bitField0_ |= 0x00000001; version_ = value; onChanged(); return this; } + /** + * required sfixed32 version = 1; + */ public Builder clearVersion() { bitField0_ = (bitField0_ & ~0x00000001); version_ = 0; onChanged(); return this; } - + // required sfixed64 writeOrderID = 2; private long writeOrderID_ ; + /** + * required sfixed64 writeOrderID = 2; + */ public boolean hasWriteOrderID() { return ((bitField0_ & 0x00000002) == 0x00000002); } + /** + * required sfixed64 writeOrderID = 2; + */ public long getWriteOrderID() { return writeOrderID_; } + /** + * required sfixed64 writeOrderID = 2; + */ public Builder setWriteOrderID(long value) { bitField0_ |= 0x00000002; writeOrderID_ = value; onChanged(); return this; } + /** + * required sfixed64 writeOrderID = 2; + */ public Builder clearWriteOrderID() { bitField0_ = (bitField0_ & ~0x00000002); writeOrderID_ = 0L; onChanged(); return this; } - + // required sfixed32 queueSize = 3; private int queueSize_ ; + /** + * required sfixed32 queueSize = 3; + */ public boolean hasQueueSize() { return ((bitField0_ & 0x00000004) == 0x00000004); } + /** + * required sfixed32 queueSize = 3; + */ public int getQueueSize() { return queueSize_; } + /** + * required sfixed32 queueSize = 3; + */ public Builder setQueueSize(int value) { bitField0_ |= 0x00000004; queueSize_ = value; onChanged(); return this; } + /** + * required sfixed32 queueSize = 3; + */ public Builder clearQueueSize() { bitField0_ = (bitField0_ & ~0x00000004); queueSize_ = 0; onChanged(); return this; } - + // required sfixed32 queueHead = 4; private int queueHead_ ; + /** + * required sfixed32 queueHead = 4; + */ public boolean hasQueueHead() { return ((bitField0_ & 0x00000008) == 0x00000008); } + /** + * required sfixed32 queueHead = 4; + */ public int getQueueHead() { return queueHead_; } + /** + * required sfixed32 queueHead = 4; + */ public Builder setQueueHead(int value) { bitField0_ |= 0x00000008; queueHead_ = value; onChanged(); return this; } + /** + * required sfixed32 queueHead = 4; + */ public Builder clearQueueHead() { bitField0_ = (bitField0_ & ~0x00000008); queueHead_ = 0; onChanged(); return this; } - + // repeated .ActiveLog activeLogs = 5; private java.util.List activeLogs_ = java.util.Collections.emptyList(); @@ -664,10 +829,13 @@ private void ensureActiveLogsIsMutable() { bitField0_ |= 0x00000010; } } - + private com.google.protobuf.RepeatedFieldBuilder< org.apache.flume.channel.file.proto.ProtosFactory.ActiveLog, org.apache.flume.channel.file.proto.ProtosFactory.ActiveLog.Builder, org.apache.flume.channel.file.proto.ProtosFactory.ActiveLogOrBuilder> activeLogsBuilder_; - + + /** + * repeated .ActiveLog activeLogs = 5; + */ public java.util.List getActiveLogsList() { if (activeLogsBuilder_ == null) { return java.util.Collections.unmodifiableList(activeLogs_); @@ -675,6 +843,9 @@ public java.util.Listrepeated .ActiveLog activeLogs = 5; + */ public int getActiveLogsCount() { if (activeLogsBuilder_ == null) { return activeLogs_.size(); @@ -682,6 +853,9 @@ public int getActiveLogsCount() { return activeLogsBuilder_.getCount(); } } + /** + * repeated .ActiveLog activeLogs = 5; + */ public org.apache.flume.channel.file.proto.ProtosFactory.ActiveLog getActiveLogs(int index) { if (activeLogsBuilder_ == null) { return activeLogs_.get(index); @@ -689,6 +863,9 @@ public org.apache.flume.channel.file.proto.ProtosFactory.ActiveLog getActiveLogs return activeLogsBuilder_.getMessage(index); } } + /** + * repeated .ActiveLog activeLogs = 5; + */ public Builder setActiveLogs( int index, org.apache.flume.channel.file.proto.ProtosFactory.ActiveLog value) { if (activeLogsBuilder_ == null) { @@ -703,6 +880,9 @@ public Builder setActiveLogs( } return this; } + /** + * repeated .ActiveLog activeLogs = 5; + */ public Builder setActiveLogs( int index, org.apache.flume.channel.file.proto.ProtosFactory.ActiveLog.Builder builderForValue) { if (activeLogsBuilder_ == null) { @@ -714,6 +894,9 @@ public Builder setActiveLogs( } return this; } + /** + * repeated .ActiveLog activeLogs = 5; + */ public Builder addActiveLogs(org.apache.flume.channel.file.proto.ProtosFactory.ActiveLog value) { if (activeLogsBuilder_ == null) { if (value == null) { @@ -727,6 +910,9 @@ public Builder addActiveLogs(org.apache.flume.channel.file.proto.ProtosFactory.A } return this; } + /** + * repeated .ActiveLog activeLogs = 5; + */ public Builder addActiveLogs( int index, org.apache.flume.channel.file.proto.ProtosFactory.ActiveLog value) { if (activeLogsBuilder_ == null) { @@ -741,6 +927,9 @@ public Builder addActiveLogs( } return this; } + /** + * repeated .ActiveLog activeLogs = 5; + */ public Builder addActiveLogs( org.apache.flume.channel.file.proto.ProtosFactory.ActiveLog.Builder builderForValue) { if (activeLogsBuilder_ == null) { @@ -752,6 +941,9 @@ public Builder addActiveLogs( } return this; } + /** + * repeated .ActiveLog activeLogs = 5; + */ public Builder addActiveLogs( int index, org.apache.flume.channel.file.proto.ProtosFactory.ActiveLog.Builder builderForValue) { if (activeLogsBuilder_ == null) { @@ -763,6 +955,9 @@ public Builder addActiveLogs( } return this; } + /** + * repeated .ActiveLog activeLogs = 5; + */ public Builder addAllActiveLogs( java.lang.Iterable values) { if (activeLogsBuilder_ == null) { @@ -774,6 +969,9 @@ public Builder addAllActiveLogs( } return this; } + /** + * repeated .ActiveLog activeLogs = 5; + */ public Builder clearActiveLogs() { if (activeLogsBuilder_ == null) { activeLogs_ = java.util.Collections.emptyList(); @@ -784,6 +982,9 @@ public Builder clearActiveLogs() { } return this; } + /** + * repeated .ActiveLog activeLogs = 5; + */ public Builder removeActiveLogs(int index) { if (activeLogsBuilder_ == null) { ensureActiveLogsIsMutable(); @@ -794,10 +995,16 @@ public Builder removeActiveLogs(int index) { } return this; } + /** + * repeated .ActiveLog activeLogs = 5; + */ public org.apache.flume.channel.file.proto.ProtosFactory.ActiveLog.Builder getActiveLogsBuilder( int index) { return getActiveLogsFieldBuilder().getBuilder(index); } + /** + * repeated .ActiveLog activeLogs = 5; + */ public org.apache.flume.channel.file.proto.ProtosFactory.ActiveLogOrBuilder getActiveLogsOrBuilder( int index) { if (activeLogsBuilder_ == null) { @@ -805,6 +1012,9 @@ public org.apache.flume.channel.file.proto.ProtosFactory.ActiveLogOrBuilder getA return activeLogsBuilder_.getMessageOrBuilder(index); } } + /** + * repeated .ActiveLog activeLogs = 5; + */ public java.util.List getActiveLogsOrBuilderList() { if (activeLogsBuilder_ != null) { @@ -813,15 +1023,24 @@ public org.apache.flume.channel.file.proto.ProtosFactory.ActiveLogOrBuilder getA return java.util.Collections.unmodifiableList(activeLogs_); } } + /** + * repeated .ActiveLog activeLogs = 5; + */ public org.apache.flume.channel.file.proto.ProtosFactory.ActiveLog.Builder addActiveLogsBuilder() { return getActiveLogsFieldBuilder().addBuilder( org.apache.flume.channel.file.proto.ProtosFactory.ActiveLog.getDefaultInstance()); } + /** + * repeated .ActiveLog activeLogs = 5; + */ public org.apache.flume.channel.file.proto.ProtosFactory.ActiveLog.Builder addActiveLogsBuilder( int index) { return getActiveLogsFieldBuilder().addBuilder( index, org.apache.flume.channel.file.proto.ProtosFactory.ActiveLog.getDefaultInstance()); } + /** + * repeated .ActiveLog activeLogs = 5; + */ public java.util.List getActiveLogsBuilderList() { return getActiveLogsFieldBuilder().getBuilderList(); @@ -840,78 +1059,174 @@ public org.apache.flume.channel.file.proto.ProtosFactory.ActiveLog.Builder addAc } return activeLogsBuilder_; } - + // @@protoc_insertion_point(builder_scope:Checkpoint) } - + static { defaultInstance = new Checkpoint(true); defaultInstance.initFields(); } - + // @@protoc_insertion_point(class_scope:Checkpoint) } - + public interface ActiveLogOrBuilder extends com.google.protobuf.MessageOrBuilder { - + // required sfixed32 logFileID = 1; + /** + * required sfixed32 logFileID = 1; + */ boolean hasLogFileID(); + /** + * required sfixed32 logFileID = 1; + */ int getLogFileID(); - + // required sfixed32 count = 2; + /** + * required sfixed32 count = 2; + */ boolean hasCount(); + /** + * required sfixed32 count = 2; + */ int getCount(); } + /** + * Protobuf type {@code ActiveLog} + */ public static final class ActiveLog extends com.google.protobuf.GeneratedMessage implements ActiveLogOrBuilder { // Use ActiveLog.newBuilder() to construct. - private ActiveLog(Builder builder) { + private ActiveLog(com.google.protobuf.GeneratedMessage.Builder builder) { super(builder); + this.unknownFields = builder.getUnknownFields(); } - private ActiveLog(boolean noInit) {} - + private ActiveLog(boolean noInit) { this.unknownFields = com.google.protobuf.UnknownFieldSet.getDefaultInstance(); } + private static final ActiveLog defaultInstance; public static ActiveLog getDefaultInstance() { return defaultInstance; } - + public ActiveLog getDefaultInstanceForType() { return defaultInstance; } - + + private final com.google.protobuf.UnknownFieldSet unknownFields; + @java.lang.Override + public final com.google.protobuf.UnknownFieldSet + getUnknownFields() { + return this.unknownFields; + } + private ActiveLog( + com.google.protobuf.CodedInputStream input, + com.google.protobuf.ExtensionRegistryLite extensionRegistry) + throws com.google.protobuf.InvalidProtocolBufferException { + initFields(); + int mutable_bitField0_ = 0; + com.google.protobuf.UnknownFieldSet.Builder unknownFields = + com.google.protobuf.UnknownFieldSet.newBuilder(); + try { + boolean done = false; + while (!done) { + int tag = input.readTag(); + switch (tag) { + case 0: + done = true; + break; + default: { + if (!parseUnknownField(input, unknownFields, + extensionRegistry, tag)) { + done = true; + } + break; + } + case 13: { + bitField0_ |= 0x00000001; + logFileID_ = input.readSFixed32(); + break; + } + case 21: { + bitField0_ |= 0x00000002; + count_ = input.readSFixed32(); + break; + } + } + } + } catch (com.google.protobuf.InvalidProtocolBufferException e) { + throw e.setUnfinishedMessage(this); + } catch (java.io.IOException e) { + throw new com.google.protobuf.InvalidProtocolBufferException( + e.getMessage()).setUnfinishedMessage(this); + } finally { + this.unknownFields = unknownFields.build(); + makeExtensionsImmutable(); + } + } public static final com.google.protobuf.Descriptors.Descriptor getDescriptor() { return org.apache.flume.channel.file.proto.ProtosFactory.internal_static_ActiveLog_descriptor; } - + protected com.google.protobuf.GeneratedMessage.FieldAccessorTable internalGetFieldAccessorTable() { - return org.apache.flume.channel.file.proto.ProtosFactory.internal_static_ActiveLog_fieldAccessorTable; + return org.apache.flume.channel.file.proto.ProtosFactory.internal_static_ActiveLog_fieldAccessorTable + .ensureFieldAccessorsInitialized( + org.apache.flume.channel.file.proto.ProtosFactory.ActiveLog.class, org.apache.flume.channel.file.proto.ProtosFactory.ActiveLog.Builder.class); + } + + public static com.google.protobuf.Parser PARSER = + new com.google.protobuf.AbstractParser() { + public ActiveLog parsePartialFrom( + com.google.protobuf.CodedInputStream input, + com.google.protobuf.ExtensionRegistryLite extensionRegistry) + throws com.google.protobuf.InvalidProtocolBufferException { + return new ActiveLog(input, extensionRegistry); + } + }; + + @java.lang.Override + public com.google.protobuf.Parser getParserForType() { + return PARSER; } - + private int bitField0_; // required sfixed32 logFileID = 1; public static final int LOGFILEID_FIELD_NUMBER = 1; private int logFileID_; + /** + * required sfixed32 logFileID = 1; + */ public boolean hasLogFileID() { return ((bitField0_ & 0x00000001) == 0x00000001); } + /** + * required sfixed32 logFileID = 1; + */ public int getLogFileID() { return logFileID_; } - + // required sfixed32 count = 2; public static final int COUNT_FIELD_NUMBER = 2; private int count_; + /** + * required sfixed32 count = 2; + */ public boolean hasCount() { return ((bitField0_ & 0x00000002) == 0x00000002); } + /** + * required sfixed32 count = 2; + */ public int getCount() { return count_; } - + private void initFields() { logFileID_ = 0; count_ = 0; @@ -920,7 +1235,7 @@ private void initFields() { public final boolean isInitialized() { byte isInitialized = memoizedIsInitialized; if (isInitialized != -1) return isInitialized == 1; - + if (!hasLogFileID()) { memoizedIsInitialized = 0; return false; @@ -932,7 +1247,7 @@ public final boolean isInitialized() { memoizedIsInitialized = 1; return true; } - + public void writeTo(com.google.protobuf.CodedOutputStream output) throws java.io.IOException { getSerializedSize(); @@ -944,12 +1259,12 @@ public void writeTo(com.google.protobuf.CodedOutputStream output) } getUnknownFields().writeTo(output); } - + private int memoizedSerializedSize = -1; public int getSerializedSize() { int size = memoizedSerializedSize; if (size != -1) return size; - + size = 0; if (((bitField0_ & 0x00000001) == 0x00000001)) { size += com.google.protobuf.CodedOutputStream @@ -963,94 +1278,83 @@ public int getSerializedSize() { memoizedSerializedSize = size; return size; } - + private static final long serialVersionUID = 0L; @java.lang.Override protected java.lang.Object writeReplace() throws java.io.ObjectStreamException { return super.writeReplace(); } - + public static org.apache.flume.channel.file.proto.ProtosFactory.ActiveLog parseFrom( com.google.protobuf.ByteString data) throws com.google.protobuf.InvalidProtocolBufferException { - return newBuilder().mergeFrom(data).buildParsed(); + return PARSER.parseFrom(data); } public static org.apache.flume.channel.file.proto.ProtosFactory.ActiveLog parseFrom( com.google.protobuf.ByteString data, com.google.protobuf.ExtensionRegistryLite extensionRegistry) throws com.google.protobuf.InvalidProtocolBufferException { - return newBuilder().mergeFrom(data, extensionRegistry) - .buildParsed(); + return PARSER.parseFrom(data, extensionRegistry); } public static org.apache.flume.channel.file.proto.ProtosFactory.ActiveLog parseFrom(byte[] data) throws com.google.protobuf.InvalidProtocolBufferException { - return newBuilder().mergeFrom(data).buildParsed(); + return PARSER.parseFrom(data); } public static org.apache.flume.channel.file.proto.ProtosFactory.ActiveLog parseFrom( byte[] data, com.google.protobuf.ExtensionRegistryLite extensionRegistry) throws com.google.protobuf.InvalidProtocolBufferException { - return newBuilder().mergeFrom(data, extensionRegistry) - .buildParsed(); + return PARSER.parseFrom(data, extensionRegistry); } public static org.apache.flume.channel.file.proto.ProtosFactory.ActiveLog parseFrom(java.io.InputStream input) throws java.io.IOException { - return newBuilder().mergeFrom(input).buildParsed(); + return PARSER.parseFrom(input); } public static org.apache.flume.channel.file.proto.ProtosFactory.ActiveLog parseFrom( java.io.InputStream input, com.google.protobuf.ExtensionRegistryLite extensionRegistry) throws java.io.IOException { - return newBuilder().mergeFrom(input, extensionRegistry) - .buildParsed(); + return PARSER.parseFrom(input, extensionRegistry); } public static org.apache.flume.channel.file.proto.ProtosFactory.ActiveLog parseDelimitedFrom(java.io.InputStream input) throws java.io.IOException { - Builder builder = newBuilder(); - if (builder.mergeDelimitedFrom(input)) { - return builder.buildParsed(); - } else { - return null; - } + return PARSER.parseDelimitedFrom(input); } public static org.apache.flume.channel.file.proto.ProtosFactory.ActiveLog parseDelimitedFrom( java.io.InputStream input, com.google.protobuf.ExtensionRegistryLite extensionRegistry) throws java.io.IOException { - Builder builder = newBuilder(); - if (builder.mergeDelimitedFrom(input, extensionRegistry)) { - return builder.buildParsed(); - } else { - return null; - } + return PARSER.parseDelimitedFrom(input, extensionRegistry); } public static org.apache.flume.channel.file.proto.ProtosFactory.ActiveLog parseFrom( com.google.protobuf.CodedInputStream input) throws java.io.IOException { - return newBuilder().mergeFrom(input).buildParsed(); + return PARSER.parseFrom(input); } public static org.apache.flume.channel.file.proto.ProtosFactory.ActiveLog parseFrom( com.google.protobuf.CodedInputStream input, com.google.protobuf.ExtensionRegistryLite extensionRegistry) throws java.io.IOException { - return newBuilder().mergeFrom(input, extensionRegistry) - .buildParsed(); + return PARSER.parseFrom(input, extensionRegistry); } - + public static Builder newBuilder() { return Builder.create(); } public Builder newBuilderForType() { return newBuilder(); } public static Builder newBuilder(org.apache.flume.channel.file.proto.ProtosFactory.ActiveLog prototype) { return newBuilder().mergeFrom(prototype); } public Builder toBuilder() { return newBuilder(this); } - + @java.lang.Override protected Builder newBuilderForType( com.google.protobuf.GeneratedMessage.BuilderParent parent) { Builder builder = new Builder(parent); return builder; } + /** + * Protobuf type {@code ActiveLog} + */ public static final class Builder extends com.google.protobuf.GeneratedMessage.Builder implements org.apache.flume.channel.file.proto.ProtosFactory.ActiveLogOrBuilder { @@ -1058,18 +1362,21 @@ public static final class Builder extends getDescriptor() { return org.apache.flume.channel.file.proto.ProtosFactory.internal_static_ActiveLog_descriptor; } - + protected com.google.protobuf.GeneratedMessage.FieldAccessorTable internalGetFieldAccessorTable() { - return org.apache.flume.channel.file.proto.ProtosFactory.internal_static_ActiveLog_fieldAccessorTable; + return org.apache.flume.channel.file.proto.ProtosFactory.internal_static_ActiveLog_fieldAccessorTable + .ensureFieldAccessorsInitialized( + org.apache.flume.channel.file.proto.ProtosFactory.ActiveLog.class, org.apache.flume.channel.file.proto.ProtosFactory.ActiveLog.Builder.class); } - + // Construct using org.apache.flume.channel.file.proto.ProtosFactory.ActiveLog.newBuilder() private Builder() { maybeForceBuilderInitialization(); } - - private Builder(BuilderParent parent) { + + private Builder( + com.google.protobuf.GeneratedMessage.BuilderParent parent) { super(parent); maybeForceBuilderInitialization(); } @@ -1080,7 +1387,7 @@ private void maybeForceBuilderInitialization() { private static Builder create() { return new Builder(); } - + public Builder clear() { super.clear(); logFileID_ = 0; @@ -1089,20 +1396,20 @@ public Builder clear() { bitField0_ = (bitField0_ & ~0x00000002); return this; } - + public Builder clone() { return create().mergeFrom(buildPartial()); } - + public com.google.protobuf.Descriptors.Descriptor getDescriptorForType() { - return org.apache.flume.channel.file.proto.ProtosFactory.ActiveLog.getDescriptor(); + return org.apache.flume.channel.file.proto.ProtosFactory.internal_static_ActiveLog_descriptor; } - + public org.apache.flume.channel.file.proto.ProtosFactory.ActiveLog getDefaultInstanceForType() { return org.apache.flume.channel.file.proto.ProtosFactory.ActiveLog.getDefaultInstance(); } - + public org.apache.flume.channel.file.proto.ProtosFactory.ActiveLog build() { org.apache.flume.channel.file.proto.ProtosFactory.ActiveLog result = buildPartial(); if (!result.isInitialized()) { @@ -1110,17 +1417,7 @@ public org.apache.flume.channel.file.proto.ProtosFactory.ActiveLog build() { } return result; } - - private org.apache.flume.channel.file.proto.ProtosFactory.ActiveLog buildParsed() - throws com.google.protobuf.InvalidProtocolBufferException { - org.apache.flume.channel.file.proto.ProtosFactory.ActiveLog result = buildPartial(); - if (!result.isInitialized()) { - throw newUninitializedMessageException( - result).asInvalidProtocolBufferException(); - } - return result; - } - + public org.apache.flume.channel.file.proto.ProtosFactory.ActiveLog buildPartial() { org.apache.flume.channel.file.proto.ProtosFactory.ActiveLog result = new org.apache.flume.channel.file.proto.ProtosFactory.ActiveLog(this); int from_bitField0_ = bitField0_; @@ -1137,7 +1434,7 @@ public org.apache.flume.channel.file.proto.ProtosFactory.ActiveLog buildPartial( onBuilt(); return result; } - + public Builder mergeFrom(com.google.protobuf.Message other) { if (other instanceof org.apache.flume.channel.file.proto.ProtosFactory.ActiveLog) { return mergeFrom((org.apache.flume.channel.file.proto.ProtosFactory.ActiveLog)other); @@ -1146,7 +1443,7 @@ public Builder mergeFrom(com.google.protobuf.Message other) { return this; } } - + public Builder mergeFrom(org.apache.flume.channel.file.proto.ProtosFactory.ActiveLog other) { if (other == org.apache.flume.channel.file.proto.ProtosFactory.ActiveLog.getDefaultInstance()) return this; if (other.hasLogFileID()) { @@ -1158,7 +1455,7 @@ public Builder mergeFrom(org.apache.flume.channel.file.proto.ProtosFactory.Activ this.mergeUnknownFields(other.getUnknownFields()); return this; } - + public final boolean isInitialized() { if (!hasLogFileID()) { @@ -1170,233 +1467,432 @@ public final boolean isInitialized() { } return true; } - + public Builder mergeFrom( com.google.protobuf.CodedInputStream input, com.google.protobuf.ExtensionRegistryLite extensionRegistry) throws java.io.IOException { - com.google.protobuf.UnknownFieldSet.Builder unknownFields = - com.google.protobuf.UnknownFieldSet.newBuilder( - this.getUnknownFields()); - while (true) { - int tag = input.readTag(); - switch (tag) { - case 0: - this.setUnknownFields(unknownFields.build()); - onChanged(); - return this; - default: { - if (!parseUnknownField(input, unknownFields, - extensionRegistry, tag)) { - this.setUnknownFields(unknownFields.build()); - onChanged(); - return this; - } - break; - } - case 13: { - bitField0_ |= 0x00000001; - logFileID_ = input.readSFixed32(); - break; - } - case 21: { - bitField0_ |= 0x00000002; - count_ = input.readSFixed32(); - break; - } + org.apache.flume.channel.file.proto.ProtosFactory.ActiveLog parsedMessage = null; + try { + parsedMessage = PARSER.parsePartialFrom(input, extensionRegistry); + } catch (com.google.protobuf.InvalidProtocolBufferException e) { + parsedMessage = (org.apache.flume.channel.file.proto.ProtosFactory.ActiveLog) e.getUnfinishedMessage(); + throw e; + } finally { + if (parsedMessage != null) { + mergeFrom(parsedMessage); } } + return this; } - private int bitField0_; - + // required sfixed32 logFileID = 1; private int logFileID_ ; + /** + * required sfixed32 logFileID = 1; + */ public boolean hasLogFileID() { return ((bitField0_ & 0x00000001) == 0x00000001); } + /** + * required sfixed32 logFileID = 1; + */ public int getLogFileID() { return logFileID_; } + /** + * required sfixed32 logFileID = 1; + */ public Builder setLogFileID(int value) { bitField0_ |= 0x00000001; logFileID_ = value; onChanged(); return this; } + /** + * required sfixed32 logFileID = 1; + */ public Builder clearLogFileID() { bitField0_ = (bitField0_ & ~0x00000001); logFileID_ = 0; onChanged(); return this; } - + // required sfixed32 count = 2; private int count_ ; + /** + * required sfixed32 count = 2; + */ public boolean hasCount() { return ((bitField0_ & 0x00000002) == 0x00000002); } + /** + * required sfixed32 count = 2; + */ public int getCount() { return count_; } + /** + * required sfixed32 count = 2; + */ public Builder setCount(int value) { bitField0_ |= 0x00000002; count_ = value; onChanged(); return this; } + /** + * required sfixed32 count = 2; + */ public Builder clearCount() { bitField0_ = (bitField0_ & ~0x00000002); count_ = 0; onChanged(); return this; } - + // @@protoc_insertion_point(builder_scope:ActiveLog) } - + static { defaultInstance = new ActiveLog(true); defaultInstance.initFields(); } - + // @@protoc_insertion_point(class_scope:ActiveLog) } - + public interface LogFileMetaDataOrBuilder extends com.google.protobuf.MessageOrBuilder { - + // required sfixed32 version = 1; + /** + * required sfixed32 version = 1; + */ boolean hasVersion(); + /** + * required sfixed32 version = 1; + */ int getVersion(); - + // required sfixed32 logFileID = 2; + /** + * required sfixed32 logFileID = 2; + */ boolean hasLogFileID(); + /** + * required sfixed32 logFileID = 2; + */ int getLogFileID(); - + // required sfixed64 checkpointPosition = 3; + /** + * required sfixed64 checkpointPosition = 3; + */ boolean hasCheckpointPosition(); + /** + * required sfixed64 checkpointPosition = 3; + */ long getCheckpointPosition(); - + // required sfixed64 checkpointWriteOrderID = 4; + /** + * required sfixed64 checkpointWriteOrderID = 4; + */ boolean hasCheckpointWriteOrderID(); + /** + * required sfixed64 checkpointWriteOrderID = 4; + */ long getCheckpointWriteOrderID(); - + // optional .LogFileEncryption encryption = 5; + /** + * optional .LogFileEncryption encryption = 5; + */ boolean hasEncryption(); + /** + * optional .LogFileEncryption encryption = 5; + */ org.apache.flume.channel.file.proto.ProtosFactory.LogFileEncryption getEncryption(); + /** + * optional .LogFileEncryption encryption = 5; + */ org.apache.flume.channel.file.proto.ProtosFactory.LogFileEncryptionOrBuilder getEncryptionOrBuilder(); - + // optional sfixed64 backupCheckpointPosition = 6; + /** + * optional sfixed64 backupCheckpointPosition = 6; + */ boolean hasBackupCheckpointPosition(); + /** + * optional sfixed64 backupCheckpointPosition = 6; + */ long getBackupCheckpointPosition(); - + // optional sfixed64 backupCheckpointWriteOrderID = 7; + /** + * optional sfixed64 backupCheckpointWriteOrderID = 7; + */ boolean hasBackupCheckpointWriteOrderID(); + /** + * optional sfixed64 backupCheckpointWriteOrderID = 7; + */ long getBackupCheckpointWriteOrderID(); } + /** + * Protobuf type {@code LogFileMetaData} + */ public static final class LogFileMetaData extends com.google.protobuf.GeneratedMessage implements LogFileMetaDataOrBuilder { // Use LogFileMetaData.newBuilder() to construct. - private LogFileMetaData(Builder builder) { + private LogFileMetaData(com.google.protobuf.GeneratedMessage.Builder builder) { super(builder); + this.unknownFields = builder.getUnknownFields(); } - private LogFileMetaData(boolean noInit) {} - + private LogFileMetaData(boolean noInit) { this.unknownFields = com.google.protobuf.UnknownFieldSet.getDefaultInstance(); } + private static final LogFileMetaData defaultInstance; public static LogFileMetaData getDefaultInstance() { return defaultInstance; } - + public LogFileMetaData getDefaultInstanceForType() { return defaultInstance; } - + + private final com.google.protobuf.UnknownFieldSet unknownFields; + @java.lang.Override + public final com.google.protobuf.UnknownFieldSet + getUnknownFields() { + return this.unknownFields; + } + private LogFileMetaData( + com.google.protobuf.CodedInputStream input, + com.google.protobuf.ExtensionRegistryLite extensionRegistry) + throws com.google.protobuf.InvalidProtocolBufferException { + initFields(); + int mutable_bitField0_ = 0; + com.google.protobuf.UnknownFieldSet.Builder unknownFields = + com.google.protobuf.UnknownFieldSet.newBuilder(); + try { + boolean done = false; + while (!done) { + int tag = input.readTag(); + switch (tag) { + case 0: + done = true; + break; + default: { + if (!parseUnknownField(input, unknownFields, + extensionRegistry, tag)) { + done = true; + } + break; + } + case 13: { + bitField0_ |= 0x00000001; + version_ = input.readSFixed32(); + break; + } + case 21: { + bitField0_ |= 0x00000002; + logFileID_ = input.readSFixed32(); + break; + } + case 25: { + bitField0_ |= 0x00000004; + checkpointPosition_ = input.readSFixed64(); + break; + } + case 33: { + bitField0_ |= 0x00000008; + checkpointWriteOrderID_ = input.readSFixed64(); + break; + } + case 42: { + org.apache.flume.channel.file.proto.ProtosFactory.LogFileEncryption.Builder subBuilder = null; + if (((bitField0_ & 0x00000010) == 0x00000010)) { + subBuilder = encryption_.toBuilder(); + } + encryption_ = input.readMessage(org.apache.flume.channel.file.proto.ProtosFactory.LogFileEncryption.PARSER, extensionRegistry); + if (subBuilder != null) { + subBuilder.mergeFrom(encryption_); + encryption_ = subBuilder.buildPartial(); + } + bitField0_ |= 0x00000010; + break; + } + case 49: { + bitField0_ |= 0x00000020; + backupCheckpointPosition_ = input.readSFixed64(); + break; + } + case 57: { + bitField0_ |= 0x00000040; + backupCheckpointWriteOrderID_ = input.readSFixed64(); + break; + } + } + } + } catch (com.google.protobuf.InvalidProtocolBufferException e) { + throw e.setUnfinishedMessage(this); + } catch (java.io.IOException e) { + throw new com.google.protobuf.InvalidProtocolBufferException( + e.getMessage()).setUnfinishedMessage(this); + } finally { + this.unknownFields = unknownFields.build(); + makeExtensionsImmutable(); + } + } public static final com.google.protobuf.Descriptors.Descriptor getDescriptor() { return org.apache.flume.channel.file.proto.ProtosFactory.internal_static_LogFileMetaData_descriptor; } - + protected com.google.protobuf.GeneratedMessage.FieldAccessorTable internalGetFieldAccessorTable() { - return org.apache.flume.channel.file.proto.ProtosFactory.internal_static_LogFileMetaData_fieldAccessorTable; + return org.apache.flume.channel.file.proto.ProtosFactory.internal_static_LogFileMetaData_fieldAccessorTable + .ensureFieldAccessorsInitialized( + org.apache.flume.channel.file.proto.ProtosFactory.LogFileMetaData.class, org.apache.flume.channel.file.proto.ProtosFactory.LogFileMetaData.Builder.class); + } + + public static com.google.protobuf.Parser PARSER = + new com.google.protobuf.AbstractParser() { + public LogFileMetaData parsePartialFrom( + com.google.protobuf.CodedInputStream input, + com.google.protobuf.ExtensionRegistryLite extensionRegistry) + throws com.google.protobuf.InvalidProtocolBufferException { + return new LogFileMetaData(input, extensionRegistry); + } + }; + + @java.lang.Override + public com.google.protobuf.Parser getParserForType() { + return PARSER; } - + private int bitField0_; // required sfixed32 version = 1; public static final int VERSION_FIELD_NUMBER = 1; private int version_; + /** + * required sfixed32 version = 1; + */ public boolean hasVersion() { return ((bitField0_ & 0x00000001) == 0x00000001); } + /** + * required sfixed32 version = 1; + */ public int getVersion() { return version_; } - + // required sfixed32 logFileID = 2; public static final int LOGFILEID_FIELD_NUMBER = 2; private int logFileID_; + /** + * required sfixed32 logFileID = 2; + */ public boolean hasLogFileID() { return ((bitField0_ & 0x00000002) == 0x00000002); } + /** + * required sfixed32 logFileID = 2; + */ public int getLogFileID() { return logFileID_; } - + // required sfixed64 checkpointPosition = 3; public static final int CHECKPOINTPOSITION_FIELD_NUMBER = 3; private long checkpointPosition_; + /** + * required sfixed64 checkpointPosition = 3; + */ public boolean hasCheckpointPosition() { return ((bitField0_ & 0x00000004) == 0x00000004); } + /** + * required sfixed64 checkpointPosition = 3; + */ public long getCheckpointPosition() { return checkpointPosition_; } - + // required sfixed64 checkpointWriteOrderID = 4; public static final int CHECKPOINTWRITEORDERID_FIELD_NUMBER = 4; private long checkpointWriteOrderID_; + /** + * required sfixed64 checkpointWriteOrderID = 4; + */ public boolean hasCheckpointWriteOrderID() { return ((bitField0_ & 0x00000008) == 0x00000008); } + /** + * required sfixed64 checkpointWriteOrderID = 4; + */ public long getCheckpointWriteOrderID() { return checkpointWriteOrderID_; } - + // optional .LogFileEncryption encryption = 5; public static final int ENCRYPTION_FIELD_NUMBER = 5; private org.apache.flume.channel.file.proto.ProtosFactory.LogFileEncryption encryption_; + /** + * optional .LogFileEncryption encryption = 5; + */ public boolean hasEncryption() { return ((bitField0_ & 0x00000010) == 0x00000010); } + /** + * optional .LogFileEncryption encryption = 5; + */ public org.apache.flume.channel.file.proto.ProtosFactory.LogFileEncryption getEncryption() { return encryption_; } + /** + * optional .LogFileEncryption encryption = 5; + */ public org.apache.flume.channel.file.proto.ProtosFactory.LogFileEncryptionOrBuilder getEncryptionOrBuilder() { return encryption_; } - + // optional sfixed64 backupCheckpointPosition = 6; public static final int BACKUPCHECKPOINTPOSITION_FIELD_NUMBER = 6; private long backupCheckpointPosition_; + /** + * optional sfixed64 backupCheckpointPosition = 6; + */ public boolean hasBackupCheckpointPosition() { return ((bitField0_ & 0x00000020) == 0x00000020); } + /** + * optional sfixed64 backupCheckpointPosition = 6; + */ public long getBackupCheckpointPosition() { return backupCheckpointPosition_; } - + // optional sfixed64 backupCheckpointWriteOrderID = 7; public static final int BACKUPCHECKPOINTWRITEORDERID_FIELD_NUMBER = 7; private long backupCheckpointWriteOrderID_; + /** + * optional sfixed64 backupCheckpointWriteOrderID = 7; + */ public boolean hasBackupCheckpointWriteOrderID() { return ((bitField0_ & 0x00000040) == 0x00000040); } + /** + * optional sfixed64 backupCheckpointWriteOrderID = 7; + */ public long getBackupCheckpointWriteOrderID() { return backupCheckpointWriteOrderID_; } - + private void initFields() { version_ = 0; logFileID_ = 0; @@ -1410,7 +1906,7 @@ private void initFields() { public final boolean isInitialized() { byte isInitialized = memoizedIsInitialized; if (isInitialized != -1) return isInitialized == 1; - + if (!hasVersion()) { memoizedIsInitialized = 0; return false; @@ -1436,7 +1932,7 @@ public final boolean isInitialized() { memoizedIsInitialized = 1; return true; } - + public void writeTo(com.google.protobuf.CodedOutputStream output) throws java.io.IOException { getSerializedSize(); @@ -1463,12 +1959,12 @@ public void writeTo(com.google.protobuf.CodedOutputStream output) } getUnknownFields().writeTo(output); } - + private int memoizedSerializedSize = -1; public int getSerializedSize() { int size = memoizedSerializedSize; if (size != -1) return size; - + size = 0; if (((bitField0_ & 0x00000001) == 0x00000001)) { size += com.google.protobuf.CodedOutputStream @@ -1502,94 +1998,83 @@ public int getSerializedSize() { memoizedSerializedSize = size; return size; } - + private static final long serialVersionUID = 0L; @java.lang.Override protected java.lang.Object writeReplace() throws java.io.ObjectStreamException { return super.writeReplace(); } - + public static org.apache.flume.channel.file.proto.ProtosFactory.LogFileMetaData parseFrom( com.google.protobuf.ByteString data) throws com.google.protobuf.InvalidProtocolBufferException { - return newBuilder().mergeFrom(data).buildParsed(); + return PARSER.parseFrom(data); } public static org.apache.flume.channel.file.proto.ProtosFactory.LogFileMetaData parseFrom( com.google.protobuf.ByteString data, com.google.protobuf.ExtensionRegistryLite extensionRegistry) throws com.google.protobuf.InvalidProtocolBufferException { - return newBuilder().mergeFrom(data, extensionRegistry) - .buildParsed(); + return PARSER.parseFrom(data, extensionRegistry); } public static org.apache.flume.channel.file.proto.ProtosFactory.LogFileMetaData parseFrom(byte[] data) throws com.google.protobuf.InvalidProtocolBufferException { - return newBuilder().mergeFrom(data).buildParsed(); + return PARSER.parseFrom(data); } public static org.apache.flume.channel.file.proto.ProtosFactory.LogFileMetaData parseFrom( byte[] data, com.google.protobuf.ExtensionRegistryLite extensionRegistry) throws com.google.protobuf.InvalidProtocolBufferException { - return newBuilder().mergeFrom(data, extensionRegistry) - .buildParsed(); + return PARSER.parseFrom(data, extensionRegistry); } public static org.apache.flume.channel.file.proto.ProtosFactory.LogFileMetaData parseFrom(java.io.InputStream input) throws java.io.IOException { - return newBuilder().mergeFrom(input).buildParsed(); + return PARSER.parseFrom(input); } public static org.apache.flume.channel.file.proto.ProtosFactory.LogFileMetaData parseFrom( java.io.InputStream input, com.google.protobuf.ExtensionRegistryLite extensionRegistry) throws java.io.IOException { - return newBuilder().mergeFrom(input, extensionRegistry) - .buildParsed(); + return PARSER.parseFrom(input, extensionRegistry); } public static org.apache.flume.channel.file.proto.ProtosFactory.LogFileMetaData parseDelimitedFrom(java.io.InputStream input) throws java.io.IOException { - Builder builder = newBuilder(); - if (builder.mergeDelimitedFrom(input)) { - return builder.buildParsed(); - } else { - return null; - } + return PARSER.parseDelimitedFrom(input); } public static org.apache.flume.channel.file.proto.ProtosFactory.LogFileMetaData parseDelimitedFrom( java.io.InputStream input, com.google.protobuf.ExtensionRegistryLite extensionRegistry) throws java.io.IOException { - Builder builder = newBuilder(); - if (builder.mergeDelimitedFrom(input, extensionRegistry)) { - return builder.buildParsed(); - } else { - return null; - } + return PARSER.parseDelimitedFrom(input, extensionRegistry); } public static org.apache.flume.channel.file.proto.ProtosFactory.LogFileMetaData parseFrom( com.google.protobuf.CodedInputStream input) throws java.io.IOException { - return newBuilder().mergeFrom(input).buildParsed(); + return PARSER.parseFrom(input); } public static org.apache.flume.channel.file.proto.ProtosFactory.LogFileMetaData parseFrom( com.google.protobuf.CodedInputStream input, com.google.protobuf.ExtensionRegistryLite extensionRegistry) throws java.io.IOException { - return newBuilder().mergeFrom(input, extensionRegistry) - .buildParsed(); + return PARSER.parseFrom(input, extensionRegistry); } - + public static Builder newBuilder() { return Builder.create(); } public Builder newBuilderForType() { return newBuilder(); } public static Builder newBuilder(org.apache.flume.channel.file.proto.ProtosFactory.LogFileMetaData prototype) { return newBuilder().mergeFrom(prototype); } public Builder toBuilder() { return newBuilder(this); } - + @java.lang.Override protected Builder newBuilderForType( com.google.protobuf.GeneratedMessage.BuilderParent parent) { Builder builder = new Builder(parent); return builder; } + /** + * Protobuf type {@code LogFileMetaData} + */ public static final class Builder extends com.google.protobuf.GeneratedMessage.Builder implements org.apache.flume.channel.file.proto.ProtosFactory.LogFileMetaDataOrBuilder { @@ -1597,18 +2082,21 @@ public static final class Builder extends getDescriptor() { return org.apache.flume.channel.file.proto.ProtosFactory.internal_static_LogFileMetaData_descriptor; } - + protected com.google.protobuf.GeneratedMessage.FieldAccessorTable internalGetFieldAccessorTable() { - return org.apache.flume.channel.file.proto.ProtosFactory.internal_static_LogFileMetaData_fieldAccessorTable; + return org.apache.flume.channel.file.proto.ProtosFactory.internal_static_LogFileMetaData_fieldAccessorTable + .ensureFieldAccessorsInitialized( + org.apache.flume.channel.file.proto.ProtosFactory.LogFileMetaData.class, org.apache.flume.channel.file.proto.ProtosFactory.LogFileMetaData.Builder.class); } - + // Construct using org.apache.flume.channel.file.proto.ProtosFactory.LogFileMetaData.newBuilder() private Builder() { maybeForceBuilderInitialization(); } - - private Builder(BuilderParent parent) { + + private Builder( + com.google.protobuf.GeneratedMessage.BuilderParent parent) { super(parent); maybeForceBuilderInitialization(); } @@ -1620,7 +2108,7 @@ private void maybeForceBuilderInitialization() { private static Builder create() { return new Builder(); } - + public Builder clear() { super.clear(); version_ = 0; @@ -1643,20 +2131,20 @@ public Builder clear() { bitField0_ = (bitField0_ & ~0x00000040); return this; } - + public Builder clone() { return create().mergeFrom(buildPartial()); } - + public com.google.protobuf.Descriptors.Descriptor getDescriptorForType() { - return org.apache.flume.channel.file.proto.ProtosFactory.LogFileMetaData.getDescriptor(); + return org.apache.flume.channel.file.proto.ProtosFactory.internal_static_LogFileMetaData_descriptor; } - + public org.apache.flume.channel.file.proto.ProtosFactory.LogFileMetaData getDefaultInstanceForType() { return org.apache.flume.channel.file.proto.ProtosFactory.LogFileMetaData.getDefaultInstance(); } - + public org.apache.flume.channel.file.proto.ProtosFactory.LogFileMetaData build() { org.apache.flume.channel.file.proto.ProtosFactory.LogFileMetaData result = buildPartial(); if (!result.isInitialized()) { @@ -1664,17 +2152,7 @@ public org.apache.flume.channel.file.proto.ProtosFactory.LogFileMetaData build() } return result; } - - private org.apache.flume.channel.file.proto.ProtosFactory.LogFileMetaData buildParsed() - throws com.google.protobuf.InvalidProtocolBufferException { - org.apache.flume.channel.file.proto.ProtosFactory.LogFileMetaData result = buildPartial(); - if (!result.isInitialized()) { - throw newUninitializedMessageException( - result).asInvalidProtocolBufferException(); - } - return result; - } - + public org.apache.flume.channel.file.proto.ProtosFactory.LogFileMetaData buildPartial() { org.apache.flume.channel.file.proto.ProtosFactory.LogFileMetaData result = new org.apache.flume.channel.file.proto.ProtosFactory.LogFileMetaData(this); int from_bitField0_ = bitField0_; @@ -1715,7 +2193,7 @@ public org.apache.flume.channel.file.proto.ProtosFactory.LogFileMetaData buildPa onBuilt(); return result; } - + public Builder mergeFrom(com.google.protobuf.Message other) { if (other instanceof org.apache.flume.channel.file.proto.ProtosFactory.LogFileMetaData) { return mergeFrom((org.apache.flume.channel.file.proto.ProtosFactory.LogFileMetaData)other); @@ -1724,7 +2202,7 @@ public Builder mergeFrom(com.google.protobuf.Message other) { return this; } } - + public Builder mergeFrom(org.apache.flume.channel.file.proto.ProtosFactory.LogFileMetaData other) { if (other == org.apache.flume.channel.file.proto.ProtosFactory.LogFileMetaData.getDefaultInstance()) return this; if (other.hasVersion()) { @@ -1751,7 +2229,7 @@ public Builder mergeFrom(org.apache.flume.channel.file.proto.ProtosFactory.LogFi this.mergeUnknownFields(other.getUnknownFields()); return this; } - + public final boolean isInitialized() { if (!hasVersion()) { @@ -1777,166 +2255,171 @@ public final boolean isInitialized() { } return true; } - + public Builder mergeFrom( com.google.protobuf.CodedInputStream input, com.google.protobuf.ExtensionRegistryLite extensionRegistry) throws java.io.IOException { - com.google.protobuf.UnknownFieldSet.Builder unknownFields = - com.google.protobuf.UnknownFieldSet.newBuilder( - this.getUnknownFields()); - while (true) { - int tag = input.readTag(); - switch (tag) { - case 0: - this.setUnknownFields(unknownFields.build()); - onChanged(); - return this; - default: { - if (!parseUnknownField(input, unknownFields, - extensionRegistry, tag)) { - this.setUnknownFields(unknownFields.build()); - onChanged(); - return this; - } - break; - } - case 13: { - bitField0_ |= 0x00000001; - version_ = input.readSFixed32(); - break; - } - case 21: { - bitField0_ |= 0x00000002; - logFileID_ = input.readSFixed32(); - break; - } - case 25: { - bitField0_ |= 0x00000004; - checkpointPosition_ = input.readSFixed64(); - break; - } - case 33: { - bitField0_ |= 0x00000008; - checkpointWriteOrderID_ = input.readSFixed64(); - break; - } - case 42: { - org.apache.flume.channel.file.proto.ProtosFactory.LogFileEncryption.Builder subBuilder = org.apache.flume.channel.file.proto.ProtosFactory.LogFileEncryption.newBuilder(); - if (hasEncryption()) { - subBuilder.mergeFrom(getEncryption()); - } - input.readMessage(subBuilder, extensionRegistry); - setEncryption(subBuilder.buildPartial()); - break; - } - case 49: { - bitField0_ |= 0x00000020; - backupCheckpointPosition_ = input.readSFixed64(); - break; - } - case 57: { - bitField0_ |= 0x00000040; - backupCheckpointWriteOrderID_ = input.readSFixed64(); - break; - } + org.apache.flume.channel.file.proto.ProtosFactory.LogFileMetaData parsedMessage = null; + try { + parsedMessage = PARSER.parsePartialFrom(input, extensionRegistry); + } catch (com.google.protobuf.InvalidProtocolBufferException e) { + parsedMessage = (org.apache.flume.channel.file.proto.ProtosFactory.LogFileMetaData) e.getUnfinishedMessage(); + throw e; + } finally { + if (parsedMessage != null) { + mergeFrom(parsedMessage); } } + return this; } - private int bitField0_; - + // required sfixed32 version = 1; private int version_ ; + /** + * required sfixed32 version = 1; + */ public boolean hasVersion() { return ((bitField0_ & 0x00000001) == 0x00000001); } + /** + * required sfixed32 version = 1; + */ public int getVersion() { return version_; } + /** + * required sfixed32 version = 1; + */ public Builder setVersion(int value) { bitField0_ |= 0x00000001; version_ = value; onChanged(); return this; } + /** + * required sfixed32 version = 1; + */ public Builder clearVersion() { bitField0_ = (bitField0_ & ~0x00000001); version_ = 0; onChanged(); return this; } - + // required sfixed32 logFileID = 2; private int logFileID_ ; + /** + * required sfixed32 logFileID = 2; + */ public boolean hasLogFileID() { return ((bitField0_ & 0x00000002) == 0x00000002); } + /** + * required sfixed32 logFileID = 2; + */ public int getLogFileID() { return logFileID_; } + /** + * required sfixed32 logFileID = 2; + */ public Builder setLogFileID(int value) { bitField0_ |= 0x00000002; logFileID_ = value; onChanged(); return this; } + /** + * required sfixed32 logFileID = 2; + */ public Builder clearLogFileID() { bitField0_ = (bitField0_ & ~0x00000002); logFileID_ = 0; onChanged(); return this; } - + // required sfixed64 checkpointPosition = 3; private long checkpointPosition_ ; + /** + * required sfixed64 checkpointPosition = 3; + */ public boolean hasCheckpointPosition() { return ((bitField0_ & 0x00000004) == 0x00000004); } + /** + * required sfixed64 checkpointPosition = 3; + */ public long getCheckpointPosition() { return checkpointPosition_; } + /** + * required sfixed64 checkpointPosition = 3; + */ public Builder setCheckpointPosition(long value) { bitField0_ |= 0x00000004; checkpointPosition_ = value; onChanged(); return this; } + /** + * required sfixed64 checkpointPosition = 3; + */ public Builder clearCheckpointPosition() { bitField0_ = (bitField0_ & ~0x00000004); checkpointPosition_ = 0L; onChanged(); return this; } - + // required sfixed64 checkpointWriteOrderID = 4; private long checkpointWriteOrderID_ ; + /** + * required sfixed64 checkpointWriteOrderID = 4; + */ public boolean hasCheckpointWriteOrderID() { return ((bitField0_ & 0x00000008) == 0x00000008); } + /** + * required sfixed64 checkpointWriteOrderID = 4; + */ public long getCheckpointWriteOrderID() { return checkpointWriteOrderID_; } + /** + * required sfixed64 checkpointWriteOrderID = 4; + */ public Builder setCheckpointWriteOrderID(long value) { bitField0_ |= 0x00000008; checkpointWriteOrderID_ = value; onChanged(); return this; } + /** + * required sfixed64 checkpointWriteOrderID = 4; + */ public Builder clearCheckpointWriteOrderID() { bitField0_ = (bitField0_ & ~0x00000008); checkpointWriteOrderID_ = 0L; onChanged(); return this; } - + // optional .LogFileEncryption encryption = 5; private org.apache.flume.channel.file.proto.ProtosFactory.LogFileEncryption encryption_ = org.apache.flume.channel.file.proto.ProtosFactory.LogFileEncryption.getDefaultInstance(); private com.google.protobuf.SingleFieldBuilder< org.apache.flume.channel.file.proto.ProtosFactory.LogFileEncryption, org.apache.flume.channel.file.proto.ProtosFactory.LogFileEncryption.Builder, org.apache.flume.channel.file.proto.ProtosFactory.LogFileEncryptionOrBuilder> encryptionBuilder_; + /** + * optional .LogFileEncryption encryption = 5; + */ public boolean hasEncryption() { return ((bitField0_ & 0x00000010) == 0x00000010); } + /** + * optional .LogFileEncryption encryption = 5; + */ public org.apache.flume.channel.file.proto.ProtosFactory.LogFileEncryption getEncryption() { if (encryptionBuilder_ == null) { return encryption_; @@ -1944,6 +2427,9 @@ public org.apache.flume.channel.file.proto.ProtosFactory.LogFileEncryption getEn return encryptionBuilder_.getMessage(); } } + /** + * optional .LogFileEncryption encryption = 5; + */ public Builder setEncryption(org.apache.flume.channel.file.proto.ProtosFactory.LogFileEncryption value) { if (encryptionBuilder_ == null) { if (value == null) { @@ -1957,6 +2443,9 @@ public Builder setEncryption(org.apache.flume.channel.file.proto.ProtosFactory.L bitField0_ |= 0x00000010; return this; } + /** + * optional .LogFileEncryption encryption = 5; + */ public Builder setEncryption( org.apache.flume.channel.file.proto.ProtosFactory.LogFileEncryption.Builder builderForValue) { if (encryptionBuilder_ == null) { @@ -1968,6 +2457,9 @@ public Builder setEncryption( bitField0_ |= 0x00000010; return this; } + /** + * optional .LogFileEncryption encryption = 5; + */ public Builder mergeEncryption(org.apache.flume.channel.file.proto.ProtosFactory.LogFileEncryption value) { if (encryptionBuilder_ == null) { if (((bitField0_ & 0x00000010) == 0x00000010) && @@ -1984,6 +2476,9 @@ public Builder mergeEncryption(org.apache.flume.channel.file.proto.ProtosFactory bitField0_ |= 0x00000010; return this; } + /** + * optional .LogFileEncryption encryption = 5; + */ public Builder clearEncryption() { if (encryptionBuilder_ == null) { encryption_ = org.apache.flume.channel.file.proto.ProtosFactory.LogFileEncryption.getDefaultInstance(); @@ -1994,11 +2489,17 @@ public Builder clearEncryption() { bitField0_ = (bitField0_ & ~0x00000010); return this; } + /** + * optional .LogFileEncryption encryption = 5; + */ public org.apache.flume.channel.file.proto.ProtosFactory.LogFileEncryption.Builder getEncryptionBuilder() { bitField0_ |= 0x00000010; onChanged(); return getEncryptionFieldBuilder().getBuilder(); } + /** + * optional .LogFileEncryption encryption = 5; + */ public org.apache.flume.channel.file.proto.ProtosFactory.LogFileEncryptionOrBuilder getEncryptionOrBuilder() { if (encryptionBuilder_ != null) { return encryptionBuilder_.getMessageOrBuilder(); @@ -2006,6 +2507,9 @@ public org.apache.flume.channel.file.proto.ProtosFactory.LogFileEncryptionOrBuil return encryption_; } } + /** + * optional .LogFileEncryption encryption = 5; + */ private com.google.protobuf.SingleFieldBuilder< org.apache.flume.channel.file.proto.ProtosFactory.LogFileEncryption, org.apache.flume.channel.file.proto.ProtosFactory.LogFileEncryption.Builder, org.apache.flume.channel.file.proto.ProtosFactory.LogFileEncryptionOrBuilder> getEncryptionFieldBuilder() { @@ -2019,178 +2523,335 @@ public org.apache.flume.channel.file.proto.ProtosFactory.LogFileEncryptionOrBuil } return encryptionBuilder_; } - + // optional sfixed64 backupCheckpointPosition = 6; private long backupCheckpointPosition_ ; + /** + * optional sfixed64 backupCheckpointPosition = 6; + */ public boolean hasBackupCheckpointPosition() { return ((bitField0_ & 0x00000020) == 0x00000020); } + /** + * optional sfixed64 backupCheckpointPosition = 6; + */ public long getBackupCheckpointPosition() { return backupCheckpointPosition_; } + /** + * optional sfixed64 backupCheckpointPosition = 6; + */ public Builder setBackupCheckpointPosition(long value) { bitField0_ |= 0x00000020; backupCheckpointPosition_ = value; onChanged(); return this; } + /** + * optional sfixed64 backupCheckpointPosition = 6; + */ public Builder clearBackupCheckpointPosition() { bitField0_ = (bitField0_ & ~0x00000020); backupCheckpointPosition_ = 0L; onChanged(); return this; } - + // optional sfixed64 backupCheckpointWriteOrderID = 7; private long backupCheckpointWriteOrderID_ ; + /** + * optional sfixed64 backupCheckpointWriteOrderID = 7; + */ public boolean hasBackupCheckpointWriteOrderID() { return ((bitField0_ & 0x00000040) == 0x00000040); } + /** + * optional sfixed64 backupCheckpointWriteOrderID = 7; + */ public long getBackupCheckpointWriteOrderID() { return backupCheckpointWriteOrderID_; } + /** + * optional sfixed64 backupCheckpointWriteOrderID = 7; + */ public Builder setBackupCheckpointWriteOrderID(long value) { bitField0_ |= 0x00000040; backupCheckpointWriteOrderID_ = value; onChanged(); return this; } + /** + * optional sfixed64 backupCheckpointWriteOrderID = 7; + */ public Builder clearBackupCheckpointWriteOrderID() { bitField0_ = (bitField0_ & ~0x00000040); backupCheckpointWriteOrderID_ = 0L; onChanged(); return this; } - + // @@protoc_insertion_point(builder_scope:LogFileMetaData) } - + static { defaultInstance = new LogFileMetaData(true); defaultInstance.initFields(); } - + // @@protoc_insertion_point(class_scope:LogFileMetaData) } - + public interface LogFileEncryptionOrBuilder extends com.google.protobuf.MessageOrBuilder { - + // required string cipherProvider = 1; + /** + * required string cipherProvider = 1; + */ boolean hasCipherProvider(); - String getCipherProvider(); - + /** + * required string cipherProvider = 1; + */ + java.lang.String getCipherProvider(); + /** + * required string cipherProvider = 1; + */ + com.google.protobuf.ByteString + getCipherProviderBytes(); + // required string keyAlias = 2; + /** + * required string keyAlias = 2; + */ boolean hasKeyAlias(); - String getKeyAlias(); - + /** + * required string keyAlias = 2; + */ + java.lang.String getKeyAlias(); + /** + * required string keyAlias = 2; + */ + com.google.protobuf.ByteString + getKeyAliasBytes(); + // optional bytes parameters = 3; + /** + * optional bytes parameters = 3; + */ boolean hasParameters(); + /** + * optional bytes parameters = 3; + */ com.google.protobuf.ByteString getParameters(); } + /** + * Protobuf type {@code LogFileEncryption} + */ public static final class LogFileEncryption extends com.google.protobuf.GeneratedMessage implements LogFileEncryptionOrBuilder { // Use LogFileEncryption.newBuilder() to construct. - private LogFileEncryption(Builder builder) { + private LogFileEncryption(com.google.protobuf.GeneratedMessage.Builder builder) { super(builder); + this.unknownFields = builder.getUnknownFields(); } - private LogFileEncryption(boolean noInit) {} - + private LogFileEncryption(boolean noInit) { this.unknownFields = com.google.protobuf.UnknownFieldSet.getDefaultInstance(); } + private static final LogFileEncryption defaultInstance; public static LogFileEncryption getDefaultInstance() { return defaultInstance; } - + public LogFileEncryption getDefaultInstanceForType() { return defaultInstance; } - + + private final com.google.protobuf.UnknownFieldSet unknownFields; + @java.lang.Override + public final com.google.protobuf.UnknownFieldSet + getUnknownFields() { + return this.unknownFields; + } + private LogFileEncryption( + com.google.protobuf.CodedInputStream input, + com.google.protobuf.ExtensionRegistryLite extensionRegistry) + throws com.google.protobuf.InvalidProtocolBufferException { + initFields(); + int mutable_bitField0_ = 0; + com.google.protobuf.UnknownFieldSet.Builder unknownFields = + com.google.protobuf.UnknownFieldSet.newBuilder(); + try { + boolean done = false; + while (!done) { + int tag = input.readTag(); + switch (tag) { + case 0: + done = true; + break; + default: { + if (!parseUnknownField(input, unknownFields, + extensionRegistry, tag)) { + done = true; + } + break; + } + case 10: { + bitField0_ |= 0x00000001; + cipherProvider_ = input.readBytes(); + break; + } + case 18: { + bitField0_ |= 0x00000002; + keyAlias_ = input.readBytes(); + break; + } + case 26: { + bitField0_ |= 0x00000004; + parameters_ = input.readBytes(); + break; + } + } + } + } catch (com.google.protobuf.InvalidProtocolBufferException e) { + throw e.setUnfinishedMessage(this); + } catch (java.io.IOException e) { + throw new com.google.protobuf.InvalidProtocolBufferException( + e.getMessage()).setUnfinishedMessage(this); + } finally { + this.unknownFields = unknownFields.build(); + makeExtensionsImmutable(); + } + } public static final com.google.protobuf.Descriptors.Descriptor getDescriptor() { return org.apache.flume.channel.file.proto.ProtosFactory.internal_static_LogFileEncryption_descriptor; } - + protected com.google.protobuf.GeneratedMessage.FieldAccessorTable internalGetFieldAccessorTable() { - return org.apache.flume.channel.file.proto.ProtosFactory.internal_static_LogFileEncryption_fieldAccessorTable; + return org.apache.flume.channel.file.proto.ProtosFactory.internal_static_LogFileEncryption_fieldAccessorTable + .ensureFieldAccessorsInitialized( + org.apache.flume.channel.file.proto.ProtosFactory.LogFileEncryption.class, org.apache.flume.channel.file.proto.ProtosFactory.LogFileEncryption.Builder.class); + } + + public static com.google.protobuf.Parser PARSER = + new com.google.protobuf.AbstractParser() { + public LogFileEncryption parsePartialFrom( + com.google.protobuf.CodedInputStream input, + com.google.protobuf.ExtensionRegistryLite extensionRegistry) + throws com.google.protobuf.InvalidProtocolBufferException { + return new LogFileEncryption(input, extensionRegistry); + } + }; + + @java.lang.Override + public com.google.protobuf.Parser getParserForType() { + return PARSER; } - + private int bitField0_; // required string cipherProvider = 1; public static final int CIPHERPROVIDER_FIELD_NUMBER = 1; private java.lang.Object cipherProvider_; + /** + * required string cipherProvider = 1; + */ public boolean hasCipherProvider() { return ((bitField0_ & 0x00000001) == 0x00000001); } - public String getCipherProvider() { + /** + * required string cipherProvider = 1; + */ + public java.lang.String getCipherProvider() { java.lang.Object ref = cipherProvider_; - if (ref instanceof String) { - return (String) ref; + if (ref instanceof java.lang.String) { + return (java.lang.String) ref; } else { com.google.protobuf.ByteString bs = (com.google.protobuf.ByteString) ref; - String s = bs.toStringUtf8(); - if (com.google.protobuf.Internal.isValidUtf8(bs)) { + java.lang.String s = bs.toStringUtf8(); + if (bs.isValidUtf8()) { cipherProvider_ = s; } return s; } } - private com.google.protobuf.ByteString getCipherProviderBytes() { + /** + * required string cipherProvider = 1; + */ + public com.google.protobuf.ByteString + getCipherProviderBytes() { java.lang.Object ref = cipherProvider_; - if (ref instanceof String) { + if (ref instanceof java.lang.String) { com.google.protobuf.ByteString b = - com.google.protobuf.ByteString.copyFromUtf8((String) ref); + com.google.protobuf.ByteString.copyFromUtf8( + (java.lang.String) ref); cipherProvider_ = b; return b; } else { return (com.google.protobuf.ByteString) ref; } } - + // required string keyAlias = 2; public static final int KEYALIAS_FIELD_NUMBER = 2; private java.lang.Object keyAlias_; + /** + * required string keyAlias = 2; + */ public boolean hasKeyAlias() { return ((bitField0_ & 0x00000002) == 0x00000002); } - public String getKeyAlias() { + /** + * required string keyAlias = 2; + */ + public java.lang.String getKeyAlias() { java.lang.Object ref = keyAlias_; - if (ref instanceof String) { - return (String) ref; + if (ref instanceof java.lang.String) { + return (java.lang.String) ref; } else { com.google.protobuf.ByteString bs = (com.google.protobuf.ByteString) ref; - String s = bs.toStringUtf8(); - if (com.google.protobuf.Internal.isValidUtf8(bs)) { + java.lang.String s = bs.toStringUtf8(); + if (bs.isValidUtf8()) { keyAlias_ = s; } return s; } } - private com.google.protobuf.ByteString getKeyAliasBytes() { + /** + * required string keyAlias = 2; + */ + public com.google.protobuf.ByteString + getKeyAliasBytes() { java.lang.Object ref = keyAlias_; - if (ref instanceof String) { + if (ref instanceof java.lang.String) { com.google.protobuf.ByteString b = - com.google.protobuf.ByteString.copyFromUtf8((String) ref); + com.google.protobuf.ByteString.copyFromUtf8( + (java.lang.String) ref); keyAlias_ = b; return b; } else { return (com.google.protobuf.ByteString) ref; } } - + // optional bytes parameters = 3; public static final int PARAMETERS_FIELD_NUMBER = 3; private com.google.protobuf.ByteString parameters_; + /** + * optional bytes parameters = 3; + */ public boolean hasParameters() { return ((bitField0_ & 0x00000004) == 0x00000004); } + /** + * optional bytes parameters = 3; + */ public com.google.protobuf.ByteString getParameters() { return parameters_; } - + private void initFields() { cipherProvider_ = ""; keyAlias_ = ""; @@ -2200,7 +2861,7 @@ private void initFields() { public final boolean isInitialized() { byte isInitialized = memoizedIsInitialized; if (isInitialized != -1) return isInitialized == 1; - + if (!hasCipherProvider()) { memoizedIsInitialized = 0; return false; @@ -2212,7 +2873,7 @@ public final boolean isInitialized() { memoizedIsInitialized = 1; return true; } - + public void writeTo(com.google.protobuf.CodedOutputStream output) throws java.io.IOException { getSerializedSize(); @@ -2227,12 +2888,12 @@ public void writeTo(com.google.protobuf.CodedOutputStream output) } getUnknownFields().writeTo(output); } - + private int memoizedSerializedSize = -1; public int getSerializedSize() { int size = memoizedSerializedSize; if (size != -1) return size; - + size = 0; if (((bitField0_ & 0x00000001) == 0x00000001)) { size += com.google.protobuf.CodedOutputStream @@ -2250,94 +2911,83 @@ public int getSerializedSize() { memoizedSerializedSize = size; return size; } - + private static final long serialVersionUID = 0L; @java.lang.Override protected java.lang.Object writeReplace() throws java.io.ObjectStreamException { return super.writeReplace(); } - + public static org.apache.flume.channel.file.proto.ProtosFactory.LogFileEncryption parseFrom( com.google.protobuf.ByteString data) throws com.google.protobuf.InvalidProtocolBufferException { - return newBuilder().mergeFrom(data).buildParsed(); + return PARSER.parseFrom(data); } public static org.apache.flume.channel.file.proto.ProtosFactory.LogFileEncryption parseFrom( com.google.protobuf.ByteString data, com.google.protobuf.ExtensionRegistryLite extensionRegistry) throws com.google.protobuf.InvalidProtocolBufferException { - return newBuilder().mergeFrom(data, extensionRegistry) - .buildParsed(); + return PARSER.parseFrom(data, extensionRegistry); } public static org.apache.flume.channel.file.proto.ProtosFactory.LogFileEncryption parseFrom(byte[] data) throws com.google.protobuf.InvalidProtocolBufferException { - return newBuilder().mergeFrom(data).buildParsed(); + return PARSER.parseFrom(data); } public static org.apache.flume.channel.file.proto.ProtosFactory.LogFileEncryption parseFrom( byte[] data, com.google.protobuf.ExtensionRegistryLite extensionRegistry) throws com.google.protobuf.InvalidProtocolBufferException { - return newBuilder().mergeFrom(data, extensionRegistry) - .buildParsed(); + return PARSER.parseFrom(data, extensionRegistry); } public static org.apache.flume.channel.file.proto.ProtosFactory.LogFileEncryption parseFrom(java.io.InputStream input) throws java.io.IOException { - return newBuilder().mergeFrom(input).buildParsed(); + return PARSER.parseFrom(input); } public static org.apache.flume.channel.file.proto.ProtosFactory.LogFileEncryption parseFrom( java.io.InputStream input, com.google.protobuf.ExtensionRegistryLite extensionRegistry) throws java.io.IOException { - return newBuilder().mergeFrom(input, extensionRegistry) - .buildParsed(); + return PARSER.parseFrom(input, extensionRegistry); } public static org.apache.flume.channel.file.proto.ProtosFactory.LogFileEncryption parseDelimitedFrom(java.io.InputStream input) throws java.io.IOException { - Builder builder = newBuilder(); - if (builder.mergeDelimitedFrom(input)) { - return builder.buildParsed(); - } else { - return null; - } + return PARSER.parseDelimitedFrom(input); } public static org.apache.flume.channel.file.proto.ProtosFactory.LogFileEncryption parseDelimitedFrom( java.io.InputStream input, com.google.protobuf.ExtensionRegistryLite extensionRegistry) throws java.io.IOException { - Builder builder = newBuilder(); - if (builder.mergeDelimitedFrom(input, extensionRegistry)) { - return builder.buildParsed(); - } else { - return null; - } + return PARSER.parseDelimitedFrom(input, extensionRegistry); } public static org.apache.flume.channel.file.proto.ProtosFactory.LogFileEncryption parseFrom( com.google.protobuf.CodedInputStream input) throws java.io.IOException { - return newBuilder().mergeFrom(input).buildParsed(); + return PARSER.parseFrom(input); } public static org.apache.flume.channel.file.proto.ProtosFactory.LogFileEncryption parseFrom( com.google.protobuf.CodedInputStream input, com.google.protobuf.ExtensionRegistryLite extensionRegistry) throws java.io.IOException { - return newBuilder().mergeFrom(input, extensionRegistry) - .buildParsed(); + return PARSER.parseFrom(input, extensionRegistry); } - + public static Builder newBuilder() { return Builder.create(); } public Builder newBuilderForType() { return newBuilder(); } public static Builder newBuilder(org.apache.flume.channel.file.proto.ProtosFactory.LogFileEncryption prototype) { return newBuilder().mergeFrom(prototype); } public Builder toBuilder() { return newBuilder(this); } - + @java.lang.Override protected Builder newBuilderForType( com.google.protobuf.GeneratedMessage.BuilderParent parent) { Builder builder = new Builder(parent); return builder; } + /** + * Protobuf type {@code LogFileEncryption} + */ public static final class Builder extends com.google.protobuf.GeneratedMessage.Builder implements org.apache.flume.channel.file.proto.ProtosFactory.LogFileEncryptionOrBuilder { @@ -2345,18 +2995,21 @@ public static final class Builder extends getDescriptor() { return org.apache.flume.channel.file.proto.ProtosFactory.internal_static_LogFileEncryption_descriptor; } - + protected com.google.protobuf.GeneratedMessage.FieldAccessorTable internalGetFieldAccessorTable() { - return org.apache.flume.channel.file.proto.ProtosFactory.internal_static_LogFileEncryption_fieldAccessorTable; + return org.apache.flume.channel.file.proto.ProtosFactory.internal_static_LogFileEncryption_fieldAccessorTable + .ensureFieldAccessorsInitialized( + org.apache.flume.channel.file.proto.ProtosFactory.LogFileEncryption.class, org.apache.flume.channel.file.proto.ProtosFactory.LogFileEncryption.Builder.class); } - + // Construct using org.apache.flume.channel.file.proto.ProtosFactory.LogFileEncryption.newBuilder() private Builder() { maybeForceBuilderInitialization(); } - - private Builder(BuilderParent parent) { + + private Builder( + com.google.protobuf.GeneratedMessage.BuilderParent parent) { super(parent); maybeForceBuilderInitialization(); } @@ -2367,7 +3020,7 @@ private void maybeForceBuilderInitialization() { private static Builder create() { return new Builder(); } - + public Builder clear() { super.clear(); cipherProvider_ = ""; @@ -2378,20 +3031,20 @@ public Builder clear() { bitField0_ = (bitField0_ & ~0x00000004); return this; } - + public Builder clone() { return create().mergeFrom(buildPartial()); } - + public com.google.protobuf.Descriptors.Descriptor getDescriptorForType() { - return org.apache.flume.channel.file.proto.ProtosFactory.LogFileEncryption.getDescriptor(); + return org.apache.flume.channel.file.proto.ProtosFactory.internal_static_LogFileEncryption_descriptor; } - + public org.apache.flume.channel.file.proto.ProtosFactory.LogFileEncryption getDefaultInstanceForType() { return org.apache.flume.channel.file.proto.ProtosFactory.LogFileEncryption.getDefaultInstance(); } - + public org.apache.flume.channel.file.proto.ProtosFactory.LogFileEncryption build() { org.apache.flume.channel.file.proto.ProtosFactory.LogFileEncryption result = buildPartial(); if (!result.isInitialized()) { @@ -2399,17 +3052,7 @@ public org.apache.flume.channel.file.proto.ProtosFactory.LogFileEncryption build } return result; } - - private org.apache.flume.channel.file.proto.ProtosFactory.LogFileEncryption buildParsed() - throws com.google.protobuf.InvalidProtocolBufferException { - org.apache.flume.channel.file.proto.ProtosFactory.LogFileEncryption result = buildPartial(); - if (!result.isInitialized()) { - throw newUninitializedMessageException( - result).asInvalidProtocolBufferException(); - } - return result; - } - + public org.apache.flume.channel.file.proto.ProtosFactory.LogFileEncryption buildPartial() { org.apache.flume.channel.file.proto.ProtosFactory.LogFileEncryption result = new org.apache.flume.channel.file.proto.ProtosFactory.LogFileEncryption(this); int from_bitField0_ = bitField0_; @@ -2430,7 +3073,7 @@ public org.apache.flume.channel.file.proto.ProtosFactory.LogFileEncryption build onBuilt(); return result; } - + public Builder mergeFrom(com.google.protobuf.Message other) { if (other instanceof org.apache.flume.channel.file.proto.ProtosFactory.LogFileEncryption) { return mergeFrom((org.apache.flume.channel.file.proto.ProtosFactory.LogFileEncryption)other); @@ -2439,14 +3082,18 @@ public Builder mergeFrom(com.google.protobuf.Message other) { return this; } } - + public Builder mergeFrom(org.apache.flume.channel.file.proto.ProtosFactory.LogFileEncryption other) { if (other == org.apache.flume.channel.file.proto.ProtosFactory.LogFileEncryption.getDefaultInstance()) return this; if (other.hasCipherProvider()) { - setCipherProvider(other.getCipherProvider()); + bitField0_ |= 0x00000001; + cipherProvider_ = other.cipherProvider_; + onChanged(); } if (other.hasKeyAlias()) { - setKeyAlias(other.getKeyAlias()); + bitField0_ |= 0x00000002; + keyAlias_ = other.keyAlias_; + onChanged(); } if (other.hasParameters()) { setParameters(other.getParameters()); @@ -2454,7 +3101,7 @@ public Builder mergeFrom(org.apache.flume.channel.file.proto.ProtosFactory.LogFi this.mergeUnknownFields(other.getUnknownFields()); return this; } - + public final boolean isInitialized() { if (!hasCipherProvider()) { @@ -2466,67 +3113,69 @@ public final boolean isInitialized() { } return true; } - + public Builder mergeFrom( com.google.protobuf.CodedInputStream input, com.google.protobuf.ExtensionRegistryLite extensionRegistry) throws java.io.IOException { - com.google.protobuf.UnknownFieldSet.Builder unknownFields = - com.google.protobuf.UnknownFieldSet.newBuilder( - this.getUnknownFields()); - while (true) { - int tag = input.readTag(); - switch (tag) { - case 0: - this.setUnknownFields(unknownFields.build()); - onChanged(); - return this; - default: { - if (!parseUnknownField(input, unknownFields, - extensionRegistry, tag)) { - this.setUnknownFields(unknownFields.build()); - onChanged(); - return this; - } - break; - } - case 10: { - bitField0_ |= 0x00000001; - cipherProvider_ = input.readBytes(); - break; - } - case 18: { - bitField0_ |= 0x00000002; - keyAlias_ = input.readBytes(); - break; - } - case 26: { - bitField0_ |= 0x00000004; - parameters_ = input.readBytes(); - break; - } + org.apache.flume.channel.file.proto.ProtosFactory.LogFileEncryption parsedMessage = null; + try { + parsedMessage = PARSER.parsePartialFrom(input, extensionRegistry); + } catch (com.google.protobuf.InvalidProtocolBufferException e) { + parsedMessage = (org.apache.flume.channel.file.proto.ProtosFactory.LogFileEncryption) e.getUnfinishedMessage(); + throw e; + } finally { + if (parsedMessage != null) { + mergeFrom(parsedMessage); } } + return this; } - private int bitField0_; - + // required string cipherProvider = 1; private java.lang.Object cipherProvider_ = ""; + /** + * required string cipherProvider = 1; + */ public boolean hasCipherProvider() { return ((bitField0_ & 0x00000001) == 0x00000001); } - public String getCipherProvider() { + /** + * required string cipherProvider = 1; + */ + public java.lang.String getCipherProvider() { java.lang.Object ref = cipherProvider_; - if (!(ref instanceof String)) { - String s = ((com.google.protobuf.ByteString) ref).toStringUtf8(); + if (!(ref instanceof java.lang.String)) { + java.lang.String s = ((com.google.protobuf.ByteString) ref) + .toStringUtf8(); cipherProvider_ = s; return s; } else { - return (String) ref; + return (java.lang.String) ref; + } + } + /** + * required string cipherProvider = 1; + */ + public com.google.protobuf.ByteString + getCipherProviderBytes() { + java.lang.Object ref = cipherProvider_; + if (ref instanceof String) { + com.google.protobuf.ByteString b = + com.google.protobuf.ByteString.copyFromUtf8( + (java.lang.String) ref); + cipherProvider_ = b; + return b; + } else { + return (com.google.protobuf.ByteString) ref; } } - public Builder setCipherProvider(String value) { + /** + * required string cipherProvider = 1; + */ + public Builder setCipherProvider( + java.lang.String value) { if (value == null) { throw new NullPointerException(); } @@ -2535,34 +3184,72 @@ public Builder setCipherProvider(String value) { onChanged(); return this; } + /** + * required string cipherProvider = 1; + */ public Builder clearCipherProvider() { bitField0_ = (bitField0_ & ~0x00000001); cipherProvider_ = getDefaultInstance().getCipherProvider(); onChanged(); return this; } - void setCipherProvider(com.google.protobuf.ByteString value) { - bitField0_ |= 0x00000001; + /** + * required string cipherProvider = 1; + */ + public Builder setCipherProviderBytes( + com.google.protobuf.ByteString value) { + if (value == null) { + throw new NullPointerException(); + } + bitField0_ |= 0x00000001; cipherProvider_ = value; onChanged(); + return this; } - + // required string keyAlias = 2; private java.lang.Object keyAlias_ = ""; + /** + * required string keyAlias = 2; + */ public boolean hasKeyAlias() { return ((bitField0_ & 0x00000002) == 0x00000002); } - public String getKeyAlias() { + /** + * required string keyAlias = 2; + */ + public java.lang.String getKeyAlias() { java.lang.Object ref = keyAlias_; - if (!(ref instanceof String)) { - String s = ((com.google.protobuf.ByteString) ref).toStringUtf8(); + if (!(ref instanceof java.lang.String)) { + java.lang.String s = ((com.google.protobuf.ByteString) ref) + .toStringUtf8(); keyAlias_ = s; return s; } else { - return (String) ref; + return (java.lang.String) ref; } } - public Builder setKeyAlias(String value) { + /** + * required string keyAlias = 2; + */ + public com.google.protobuf.ByteString + getKeyAliasBytes() { + java.lang.Object ref = keyAlias_; + if (ref instanceof String) { + com.google.protobuf.ByteString b = + com.google.protobuf.ByteString.copyFromUtf8( + (java.lang.String) ref); + keyAlias_ = b; + return b; + } else { + return (com.google.protobuf.ByteString) ref; + } + } + /** + * required string keyAlias = 2; + */ + public Builder setKeyAlias( + java.lang.String value) { if (value == null) { throw new NullPointerException(); } @@ -2571,26 +3258,46 @@ public Builder setKeyAlias(String value) { onChanged(); return this; } + /** + * required string keyAlias = 2; + */ public Builder clearKeyAlias() { bitField0_ = (bitField0_ & ~0x00000002); keyAlias_ = getDefaultInstance().getKeyAlias(); onChanged(); return this; } - void setKeyAlias(com.google.protobuf.ByteString value) { - bitField0_ |= 0x00000002; + /** + * required string keyAlias = 2; + */ + public Builder setKeyAliasBytes( + com.google.protobuf.ByteString value) { + if (value == null) { + throw new NullPointerException(); + } + bitField0_ |= 0x00000002; keyAlias_ = value; onChanged(); + return this; } - + // optional bytes parameters = 3; private com.google.protobuf.ByteString parameters_ = com.google.protobuf.ByteString.EMPTY; + /** + * optional bytes parameters = 3; + */ public boolean hasParameters() { return ((bitField0_ & 0x00000004) == 0x00000004); } + /** + * optional bytes parameters = 3; + */ public com.google.protobuf.ByteString getParameters() { return parameters_; } + /** + * optional bytes parameters = 3; + */ public Builder setParameters(com.google.protobuf.ByteString value) { if (value == null) { throw new NullPointerException(); @@ -2600,98 +3307,214 @@ public Builder setParameters(com.google.protobuf.ByteString value) { onChanged(); return this; } + /** + * optional bytes parameters = 3; + */ public Builder clearParameters() { bitField0_ = (bitField0_ & ~0x00000004); parameters_ = getDefaultInstance().getParameters(); onChanged(); return this; } - + // @@protoc_insertion_point(builder_scope:LogFileEncryption) } - + static { defaultInstance = new LogFileEncryption(true); defaultInstance.initFields(); } - + // @@protoc_insertion_point(class_scope:LogFileEncryption) } - + public interface TransactionEventHeaderOrBuilder extends com.google.protobuf.MessageOrBuilder { - + // required sfixed32 type = 1; + /** + * required sfixed32 type = 1; + */ boolean hasType(); + /** + * required sfixed32 type = 1; + */ int getType(); - + // required sfixed64 transactionID = 2; + /** + * required sfixed64 transactionID = 2; + */ boolean hasTransactionID(); + /** + * required sfixed64 transactionID = 2; + */ long getTransactionID(); - + // required sfixed64 writeOrderID = 3; + /** + * required sfixed64 writeOrderID = 3; + */ boolean hasWriteOrderID(); + /** + * required sfixed64 writeOrderID = 3; + */ long getWriteOrderID(); } + /** + * Protobuf type {@code TransactionEventHeader} + */ public static final class TransactionEventHeader extends com.google.protobuf.GeneratedMessage implements TransactionEventHeaderOrBuilder { // Use TransactionEventHeader.newBuilder() to construct. - private TransactionEventHeader(Builder builder) { + private TransactionEventHeader(com.google.protobuf.GeneratedMessage.Builder builder) { super(builder); + this.unknownFields = builder.getUnknownFields(); } - private TransactionEventHeader(boolean noInit) {} - + private TransactionEventHeader(boolean noInit) { this.unknownFields = com.google.protobuf.UnknownFieldSet.getDefaultInstance(); } + private static final TransactionEventHeader defaultInstance; public static TransactionEventHeader getDefaultInstance() { return defaultInstance; } - + public TransactionEventHeader getDefaultInstanceForType() { return defaultInstance; } - + + private final com.google.protobuf.UnknownFieldSet unknownFields; + @java.lang.Override + public final com.google.protobuf.UnknownFieldSet + getUnknownFields() { + return this.unknownFields; + } + private TransactionEventHeader( + com.google.protobuf.CodedInputStream input, + com.google.protobuf.ExtensionRegistryLite extensionRegistry) + throws com.google.protobuf.InvalidProtocolBufferException { + initFields(); + int mutable_bitField0_ = 0; + com.google.protobuf.UnknownFieldSet.Builder unknownFields = + com.google.protobuf.UnknownFieldSet.newBuilder(); + try { + boolean done = false; + while (!done) { + int tag = input.readTag(); + switch (tag) { + case 0: + done = true; + break; + default: { + if (!parseUnknownField(input, unknownFields, + extensionRegistry, tag)) { + done = true; + } + break; + } + case 13: { + bitField0_ |= 0x00000001; + type_ = input.readSFixed32(); + break; + } + case 17: { + bitField0_ |= 0x00000002; + transactionID_ = input.readSFixed64(); + break; + } + case 25: { + bitField0_ |= 0x00000004; + writeOrderID_ = input.readSFixed64(); + break; + } + } + } + } catch (com.google.protobuf.InvalidProtocolBufferException e) { + throw e.setUnfinishedMessage(this); + } catch (java.io.IOException e) { + throw new com.google.protobuf.InvalidProtocolBufferException( + e.getMessage()).setUnfinishedMessage(this); + } finally { + this.unknownFields = unknownFields.build(); + makeExtensionsImmutable(); + } + } public static final com.google.protobuf.Descriptors.Descriptor getDescriptor() { return org.apache.flume.channel.file.proto.ProtosFactory.internal_static_TransactionEventHeader_descriptor; } - + protected com.google.protobuf.GeneratedMessage.FieldAccessorTable internalGetFieldAccessorTable() { - return org.apache.flume.channel.file.proto.ProtosFactory.internal_static_TransactionEventHeader_fieldAccessorTable; + return org.apache.flume.channel.file.proto.ProtosFactory.internal_static_TransactionEventHeader_fieldAccessorTable + .ensureFieldAccessorsInitialized( + org.apache.flume.channel.file.proto.ProtosFactory.TransactionEventHeader.class, org.apache.flume.channel.file.proto.ProtosFactory.TransactionEventHeader.Builder.class); + } + + public static com.google.protobuf.Parser PARSER = + new com.google.protobuf.AbstractParser() { + public TransactionEventHeader parsePartialFrom( + com.google.protobuf.CodedInputStream input, + com.google.protobuf.ExtensionRegistryLite extensionRegistry) + throws com.google.protobuf.InvalidProtocolBufferException { + return new TransactionEventHeader(input, extensionRegistry); + } + }; + + @java.lang.Override + public com.google.protobuf.Parser getParserForType() { + return PARSER; } - + private int bitField0_; // required sfixed32 type = 1; public static final int TYPE_FIELD_NUMBER = 1; private int type_; + /** + * required sfixed32 type = 1; + */ public boolean hasType() { return ((bitField0_ & 0x00000001) == 0x00000001); } + /** + * required sfixed32 type = 1; + */ public int getType() { return type_; } - + // required sfixed64 transactionID = 2; public static final int TRANSACTIONID_FIELD_NUMBER = 2; private long transactionID_; + /** + * required sfixed64 transactionID = 2; + */ public boolean hasTransactionID() { return ((bitField0_ & 0x00000002) == 0x00000002); } + /** + * required sfixed64 transactionID = 2; + */ public long getTransactionID() { return transactionID_; } - + // required sfixed64 writeOrderID = 3; public static final int WRITEORDERID_FIELD_NUMBER = 3; private long writeOrderID_; + /** + * required sfixed64 writeOrderID = 3; + */ public boolean hasWriteOrderID() { return ((bitField0_ & 0x00000004) == 0x00000004); } + /** + * required sfixed64 writeOrderID = 3; + */ public long getWriteOrderID() { return writeOrderID_; } - + private void initFields() { type_ = 0; transactionID_ = 0L; @@ -2701,7 +3524,7 @@ private void initFields() { public final boolean isInitialized() { byte isInitialized = memoizedIsInitialized; if (isInitialized != -1) return isInitialized == 1; - + if (!hasType()) { memoizedIsInitialized = 0; return false; @@ -2717,7 +3540,7 @@ public final boolean isInitialized() { memoizedIsInitialized = 1; return true; } - + public void writeTo(com.google.protobuf.CodedOutputStream output) throws java.io.IOException { getSerializedSize(); @@ -2732,12 +3555,12 @@ public void writeTo(com.google.protobuf.CodedOutputStream output) } getUnknownFields().writeTo(output); } - + private int memoizedSerializedSize = -1; public int getSerializedSize() { int size = memoizedSerializedSize; if (size != -1) return size; - + size = 0; if (((bitField0_ & 0x00000001) == 0x00000001)) { size += com.google.protobuf.CodedOutputStream @@ -2755,94 +3578,83 @@ public int getSerializedSize() { memoizedSerializedSize = size; return size; } - + private static final long serialVersionUID = 0L; @java.lang.Override protected java.lang.Object writeReplace() throws java.io.ObjectStreamException { return super.writeReplace(); } - + public static org.apache.flume.channel.file.proto.ProtosFactory.TransactionEventHeader parseFrom( com.google.protobuf.ByteString data) throws com.google.protobuf.InvalidProtocolBufferException { - return newBuilder().mergeFrom(data).buildParsed(); + return PARSER.parseFrom(data); } public static org.apache.flume.channel.file.proto.ProtosFactory.TransactionEventHeader parseFrom( com.google.protobuf.ByteString data, com.google.protobuf.ExtensionRegistryLite extensionRegistry) throws com.google.protobuf.InvalidProtocolBufferException { - return newBuilder().mergeFrom(data, extensionRegistry) - .buildParsed(); + return PARSER.parseFrom(data, extensionRegistry); } public static org.apache.flume.channel.file.proto.ProtosFactory.TransactionEventHeader parseFrom(byte[] data) throws com.google.protobuf.InvalidProtocolBufferException { - return newBuilder().mergeFrom(data).buildParsed(); + return PARSER.parseFrom(data); } public static org.apache.flume.channel.file.proto.ProtosFactory.TransactionEventHeader parseFrom( byte[] data, com.google.protobuf.ExtensionRegistryLite extensionRegistry) throws com.google.protobuf.InvalidProtocolBufferException { - return newBuilder().mergeFrom(data, extensionRegistry) - .buildParsed(); + return PARSER.parseFrom(data, extensionRegistry); } public static org.apache.flume.channel.file.proto.ProtosFactory.TransactionEventHeader parseFrom(java.io.InputStream input) throws java.io.IOException { - return newBuilder().mergeFrom(input).buildParsed(); + return PARSER.parseFrom(input); } public static org.apache.flume.channel.file.proto.ProtosFactory.TransactionEventHeader parseFrom( java.io.InputStream input, com.google.protobuf.ExtensionRegistryLite extensionRegistry) throws java.io.IOException { - return newBuilder().mergeFrom(input, extensionRegistry) - .buildParsed(); + return PARSER.parseFrom(input, extensionRegistry); } public static org.apache.flume.channel.file.proto.ProtosFactory.TransactionEventHeader parseDelimitedFrom(java.io.InputStream input) throws java.io.IOException { - Builder builder = newBuilder(); - if (builder.mergeDelimitedFrom(input)) { - return builder.buildParsed(); - } else { - return null; - } + return PARSER.parseDelimitedFrom(input); } public static org.apache.flume.channel.file.proto.ProtosFactory.TransactionEventHeader parseDelimitedFrom( java.io.InputStream input, com.google.protobuf.ExtensionRegistryLite extensionRegistry) throws java.io.IOException { - Builder builder = newBuilder(); - if (builder.mergeDelimitedFrom(input, extensionRegistry)) { - return builder.buildParsed(); - } else { - return null; - } + return PARSER.parseDelimitedFrom(input, extensionRegistry); } public static org.apache.flume.channel.file.proto.ProtosFactory.TransactionEventHeader parseFrom( com.google.protobuf.CodedInputStream input) throws java.io.IOException { - return newBuilder().mergeFrom(input).buildParsed(); + return PARSER.parseFrom(input); } public static org.apache.flume.channel.file.proto.ProtosFactory.TransactionEventHeader parseFrom( com.google.protobuf.CodedInputStream input, com.google.protobuf.ExtensionRegistryLite extensionRegistry) throws java.io.IOException { - return newBuilder().mergeFrom(input, extensionRegistry) - .buildParsed(); + return PARSER.parseFrom(input, extensionRegistry); } - + public static Builder newBuilder() { return Builder.create(); } public Builder newBuilderForType() { return newBuilder(); } public static Builder newBuilder(org.apache.flume.channel.file.proto.ProtosFactory.TransactionEventHeader prototype) { return newBuilder().mergeFrom(prototype); } public Builder toBuilder() { return newBuilder(this); } - + @java.lang.Override protected Builder newBuilderForType( com.google.protobuf.GeneratedMessage.BuilderParent parent) { Builder builder = new Builder(parent); return builder; } + /** + * Protobuf type {@code TransactionEventHeader} + */ public static final class Builder extends com.google.protobuf.GeneratedMessage.Builder implements org.apache.flume.channel.file.proto.ProtosFactory.TransactionEventHeaderOrBuilder { @@ -2850,18 +3662,21 @@ public static final class Builder extends getDescriptor() { return org.apache.flume.channel.file.proto.ProtosFactory.internal_static_TransactionEventHeader_descriptor; } - + protected com.google.protobuf.GeneratedMessage.FieldAccessorTable internalGetFieldAccessorTable() { - return org.apache.flume.channel.file.proto.ProtosFactory.internal_static_TransactionEventHeader_fieldAccessorTable; + return org.apache.flume.channel.file.proto.ProtosFactory.internal_static_TransactionEventHeader_fieldAccessorTable + .ensureFieldAccessorsInitialized( + org.apache.flume.channel.file.proto.ProtosFactory.TransactionEventHeader.class, org.apache.flume.channel.file.proto.ProtosFactory.TransactionEventHeader.Builder.class); } - + // Construct using org.apache.flume.channel.file.proto.ProtosFactory.TransactionEventHeader.newBuilder() private Builder() { maybeForceBuilderInitialization(); } - - private Builder(BuilderParent parent) { + + private Builder( + com.google.protobuf.GeneratedMessage.BuilderParent parent) { super(parent); maybeForceBuilderInitialization(); } @@ -2872,7 +3687,7 @@ private void maybeForceBuilderInitialization() { private static Builder create() { return new Builder(); } - + public Builder clear() { super.clear(); type_ = 0; @@ -2883,20 +3698,20 @@ public Builder clear() { bitField0_ = (bitField0_ & ~0x00000004); return this; } - + public Builder clone() { return create().mergeFrom(buildPartial()); } - + public com.google.protobuf.Descriptors.Descriptor getDescriptorForType() { - return org.apache.flume.channel.file.proto.ProtosFactory.TransactionEventHeader.getDescriptor(); + return org.apache.flume.channel.file.proto.ProtosFactory.internal_static_TransactionEventHeader_descriptor; } - + public org.apache.flume.channel.file.proto.ProtosFactory.TransactionEventHeader getDefaultInstanceForType() { return org.apache.flume.channel.file.proto.ProtosFactory.TransactionEventHeader.getDefaultInstance(); } - + public org.apache.flume.channel.file.proto.ProtosFactory.TransactionEventHeader build() { org.apache.flume.channel.file.proto.ProtosFactory.TransactionEventHeader result = buildPartial(); if (!result.isInitialized()) { @@ -2904,17 +3719,7 @@ public org.apache.flume.channel.file.proto.ProtosFactory.TransactionEventHeader } return result; } - - private org.apache.flume.channel.file.proto.ProtosFactory.TransactionEventHeader buildParsed() - throws com.google.protobuf.InvalidProtocolBufferException { - org.apache.flume.channel.file.proto.ProtosFactory.TransactionEventHeader result = buildPartial(); - if (!result.isInitialized()) { - throw newUninitializedMessageException( - result).asInvalidProtocolBufferException(); - } - return result; - } - + public org.apache.flume.channel.file.proto.ProtosFactory.TransactionEventHeader buildPartial() { org.apache.flume.channel.file.proto.ProtosFactory.TransactionEventHeader result = new org.apache.flume.channel.file.proto.ProtosFactory.TransactionEventHeader(this); int from_bitField0_ = bitField0_; @@ -2935,7 +3740,7 @@ public org.apache.flume.channel.file.proto.ProtosFactory.TransactionEventHeader onBuilt(); return result; } - + public Builder mergeFrom(com.google.protobuf.Message other) { if (other instanceof org.apache.flume.channel.file.proto.ProtosFactory.TransactionEventHeader) { return mergeFrom((org.apache.flume.channel.file.proto.ProtosFactory.TransactionEventHeader)other); @@ -2944,7 +3749,7 @@ public Builder mergeFrom(com.google.protobuf.Message other) { return this; } } - + public Builder mergeFrom(org.apache.flume.channel.file.proto.ProtosFactory.TransactionEventHeader other) { if (other == org.apache.flume.channel.file.proto.ProtosFactory.TransactionEventHeader.getDefaultInstance()) return this; if (other.hasType()) { @@ -2959,7 +3764,7 @@ public Builder mergeFrom(org.apache.flume.channel.file.proto.ProtosFactory.Trans this.mergeUnknownFields(other.getUnknownFields()); return this; } - + public final boolean isInitialized() { if (!hasType()) { @@ -2975,189 +3780,310 @@ public final boolean isInitialized() { } return true; } - + public Builder mergeFrom( com.google.protobuf.CodedInputStream input, com.google.protobuf.ExtensionRegistryLite extensionRegistry) throws java.io.IOException { - com.google.protobuf.UnknownFieldSet.Builder unknownFields = - com.google.protobuf.UnknownFieldSet.newBuilder( - this.getUnknownFields()); - while (true) { - int tag = input.readTag(); - switch (tag) { - case 0: - this.setUnknownFields(unknownFields.build()); - onChanged(); - return this; - default: { - if (!parseUnknownField(input, unknownFields, - extensionRegistry, tag)) { - this.setUnknownFields(unknownFields.build()); - onChanged(); - return this; - } - break; - } - case 13: { - bitField0_ |= 0x00000001; - type_ = input.readSFixed32(); - break; - } - case 17: { - bitField0_ |= 0x00000002; - transactionID_ = input.readSFixed64(); - break; - } - case 25: { - bitField0_ |= 0x00000004; - writeOrderID_ = input.readSFixed64(); - break; - } + org.apache.flume.channel.file.proto.ProtosFactory.TransactionEventHeader parsedMessage = null; + try { + parsedMessage = PARSER.parsePartialFrom(input, extensionRegistry); + } catch (com.google.protobuf.InvalidProtocolBufferException e) { + parsedMessage = (org.apache.flume.channel.file.proto.ProtosFactory.TransactionEventHeader) e.getUnfinishedMessage(); + throw e; + } finally { + if (parsedMessage != null) { + mergeFrom(parsedMessage); } } + return this; } - private int bitField0_; - + // required sfixed32 type = 1; private int type_ ; + /** + * required sfixed32 type = 1; + */ public boolean hasType() { return ((bitField0_ & 0x00000001) == 0x00000001); } + /** + * required sfixed32 type = 1; + */ public int getType() { return type_; } + /** + * required sfixed32 type = 1; + */ public Builder setType(int value) { bitField0_ |= 0x00000001; type_ = value; onChanged(); return this; } + /** + * required sfixed32 type = 1; + */ public Builder clearType() { bitField0_ = (bitField0_ & ~0x00000001); type_ = 0; onChanged(); return this; } - + // required sfixed64 transactionID = 2; private long transactionID_ ; + /** + * required sfixed64 transactionID = 2; + */ public boolean hasTransactionID() { return ((bitField0_ & 0x00000002) == 0x00000002); } + /** + * required sfixed64 transactionID = 2; + */ public long getTransactionID() { return transactionID_; } + /** + * required sfixed64 transactionID = 2; + */ public Builder setTransactionID(long value) { bitField0_ |= 0x00000002; transactionID_ = value; onChanged(); return this; } + /** + * required sfixed64 transactionID = 2; + */ public Builder clearTransactionID() { bitField0_ = (bitField0_ & ~0x00000002); transactionID_ = 0L; onChanged(); return this; } - + // required sfixed64 writeOrderID = 3; private long writeOrderID_ ; + /** + * required sfixed64 writeOrderID = 3; + */ public boolean hasWriteOrderID() { return ((bitField0_ & 0x00000004) == 0x00000004); } + /** + * required sfixed64 writeOrderID = 3; + */ public long getWriteOrderID() { return writeOrderID_; } + /** + * required sfixed64 writeOrderID = 3; + */ public Builder setWriteOrderID(long value) { bitField0_ |= 0x00000004; writeOrderID_ = value; onChanged(); return this; } + /** + * required sfixed64 writeOrderID = 3; + */ public Builder clearWriteOrderID() { bitField0_ = (bitField0_ & ~0x00000004); writeOrderID_ = 0L; onChanged(); return this; } - + // @@protoc_insertion_point(builder_scope:TransactionEventHeader) } - + static { defaultInstance = new TransactionEventHeader(true); defaultInstance.initFields(); } - + // @@protoc_insertion_point(class_scope:TransactionEventHeader) } - + public interface PutOrBuilder extends com.google.protobuf.MessageOrBuilder { - + // required .FlumeEvent event = 1; + /** + * required .FlumeEvent event = 1; + */ boolean hasEvent(); + /** + * required .FlumeEvent event = 1; + */ org.apache.flume.channel.file.proto.ProtosFactory.FlumeEvent getEvent(); + /** + * required .FlumeEvent event = 1; + */ org.apache.flume.channel.file.proto.ProtosFactory.FlumeEventOrBuilder getEventOrBuilder(); - + // optional sfixed64 checksum = 2; + /** + * optional sfixed64 checksum = 2; + */ boolean hasChecksum(); + /** + * optional sfixed64 checksum = 2; + */ long getChecksum(); } + /** + * Protobuf type {@code Put} + */ public static final class Put extends com.google.protobuf.GeneratedMessage implements PutOrBuilder { // Use Put.newBuilder() to construct. - private Put(Builder builder) { + private Put(com.google.protobuf.GeneratedMessage.Builder builder) { super(builder); + this.unknownFields = builder.getUnknownFields(); } - private Put(boolean noInit) {} - + private Put(boolean noInit) { this.unknownFields = com.google.protobuf.UnknownFieldSet.getDefaultInstance(); } + private static final Put defaultInstance; public static Put getDefaultInstance() { return defaultInstance; } - + public Put getDefaultInstanceForType() { return defaultInstance; } - + + private final com.google.protobuf.UnknownFieldSet unknownFields; + @java.lang.Override + public final com.google.protobuf.UnknownFieldSet + getUnknownFields() { + return this.unknownFields; + } + private Put( + com.google.protobuf.CodedInputStream input, + com.google.protobuf.ExtensionRegistryLite extensionRegistry) + throws com.google.protobuf.InvalidProtocolBufferException { + initFields(); + int mutable_bitField0_ = 0; + com.google.protobuf.UnknownFieldSet.Builder unknownFields = + com.google.protobuf.UnknownFieldSet.newBuilder(); + try { + boolean done = false; + while (!done) { + int tag = input.readTag(); + switch (tag) { + case 0: + done = true; + break; + default: { + if (!parseUnknownField(input, unknownFields, + extensionRegistry, tag)) { + done = true; + } + break; + } + case 10: { + org.apache.flume.channel.file.proto.ProtosFactory.FlumeEvent.Builder subBuilder = null; + if (((bitField0_ & 0x00000001) == 0x00000001)) { + subBuilder = event_.toBuilder(); + } + event_ = input.readMessage(org.apache.flume.channel.file.proto.ProtosFactory.FlumeEvent.PARSER, extensionRegistry); + if (subBuilder != null) { + subBuilder.mergeFrom(event_); + event_ = subBuilder.buildPartial(); + } + bitField0_ |= 0x00000001; + break; + } + case 17: { + bitField0_ |= 0x00000002; + checksum_ = input.readSFixed64(); + break; + } + } + } + } catch (com.google.protobuf.InvalidProtocolBufferException e) { + throw e.setUnfinishedMessage(this); + } catch (java.io.IOException e) { + throw new com.google.protobuf.InvalidProtocolBufferException( + e.getMessage()).setUnfinishedMessage(this); + } finally { + this.unknownFields = unknownFields.build(); + makeExtensionsImmutable(); + } + } public static final com.google.protobuf.Descriptors.Descriptor getDescriptor() { return org.apache.flume.channel.file.proto.ProtosFactory.internal_static_Put_descriptor; } - + protected com.google.protobuf.GeneratedMessage.FieldAccessorTable internalGetFieldAccessorTable() { - return org.apache.flume.channel.file.proto.ProtosFactory.internal_static_Put_fieldAccessorTable; + return org.apache.flume.channel.file.proto.ProtosFactory.internal_static_Put_fieldAccessorTable + .ensureFieldAccessorsInitialized( + org.apache.flume.channel.file.proto.ProtosFactory.Put.class, org.apache.flume.channel.file.proto.ProtosFactory.Put.Builder.class); + } + + public static com.google.protobuf.Parser PARSER = + new com.google.protobuf.AbstractParser() { + public Put parsePartialFrom( + com.google.protobuf.CodedInputStream input, + com.google.protobuf.ExtensionRegistryLite extensionRegistry) + throws com.google.protobuf.InvalidProtocolBufferException { + return new Put(input, extensionRegistry); + } + }; + + @java.lang.Override + public com.google.protobuf.Parser getParserForType() { + return PARSER; } - + private int bitField0_; // required .FlumeEvent event = 1; public static final int EVENT_FIELD_NUMBER = 1; private org.apache.flume.channel.file.proto.ProtosFactory.FlumeEvent event_; + /** + * required .FlumeEvent event = 1; + */ public boolean hasEvent() { return ((bitField0_ & 0x00000001) == 0x00000001); } + /** + * required .FlumeEvent event = 1; + */ public org.apache.flume.channel.file.proto.ProtosFactory.FlumeEvent getEvent() { return event_; } + /** + * required .FlumeEvent event = 1; + */ public org.apache.flume.channel.file.proto.ProtosFactory.FlumeEventOrBuilder getEventOrBuilder() { return event_; } - + // optional sfixed64 checksum = 2; public static final int CHECKSUM_FIELD_NUMBER = 2; private long checksum_; + /** + * optional sfixed64 checksum = 2; + */ public boolean hasChecksum() { return ((bitField0_ & 0x00000002) == 0x00000002); } + /** + * optional sfixed64 checksum = 2; + */ public long getChecksum() { return checksum_; } - + private void initFields() { event_ = org.apache.flume.channel.file.proto.ProtosFactory.FlumeEvent.getDefaultInstance(); checksum_ = 0L; @@ -3166,7 +4092,7 @@ private void initFields() { public final boolean isInitialized() { byte isInitialized = memoizedIsInitialized; if (isInitialized != -1) return isInitialized == 1; - + if (!hasEvent()) { memoizedIsInitialized = 0; return false; @@ -3178,7 +4104,7 @@ public final boolean isInitialized() { memoizedIsInitialized = 1; return true; } - + public void writeTo(com.google.protobuf.CodedOutputStream output) throws java.io.IOException { getSerializedSize(); @@ -3190,12 +4116,12 @@ public void writeTo(com.google.protobuf.CodedOutputStream output) } getUnknownFields().writeTo(output); } - + private int memoizedSerializedSize = -1; public int getSerializedSize() { int size = memoizedSerializedSize; if (size != -1) return size; - + size = 0; if (((bitField0_ & 0x00000001) == 0x00000001)) { size += com.google.protobuf.CodedOutputStream @@ -3209,94 +4135,83 @@ public int getSerializedSize() { memoizedSerializedSize = size; return size; } - + private static final long serialVersionUID = 0L; @java.lang.Override protected java.lang.Object writeReplace() throws java.io.ObjectStreamException { return super.writeReplace(); } - + public static org.apache.flume.channel.file.proto.ProtosFactory.Put parseFrom( com.google.protobuf.ByteString data) throws com.google.protobuf.InvalidProtocolBufferException { - return newBuilder().mergeFrom(data).buildParsed(); + return PARSER.parseFrom(data); } public static org.apache.flume.channel.file.proto.ProtosFactory.Put parseFrom( com.google.protobuf.ByteString data, com.google.protobuf.ExtensionRegistryLite extensionRegistry) throws com.google.protobuf.InvalidProtocolBufferException { - return newBuilder().mergeFrom(data, extensionRegistry) - .buildParsed(); + return PARSER.parseFrom(data, extensionRegistry); } public static org.apache.flume.channel.file.proto.ProtosFactory.Put parseFrom(byte[] data) throws com.google.protobuf.InvalidProtocolBufferException { - return newBuilder().mergeFrom(data).buildParsed(); + return PARSER.parseFrom(data); } public static org.apache.flume.channel.file.proto.ProtosFactory.Put parseFrom( byte[] data, com.google.protobuf.ExtensionRegistryLite extensionRegistry) throws com.google.protobuf.InvalidProtocolBufferException { - return newBuilder().mergeFrom(data, extensionRegistry) - .buildParsed(); + return PARSER.parseFrom(data, extensionRegistry); } public static org.apache.flume.channel.file.proto.ProtosFactory.Put parseFrom(java.io.InputStream input) throws java.io.IOException { - return newBuilder().mergeFrom(input).buildParsed(); + return PARSER.parseFrom(input); } public static org.apache.flume.channel.file.proto.ProtosFactory.Put parseFrom( java.io.InputStream input, com.google.protobuf.ExtensionRegistryLite extensionRegistry) throws java.io.IOException { - return newBuilder().mergeFrom(input, extensionRegistry) - .buildParsed(); + return PARSER.parseFrom(input, extensionRegistry); } public static org.apache.flume.channel.file.proto.ProtosFactory.Put parseDelimitedFrom(java.io.InputStream input) throws java.io.IOException { - Builder builder = newBuilder(); - if (builder.mergeDelimitedFrom(input)) { - return builder.buildParsed(); - } else { - return null; - } + return PARSER.parseDelimitedFrom(input); } public static org.apache.flume.channel.file.proto.ProtosFactory.Put parseDelimitedFrom( java.io.InputStream input, com.google.protobuf.ExtensionRegistryLite extensionRegistry) throws java.io.IOException { - Builder builder = newBuilder(); - if (builder.mergeDelimitedFrom(input, extensionRegistry)) { - return builder.buildParsed(); - } else { - return null; - } + return PARSER.parseDelimitedFrom(input, extensionRegistry); } public static org.apache.flume.channel.file.proto.ProtosFactory.Put parseFrom( com.google.protobuf.CodedInputStream input) throws java.io.IOException { - return newBuilder().mergeFrom(input).buildParsed(); + return PARSER.parseFrom(input); } public static org.apache.flume.channel.file.proto.ProtosFactory.Put parseFrom( com.google.protobuf.CodedInputStream input, com.google.protobuf.ExtensionRegistryLite extensionRegistry) throws java.io.IOException { - return newBuilder().mergeFrom(input, extensionRegistry) - .buildParsed(); + return PARSER.parseFrom(input, extensionRegistry); } - + public static Builder newBuilder() { return Builder.create(); } public Builder newBuilderForType() { return newBuilder(); } public static Builder newBuilder(org.apache.flume.channel.file.proto.ProtosFactory.Put prototype) { return newBuilder().mergeFrom(prototype); } public Builder toBuilder() { return newBuilder(this); } - + @java.lang.Override protected Builder newBuilderForType( com.google.protobuf.GeneratedMessage.BuilderParent parent) { Builder builder = new Builder(parent); return builder; } + /** + * Protobuf type {@code Put} + */ public static final class Builder extends com.google.protobuf.GeneratedMessage.Builder implements org.apache.flume.channel.file.proto.ProtosFactory.PutOrBuilder { @@ -3304,18 +4219,21 @@ public static final class Builder extends getDescriptor() { return org.apache.flume.channel.file.proto.ProtosFactory.internal_static_Put_descriptor; } - + protected com.google.protobuf.GeneratedMessage.FieldAccessorTable internalGetFieldAccessorTable() { - return org.apache.flume.channel.file.proto.ProtosFactory.internal_static_Put_fieldAccessorTable; + return org.apache.flume.channel.file.proto.ProtosFactory.internal_static_Put_fieldAccessorTable + .ensureFieldAccessorsInitialized( + org.apache.flume.channel.file.proto.ProtosFactory.Put.class, org.apache.flume.channel.file.proto.ProtosFactory.Put.Builder.class); } - + // Construct using org.apache.flume.channel.file.proto.ProtosFactory.Put.newBuilder() private Builder() { maybeForceBuilderInitialization(); } - - private Builder(BuilderParent parent) { + + private Builder( + com.google.protobuf.GeneratedMessage.BuilderParent parent) { super(parent); maybeForceBuilderInitialization(); } @@ -3327,7 +4245,7 @@ private void maybeForceBuilderInitialization() { private static Builder create() { return new Builder(); } - + public Builder clear() { super.clear(); if (eventBuilder_ == null) { @@ -3340,20 +4258,20 @@ public Builder clear() { bitField0_ = (bitField0_ & ~0x00000002); return this; } - + public Builder clone() { return create().mergeFrom(buildPartial()); } - + public com.google.protobuf.Descriptors.Descriptor getDescriptorForType() { - return org.apache.flume.channel.file.proto.ProtosFactory.Put.getDescriptor(); + return org.apache.flume.channel.file.proto.ProtosFactory.internal_static_Put_descriptor; } - + public org.apache.flume.channel.file.proto.ProtosFactory.Put getDefaultInstanceForType() { return org.apache.flume.channel.file.proto.ProtosFactory.Put.getDefaultInstance(); } - + public org.apache.flume.channel.file.proto.ProtosFactory.Put build() { org.apache.flume.channel.file.proto.ProtosFactory.Put result = buildPartial(); if (!result.isInitialized()) { @@ -3361,17 +4279,7 @@ public org.apache.flume.channel.file.proto.ProtosFactory.Put build() { } return result; } - - private org.apache.flume.channel.file.proto.ProtosFactory.Put buildParsed() - throws com.google.protobuf.InvalidProtocolBufferException { - org.apache.flume.channel.file.proto.ProtosFactory.Put result = buildPartial(); - if (!result.isInitialized()) { - throw newUninitializedMessageException( - result).asInvalidProtocolBufferException(); - } - return result; - } - + public org.apache.flume.channel.file.proto.ProtosFactory.Put buildPartial() { org.apache.flume.channel.file.proto.ProtosFactory.Put result = new org.apache.flume.channel.file.proto.ProtosFactory.Put(this); int from_bitField0_ = bitField0_; @@ -3392,7 +4300,7 @@ public org.apache.flume.channel.file.proto.ProtosFactory.Put buildPartial() { onBuilt(); return result; } - + public Builder mergeFrom(com.google.protobuf.Message other) { if (other instanceof org.apache.flume.channel.file.proto.ProtosFactory.Put) { return mergeFrom((org.apache.flume.channel.file.proto.ProtosFactory.Put)other); @@ -3401,7 +4309,7 @@ public Builder mergeFrom(com.google.protobuf.Message other) { return this; } } - + public Builder mergeFrom(org.apache.flume.channel.file.proto.ProtosFactory.Put other) { if (other == org.apache.flume.channel.file.proto.ProtosFactory.Put.getDefaultInstance()) return this; if (other.hasEvent()) { @@ -3413,7 +4321,7 @@ public Builder mergeFrom(org.apache.flume.channel.file.proto.ProtosFactory.Put o this.mergeUnknownFields(other.getUnknownFields()); return this; } - + public final boolean isInitialized() { if (!hasEvent()) { @@ -3425,57 +4333,39 @@ public final boolean isInitialized() { } return true; } - + public Builder mergeFrom( com.google.protobuf.CodedInputStream input, com.google.protobuf.ExtensionRegistryLite extensionRegistry) throws java.io.IOException { - com.google.protobuf.UnknownFieldSet.Builder unknownFields = - com.google.protobuf.UnknownFieldSet.newBuilder( - this.getUnknownFields()); - while (true) { - int tag = input.readTag(); - switch (tag) { - case 0: - this.setUnknownFields(unknownFields.build()); - onChanged(); - return this; - default: { - if (!parseUnknownField(input, unknownFields, - extensionRegistry, tag)) { - this.setUnknownFields(unknownFields.build()); - onChanged(); - return this; - } - break; - } - case 10: { - org.apache.flume.channel.file.proto.ProtosFactory.FlumeEvent.Builder subBuilder = org.apache.flume.channel.file.proto.ProtosFactory.FlumeEvent.newBuilder(); - if (hasEvent()) { - subBuilder.mergeFrom(getEvent()); - } - input.readMessage(subBuilder, extensionRegistry); - setEvent(subBuilder.buildPartial()); - break; - } - case 17: { - bitField0_ |= 0x00000002; - checksum_ = input.readSFixed64(); - break; - } + org.apache.flume.channel.file.proto.ProtosFactory.Put parsedMessage = null; + try { + parsedMessage = PARSER.parsePartialFrom(input, extensionRegistry); + } catch (com.google.protobuf.InvalidProtocolBufferException e) { + parsedMessage = (org.apache.flume.channel.file.proto.ProtosFactory.Put) e.getUnfinishedMessage(); + throw e; + } finally { + if (parsedMessage != null) { + mergeFrom(parsedMessage); } } + return this; } - private int bitField0_; - + // required .FlumeEvent event = 1; private org.apache.flume.channel.file.proto.ProtosFactory.FlumeEvent event_ = org.apache.flume.channel.file.proto.ProtosFactory.FlumeEvent.getDefaultInstance(); private com.google.protobuf.SingleFieldBuilder< org.apache.flume.channel.file.proto.ProtosFactory.FlumeEvent, org.apache.flume.channel.file.proto.ProtosFactory.FlumeEvent.Builder, org.apache.flume.channel.file.proto.ProtosFactory.FlumeEventOrBuilder> eventBuilder_; + /** + * required .FlumeEvent event = 1; + */ public boolean hasEvent() { return ((bitField0_ & 0x00000001) == 0x00000001); } + /** + * required .FlumeEvent event = 1; + */ public org.apache.flume.channel.file.proto.ProtosFactory.FlumeEvent getEvent() { if (eventBuilder_ == null) { return event_; @@ -3483,6 +4373,9 @@ public org.apache.flume.channel.file.proto.ProtosFactory.FlumeEvent getEvent() { return eventBuilder_.getMessage(); } } + /** + * required .FlumeEvent event = 1; + */ public Builder setEvent(org.apache.flume.channel.file.proto.ProtosFactory.FlumeEvent value) { if (eventBuilder_ == null) { if (value == null) { @@ -3496,6 +4389,9 @@ public Builder setEvent(org.apache.flume.channel.file.proto.ProtosFactory.FlumeE bitField0_ |= 0x00000001; return this; } + /** + * required .FlumeEvent event = 1; + */ public Builder setEvent( org.apache.flume.channel.file.proto.ProtosFactory.FlumeEvent.Builder builderForValue) { if (eventBuilder_ == null) { @@ -3507,6 +4403,9 @@ public Builder setEvent( bitField0_ |= 0x00000001; return this; } + /** + * required .FlumeEvent event = 1; + */ public Builder mergeEvent(org.apache.flume.channel.file.proto.ProtosFactory.FlumeEvent value) { if (eventBuilder_ == null) { if (((bitField0_ & 0x00000001) == 0x00000001) && @@ -3523,6 +4422,9 @@ public Builder mergeEvent(org.apache.flume.channel.file.proto.ProtosFactory.Flum bitField0_ |= 0x00000001; return this; } + /** + * required .FlumeEvent event = 1; + */ public Builder clearEvent() { if (eventBuilder_ == null) { event_ = org.apache.flume.channel.file.proto.ProtosFactory.FlumeEvent.getDefaultInstance(); @@ -3533,11 +4435,17 @@ public Builder clearEvent() { bitField0_ = (bitField0_ & ~0x00000001); return this; } + /** + * required .FlumeEvent event = 1; + */ public org.apache.flume.channel.file.proto.ProtosFactory.FlumeEvent.Builder getEventBuilder() { bitField0_ |= 0x00000001; onChanged(); return getEventFieldBuilder().getBuilder(); } + /** + * required .FlumeEvent event = 1; + */ public org.apache.flume.channel.file.proto.ProtosFactory.FlumeEventOrBuilder getEventOrBuilder() { if (eventBuilder_ != null) { return eventBuilder_.getMessageOrBuilder(); @@ -3545,6 +4453,9 @@ public org.apache.flume.channel.file.proto.ProtosFactory.FlumeEventOrBuilder get return event_; } } + /** + * required .FlumeEvent event = 1; + */ private com.google.protobuf.SingleFieldBuilder< org.apache.flume.channel.file.proto.ProtosFactory.FlumeEvent, org.apache.flume.channel.file.proto.ProtosFactory.FlumeEvent.Builder, org.apache.flume.channel.file.proto.ProtosFactory.FlumeEventOrBuilder> getEventFieldBuilder() { @@ -3558,99 +4469,207 @@ public org.apache.flume.channel.file.proto.ProtosFactory.FlumeEventOrBuilder get } return eventBuilder_; } - + // optional sfixed64 checksum = 2; private long checksum_ ; + /** + * optional sfixed64 checksum = 2; + */ public boolean hasChecksum() { return ((bitField0_ & 0x00000002) == 0x00000002); } + /** + * optional sfixed64 checksum = 2; + */ public long getChecksum() { return checksum_; } + /** + * optional sfixed64 checksum = 2; + */ public Builder setChecksum(long value) { bitField0_ |= 0x00000002; checksum_ = value; onChanged(); return this; } + /** + * optional sfixed64 checksum = 2; + */ public Builder clearChecksum() { bitField0_ = (bitField0_ & ~0x00000002); checksum_ = 0L; onChanged(); return this; } - + // @@protoc_insertion_point(builder_scope:Put) } - + static { defaultInstance = new Put(true); defaultInstance.initFields(); } - + // @@protoc_insertion_point(class_scope:Put) } - + public interface TakeOrBuilder extends com.google.protobuf.MessageOrBuilder { - + // required sfixed32 fileID = 1; + /** + * required sfixed32 fileID = 1; + */ boolean hasFileID(); + /** + * required sfixed32 fileID = 1; + */ int getFileID(); - + // required sfixed32 offset = 2; + /** + * required sfixed32 offset = 2; + */ boolean hasOffset(); + /** + * required sfixed32 offset = 2; + */ int getOffset(); } + /** + * Protobuf type {@code Take} + */ public static final class Take extends com.google.protobuf.GeneratedMessage implements TakeOrBuilder { // Use Take.newBuilder() to construct. - private Take(Builder builder) { + private Take(com.google.protobuf.GeneratedMessage.Builder builder) { super(builder); + this.unknownFields = builder.getUnknownFields(); } - private Take(boolean noInit) {} - + private Take(boolean noInit) { this.unknownFields = com.google.protobuf.UnknownFieldSet.getDefaultInstance(); } + private static final Take defaultInstance; public static Take getDefaultInstance() { return defaultInstance; } - + public Take getDefaultInstanceForType() { return defaultInstance; } - + + private final com.google.protobuf.UnknownFieldSet unknownFields; + @java.lang.Override + public final com.google.protobuf.UnknownFieldSet + getUnknownFields() { + return this.unknownFields; + } + private Take( + com.google.protobuf.CodedInputStream input, + com.google.protobuf.ExtensionRegistryLite extensionRegistry) + throws com.google.protobuf.InvalidProtocolBufferException { + initFields(); + int mutable_bitField0_ = 0; + com.google.protobuf.UnknownFieldSet.Builder unknownFields = + com.google.protobuf.UnknownFieldSet.newBuilder(); + try { + boolean done = false; + while (!done) { + int tag = input.readTag(); + switch (tag) { + case 0: + done = true; + break; + default: { + if (!parseUnknownField(input, unknownFields, + extensionRegistry, tag)) { + done = true; + } + break; + } + case 13: { + bitField0_ |= 0x00000001; + fileID_ = input.readSFixed32(); + break; + } + case 21: { + bitField0_ |= 0x00000002; + offset_ = input.readSFixed32(); + break; + } + } + } + } catch (com.google.protobuf.InvalidProtocolBufferException e) { + throw e.setUnfinishedMessage(this); + } catch (java.io.IOException e) { + throw new com.google.protobuf.InvalidProtocolBufferException( + e.getMessage()).setUnfinishedMessage(this); + } finally { + this.unknownFields = unknownFields.build(); + makeExtensionsImmutable(); + } + } public static final com.google.protobuf.Descriptors.Descriptor getDescriptor() { return org.apache.flume.channel.file.proto.ProtosFactory.internal_static_Take_descriptor; } - + protected com.google.protobuf.GeneratedMessage.FieldAccessorTable internalGetFieldAccessorTable() { - return org.apache.flume.channel.file.proto.ProtosFactory.internal_static_Take_fieldAccessorTable; + return org.apache.flume.channel.file.proto.ProtosFactory.internal_static_Take_fieldAccessorTable + .ensureFieldAccessorsInitialized( + org.apache.flume.channel.file.proto.ProtosFactory.Take.class, org.apache.flume.channel.file.proto.ProtosFactory.Take.Builder.class); + } + + public static com.google.protobuf.Parser PARSER = + new com.google.protobuf.AbstractParser() { + public Take parsePartialFrom( + com.google.protobuf.CodedInputStream input, + com.google.protobuf.ExtensionRegistryLite extensionRegistry) + throws com.google.protobuf.InvalidProtocolBufferException { + return new Take(input, extensionRegistry); + } + }; + + @java.lang.Override + public com.google.protobuf.Parser getParserForType() { + return PARSER; } - + private int bitField0_; // required sfixed32 fileID = 1; public static final int FILEID_FIELD_NUMBER = 1; private int fileID_; + /** + * required sfixed32 fileID = 1; + */ public boolean hasFileID() { return ((bitField0_ & 0x00000001) == 0x00000001); } + /** + * required sfixed32 fileID = 1; + */ public int getFileID() { return fileID_; } - + // required sfixed32 offset = 2; public static final int OFFSET_FIELD_NUMBER = 2; private int offset_; + /** + * required sfixed32 offset = 2; + */ public boolean hasOffset() { return ((bitField0_ & 0x00000002) == 0x00000002); } + /** + * required sfixed32 offset = 2; + */ public int getOffset() { return offset_; } - + private void initFields() { fileID_ = 0; offset_ = 0; @@ -3659,7 +4678,7 @@ private void initFields() { public final boolean isInitialized() { byte isInitialized = memoizedIsInitialized; if (isInitialized != -1) return isInitialized == 1; - + if (!hasFileID()) { memoizedIsInitialized = 0; return false; @@ -3671,7 +4690,7 @@ public final boolean isInitialized() { memoizedIsInitialized = 1; return true; } - + public void writeTo(com.google.protobuf.CodedOutputStream output) throws java.io.IOException { getSerializedSize(); @@ -3683,12 +4702,12 @@ public void writeTo(com.google.protobuf.CodedOutputStream output) } getUnknownFields().writeTo(output); } - + private int memoizedSerializedSize = -1; public int getSerializedSize() { int size = memoizedSerializedSize; if (size != -1) return size; - + size = 0; if (((bitField0_ & 0x00000001) == 0x00000001)) { size += com.google.protobuf.CodedOutputStream @@ -3702,94 +4721,83 @@ public int getSerializedSize() { memoizedSerializedSize = size; return size; } - + private static final long serialVersionUID = 0L; @java.lang.Override protected java.lang.Object writeReplace() throws java.io.ObjectStreamException { return super.writeReplace(); } - + public static org.apache.flume.channel.file.proto.ProtosFactory.Take parseFrom( com.google.protobuf.ByteString data) throws com.google.protobuf.InvalidProtocolBufferException { - return newBuilder().mergeFrom(data).buildParsed(); + return PARSER.parseFrom(data); } public static org.apache.flume.channel.file.proto.ProtosFactory.Take parseFrom( com.google.protobuf.ByteString data, com.google.protobuf.ExtensionRegistryLite extensionRegistry) throws com.google.protobuf.InvalidProtocolBufferException { - return newBuilder().mergeFrom(data, extensionRegistry) - .buildParsed(); + return PARSER.parseFrom(data, extensionRegistry); } public static org.apache.flume.channel.file.proto.ProtosFactory.Take parseFrom(byte[] data) throws com.google.protobuf.InvalidProtocolBufferException { - return newBuilder().mergeFrom(data).buildParsed(); + return PARSER.parseFrom(data); } public static org.apache.flume.channel.file.proto.ProtosFactory.Take parseFrom( byte[] data, com.google.protobuf.ExtensionRegistryLite extensionRegistry) throws com.google.protobuf.InvalidProtocolBufferException { - return newBuilder().mergeFrom(data, extensionRegistry) - .buildParsed(); + return PARSER.parseFrom(data, extensionRegistry); } public static org.apache.flume.channel.file.proto.ProtosFactory.Take parseFrom(java.io.InputStream input) throws java.io.IOException { - return newBuilder().mergeFrom(input).buildParsed(); + return PARSER.parseFrom(input); } public static org.apache.flume.channel.file.proto.ProtosFactory.Take parseFrom( java.io.InputStream input, com.google.protobuf.ExtensionRegistryLite extensionRegistry) throws java.io.IOException { - return newBuilder().mergeFrom(input, extensionRegistry) - .buildParsed(); + return PARSER.parseFrom(input, extensionRegistry); } public static org.apache.flume.channel.file.proto.ProtosFactory.Take parseDelimitedFrom(java.io.InputStream input) throws java.io.IOException { - Builder builder = newBuilder(); - if (builder.mergeDelimitedFrom(input)) { - return builder.buildParsed(); - } else { - return null; - } + return PARSER.parseDelimitedFrom(input); } public static org.apache.flume.channel.file.proto.ProtosFactory.Take parseDelimitedFrom( java.io.InputStream input, com.google.protobuf.ExtensionRegistryLite extensionRegistry) throws java.io.IOException { - Builder builder = newBuilder(); - if (builder.mergeDelimitedFrom(input, extensionRegistry)) { - return builder.buildParsed(); - } else { - return null; - } + return PARSER.parseDelimitedFrom(input, extensionRegistry); } public static org.apache.flume.channel.file.proto.ProtosFactory.Take parseFrom( com.google.protobuf.CodedInputStream input) throws java.io.IOException { - return newBuilder().mergeFrom(input).buildParsed(); + return PARSER.parseFrom(input); } public static org.apache.flume.channel.file.proto.ProtosFactory.Take parseFrom( com.google.protobuf.CodedInputStream input, com.google.protobuf.ExtensionRegistryLite extensionRegistry) throws java.io.IOException { - return newBuilder().mergeFrom(input, extensionRegistry) - .buildParsed(); + return PARSER.parseFrom(input, extensionRegistry); } - + public static Builder newBuilder() { return Builder.create(); } public Builder newBuilderForType() { return newBuilder(); } public static Builder newBuilder(org.apache.flume.channel.file.proto.ProtosFactory.Take prototype) { return newBuilder().mergeFrom(prototype); } public Builder toBuilder() { return newBuilder(this); } - + @java.lang.Override protected Builder newBuilderForType( com.google.protobuf.GeneratedMessage.BuilderParent parent) { Builder builder = new Builder(parent); return builder; } + /** + * Protobuf type {@code Take} + */ public static final class Builder extends com.google.protobuf.GeneratedMessage.Builder implements org.apache.flume.channel.file.proto.ProtosFactory.TakeOrBuilder { @@ -3797,18 +4805,21 @@ public static final class Builder extends getDescriptor() { return org.apache.flume.channel.file.proto.ProtosFactory.internal_static_Take_descriptor; } - + protected com.google.protobuf.GeneratedMessage.FieldAccessorTable internalGetFieldAccessorTable() { - return org.apache.flume.channel.file.proto.ProtosFactory.internal_static_Take_fieldAccessorTable; + return org.apache.flume.channel.file.proto.ProtosFactory.internal_static_Take_fieldAccessorTable + .ensureFieldAccessorsInitialized( + org.apache.flume.channel.file.proto.ProtosFactory.Take.class, org.apache.flume.channel.file.proto.ProtosFactory.Take.Builder.class); } - + // Construct using org.apache.flume.channel.file.proto.ProtosFactory.Take.newBuilder() private Builder() { maybeForceBuilderInitialization(); } - - private Builder(BuilderParent parent) { + + private Builder( + com.google.protobuf.GeneratedMessage.BuilderParent parent) { super(parent); maybeForceBuilderInitialization(); } @@ -3819,7 +4830,7 @@ private void maybeForceBuilderInitialization() { private static Builder create() { return new Builder(); } - + public Builder clear() { super.clear(); fileID_ = 0; @@ -3828,20 +4839,20 @@ public Builder clear() { bitField0_ = (bitField0_ & ~0x00000002); return this; } - + public Builder clone() { return create().mergeFrom(buildPartial()); } - + public com.google.protobuf.Descriptors.Descriptor getDescriptorForType() { - return org.apache.flume.channel.file.proto.ProtosFactory.Take.getDescriptor(); + return org.apache.flume.channel.file.proto.ProtosFactory.internal_static_Take_descriptor; } - + public org.apache.flume.channel.file.proto.ProtosFactory.Take getDefaultInstanceForType() { return org.apache.flume.channel.file.proto.ProtosFactory.Take.getDefaultInstance(); } - + public org.apache.flume.channel.file.proto.ProtosFactory.Take build() { org.apache.flume.channel.file.proto.ProtosFactory.Take result = buildPartial(); if (!result.isInitialized()) { @@ -3849,17 +4860,7 @@ public org.apache.flume.channel.file.proto.ProtosFactory.Take build() { } return result; } - - private org.apache.flume.channel.file.proto.ProtosFactory.Take buildParsed() - throws com.google.protobuf.InvalidProtocolBufferException { - org.apache.flume.channel.file.proto.ProtosFactory.Take result = buildPartial(); - if (!result.isInitialized()) { - throw newUninitializedMessageException( - result).asInvalidProtocolBufferException(); - } - return result; - } - + public org.apache.flume.channel.file.proto.ProtosFactory.Take buildPartial() { org.apache.flume.channel.file.proto.ProtosFactory.Take result = new org.apache.flume.channel.file.proto.ProtosFactory.Take(this); int from_bitField0_ = bitField0_; @@ -3876,7 +4877,7 @@ public org.apache.flume.channel.file.proto.ProtosFactory.Take buildPartial() { onBuilt(); return result; } - + public Builder mergeFrom(com.google.protobuf.Message other) { if (other instanceof org.apache.flume.channel.file.proto.ProtosFactory.Take) { return mergeFrom((org.apache.flume.channel.file.proto.ProtosFactory.Take)other); @@ -3885,7 +4886,7 @@ public Builder mergeFrom(com.google.protobuf.Message other) { return this; } } - + public Builder mergeFrom(org.apache.flume.channel.file.proto.ProtosFactory.Take other) { if (other == org.apache.flume.channel.file.proto.ProtosFactory.Take.getDefaultInstance()) return this; if (other.hasFileID()) { @@ -3897,7 +4898,7 @@ public Builder mergeFrom(org.apache.flume.channel.file.proto.ProtosFactory.Take this.mergeUnknownFields(other.getUnknownFields()); return this; } - + public final boolean isInitialized() { if (!hasFileID()) { @@ -3909,245 +4910,299 @@ public final boolean isInitialized() { } return true; } - + public Builder mergeFrom( com.google.protobuf.CodedInputStream input, com.google.protobuf.ExtensionRegistryLite extensionRegistry) throws java.io.IOException { - com.google.protobuf.UnknownFieldSet.Builder unknownFields = - com.google.protobuf.UnknownFieldSet.newBuilder( - this.getUnknownFields()); - while (true) { - int tag = input.readTag(); - switch (tag) { - case 0: - this.setUnknownFields(unknownFields.build()); - onChanged(); - return this; - default: { - if (!parseUnknownField(input, unknownFields, - extensionRegistry, tag)) { - this.setUnknownFields(unknownFields.build()); - onChanged(); - return this; - } - break; - } - case 13: { - bitField0_ |= 0x00000001; - fileID_ = input.readSFixed32(); - break; - } - case 21: { - bitField0_ |= 0x00000002; - offset_ = input.readSFixed32(); - break; - } + org.apache.flume.channel.file.proto.ProtosFactory.Take parsedMessage = null; + try { + parsedMessage = PARSER.parsePartialFrom(input, extensionRegistry); + } catch (com.google.protobuf.InvalidProtocolBufferException e) { + parsedMessage = (org.apache.flume.channel.file.proto.ProtosFactory.Take) e.getUnfinishedMessage(); + throw e; + } finally { + if (parsedMessage != null) { + mergeFrom(parsedMessage); } } + return this; } - private int bitField0_; - + // required sfixed32 fileID = 1; private int fileID_ ; + /** + * required sfixed32 fileID = 1; + */ public boolean hasFileID() { return ((bitField0_ & 0x00000001) == 0x00000001); } + /** + * required sfixed32 fileID = 1; + */ public int getFileID() { return fileID_; } + /** + * required sfixed32 fileID = 1; + */ public Builder setFileID(int value) { bitField0_ |= 0x00000001; fileID_ = value; onChanged(); return this; } + /** + * required sfixed32 fileID = 1; + */ public Builder clearFileID() { bitField0_ = (bitField0_ & ~0x00000001); fileID_ = 0; onChanged(); return this; } - + // required sfixed32 offset = 2; private int offset_ ; + /** + * required sfixed32 offset = 2; + */ public boolean hasOffset() { return ((bitField0_ & 0x00000002) == 0x00000002); } + /** + * required sfixed32 offset = 2; + */ public int getOffset() { return offset_; } + /** + * required sfixed32 offset = 2; + */ public Builder setOffset(int value) { bitField0_ |= 0x00000002; offset_ = value; onChanged(); return this; } + /** + * required sfixed32 offset = 2; + */ public Builder clearOffset() { bitField0_ = (bitField0_ & ~0x00000002); offset_ = 0; onChanged(); return this; } - + // @@protoc_insertion_point(builder_scope:Take) } - + static { defaultInstance = new Take(true); defaultInstance.initFields(); } - + // @@protoc_insertion_point(class_scope:Take) } - + public interface RollbackOrBuilder extends com.google.protobuf.MessageOrBuilder { } + /** + * Protobuf type {@code Rollback} + */ public static final class Rollback extends com.google.protobuf.GeneratedMessage implements RollbackOrBuilder { // Use Rollback.newBuilder() to construct. - private Rollback(Builder builder) { + private Rollback(com.google.protobuf.GeneratedMessage.Builder builder) { super(builder); + this.unknownFields = builder.getUnknownFields(); } - private Rollback(boolean noInit) {} - + private Rollback(boolean noInit) { this.unknownFields = com.google.protobuf.UnknownFieldSet.getDefaultInstance(); } + private static final Rollback defaultInstance; public static Rollback getDefaultInstance() { return defaultInstance; } - + public Rollback getDefaultInstanceForType() { return defaultInstance; } - + + private final com.google.protobuf.UnknownFieldSet unknownFields; + @java.lang.Override + public final com.google.protobuf.UnknownFieldSet + getUnknownFields() { + return this.unknownFields; + } + private Rollback( + com.google.protobuf.CodedInputStream input, + com.google.protobuf.ExtensionRegistryLite extensionRegistry) + throws com.google.protobuf.InvalidProtocolBufferException { + initFields(); + com.google.protobuf.UnknownFieldSet.Builder unknownFields = + com.google.protobuf.UnknownFieldSet.newBuilder(); + try { + boolean done = false; + while (!done) { + int tag = input.readTag(); + switch (tag) { + case 0: + done = true; + break; + default: { + if (!parseUnknownField(input, unknownFields, + extensionRegistry, tag)) { + done = true; + } + break; + } + } + } + } catch (com.google.protobuf.InvalidProtocolBufferException e) { + throw e.setUnfinishedMessage(this); + } catch (java.io.IOException e) { + throw new com.google.protobuf.InvalidProtocolBufferException( + e.getMessage()).setUnfinishedMessage(this); + } finally { + this.unknownFields = unknownFields.build(); + makeExtensionsImmutable(); + } + } public static final com.google.protobuf.Descriptors.Descriptor getDescriptor() { return org.apache.flume.channel.file.proto.ProtosFactory.internal_static_Rollback_descriptor; } - + protected com.google.protobuf.GeneratedMessage.FieldAccessorTable internalGetFieldAccessorTable() { - return org.apache.flume.channel.file.proto.ProtosFactory.internal_static_Rollback_fieldAccessorTable; + return org.apache.flume.channel.file.proto.ProtosFactory.internal_static_Rollback_fieldAccessorTable + .ensureFieldAccessorsInitialized( + org.apache.flume.channel.file.proto.ProtosFactory.Rollback.class, org.apache.flume.channel.file.proto.ProtosFactory.Rollback.Builder.class); + } + + public static com.google.protobuf.Parser PARSER = + new com.google.protobuf.AbstractParser() { + public Rollback parsePartialFrom( + com.google.protobuf.CodedInputStream input, + com.google.protobuf.ExtensionRegistryLite extensionRegistry) + throws com.google.protobuf.InvalidProtocolBufferException { + return new Rollback(input, extensionRegistry); + } + }; + + @java.lang.Override + public com.google.protobuf.Parser getParserForType() { + return PARSER; } - + private void initFields() { } private byte memoizedIsInitialized = -1; public final boolean isInitialized() { byte isInitialized = memoizedIsInitialized; if (isInitialized != -1) return isInitialized == 1; - + memoizedIsInitialized = 1; return true; } - + public void writeTo(com.google.protobuf.CodedOutputStream output) throws java.io.IOException { getSerializedSize(); getUnknownFields().writeTo(output); } - + private int memoizedSerializedSize = -1; public int getSerializedSize() { int size = memoizedSerializedSize; if (size != -1) return size; - + size = 0; size += getUnknownFields().getSerializedSize(); memoizedSerializedSize = size; return size; } - + private static final long serialVersionUID = 0L; @java.lang.Override protected java.lang.Object writeReplace() throws java.io.ObjectStreamException { return super.writeReplace(); } - + public static org.apache.flume.channel.file.proto.ProtosFactory.Rollback parseFrom( com.google.protobuf.ByteString data) throws com.google.protobuf.InvalidProtocolBufferException { - return newBuilder().mergeFrom(data).buildParsed(); + return PARSER.parseFrom(data); } public static org.apache.flume.channel.file.proto.ProtosFactory.Rollback parseFrom( com.google.protobuf.ByteString data, com.google.protobuf.ExtensionRegistryLite extensionRegistry) throws com.google.protobuf.InvalidProtocolBufferException { - return newBuilder().mergeFrom(data, extensionRegistry) - .buildParsed(); + return PARSER.parseFrom(data, extensionRegistry); } public static org.apache.flume.channel.file.proto.ProtosFactory.Rollback parseFrom(byte[] data) throws com.google.protobuf.InvalidProtocolBufferException { - return newBuilder().mergeFrom(data).buildParsed(); + return PARSER.parseFrom(data); } public static org.apache.flume.channel.file.proto.ProtosFactory.Rollback parseFrom( byte[] data, com.google.protobuf.ExtensionRegistryLite extensionRegistry) throws com.google.protobuf.InvalidProtocolBufferException { - return newBuilder().mergeFrom(data, extensionRegistry) - .buildParsed(); + return PARSER.parseFrom(data, extensionRegistry); } public static org.apache.flume.channel.file.proto.ProtosFactory.Rollback parseFrom(java.io.InputStream input) throws java.io.IOException { - return newBuilder().mergeFrom(input).buildParsed(); + return PARSER.parseFrom(input); } public static org.apache.flume.channel.file.proto.ProtosFactory.Rollback parseFrom( java.io.InputStream input, com.google.protobuf.ExtensionRegistryLite extensionRegistry) throws java.io.IOException { - return newBuilder().mergeFrom(input, extensionRegistry) - .buildParsed(); + return PARSER.parseFrom(input, extensionRegistry); } public static org.apache.flume.channel.file.proto.ProtosFactory.Rollback parseDelimitedFrom(java.io.InputStream input) throws java.io.IOException { - Builder builder = newBuilder(); - if (builder.mergeDelimitedFrom(input)) { - return builder.buildParsed(); - } else { - return null; - } + return PARSER.parseDelimitedFrom(input); } public static org.apache.flume.channel.file.proto.ProtosFactory.Rollback parseDelimitedFrom( java.io.InputStream input, com.google.protobuf.ExtensionRegistryLite extensionRegistry) throws java.io.IOException { - Builder builder = newBuilder(); - if (builder.mergeDelimitedFrom(input, extensionRegistry)) { - return builder.buildParsed(); - } else { - return null; - } + return PARSER.parseDelimitedFrom(input, extensionRegistry); } public static org.apache.flume.channel.file.proto.ProtosFactory.Rollback parseFrom( com.google.protobuf.CodedInputStream input) throws java.io.IOException { - return newBuilder().mergeFrom(input).buildParsed(); + return PARSER.parseFrom(input); } public static org.apache.flume.channel.file.proto.ProtosFactory.Rollback parseFrom( com.google.protobuf.CodedInputStream input, com.google.protobuf.ExtensionRegistryLite extensionRegistry) throws java.io.IOException { - return newBuilder().mergeFrom(input, extensionRegistry) - .buildParsed(); + return PARSER.parseFrom(input, extensionRegistry); } - + public static Builder newBuilder() { return Builder.create(); } public Builder newBuilderForType() { return newBuilder(); } public static Builder newBuilder(org.apache.flume.channel.file.proto.ProtosFactory.Rollback prototype) { return newBuilder().mergeFrom(prototype); } public Builder toBuilder() { return newBuilder(this); } - + @java.lang.Override protected Builder newBuilderForType( com.google.protobuf.GeneratedMessage.BuilderParent parent) { Builder builder = new Builder(parent); return builder; } + /** + * Protobuf type {@code Rollback} + */ public static final class Builder extends com.google.protobuf.GeneratedMessage.Builder implements org.apache.flume.channel.file.proto.ProtosFactory.RollbackOrBuilder { @@ -4155,18 +5210,21 @@ public static final class Builder extends getDescriptor() { return org.apache.flume.channel.file.proto.ProtosFactory.internal_static_Rollback_descriptor; } - + protected com.google.protobuf.GeneratedMessage.FieldAccessorTable internalGetFieldAccessorTable() { - return org.apache.flume.channel.file.proto.ProtosFactory.internal_static_Rollback_fieldAccessorTable; + return org.apache.flume.channel.file.proto.ProtosFactory.internal_static_Rollback_fieldAccessorTable + .ensureFieldAccessorsInitialized( + org.apache.flume.channel.file.proto.ProtosFactory.Rollback.class, org.apache.flume.channel.file.proto.ProtosFactory.Rollback.Builder.class); } - + // Construct using org.apache.flume.channel.file.proto.ProtosFactory.Rollback.newBuilder() private Builder() { maybeForceBuilderInitialization(); } - - private Builder(BuilderParent parent) { + + private Builder( + com.google.protobuf.GeneratedMessage.BuilderParent parent) { super(parent); maybeForceBuilderInitialization(); } @@ -4177,25 +5235,25 @@ private void maybeForceBuilderInitialization() { private static Builder create() { return new Builder(); } - + public Builder clear() { super.clear(); return this; } - + public Builder clone() { return create().mergeFrom(buildPartial()); } - + public com.google.protobuf.Descriptors.Descriptor getDescriptorForType() { - return org.apache.flume.channel.file.proto.ProtosFactory.Rollback.getDescriptor(); + return org.apache.flume.channel.file.proto.ProtosFactory.internal_static_Rollback_descriptor; } - + public org.apache.flume.channel.file.proto.ProtosFactory.Rollback getDefaultInstanceForType() { return org.apache.flume.channel.file.proto.ProtosFactory.Rollback.getDefaultInstance(); } - + public org.apache.flume.channel.file.proto.ProtosFactory.Rollback build() { org.apache.flume.channel.file.proto.ProtosFactory.Rollback result = buildPartial(); if (!result.isInitialized()) { @@ -4203,23 +5261,13 @@ public org.apache.flume.channel.file.proto.ProtosFactory.Rollback build() { } return result; } - - private org.apache.flume.channel.file.proto.ProtosFactory.Rollback buildParsed() - throws com.google.protobuf.InvalidProtocolBufferException { - org.apache.flume.channel.file.proto.ProtosFactory.Rollback result = buildPartial(); - if (!result.isInitialized()) { - throw newUninitializedMessageException( - result).asInvalidProtocolBufferException(); - } - return result; - } - + public org.apache.flume.channel.file.proto.ProtosFactory.Rollback buildPartial() { org.apache.flume.channel.file.proto.ProtosFactory.Rollback result = new org.apache.flume.channel.file.proto.ProtosFactory.Rollback(this); onBuilt(); return result; } - + public Builder mergeFrom(com.google.protobuf.Message other) { if (other instanceof org.apache.flume.channel.file.proto.ProtosFactory.Rollback) { return mergeFrom((org.apache.flume.channel.file.proto.ProtosFactory.Rollback)other); @@ -4228,102 +5276,171 @@ public Builder mergeFrom(com.google.protobuf.Message other) { return this; } } - + public Builder mergeFrom(org.apache.flume.channel.file.proto.ProtosFactory.Rollback other) { if (other == org.apache.flume.channel.file.proto.ProtosFactory.Rollback.getDefaultInstance()) return this; this.mergeUnknownFields(other.getUnknownFields()); return this; } - + public final boolean isInitialized() { return true; } - + public Builder mergeFrom( com.google.protobuf.CodedInputStream input, com.google.protobuf.ExtensionRegistryLite extensionRegistry) throws java.io.IOException { - com.google.protobuf.UnknownFieldSet.Builder unknownFields = - com.google.protobuf.UnknownFieldSet.newBuilder( - this.getUnknownFields()); - while (true) { - int tag = input.readTag(); - switch (tag) { - case 0: - this.setUnknownFields(unknownFields.build()); - onChanged(); - return this; - default: { - if (!parseUnknownField(input, unknownFields, - extensionRegistry, tag)) { - this.setUnknownFields(unknownFields.build()); - onChanged(); - return this; - } - break; - } + org.apache.flume.channel.file.proto.ProtosFactory.Rollback parsedMessage = null; + try { + parsedMessage = PARSER.parsePartialFrom(input, extensionRegistry); + } catch (com.google.protobuf.InvalidProtocolBufferException e) { + parsedMessage = (org.apache.flume.channel.file.proto.ProtosFactory.Rollback) e.getUnfinishedMessage(); + throw e; + } finally { + if (parsedMessage != null) { + mergeFrom(parsedMessage); } } + return this; } - - + // @@protoc_insertion_point(builder_scope:Rollback) } - + static { defaultInstance = new Rollback(true); defaultInstance.initFields(); } - + // @@protoc_insertion_point(class_scope:Rollback) } - + public interface CommitOrBuilder extends com.google.protobuf.MessageOrBuilder { - + // required sfixed32 type = 1; + /** + * required sfixed32 type = 1; + */ boolean hasType(); + /** + * required sfixed32 type = 1; + */ int getType(); } + /** + * Protobuf type {@code Commit} + */ public static final class Commit extends com.google.protobuf.GeneratedMessage implements CommitOrBuilder { // Use Commit.newBuilder() to construct. - private Commit(Builder builder) { + private Commit(com.google.protobuf.GeneratedMessage.Builder builder) { super(builder); + this.unknownFields = builder.getUnknownFields(); } - private Commit(boolean noInit) {} - + private Commit(boolean noInit) { this.unknownFields = com.google.protobuf.UnknownFieldSet.getDefaultInstance(); } + private static final Commit defaultInstance; public static Commit getDefaultInstance() { return defaultInstance; } - + public Commit getDefaultInstanceForType() { return defaultInstance; } - + + private final com.google.protobuf.UnknownFieldSet unknownFields; + @java.lang.Override + public final com.google.protobuf.UnknownFieldSet + getUnknownFields() { + return this.unknownFields; + } + private Commit( + com.google.protobuf.CodedInputStream input, + com.google.protobuf.ExtensionRegistryLite extensionRegistry) + throws com.google.protobuf.InvalidProtocolBufferException { + initFields(); + int mutable_bitField0_ = 0; + com.google.protobuf.UnknownFieldSet.Builder unknownFields = + com.google.protobuf.UnknownFieldSet.newBuilder(); + try { + boolean done = false; + while (!done) { + int tag = input.readTag(); + switch (tag) { + case 0: + done = true; + break; + default: { + if (!parseUnknownField(input, unknownFields, + extensionRegistry, tag)) { + done = true; + } + break; + } + case 13: { + bitField0_ |= 0x00000001; + type_ = input.readSFixed32(); + break; + } + } + } + } catch (com.google.protobuf.InvalidProtocolBufferException e) { + throw e.setUnfinishedMessage(this); + } catch (java.io.IOException e) { + throw new com.google.protobuf.InvalidProtocolBufferException( + e.getMessage()).setUnfinishedMessage(this); + } finally { + this.unknownFields = unknownFields.build(); + makeExtensionsImmutable(); + } + } public static final com.google.protobuf.Descriptors.Descriptor getDescriptor() { return org.apache.flume.channel.file.proto.ProtosFactory.internal_static_Commit_descriptor; } - + protected com.google.protobuf.GeneratedMessage.FieldAccessorTable internalGetFieldAccessorTable() { - return org.apache.flume.channel.file.proto.ProtosFactory.internal_static_Commit_fieldAccessorTable; + return org.apache.flume.channel.file.proto.ProtosFactory.internal_static_Commit_fieldAccessorTable + .ensureFieldAccessorsInitialized( + org.apache.flume.channel.file.proto.ProtosFactory.Commit.class, org.apache.flume.channel.file.proto.ProtosFactory.Commit.Builder.class); + } + + public static com.google.protobuf.Parser PARSER = + new com.google.protobuf.AbstractParser() { + public Commit parsePartialFrom( + com.google.protobuf.CodedInputStream input, + com.google.protobuf.ExtensionRegistryLite extensionRegistry) + throws com.google.protobuf.InvalidProtocolBufferException { + return new Commit(input, extensionRegistry); + } + }; + + @java.lang.Override + public com.google.protobuf.Parser getParserForType() { + return PARSER; } - + private int bitField0_; // required sfixed32 type = 1; public static final int TYPE_FIELD_NUMBER = 1; private int type_; + /** + * required sfixed32 type = 1; + */ public boolean hasType() { return ((bitField0_ & 0x00000001) == 0x00000001); } + /** + * required sfixed32 type = 1; + */ public int getType() { return type_; } - + private void initFields() { type_ = 0; } @@ -4331,7 +5448,7 @@ private void initFields() { public final boolean isInitialized() { byte isInitialized = memoizedIsInitialized; if (isInitialized != -1) return isInitialized == 1; - + if (!hasType()) { memoizedIsInitialized = 0; return false; @@ -4339,7 +5456,7 @@ public final boolean isInitialized() { memoizedIsInitialized = 1; return true; } - + public void writeTo(com.google.protobuf.CodedOutputStream output) throws java.io.IOException { getSerializedSize(); @@ -4348,12 +5465,12 @@ public void writeTo(com.google.protobuf.CodedOutputStream output) } getUnknownFields().writeTo(output); } - + private int memoizedSerializedSize = -1; public int getSerializedSize() { int size = memoizedSerializedSize; if (size != -1) return size; - + size = 0; if (((bitField0_ & 0x00000001) == 0x00000001)) { size += com.google.protobuf.CodedOutputStream @@ -4363,94 +5480,83 @@ public int getSerializedSize() { memoizedSerializedSize = size; return size; } - + private static final long serialVersionUID = 0L; @java.lang.Override protected java.lang.Object writeReplace() throws java.io.ObjectStreamException { return super.writeReplace(); } - + public static org.apache.flume.channel.file.proto.ProtosFactory.Commit parseFrom( com.google.protobuf.ByteString data) throws com.google.protobuf.InvalidProtocolBufferException { - return newBuilder().mergeFrom(data).buildParsed(); + return PARSER.parseFrom(data); } public static org.apache.flume.channel.file.proto.ProtosFactory.Commit parseFrom( com.google.protobuf.ByteString data, com.google.protobuf.ExtensionRegistryLite extensionRegistry) throws com.google.protobuf.InvalidProtocolBufferException { - return newBuilder().mergeFrom(data, extensionRegistry) - .buildParsed(); + return PARSER.parseFrom(data, extensionRegistry); } public static org.apache.flume.channel.file.proto.ProtosFactory.Commit parseFrom(byte[] data) throws com.google.protobuf.InvalidProtocolBufferException { - return newBuilder().mergeFrom(data).buildParsed(); + return PARSER.parseFrom(data); } public static org.apache.flume.channel.file.proto.ProtosFactory.Commit parseFrom( byte[] data, com.google.protobuf.ExtensionRegistryLite extensionRegistry) throws com.google.protobuf.InvalidProtocolBufferException { - return newBuilder().mergeFrom(data, extensionRegistry) - .buildParsed(); + return PARSER.parseFrom(data, extensionRegistry); } public static org.apache.flume.channel.file.proto.ProtosFactory.Commit parseFrom(java.io.InputStream input) throws java.io.IOException { - return newBuilder().mergeFrom(input).buildParsed(); + return PARSER.parseFrom(input); } public static org.apache.flume.channel.file.proto.ProtosFactory.Commit parseFrom( java.io.InputStream input, com.google.protobuf.ExtensionRegistryLite extensionRegistry) throws java.io.IOException { - return newBuilder().mergeFrom(input, extensionRegistry) - .buildParsed(); + return PARSER.parseFrom(input, extensionRegistry); } public static org.apache.flume.channel.file.proto.ProtosFactory.Commit parseDelimitedFrom(java.io.InputStream input) throws java.io.IOException { - Builder builder = newBuilder(); - if (builder.mergeDelimitedFrom(input)) { - return builder.buildParsed(); - } else { - return null; - } + return PARSER.parseDelimitedFrom(input); } public static org.apache.flume.channel.file.proto.ProtosFactory.Commit parseDelimitedFrom( java.io.InputStream input, com.google.protobuf.ExtensionRegistryLite extensionRegistry) throws java.io.IOException { - Builder builder = newBuilder(); - if (builder.mergeDelimitedFrom(input, extensionRegistry)) { - return builder.buildParsed(); - } else { - return null; - } + return PARSER.parseDelimitedFrom(input, extensionRegistry); } public static org.apache.flume.channel.file.proto.ProtosFactory.Commit parseFrom( com.google.protobuf.CodedInputStream input) throws java.io.IOException { - return newBuilder().mergeFrom(input).buildParsed(); + return PARSER.parseFrom(input); } public static org.apache.flume.channel.file.proto.ProtosFactory.Commit parseFrom( com.google.protobuf.CodedInputStream input, com.google.protobuf.ExtensionRegistryLite extensionRegistry) throws java.io.IOException { - return newBuilder().mergeFrom(input, extensionRegistry) - .buildParsed(); + return PARSER.parseFrom(input, extensionRegistry); } - + public static Builder newBuilder() { return Builder.create(); } public Builder newBuilderForType() { return newBuilder(); } public static Builder newBuilder(org.apache.flume.channel.file.proto.ProtosFactory.Commit prototype) { return newBuilder().mergeFrom(prototype); } public Builder toBuilder() { return newBuilder(this); } - + @java.lang.Override protected Builder newBuilderForType( com.google.protobuf.GeneratedMessage.BuilderParent parent) { Builder builder = new Builder(parent); return builder; } + /** + * Protobuf type {@code Commit} + */ public static final class Builder extends com.google.protobuf.GeneratedMessage.Builder implements org.apache.flume.channel.file.proto.ProtosFactory.CommitOrBuilder { @@ -4458,18 +5564,21 @@ public static final class Builder extends getDescriptor() { return org.apache.flume.channel.file.proto.ProtosFactory.internal_static_Commit_descriptor; } - + protected com.google.protobuf.GeneratedMessage.FieldAccessorTable internalGetFieldAccessorTable() { - return org.apache.flume.channel.file.proto.ProtosFactory.internal_static_Commit_fieldAccessorTable; + return org.apache.flume.channel.file.proto.ProtosFactory.internal_static_Commit_fieldAccessorTable + .ensureFieldAccessorsInitialized( + org.apache.flume.channel.file.proto.ProtosFactory.Commit.class, org.apache.flume.channel.file.proto.ProtosFactory.Commit.Builder.class); } - + // Construct using org.apache.flume.channel.file.proto.ProtosFactory.Commit.newBuilder() private Builder() { maybeForceBuilderInitialization(); } - - private Builder(BuilderParent parent) { + + private Builder( + com.google.protobuf.GeneratedMessage.BuilderParent parent) { super(parent); maybeForceBuilderInitialization(); } @@ -4480,27 +5589,27 @@ private void maybeForceBuilderInitialization() { private static Builder create() { return new Builder(); } - + public Builder clear() { super.clear(); type_ = 0; bitField0_ = (bitField0_ & ~0x00000001); return this; } - + public Builder clone() { return create().mergeFrom(buildPartial()); } - + public com.google.protobuf.Descriptors.Descriptor getDescriptorForType() { - return org.apache.flume.channel.file.proto.ProtosFactory.Commit.getDescriptor(); + return org.apache.flume.channel.file.proto.ProtosFactory.internal_static_Commit_descriptor; } - + public org.apache.flume.channel.file.proto.ProtosFactory.Commit getDefaultInstanceForType() { return org.apache.flume.channel.file.proto.ProtosFactory.Commit.getDefaultInstance(); } - + public org.apache.flume.channel.file.proto.ProtosFactory.Commit build() { org.apache.flume.channel.file.proto.ProtosFactory.Commit result = buildPartial(); if (!result.isInitialized()) { @@ -4508,17 +5617,7 @@ public org.apache.flume.channel.file.proto.ProtosFactory.Commit build() { } return result; } - - private org.apache.flume.channel.file.proto.ProtosFactory.Commit buildParsed() - throws com.google.protobuf.InvalidProtocolBufferException { - org.apache.flume.channel.file.proto.ProtosFactory.Commit result = buildPartial(); - if (!result.isInitialized()) { - throw newUninitializedMessageException( - result).asInvalidProtocolBufferException(); - } - return result; - } - + public org.apache.flume.channel.file.proto.ProtosFactory.Commit buildPartial() { org.apache.flume.channel.file.proto.ProtosFactory.Commit result = new org.apache.flume.channel.file.proto.ProtosFactory.Commit(this); int from_bitField0_ = bitField0_; @@ -4531,7 +5630,7 @@ public org.apache.flume.channel.file.proto.ProtosFactory.Commit buildPartial() { onBuilt(); return result; } - + public Builder mergeFrom(com.google.protobuf.Message other) { if (other instanceof org.apache.flume.channel.file.proto.ProtosFactory.Commit) { return mergeFrom((org.apache.flume.channel.file.proto.ProtosFactory.Commit)other); @@ -4540,7 +5639,7 @@ public Builder mergeFrom(com.google.protobuf.Message other) { return this; } } - + public Builder mergeFrom(org.apache.flume.channel.file.proto.ProtosFactory.Commit other) { if (other == org.apache.flume.channel.file.proto.ProtosFactory.Commit.getDefaultInstance()) return this; if (other.hasType()) { @@ -4549,7 +5648,7 @@ public Builder mergeFrom(org.apache.flume.channel.file.proto.ProtosFactory.Commi this.mergeUnknownFields(other.getUnknownFields()); return this; } - + public final boolean isInitialized() { if (!hasType()) { @@ -4557,219 +5656,266 @@ public final boolean isInitialized() { } return true; } - + public Builder mergeFrom( com.google.protobuf.CodedInputStream input, com.google.protobuf.ExtensionRegistryLite extensionRegistry) throws java.io.IOException { - com.google.protobuf.UnknownFieldSet.Builder unknownFields = - com.google.protobuf.UnknownFieldSet.newBuilder( - this.getUnknownFields()); - while (true) { - int tag = input.readTag(); - switch (tag) { - case 0: - this.setUnknownFields(unknownFields.build()); - onChanged(); - return this; - default: { - if (!parseUnknownField(input, unknownFields, - extensionRegistry, tag)) { - this.setUnknownFields(unknownFields.build()); - onChanged(); - return this; - } - break; - } - case 13: { - bitField0_ |= 0x00000001; - type_ = input.readSFixed32(); - break; - } + org.apache.flume.channel.file.proto.ProtosFactory.Commit parsedMessage = null; + try { + parsedMessage = PARSER.parsePartialFrom(input, extensionRegistry); + } catch (com.google.protobuf.InvalidProtocolBufferException e) { + parsedMessage = (org.apache.flume.channel.file.proto.ProtosFactory.Commit) e.getUnfinishedMessage(); + throw e; + } finally { + if (parsedMessage != null) { + mergeFrom(parsedMessage); } } + return this; } - private int bitField0_; - + // required sfixed32 type = 1; private int type_ ; + /** + * required sfixed32 type = 1; + */ public boolean hasType() { return ((bitField0_ & 0x00000001) == 0x00000001); } + /** + * required sfixed32 type = 1; + */ public int getType() { return type_; } + /** + * required sfixed32 type = 1; + */ public Builder setType(int value) { bitField0_ |= 0x00000001; type_ = value; onChanged(); return this; } + /** + * required sfixed32 type = 1; + */ public Builder clearType() { bitField0_ = (bitField0_ & ~0x00000001); type_ = 0; onChanged(); return this; } - + // @@protoc_insertion_point(builder_scope:Commit) } - + static { defaultInstance = new Commit(true); defaultInstance.initFields(); } - + // @@protoc_insertion_point(class_scope:Commit) } - + public interface TransactionEventFooterOrBuilder extends com.google.protobuf.MessageOrBuilder { } + /** + * Protobuf type {@code TransactionEventFooter} + */ public static final class TransactionEventFooter extends com.google.protobuf.GeneratedMessage implements TransactionEventFooterOrBuilder { // Use TransactionEventFooter.newBuilder() to construct. - private TransactionEventFooter(Builder builder) { + private TransactionEventFooter(com.google.protobuf.GeneratedMessage.Builder builder) { super(builder); + this.unknownFields = builder.getUnknownFields(); } - private TransactionEventFooter(boolean noInit) {} - + private TransactionEventFooter(boolean noInit) { this.unknownFields = com.google.protobuf.UnknownFieldSet.getDefaultInstance(); } + private static final TransactionEventFooter defaultInstance; public static TransactionEventFooter getDefaultInstance() { return defaultInstance; } - + public TransactionEventFooter getDefaultInstanceForType() { return defaultInstance; } - + + private final com.google.protobuf.UnknownFieldSet unknownFields; + @java.lang.Override + public final com.google.protobuf.UnknownFieldSet + getUnknownFields() { + return this.unknownFields; + } + private TransactionEventFooter( + com.google.protobuf.CodedInputStream input, + com.google.protobuf.ExtensionRegistryLite extensionRegistry) + throws com.google.protobuf.InvalidProtocolBufferException { + initFields(); + com.google.protobuf.UnknownFieldSet.Builder unknownFields = + com.google.protobuf.UnknownFieldSet.newBuilder(); + try { + boolean done = false; + while (!done) { + int tag = input.readTag(); + switch (tag) { + case 0: + done = true; + break; + default: { + if (!parseUnknownField(input, unknownFields, + extensionRegistry, tag)) { + done = true; + } + break; + } + } + } + } catch (com.google.protobuf.InvalidProtocolBufferException e) { + throw e.setUnfinishedMessage(this); + } catch (java.io.IOException e) { + throw new com.google.protobuf.InvalidProtocolBufferException( + e.getMessage()).setUnfinishedMessage(this); + } finally { + this.unknownFields = unknownFields.build(); + makeExtensionsImmutable(); + } + } public static final com.google.protobuf.Descriptors.Descriptor getDescriptor() { return org.apache.flume.channel.file.proto.ProtosFactory.internal_static_TransactionEventFooter_descriptor; } - + protected com.google.protobuf.GeneratedMessage.FieldAccessorTable internalGetFieldAccessorTable() { - return org.apache.flume.channel.file.proto.ProtosFactory.internal_static_TransactionEventFooter_fieldAccessorTable; + return org.apache.flume.channel.file.proto.ProtosFactory.internal_static_TransactionEventFooter_fieldAccessorTable + .ensureFieldAccessorsInitialized( + org.apache.flume.channel.file.proto.ProtosFactory.TransactionEventFooter.class, org.apache.flume.channel.file.proto.ProtosFactory.TransactionEventFooter.Builder.class); + } + + public static com.google.protobuf.Parser PARSER = + new com.google.protobuf.AbstractParser() { + public TransactionEventFooter parsePartialFrom( + com.google.protobuf.CodedInputStream input, + com.google.protobuf.ExtensionRegistryLite extensionRegistry) + throws com.google.protobuf.InvalidProtocolBufferException { + return new TransactionEventFooter(input, extensionRegistry); + } + }; + + @java.lang.Override + public com.google.protobuf.Parser getParserForType() { + return PARSER; } - + private void initFields() { } private byte memoizedIsInitialized = -1; public final boolean isInitialized() { byte isInitialized = memoizedIsInitialized; if (isInitialized != -1) return isInitialized == 1; - + memoizedIsInitialized = 1; return true; } - + public void writeTo(com.google.protobuf.CodedOutputStream output) throws java.io.IOException { getSerializedSize(); getUnknownFields().writeTo(output); } - + private int memoizedSerializedSize = -1; public int getSerializedSize() { int size = memoizedSerializedSize; if (size != -1) return size; - + size = 0; size += getUnknownFields().getSerializedSize(); memoizedSerializedSize = size; return size; } - + private static final long serialVersionUID = 0L; @java.lang.Override protected java.lang.Object writeReplace() throws java.io.ObjectStreamException { return super.writeReplace(); } - + public static org.apache.flume.channel.file.proto.ProtosFactory.TransactionEventFooter parseFrom( com.google.protobuf.ByteString data) throws com.google.protobuf.InvalidProtocolBufferException { - return newBuilder().mergeFrom(data).buildParsed(); + return PARSER.parseFrom(data); } public static org.apache.flume.channel.file.proto.ProtosFactory.TransactionEventFooter parseFrom( com.google.protobuf.ByteString data, com.google.protobuf.ExtensionRegistryLite extensionRegistry) throws com.google.protobuf.InvalidProtocolBufferException { - return newBuilder().mergeFrom(data, extensionRegistry) - .buildParsed(); + return PARSER.parseFrom(data, extensionRegistry); } public static org.apache.flume.channel.file.proto.ProtosFactory.TransactionEventFooter parseFrom(byte[] data) throws com.google.protobuf.InvalidProtocolBufferException { - return newBuilder().mergeFrom(data).buildParsed(); + return PARSER.parseFrom(data); } public static org.apache.flume.channel.file.proto.ProtosFactory.TransactionEventFooter parseFrom( byte[] data, com.google.protobuf.ExtensionRegistryLite extensionRegistry) throws com.google.protobuf.InvalidProtocolBufferException { - return newBuilder().mergeFrom(data, extensionRegistry) - .buildParsed(); + return PARSER.parseFrom(data, extensionRegistry); } public static org.apache.flume.channel.file.proto.ProtosFactory.TransactionEventFooter parseFrom(java.io.InputStream input) throws java.io.IOException { - return newBuilder().mergeFrom(input).buildParsed(); + return PARSER.parseFrom(input); } public static org.apache.flume.channel.file.proto.ProtosFactory.TransactionEventFooter parseFrom( java.io.InputStream input, com.google.protobuf.ExtensionRegistryLite extensionRegistry) throws java.io.IOException { - return newBuilder().mergeFrom(input, extensionRegistry) - .buildParsed(); + return PARSER.parseFrom(input, extensionRegistry); } public static org.apache.flume.channel.file.proto.ProtosFactory.TransactionEventFooter parseDelimitedFrom(java.io.InputStream input) throws java.io.IOException { - Builder builder = newBuilder(); - if (builder.mergeDelimitedFrom(input)) { - return builder.buildParsed(); - } else { - return null; - } + return PARSER.parseDelimitedFrom(input); } public static org.apache.flume.channel.file.proto.ProtosFactory.TransactionEventFooter parseDelimitedFrom( java.io.InputStream input, com.google.protobuf.ExtensionRegistryLite extensionRegistry) throws java.io.IOException { - Builder builder = newBuilder(); - if (builder.mergeDelimitedFrom(input, extensionRegistry)) { - return builder.buildParsed(); - } else { - return null; - } + return PARSER.parseDelimitedFrom(input, extensionRegistry); } public static org.apache.flume.channel.file.proto.ProtosFactory.TransactionEventFooter parseFrom( com.google.protobuf.CodedInputStream input) throws java.io.IOException { - return newBuilder().mergeFrom(input).buildParsed(); + return PARSER.parseFrom(input); } public static org.apache.flume.channel.file.proto.ProtosFactory.TransactionEventFooter parseFrom( com.google.protobuf.CodedInputStream input, com.google.protobuf.ExtensionRegistryLite extensionRegistry) throws java.io.IOException { - return newBuilder().mergeFrom(input, extensionRegistry) - .buildParsed(); + return PARSER.parseFrom(input, extensionRegistry); } - + public static Builder newBuilder() { return Builder.create(); } public Builder newBuilderForType() { return newBuilder(); } public static Builder newBuilder(org.apache.flume.channel.file.proto.ProtosFactory.TransactionEventFooter prototype) { return newBuilder().mergeFrom(prototype); } public Builder toBuilder() { return newBuilder(this); } - + @java.lang.Override protected Builder newBuilderForType( com.google.protobuf.GeneratedMessage.BuilderParent parent) { Builder builder = new Builder(parent); return builder; } + /** + * Protobuf type {@code TransactionEventFooter} + */ public static final class Builder extends com.google.protobuf.GeneratedMessage.Builder implements org.apache.flume.channel.file.proto.ProtosFactory.TransactionEventFooterOrBuilder { @@ -4777,18 +5923,21 @@ public static final class Builder extends getDescriptor() { return org.apache.flume.channel.file.proto.ProtosFactory.internal_static_TransactionEventFooter_descriptor; } - + protected com.google.protobuf.GeneratedMessage.FieldAccessorTable internalGetFieldAccessorTable() { - return org.apache.flume.channel.file.proto.ProtosFactory.internal_static_TransactionEventFooter_fieldAccessorTable; + return org.apache.flume.channel.file.proto.ProtosFactory.internal_static_TransactionEventFooter_fieldAccessorTable + .ensureFieldAccessorsInitialized( + org.apache.flume.channel.file.proto.ProtosFactory.TransactionEventFooter.class, org.apache.flume.channel.file.proto.ProtosFactory.TransactionEventFooter.Builder.class); } - + // Construct using org.apache.flume.channel.file.proto.ProtosFactory.TransactionEventFooter.newBuilder() private Builder() { maybeForceBuilderInitialization(); } - - private Builder(BuilderParent parent) { + + private Builder( + com.google.protobuf.GeneratedMessage.BuilderParent parent) { super(parent); maybeForceBuilderInitialization(); } @@ -4799,49 +5948,39 @@ private void maybeForceBuilderInitialization() { private static Builder create() { return new Builder(); } - + public Builder clear() { super.clear(); return this; } - + public Builder clone() { return create().mergeFrom(buildPartial()); } - + public com.google.protobuf.Descriptors.Descriptor getDescriptorForType() { - return org.apache.flume.channel.file.proto.ProtosFactory.TransactionEventFooter.getDescriptor(); + return org.apache.flume.channel.file.proto.ProtosFactory.internal_static_TransactionEventFooter_descriptor; } - + public org.apache.flume.channel.file.proto.ProtosFactory.TransactionEventFooter getDefaultInstanceForType() { return org.apache.flume.channel.file.proto.ProtosFactory.TransactionEventFooter.getDefaultInstance(); } - - public org.apache.flume.channel.file.proto.ProtosFactory.TransactionEventFooter build() { - org.apache.flume.channel.file.proto.ProtosFactory.TransactionEventFooter result = buildPartial(); - if (!result.isInitialized()) { - throw newUninitializedMessageException(result); - } - return result; - } - - private org.apache.flume.channel.file.proto.ProtosFactory.TransactionEventFooter buildParsed() - throws com.google.protobuf.InvalidProtocolBufferException { + + public org.apache.flume.channel.file.proto.ProtosFactory.TransactionEventFooter build() { org.apache.flume.channel.file.proto.ProtosFactory.TransactionEventFooter result = buildPartial(); if (!result.isInitialized()) { - throw newUninitializedMessageException( - result).asInvalidProtocolBufferException(); + throw newUninitializedMessageException(result); } return result; } - + public org.apache.flume.channel.file.proto.ProtosFactory.TransactionEventFooter buildPartial() { org.apache.flume.channel.file.proto.ProtosFactory.TransactionEventFooter result = new org.apache.flume.channel.file.proto.ProtosFactory.TransactionEventFooter(this); onBuilt(); return result; } - + public Builder mergeFrom(com.google.protobuf.Message other) { if (other instanceof org.apache.flume.channel.file.proto.ProtosFactory.TransactionEventFooter) { return mergeFrom((org.apache.flume.channel.file.proto.ProtosFactory.TransactionEventFooter)other); @@ -4850,133 +5989,243 @@ public Builder mergeFrom(com.google.protobuf.Message other) { return this; } } - + public Builder mergeFrom(org.apache.flume.channel.file.proto.ProtosFactory.TransactionEventFooter other) { if (other == org.apache.flume.channel.file.proto.ProtosFactory.TransactionEventFooter.getDefaultInstance()) return this; this.mergeUnknownFields(other.getUnknownFields()); return this; } - + public final boolean isInitialized() { return true; } - + public Builder mergeFrom( com.google.protobuf.CodedInputStream input, com.google.protobuf.ExtensionRegistryLite extensionRegistry) throws java.io.IOException { - com.google.protobuf.UnknownFieldSet.Builder unknownFields = - com.google.protobuf.UnknownFieldSet.newBuilder( - this.getUnknownFields()); - while (true) { - int tag = input.readTag(); - switch (tag) { - case 0: - this.setUnknownFields(unknownFields.build()); - onChanged(); - return this; - default: { - if (!parseUnknownField(input, unknownFields, - extensionRegistry, tag)) { - this.setUnknownFields(unknownFields.build()); - onChanged(); - return this; - } - break; - } + org.apache.flume.channel.file.proto.ProtosFactory.TransactionEventFooter parsedMessage = null; + try { + parsedMessage = PARSER.parsePartialFrom(input, extensionRegistry); + } catch (com.google.protobuf.InvalidProtocolBufferException e) { + parsedMessage = (org.apache.flume.channel.file.proto.ProtosFactory.TransactionEventFooter) e.getUnfinishedMessage(); + throw e; + } finally { + if (parsedMessage != null) { + mergeFrom(parsedMessage); } } + return this; } - - + // @@protoc_insertion_point(builder_scope:TransactionEventFooter) } - + static { defaultInstance = new TransactionEventFooter(true); defaultInstance.initFields(); } - + // @@protoc_insertion_point(class_scope:TransactionEventFooter) } - + public interface FlumeEventOrBuilder extends com.google.protobuf.MessageOrBuilder { - + // repeated .FlumeEventHeader headers = 1; + /** + * repeated .FlumeEventHeader headers = 1; + */ java.util.List getHeadersList(); + /** + * repeated .FlumeEventHeader headers = 1; + */ org.apache.flume.channel.file.proto.ProtosFactory.FlumeEventHeader getHeaders(int index); + /** + * repeated .FlumeEventHeader headers = 1; + */ int getHeadersCount(); + /** + * repeated .FlumeEventHeader headers = 1; + */ java.util.List getHeadersOrBuilderList(); + /** + * repeated .FlumeEventHeader headers = 1; + */ org.apache.flume.channel.file.proto.ProtosFactory.FlumeEventHeaderOrBuilder getHeadersOrBuilder( int index); - + // required bytes body = 2; + /** + * required bytes body = 2; + */ boolean hasBody(); + /** + * required bytes body = 2; + */ com.google.protobuf.ByteString getBody(); } + /** + * Protobuf type {@code FlumeEvent} + */ public static final class FlumeEvent extends com.google.protobuf.GeneratedMessage implements FlumeEventOrBuilder { // Use FlumeEvent.newBuilder() to construct. - private FlumeEvent(Builder builder) { + private FlumeEvent(com.google.protobuf.GeneratedMessage.Builder builder) { super(builder); + this.unknownFields = builder.getUnknownFields(); } - private FlumeEvent(boolean noInit) {} - + private FlumeEvent(boolean noInit) { this.unknownFields = com.google.protobuf.UnknownFieldSet.getDefaultInstance(); } + private static final FlumeEvent defaultInstance; public static FlumeEvent getDefaultInstance() { return defaultInstance; } - + public FlumeEvent getDefaultInstanceForType() { return defaultInstance; } - + + private final com.google.protobuf.UnknownFieldSet unknownFields; + @java.lang.Override + public final com.google.protobuf.UnknownFieldSet + getUnknownFields() { + return this.unknownFields; + } + private FlumeEvent( + com.google.protobuf.CodedInputStream input, + com.google.protobuf.ExtensionRegistryLite extensionRegistry) + throws com.google.protobuf.InvalidProtocolBufferException { + initFields(); + int mutable_bitField0_ = 0; + com.google.protobuf.UnknownFieldSet.Builder unknownFields = + com.google.protobuf.UnknownFieldSet.newBuilder(); + try { + boolean done = false; + while (!done) { + int tag = input.readTag(); + switch (tag) { + case 0: + done = true; + break; + default: { + if (!parseUnknownField(input, unknownFields, + extensionRegistry, tag)) { + done = true; + } + break; + } + case 10: { + if (!((mutable_bitField0_ & 0x00000001) == 0x00000001)) { + headers_ = new java.util.ArrayList(); + mutable_bitField0_ |= 0x00000001; + } + headers_.add(input.readMessage(org.apache.flume.channel.file.proto.ProtosFactory.FlumeEventHeader.PARSER, extensionRegistry)); + break; + } + case 18: { + bitField0_ |= 0x00000001; + body_ = input.readBytes(); + break; + } + } + } + } catch (com.google.protobuf.InvalidProtocolBufferException e) { + throw e.setUnfinishedMessage(this); + } catch (java.io.IOException e) { + throw new com.google.protobuf.InvalidProtocolBufferException( + e.getMessage()).setUnfinishedMessage(this); + } finally { + if (((mutable_bitField0_ & 0x00000001) == 0x00000001)) { + headers_ = java.util.Collections.unmodifiableList(headers_); + } + this.unknownFields = unknownFields.build(); + makeExtensionsImmutable(); + } + } public static final com.google.protobuf.Descriptors.Descriptor getDescriptor() { return org.apache.flume.channel.file.proto.ProtosFactory.internal_static_FlumeEvent_descriptor; } - + protected com.google.protobuf.GeneratedMessage.FieldAccessorTable internalGetFieldAccessorTable() { - return org.apache.flume.channel.file.proto.ProtosFactory.internal_static_FlumeEvent_fieldAccessorTable; + return org.apache.flume.channel.file.proto.ProtosFactory.internal_static_FlumeEvent_fieldAccessorTable + .ensureFieldAccessorsInitialized( + org.apache.flume.channel.file.proto.ProtosFactory.FlumeEvent.class, org.apache.flume.channel.file.proto.ProtosFactory.FlumeEvent.Builder.class); + } + + public static com.google.protobuf.Parser PARSER = + new com.google.protobuf.AbstractParser() { + public FlumeEvent parsePartialFrom( + com.google.protobuf.CodedInputStream input, + com.google.protobuf.ExtensionRegistryLite extensionRegistry) + throws com.google.protobuf.InvalidProtocolBufferException { + return new FlumeEvent(input, extensionRegistry); + } + }; + + @java.lang.Override + public com.google.protobuf.Parser getParserForType() { + return PARSER; } - + private int bitField0_; // repeated .FlumeEventHeader headers = 1; public static final int HEADERS_FIELD_NUMBER = 1; private java.util.List headers_; + /** + * repeated .FlumeEventHeader headers = 1; + */ public java.util.List getHeadersList() { return headers_; } + /** + * repeated .FlumeEventHeader headers = 1; + */ public java.util.List getHeadersOrBuilderList() { return headers_; } + /** + * repeated .FlumeEventHeader headers = 1; + */ public int getHeadersCount() { return headers_.size(); } + /** + * repeated .FlumeEventHeader headers = 1; + */ public org.apache.flume.channel.file.proto.ProtosFactory.FlumeEventHeader getHeaders(int index) { return headers_.get(index); } + /** + * repeated .FlumeEventHeader headers = 1; + */ public org.apache.flume.channel.file.proto.ProtosFactory.FlumeEventHeaderOrBuilder getHeadersOrBuilder( int index) { return headers_.get(index); } - + // required bytes body = 2; public static final int BODY_FIELD_NUMBER = 2; private com.google.protobuf.ByteString body_; + /** + * required bytes body = 2; + */ public boolean hasBody() { return ((bitField0_ & 0x00000001) == 0x00000001); } + /** + * required bytes body = 2; + */ public com.google.protobuf.ByteString getBody() { return body_; } - + private void initFields() { headers_ = java.util.Collections.emptyList(); body_ = com.google.protobuf.ByteString.EMPTY; @@ -4985,7 +6234,7 @@ private void initFields() { public final boolean isInitialized() { byte isInitialized = memoizedIsInitialized; if (isInitialized != -1) return isInitialized == 1; - + if (!hasBody()) { memoizedIsInitialized = 0; return false; @@ -4999,7 +6248,7 @@ public final boolean isInitialized() { memoizedIsInitialized = 1; return true; } - + public void writeTo(com.google.protobuf.CodedOutputStream output) throws java.io.IOException { getSerializedSize(); @@ -5011,12 +6260,12 @@ public void writeTo(com.google.protobuf.CodedOutputStream output) } getUnknownFields().writeTo(output); } - + private int memoizedSerializedSize = -1; public int getSerializedSize() { int size = memoizedSerializedSize; if (size != -1) return size; - + size = 0; for (int i = 0; i < headers_.size(); i++) { size += com.google.protobuf.CodedOutputStream @@ -5030,94 +6279,83 @@ public int getSerializedSize() { memoizedSerializedSize = size; return size; } - + private static final long serialVersionUID = 0L; @java.lang.Override protected java.lang.Object writeReplace() throws java.io.ObjectStreamException { return super.writeReplace(); } - + public static org.apache.flume.channel.file.proto.ProtosFactory.FlumeEvent parseFrom( com.google.protobuf.ByteString data) throws com.google.protobuf.InvalidProtocolBufferException { - return newBuilder().mergeFrom(data).buildParsed(); + return PARSER.parseFrom(data); } public static org.apache.flume.channel.file.proto.ProtosFactory.FlumeEvent parseFrom( com.google.protobuf.ByteString data, com.google.protobuf.ExtensionRegistryLite extensionRegistry) throws com.google.protobuf.InvalidProtocolBufferException { - return newBuilder().mergeFrom(data, extensionRegistry) - .buildParsed(); + return PARSER.parseFrom(data, extensionRegistry); } public static org.apache.flume.channel.file.proto.ProtosFactory.FlumeEvent parseFrom(byte[] data) throws com.google.protobuf.InvalidProtocolBufferException { - return newBuilder().mergeFrom(data).buildParsed(); + return PARSER.parseFrom(data); } public static org.apache.flume.channel.file.proto.ProtosFactory.FlumeEvent parseFrom( byte[] data, com.google.protobuf.ExtensionRegistryLite extensionRegistry) throws com.google.protobuf.InvalidProtocolBufferException { - return newBuilder().mergeFrom(data, extensionRegistry) - .buildParsed(); + return PARSER.parseFrom(data, extensionRegistry); } public static org.apache.flume.channel.file.proto.ProtosFactory.FlumeEvent parseFrom(java.io.InputStream input) throws java.io.IOException { - return newBuilder().mergeFrom(input).buildParsed(); + return PARSER.parseFrom(input); } public static org.apache.flume.channel.file.proto.ProtosFactory.FlumeEvent parseFrom( java.io.InputStream input, com.google.protobuf.ExtensionRegistryLite extensionRegistry) throws java.io.IOException { - return newBuilder().mergeFrom(input, extensionRegistry) - .buildParsed(); + return PARSER.parseFrom(input, extensionRegistry); } public static org.apache.flume.channel.file.proto.ProtosFactory.FlumeEvent parseDelimitedFrom(java.io.InputStream input) throws java.io.IOException { - Builder builder = newBuilder(); - if (builder.mergeDelimitedFrom(input)) { - return builder.buildParsed(); - } else { - return null; - } + return PARSER.parseDelimitedFrom(input); } public static org.apache.flume.channel.file.proto.ProtosFactory.FlumeEvent parseDelimitedFrom( java.io.InputStream input, com.google.protobuf.ExtensionRegistryLite extensionRegistry) throws java.io.IOException { - Builder builder = newBuilder(); - if (builder.mergeDelimitedFrom(input, extensionRegistry)) { - return builder.buildParsed(); - } else { - return null; - } + return PARSER.parseDelimitedFrom(input, extensionRegistry); } public static org.apache.flume.channel.file.proto.ProtosFactory.FlumeEvent parseFrom( com.google.protobuf.CodedInputStream input) throws java.io.IOException { - return newBuilder().mergeFrom(input).buildParsed(); + return PARSER.parseFrom(input); } public static org.apache.flume.channel.file.proto.ProtosFactory.FlumeEvent parseFrom( com.google.protobuf.CodedInputStream input, com.google.protobuf.ExtensionRegistryLite extensionRegistry) throws java.io.IOException { - return newBuilder().mergeFrom(input, extensionRegistry) - .buildParsed(); + return PARSER.parseFrom(input, extensionRegistry); } - + public static Builder newBuilder() { return Builder.create(); } public Builder newBuilderForType() { return newBuilder(); } public static Builder newBuilder(org.apache.flume.channel.file.proto.ProtosFactory.FlumeEvent prototype) { return newBuilder().mergeFrom(prototype); } public Builder toBuilder() { return newBuilder(this); } - + @java.lang.Override protected Builder newBuilderForType( com.google.protobuf.GeneratedMessage.BuilderParent parent) { Builder builder = new Builder(parent); return builder; } + /** + * Protobuf type {@code FlumeEvent} + */ public static final class Builder extends com.google.protobuf.GeneratedMessage.Builder implements org.apache.flume.channel.file.proto.ProtosFactory.FlumeEventOrBuilder { @@ -5125,18 +6363,21 @@ public static final class Builder extends getDescriptor() { return org.apache.flume.channel.file.proto.ProtosFactory.internal_static_FlumeEvent_descriptor; } - + protected com.google.protobuf.GeneratedMessage.FieldAccessorTable internalGetFieldAccessorTable() { - return org.apache.flume.channel.file.proto.ProtosFactory.internal_static_FlumeEvent_fieldAccessorTable; + return org.apache.flume.channel.file.proto.ProtosFactory.internal_static_FlumeEvent_fieldAccessorTable + .ensureFieldAccessorsInitialized( + org.apache.flume.channel.file.proto.ProtosFactory.FlumeEvent.class, org.apache.flume.channel.file.proto.ProtosFactory.FlumeEvent.Builder.class); } - + // Construct using org.apache.flume.channel.file.proto.ProtosFactory.FlumeEvent.newBuilder() private Builder() { maybeForceBuilderInitialization(); } - - private Builder(BuilderParent parent) { + + private Builder( + com.google.protobuf.GeneratedMessage.BuilderParent parent) { super(parent); maybeForceBuilderInitialization(); } @@ -5148,7 +6389,7 @@ private void maybeForceBuilderInitialization() { private static Builder create() { return new Builder(); } - + public Builder clear() { super.clear(); if (headersBuilder_ == null) { @@ -5161,20 +6402,20 @@ public Builder clear() { bitField0_ = (bitField0_ & ~0x00000002); return this; } - + public Builder clone() { return create().mergeFrom(buildPartial()); } - + public com.google.protobuf.Descriptors.Descriptor getDescriptorForType() { - return org.apache.flume.channel.file.proto.ProtosFactory.FlumeEvent.getDescriptor(); + return org.apache.flume.channel.file.proto.ProtosFactory.internal_static_FlumeEvent_descriptor; } - + public org.apache.flume.channel.file.proto.ProtosFactory.FlumeEvent getDefaultInstanceForType() { return org.apache.flume.channel.file.proto.ProtosFactory.FlumeEvent.getDefaultInstance(); } - + public org.apache.flume.channel.file.proto.ProtosFactory.FlumeEvent build() { org.apache.flume.channel.file.proto.ProtosFactory.FlumeEvent result = buildPartial(); if (!result.isInitialized()) { @@ -5182,17 +6423,7 @@ public org.apache.flume.channel.file.proto.ProtosFactory.FlumeEvent build() { } return result; } - - private org.apache.flume.channel.file.proto.ProtosFactory.FlumeEvent buildParsed() - throws com.google.protobuf.InvalidProtocolBufferException { - org.apache.flume.channel.file.proto.ProtosFactory.FlumeEvent result = buildPartial(); - if (!result.isInitialized()) { - throw newUninitializedMessageException( - result).asInvalidProtocolBufferException(); - } - return result; - } - + public org.apache.flume.channel.file.proto.ProtosFactory.FlumeEvent buildPartial() { org.apache.flume.channel.file.proto.ProtosFactory.FlumeEvent result = new org.apache.flume.channel.file.proto.ProtosFactory.FlumeEvent(this); int from_bitField0_ = bitField0_; @@ -5214,7 +6445,7 @@ public org.apache.flume.channel.file.proto.ProtosFactory.FlumeEvent buildPartial onBuilt(); return result; } - + public Builder mergeFrom(com.google.protobuf.Message other) { if (other instanceof org.apache.flume.channel.file.proto.ProtosFactory.FlumeEvent) { return mergeFrom((org.apache.flume.channel.file.proto.ProtosFactory.FlumeEvent)other); @@ -5223,7 +6454,7 @@ public Builder mergeFrom(com.google.protobuf.Message other) { return this; } } - + public Builder mergeFrom(org.apache.flume.channel.file.proto.ProtosFactory.FlumeEvent other) { if (other == org.apache.flume.channel.file.proto.ProtosFactory.FlumeEvent.getDefaultInstance()) return this; if (headersBuilder_ == null) { @@ -5258,7 +6489,7 @@ public Builder mergeFrom(org.apache.flume.channel.file.proto.ProtosFactory.Flume this.mergeUnknownFields(other.getUnknownFields()); return this; } - + public final boolean isInitialized() { if (!hasBody()) { @@ -5272,47 +6503,26 @@ public final boolean isInitialized() { } return true; } - + public Builder mergeFrom( com.google.protobuf.CodedInputStream input, com.google.protobuf.ExtensionRegistryLite extensionRegistry) throws java.io.IOException { - com.google.protobuf.UnknownFieldSet.Builder unknownFields = - com.google.protobuf.UnknownFieldSet.newBuilder( - this.getUnknownFields()); - while (true) { - int tag = input.readTag(); - switch (tag) { - case 0: - this.setUnknownFields(unknownFields.build()); - onChanged(); - return this; - default: { - if (!parseUnknownField(input, unknownFields, - extensionRegistry, tag)) { - this.setUnknownFields(unknownFields.build()); - onChanged(); - return this; - } - break; - } - case 10: { - org.apache.flume.channel.file.proto.ProtosFactory.FlumeEventHeader.Builder subBuilder = org.apache.flume.channel.file.proto.ProtosFactory.FlumeEventHeader.newBuilder(); - input.readMessage(subBuilder, extensionRegistry); - addHeaders(subBuilder.buildPartial()); - break; - } - case 18: { - bitField0_ |= 0x00000002; - body_ = input.readBytes(); - break; - } + org.apache.flume.channel.file.proto.ProtosFactory.FlumeEvent parsedMessage = null; + try { + parsedMessage = PARSER.parsePartialFrom(input, extensionRegistry); + } catch (com.google.protobuf.InvalidProtocolBufferException e) { + parsedMessage = (org.apache.flume.channel.file.proto.ProtosFactory.FlumeEvent) e.getUnfinishedMessage(); + throw e; + } finally { + if (parsedMessage != null) { + mergeFrom(parsedMessage); } } + return this; } - private int bitField0_; - + // repeated .FlumeEventHeader headers = 1; private java.util.List headers_ = java.util.Collections.emptyList(); @@ -5322,10 +6532,13 @@ private void ensureHeadersIsMutable() { bitField0_ |= 0x00000001; } } - + private com.google.protobuf.RepeatedFieldBuilder< org.apache.flume.channel.file.proto.ProtosFactory.FlumeEventHeader, org.apache.flume.channel.file.proto.ProtosFactory.FlumeEventHeader.Builder, org.apache.flume.channel.file.proto.ProtosFactory.FlumeEventHeaderOrBuilder> headersBuilder_; - + + /** + * repeated .FlumeEventHeader headers = 1; + */ public java.util.List getHeadersList() { if (headersBuilder_ == null) { return java.util.Collections.unmodifiableList(headers_); @@ -5333,6 +6546,9 @@ public java.util.Listrepeated .FlumeEventHeader headers = 1; + */ public int getHeadersCount() { if (headersBuilder_ == null) { return headers_.size(); @@ -5340,6 +6556,9 @@ public int getHeadersCount() { return headersBuilder_.getCount(); } } + /** + * repeated .FlumeEventHeader headers = 1; + */ public org.apache.flume.channel.file.proto.ProtosFactory.FlumeEventHeader getHeaders(int index) { if (headersBuilder_ == null) { return headers_.get(index); @@ -5347,6 +6566,9 @@ public org.apache.flume.channel.file.proto.ProtosFactory.FlumeEventHeader getHea return headersBuilder_.getMessage(index); } } + /** + * repeated .FlumeEventHeader headers = 1; + */ public Builder setHeaders( int index, org.apache.flume.channel.file.proto.ProtosFactory.FlumeEventHeader value) { if (headersBuilder_ == null) { @@ -5361,6 +6583,9 @@ public Builder setHeaders( } return this; } + /** + * repeated .FlumeEventHeader headers = 1; + */ public Builder setHeaders( int index, org.apache.flume.channel.file.proto.ProtosFactory.FlumeEventHeader.Builder builderForValue) { if (headersBuilder_ == null) { @@ -5372,6 +6597,9 @@ public Builder setHeaders( } return this; } + /** + * repeated .FlumeEventHeader headers = 1; + */ public Builder addHeaders(org.apache.flume.channel.file.proto.ProtosFactory.FlumeEventHeader value) { if (headersBuilder_ == null) { if (value == null) { @@ -5385,6 +6613,9 @@ public Builder addHeaders(org.apache.flume.channel.file.proto.ProtosFactory.Flum } return this; } + /** + * repeated .FlumeEventHeader headers = 1; + */ public Builder addHeaders( int index, org.apache.flume.channel.file.proto.ProtosFactory.FlumeEventHeader value) { if (headersBuilder_ == null) { @@ -5399,6 +6630,9 @@ public Builder addHeaders( } return this; } + /** + * repeated .FlumeEventHeader headers = 1; + */ public Builder addHeaders( org.apache.flume.channel.file.proto.ProtosFactory.FlumeEventHeader.Builder builderForValue) { if (headersBuilder_ == null) { @@ -5410,6 +6644,9 @@ public Builder addHeaders( } return this; } + /** + * repeated .FlumeEventHeader headers = 1; + */ public Builder addHeaders( int index, org.apache.flume.channel.file.proto.ProtosFactory.FlumeEventHeader.Builder builderForValue) { if (headersBuilder_ == null) { @@ -5421,6 +6658,9 @@ public Builder addHeaders( } return this; } + /** + * repeated .FlumeEventHeader headers = 1; + */ public Builder addAllHeaders( java.lang.Iterable values) { if (headersBuilder_ == null) { @@ -5432,6 +6672,9 @@ public Builder addAllHeaders( } return this; } + /** + * repeated .FlumeEventHeader headers = 1; + */ public Builder clearHeaders() { if (headersBuilder_ == null) { headers_ = java.util.Collections.emptyList(); @@ -5442,6 +6685,9 @@ public Builder clearHeaders() { } return this; } + /** + * repeated .FlumeEventHeader headers = 1; + */ public Builder removeHeaders(int index) { if (headersBuilder_ == null) { ensureHeadersIsMutable(); @@ -5452,10 +6698,16 @@ public Builder removeHeaders(int index) { } return this; } + /** + * repeated .FlumeEventHeader headers = 1; + */ public org.apache.flume.channel.file.proto.ProtosFactory.FlumeEventHeader.Builder getHeadersBuilder( int index) { return getHeadersFieldBuilder().getBuilder(index); } + /** + * repeated .FlumeEventHeader headers = 1; + */ public org.apache.flume.channel.file.proto.ProtosFactory.FlumeEventHeaderOrBuilder getHeadersOrBuilder( int index) { if (headersBuilder_ == null) { @@ -5463,6 +6715,9 @@ public org.apache.flume.channel.file.proto.ProtosFactory.FlumeEventHeaderOrBuild return headersBuilder_.getMessageOrBuilder(index); } } + /** + * repeated .FlumeEventHeader headers = 1; + */ public java.util.List getHeadersOrBuilderList() { if (headersBuilder_ != null) { @@ -5471,15 +6726,24 @@ public org.apache.flume.channel.file.proto.ProtosFactory.FlumeEventHeaderOrBuild return java.util.Collections.unmodifiableList(headers_); } } + /** + * repeated .FlumeEventHeader headers = 1; + */ public org.apache.flume.channel.file.proto.ProtosFactory.FlumeEventHeader.Builder addHeadersBuilder() { return getHeadersFieldBuilder().addBuilder( org.apache.flume.channel.file.proto.ProtosFactory.FlumeEventHeader.getDefaultInstance()); } + /** + * repeated .FlumeEventHeader headers = 1; + */ public org.apache.flume.channel.file.proto.ProtosFactory.FlumeEventHeader.Builder addHeadersBuilder( int index) { return getHeadersFieldBuilder().addBuilder( index, org.apache.flume.channel.file.proto.ProtosFactory.FlumeEventHeader.getDefaultInstance()); } + /** + * repeated .FlumeEventHeader headers = 1; + */ public java.util.List getHeadersBuilderList() { return getHeadersFieldBuilder().getBuilderList(); @@ -5498,15 +6762,24 @@ public org.apache.flume.channel.file.proto.ProtosFactory.FlumeEventHeader.Builde } return headersBuilder_; } - + // required bytes body = 2; private com.google.protobuf.ByteString body_ = com.google.protobuf.ByteString.EMPTY; + /** + * required bytes body = 2; + */ public boolean hasBody() { return ((bitField0_ & 0x00000002) == 0x00000002); } + /** + * required bytes body = 2; + */ public com.google.protobuf.ByteString getBody() { return body_; } + /** + * required bytes body = 2; + */ public Builder setBody(com.google.protobuf.ByteString value) { if (value == null) { throw new NullPointerException(); @@ -5516,128 +6789,247 @@ public Builder setBody(com.google.protobuf.ByteString value) { onChanged(); return this; } + /** + * required bytes body = 2; + */ public Builder clearBody() { bitField0_ = (bitField0_ & ~0x00000002); body_ = getDefaultInstance().getBody(); onChanged(); return this; } - + // @@protoc_insertion_point(builder_scope:FlumeEvent) } - + static { defaultInstance = new FlumeEvent(true); defaultInstance.initFields(); } - + // @@protoc_insertion_point(class_scope:FlumeEvent) } - + public interface FlumeEventHeaderOrBuilder extends com.google.protobuf.MessageOrBuilder { - + // required string key = 1; + /** + * required string key = 1; + */ boolean hasKey(); - String getKey(); - + /** + * required string key = 1; + */ + java.lang.String getKey(); + /** + * required string key = 1; + */ + com.google.protobuf.ByteString + getKeyBytes(); + // required string value = 2; + /** + * required string value = 2; + */ boolean hasValue(); - String getValue(); + /** + * required string value = 2; + */ + java.lang.String getValue(); + /** + * required string value = 2; + */ + com.google.protobuf.ByteString + getValueBytes(); } + /** + * Protobuf type {@code FlumeEventHeader} + */ public static final class FlumeEventHeader extends com.google.protobuf.GeneratedMessage implements FlumeEventHeaderOrBuilder { // Use FlumeEventHeader.newBuilder() to construct. - private FlumeEventHeader(Builder builder) { + private FlumeEventHeader(com.google.protobuf.GeneratedMessage.Builder builder) { super(builder); + this.unknownFields = builder.getUnknownFields(); } - private FlumeEventHeader(boolean noInit) {} - + private FlumeEventHeader(boolean noInit) { this.unknownFields = com.google.protobuf.UnknownFieldSet.getDefaultInstance(); } + private static final FlumeEventHeader defaultInstance; public static FlumeEventHeader getDefaultInstance() { return defaultInstance; } - + public FlumeEventHeader getDefaultInstanceForType() { return defaultInstance; } - + + private final com.google.protobuf.UnknownFieldSet unknownFields; + @java.lang.Override + public final com.google.protobuf.UnknownFieldSet + getUnknownFields() { + return this.unknownFields; + } + private FlumeEventHeader( + com.google.protobuf.CodedInputStream input, + com.google.protobuf.ExtensionRegistryLite extensionRegistry) + throws com.google.protobuf.InvalidProtocolBufferException { + initFields(); + int mutable_bitField0_ = 0; + com.google.protobuf.UnknownFieldSet.Builder unknownFields = + com.google.protobuf.UnknownFieldSet.newBuilder(); + try { + boolean done = false; + while (!done) { + int tag = input.readTag(); + switch (tag) { + case 0: + done = true; + break; + default: { + if (!parseUnknownField(input, unknownFields, + extensionRegistry, tag)) { + done = true; + } + break; + } + case 10: { + bitField0_ |= 0x00000001; + key_ = input.readBytes(); + break; + } + case 18: { + bitField0_ |= 0x00000002; + value_ = input.readBytes(); + break; + } + } + } + } catch (com.google.protobuf.InvalidProtocolBufferException e) { + throw e.setUnfinishedMessage(this); + } catch (java.io.IOException e) { + throw new com.google.protobuf.InvalidProtocolBufferException( + e.getMessage()).setUnfinishedMessage(this); + } finally { + this.unknownFields = unknownFields.build(); + makeExtensionsImmutable(); + } + } public static final com.google.protobuf.Descriptors.Descriptor getDescriptor() { return org.apache.flume.channel.file.proto.ProtosFactory.internal_static_FlumeEventHeader_descriptor; } - + protected com.google.protobuf.GeneratedMessage.FieldAccessorTable internalGetFieldAccessorTable() { - return org.apache.flume.channel.file.proto.ProtosFactory.internal_static_FlumeEventHeader_fieldAccessorTable; + return org.apache.flume.channel.file.proto.ProtosFactory.internal_static_FlumeEventHeader_fieldAccessorTable + .ensureFieldAccessorsInitialized( + org.apache.flume.channel.file.proto.ProtosFactory.FlumeEventHeader.class, org.apache.flume.channel.file.proto.ProtosFactory.FlumeEventHeader.Builder.class); + } + + public static com.google.protobuf.Parser PARSER = + new com.google.protobuf.AbstractParser() { + public FlumeEventHeader parsePartialFrom( + com.google.protobuf.CodedInputStream input, + com.google.protobuf.ExtensionRegistryLite extensionRegistry) + throws com.google.protobuf.InvalidProtocolBufferException { + return new FlumeEventHeader(input, extensionRegistry); + } + }; + + @java.lang.Override + public com.google.protobuf.Parser getParserForType() { + return PARSER; } - + private int bitField0_; // required string key = 1; public static final int KEY_FIELD_NUMBER = 1; private java.lang.Object key_; + /** + * required string key = 1; + */ public boolean hasKey() { return ((bitField0_ & 0x00000001) == 0x00000001); } - public String getKey() { + /** + * required string key = 1; + */ + public java.lang.String getKey() { java.lang.Object ref = key_; - if (ref instanceof String) { - return (String) ref; + if (ref instanceof java.lang.String) { + return (java.lang.String) ref; } else { com.google.protobuf.ByteString bs = (com.google.protobuf.ByteString) ref; - String s = bs.toStringUtf8(); - if (com.google.protobuf.Internal.isValidUtf8(bs)) { + java.lang.String s = bs.toStringUtf8(); + if (bs.isValidUtf8()) { key_ = s; } return s; } } - private com.google.protobuf.ByteString getKeyBytes() { + /** + * required string key = 1; + */ + public com.google.protobuf.ByteString + getKeyBytes() { java.lang.Object ref = key_; - if (ref instanceof String) { + if (ref instanceof java.lang.String) { com.google.protobuf.ByteString b = - com.google.protobuf.ByteString.copyFromUtf8((String) ref); + com.google.protobuf.ByteString.copyFromUtf8( + (java.lang.String) ref); key_ = b; return b; } else { return (com.google.protobuf.ByteString) ref; } } - + // required string value = 2; public static final int VALUE_FIELD_NUMBER = 2; private java.lang.Object value_; + /** + * required string value = 2; + */ public boolean hasValue() { return ((bitField0_ & 0x00000002) == 0x00000002); } - public String getValue() { + /** + * required string value = 2; + */ + public java.lang.String getValue() { java.lang.Object ref = value_; - if (ref instanceof String) { - return (String) ref; + if (ref instanceof java.lang.String) { + return (java.lang.String) ref; } else { com.google.protobuf.ByteString bs = (com.google.protobuf.ByteString) ref; - String s = bs.toStringUtf8(); - if (com.google.protobuf.Internal.isValidUtf8(bs)) { + java.lang.String s = bs.toStringUtf8(); + if (bs.isValidUtf8()) { value_ = s; } return s; } } - private com.google.protobuf.ByteString getValueBytes() { + /** + * required string value = 2; + */ + public com.google.protobuf.ByteString + getValueBytes() { java.lang.Object ref = value_; - if (ref instanceof String) { + if (ref instanceof java.lang.String) { com.google.protobuf.ByteString b = - com.google.protobuf.ByteString.copyFromUtf8((String) ref); + com.google.protobuf.ByteString.copyFromUtf8( + (java.lang.String) ref); value_ = b; return b; } else { return (com.google.protobuf.ByteString) ref; } } - + private void initFields() { key_ = ""; value_ = ""; @@ -5646,7 +7038,7 @@ private void initFields() { public final boolean isInitialized() { byte isInitialized = memoizedIsInitialized; if (isInitialized != -1) return isInitialized == 1; - + if (!hasKey()) { memoizedIsInitialized = 0; return false; @@ -5658,7 +7050,7 @@ public final boolean isInitialized() { memoizedIsInitialized = 1; return true; } - + public void writeTo(com.google.protobuf.CodedOutputStream output) throws java.io.IOException { getSerializedSize(); @@ -5670,12 +7062,12 @@ public void writeTo(com.google.protobuf.CodedOutputStream output) } getUnknownFields().writeTo(output); } - + private int memoizedSerializedSize = -1; public int getSerializedSize() { int size = memoizedSerializedSize; if (size != -1) return size; - + size = 0; if (((bitField0_ & 0x00000001) == 0x00000001)) { size += com.google.protobuf.CodedOutputStream @@ -5689,94 +7081,83 @@ public int getSerializedSize() { memoizedSerializedSize = size; return size; } - + private static final long serialVersionUID = 0L; @java.lang.Override protected java.lang.Object writeReplace() throws java.io.ObjectStreamException { return super.writeReplace(); } - + public static org.apache.flume.channel.file.proto.ProtosFactory.FlumeEventHeader parseFrom( com.google.protobuf.ByteString data) throws com.google.protobuf.InvalidProtocolBufferException { - return newBuilder().mergeFrom(data).buildParsed(); + return PARSER.parseFrom(data); } public static org.apache.flume.channel.file.proto.ProtosFactory.FlumeEventHeader parseFrom( com.google.protobuf.ByteString data, com.google.protobuf.ExtensionRegistryLite extensionRegistry) throws com.google.protobuf.InvalidProtocolBufferException { - return newBuilder().mergeFrom(data, extensionRegistry) - .buildParsed(); + return PARSER.parseFrom(data, extensionRegistry); } public static org.apache.flume.channel.file.proto.ProtosFactory.FlumeEventHeader parseFrom(byte[] data) throws com.google.protobuf.InvalidProtocolBufferException { - return newBuilder().mergeFrom(data).buildParsed(); + return PARSER.parseFrom(data); } public static org.apache.flume.channel.file.proto.ProtosFactory.FlumeEventHeader parseFrom( byte[] data, com.google.protobuf.ExtensionRegistryLite extensionRegistry) throws com.google.protobuf.InvalidProtocolBufferException { - return newBuilder().mergeFrom(data, extensionRegistry) - .buildParsed(); + return PARSER.parseFrom(data, extensionRegistry); } public static org.apache.flume.channel.file.proto.ProtosFactory.FlumeEventHeader parseFrom(java.io.InputStream input) throws java.io.IOException { - return newBuilder().mergeFrom(input).buildParsed(); + return PARSER.parseFrom(input); } public static org.apache.flume.channel.file.proto.ProtosFactory.FlumeEventHeader parseFrom( java.io.InputStream input, com.google.protobuf.ExtensionRegistryLite extensionRegistry) throws java.io.IOException { - return newBuilder().mergeFrom(input, extensionRegistry) - .buildParsed(); + return PARSER.parseFrom(input, extensionRegistry); } public static org.apache.flume.channel.file.proto.ProtosFactory.FlumeEventHeader parseDelimitedFrom(java.io.InputStream input) throws java.io.IOException { - Builder builder = newBuilder(); - if (builder.mergeDelimitedFrom(input)) { - return builder.buildParsed(); - } else { - return null; - } + return PARSER.parseDelimitedFrom(input); } public static org.apache.flume.channel.file.proto.ProtosFactory.FlumeEventHeader parseDelimitedFrom( java.io.InputStream input, com.google.protobuf.ExtensionRegistryLite extensionRegistry) throws java.io.IOException { - Builder builder = newBuilder(); - if (builder.mergeDelimitedFrom(input, extensionRegistry)) { - return builder.buildParsed(); - } else { - return null; - } + return PARSER.parseDelimitedFrom(input, extensionRegistry); } public static org.apache.flume.channel.file.proto.ProtosFactory.FlumeEventHeader parseFrom( com.google.protobuf.CodedInputStream input) throws java.io.IOException { - return newBuilder().mergeFrom(input).buildParsed(); + return PARSER.parseFrom(input); } public static org.apache.flume.channel.file.proto.ProtosFactory.FlumeEventHeader parseFrom( com.google.protobuf.CodedInputStream input, com.google.protobuf.ExtensionRegistryLite extensionRegistry) throws java.io.IOException { - return newBuilder().mergeFrom(input, extensionRegistry) - .buildParsed(); + return PARSER.parseFrom(input, extensionRegistry); } - + public static Builder newBuilder() { return Builder.create(); } public Builder newBuilderForType() { return newBuilder(); } public static Builder newBuilder(org.apache.flume.channel.file.proto.ProtosFactory.FlumeEventHeader prototype) { return newBuilder().mergeFrom(prototype); } public Builder toBuilder() { return newBuilder(this); } - + @java.lang.Override protected Builder newBuilderForType( com.google.protobuf.GeneratedMessage.BuilderParent parent) { Builder builder = new Builder(parent); return builder; } + /** + * Protobuf type {@code FlumeEventHeader} + */ public static final class Builder extends com.google.protobuf.GeneratedMessage.Builder implements org.apache.flume.channel.file.proto.ProtosFactory.FlumeEventHeaderOrBuilder { @@ -5784,18 +7165,21 @@ public static final class Builder extends getDescriptor() { return org.apache.flume.channel.file.proto.ProtosFactory.internal_static_FlumeEventHeader_descriptor; } - + protected com.google.protobuf.GeneratedMessage.FieldAccessorTable internalGetFieldAccessorTable() { - return org.apache.flume.channel.file.proto.ProtosFactory.internal_static_FlumeEventHeader_fieldAccessorTable; + return org.apache.flume.channel.file.proto.ProtosFactory.internal_static_FlumeEventHeader_fieldAccessorTable + .ensureFieldAccessorsInitialized( + org.apache.flume.channel.file.proto.ProtosFactory.FlumeEventHeader.class, org.apache.flume.channel.file.proto.ProtosFactory.FlumeEventHeader.Builder.class); } - + // Construct using org.apache.flume.channel.file.proto.ProtosFactory.FlumeEventHeader.newBuilder() private Builder() { maybeForceBuilderInitialization(); } - - private Builder(BuilderParent parent) { + + private Builder( + com.google.protobuf.GeneratedMessage.BuilderParent parent) { super(parent); maybeForceBuilderInitialization(); } @@ -5806,7 +7190,7 @@ private void maybeForceBuilderInitialization() { private static Builder create() { return new Builder(); } - + public Builder clear() { super.clear(); key_ = ""; @@ -5815,20 +7199,20 @@ public Builder clear() { bitField0_ = (bitField0_ & ~0x00000002); return this; } - + public Builder clone() { return create().mergeFrom(buildPartial()); } - + public com.google.protobuf.Descriptors.Descriptor getDescriptorForType() { - return org.apache.flume.channel.file.proto.ProtosFactory.FlumeEventHeader.getDescriptor(); + return org.apache.flume.channel.file.proto.ProtosFactory.internal_static_FlumeEventHeader_descriptor; } - + public org.apache.flume.channel.file.proto.ProtosFactory.FlumeEventHeader getDefaultInstanceForType() { return org.apache.flume.channel.file.proto.ProtosFactory.FlumeEventHeader.getDefaultInstance(); } - + public org.apache.flume.channel.file.proto.ProtosFactory.FlumeEventHeader build() { org.apache.flume.channel.file.proto.ProtosFactory.FlumeEventHeader result = buildPartial(); if (!result.isInitialized()) { @@ -5836,17 +7220,7 @@ public org.apache.flume.channel.file.proto.ProtosFactory.FlumeEventHeader build( } return result; } - - private org.apache.flume.channel.file.proto.ProtosFactory.FlumeEventHeader buildParsed() - throws com.google.protobuf.InvalidProtocolBufferException { - org.apache.flume.channel.file.proto.ProtosFactory.FlumeEventHeader result = buildPartial(); - if (!result.isInitialized()) { - throw newUninitializedMessageException( - result).asInvalidProtocolBufferException(); - } - return result; - } - + public org.apache.flume.channel.file.proto.ProtosFactory.FlumeEventHeader buildPartial() { org.apache.flume.channel.file.proto.ProtosFactory.FlumeEventHeader result = new org.apache.flume.channel.file.proto.ProtosFactory.FlumeEventHeader(this); int from_bitField0_ = bitField0_; @@ -5863,7 +7237,7 @@ public org.apache.flume.channel.file.proto.ProtosFactory.FlumeEventHeader buildP onBuilt(); return result; } - + public Builder mergeFrom(com.google.protobuf.Message other) { if (other instanceof org.apache.flume.channel.file.proto.ProtosFactory.FlumeEventHeader) { return mergeFrom((org.apache.flume.channel.file.proto.ProtosFactory.FlumeEventHeader)other); @@ -5872,19 +7246,23 @@ public Builder mergeFrom(com.google.protobuf.Message other) { return this; } } - + public Builder mergeFrom(org.apache.flume.channel.file.proto.ProtosFactory.FlumeEventHeader other) { if (other == org.apache.flume.channel.file.proto.ProtosFactory.FlumeEventHeader.getDefaultInstance()) return this; if (other.hasKey()) { - setKey(other.getKey()); + bitField0_ |= 0x00000001; + key_ = other.key_; + onChanged(); } if (other.hasValue()) { - setValue(other.getValue()); + bitField0_ |= 0x00000002; + value_ = other.value_; + onChanged(); } this.mergeUnknownFields(other.getUnknownFields()); return this; } - + public final boolean isInitialized() { if (!hasKey()) { @@ -5896,62 +7274,69 @@ public final boolean isInitialized() { } return true; } - + public Builder mergeFrom( com.google.protobuf.CodedInputStream input, com.google.protobuf.ExtensionRegistryLite extensionRegistry) throws java.io.IOException { - com.google.protobuf.UnknownFieldSet.Builder unknownFields = - com.google.protobuf.UnknownFieldSet.newBuilder( - this.getUnknownFields()); - while (true) { - int tag = input.readTag(); - switch (tag) { - case 0: - this.setUnknownFields(unknownFields.build()); - onChanged(); - return this; - default: { - if (!parseUnknownField(input, unknownFields, - extensionRegistry, tag)) { - this.setUnknownFields(unknownFields.build()); - onChanged(); - return this; - } - break; - } - case 10: { - bitField0_ |= 0x00000001; - key_ = input.readBytes(); - break; - } - case 18: { - bitField0_ |= 0x00000002; - value_ = input.readBytes(); - break; - } + org.apache.flume.channel.file.proto.ProtosFactory.FlumeEventHeader parsedMessage = null; + try { + parsedMessage = PARSER.parsePartialFrom(input, extensionRegistry); + } catch (com.google.protobuf.InvalidProtocolBufferException e) { + parsedMessage = (org.apache.flume.channel.file.proto.ProtosFactory.FlumeEventHeader) e.getUnfinishedMessage(); + throw e; + } finally { + if (parsedMessage != null) { + mergeFrom(parsedMessage); } } + return this; } - private int bitField0_; - + // required string key = 1; private java.lang.Object key_ = ""; + /** + * required string key = 1; + */ public boolean hasKey() { return ((bitField0_ & 0x00000001) == 0x00000001); } - public String getKey() { + /** + * required string key = 1; + */ + public java.lang.String getKey() { java.lang.Object ref = key_; - if (!(ref instanceof String)) { - String s = ((com.google.protobuf.ByteString) ref).toStringUtf8(); + if (!(ref instanceof java.lang.String)) { + java.lang.String s = ((com.google.protobuf.ByteString) ref) + .toStringUtf8(); key_ = s; return s; } else { - return (String) ref; + return (java.lang.String) ref; + } + } + /** + * required string key = 1; + */ + public com.google.protobuf.ByteString + getKeyBytes() { + java.lang.Object ref = key_; + if (ref instanceof String) { + com.google.protobuf.ByteString b = + com.google.protobuf.ByteString.copyFromUtf8( + (java.lang.String) ref); + key_ = b; + return b; + } else { + return (com.google.protobuf.ByteString) ref; } } - public Builder setKey(String value) { + /** + * required string key = 1; + */ + public Builder setKey( + java.lang.String value) { if (value == null) { throw new NullPointerException(); } @@ -5960,34 +7345,72 @@ public Builder setKey(String value) { onChanged(); return this; } + /** + * required string key = 1; + */ public Builder clearKey() { bitField0_ = (bitField0_ & ~0x00000001); key_ = getDefaultInstance().getKey(); onChanged(); return this; } - void setKey(com.google.protobuf.ByteString value) { - bitField0_ |= 0x00000001; + /** + * required string key = 1; + */ + public Builder setKeyBytes( + com.google.protobuf.ByteString value) { + if (value == null) { + throw new NullPointerException(); + } + bitField0_ |= 0x00000001; key_ = value; onChanged(); + return this; } - + // required string value = 2; private java.lang.Object value_ = ""; + /** + * required string value = 2; + */ public boolean hasValue() { return ((bitField0_ & 0x00000002) == 0x00000002); } - public String getValue() { + /** + * required string value = 2; + */ + public java.lang.String getValue() { java.lang.Object ref = value_; - if (!(ref instanceof String)) { - String s = ((com.google.protobuf.ByteString) ref).toStringUtf8(); + if (!(ref instanceof java.lang.String)) { + java.lang.String s = ((com.google.protobuf.ByteString) ref) + .toStringUtf8(); value_ = s; return s; } else { - return (String) ref; + return (java.lang.String) ref; } } - public Builder setValue(String value) { + /** + * required string value = 2; + */ + public com.google.protobuf.ByteString + getValueBytes() { + java.lang.Object ref = value_; + if (ref instanceof String) { + com.google.protobuf.ByteString b = + com.google.protobuf.ByteString.copyFromUtf8( + (java.lang.String) ref); + value_ = b; + return b; + } else { + return (com.google.protobuf.ByteString) ref; + } + } + /** + * required string value = 2; + */ + public Builder setValue( + java.lang.String value) { if (value == null) { throw new NullPointerException(); } @@ -5996,29 +7419,40 @@ public Builder setValue(String value) { onChanged(); return this; } + /** + * required string value = 2; + */ public Builder clearValue() { bitField0_ = (bitField0_ & ~0x00000002); value_ = getDefaultInstance().getValue(); onChanged(); return this; } - void setValue(com.google.protobuf.ByteString value) { - bitField0_ |= 0x00000002; + /** + * required string value = 2; + */ + public Builder setValueBytes( + com.google.protobuf.ByteString value) { + if (value == null) { + throw new NullPointerException(); + } + bitField0_ |= 0x00000002; value_ = value; onChanged(); + return this; } - + // @@protoc_insertion_point(builder_scope:FlumeEventHeader) } - + static { defaultInstance = new FlumeEventHeader(true); defaultInstance.initFields(); } - + // @@protoc_insertion_point(class_scope:FlumeEventHeader) } - + private static com.google.protobuf.Descriptors.Descriptor internal_static_Checkpoint_descriptor; private static @@ -6079,7 +7513,7 @@ void setValue(com.google.protobuf.ByteString value) { private static com.google.protobuf.GeneratedMessage.FieldAccessorTable internal_static_FlumeEventHeader_fieldAccessorTable; - + public static com.google.protobuf.Descriptors.FileDescriptor getDescriptor() { return descriptor; @@ -6122,97 +7556,73 @@ public com.google.protobuf.ExtensionRegistry assignDescriptors( internal_static_Checkpoint_fieldAccessorTable = new com.google.protobuf.GeneratedMessage.FieldAccessorTable( internal_static_Checkpoint_descriptor, - new java.lang.String[] { "Version", "WriteOrderID", "QueueSize", "QueueHead", "ActiveLogs", }, - org.apache.flume.channel.file.proto.ProtosFactory.Checkpoint.class, - org.apache.flume.channel.file.proto.ProtosFactory.Checkpoint.Builder.class); + new java.lang.String[] { "Version", "WriteOrderID", "QueueSize", "QueueHead", "ActiveLogs", }); internal_static_ActiveLog_descriptor = getDescriptor().getMessageTypes().get(1); internal_static_ActiveLog_fieldAccessorTable = new com.google.protobuf.GeneratedMessage.FieldAccessorTable( internal_static_ActiveLog_descriptor, - new java.lang.String[] { "LogFileID", "Count", }, - org.apache.flume.channel.file.proto.ProtosFactory.ActiveLog.class, - org.apache.flume.channel.file.proto.ProtosFactory.ActiveLog.Builder.class); + new java.lang.String[] { "LogFileID", "Count", }); internal_static_LogFileMetaData_descriptor = getDescriptor().getMessageTypes().get(2); internal_static_LogFileMetaData_fieldAccessorTable = new com.google.protobuf.GeneratedMessage.FieldAccessorTable( internal_static_LogFileMetaData_descriptor, - new java.lang.String[] { "Version", "LogFileID", "CheckpointPosition", "CheckpointWriteOrderID", "Encryption", "BackupCheckpointPosition", "BackupCheckpointWriteOrderID", }, - org.apache.flume.channel.file.proto.ProtosFactory.LogFileMetaData.class, - org.apache.flume.channel.file.proto.ProtosFactory.LogFileMetaData.Builder.class); + new java.lang.String[] { "Version", "LogFileID", "CheckpointPosition", "CheckpointWriteOrderID", "Encryption", "BackupCheckpointPosition", "BackupCheckpointWriteOrderID", }); internal_static_LogFileEncryption_descriptor = getDescriptor().getMessageTypes().get(3); internal_static_LogFileEncryption_fieldAccessorTable = new com.google.protobuf.GeneratedMessage.FieldAccessorTable( internal_static_LogFileEncryption_descriptor, - new java.lang.String[] { "CipherProvider", "KeyAlias", "Parameters", }, - org.apache.flume.channel.file.proto.ProtosFactory.LogFileEncryption.class, - org.apache.flume.channel.file.proto.ProtosFactory.LogFileEncryption.Builder.class); + new java.lang.String[] { "CipherProvider", "KeyAlias", "Parameters", }); internal_static_TransactionEventHeader_descriptor = getDescriptor().getMessageTypes().get(4); internal_static_TransactionEventHeader_fieldAccessorTable = new com.google.protobuf.GeneratedMessage.FieldAccessorTable( internal_static_TransactionEventHeader_descriptor, - new java.lang.String[] { "Type", "TransactionID", "WriteOrderID", }, - org.apache.flume.channel.file.proto.ProtosFactory.TransactionEventHeader.class, - org.apache.flume.channel.file.proto.ProtosFactory.TransactionEventHeader.Builder.class); + new java.lang.String[] { "Type", "TransactionID", "WriteOrderID", }); internal_static_Put_descriptor = getDescriptor().getMessageTypes().get(5); internal_static_Put_fieldAccessorTable = new com.google.protobuf.GeneratedMessage.FieldAccessorTable( internal_static_Put_descriptor, - new java.lang.String[] { "Event", "Checksum", }, - org.apache.flume.channel.file.proto.ProtosFactory.Put.class, - org.apache.flume.channel.file.proto.ProtosFactory.Put.Builder.class); + new java.lang.String[] { "Event", "Checksum", }); internal_static_Take_descriptor = getDescriptor().getMessageTypes().get(6); internal_static_Take_fieldAccessorTable = new com.google.protobuf.GeneratedMessage.FieldAccessorTable( internal_static_Take_descriptor, - new java.lang.String[] { "FileID", "Offset", }, - org.apache.flume.channel.file.proto.ProtosFactory.Take.class, - org.apache.flume.channel.file.proto.ProtosFactory.Take.Builder.class); + new java.lang.String[] { "FileID", "Offset", }); internal_static_Rollback_descriptor = getDescriptor().getMessageTypes().get(7); internal_static_Rollback_fieldAccessorTable = new com.google.protobuf.GeneratedMessage.FieldAccessorTable( internal_static_Rollback_descriptor, - new java.lang.String[] { }, - org.apache.flume.channel.file.proto.ProtosFactory.Rollback.class, - org.apache.flume.channel.file.proto.ProtosFactory.Rollback.Builder.class); + new java.lang.String[] { }); internal_static_Commit_descriptor = getDescriptor().getMessageTypes().get(8); internal_static_Commit_fieldAccessorTable = new com.google.protobuf.GeneratedMessage.FieldAccessorTable( internal_static_Commit_descriptor, - new java.lang.String[] { "Type", }, - org.apache.flume.channel.file.proto.ProtosFactory.Commit.class, - org.apache.flume.channel.file.proto.ProtosFactory.Commit.Builder.class); + new java.lang.String[] { "Type", }); internal_static_TransactionEventFooter_descriptor = getDescriptor().getMessageTypes().get(9); internal_static_TransactionEventFooter_fieldAccessorTable = new com.google.protobuf.GeneratedMessage.FieldAccessorTable( internal_static_TransactionEventFooter_descriptor, - new java.lang.String[] { }, - org.apache.flume.channel.file.proto.ProtosFactory.TransactionEventFooter.class, - org.apache.flume.channel.file.proto.ProtosFactory.TransactionEventFooter.Builder.class); + new java.lang.String[] { }); internal_static_FlumeEvent_descriptor = getDescriptor().getMessageTypes().get(10); internal_static_FlumeEvent_fieldAccessorTable = new com.google.protobuf.GeneratedMessage.FieldAccessorTable( internal_static_FlumeEvent_descriptor, - new java.lang.String[] { "Headers", "Body", }, - org.apache.flume.channel.file.proto.ProtosFactory.FlumeEvent.class, - org.apache.flume.channel.file.proto.ProtosFactory.FlumeEvent.Builder.class); + new java.lang.String[] { "Headers", "Body", }); internal_static_FlumeEventHeader_descriptor = getDescriptor().getMessageTypes().get(11); internal_static_FlumeEventHeader_fieldAccessorTable = new com.google.protobuf.GeneratedMessage.FieldAccessorTable( internal_static_FlumeEventHeader_descriptor, - new java.lang.String[] { "Key", "Value", }, - org.apache.flume.channel.file.proto.ProtosFactory.FlumeEventHeader.class, - org.apache.flume.channel.file.proto.ProtosFactory.FlumeEventHeader.Builder.class); + new java.lang.String[] { "Key", "Value", }); return null; } }; @@ -6221,6 +7631,6 @@ public com.google.protobuf.ExtensionRegistry assignDescriptors( new com.google.protobuf.Descriptors.FileDescriptor[] { }, assigner); } - + // @@protoc_insertion_point(outer_class_scope) } diff --git a/pom.xml b/pom.xml index 1405cdae9b..25ea4e798e 100644 --- a/pom.xml +++ b/pom.xml @@ -747,7 +747,7 @@ limitations under the License. com.google.protobuf protobuf-java compile - 2.4.1 + 2.5.0 From 629b7e6b888190ebd5b9931caeec90c5d1c2ba58 Mon Sep 17 00:00:00 2001 From: Hari Shreedharan Date: Mon, 16 Sep 2013 15:38:13 -0700 Subject: [PATCH 018/341] FLUME-2185. Upgrade morphlines to 0.7.0 (Wolfgang Hoschek via Hari Shreedharan) --- flume-ng-doc/sphinx/FlumeUserGuide.rst | 4 ++-- .../flume-ng-morphline-solr-sink/pom.xml | 20 +------------------ 2 files changed, 3 insertions(+), 21 deletions(-) diff --git a/flume-ng-doc/sphinx/FlumeUserGuide.rst b/flume-ng-doc/sphinx/FlumeUserGuide.rst index c61499134c..bbfb5d0c3c 100644 --- a/flume-ng-doc/sphinx/FlumeUserGuide.rst +++ b/flume-ng-doc/sphinx/FlumeUserGuide.rst @@ -1839,7 +1839,7 @@ This sink extracts data from Flume events, transforms it, and loads it in near-r This sink is well suited for use cases that stream raw data into HDFS (via the HdfsSink) and simultaneously extract, transform and load the same data into Solr (via MorphlineSolrSink). In particular, this sink can process arbitrary heterogeneous raw data from disparate data sources and turn it into a data model that is useful to Search applications. -The ETL functionality is customizable using a `morphline configuration file `_ that defines a chain of transformation commands that pipe event records from one command to another. +The ETL functionality is customizable using a `morphline configuration file `_ that defines a chain of transformation commands that pipe event records from one command to another. Morphlines can be seen as an evolution of Unix pipelines where the data model is generalized to work with streams of generic records, including arbitrary binary payloads. A morphline command is a bit like a Flume Interceptor. Morphlines can be embedded into Hadoop components such as Flume. @@ -2599,7 +2599,7 @@ prefix "" The prefix string constant to prepend to each generat Morphline Interceptor ~~~~~~~~~~~~~~~~~~~~~~~~~~~ -This interceptor filters the events through a `morphline configuration file `_ that defines a chain of transformation commands that pipe records from one command to another. +This interceptor filters the events through a `morphline configuration file `_ that defines a chain of transformation commands that pipe records from one command to another. For example the morphline can ignore certain events or alter or insert certain event headers via regular expression based pattern matching, or it can auto-detect and set a MIME type via Apache Tika on events that are intercepted. For example, this kind of packet sniffing can be used for content based dynamic routing in a Flume topology. MorphlineInterceptor can also help to implement dynamic routing to multiple Apache Solr collections (e.g. for multi-tenancy). diff --git a/flume-ng-sinks/flume-ng-morphline-solr-sink/pom.xml b/flume-ng-sinks/flume-ng-morphline-solr-sink/pom.xml index a2fb931e86..b2640d9c4b 100644 --- a/flume-ng-sinks/flume-ng-morphline-solr-sink/pom.xml +++ b/flume-ng-sinks/flume-ng-morphline-solr-sink/pom.xml @@ -33,8 +33,7 @@ limitations under the License. UTF-8 4.3.0 4.3.0 - 1.3 - 0.6.0 + 0.7.0 1.6.1 2.12.4 @@ -108,23 +107,6 @@ limitations under the License. - - org.apache.tika - tika-xmp - ${tika.version} - test - - - org.apache.geronimo.specs - geronimo-stax-api_1.0_spec - - - xerces - xercesImpl - - - - com.cloudera.cdk cdk-morphlines-solr-core From bb7fb11b6ad3945704a6511140538b998e5143d1 Mon Sep 17 00:00:00 2001 From: Hari Shreedharan Date: Thu, 19 Sep 2013 21:53:55 -0700 Subject: [PATCH 019/341] FLUME-2189. Add support for IP filtering on AvroSource (Ted Malaska via Hari Shreedharan) --- .../org/apache/flume/source/AvroSource.java | 122 ++++++++++++- .../apache/flume/source/TestAvroSource.java | 163 +++++++++++++++++- flume-ng-doc/sphinx/FlumeUserGuide.rst | 17 ++ pom.xml | 2 +- 4 files changed, 292 insertions(+), 12 deletions(-) diff --git a/flume-ng-core/src/main/java/org/apache/flume/source/AvroSource.java b/flume-ng-core/src/main/java/org/apache/flume/source/AvroSource.java index f23cd938f7..f6e4cfe693 100644 --- a/flume-ng-core/src/main/java/org/apache/flume/source/AvroSource.java +++ b/flume-ng-core/src/main/java/org/apache/flume/source/AvroSource.java @@ -35,7 +35,6 @@ import javax.net.ssl.KeyManagerFactory; import javax.net.ssl.SSLContext; import javax.net.ssl.SSLEngine; - import org.apache.avro.ipc.NettyServer; import org.apache.avro.ipc.Responder; import org.apache.avro.ipc.Server; @@ -60,6 +59,8 @@ import org.jboss.netty.channel.Channels; import org.jboss.netty.handler.codec.compression.ZlibDecoder; import org.jboss.netty.handler.codec.compression.ZlibEncoder; +import org.jboss.netty.handler.ipfilter.IpFilterRuleHandler; +import org.jboss.netty.handler.ipfilter.PatternRule; import org.jboss.netty.handler.ssl.SslHandler; import org.slf4j.Logger; import org.slf4j.LoggerFactory; @@ -131,6 +132,8 @@ public class AvroSource extends AbstractSource implements EventDrivenSource, private static final String BIND_KEY = "bind"; private static final String COMPRESSION_TYPE = "compression-type"; private static final String SSL_KEY = "ssl"; + private static final String IP_FILTER_KEY = "ipFilter"; + private static final String IP_FILTER_RULES_KEY = "ipFilterRules"; private static final String KEYSTORE_KEY = "keystore"; private static final String KEYSTORE_PASSWORD_KEY = "keystore-password"; private static final String KEYSTORE_TYPE_KEY = "keystore-type"; @@ -141,6 +144,8 @@ public class AvroSource extends AbstractSource implements EventDrivenSource, private String keystorePassword; private String keystoreType; private boolean enableSsl = false; + private boolean enableIpFilter; + private String patternRuleConfigDefinition; private Server server; private SourceCounter sourceCounter; @@ -182,6 +187,17 @@ public void configure(Context context) { } } + enableIpFilter = context.getBoolean(IP_FILTER_KEY, false); + if (enableIpFilter) { + patternRuleConfigDefinition = context.getString(IP_FILTER_RULES_KEY); + if (patternRuleConfigDefinition == null || + patternRuleConfigDefinition.isEmpty()) { + throw new FlumeException( + "ipFilter is configured with true but ipFilterRules is not defined:" + + " "); + } + } + if (sourceCounter == null) { sourceCounter = new SourceCounter(getName()); } @@ -233,10 +249,11 @@ private NioServerSocketChannelFactory initSocketChannelFactory() { private ChannelPipelineFactory initChannelPipelineFactory() { ChannelPipelineFactory pipelineFactory; boolean enableCompression = compressionType.equalsIgnoreCase("deflate"); - if (enableCompression || enableSsl) { - pipelineFactory = new SSLCompressionChannelPipelineFactory( - enableCompression, enableSsl, keystore, - keystorePassword, keystoreType); + if (enableCompression || enableSsl || enableIpFilter) { + pipelineFactory = new AdvancedChannelPipelineFactory( + enableCompression, enableSsl, keystore, + keystorePassword, keystoreType, enableIpFilter, + patternRuleConfigDefinition); } else { pipelineFactory = new ChannelPipelineFactory() { @Override @@ -356,7 +373,7 @@ public Status appendBatch(List events) { * Factory of SSL-enabled server worker channel pipelines * Copied from Avro's org.apache.avro.ipc.TestNettyServerWithSSL test */ - private static class SSLCompressionChannelPipelineFactory + private static class AdvancedChannelPipelineFactory implements ChannelPipelineFactory { private boolean enableCompression; @@ -365,12 +382,20 @@ private static class SSLCompressionChannelPipelineFactory private String keystorePassword; private String keystoreType; - public SSLCompressionChannelPipelineFactory(boolean enableCompression, boolean enableSsl, String keystore, String keystorePassword, String keystoreType) { + private boolean enableIpFilter; + private String patternRuleConfigDefinition; + + public AdvancedChannelPipelineFactory(boolean enableCompression, + boolean enableSsl, String keystore, String keystorePassword, + String keystoreType, boolean enableIpFilter, + String patternRuleConfigDefinition) { this.enableCompression = enableCompression; this.enableSsl = enableSsl; this.keystore = keystore; this.keystorePassword = keystorePassword; this.keystoreType = keystoreType; + this.enableIpFilter = enableIpFilter; + this.patternRuleConfigDefinition = patternRuleConfigDefinition; } private SSLContext createServerSSLContext() { @@ -407,6 +432,8 @@ public ChannelPipeline getPipeline() throws Exception { pipeline.addFirst("deflater", encoder); pipeline.addFirst("inflater", new ZlibDecoder()); } + + if (enableSsl) { SSLEngine sslEngine = createServerSSLContext().createSSLEngine(); sslEngine.setUseClientMode(false); @@ -415,7 +442,88 @@ public ChannelPipeline getPipeline() throws Exception { // adding compression handling above pipeline.addFirst("ssl", new SslHandler(sslEngine)); } + + if (enableIpFilter) { + + logger.info("Setting up ipFilter with the following rule definition: " + + patternRuleConfigDefinition); + IpFilterRuleHandler ipFilterHandler = new IpFilterRuleHandler(); + + if (patternRuleConfigDefinition != null && + !patternRuleConfigDefinition.isEmpty()) { + String[] patternRuleDefinitions = patternRuleConfigDefinition.split( + ","); + for (String patternRuleDefinition : patternRuleDefinitions) { + + PatternRule patternRule + = PatternRuleBuilder.withConfigRuleDefinition( + patternRuleDefinition); + + if (patternRule != null) { + ipFilterHandler.add(patternRule); + } + } + } + + logger.info( + "Adding ipFilter with " + ipFilterHandler.size() + " rules"); + + pipeline.addFirst("ipFilter", ipFilterHandler); + } + return pipeline; } + + public static class PatternRuleBuilder { + public static PatternRule withConfigRuleDefinition( + String patternRuleDefinition) throws FlumeException { + patternRuleDefinition = patternRuleDefinition.trim(); + //first validation the format + + int firstColonIndex = patternRuleDefinition.indexOf(":"); + if (firstColonIndex == -1) { + logger.error( + "Invalid ipFilter patternRule '" + patternRuleDefinition + + "' should look like <'allow' or 'deny'>:<'ip' or " + + "'name'>:"); + return null; + } else { + + String ruleAccessFlag = patternRuleDefinition.substring(0, + firstColonIndex); + int secondColonIndex = patternRuleDefinition.indexOf(":", + firstColonIndex + 1); + if ((!ruleAccessFlag.equals("allow") && + !ruleAccessFlag.equals("deny")) || secondColonIndex == -1) { + logger.error( + "Invalid ipFilter patternRule '" + patternRuleDefinition + + "' should look like <'allow' or 'deny'>:<'ip' or " + + "'name'>:"); + return null; + } + + String patternTypeFlag = patternRuleDefinition.substring( + firstColonIndex + 1, secondColonIndex); + if ((!patternTypeFlag.equals("ip") && + !patternTypeFlag.equals("name"))) { + logger.error( + "Invalid ipFilter patternRule '" + patternRuleDefinition + + "' should look like <'allow' or 'deny'>:<'ip' or " + + "'name'>:"); + return null; + } + + boolean isAllow = ruleAccessFlag.equals("allow"); + String patternRuleString = + (patternTypeFlag.equals("ip") ? "i" : "n") + ":" + + patternRuleDefinition.substring(secondColonIndex + 1); + logger.info("Adding ipFilter PatternRule: " + + (isAllow ? "Allow" : "deny") + + " " + patternRuleString); + return new PatternRule(isAllow, patternRuleString); + } + } + } + } } diff --git a/flume-ng-core/src/test/java/org/apache/flume/source/TestAvroSource.java b/flume-ng-core/src/test/java/org/apache/flume/source/TestAvroSource.java index 2667a6fdac..e208fffafb 100644 --- a/flume-ng-core/src/test/java/org/apache/flume/source/TestAvroSource.java +++ b/flume-ng-core/src/test/java/org/apache/flume/source/TestAvroSource.java @@ -20,6 +20,7 @@ package org.apache.flume.source; import java.io.IOException; +import java.net.Inet4Address; import java.net.InetSocketAddress; import java.nio.ByteBuffer; import java.security.cert.X509Certificate; @@ -39,6 +40,7 @@ import org.apache.flume.ChannelSelector; import org.apache.flume.Context; import org.apache.flume.Event; +import org.apache.flume.FlumeException; import org.apache.flume.Transaction; import org.apache.flume.channel.ChannelProcessor; import org.apache.flume.channel.MemoryChannel; @@ -199,14 +201,19 @@ private void doRequest(boolean serverEnableCompression, boolean clientEnableComp source.getLifecycleState()); AvroSourceProtocol client; + NettyTransceiver nettyTransceiver; if (clientEnableCompression) { + + nettyTransceiver = new NettyTransceiver(new InetSocketAddress( + selectedPort), new CompressionChannelFactory(compressionLevel)); + client = SpecificRequestor.getClient( - AvroSourceProtocol.class, new NettyTransceiver(new InetSocketAddress( - selectedPort), new CompressionChannelFactory(6))); + AvroSourceProtocol.class, nettyTransceiver); } else { + nettyTransceiver = new NettyTransceiver(new InetSocketAddress(selectedPort)); + client = SpecificRequestor.getClient( - AvroSourceProtocol.class, new NettyTransceiver(new InetSocketAddress( - selectedPort))); + AvroSourceProtocol.class, nettyTransceiver); } AvroFlumeEvent avroEvent = new AvroFlumeEvent(); @@ -230,6 +237,8 @@ AvroSourceProtocol.class, new NettyTransceiver(new InetSocketAddress( logger.debug("Round trip event:{}", event); + + nettyTransceiver.close(); source.stop(); Assert.assertTrue("Reached stop or error", LifecycleController.waitForOneOf(source, LifecycleState.STOP_OR_ERROR)); @@ -372,4 +381,150 @@ public X509Certificate[] getAcceptedIssuers() { return new X509Certificate[0]; } } + + @Test + public void testValidIpFilterAllows() throws InterruptedException, IOException { + + doIpFilterTest("allow:name:localhost,deny:ip:*", true, false); + doIpFilterTest("allow:ip:" + Inet4Address.getLocalHost().getHostAddress() + ",deny:ip:*", true, false); + doIpFilterTest("allow:ip:*", true, false); + doIpFilterTest("allow:ip:" + Inet4Address.getLocalHost().getHostAddress().substring(0, 3) + "*,deny:ip:*", true, false); + doIpFilterTest("allow:ip:127.0.0.2,allow:ip:" + Inet4Address.getLocalHost().getHostAddress().substring(0, 3) + "*,deny:ip:*", true, false); + + doIpFilterTest("allow:name:localhost,deny:ip:*", true, true); + doIpFilterTest("allow:ip:*", true, true); + + } + + @Test + public void testValidIpFilterDenys() throws InterruptedException, IOException { + + doIpFilterTest("deny:ip:*", false, false); + doIpFilterTest("deny:name:localhost", false, false); + doIpFilterTest("deny:ip:" + Inet4Address.getLocalHost().getHostAddress() + ",allow:ip:*", false, false); + doIpFilterTest("deny:ip:*", false, false); + doIpFilterTest("allow:ip:45.2.2.2,deny:ip:*", false, false); + doIpFilterTest("deny:ip:" + Inet4Address.getLocalHost().getHostAddress().substring(0, 3) + "*,allow:ip:*", false, false); + + + doIpFilterTest("deny:ip:*", false, true); + } + + @Test + public void testInvalidIpFilter() throws InterruptedException, IOException { + + doIpFilterTest("deny:ip?*", true, false); + doIpFilterTest("deny?name:localhost", true, false); + doIpFilterTest("deny:ip:127.0.0.2,allow:ip?*,deny:ip:" + Inet4Address.getLocalHost().getHostAddress() + "", false, false); + doIpFilterTest("deny:*", true, false); + doIpFilterTest("deny:id:" + Inet4Address.getLocalHost().getHostAddress().substring(0, 3) + "*,allow:ip:*", true, false); + try { + doIpFilterTest(null, true, false); + Assert.fail("The null ipFilterRules config should had thrown an exception."); + } catch (FlumeException e) { + //Do nothing + } + + try{ + doIpFilterTest("", true, false); + Assert.fail("The empty string ipFilterRules config should had thrown an exception."); + } catch (FlumeException e) { + //Do nothing + } + + + } + + public void doIpFilterTest(String ruleDefinition, boolean eventShouldBeAllowed, boolean testWithSSL) throws InterruptedException, IOException { + boolean bound = false; + + for (int i = 0; i < 100 && !bound; i++) { + try { + Context context = new Context(); + + context.put("port", String.valueOf(selectedPort = 41414 + i)); + context.put("bind", "0.0.0.0"); + context.put("ipFilter", "true"); + if (ruleDefinition != null) { + context.put("ipFilterRules", ruleDefinition); + } + if (testWithSSL) { + logger.info("Client testWithSSL" + testWithSSL); + context.put("ssl", "true"); + context.put("keystore", "src/test/resources/server.p12"); + context.put("keystore-password", "password"); + context.put("keystore-type", "PKCS12"); + } + + Configurables.configure(source, context); + + source.start(); + bound = true; + } catch (ChannelException e) { + /* + * NB: This assume we're using the Netty server under the hood and the + * failure is to bind. Yucky. + */ + Thread.sleep(100); + } + } + + Assert + .assertTrue("Reached start or error", LifecycleController.waitForOneOf( + source, LifecycleState.START_OR_ERROR)); + Assert.assertEquals("Server is started", LifecycleState.START, + source.getLifecycleState()); + + AvroSourceProtocol client; + NettyTransceiver nettyTransceiver; + + if (testWithSSL) { + nettyTransceiver = new NettyTransceiver(new InetSocketAddress(selectedPort), new SSLChannelFactory()); + client = SpecificRequestor.getClient( + AvroSourceProtocol.class, nettyTransceiver ); + } else { + nettyTransceiver = new NettyTransceiver(new InetSocketAddress(selectedPort)); + client = SpecificRequestor.getClient( + AvroSourceProtocol.class, nettyTransceiver); + } + + AvroFlumeEvent avroEvent = new AvroFlumeEvent(); + avroEvent.setHeaders(new HashMap()); + avroEvent.setBody(ByteBuffer.wrap("Hello avro ipFilter".getBytes())); + + try { + logger.info("Client about to append"); + Status status = client.append(avroEvent); + logger.info("Client appended"); + Assert.assertEquals(Status.OK, status); + } catch(IOException e) { + Assert.assertTrue("Should have been Allowed:" + ruleDefinition, !eventShouldBeAllowed); + return; + } + Assert.assertTrue("Should have been denied:" + ruleDefinition, eventShouldBeAllowed); + + + + Transaction transaction = channel.getTransaction(); + transaction.begin(); + + Event event = channel.take(); + Assert.assertNotNull(event); + Assert.assertEquals("Channel contained our event", "Hello avro ipFilter", + new String(event.getBody())); + transaction.commit(); + transaction.close(); + + logger.debug("Round trip event:{}", event); + + nettyTransceiver.close(); + source.stop(); + Assert.assertTrue("Reached stop or error", + LifecycleController.waitForOneOf(source, LifecycleState.STOP_OR_ERROR)); + Assert.assertEquals("Server is stopped", LifecycleState.STOP, + source.getLifecycleState()); + + + } + } diff --git a/flume-ng-doc/sphinx/FlumeUserGuide.rst b/flume-ng-doc/sphinx/FlumeUserGuide.rst index bbfb5d0c3c..dac3ce725d 100644 --- a/flume-ng-doc/sphinx/FlumeUserGuide.rst +++ b/flume-ng-doc/sphinx/FlumeUserGuide.rst @@ -705,6 +705,8 @@ ssl false Set this to true to enable SSL encryption. You keystore -- This is the path to a Java keystore file. Required for SSL. keystore-password -- The password for the Java keystore. Required for SSL. keystore-type JKS The type of the Java keystore. This can be "JKS" or "PKCS12". +ipFilter false Set this to true to enable ipFiltering for netty +ipFilter.rules -- Define N netty ipFilter pattern rules with this config. ================== =========== =================================================== Example for agent named a1: @@ -718,6 +720,21 @@ Example for agent named a1: a1.sources.r1.bind = 0.0.0.0 a1.sources.r1.port = 4141 +Example of ipFilter.rules + +ipFilter.rules defines N netty ipFilters separated by a comma a pattern rule must be in this format. + +<'allow' or deny>:<'ip' or 'name' for computer name>: +or +allow/deny:ip/name:pattern + +example: ipFilter.rules=allow:ip:127.*,allow:name:localhost,deny:ip:* + +Note that the first rule to match will apply as the example below shows from a client on the localhost + +This will Allow the client on localhost be deny clients from any other ip "allow:name:localhost,deny:ip:*" +This will deny the client on localhost be allow clients from any other ip "deny:name:localhost,allow:ip:*" + Thrift Source ~~~~~~~~~~~~~ diff --git a/pom.xml b/pom.xml index 25ea4e798e..8b3640268f 100644 --- a/pom.xml +++ b/pom.xml @@ -1049,7 +1049,7 @@ limitations under the License. io.netty netty - 3.4.0.Final + 3.5.12.Final From 0f4a66fb0f2946cd61dd8df31bd255fef7581cbc Mon Sep 17 00:00:00 2001 From: Hari Shreedharan Date: Fri, 27 Sep 2013 13:21:48 -0700 Subject: [PATCH 020/341] FLUME-2190. Add a source capable of feeding off of the Twitter Streaming API (Roman Shaposhnik via Hari Shreedharan) --- flume-ng-dist/pom.xml | 4 + flume-ng-doc/sphinx/FlumeUserGuide.rst | 41 +++ flume-ng-sources/flume-twitter-source/pom.xml | 61 ++++ .../flume/source/twitter/TwitterSource.java | 333 ++++++++++++++++++ .../source/twitter/TestTwitterSource.java | 112 ++++++ .../src/test/resources/log4j.properties | 33 ++ .../src/test/resources/twitter-flume.conf | 92 +++++ flume-ng-sources/pom.xml | 1 + pom.xml | 23 ++ 9 files changed, 700 insertions(+) create mode 100644 flume-ng-sources/flume-twitter-source/pom.xml create mode 100644 flume-ng-sources/flume-twitter-source/src/main/java/org/apache/flume/source/twitter/TwitterSource.java create mode 100644 flume-ng-sources/flume-twitter-source/src/test/java/org/apache/flume/source/twitter/TestTwitterSource.java create mode 100644 flume-ng-sources/flume-twitter-source/src/test/resources/log4j.properties create mode 100644 flume-ng-sources/flume-twitter-source/src/test/resources/twitter-flume.conf diff --git a/flume-ng-dist/pom.xml b/flume-ng-dist/pom.xml index 83332a91a6..2d0ee47718 100644 --- a/flume-ng-dist/pom.xml +++ b/flume-ng-dist/pom.xml @@ -120,6 +120,10 @@ org.apache.flume.flume-ng-sources flume-jms-source + + org.apache.flume.flume-ng-sources + flume-twitter-source + org.apache.flume.flume-ng-legacy-sources flume-avro-source diff --git a/flume-ng-doc/sphinx/FlumeUserGuide.rst b/flume-ng-doc/sphinx/FlumeUserGuide.rst index dac3ce725d..007436b6b5 100644 --- a/flume-ng-doc/sphinx/FlumeUserGuide.rst +++ b/flume-ng-doc/sphinx/FlumeUserGuide.rst @@ -977,6 +977,47 @@ Example for an agent named agent-1: agent-1.sources.src-1.spoolDir = /var/log/apache/flumeSpool agent-1.sources.src-1.fileHeader = true +Twitter 1% firehose Source (experimental) +~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~ + +.. warning:: + This source is hightly experimental and may change between minor versions of Flume. + Use at your own risk. + +Experimental source that connects via Streaming API to the 1% sample twitter +firehose, continously downloads tweets, converts them to Avro format and +sends Avro events to a downstream Flume sink. Requires the consumer and +access tokens and secrets of a Twitter developer account. +Required properties are in **bold**. + +================== =========== =================================================== +Property Name Default Description +================== =========== =================================================== +**channels** -- +**type** -- The component type name, needs to be ``org.apache.flume.source.twitter.TwitterSource`` +**consumerKey** -- OAuth consumer key +**consumerSecret** -- OAuth consumer secret +**accessToken** -- OAuth access token +**accessTokenSecret** -- OAuth toekn secret +maxBatchSize 1000 Maximum number of twitter messages to put in a single batch +maxBatchDurationMillis 1000 Maximum number of milliseconds to wait before closing a batch +================== =========== =================================================== + +Example for agent named a1: + +.. code-block:: properties + + a1.sources = r1 + a1.channels = c1 + a1.sources.r1.type = org.apache.flume.source.twitter.TwitterSource + a1.sources.r1.channels = c1 + a1.sources.r1.consumerKey = YOUR_TWITTER_CONSUMER_KEY + a1.sources.r1.consumerSecret = YOUR_TWITTER_CONSUMER_SECRET + a1.sources.r1.accessToken = YOUR_TWITTER_ACCESS_TOKEN + a1.sources.r1.accessTokenSecret = YOUR_TWITTER_ACCESS_TOKEN_SECRET + a1.sources.r1.maxBatchSize = 10 + a1.sources.r1.maxBatchDurationMillis = 200 + Event Deserializers ''''''''''''''''''' diff --git a/flume-ng-sources/flume-twitter-source/pom.xml b/flume-ng-sources/flume-twitter-source/pom.xml new file mode 100644 index 0000000000..a5a27cfec2 --- /dev/null +++ b/flume-ng-sources/flume-twitter-source/pom.xml @@ -0,0 +1,61 @@ + + + + + 4.0.0 + + + flume-ng-sources + org.apache.flume + 1.5.0-SNAPSHOT + + + org.apache.flume.flume-ng-sources + flume-twitter-source + Flume Twitter Source + + + + org.apache.flume + flume-ng-core + + + + junit + junit + test + + + + org.twitter4j + twitter4j-core + + + org.twitter4j + twitter4j-media-support + + + org.twitter4j + twitter4j-stream + + + + + + diff --git a/flume-ng-sources/flume-twitter-source/src/main/java/org/apache/flume/source/twitter/TwitterSource.java b/flume-ng-sources/flume-twitter-source/src/main/java/org/apache/flume/source/twitter/TwitterSource.java new file mode 100644 index 0000000000..27b2c3ff3d --- /dev/null +++ b/flume-ng-sources/flume-twitter-source/src/main/java/org/apache/flume/source/twitter/TwitterSource.java @@ -0,0 +1,333 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, + * software distributed under the License is distributed on an + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + * KIND, either express or implied. See the License for the + * specific language governing permissions and limitations + * under the License. + */ +package org.apache.flume.source.twitter; + +import java.io.ByteArrayOutputStream; +import java.io.IOException; +import java.text.DecimalFormat; +import java.text.SimpleDateFormat; +import java.util.ArrayList; +import java.util.Arrays; +import java.util.List; + +import org.apache.avro.Schema; +import org.apache.avro.Schema.Field; +import org.apache.avro.Schema.Type; +import org.apache.avro.file.DataFileWriter; +import org.apache.avro.generic.GenericData.Record; +import org.apache.avro.generic.GenericDatumWriter; +import org.apache.avro.generic.GenericRecord; +import org.apache.avro.io.DatumWriter; +import org.apache.flume.Context; +import org.apache.flume.Event; +import org.apache.flume.EventDrivenSource; +import org.apache.flume.annotations.InterfaceAudience; +import org.apache.flume.annotations.InterfaceStability; +import org.apache.flume.conf.Configurable; +import org.apache.flume.event.EventBuilder; +import org.apache.flume.source.AbstractSource; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +import twitter4j.MediaEntity; +import twitter4j.StallWarning; +import twitter4j.Status; +import twitter4j.StatusDeletionNotice; +import twitter4j.StatusListener; +import twitter4j.TwitterStream; +import twitter4j.TwitterStreamFactory; +import twitter4j.User; +import twitter4j.auth.AccessToken; + +/** + * Demo Flume source that connects via Streaming API to the 1% sample twitter + * firehose, continously downloads tweets, converts them to Avro format and + * sends Avro events to a downstream Flume sink. + * + * Requires the consumer and access tokens and secrets of a Twitter developer + * account + */ + +@InterfaceAudience.Private +@InterfaceStability.Unstable +public class TwitterSource + extends AbstractSource + implements EventDrivenSource, Configurable, StatusListener { + + private TwitterStream twitterStream; + private Schema avroSchema; + + private long docCount = 0; + private long startTime = 0; + private long exceptionCount = 0; + private long totalTextIndexed = 0; + private long skippedDocs = 0; + private long batchEndTime = 0; + private final List docs = new ArrayList(); + private final ByteArrayOutputStream serializationBuffer = + new ByteArrayOutputStream(); + private DataFileWriter dataFileWriter; + + private int maxBatchSize = 1000; + private int maxBatchDurationMillis = 1000; + + // Fri May 14 02:52:55 +0000 2010 + private SimpleDateFormat formatterTo = + new SimpleDateFormat("yyyy-MM-dd'T'HH:mm:ss'Z'"); + private DecimalFormat numFormatter = new DecimalFormat("###,###.###"); + + private static int REPORT_INTERVAL = 100; + private static int STATS_INTERVAL = REPORT_INTERVAL * 10; + private static final Logger LOGGER = + LoggerFactory.getLogger(TwitterSource.class); + + public TwitterSource() { + } + + @Override + public void configure(Context context) { + String consumerKey = context.getString("consumerKey"); + String consumerSecret = context.getString("consumerSecret"); + String accessToken = context.getString("accessToken"); + String accessTokenSecret = context.getString("accessTokenSecret"); + + LOGGER.info("Consumer Key: '" + consumerKey + "'"); + LOGGER.info("Consumer Secret: '" + consumerSecret + "'"); + LOGGER.info("Access Token: '" + accessToken + "'"); + LOGGER.info("Access Token Secret: '" + accessTokenSecret + "'"); + + twitterStream = new TwitterStreamFactory().getInstance(); + twitterStream.setOAuthConsumer(consumerKey, consumerSecret); + twitterStream.setOAuthAccessToken(new AccessToken(accessToken, + accessTokenSecret)); + twitterStream.addListener(this); + avroSchema = createAvroSchema(); + dataFileWriter = new DataFileWriter( + new GenericDatumWriter(avroSchema)); + + maxBatchSize = context.getInteger("maxBatchSize", maxBatchSize); + maxBatchDurationMillis = context.getInteger("maxBatchDurationMillis", + maxBatchDurationMillis); + } + + @Override + public synchronized void start() { + LOGGER.info("Starting twitter source {} ...", this); + docCount = 0; + startTime = System.currentTimeMillis(); + exceptionCount = 0; + totalTextIndexed = 0; + skippedDocs = 0; + batchEndTime = System.currentTimeMillis() + maxBatchDurationMillis; + twitterStream.sample(); + LOGGER.info("Twitter source {} started.", getName()); + // This should happen at the end of the start method, since this will + // change the lifecycle status of the component to tell the Flume + // framework that this component has started. Doing this any earlier + // tells the framework that the component started successfully, even + // if the method actually fails later. + super.start(); + } + + @Override + public synchronized void stop() { + LOGGER.info("Twitter source {} stopping...", getName()); + twitterStream.shutdown(); + super.stop(); + LOGGER.info("Twitter source {} stopped.", getName()); + } + + public void onStatus(Status status) { + Record doc = extractRecord("", avroSchema, status); + if (doc == null) { + return; // skip + } + docs.add(doc); + if (docs.size() >= maxBatchSize || + System.currentTimeMillis() >= batchEndTime) { + batchEndTime = System.currentTimeMillis() + maxBatchDurationMillis; + byte[] bytes; + try { + bytes = serializeToAvro(avroSchema, docs); + } catch (IOException e) { + LOGGER.error("Exception while serializing tweet", e); + return; //skip + } + Event event = EventBuilder.withBody(bytes); + getChannelProcessor().processEvent(event); // send event to the flume sink + docs.clear(); + } + docCount++; + if ((docCount % REPORT_INTERVAL) == 0) { + LOGGER.info(String.format("Processed %s docs", + numFormatter.format(docCount))); + } + if ((docCount % STATS_INTERVAL) == 0) { + logStats(); + } + } + + private Schema createAvroSchema() { + Schema avroSchema = Schema.createRecord("Doc", "adoc", null, false); + List fields = new ArrayList(); + fields.add(new Field("id", Schema.create(Type.STRING), null, null)); + fields.add(new Field("user_friends_count", + createOptional(Schema.create(Type.INT)), + null, null)); + fields.add(new Field("user_location", + createOptional(Schema.create(Type.STRING)), + null, null)); + fields.add(new Field("user_description", + createOptional(Schema.create(Type.STRING)), + null, null)); + fields.add(new Field("user_statuses_count", + createOptional(Schema.create(Type.INT)), + null, null)); + fields.add(new Field("user_followers_count", + createOptional(Schema.create(Type.INT)), + null, null)); + fields.add(new Field("user_name", + createOptional(Schema.create(Type.STRING)), + null, null)); + fields.add(new Field("user_screen_name", + createOptional(Schema.create(Type.STRING)), + null, null)); + fields.add(new Field("created_at", + createOptional(Schema.create(Type.STRING)), + null, null)); + fields.add(new Field("text", + createOptional(Schema.create(Type.STRING)), + null, null)); + fields.add(new Field("retweet_count", + createOptional(Schema.create(Type.LONG)), + null, null)); + fields.add(new Field("retweeted", + createOptional(Schema.create(Type.BOOLEAN)), + null, null)); + fields.add(new Field("in_reply_to_user_id", + createOptional(Schema.create(Type.LONG)), + null, null)); + fields.add(new Field("source", + createOptional(Schema.create(Type.STRING)), + null, null)); + fields.add(new Field("in_reply_to_status_id", + createOptional(Schema.create(Type.LONG)), + null, null)); + fields.add(new Field("media_url_https", + createOptional(Schema.create(Type.STRING)), + null, null)); + fields.add(new Field("expanded_url", + createOptional(Schema.create(Type.STRING)), + null, null)); + avroSchema.setFields(fields); + return avroSchema; + } + + private Record extractRecord(String idPrefix, Schema avroSchema, Status status) { + User user = status.getUser(); + Record doc = new Record(avroSchema); + + doc.put("id", idPrefix + status.getId()); + doc.put("created_at", formatterTo.format(status.getCreatedAt())); + doc.put("retweet_count", status.getRetweetCount()); + doc.put("retweeted", status.isRetweet()); + doc.put("in_reply_to_user_id", status.getInReplyToUserId()); + doc.put("in_reply_to_status_id", status.getInReplyToStatusId()); + + addString(doc, "source", status.getSource()); + addString(doc, "text", status.getText()); + + MediaEntity[] mediaEntities = status.getMediaEntities(); + if (mediaEntities.length > 0) { + addString(doc, "media_url_https", mediaEntities[0].getMediaURLHttps()); + addString(doc, "expanded_url", mediaEntities[0].getExpandedURL()); + } + + doc.put("user_friends_count", user.getFriendsCount()); + doc.put("user_statuses_count", user.getStatusesCount()); + doc.put("user_followers_count", user.getFollowersCount()); + addString(doc, "user_location", user.getLocation()); + addString(doc, "user_description", user.getDescription()); + addString(doc, "user_screen_name", user.getScreenName()); + addString(doc, "user_name", user.getName()); + return doc; + } + + private byte[] serializeToAvro(Schema avroSchema, List docList) + throws IOException { + serializationBuffer.reset(); + dataFileWriter.create(avroSchema, serializationBuffer); + for (Record doc2 : docList) { + dataFileWriter.append(doc2); + } + dataFileWriter.close(); + return serializationBuffer.toByteArray(); + } + + private Schema createOptional(Schema schema) { + return Schema.createUnion(Arrays.asList( + new Schema[] { schema, Schema.create(Type.NULL) })); + } + + private void addString(Record doc, String avroField, String val) { + if (val == null) { + return; + } + doc.put(avroField, val); + totalTextIndexed += val.length(); + } + + private void logStats() { + double mbIndexed = totalTextIndexed / (1024 * 1024.0); + long seconds = (System.currentTimeMillis() - startTime) / 1000; + seconds = Math.max(seconds, 1); + LOGGER.info(String.format("Total docs indexed: %s, total skipped docs: %s", + numFormatter.format(docCount), numFormatter.format(skippedDocs))); + LOGGER.info(String.format(" %s docs/second", + numFormatter.format(docCount / seconds))); + LOGGER.info(String.format("Run took %s seconds and processed:", + numFormatter.format(seconds))); + LOGGER.info(String.format(" %s MB/sec sent to index", + numFormatter.format(((float) totalTextIndexed / (1024 * 1024)) / seconds))); + LOGGER.info(String.format(" %s MB text sent to index", + numFormatter.format(mbIndexed))); + LOGGER.info(String.format("There were %s exceptions ignored: ", + numFormatter.format(exceptionCount))); + } + + public void onDeletionNotice(StatusDeletionNotice statusDeletionNotice) { + // Do nothing... + } + + public void onScrubGeo(long userId, long upToStatusId) { + // Do nothing... + } + + public void onStallWarning(StallWarning warning) { + // Do nothing... + } + + public void onTrackLimitationNotice(int numberOfLimitedStatuses) { + // Do nothing... + } + + public void onException(Exception e) { + LOGGER.error("Exception while streaming tweets", e); + } +} diff --git a/flume-ng-sources/flume-twitter-source/src/test/java/org/apache/flume/source/twitter/TestTwitterSource.java b/flume-ng-sources/flume-twitter-source/src/test/java/org/apache/flume/source/twitter/TestTwitterSource.java new file mode 100644 index 0000000000..f6cc2c9f9e --- /dev/null +++ b/flume-ng-sources/flume-twitter-source/src/test/java/org/apache/flume/source/twitter/TestTwitterSource.java @@ -0,0 +1,112 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, + * software distributed under the License is distributed on an + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + * KIND, either express or implied. See the License for the + * specific language governing permissions and limitations + * under the License. + */ +package org.apache.flume.source.twitter; + +import java.net.InetAddress; +import java.net.UnknownHostException; +import java.text.SimpleDateFormat; +import java.util.Collections; +import java.util.HashMap; +import java.util.Map; + +import org.apache.flume.Channel; +import org.apache.flume.ChannelSelector; +import org.apache.flume.Context; +import org.apache.flume.Sink; +import org.apache.flume.SinkRunner; +import org.apache.flume.channel.ChannelProcessor; +import org.apache.flume.channel.MemoryChannel; +import org.apache.flume.channel.ReplicatingChannelSelector; +import org.apache.flume.conf.Configurables; +import org.apache.flume.sink.DefaultSinkProcessor; +import org.apache.flume.sink.LoggerSink; +import org.apache.flume.source.twitter.TwitterSource; +import org.junit.Assert; +import org.junit.Assume; +import org.junit.BeforeClass; +import org.junit.Test; + +public class TestTwitterSource extends Assert { + + @BeforeClass + public static void setUp() { + try { + Assume.assumeNotNull(InetAddress.getByName("stream.twitter.com")); + } catch (UnknownHostException e) { + Assume.assumeTrue(false); // ignore Test if twitter is unreachable + } + } + + @Test + public void testBasic() throws Exception { + String consumerKey = System.getProperty("twitter.consumerKey"); + Assume.assumeNotNull(consumerKey); + + String consumerSecret = System.getProperty("twitter.consumerSecret"); + Assume.assumeNotNull(consumerSecret); + + String accessToken = System.getProperty("twitter.accessToken"); + Assume.assumeNotNull(accessToken); + + String accessTokenSecret = System.getProperty("twitter.accessTokenSecret"); + Assume.assumeNotNull(accessTokenSecret); + + Context context = new Context(); + context.put("consumerKey", consumerKey); + context.put("consumerSecret", consumerSecret); + context.put("accessToken", accessToken); + context.put("accessTokenSecret", accessTokenSecret); + context.put("maxBatchDurationMillis", "1000"); + + TwitterSource source = new TwitterSource(); + source.configure(context); + + Map channelContext = new HashMap(); + channelContext.put("capacity", "1000000"); + channelContext.put("keep-alive", "0"); // for faster tests + Channel channel = new MemoryChannel(); + Configurables.configure(channel, new Context(channelContext)); + + Sink sink = new LoggerSink(); + sink.setChannel(channel); + sink.start(); + DefaultSinkProcessor proc = new DefaultSinkProcessor(); + proc.setSinks(Collections.singletonList(sink)); + SinkRunner sinkRunner = new SinkRunner(proc); + sinkRunner.start(); + + ChannelSelector rcs = new ReplicatingChannelSelector(); + rcs.setChannels(Collections.singletonList(channel)); + ChannelProcessor chp = new ChannelProcessor(rcs); + source.setChannelProcessor(chp); + source.start(); + + Thread.sleep(5000); + source.stop(); + sinkRunner.stop(); + sink.stop(); + } + + @Test + public void testCarrotDateFormatBug() throws Exception { + SimpleDateFormat formatterFrom = new SimpleDateFormat("EEE MMM dd HH:mm:ss Z yyyy"); + formatterFrom.parse("Fri Oct 26 22:53:55 +0000 2012"); + } + +} diff --git a/flume-ng-sources/flume-twitter-source/src/test/resources/log4j.properties b/flume-ng-sources/flume-twitter-source/src/test/resources/log4j.properties new file mode 100644 index 0000000000..7755024429 --- /dev/null +++ b/flume-ng-sources/flume-twitter-source/src/test/resources/log4j.properties @@ -0,0 +1,33 @@ +# Licensed to the Apache Software Foundation (ASF) under one +# or more contributor license agreements. See the NOTICE file +# distributed with this work for additional information +# regarding copyright ownership. The ASF licenses this file +# to you under the Apache License, Version 2.0 (the +# "License"); you may not use this file except in compliance +# with the License. You may obtain a copy of the License at +# +# http://www.apache.org/licenses/LICENSE-2.0 +# +# Unless required by applicable law or agreed to in writing, +# software distributed under the License is distributed on an +# "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY +# KIND, either express or implied. See the License for the +# specific language governing permissions and limitations +# under the License. +log4j.rootLogger=WARN, A1 + +log4j.logger.org.apache.flume.sink=INFO +#log4j.logger.org.apache.flume.sink.solr=DEBUG +log4j.logger.org.apache.solr=INFO +#log4j.logger.org.apache.solr.hadoop=DEBUG +log4j.logger.org.apache.solr.morphline=DEBUG +log4j.logger.org.apache.solr.update.processor.LogUpdateProcessor=WARN +log4j.logger.org.apache.solr.core.SolrCore=WARN +log4j.logger.org.apache.solr.search.SolrIndexSearcher=ERROR + +# A1 is set to be a ConsoleAppender. +log4j.appender.A1=org.apache.log4j.ConsoleAppender + +# A1 uses PatternLayout. +log4j.appender.A1.layout=org.apache.log4j.PatternLayout +log4j.appender.A1.layout.ConversionPattern=%-4r [%t] %-5p %c %x - %m%n diff --git a/flume-ng-sources/flume-twitter-source/src/test/resources/twitter-flume.conf b/flume-ng-sources/flume-twitter-source/src/test/resources/twitter-flume.conf new file mode 100644 index 0000000000..72fe4efc9f --- /dev/null +++ b/flume-ng-sources/flume-twitter-source/src/test/resources/twitter-flume.conf @@ -0,0 +1,92 @@ +# Licensed under the Apache License, Version 2.0 (the "License"); +# you may not use this file except in compliance with the License. +# You may obtain a copy of the License at +# +# http://www.apache.org/licenses/LICENSE-2.0 +# +# Unless required by applicable law or agreed to in writing, software +# distributed under the License is distributed on an "AS IS" BASIS, +# WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +# See the License for the specific language governing permissions and +# limitations under the License. + + +# The configuration file needs to define the sources, +# the channels and the sinks. +# Sources, channels and sinks are defined per agent, +# in this case called 'agent' + +agent.sources = twitterSrc +#agent.sources = httpSrc +#agent.sources = spoolSrc +#agent.sources = avroSrc +agent.channels = memoryChannel +agent.sinks = solrSink +#agent.sinks = loggerSink + +agent.sources.twitterSrc.type = org.apache.flume.source.twitter.TwitterSource +agent.sources.twitterSrc.consumerKey = YOUR_TWITTER_CONSUMER_KEY +agent.sources.twitterSrc.consumerSecret = YOUR_TWITTER_CONSUMER_SECRET +agent.sources.twitterSrc.accessToken = YOUR_TWITTER_ACCESS_TOKEN +agent.sources.twitterSrc.accessTokenSecret = YOUR_TWITTER_ACCESS_TOKEN_SECRET +agent.sources.twitterSrc.maxBatchDurationMillis = 200 +agent.sources.twitterSrc.channels = memoryChannel + +agent.sources.httpSrc.type = org.apache.flume.source.http.HTTPSource +agent.sources.httpSrc.port = 5140 +agent.sources.httpSrc.handler = org.apache.flume.sink.solr.morphline.BlobHandler +agent.sources.httpSrc.handler.maxBlobLength = 2000000000 +#agent.sources.httpSrc.interceptors = uuidinterceptor +#agent.sources.httpSrc.interceptors.uuidinterceptor.type = org.apache.flume.sink.solr.morphline.UUIDInterceptor$Builder +#agent.sources.httpSrc.interceptors.uuidinterceptor.headerName = id +##agent.sources.httpSrc.interceptors.uuidinterceptor.preserveExisting = false +##agent.sources.httpSrc.interceptors.uuidinterceptor.prefix = myhostname +agent.sources.httpSrc.channels = memoryChannel + +agent.sources.spoolSrc.type = spooldir +agent.sources.spoolSrc.spoolDir = /tmp/myspooldir +agent.sources.spoolSrc.ignorePattern = \. +agent.sources.spoolSrc.deserializer = org.apache.flume.sink.solr.morphline.BlobDeserializer$Builder +agent.sources.spoolSrc.deserializer.maxBlobLength = 2000000000 +agent.sources.spoolSrc.batchSize = 1 +agent.sources.spoolSrc.fileHeader = true +agent.sources.spoolSrc.fileHeaderKey = resourceName +#agent.sources.spoolSrc.interceptors = uuidinterceptor +#agent.sources.spoolSrc.interceptors.uuidinterceptor.type = org.apache.flume.sink.solr.morphline.UUIDInterceptor$Builder +#agent.sources.spoolSrc.interceptors.uuidinterceptor.headerName = id +##agent.sources.spoolSrc.interceptors.uuidinterceptor.preserveExisting = false +##agent.sources.spoolSrc.interceptors.uuidinterceptor.prefix = myhostname +agent.sources.spoolSrc.channels = memoryChannel + +agent.sources.avroSrc.type = avro +agent.sources.avroSrc.bind = 127.0.0.1 +agent.sources.avroSrc.port = 10000 +agent.sources.avroSrc.channels = memoryChannel +agent.sources.avroSrc.interceptors = uuidinterceptor +agent.sources.avroSrc.interceptors.uuidinterceptor.type = org.apache.flume.sink.solr.morphline.UUIDInterceptor$Builder +agent.sources.avroSrc.interceptors.uuidinterceptor.headerName = id +#agent.sources.avroSrc.interceptors.uuidinterceptor.preserveExisting = false +#agent.sources.avroSrc.interceptors.uuidinterceptor.prefix = myhostname +#agent.sources.avroSrc.interceptors = morphlineinterceptor +#agent.sources.avroSrc.interceptors.morphlineinterceptor.type = org.apache.flume.sink.solr.morphline.MorphlineInterceptor$Builder +#agent.sources.avroSrc.interceptors.morphlineinterceptor.morphlineFile = /etc/flume-ng/conf/morphline.conf +#agent.sources.avroSrc.interceptors.morphlineinterceptor.morphlineId = morphline1 + +agent.channels.memoryChannel.type = memory +agent.channels.memoryChannel.capacity = 10000 +agent.channels.memoryChannel.transactionCapacity = 1000 + +#agent.channels.fileChannel.type = file +#agent.channels.fileChannel.capacity = 1000000 +#agent.channels.fileChannel.transactionCapacity = 1000 +#agent.channels.fileChannel.write-timeout = 1 + +agent.sinks.solrSink.type = org.apache.flume.sink.solr.morphline.MorphlineSolrSink +agent.sinks.solrSink.channel = memoryChannel +#agent.sinks.solrSink.batchSize = 1000 +#agent.sinks.solrSink.batchDurationMillis = 1000 +agent.sinks.solrSink.morphlineFile = /etc/flume-ng/conf/morphline.conf +#agent.sinks.solrSink.morphlineId = morphline1 + +#agent.sinks.loggerSink.type = logger +#agent.sinks.loggerSink.channel = memoryChannel diff --git a/flume-ng-sources/pom.xml b/flume-ng-sources/pom.xml index 6006fa1c25..0b57d6d44f 100644 --- a/flume-ng-sources/pom.xml +++ b/flume-ng-sources/pom.xml @@ -43,6 +43,7 @@ limitations under the License. flume-scribe-source flume-jms-source + flume-twitter-source diff --git a/pom.xml b/pom.xml index 8b3640268f..f0fd22ecb7 100644 --- a/pom.xml +++ b/pom.xml @@ -967,6 +967,12 @@ limitations under the License. 1.5.0-SNAPSHOT + + org.apache.flume.flume-ng-sources + flume-twitter-source + 1.5.0-SNAPSHOT + + org.apache.flume.flume-ng-legacy-sources flume-thrift-source @@ -1058,6 +1064,23 @@ limitations under the License. 4.3.0 + + + org.twitter4j + twitter4j-core + 3.0.3 + + + org.twitter4j + twitter4j-media-support + 3.0.3 + + + org.twitter4j + twitter4j-stream + 3.0.3 + + From ffa706429186df2cf8ad04fd9dcba37b6a35d7f1 Mon Sep 17 00:00:00 2001 From: Mike Percy Date: Fri, 27 Sep 2013 17:01:41 -0700 Subject: [PATCH 021/341] FLUME-2182. Spooling Directory Source will not ingest data completely when a wide character appears at the edge of a buffer (Sven Meys via Mike Percy) --- .../ResettableFileInputStream.java | 9 +++- .../TestResettableFileInputStream.java | 46 +++++++++++++++++++ 2 files changed, 54 insertions(+), 1 deletion(-) diff --git a/flume-ng-core/src/main/java/org/apache/flume/serialization/ResettableFileInputStream.java b/flume-ng-core/src/main/java/org/apache/flume/serialization/ResettableFileInputStream.java index 09f490f257..ecea5e205a 100644 --- a/flume-ng-core/src/main/java/org/apache/flume/serialization/ResettableFileInputStream.java +++ b/flume-ng-core/src/main/java/org/apache/flume/serialization/ResettableFileInputStream.java @@ -64,6 +64,7 @@ public class ResettableFileInputStream extends ResettableInputStream private final CharsetDecoder decoder; private long position; private long syncPosition; + private int maxCharWidth; /** * @@ -112,6 +113,7 @@ public ResettableFileInputStream(File file, PositionTracker tracker, this.decoder = charset.newDecoder(); this.position = 0; this.syncPosition = 0; + this.maxCharWidth = (int)Math.ceil(charset.newEncoder().maxBytesPerChar()); seek(tracker.getPosition()); } @@ -152,7 +154,12 @@ public synchronized int read(byte[] b, int off, int len) throws IOException { @Override public synchronized int readChar() throws IOException { - if (!buf.hasRemaining()) { + // The decoder can have issues with multi-byte characters. + // This check ensures that there are at least maxCharWidth bytes in the buffer + // before reaching EOF. + if (buf.remaining() < maxCharWidth) { + buf.clear(); + buf.flip(); refillBuf(); } diff --git a/flume-ng-core/src/test/java/org/apache/flume/serialization/TestResettableFileInputStream.java b/flume-ng-core/src/test/java/org/apache/flume/serialization/TestResettableFileInputStream.java index 5ad6a0a70e..066765c692 100644 --- a/flume-ng-core/src/test/java/org/apache/flume/serialization/TestResettableFileInputStream.java +++ b/flume-ng-core/src/test/java/org/apache/flume/serialization/TestResettableFileInputStream.java @@ -18,6 +18,7 @@ package org.apache.flume.serialization; import com.google.common.base.Charsets; +import com.google.common.base.Strings; import com.google.common.collect.Lists; import com.google.common.io.Files; import junit.framework.Assert; @@ -86,6 +87,27 @@ public void testBasicRead() throws IOException { in.close(); } + /** + * Ensure that we can process lines that contain multi byte characters in weird places + * such as at the end of a buffer. + * @throws IOException + */ + @Test + public void testWideCharRead() throws IOException { + String output = wideCharFileInit(file, Charsets.UTF_8); + + PositionTracker tracker = new DurablePositionTracker(meta, file.getPath()); + ResettableInputStream in = new ResettableFileInputStream(file, tracker); + + String result = readLine(in, output.length()); + assertEquals(output, result); + + String afterEOF = readLine(in, output.length()); + assertNull(afterEOF); + + in.close(); + } + /** * Ensure a reset() brings us back to the default mark (beginning of file) * @throws IOException @@ -229,6 +251,30 @@ public void testSeek() throws IOException { assertEquals(11, Integer.parseInt(readLine(in, LINE_LEN).substring(0, 10))); } + /** + * Helper method that generates a line to test if parts of multi-byte characters on the + * edge of a buffer are handled properly. + */ + private static String generateWideCharLine(){ + String s = "éllo Wörld!\n"; + int size = (ResettableFileInputStream.DEFAULT_BUF_SIZE - 1) + s.length(); + return Strings.padStart(s, size , 'H'); + } + + /** + * Creates a file that contains a line that contains wide characters + * @param file + * @param charset + * @return + * @throws IOException + */ + private static String wideCharFileInit(File file, Charset charset) + throws IOException { + String output = generateWideCharLine(); + Files.write(output.getBytes(charset), file); + return output; + } + /** * Helper function to read a line from a character stream. * @param in From 49933493f53633fec5846edf88aca4dd0dfdd52a Mon Sep 17 00:00:00 2001 From: Mike Percy Date: Fri, 27 Sep 2013 21:48:05 -0700 Subject: [PATCH 022/341] FLUME-2198. Avro Source should disable itself if ipFilterRules contains invalid rules (Hari Shreedharan via Mike Percy) --- .../org/apache/flume/source/AvroSource.java | 125 +++++++-------- .../apache/flume/source/TestAvroSource.java | 151 ++++++++++-------- 2 files changed, 141 insertions(+), 135 deletions(-) diff --git a/flume-ng-core/src/main/java/org/apache/flume/source/AvroSource.java b/flume-ng-core/src/main/java/org/apache/flume/source/AvroSource.java index f6e4cfe693..c1ee3a93ce 100644 --- a/flume-ng-core/src/main/java/org/apache/flume/source/AvroSource.java +++ b/flume-ng-core/src/main/java/org/apache/flume/source/AvroSource.java @@ -59,6 +59,7 @@ import org.jboss.netty.channel.Channels; import org.jboss.netty.handler.codec.compression.ZlibDecoder; import org.jboss.netty.handler.codec.compression.ZlibEncoder; +import org.jboss.netty.handler.ipfilter.IpFilterRule; import org.jboss.netty.handler.ipfilter.IpFilterRuleHandler; import org.jboss.netty.handler.ipfilter.PatternRule; import org.jboss.netty.handler.ssl.SslHandler; @@ -153,6 +154,8 @@ public class AvroSource extends AbstractSource implements EventDrivenSource, private int maxThreads; private ScheduledExecutorService connectionCountUpdater; + private List rules; + @Override public void configure(Context context) { Configurables.ensureRequiredNonNull(context, PORT_KEY, BIND_KEY); @@ -191,11 +194,17 @@ public void configure(Context context) { if (enableIpFilter) { patternRuleConfigDefinition = context.getString(IP_FILTER_RULES_KEY); if (patternRuleConfigDefinition == null || - patternRuleConfigDefinition.isEmpty()) { + patternRuleConfigDefinition.trim().isEmpty()) { throw new FlumeException( "ipFilter is configured with true but ipFilterRules is not defined:" + " "); } + String[] patternRuleDefinitions = patternRuleConfigDefinition.split( + ","); + rules = new ArrayList(patternRuleDefinitions.length); + for (String patternRuleDefinition : patternRuleDefinitions) { + rules.add(generateRule(patternRuleDefinition)); + } } if (sourceCounter == null) { @@ -369,11 +378,53 @@ public Status appendBatch(List events) { return Status.OK; } + private PatternRule generateRule( + String patternRuleDefinition) throws FlumeException { + patternRuleDefinition = patternRuleDefinition.trim(); + //first validate the format + int firstColonIndex = patternRuleDefinition.indexOf(":"); + if (firstColonIndex == -1) { + throw new FlumeException( + "Invalid ipFilter patternRule '" + patternRuleDefinition + + "' should look like <'allow' or 'deny'>:<'ip' or " + + "'name'>:"); + } else { + String ruleAccessFlag = patternRuleDefinition.substring(0, + firstColonIndex); + int secondColonIndex = patternRuleDefinition.indexOf(":", + firstColonIndex + 1); + if ((!ruleAccessFlag.equals("allow") && + !ruleAccessFlag.equals("deny")) || secondColonIndex == -1) { + throw new FlumeException( + "Invalid ipFilter patternRule '" + patternRuleDefinition + + "' should look like <'allow' or 'deny'>:<'ip' or " + + "'name'>:"); + } + + String patternTypeFlag = patternRuleDefinition.substring( + firstColonIndex + 1, secondColonIndex); + if ((!patternTypeFlag.equals("ip") && + !patternTypeFlag.equals("name"))) { + throw new FlumeException( + "Invalid ipFilter patternRule '" + patternRuleDefinition + + "' should look like <'allow' or 'deny'>:<'ip' or " + + "'name'>:"); + } + + boolean isAllow = ruleAccessFlag.equals("allow"); + String patternRuleString = (patternTypeFlag.equals("ip") ? "i" : "n") + + ":" + patternRuleDefinition.substring(secondColonIndex + 1); + logger.info("Adding ipFilter PatternRule: " + + (isAllow ? "Allow" : "deny") + " " + patternRuleString); + return new PatternRule(isAllow, patternRuleString); + } + } + /** * Factory of SSL-enabled server worker channel pipelines * Copied from Avro's org.apache.avro.ipc.TestNettyServerWithSSL test */ - private static class AdvancedChannelPipelineFactory + private class AdvancedChannelPipelineFactory implements ChannelPipelineFactory { private boolean enableCompression; @@ -448,23 +499,7 @@ public ChannelPipeline getPipeline() throws Exception { logger.info("Setting up ipFilter with the following rule definition: " + patternRuleConfigDefinition); IpFilterRuleHandler ipFilterHandler = new IpFilterRuleHandler(); - - if (patternRuleConfigDefinition != null && - !patternRuleConfigDefinition.isEmpty()) { - String[] patternRuleDefinitions = patternRuleConfigDefinition.split( - ","); - for (String patternRuleDefinition : patternRuleDefinitions) { - - PatternRule patternRule - = PatternRuleBuilder.withConfigRuleDefinition( - patternRuleDefinition); - - if (patternRule != null) { - ipFilterHandler.add(patternRule); - } - } - } - + ipFilterHandler.addAll(rules); logger.info( "Adding ipFilter with " + ipFilterHandler.size() + " rules"); @@ -473,57 +508,5 @@ public ChannelPipeline getPipeline() throws Exception { return pipeline; } - - public static class PatternRuleBuilder { - public static PatternRule withConfigRuleDefinition( - String patternRuleDefinition) throws FlumeException { - patternRuleDefinition = patternRuleDefinition.trim(); - //first validation the format - - int firstColonIndex = patternRuleDefinition.indexOf(":"); - if (firstColonIndex == -1) { - logger.error( - "Invalid ipFilter patternRule '" + patternRuleDefinition + - "' should look like <'allow' or 'deny'>:<'ip' or " + - "'name'>:"); - return null; - } else { - - String ruleAccessFlag = patternRuleDefinition.substring(0, - firstColonIndex); - int secondColonIndex = patternRuleDefinition.indexOf(":", - firstColonIndex + 1); - if ((!ruleAccessFlag.equals("allow") && - !ruleAccessFlag.equals("deny")) || secondColonIndex == -1) { - logger.error( - "Invalid ipFilter patternRule '" + patternRuleDefinition + - "' should look like <'allow' or 'deny'>:<'ip' or " + - "'name'>:"); - return null; - } - - String patternTypeFlag = patternRuleDefinition.substring( - firstColonIndex + 1, secondColonIndex); - if ((!patternTypeFlag.equals("ip") && - !patternTypeFlag.equals("name"))) { - logger.error( - "Invalid ipFilter patternRule '" + patternRuleDefinition + - "' should look like <'allow' or 'deny'>:<'ip' or " + - "'name'>:"); - return null; - } - - boolean isAllow = ruleAccessFlag.equals("allow"); - String patternRuleString = - (patternTypeFlag.equals("ip") ? "i" : "n") + ":" + - patternRuleDefinition.substring(secondColonIndex + 1); - logger.info("Adding ipFilter PatternRule: " - + (isAllow ? "Allow" : "deny") + - " " + patternRuleString); - return new PatternRule(isAllow, patternRuleString); - } - } - } - } } diff --git a/flume-ng-core/src/test/java/org/apache/flume/source/TestAvroSource.java b/flume-ng-core/src/test/java/org/apache/flume/source/TestAvroSource.java index e208fffafb..c75d098141 100644 --- a/flume-ng-core/src/test/java/org/apache/flume/source/TestAvroSource.java +++ b/flume-ng-core/src/test/java/org/apache/flume/source/TestAvroSource.java @@ -21,13 +21,14 @@ import java.io.IOException; import java.net.Inet4Address; +import java.net.InetAddress; import java.net.InetSocketAddress; +import java.net.UnknownHostException; import java.nio.ByteBuffer; import java.security.cert.X509Certificate; import java.util.ArrayList; import java.util.HashMap; import java.util.List; -import java.util.concurrent.Executor; import java.util.concurrent.Executors; import javax.net.ssl.SSLContext; import javax.net.ssl.SSLEngine; @@ -72,9 +73,11 @@ public class TestAvroSource { private int selectedPort; private AvroSource source; private Channel channel; + private InetAddress localhost; @Before - public void setUp() { + public void setUp() throws UnknownHostException { + localhost = InetAddress.getByName("127.0.0.1"); source = new AvroSource(); channel = new MemoryChannel(); @@ -383,65 +386,84 @@ public X509Certificate[] getAcceptedIssuers() { } @Test - public void testValidIpFilterAllows() throws InterruptedException, IOException { - - doIpFilterTest("allow:name:localhost,deny:ip:*", true, false); - doIpFilterTest("allow:ip:" + Inet4Address.getLocalHost().getHostAddress() + ",deny:ip:*", true, false); - doIpFilterTest("allow:ip:*", true, false); - doIpFilterTest("allow:ip:" + Inet4Address.getLocalHost().getHostAddress().substring(0, 3) + "*,deny:ip:*", true, false); - doIpFilterTest("allow:ip:127.0.0.2,allow:ip:" + Inet4Address.getLocalHost().getHostAddress().substring(0, 3) + "*,deny:ip:*", true, false); - - doIpFilterTest("allow:name:localhost,deny:ip:*", true, true); - doIpFilterTest("allow:ip:*", true, true); - + public void testValidIpFilterAllows() + throws InterruptedException, IOException { + doIpFilterTest(localhost, "allow:name:localhost,deny:ip:*", true, false); + doIpFilterTest(localhost, "allow:ip:" + localhost.getHostAddress() + + ",deny:ip:*", true, false); + doIpFilterTest(localhost, "allow:ip:*", true, false); + doIpFilterTest(localhost, "allow:ip:" + + localhost.getHostAddress().substring(0, 3) + + "*,deny:ip:*", true, false); + doIpFilterTest(localhost, "allow:ip:127.0.0.2,allow:ip:" + + localhost.getHostAddress().substring(0, 3) + + "*,deny:ip:*", true, false); + doIpFilterTest(localhost, "allow:name:localhost,deny:ip:*", true, true); + doIpFilterTest(localhost, "allow:ip:*", true, true); } @Test - public void testValidIpFilterDenys() throws InterruptedException, IOException { - - doIpFilterTest("deny:ip:*", false, false); - doIpFilterTest("deny:name:localhost", false, false); - doIpFilterTest("deny:ip:" + Inet4Address.getLocalHost().getHostAddress() + ",allow:ip:*", false, false); - doIpFilterTest("deny:ip:*", false, false); - doIpFilterTest("allow:ip:45.2.2.2,deny:ip:*", false, false); - doIpFilterTest("deny:ip:" + Inet4Address.getLocalHost().getHostAddress().substring(0, 3) + "*,allow:ip:*", false, false); - - - doIpFilterTest("deny:ip:*", false, true); + public void testValidIpFilterDenys() + throws InterruptedException, IOException { + doIpFilterTest(localhost, "deny:ip:*", false, false); + doIpFilterTest(localhost, "deny:name:localhost", false, false); + doIpFilterTest(localhost, "deny:ip:" + localhost.getHostAddress() + + ",allow:ip:*", false, false); + doIpFilterTest(localhost, "deny:ip:*", false, false); + doIpFilterTest(localhost, "allow:ip:45.2.2.2,deny:ip:*", false, false); + doIpFilterTest(localhost, "deny:ip:" + + localhost.getHostAddress().substring(0, 3) + + "*,allow:ip:*", false, false); + doIpFilterTest(localhost, "deny:ip:*", false, true); } @Test public void testInvalidIpFilter() throws InterruptedException, IOException { - - doIpFilterTest("deny:ip?*", true, false); - doIpFilterTest("deny?name:localhost", true, false); - doIpFilterTest("deny:ip:127.0.0.2,allow:ip?*,deny:ip:" + Inet4Address.getLocalHost().getHostAddress() + "", false, false); - doIpFilterTest("deny:*", true, false); - doIpFilterTest("deny:id:" + Inet4Address.getLocalHost().getHostAddress().substring(0, 3) + "*,allow:ip:*", true, false); + doIpFilterTest(localhost, "deny:ip:*", false, false); + doIpFilterTest(localhost, "allow:name:localhost", true, false); + doIpFilterTest(localhost, "deny:ip:127.0.0.2,allow:ip:*,deny:ip:" + + localhost.getHostAddress(), true, false); + doIpFilterTest(localhost, "deny:ip:" + + localhost.getHostAddress().substring(0, 3) + "*,allow:ip:*", + false, false); try { - doIpFilterTest(null, true, false); - Assert.fail("The null ipFilterRules config should had thrown an exception."); + doIpFilterTest(localhost, null, false, false); + Assert.fail( + "The null ipFilterRules config should have thrown an exception."); } catch (FlumeException e) { //Do nothing } - try{ - doIpFilterTest("", true, false); - Assert.fail("The empty string ipFilterRules config should had thrown an exception."); - } catch (FlumeException e) { + try { + doIpFilterTest(localhost, "", true, false); + Assert.fail("The empty string ipFilterRules config should have thrown " + + "an exception"); + } catch (FlumeException e) { //Do nothing } - + try { + doIpFilterTest(localhost, "homer:ip:45.4.23.1", true, false); + Assert.fail("Bad ipFilterRules config should have thrown an exception."); + } catch (FlumeException e) { + //Do nothing + } + try { + doIpFilterTest(localhost, "allow:sleeps:45.4.23.1", true, false); + Assert.fail("Bad ipFilterRules config should have thrown an exception."); + } catch (FlumeException e) { + //Do nothing + } } - public void doIpFilterTest(String ruleDefinition, boolean eventShouldBeAllowed, boolean testWithSSL) throws InterruptedException, IOException { + public void doIpFilterTest(InetAddress dest, String ruleDefinition, + boolean eventShouldBeAllowed, boolean testWithSSL) + throws InterruptedException, IOException { boolean bound = false; for (int i = 0; i < 100 && !bound; i++) { try { Context context = new Context(); - context.put("port", String.valueOf(selectedPort = 41414 + i)); context.put("bind", "0.0.0.0"); context.put("ipFilter", "true"); @@ -476,34 +498,41 @@ public void doIpFilterTest(String ruleDefinition, boolean eventShouldBeAllowed, source.getLifecycleState()); AvroSourceProtocol client; - NettyTransceiver nettyTransceiver; - - if (testWithSSL) { - nettyTransceiver = new NettyTransceiver(new InetSocketAddress(selectedPort), new SSLChannelFactory()); - client = SpecificRequestor.getClient( - AvroSourceProtocol.class, nettyTransceiver ); - } else { - nettyTransceiver = new NettyTransceiver(new InetSocketAddress(selectedPort)); - client = SpecificRequestor.getClient( + NettyTransceiver nettyTransceiver = null; + try { + if (testWithSSL) { + nettyTransceiver = new NettyTransceiver( + new InetSocketAddress (dest, selectedPort), + new SSLChannelFactory()); + client = SpecificRequestor.getClient( AvroSourceProtocol.class, nettyTransceiver); - } + } else { + nettyTransceiver = new NettyTransceiver( + new InetSocketAddress (dest, selectedPort)); + client = SpecificRequestor.getClient( + AvroSourceProtocol.class, nettyTransceiver); + } - AvroFlumeEvent avroEvent = new AvroFlumeEvent(); - avroEvent.setHeaders(new HashMap()); - avroEvent.setBody(ByteBuffer.wrap("Hello avro ipFilter".getBytes())); + AvroFlumeEvent avroEvent = new AvroFlumeEvent(); + avroEvent.setHeaders(new HashMap()); + avroEvent.setBody(ByteBuffer.wrap("Hello avro ipFilter".getBytes())); - try { logger.info("Client about to append"); Status status = client.append(avroEvent); logger.info("Client appended"); Assert.assertEquals(Status.OK, status); - } catch(IOException e) { - Assert.assertTrue("Should have been Allowed:" + ruleDefinition, !eventShouldBeAllowed); + } catch (IOException e) { + Assert.assertTrue("Should have been allowed: " + ruleDefinition, + !eventShouldBeAllowed); return; + } finally { + if (nettyTransceiver != null) { + nettyTransceiver.close(); + } + source.stop(); } - Assert.assertTrue("Should have been denied:" + ruleDefinition, eventShouldBeAllowed); - - + Assert.assertTrue("Should have been denied: " + ruleDefinition, + eventShouldBeAllowed); Transaction transaction = channel.getTransaction(); transaction.begin(); @@ -514,17 +543,11 @@ public void doIpFilterTest(String ruleDefinition, boolean eventShouldBeAllowed, new String(event.getBody())); transaction.commit(); transaction.close(); - logger.debug("Round trip event:{}", event); - nettyTransceiver.close(); - source.stop(); Assert.assertTrue("Reached stop or error", LifecycleController.waitForOneOf(source, LifecycleState.STOP_OR_ERROR)); Assert.assertEquals("Server is stopped", LifecycleState.STOP, source.getLifecycleState()); - - } - } From b84d01615a47c8152cfa1119a52a1a1f1b445843 Mon Sep 17 00:00:00 2001 From: Hari Shreedharan Date: Sat, 28 Sep 2013 09:35:03 -0700 Subject: [PATCH 023/341] FLUME-2052. Spooling directory source should be able to replace or ignore malformed characters (Mike Percy via Hari Shreedharan) --- .../avro/ReliableSpoolingFileEventReader.java | 32 ++++- .../serialization/DecodeErrorPolicy.java | 31 +++++ .../ResettableFileInputStream.java | 23 +++- .../flume/source/SpoolDirectorySource.java | 6 + ...DirectorySourceConfigurationConstants.java | 8 ++ .../TestResettableFileInputStream.java | 118 +++++++++++++++++- flume-ng-doc/sphinx/FlumeUserGuide.rst | 5 + 7 files changed, 215 insertions(+), 8 deletions(-) create mode 100644 flume-ng-core/src/main/java/org/apache/flume/serialization/DecodeErrorPolicy.java diff --git a/flume-ng-core/src/main/java/org/apache/flume/client/avro/ReliableSpoolingFileEventReader.java b/flume-ng-core/src/main/java/org/apache/flume/client/avro/ReliableSpoolingFileEventReader.java index 724ab38f1a..bd684ed848 100644 --- a/flume-ng-core/src/main/java/org/apache/flume/client/avro/ReliableSpoolingFileEventReader.java +++ b/flume-ng-core/src/main/java/org/apache/flume/client/avro/ReliableSpoolingFileEventReader.java @@ -24,7 +24,10 @@ import java.io.FileNotFoundException; import java.io.IOException; import java.nio.charset.Charset; -import java.util.*; +import java.util.Arrays; +import java.util.Collections; +import java.util.Comparator; +import java.util.List; import java.util.regex.Pattern; import org.apache.flume.Context; @@ -32,7 +35,13 @@ import org.apache.flume.FlumeException; import org.apache.flume.annotations.InterfaceAudience; import org.apache.flume.annotations.InterfaceStability; -import org.apache.flume.serialization.*; +import org.apache.flume.serialization.DecodeErrorPolicy; +import org.apache.flume.serialization.DurablePositionTracker; +import org.apache.flume.serialization.EventDeserializer; +import org.apache.flume.serialization.EventDeserializerFactory; +import org.apache.flume.serialization.PositionTracker; +import org.apache.flume.serialization.ResettableFileInputStream; +import org.apache.flume.serialization.ResettableInputStream; import org.apache.flume.source.SpoolDirectorySourceConfigurationConstants; import org.apache.flume.tools.PlatformDetect; import org.slf4j.Logger; @@ -86,6 +95,7 @@ public class ReliableSpoolingFileEventReader implements ReliableEventReader { private final String fileNameHeader; private final String deletePolicy; private final Charset inputCharset; + private final DecodeErrorPolicy decodeErrorPolicy; private Optional currentFile = Optional.absent(); /** Always contains the last file from which lines have been read. **/ @@ -99,7 +109,8 @@ private ReliableSpoolingFileEventReader(File spoolDirectory, String completedSuffix, String ignorePattern, String trackerDirPath, boolean annotateFileName, String fileNameHeader, String deserializerType, Context deserializerContext, - String deletePolicy, String inputCharset) throws IOException { + String deletePolicy, String inputCharset, + DecodeErrorPolicy decodeErrorPolicy) throws IOException { // Sanity checks Preconditions.checkNotNull(spoolDirectory); @@ -156,6 +167,7 @@ private ReliableSpoolingFileEventReader(File spoolDirectory, this.ignorePattern = Pattern.compile(ignorePattern); this.deletePolicy = deletePolicy; this.inputCharset = Charset.forName(inputCharset); + this.decodeErrorPolicy = Preconditions.checkNotNull(decodeErrorPolicy); File trackerDirectory = new File(trackerDirPath); @@ -432,7 +444,8 @@ public int compare(File a, File b) { ResettableInputStream in = new ResettableFileInputStream(nextFile, tracker, - ResettableFileInputStream.DEFAULT_BUF_SIZE, inputCharset); + ResettableFileInputStream.DEFAULT_BUF_SIZE, inputCharset, + decodeErrorPolicy); EventDeserializer deserializer = EventDeserializerFactory.getInstance (deserializerType, deserializerContext, in); @@ -504,6 +517,9 @@ public static class Builder { SpoolDirectorySourceConfigurationConstants.DEFAULT_DELETE_POLICY; private String inputCharset = SpoolDirectorySourceConfigurationConstants.DEFAULT_INPUT_CHARSET; + private DecodeErrorPolicy decodeErrorPolicy = DecodeErrorPolicy.valueOf( + SpoolDirectorySourceConfigurationConstants.DEFAULT_DECODE_ERROR_POLICY + .toUpperCase()); public Builder spoolDirectory(File directory) { this.spoolDirectory = directory; @@ -555,10 +571,16 @@ public Builder inputCharset(String inputCharset) { return this; } + public Builder decodeErrorPolicy(DecodeErrorPolicy decodeErrorPolicy) { + this.decodeErrorPolicy = decodeErrorPolicy; + return this; + } + public ReliableSpoolingFileEventReader build() throws IOException { return new ReliableSpoolingFileEventReader(spoolDirectory, completedSuffix, ignorePattern, trackerDirPath, annotateFileName, fileNameHeader, - deserializerType, deserializerContext, deletePolicy, inputCharset); + deserializerType, deserializerContext, deletePolicy, inputCharset, + decodeErrorPolicy); } } diff --git a/flume-ng-core/src/main/java/org/apache/flume/serialization/DecodeErrorPolicy.java b/flume-ng-core/src/main/java/org/apache/flume/serialization/DecodeErrorPolicy.java new file mode 100644 index 0000000000..a55b6b6ee7 --- /dev/null +++ b/flume-ng-core/src/main/java/org/apache/flume/serialization/DecodeErrorPolicy.java @@ -0,0 +1,31 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, + * software distributed under the License is distributed on an + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + * KIND, either express or implied. See the License for the + * specific language governing permissions and limitations + * under the License. + */ + +package org.apache.flume.serialization; + +import org.apache.flume.annotations.InterfaceAudience; +import org.apache.flume.annotations.InterfaceStability; + +@InterfaceAudience.Private +@InterfaceStability.Evolving +public enum DecodeErrorPolicy { + FAIL, + REPLACE, + IGNORE +} diff --git a/flume-ng-core/src/main/java/org/apache/flume/serialization/ResettableFileInputStream.java b/flume-ng-core/src/main/java/org/apache/flume/serialization/ResettableFileInputStream.java index ecea5e205a..587ab299b8 100644 --- a/flume-ng-core/src/main/java/org/apache/flume/serialization/ResettableFileInputStream.java +++ b/flume-ng-core/src/main/java/org/apache/flume/serialization/ResettableFileInputStream.java @@ -34,6 +34,7 @@ import java.nio.charset.Charset; import java.nio.charset.CharsetDecoder; import java.nio.charset.CoderResult; +import java.nio.charset.CodingErrorAction; /** *

This class makes the following assumptions: @@ -78,7 +79,7 @@ public class ResettableFileInputStream extends ResettableInputStream */ public ResettableFileInputStream(File file, PositionTracker tracker) throws IOException { - this(file, tracker, DEFAULT_BUF_SIZE, Charsets.UTF_8); + this(file, tracker, DEFAULT_BUF_SIZE, Charsets.UTF_8, DecodeErrorPolicy.FAIL); } /** @@ -98,7 +99,7 @@ public ResettableFileInputStream(File file, PositionTracker tracker) * @throws FileNotFoundException */ public ResettableFileInputStream(File file, PositionTracker tracker, - int bufSize, Charset charset) + int bufSize, Charset charset, DecodeErrorPolicy decodeErrorPolicy) throws IOException { this.file = file; this.tracker = tracker; @@ -115,6 +116,24 @@ public ResettableFileInputStream(File file, PositionTracker tracker, this.syncPosition = 0; this.maxCharWidth = (int)Math.ceil(charset.newEncoder().maxBytesPerChar()); + CodingErrorAction errorAction; + switch (decodeErrorPolicy) { + case FAIL: + errorAction = CodingErrorAction.REPORT; + break; + case REPLACE: + errorAction = CodingErrorAction.REPLACE; + break; + case IGNORE: + errorAction = CodingErrorAction.IGNORE; + break; + default: + throw new IllegalArgumentException( + "Unexpected value for decode error policy: " + decodeErrorPolicy); + } + decoder.onMalformedInput(errorAction); + decoder.onUnmappableCharacter(errorAction); + seek(tracker.getPosition()); } diff --git a/flume-ng-core/src/main/java/org/apache/flume/source/SpoolDirectorySource.java b/flume-ng-core/src/main/java/org/apache/flume/source/SpoolDirectorySource.java index 957eb8bc24..72c405908f 100644 --- a/flume-ng-core/src/main/java/org/apache/flume/source/SpoolDirectorySource.java +++ b/flume-ng-core/src/main/java/org/apache/flume/source/SpoolDirectorySource.java @@ -30,6 +30,7 @@ import org.apache.flume.client.avro.ReliableSpoolingFileEventReader; import org.apache.flume.conf.Configurable; import org.apache.flume.instrumentation.SourceCounter; +import org.apache.flume.serialization.DecodeErrorPolicy; import org.apache.flume.serialization.LineDeserializer; import org.slf4j.Logger; import org.slf4j.LoggerFactory; @@ -60,6 +61,7 @@ public class SpoolDirectorySource extends AbstractSource implements private Context deserializerContext; private String deletePolicy; private String inputCharset; + private DecodeErrorPolicy decodeErrorPolicy; private volatile boolean hasFatalError = false; private SourceCounter sourceCounter; @@ -86,6 +88,7 @@ public synchronized void start() { .deserializerContext(deserializerContext) .deletePolicy(deletePolicy) .inputCharset(inputCharset) + .decodeErrorPolicy(decodeErrorPolicy) .build(); } catch (IOException ioe) { throw new FlumeException("Error instantiating spooling event parser", @@ -139,6 +142,9 @@ public synchronized void configure(Context context) { batchSize = context.getInteger(BATCH_SIZE, DEFAULT_BATCH_SIZE); inputCharset = context.getString(INPUT_CHARSET, DEFAULT_INPUT_CHARSET); + decodeErrorPolicy = DecodeErrorPolicy.valueOf( + context.getString(DECODE_ERROR_POLICY, DEFAULT_DECODE_ERROR_POLICY) + .toUpperCase()); ignorePattern = context.getString(IGNORE_PAT, DEFAULT_IGNORE_PAT); trackerDirPath = context.getString(TRACKER_DIR, DEFAULT_TRACKER_DIR); diff --git a/flume-ng-core/src/main/java/org/apache/flume/source/SpoolDirectorySourceConfigurationConstants.java b/flume-ng-core/src/main/java/org/apache/flume/source/SpoolDirectorySourceConfigurationConstants.java index f3cc70320a..7bfb0ee924 100644 --- a/flume-ng-core/src/main/java/org/apache/flume/source/SpoolDirectorySourceConfigurationConstants.java +++ b/flume-ng-core/src/main/java/org/apache/flume/source/SpoolDirectorySourceConfigurationConstants.java @@ -17,6 +17,8 @@ package org.apache.flume.source; +import org.apache.flume.serialization.DecodeErrorPolicy; + public class SpoolDirectorySourceConfigurationConstants { /** Directory where files are deposited. */ public static final String SPOOL_DIRECTORY = "spoolDir"; @@ -64,6 +66,12 @@ public class SpoolDirectorySourceConfigurationConstants { public static final String DELETE_POLICY = "deletePolicy"; public static final String DEFAULT_DELETE_POLICY = "never"; + /** Character set used when reading the input. */ public static final String INPUT_CHARSET = "inputCharset"; public static final String DEFAULT_INPUT_CHARSET = "UTF-8"; + + /** What to do when there is a character set decoding error. */ + public static final String DECODE_ERROR_POLICY = "decodeErrorPolicy"; + public static final String DEFAULT_DECODE_ERROR_POLICY = + DecodeErrorPolicy.FAIL.name(); } diff --git a/flume-ng-core/src/test/java/org/apache/flume/serialization/TestResettableFileInputStream.java b/flume-ng-core/src/test/java/org/apache/flume/serialization/TestResettableFileInputStream.java index 066765c692..73df9c9155 100644 --- a/flume-ng-core/src/test/java/org/apache/flume/serialization/TestResettableFileInputStream.java +++ b/flume-ng-core/src/test/java/org/apache/flume/serialization/TestResettableFileInputStream.java @@ -32,11 +32,13 @@ import static org.junit.Assert.assertEquals; import java.io.BufferedOutputStream; +import java.io.ByteArrayOutputStream; import java.io.File; import java.io.FileOutputStream; import java.io.IOException; import java.io.OutputStream; import java.nio.charset.Charset; +import java.nio.charset.MalformedInputException; import java.util.List; public class TestResettableFileInputStream { @@ -108,6 +110,120 @@ public void testWideCharRead() throws IOException { in.close(); } + @Test(expected = MalformedInputException.class) + public void testUtf8DecodeErrorHandlingFailMalformed() throws IOException { + ResettableInputStream in = initUtf8DecodeTest(DecodeErrorPolicy.FAIL); + while (in.readChar() != -1) { + // Do nothing... read the whole file and throw away the bytes. + } + fail("Expected MalformedInputException!"); + } + + + @Test + public void testUtf8DecodeErrorHandlingIgnore() throws IOException { + ResettableInputStream in = initUtf8DecodeTest(DecodeErrorPolicy.IGNORE); + int c; + StringBuilder sb = new StringBuilder(); + while ((c = in.readChar()) != -1) { + sb.append((char)c); + } + assertEquals("Latin1: ()\nLong: ()\nNonUnicode: ()\n", sb.toString()); + } + + @Test + public void testUtf8DecodeErrorHandlingReplace() throws IOException { + ResettableInputStream in = initUtf8DecodeTest(DecodeErrorPolicy.REPLACE); + int c; + StringBuilder sb = new StringBuilder(); + while ((c = in.readChar()) != -1) { + sb.append((char)c); + } + assertEquals("Latin1: (X)\nLong: (XXX)\nNonUnicode: (X)\n" + .replaceAll("X", "\ufffd"), sb.toString()); + } + + @Test(expected = MalformedInputException.class) + public void testLatin1DecodeErrorHandlingFailMalformed() throws IOException { + ByteArrayOutputStream out = new ByteArrayOutputStream(); + generateLatin1InvalidSequence(out); + Files.write(out.toByteArray(), file); + ResettableInputStream in = initInputStream(DecodeErrorPolicy.FAIL); + while (in.readChar() != -1) { + // Do nothing... read the whole file and throw away the bytes. + } + fail("Expected MalformedInputException!"); + } + + @Test + public void testLatin1DecodeErrorHandlingReplace() throws IOException { + ByteArrayOutputStream out = new ByteArrayOutputStream(); + generateLatin1InvalidSequence(out); + Files.write(out.toByteArray(), file); + ResettableInputStream in = initInputStream(DecodeErrorPolicy.REPLACE); + + int c; + StringBuilder sb = new StringBuilder(); + while ((c = in.readChar()) != -1) { + sb.append((char)c); + } + assertEquals("Invalid: (X)\n".replaceAll("X", "\ufffd"), sb.toString()); + } + + private ResettableInputStream initUtf8DecodeTest(DecodeErrorPolicy policy) + throws IOException { + writeBigBadUtf8Sequence(file); + return initInputStream(policy); + } + + private ResettableInputStream initInputStream(DecodeErrorPolicy policy) + throws IOException { + PositionTracker tracker = new DurablePositionTracker(meta, file.getPath()); + ResettableInputStream in = new ResettableFileInputStream(file, tracker, + 2048, Charsets.UTF_8, policy); + return in; + } + + private void writeBigBadUtf8Sequence(File file) throws IOException { + ByteArrayOutputStream out = new ByteArrayOutputStream(); + generateUtf8Latin1Sequence(out); + generateUtf8OverlyLongSequence(out); + generateUtf8NonUnicodeSequence(out); + Files.write(out.toByteArray(), file); + } + + private void generateUtf8OverlyLongSequence(OutputStream out) + throws IOException { + out.write("Long: (".getBytes(Charsets.UTF_8)); + // Overly-long slash character should not be accepted. + out.write(new byte[] { (byte)0xe0, (byte)0x80, (byte)0xaf }); + out.write(")\n".getBytes(Charsets.UTF_8)); + } + + private void generateUtf8NonUnicodeSequence(OutputStream out) + throws IOException { + out.write("NonUnicode: (".getBytes(Charsets.UTF_8)); + // This is a valid 5-octet sequence but is not Unicode + out.write(new byte[] { (byte)0xf8, (byte)0xa1, (byte)0xa1, (byte)0xa1, + (byte)0xa1 } ); + out.write(")\n".getBytes(Charsets.UTF_8)); + } + + private void generateUtf8Latin1Sequence(OutputStream out) throws IOException { + out.write("Latin1: (".getBytes(Charsets.UTF_8)); + // This is "e" with an accent in Latin-1 + out.write(new byte[] { (byte)0xe9 } ); + out.write(")\n".getBytes(Charsets.UTF_8)); + } + + private void generateLatin1InvalidSequence(OutputStream out) + throws IOException { + out.write("Invalid: (".getBytes(Charsets.UTF_8)); + // Not a valid character in Latin 1. + out.write(new byte[] { (byte)0x81 } ); + out.write(")\n".getBytes(Charsets.UTF_8)); + } + /** * Ensure a reset() brings us back to the default mark (beginning of file) * @throws IOException @@ -206,7 +322,7 @@ public void testSeek() throws IOException { PositionTracker tracker = new DurablePositionTracker(meta, file.getPath()); ResettableInputStream in = new ResettableFileInputStream(file, tracker, - 10 * LINE_LEN, Charsets.UTF_8); + 10 * LINE_LEN, Charsets.UTF_8, DecodeErrorPolicy.FAIL); String line = ""; for (int i = 0; i < 9; i++) { diff --git a/flume-ng-doc/sphinx/FlumeUserGuide.rst b/flume-ng-doc/sphinx/FlumeUserGuide.rst index 007436b6b5..5a59b56272 100644 --- a/flume-ng-doc/sphinx/FlumeUserGuide.rst +++ b/flume-ng-doc/sphinx/FlumeUserGuide.rst @@ -953,6 +953,11 @@ trackerDir .flumespool Directory to store metadata related to pro If this path is not an absolute path, then it is interpreted as relative to the spoolDir. batchSize 100 Granularity at which to batch transfer to the channel inputCharset UTF-8 Character set used by deserializers that treat the input file as text. +decodeErrorPolicy ``FAIL`` What to do when we see a non-decodable character in the input file. + ``FAIL``: Throw an exception and fail to parse the file. + ``REPLACE``: Replace the unparseable character with the "replacement character" char, + typically Unicode U+FFFD. + ``IGNORE``: Drop the unparseable character sequence. deserializer ``LINE`` Specify the deserializer used to parse the file into events. Defaults to parsing each line as an event. The class specified must implement ``EventDeserializer.Builder``. From c4e2129fd12f97303a1b8120a2ecf7da456e1b77 Mon Sep 17 00:00:00 2001 From: Mike Percy Date: Thu, 3 Oct 2013 17:25:57 -0700 Subject: [PATCH 024/341] FLUME-2202. AsyncHBaseSink should coalesce increments to reduce RPC roundtrips (Hari Shreedharan via Mike Percy) --- flume-ng-doc/sphinx/FlumeUserGuide.rst | 38 +++--- .../flume/sink/hbase/AsyncHBaseSink.java | 121 ++++++++++++++++-- .../HBaseSinkConfigurationConstants.java | 4 + .../hbase/IncrementAsyncHBaseSerializer.java | 78 +++++++++++ .../flume/sink/hbase/TestAsyncHBaseSink.java | 82 +++++++++++- 5 files changed, 291 insertions(+), 32 deletions(-) create mode 100644 flume-ng-sinks/flume-ng-hbase-sink/src/test/java/org/apache/flume/sink/hbase/IncrementAsyncHBaseSerializer.java diff --git a/flume-ng-doc/sphinx/FlumeUserGuide.rst b/flume-ng-doc/sphinx/FlumeUserGuide.rst index 5a59b56272..dc8d05d852 100644 --- a/flume-ng-doc/sphinx/FlumeUserGuide.rst +++ b/flume-ng-doc/sphinx/FlumeUserGuide.rst @@ -1850,30 +1850,32 @@ AsyncHBaseSink '''''''''''''' This sink writes data to HBase using an asynchronous model. A class implementing -AsyncHbaseEventSerializer -which is specified by the configuration is used to convert the events into +AsyncHbaseEventSerializer which is specified by the configuration is used to convert the events into HBase puts and/or increments. These puts and increments are then written -to HBase. This sink provides the same consistency guarantees as HBase, +to HBase. This sink uses the `Asynchbase API `_ to write to +HBase. This sink provides the same consistency guarantees as HBase, which is currently row-wise atomicity. In the event of Hbase failing to write certain events, the sink will replay all events in that transaction. The type is the FQCN: org.apache.flume.sink.hbase.AsyncHBaseSink. Required properties are in **bold**. -================ ============================================================ ==================================================================================== -Property Name Default Description -================ ============================================================ ==================================================================================== -**channel** -- -**type** -- The component type name, needs to be ``asynchbase`` -**table** -- The name of the table in Hbase to write to. -zookeeperQuorum -- The quorum spec. This is the value for the property ``hbase.zookeeper.quorum`` in hbase-site.xml -znodeParent /hbase The base path for the znode for the -ROOT- region. Value of ``zookeeper.znode.parent`` in hbase-site.xml -**columnFamily** -- The column family in Hbase to write to. -batchSize 100 Number of events to be written per txn. -timeout 60000 The length of time (in milliseconds) the sink waits for acks from hbase for - all events in a transaction. -serializer org.apache.flume.sink.hbase.SimpleAsyncHbaseEventSerializer -serializer.* -- Properties to be passed to the serializer. -================ ============================================================ ==================================================================================== +=================== ============================================================ ==================================================================================== +Property Name Default Description +=================== ============================================================ ==================================================================================== +**channel** -- +**type** -- The component type name, needs to be ``asynchbase`` +**table** -- The name of the table in Hbase to write to. +zookeeperQuorum -- The quorum spec. This is the value for the property ``hbase.zookeeper.quorum`` in hbase-site.xml +znodeParent /hbase The base path for the znode for the -ROOT- region. Value of ``zookeeper.znode.parent`` in hbase-site.xml +**columnFamily** -- The column family in Hbase to write to. +batchSize 100 Number of events to be written per txn. +coalesceIncrements false Should the sink coalesce multiple increments to a cell per batch. This might give + better performance if there are multiple increments to a limited number of cells. +timeout 60000 The length of time (in milliseconds) the sink waits for acks from hbase for + all events in a transaction. +serializer org.apache.flume.sink.hbase.SimpleAsyncHbaseEventSerializer +serializer.* -- Properties to be passed to the serializer. +=================== ============================================================ ==================================================================================== Note that this sink takes the Zookeeper Quorum and parent znode information in the configuration. Zookeeper Quorum and parent node configuration may be diff --git a/flume-ng-sinks/flume-ng-hbase-sink/src/main/java/org/apache/flume/sink/hbase/AsyncHBaseSink.java b/flume-ng-sinks/flume-ng-hbase-sink/src/main/java/org/apache/flume/sink/hbase/AsyncHBaseSink.java index 5e297b1333..0545554c08 100644 --- a/flume-ng-sinks/flume-ng-hbase-sink/src/main/java/org/apache/flume/sink/hbase/AsyncHBaseSink.java +++ b/flume-ng-sinks/flume-ng-hbase-sink/src/main/java/org/apache/flume/sink/hbase/AsyncHBaseSink.java @@ -18,13 +18,19 @@ */ package org.apache.flume.sink.hbase; +import java.util.Arrays; +import java.util.Collection; +import java.util.Comparator; import java.util.List; +import java.util.Map; import java.util.concurrent.CountDownLatch; import java.util.concurrent.ExecutorService; import java.util.concurrent.Executors; import java.util.concurrent.atomic.AtomicBoolean; import com.google.common.annotations.VisibleForTesting; +import com.google.common.collect.Maps; +import com.google.common.primitives.UnsignedBytes; import com.google.common.util.concurrent.ThreadFactoryBuilder; import org.apache.flume.Channel; import org.apache.flume.Context; @@ -113,20 +119,32 @@ public class AsyncHBaseSink extends AbstractSink implements Configurable { private String zkQuorum; private String zkBaseDir; private ExecutorService sinkCallbackPool; - private boolean isTest; + private boolean isTimeoutTest; + private boolean isCoalesceTest; private boolean enableWal = true; + private boolean batchIncrements = false; + private volatile int totalCallbacksReceived = 0; + private Map incrementBuffer; + + // Does not need to be thread-safe. Always called only from the sink's + // process method. + private final Comparator COMPARATOR = UnsignedBytes + .lexicographicalComparator(); public AsyncHBaseSink(){ this(null); } public AsyncHBaseSink(Configuration conf) { - this(conf, false); + this(conf, false, false); } - AsyncHBaseSink(Configuration conf, boolean isTimeoutTesting) { + @VisibleForTesting + AsyncHBaseSink(Configuration conf, boolean isTimeoutTest, + boolean isCoalesceTest) { this.conf = conf; - isTest = isTimeoutTesting; + this.isTimeoutTest = isTimeoutTest; + this.isCoalesceTest = isCoalesceTest; } @Override @@ -138,7 +156,7 @@ public Status process() throws EventDeliveryException { * the next one is being processed. * */ - if(!open){ + if (!open) { throw new EventDeliveryException("Sink was never opened. " + "Please fix the configuration."); } @@ -147,6 +165,9 @@ public Status process() throws EventDeliveryException { AtomicInteger callbacksExpected = new AtomicInteger(0); final Lock lock = new ReentrantLock(); final Condition condition = lock.newCondition(); + if (incrementBuffer != null) { + incrementBuffer.clear(); + } /* * Callbacks can be reused per transaction, since they share the same * locks and conditions. @@ -185,18 +206,41 @@ public Status process() throws EventDeliveryException { serializer.setEvent(event); List actions = serializer.getActions(); List increments = serializer.getIncrements(); - callbacksExpected.addAndGet(actions.size() + increments.size()); + callbacksExpected.addAndGet(actions.size()); + if (!batchIncrements) { + callbacksExpected.addAndGet(increments.size()); + } for (PutRequest action : actions) { action.setDurable(enableWal); client.put(action).addCallbacks(putSuccessCallback, putFailureCallback); } for (AtomicIncrementRequest increment : increments) { - client.atomicIncrement(increment).addCallbacks( - incrementSuccessCallback, incrementFailureCallback); + if (batchIncrements) { + CellIdentifier identifier = new CellIdentifier(increment.key(), + increment.qualifier()); + AtomicIncrementRequest request + = incrementBuffer.get(identifier); + if (request == null) { + incrementBuffer.put(identifier, increment); + } else { + request.setAmount(request.getAmount() + increment.getAmount()); + } + } else { + client.atomicIncrement(increment).addCallbacks( + incrementSuccessCallback, incrementFailureCallback); + } } } } + if (batchIncrements) { + Collection increments = incrementBuffer.values(); + for (AtomicIncrementRequest increment : increments) { + client.atomicIncrement(increment).addCallbacks( + incrementSuccessCallback, incrementFailureCallback); + } + callbacksExpected.addAndGet(increments.size()); + } client.flush(); } catch (Throwable e) { this.handleTransactionFailure(txn); @@ -216,7 +260,7 @@ public Status process() throws EventDeliveryException { timeRemaining = timeout - (System.nanoTime() - startTime); timeRemaining = (timeRemaining >= 0) ? timeRemaining : 0; try { - if(!condition.await(timeRemaining, TimeUnit.NANOSECONDS)){ + if (!condition.await(timeRemaining, TimeUnit.NANOSECONDS)) { txnFail.set(true); logger.warn("HBase callbacks timed out. " + "Transaction will be rolled back."); @@ -231,6 +275,10 @@ public Status process() throws EventDeliveryException { lock.unlock(); } + if (isCoalesceTest) { + totalCallbacksReceived += callbacksReceived.get(); + } + /* * At this point, either the txn has failed * or all callbacks received and txn is successful. @@ -246,7 +294,7 @@ public Status process() throws EventDeliveryException { throw new EventDeliveryException("Could not write events to Hbase. " + "Transaction failed, and rolled back."); } else { - try{ + try { txn.commit(); txn.close(); sinkCounter.addToEventDrainSuccessCount(i); @@ -334,6 +382,21 @@ public void configure(Context context) { "All writes to HBase will have WAL disabled, and any data in the " + "memstore of this region in the Region Server could be lost!"); } + + batchIncrements = context.getBoolean( + HBaseSinkConfigurationConstants.CONFIG_COALESCE_INCREMENTS, + HBaseSinkConfigurationConstants.DEFAULT_COALESCE_INCREMENTS); + + if(batchIncrements) { + incrementBuffer = Maps.newHashMap(); + logger.info("Increment coalescing is enabled. Increments will be " + + "buffered."); + } + } + + @VisibleForTesting + int getTotalCallbacksReceived() { + return totalCallbacksReceived; } @VisibleForTesting @@ -346,7 +409,7 @@ public void start(){ + "before calling start on an old instance."); sinkCounter.start(); sinkCounter.incrementConnectionCreatedCount(); - if (!isTest) { + if (!isTimeoutTest) { sinkCallbackPool = Executors.newCachedThreadPool(new ThreadFactoryBuilder() .setNameFormat(this.getName() + " HBase Call Pool").build()); } else { @@ -447,7 +510,7 @@ public SuccessCallback(Lock lck, AtomicInteger callbacksReceived, lock = lck; this.callbacksReceived = callbacksReceived; this.condition = condition; - isTimeoutTesting = isTest; + isTimeoutTesting = isTimeoutTest; } @Override @@ -487,7 +550,7 @@ public FailureCallback(Lock lck, AtomicInteger callbacksReceived, this.callbacksReceived = callbacksReceived; this.txnFail = txnFail; this.condition = condition; - isTimeoutTesting = isTest; + isTimeoutTesting = isTimeoutTest; } @Override @@ -525,4 +588,36 @@ private void checkIfChannelExceptionAndThrow(Throwable e) } throw new EventDeliveryException("Error in processing transaction.", e); } + + private class CellIdentifier { + private final byte[] row; + private final byte[] column; + private final int hashCode; + // Since the sink operates only on one table and one cf, + // we use the data from the owning sink + public CellIdentifier(byte[] row, byte[] column) { + this.row = row; + this.column = column; + this.hashCode = + (Arrays.hashCode(row) * 31) * (Arrays.hashCode(column) * 31); + } + + @Override + public int hashCode() { + return hashCode; + } + + // Since we know that this class is used from only this class, + // skip the class comparison to save time + @Override + public boolean equals(Object other) { + CellIdentifier o = (CellIdentifier) other; + if (other == null) { + return false; + } else { + return (COMPARATOR.compare(row, o.row) == 0 + && COMPARATOR.compare(column, o.column) == 0); + } + } + } } diff --git a/flume-ng-sinks/flume-ng-hbase-sink/src/main/java/org/apache/flume/sink/hbase/HBaseSinkConfigurationConstants.java b/flume-ng-sinks/flume-ng-hbase-sink/src/main/java/org/apache/flume/sink/hbase/HBaseSinkConfigurationConstants.java index 7fdc75bd3a..1a78071661 100644 --- a/flume-ng-sinks/flume-ng-hbase-sink/src/main/java/org/apache/flume/sink/hbase/HBaseSinkConfigurationConstants.java +++ b/flume-ng-sinks/flume-ng-hbase-sink/src/main/java/org/apache/flume/sink/hbase/HBaseSinkConfigurationConstants.java @@ -66,4 +66,8 @@ public class HBaseSinkConfigurationConstants { public static final String DEFAULT_ZK_ZNODE_PARENT = HConstants.DEFAULT_ZOOKEEPER_ZNODE_PARENT; + public static final String CONFIG_COALESCE_INCREMENTS = "coalesceIncrements"; + + public static final Boolean DEFAULT_COALESCE_INCREMENTS = false; + } diff --git a/flume-ng-sinks/flume-ng-hbase-sink/src/test/java/org/apache/flume/sink/hbase/IncrementAsyncHBaseSerializer.java b/flume-ng-sinks/flume-ng-hbase-sink/src/test/java/org/apache/flume/sink/hbase/IncrementAsyncHBaseSerializer.java new file mode 100644 index 0000000000..b8aefe8d4f --- /dev/null +++ b/flume-ng-sinks/flume-ng-hbase-sink/src/test/java/org/apache/flume/sink/hbase/IncrementAsyncHBaseSerializer.java @@ -0,0 +1,78 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, + * software distributed under the License is distributed on an + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + * KIND, either express or implied. See the License for the + * specific language governing permissions and limitations + * under the License. + */ +package org.apache.flume.sink.hbase; + +import org.apache.flume.Context; +import org.apache.flume.Event; +import org.apache.flume.conf.ComponentConfiguration; +import org.hbase.async.AtomicIncrementRequest; +import org.hbase.async.PutRequest; + +import java.util.ArrayList; +import java.util.Collections; +import java.util.List; + +/** + * An AsyncHBaseEventSerializer implementation that increments a configured + * column for the row whose row key is the event's body bytes. + */ +public class IncrementAsyncHBaseSerializer implements AsyncHbaseEventSerializer { + private byte[] table; + private byte[] cf; + private byte[] column; + private Event currentEvent; + @Override + public void initialize(byte[] table, byte[] cf) { + this.table = table; + this.cf = cf; + } + + @Override + public void setEvent(Event event) { + this.currentEvent = event; + } + + @Override + public List getActions() { + return Collections.emptyList(); + } + + @Override + public List getIncrements() { + List incrs + = new ArrayList(); + AtomicIncrementRequest incr = new AtomicIncrementRequest(table, + currentEvent.getBody(), cf, column, 1); + incrs.add(incr); + return incrs; + } + + @Override + public void cleanUp() { + } + + @Override + public void configure(Context context) { + column = context.getString("column", "col").getBytes(); + } + + @Override + public void configure(ComponentConfiguration conf) { + } +} diff --git a/flume-ng-sinks/flume-ng-hbase-sink/src/test/java/org/apache/flume/sink/hbase/TestAsyncHBaseSink.java b/flume-ng-sinks/flume-ng-hbase-sink/src/test/java/org/apache/flume/sink/hbase/TestAsyncHBaseSink.java index a0c04eb6f9..ccbc086107 100644 --- a/flume-ng-sinks/flume-ng-hbase-sink/src/test/java/org/apache/flume/sink/hbase/TestAsyncHBaseSink.java +++ b/flume-ng-sinks/flume-ng-hbase-sink/src/test/java/org/apache/flume/sink/hbase/TestAsyncHBaseSink.java @@ -204,7 +204,7 @@ public void testTimeOut() throws Exception { testUtility.createTable(tableName.getBytes(), columnFamily.getBytes()); deleteTable = true; AsyncHBaseSink sink = new AsyncHBaseSink(testUtility.getConfiguration(), - true); + true, false); Configurables.configure(sink, ctx); Channel channel = new MemoryChannel(); Configurables.configure(channel, ctx); @@ -270,6 +270,86 @@ public void testMultipleBatches() throws Exception { Assert.assertArrayEquals(Longs.toByteArray(3), out); } + @Test + public void testMultipleBatchesBatchIncrementsWithCoalescing() + throws Exception { + doTestMultipleBatchesBatchIncrements(true); + } + + @Test + public void testMultipleBatchesBatchIncrementsNoCoalescing() + throws Exception { + doTestMultipleBatchesBatchIncrements(false); + } + + public void doTestMultipleBatchesBatchIncrements(boolean coalesce) throws + Exception { + testUtility.createTable(tableName.getBytes(), columnFamily.getBytes()); + deleteTable = true; + AsyncHBaseSink sink = new AsyncHBaseSink(testUtility.getConfiguration(), + false, true); + if (coalesce) { + ctx.put(HBaseSinkConfigurationConstants.CONFIG_COALESCE_INCREMENTS, + "true"); + } + ctx.put("batchSize", "2"); + ctx.put("serializer", IncrementAsyncHBaseSerializer.class.getName()); + ctx.put("serializer.column", "test"); + Configurables.configure(sink, ctx); + //Reset the context to a higher batchSize + ctx.put("batchSize", "100"); + // Restore the original serializer + ctx.put("serializer", SimpleAsyncHbaseEventSerializer.class.getName()); + //Restore the no coalescing behavior + ctx.put(HBaseSinkConfigurationConstants.CONFIG_COALESCE_INCREMENTS, + "false"); + Channel channel = new MemoryChannel(); + Configurables.configure(channel, ctx); + sink.setChannel(channel); + sink.start(); + Transaction tx = channel.getTransaction(); + tx.begin(); + for (int i = 0; i < 4; i++) { + for (int j = 0; j < 3; j++) { + Event e = EventBuilder.withBody(Bytes.toBytes(valBase + "-" + i)); + channel.put(e); + } + } + tx.commit(); + tx.close(); + int count = 0; + Status status = Status.READY; + while (status != Status.BACKOFF) { + count++; + status = sink.process(); + } + Assert.assertFalse(sink.isConfNull()); + sink.stop(); + Assert.assertEquals(7, count); + HTable table = new HTable(testUtility.getConfiguration(), tableName); + Scan scan = new Scan(); + scan.addColumn(columnFamily.getBytes(),"test".getBytes()); + scan.setStartRow(Bytes.toBytes(valBase)); + ResultScanner rs = table.getScanner(scan); + int i = 0; + try { + for (Result r = rs.next(); r != null; r = rs.next()) { + byte[] out = r.getValue(columnFamily.getBytes(), "test".getBytes()); + Assert.assertArrayEquals(Longs.toByteArray(3), out); + Assert.assertTrue(new String(r.getRow()).startsWith(valBase)); + i++; + } + } finally { + rs.close(); + } + Assert.assertEquals(4, i); + if (coalesce) { + Assert.assertEquals(8, sink.getTotalCallbacksReceived()); + } else { + Assert.assertEquals(12, sink.getTotalCallbacksReceived()); + } + } + @Test public void testWithoutConfigurationObject() throws Exception{ testUtility.createTable(tableName.getBytes(), columnFamily.getBytes()); From 9c59a309764498c013ccd202926d86413da01078 Mon Sep 17 00:00:00 2001 From: Arvind Prabhakar Date: Thu, 3 Oct 2013 18:25:02 -0700 Subject: [PATCH 025/341] FLUME-2191. HDFS Minicluster tests failing after protobuf upgrade. (Hari Shreedharan via Arvind Prabhakar) --- flume-ng-sinks/flume-hdfs-sink/pom.xml | 4 ---- pom.xml | 4 ++-- 2 files changed, 2 insertions(+), 6 deletions(-) diff --git a/flume-ng-sinks/flume-hdfs-sink/pom.xml b/flume-ng-sinks/flume-hdfs-sink/pom.xml index 95fcdea2cf..c646a89e2c 100644 --- a/flume-ng-sinks/flume-hdfs-sink/pom.xml +++ b/flume-ng-sinks/flume-hdfs-sink/pom.xml @@ -142,10 +142,6 @@ limitations under the License. 2 - - 2.0.0-alpha - hadoop-common - diff --git a/pom.xml b/pom.xml index f0fd22ecb7..267925f70b 100644 --- a/pom.xml +++ b/pom.xml @@ -106,7 +106,7 @@ limitations under the License. - 2.0.0-alpha + 2.1.0-beta 0.94.2 hadoop-common 0.8.0 @@ -666,7 +666,7 @@ limitations under the License. com.google.guava guava - 10.0.1 + 11.0.2 From 20eed3fdcbee57b84504ec0e1adada46950c4f90 Mon Sep 17 00:00:00 2001 From: Mike Percy Date: Tue, 8 Oct 2013 14:00:31 -0700 Subject: [PATCH 026/341] FLUME-2200. HTTP Source should use "port" param for both SSL & cleartext (Hari Shreedharan via Mike Percy) --- .../apache/flume/source/http/HTTPSource.java | 30 ++++++++----------- .../HTTPSourceConfigurationConstants.java | 1 - .../flume/source/http/TestHTTPSource.java | 3 +- flume-ng-doc/sphinx/FlumeUserGuide.rst | 1 - 4 files changed, 14 insertions(+), 21 deletions(-) diff --git a/flume-ng-core/src/main/java/org/apache/flume/source/http/HTTPSource.java b/flume-ng-core/src/main/java/org/apache/flume/source/http/HTTPSource.java index 84ee33b2b5..de79e8be5c 100644 --- a/flume-ng-core/src/main/java/org/apache/flume/source/http/HTTPSource.java +++ b/flume-ng-core/src/main/java/org/apache/flume/source/http/HTTPSource.java @@ -38,7 +38,6 @@ import javax.servlet.http.HttpServletRequest; import javax.servlet.http.HttpServletResponse; import java.io.IOException; -import java.util.ArrayList; import java.util.Collections; import java.util.List; import java.util.Map; @@ -90,7 +89,6 @@ public class HTTPSource extends AbstractSource implements private SourceCounter sourceCounter; // SSL configuration variable - private volatile Integer sslPort; private volatile String keyStorePath; private volatile String keyStorePassword; private volatile Boolean sslEnabled; @@ -108,11 +106,8 @@ public void configure(Context context) { Preconditions.checkState(host != null && !host.isEmpty(), "HTTPSource hostname specified is empty"); - // verify port only if its not ssl - if(!sslEnabled) { - Preconditions.checkNotNull(port, "HTTPSource requires a port number to be" - + " specified"); - } + Preconditions.checkNotNull(port, "HTTPSource requires a port number to be" + + " specified"); String handlerClassName = context.getString( HTTPSourceConfigurationConstants.CONFIG_HANDLER, @@ -120,8 +115,6 @@ public void configure(Context context) { if(sslEnabled) { LOG.debug("SSL configuration enabled"); - sslPort = context.getInteger(HTTPSourceConfigurationConstants.SSL_PORT); - Preconditions.checkArgument(sslPort != null && sslPort > 0, "SSL Port cannot be null or less than 0" ); keyStorePath = context.getString(HTTPSourceConfigurationConstants.SSL_KEYSTORE); Preconditions.checkArgument(keyStorePath != null && !keyStorePath.isEmpty(), "Keystore is required for SSL Conifguration" ); @@ -129,6 +122,8 @@ public void configure(Context context) { Preconditions.checkArgument(keyStorePassword != null, "Keystore password is required for SSL Configuration"); } + + @SuppressWarnings("unchecked") Class clazz = (Class) @@ -163,7 +158,7 @@ private void checkHostAndPort() { + " specified"); } - @Override + @Override public void start() { Preconditions.checkState(srv == null, "Running HTTP Server found in source: " + getName() @@ -175,24 +170,23 @@ public void start() { Connector[] connectors = new Connector[1]; - if(sslEnabled) { + if (sslEnabled) { SslSocketConnector sslSocketConnector = new SslSocketConnector(); sslSocketConnector.setKeystore(keyStorePath); sslSocketConnector.setKeyPassword(keyStorePassword); - sslSocketConnector.setPort(sslPort); connectors[0] = sslSocketConnector; } else { - SocketConnector connector = new SocketConnector(); - connector.setPort(port); - connector.setHost(host); - connectors[0] = connector; + SocketConnector connector = new SocketConnector(); + connectors[0] = connector; } + connectors[0].setHost(host); + connectors[0].setPort(port); srv.setConnectors(connectors); try { org.mortbay.jetty.servlet.Context root = - new org.mortbay.jetty.servlet.Context( - srv, "/", org.mortbay.jetty.servlet.Context.SESSIONS); + new org.mortbay.jetty.servlet.Context( + srv, "/", org.mortbay.jetty.servlet.Context.SESSIONS); root.addServlet(new ServletHolder(new FlumeHTTPServlet()), "/"); srv.start(); Preconditions.checkArgument(srv.getHandler().equals(root)); diff --git a/flume-ng-core/src/main/java/org/apache/flume/source/http/HTTPSourceConfigurationConstants.java b/flume-ng-core/src/main/java/org/apache/flume/source/http/HTTPSourceConfigurationConstants.java index 205aeab3e4..ed52827ed8 100644 --- a/flume-ng-core/src/main/java/org/apache/flume/source/http/HTTPSourceConfigurationConstants.java +++ b/flume-ng-core/src/main/java/org/apache/flume/source/http/HTTPSourceConfigurationConstants.java @@ -34,7 +34,6 @@ public class HTTPSourceConfigurationConstants { public static final String DEFAULT_HANDLER = "org.apache.flume.source.http.JSONHandler"; - public static final String SSL_PORT = "sslPort"; public static final String SSL_KEYSTORE = "keystore"; public static final String SSL_KEYSTORE_PASSWORD = "keystorePassword"; public static final String SSL_ENABLED = "enableSSL"; diff --git a/flume-ng-core/src/test/java/org/apache/flume/source/http/TestHTTPSource.java b/flume-ng-core/src/test/java/org/apache/flume/source/http/TestHTTPSource.java index 9e14648b13..ab8ec094a3 100644 --- a/flume-ng-core/src/test/java/org/apache/flume/source/http/TestHTTPSource.java +++ b/flume-ng-core/src/test/java/org/apache/flume/source/http/TestHTTPSource.java @@ -112,7 +112,8 @@ public static void setUpClass() throws Exception { Context sslContext = new Context(); sslContext.put(HTTPSourceConfigurationConstants.SSL_ENABLED, "true"); sslPort = findFreePort(); - sslContext.put(HTTPSourceConfigurationConstants.SSL_PORT, String.valueOf(sslPort)); + sslContext.put(HTTPSourceConfigurationConstants.CONFIG_PORT, + String.valueOf(sslPort)); sslContext.put(HTTPSourceConfigurationConstants.SSL_KEYSTORE_PASSWORD, "password"); sslContext.put(HTTPSourceConfigurationConstants.SSL_KEYSTORE, "src/test/resources/jettykeystore"); diff --git a/flume-ng-doc/sphinx/FlumeUserGuide.rst b/flume-ng-doc/sphinx/FlumeUserGuide.rst index dc8d05d852..4892dfcd51 100644 --- a/flume-ng-doc/sphinx/FlumeUserGuide.rst +++ b/flume-ng-doc/sphinx/FlumeUserGuide.rst @@ -1287,7 +1287,6 @@ selector.* Depends on the sel interceptors -- Space-separated list of interceptors interceptors.* enableSSL false Set the property true, to enable SSL -sslPort The port to be used for SSL keystore Location of the keystore includng keystore file name keystorePassword Keystore password ================================================================================================================================== From 02fc1a8cf436dbc9327e96d21452b826978479f8 Mon Sep 17 00:00:00 2001 From: Mike Percy Date: Tue, 8 Oct 2013 18:12:10 -0700 Subject: [PATCH 027/341] FLUME-2208. Jetty's default SocketSelector leaks File descriptors (Hari Shreedharan via Mike Percy) --- .../flume/instrumentation/http/HTTPMetricsServer.java | 8 +++++++- .../java/org/apache/flume/source/http/HTTPSource.java | 6 ++++-- 2 files changed, 11 insertions(+), 3 deletions(-) diff --git a/flume-ng-core/src/main/java/org/apache/flume/instrumentation/http/HTTPMetricsServer.java b/flume-ng-core/src/main/java/org/apache/flume/instrumentation/http/HTTPMetricsServer.java index 373e3448b7..2c2c6f3f78 100644 --- a/flume-ng-core/src/main/java/org/apache/flume/instrumentation/http/HTTPMetricsServer.java +++ b/flume-ng-core/src/main/java/org/apache/flume/instrumentation/http/HTTPMetricsServer.java @@ -29,9 +29,11 @@ import org.apache.flume.Context; import org.apache.flume.instrumentation.MonitorService; import org.apache.flume.instrumentation.util.JMXPollUtil; +import org.mortbay.jetty.Connector; import org.mortbay.jetty.Request; import org.mortbay.jetty.Server; import org.mortbay.jetty.handler.AbstractHandler; +import org.mortbay.jetty.nio.SelectChannelConnector; import org.slf4j.Logger; import org.slf4j.LoggerFactory; @@ -55,9 +57,13 @@ public class HTTPMetricsServer implements MonitorService { @Override public void start() { - jettyServer = new Server(port); + jettyServer = new Server(); //We can use Contexts etc if we have many urls to handle. For one url, //specifying a handler directly is the most efficient. + SelectChannelConnector connector = new SelectChannelConnector(); + connector.setReuseAddress(true); + connector.setPort(port); + jettyServer.setConnectors(new Connector[] {connector}); jettyServer.setHandler(new HTTPMetricsHandler()); try { jettyServer.start(); diff --git a/flume-ng-core/src/main/java/org/apache/flume/source/http/HTTPSource.java b/flume-ng-core/src/main/java/org/apache/flume/source/http/HTTPSource.java index de79e8be5c..48c64922b8 100644 --- a/flume-ng-core/src/main/java/org/apache/flume/source/http/HTTPSource.java +++ b/flume-ng-core/src/main/java/org/apache/flume/source/http/HTTPSource.java @@ -28,7 +28,7 @@ import org.apache.flume.source.AbstractSource; import org.mortbay.jetty.Connector; import org.mortbay.jetty.Server; -import org.mortbay.jetty.bio.SocketConnector; +import org.mortbay.jetty.nio.SelectChannelConnector; import org.mortbay.jetty.security.SslSocketConnector; import org.mortbay.jetty.servlet.ServletHolder; import org.slf4j.Logger; @@ -174,9 +174,11 @@ public void start() { SslSocketConnector sslSocketConnector = new SslSocketConnector(); sslSocketConnector.setKeystore(keyStorePath); sslSocketConnector.setKeyPassword(keyStorePassword); + sslSocketConnector.setReuseAddress(true); connectors[0] = sslSocketConnector; } else { - SocketConnector connector = new SocketConnector(); + SelectChannelConnector connector = new SelectChannelConnector(); + connector.setReuseAddress(true); connectors[0] = connector; } From 1f95219ea6f87173018bde126a3485575a8ee252 Mon Sep 17 00:00:00 2001 From: Mike Percy Date: Wed, 9 Oct 2013 18:49:31 -0700 Subject: [PATCH 028/341] FLUME-1666. Syslog source strips timestamp and hostname from log message body (Jeff Lord via Mike Percy) --- .../SyslogSourceConfigurationConstants.java | 3 +++ .../apache/flume/source/SyslogTcpSource.java | 20 +++++++++++++++++++ .../org/apache/flume/source/SyslogUtils.java | 19 ++++++++++++++---- .../flume/source/TestSyslogUdpSource.java | 4 ++++ .../apache/flume/source/TestSyslogUtils.java | 5 +++-- flume-ng-doc/sphinx/FlumeDeveloperGuide.rst | 2 +- flume-ng-doc/sphinx/FlumeUserGuide.rst | 2 ++ 7 files changed, 48 insertions(+), 7 deletions(-) diff --git a/flume-ng-core/src/main/java/org/apache/flume/source/SyslogSourceConfigurationConstants.java b/flume-ng-core/src/main/java/org/apache/flume/source/SyslogSourceConfigurationConstants.java index 5a73c882fa..985949cffc 100644 --- a/flume-ng-core/src/main/java/org/apache/flume/source/SyslogSourceConfigurationConstants.java +++ b/flume-ng-core/src/main/java/org/apache/flume/source/SyslogSourceConfigurationConstants.java @@ -66,6 +66,9 @@ public final class SyslogSourceConfigurationConstants { public static final String CONFIG_READBUF_SIZE = "readBufferBytes"; public static final int DEFAULT_READBUF_SIZE = 1024; + public static final String CONFIG_KEEP_FIELDS = "keepFields"; + public static final boolean DEFAULT_KEEP_FIELDS = false; + private SyslogSourceConfigurationConstants() { // Disable explicit creation of objects. } diff --git a/flume-ng-core/src/main/java/org/apache/flume/source/SyslogTcpSource.java b/flume-ng-core/src/main/java/org/apache/flume/source/SyslogTcpSource.java index db9e0fd18a..7a12d275c5 100644 --- a/flume-ng-core/src/main/java/org/apache/flume/source/SyslogTcpSource.java +++ b/flume-ng-core/src/main/java/org/apache/flume/source/SyslogTcpSource.java @@ -19,10 +19,12 @@ package org.apache.flume.source; import java.net.InetSocketAddress; +import java.net.SocketAddress; import java.util.Map; import java.util.concurrent.Executors; import java.util.concurrent.TimeUnit; +import com.google.common.annotations.VisibleForTesting; import org.apache.flume.ChannelException; import org.apache.flume.Context; import org.apache.flume.CounterGroup; @@ -56,6 +58,7 @@ public class SyslogTcpSource extends AbstractSource private Integer eventSize; private Map formaterProp; private CounterGroup counterGroup = new CounterGroup(); + private Boolean keepFields; public class syslogTcpHandler extends SimpleChannelHandler { @@ -65,6 +68,10 @@ public void setEventSize(int eventSize){ syslogUtils.setEventSize(eventSize); } + public void setKeepFields(boolean removeFields){ + syslogUtils.setKeepFields(removeFields); + } + public void setFormater(Map prop) { syslogUtils.addFormats(prop); } @@ -103,6 +110,7 @@ public ChannelPipeline getPipeline() { syslogTcpHandler handler = new syslogTcpHandler(); handler.setEventSize(eventSize); handler.setFormater(formaterProp); + handler.setKeepFields(keepFields); return Channels.pipeline(handler); } }); @@ -146,6 +154,18 @@ public void configure(Context context) { eventSize = context.getInteger("eventSize", SyslogUtils.DEFAULT_SIZE); formaterProp = context.getSubProperties( SyslogSourceConfigurationConstants.CONFIG_FORMAT_PREFIX); + keepFields = context.getBoolean + (SyslogSourceConfigurationConstants.CONFIG_KEEP_FIELDS, false); + } + + @VisibleForTesting + public int getSourcePort() { + SocketAddress localAddress = nettyChannel.getLocalAddress(); + if (localAddress instanceof InetSocketAddress) { + InetSocketAddress addr = (InetSocketAddress) localAddress; + return addr.getPort(); + } + return 0; } } diff --git a/flume-ng-core/src/main/java/org/apache/flume/source/SyslogUtils.java b/flume-ng-core/src/main/java/org/apache/flume/source/SyslogUtils.java index c2a29a1d79..f2ea932dbc 100644 --- a/flume-ng-core/src/main/java/org/apache/flume/source/SyslogUtils.java +++ b/flume-ng-core/src/main/java/org/apache/flume/source/SyslogUtils.java @@ -20,6 +20,8 @@ package org.apache.flume.source; import org.apache.flume.Event; +import org.apache.flume.annotations.InterfaceAudience; +import org.apache.flume.annotations.InterfaceStability; import org.apache.flume.event.EventBuilder; import org.jboss.netty.buffer.ChannelBuffer; import org.slf4j.Logger; @@ -37,6 +39,8 @@ import java.util.regex.Matcher; import java.util.regex.Pattern; +@InterfaceAudience.Private +@InterfaceStability.Evolving public class SyslogUtils { final public static String SYSLOG_TIMESTAMP_FORMAT_RFC5424_2 = "yyyy-MM-dd'T'HH:mm:ss.SZ"; final public static String SYSLOG_TIMESTAMP_FORMAT_RFC5424_1 = "yyyy-MM-dd'T'HH:mm:ss.S"; @@ -79,6 +83,7 @@ public class SyslogUtils { private boolean isBadEvent; private boolean isIncompleteEvent; private Integer maxSize; + private boolean keepFields; private class SyslogFormatter { public Pattern regexPattern; @@ -98,15 +103,16 @@ public SyslogUtils() { } public SyslogUtils(boolean isUdp) { - this(DEFAULT_SIZE, isUdp); + this(DEFAULT_SIZE, SyslogSourceConfigurationConstants.DEFAULT_KEEP_FIELDS, isUdp); } - public SyslogUtils(Integer eventSize, boolean isUdp){ + public SyslogUtils(Integer eventSize, boolean keepFields, boolean isUdp) { this.isUdp = isUdp; isBadEvent = false; isIncompleteEvent = false; maxSize = (eventSize < MIN_SIZE) ? MIN_SIZE : eventSize; baos = new ByteArrayOutputStream(eventSize); + this.keepFields = keepFields; initHeaderFormats(); } @@ -219,7 +225,7 @@ Event buildEvent() { headers.put(EVENT_STATUS, SyslogStatus.INCOMPLETE.getSyslogStatus()); } - if ((msgBody != null) && (msgBody.length() > 0)) { + if ((msgBody != null) && (msgBody.length() > 0) && !keepFields) { body = msgBody.getBytes(); } else { body = baos.toByteArray(); @@ -380,4 +386,9 @@ public void setEventSize(Integer eventSize) { this.maxSize = eventSize; } -} + public void setKeepFields(Boolean keepFields) { + this.keepFields= keepFields; + } + } + + diff --git a/flume-ng-core/src/test/java/org/apache/flume/source/TestSyslogUdpSource.java b/flume-ng-core/src/test/java/org/apache/flume/source/TestSyslogUdpSource.java index 2d7a429ce9..eae26edcdb 100644 --- a/flume-ng-core/src/test/java/org/apache/flume/source/TestSyslogUdpSource.java +++ b/flume-ng-core/src/test/java/org/apache/flume/source/TestSyslogUdpSource.java @@ -20,6 +20,8 @@ import java.util.ArrayList; import java.util.List; + +import com.google.common.base.Charsets; import org.apache.log4j.Logger; import org.apache.log4j.net.SyslogAppender; @@ -91,6 +93,8 @@ public void testAppend() throws InterruptedException { source.stop(); logger.removeAppender(appender); + String str = new String(e.getBody(), Charsets.UTF_8); + logger.info(str); Assert.assertNotNull(e); Assert.assertEquals(String.valueOf(SyslogAppender.LOG_FTP / 8), e.getHeaders().get(SyslogUtils.SYSLOG_FACILITY)); diff --git a/flume-ng-core/src/test/java/org/apache/flume/source/TestSyslogUtils.java b/flume-ng-core/src/test/java/org/apache/flume/source/TestSyslogUtils.java index 720846460f..898096b1e8 100644 --- a/flume-ng-core/src/test/java/org/apache/flume/source/TestSyslogUtils.java +++ b/flume-ng-core/src/test/java/org/apache/flume/source/TestSyslogUtils.java @@ -162,7 +162,8 @@ public void TestRfc3164HeaderApacheLogWithNulls() throws ParseException { format1, host1, data1); } - public void checkHeader(String msg1, String stamp1, String format1, String host1, String data1) throws ParseException { + public void checkHeader(String msg1, String stamp1, String format1, + String host1, String data1) throws ParseException { SyslogUtils util = new SyslogUtils(false); ChannelBuffer buff = ChannelBuffers.buffer(200); @@ -397,7 +398,7 @@ public void testGoodEventGoodEvent() { @Test public void testExtractBadEventLarge() { String badData1 = "<10> bad bad data bad bad\n"; - SyslogUtils util = new SyslogUtils(5, false); + SyslogUtils util = new SyslogUtils(5, true, false); ChannelBuffer buff = ChannelBuffers.buffer(100); buff.writeBytes(badData1.getBytes()); Event e = util.extractEvent(buff); diff --git a/flume-ng-doc/sphinx/FlumeDeveloperGuide.rst b/flume-ng-doc/sphinx/FlumeDeveloperGuide.rst index 2be9c68362..ee7b89b43d 100644 --- a/flume-ng-doc/sphinx/FlumeDeveloperGuide.rst +++ b/flume-ng-doc/sphinx/FlumeDeveloperGuide.rst @@ -166,7 +166,7 @@ RPC clients - Avro and Thrift As of Flume 1.4.0, Avro is the default RPC protocol. The ``NettyAvroRpcClient`` and ``ThriftRpcClient`` implement the ``RpcClient`` interface. The client needs to create this object with the host and port of -the target Flume agent, and canthen use the ``RpcClient`` to send data into +the target Flume agent, and can then use the ``RpcClient`` to send data into the agent. The following example shows how to use the Flume Client SDK API within a user's data-generating application: diff --git a/flume-ng-doc/sphinx/FlumeUserGuide.rst b/flume-ng-doc/sphinx/FlumeUserGuide.rst index 4892dfcd51..98859cea5a 100644 --- a/flume-ng-doc/sphinx/FlumeUserGuide.rst +++ b/flume-ng-doc/sphinx/FlumeUserGuide.rst @@ -1170,6 +1170,8 @@ Property Name Default Description **host** -- Host name or IP address to bind to **port** -- Port # to bind to eventSize 2500 Maximum size of a single event line, in bytes +keepFields false Setting this to true will preserve the + Timestamp and Hostname in the body of the event. selector.type replicating or multiplexing selector.* replicating Depends on the selector.type value interceptors -- Space-separated list of interceptors From c9ddf93701e16a93f3b69355bc56545effdc7230 Mon Sep 17 00:00:00 2001 From: Hari Shreedharan Date: Fri, 11 Oct 2013 21:26:40 -0700 Subject: [PATCH 029/341] FLUME-2212. Upgrade to Morphlines-0.8.0 (Wolfgang Hoschek via Hari Shreedharan) --- flume-ng-sinks/flume-ng-morphline-solr-sink/pom.xml | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/flume-ng-sinks/flume-ng-morphline-solr-sink/pom.xml b/flume-ng-sinks/flume-ng-morphline-solr-sink/pom.xml index b2640d9c4b..c78788df57 100644 --- a/flume-ng-sinks/flume-ng-morphline-solr-sink/pom.xml +++ b/flume-ng-sinks/flume-ng-morphline-solr-sink/pom.xml @@ -33,7 +33,7 @@ limitations under the License. UTF-8 4.3.0 4.3.0 - 0.7.0 + 0.8.0 1.6.1 2.12.4 From 68fe4d45123473adbef1077c5de20b4dd48d3a1d Mon Sep 17 00:00:00 2001 From: Hari Shreedharan Date: Mon, 14 Oct 2013 15:56:54 -0700 Subject: [PATCH 030/341] FLUME-2159. Remove TestNettyAvroRpcClient.spinThreadsCrazily. (Roshan Naik via Hari Shreedharan) --- .../flume/api/TestNettyAvroRpcClient.java | 45 ------------------- 1 file changed, 45 deletions(-) diff --git a/flume-ng-sdk/src/test/java/org/apache/flume/api/TestNettyAvroRpcClient.java b/flume-ng-sdk/src/test/java/org/apache/flume/api/TestNettyAvroRpcClient.java index 72e331b59d..bfb1fa60e9 100644 --- a/flume-ng-sdk/src/test/java/org/apache/flume/api/TestNettyAvroRpcClient.java +++ b/flume-ng-sdk/src/test/java/org/apache/flume/api/TestNettyAvroRpcClient.java @@ -333,49 +333,4 @@ public void testThrowingServerBatch() throws FlumeException, RpcTestUtils.handlerBatchAppendTest(new ThrowingAvroHandler()); logger.error("Throwing: I should never have gotten here!"); } - - @Test - public void spinThreadsCrazily() throws IOException { - - int initThreadCount = ManagementFactory.getThreadMXBean().getThreadCount(); - - // find a port we know is closed by opening a free one then closing it - ServerSocket sock = new ServerSocket(0); - int port = sock.getLocalPort(); - sock.close(); - - Properties props = new Properties(); - props.put(RpcClientConfigurationConstants.CONFIG_CLIENT_TYPE, - RpcClientConfigurationConstants.DEFAULT_CLIENT_TYPE); - props.put(RpcClientConfigurationConstants.CONFIG_HOSTS, "h1"); - props.put(RpcClientConfigurationConstants.CONFIG_HOSTS_PREFIX + "h1", - "localhost:" + port); - props.put(RpcClientConfigurationConstants.CONFIG_CONNECT_TIMEOUT, "20"); - props.put(RpcClientConfigurationConstants.CONFIG_REQUEST_TIMEOUT, "20"); - props.put(RpcClientConfigurationConstants.CONFIG_BATCH_SIZE, "1"); - - for (int i = 0; i < 1000; i++) { - RpcClient client = null; - try { - client = RpcClientFactory.getDefaultInstance("localhost", port); - client.append(EventBuilder.withBody("Hello", Charset.forName("UTF-8"))); - } catch (FlumeException e) { - logger.warn("Unexpected error", e); - } catch (EventDeliveryException e) { - logger.warn("Expected error", e); - } finally { - if (client != null) { - client.close(); - } - } - } - - int threadCount = ManagementFactory.getThreadMXBean().getThreadCount(); - logger.warn("Init thread count: {}, thread count: {}", - initThreadCount, threadCount); - Assert.assertEquals("Thread leak in RPC client", - initThreadCount, threadCount); - - } - } From c420fad5d03dc8d17dce7fe3e59bf3b742f3d22d Mon Sep 17 00:00:00 2001 From: Hari Shreedharan Date: Wed, 16 Oct 2013 14:24:13 -0700 Subject: [PATCH 031/341] FLUME-2213. MorphlineInterceptor should share metric registry across threads for better (aggregate) reporting (Wolfgang Hoschek via Hari Shreedharan) --- .../solr/morphline/MorphlineHandlerImpl.java | 67 +++++++++++++------ 1 file changed, 47 insertions(+), 20 deletions(-) diff --git a/flume-ng-sinks/flume-ng-morphline-solr-sink/src/main/java/org/apache/flume/sink/solr/morphline/MorphlineHandlerImpl.java b/flume-ng-sinks/flume-ng-morphline-solr-sink/src/main/java/org/apache/flume/sink/solr/morphline/MorphlineHandlerImpl.java index ea76322976..cb88dc28e8 100644 --- a/flume-ng-sinks/flume-ng-morphline-solr-sink/src/main/java/org/apache/flume/sink/solr/morphline/MorphlineHandlerImpl.java +++ b/flume-ng-sinks/flume-ng-morphline-solr-sink/src/main/java/org/apache/flume/sink/solr/morphline/MorphlineHandlerImpl.java @@ -31,8 +31,12 @@ import com.cloudera.cdk.morphline.base.Compiler; import com.cloudera.cdk.morphline.base.FaultTolerance; import com.cloudera.cdk.morphline.base.Fields; +import com.cloudera.cdk.morphline.base.Metrics; import com.cloudera.cdk.morphline.base.Notifications; +import com.codahale.metrics.Meter; import com.codahale.metrics.MetricRegistry; +import com.codahale.metrics.SharedMetricRegistries; +import com.codahale.metrics.Timer; import com.typesafe.config.Config; import com.typesafe.config.ConfigFactory; @@ -46,6 +50,11 @@ public class MorphlineHandlerImpl implements MorphlineHandler { private Command finalChild; private String morphlineFileAndId; + private Timer mappingTimer; + private Meter numRecords; + private Meter numFailedRecords; + private Meter numExceptionRecords; + public static final String MORPHLINE_FILE_PARAM = "morphlineFile"; public static final String MORPHLINE_ID_PARAM = "morphlineId"; @@ -69,6 +78,13 @@ void setFinalChild(Command finalChild) { @Override public void configure(Context context) { + String morphlineFile = context.getString(MORPHLINE_FILE_PARAM); + String morphlineId = context.getString(MORPHLINE_ID_PARAM); + if (morphlineFile == null || morphlineFile.trim().length() == 0) { + throw new MorphlineCompilationException("Missing parameter: " + MORPHLINE_FILE_PARAM, null); + } + morphlineFileAndId = morphlineFile + "@" + morphlineId; + if (morphlineContext == null) { FaultTolerance faultTolerance = new FaultTolerance( context.getBoolean(FaultTolerance.IS_PRODUCTION_MODE, false), @@ -77,37 +93,48 @@ public void configure(Context context) { morphlineContext = new MorphlineContext.Builder() .setExceptionHandler(faultTolerance) - .setMetricRegistry(new MetricRegistry()) + .setMetricRegistry(SharedMetricRegistries.getOrCreate(morphlineFileAndId)) .build(); } - String morphlineFile = context.getString(MORPHLINE_FILE_PARAM); - String morphlineId = context.getString(MORPHLINE_ID_PARAM); - if (morphlineFile == null || morphlineFile.trim().length() == 0) { - throw new MorphlineCompilationException("Missing parameter: " + MORPHLINE_FILE_PARAM, null); - } Config override = ConfigFactory.parseMap(context.getSubProperties(MORPHLINE_VARIABLE_PARAM + ".")); morphline = new Compiler().compile(new File(morphlineFile), morphlineId, morphlineContext, finalChild, override); - morphlineFileAndId = morphlineFile + "@" + morphlineId; + + this.mappingTimer = morphlineContext.getMetricRegistry().timer( + MetricRegistry.name("morphline.app", Metrics.ELAPSED_TIME)); + this.numRecords = morphlineContext.getMetricRegistry().meter( + MetricRegistry.name("morphline.app", Metrics.NUM_RECORDS)); + this.numFailedRecords = morphlineContext.getMetricRegistry().meter( + MetricRegistry.name("morphline.app", "numFailedRecords")); + this.numExceptionRecords = morphlineContext.getMetricRegistry().meter( + MetricRegistry.name("morphline.app", "numExceptionRecords")); } @Override public void process(Event event) { - Record record = new Record(); - for (Entry entry : event.getHeaders().entrySet()) { - record.put(entry.getKey(), entry.getValue()); - } - byte[] bytes = event.getBody(); - if (bytes != null && bytes.length > 0) { - record.put(Fields.ATTACHMENT_BODY, bytes); - } + numRecords.mark(); + Timer.Context timerContext = mappingTimer.time(); try { - Notifications.notifyStartSession(morphline); - if (!morphline.process(record)) { - LOG.warn("Morphline {} failed to process record: {}", morphlineFileAndId, record); + Record record = new Record(); + for (Entry entry : event.getHeaders().entrySet()) { + record.put(entry.getKey(), entry.getValue()); + } + byte[] bytes = event.getBody(); + if (bytes != null && bytes.length > 0) { + record.put(Fields.ATTACHMENT_BODY, bytes); + } + try { + Notifications.notifyStartSession(morphline); + if (!morphline.process(record)) { + numFailedRecords.mark(); + LOG.warn("Morphline {} failed to process record: {}", morphlineFileAndId, record); + } + } catch (RuntimeException t) { + numExceptionRecords.mark(); + morphlineContext.getExceptionHandler().handleException(t, record); } - } catch (RuntimeException t) { - morphlineContext.getExceptionHandler().handleException(t, record); + } finally { + timerContext.stop(); } } From 8db5de8f85f79d91818f85a241faec5d8eee9b54 Mon Sep 17 00:00:00 2001 From: Roshan Naik Date: Mon, 21 Oct 2013 12:06:51 -0700 Subject: [PATCH 032/341] FLUME-2064: Typo/Grammar in flume main user doc under Scribe (Ashish Paliwal via Roshan Naik) --- flume-ng-doc/sphinx/FlumeUserGuide.rst | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/flume-ng-doc/sphinx/FlumeUserGuide.rst b/flume-ng-doc/sphinx/FlumeUserGuide.rst index 98859cea5a..e38bb672e3 100644 --- a/flume-ng-doc/sphinx/FlumeUserGuide.rst +++ b/flume-ng-doc/sphinx/FlumeUserGuide.rst @@ -1465,7 +1465,7 @@ Scribe Source Scribe is another type of ingest system. To adopt existing Scribe ingest system, Flume should use ScribeSource based on Thrift with compatible transfering protocol. -The deployment of Scribe please following guide from Facebook. +For deployment of Scribe please follow the guide from Facebook. Required properties are in **bold**. ============== =========== ============================================== From 730c822c8fd3c393558ee63b48c82bb5a0763266 Mon Sep 17 00:00:00 2001 From: Mike Percy Date: Mon, 21 Oct 2013 12:04:22 -0700 Subject: [PATCH 033/341] FLUME-1666. Oops, forgot new test in previous commit --- .../flume/source/TestSyslogTcpSource.java | 136 ++++++++++++++++++ 1 file changed, 136 insertions(+) create mode 100644 flume-ng-core/src/test/java/org/apache/flume/source/TestSyslogTcpSource.java diff --git a/flume-ng-core/src/test/java/org/apache/flume/source/TestSyslogTcpSource.java b/flume-ng-core/src/test/java/org/apache/flume/source/TestSyslogTcpSource.java new file mode 100644 index 0000000000..a6a1d5bba4 --- /dev/null +++ b/flume-ng-core/src/test/java/org/apache/flume/source/TestSyslogTcpSource.java @@ -0,0 +1,136 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, + * software distributed under the License is distributed on an + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + * KIND, either express or implied. See the License for the + * specific language governing permissions and limitations + * under the License. + */ +package org.apache.flume.source; + +import com.google.common.base.Charsets; +import org.apache.flume.Channel; +import org.apache.flume.ChannelSelector; +import org.apache.flume.Context; +import org.apache.flume.Event; +import org.apache.flume.Transaction; +import org.apache.flume.channel.ChannelProcessor; +import org.apache.flume.channel.MemoryChannel; +import org.apache.flume.channel.ReplicatingChannelSelector; +import org.apache.flume.conf.Configurables; +import org.joda.time.DateTime; +import org.junit.Assert; +import org.junit.Test; +import org.slf4j.LoggerFactory; + +import java.io.IOException; +import java.net.InetAddress; +import java.net.Socket; +import java.util.ArrayList; +import java.util.List; + +public class TestSyslogTcpSource { + private static final org.slf4j.Logger logger = + LoggerFactory.getLogger(TestSyslogTcpSource.class); + private SyslogTcpSource source; + private Channel channel; + private static final int TEST_SYSLOG_PORT = 0; + private final DateTime time = new DateTime(); + private final String stamp1 = time.toString(); + private final String host1 = "localhost.localdomain"; + private final String data1 = "test syslog data"; + private final String bodyWithTandH = stamp1 + " " + host1 + " " + data1; + // Helper function to generate a syslog message. + private byte[] getEvent() { + // timestamp with 'Z' appended, translates to UTC + final String msg1 = "<10>" + stamp1 + " " + host1 + " " + data1 + "\n"; + return msg1.getBytes(); + } + + private void init(boolean keepFields){ + source = new SyslogTcpSource(); + channel = new MemoryChannel(); + + Configurables.configure(channel, new Context()); + + List channels = new ArrayList(); + channels.add(channel); + + ChannelSelector rcs = new ReplicatingChannelSelector(); + rcs.setChannels(channels); + + source.setChannelProcessor(new ChannelProcessor(rcs)); + Context context = new Context(); + context.put("port", String.valueOf(TEST_SYSLOG_PORT)); + context.put("keepFields", String.valueOf(keepFields)); + + source.configure(context); + + } + /** Tests the keepFields configuration parameter (enabled or disabled) + using SyslogTcpSource.*/ + private void runKeepFieldsTest(boolean keepFields) throws IOException { + init(keepFields); + source.start(); + // Write some message to the syslog port + Socket syslogSocket; + for (int i = 0; i < 10 ; i++) { + syslogSocket = new Socket( + InetAddress.getLocalHost(), source.getSourcePort()); + syslogSocket.getOutputStream().write(getEvent()); + syslogSocket.close(); + } + + List channelEvents = new ArrayList(); + Transaction txn = channel.getTransaction(); + txn.begin(); + for (int i = 0; i < 10; i++) { + Event e = channel.take(); + if (e == null) { + throw new NullPointerException("Event is null"); + } + channelEvents.add(e); + } + + try { + txn.commit(); + } catch (Throwable t) { + txn.rollback(); + } finally { + txn.close(); + } + + source.stop(); + for (Event e : channelEvents) { + Assert.assertNotNull(e); + String str = new String(e.getBody(), Charsets.UTF_8); + logger.info(str); + if (keepFields) { + Assert.assertArrayEquals(bodyWithTandH.getBytes(), e.getBody()); + } else if (!keepFields) { + Assert.assertArrayEquals(data1.getBytes(), e.getBody()); + } + } + } + + @Test + public void testKeepFields () throws IOException { + runKeepFieldsTest(true); + } + + @Test + public void testRemoveFields() throws IOException{ + runKeepFieldsTest(false); + } + } + From 603bcf2d0ef0d68357d0d40e34484fbdb96aa3f9 Mon Sep 17 00:00:00 2001 From: Hari Shreedharan Date: Thu, 24 Oct 2013 17:47:53 -0700 Subject: [PATCH 034/341] FLUME-2210. UnresolvedAddressException when using multiple hostNames in Elasticsearch sink configuration (Dib Ghosh via Hari Shreedharan) --- .../sink/elasticsearch/ElasticSearchSink.java | 6 ++-- .../elasticsearch/TestElasticSearchSink.java | 30 +++++++++++++++++++ 2 files changed, 33 insertions(+), 3 deletions(-) diff --git a/flume-ng-sinks/flume-ng-elasticsearch-sink/src/main/java/org/apache/flume/sink/elasticsearch/ElasticSearchSink.java b/flume-ng-sinks/flume-ng-elasticsearch-sink/src/main/java/org/apache/flume/sink/elasticsearch/ElasticSearchSink.java index 3286412453..3d01173274 100644 --- a/flume-ng-sinks/flume-ng-elasticsearch-sink/src/main/java/org/apache/flume/sink/elasticsearch/ElasticSearchSink.java +++ b/flume-ng-sinks/flume-ng-elasticsearch-sink/src/main/java/org/apache/flume/sink/elasticsearch/ElasticSearchSink.java @@ -241,9 +241,9 @@ public void configure(Context context) { serverAddresses = new InetSocketTransportAddress[hostNames.length]; for (int i = 0; i < hostNames.length; i++) { - String[] hostPort = hostNames[i].split(":"); - String host = hostPort[0]; - int port = hostPort.length == 2 ? Integer.parseInt(hostPort[1]) + String[] hostPort = hostNames[i].trim().split(":"); + String host = hostPort[0].trim(); + int port = hostPort.length == 2 ? Integer.parseInt(hostPort[1].trim()) : DEFAULT_PORT; serverAddresses[i] = new InetSocketTransportAddress(host, port); } diff --git a/flume-ng-sinks/flume-ng-elasticsearch-sink/src/test/java/org/apache/flume/sink/elasticsearch/TestElasticSearchSink.java b/flume-ng-sinks/flume-ng-elasticsearch-sink/src/test/java/org/apache/flume/sink/elasticsearch/TestElasticSearchSink.java index ad40a3ca55..3f2ec6ec03 100644 --- a/flume-ng-sinks/flume-ng-elasticsearch-sink/src/test/java/org/apache/flume/sink/elasticsearch/TestElasticSearchSink.java +++ b/flume-ng-sinks/flume-ng-elasticsearch-sink/src/test/java/org/apache/flume/sink/elasticsearch/TestElasticSearchSink.java @@ -211,6 +211,21 @@ public void shouldParseMultipleHostUsingDefaultPorts() { assertArrayEquals(expected, fixture.getServerAddresses()); } + @Test + public void shouldParseMultipleHostWithWhitespacesUsingDefaultPorts() { + parameters.put(HOSTNAMES, " 10.5.5.27 , 10.5.5.28 , 10.5.5.29 "); + + fixture = new ElasticSearchSink(); + fixture.configure(new Context(parameters)); + + InetSocketTransportAddress[] expected = { + new InetSocketTransportAddress("10.5.5.27", DEFAULT_PORT), + new InetSocketTransportAddress("10.5.5.28", DEFAULT_PORT), + new InetSocketTransportAddress("10.5.5.29", DEFAULT_PORT) }; + + assertArrayEquals(expected, fixture.getServerAddresses()); + } + @Test public void shouldParseMultipleHostAndPorts() { parameters.put(HOSTNAMES, "10.5.5.27:9300,10.5.5.28:9301,10.5.5.29:9302"); @@ -226,6 +241,21 @@ public void shouldParseMultipleHostAndPorts() { assertArrayEquals(expected, fixture.getServerAddresses()); } + @Test + public void shouldParseMultipleHostAndPortsWithWhitespaces() { + parameters.put(HOSTNAMES, " 10.5.5.27 : 9300 , 10.5.5.28 : 9301 , 10.5.5.29 : 9302 "); + + fixture = new ElasticSearchSink(); + fixture.configure(new Context(parameters)); + + InetSocketTransportAddress[] expected = { + new InetSocketTransportAddress("10.5.5.27", 9300), + new InetSocketTransportAddress("10.5.5.28", 9301), + new InetSocketTransportAddress("10.5.5.29", 9302) }; + + assertArrayEquals(expected, fixture.getServerAddresses()); + } + @Test public void shouldAllowCustomElasticSearchIndexRequestBuilderFactory() throws Exception { From f017ce5aca00d280ad6ee94e63fe3b44c326c5cf Mon Sep 17 00:00:00 2001 From: Hari Shreedharan Date: Thu, 24 Oct 2013 22:18:37 -0700 Subject: [PATCH 035/341] FLUME-2192. AbstractSinkProcessor stop incorrectly calls start (Jeremy Karlson via Hari Shreedharan) --- .../main/java/org/apache/flume/sink/AbstractSinkProcessor.java | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/flume-ng-core/src/main/java/org/apache/flume/sink/AbstractSinkProcessor.java b/flume-ng-core/src/main/java/org/apache/flume/sink/AbstractSinkProcessor.java index 528a309124..1c30592982 100644 --- a/flume-ng-core/src/main/java/org/apache/flume/sink/AbstractSinkProcessor.java +++ b/flume-ng-core/src/main/java/org/apache/flume/sink/AbstractSinkProcessor.java @@ -47,7 +47,7 @@ public void start() { @Override public void stop() { for(Sink s : sinkList) { - s.start(); + s.stop(); } state = LifecycleState.STOP; } From 3cc8cec0ec37e6575efdbe3badcc28eceee017c0 Mon Sep 17 00:00:00 2001 From: Hari Shreedharan Date: Wed, 30 Oct 2013 18:30:49 -0700 Subject: [PATCH 036/341] FLUME-1851. Fix grammatical error in Flume User Guide. (Ashish Paliwal via Hari Shreedharan) --- flume-ng-doc/sphinx/FlumeUserGuide.rst | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/flume-ng-doc/sphinx/FlumeUserGuide.rst b/flume-ng-doc/sphinx/FlumeUserGuide.rst index e38bb672e3..a768383a1e 100644 --- a/flume-ng-doc/sphinx/FlumeUserGuide.rst +++ b/flume-ng-doc/sphinx/FlumeUserGuide.rst @@ -128,7 +128,7 @@ Setting up an agent ------------------- Flume agent configuration is stored in a local configuration file. This is a -text file which has a format follows the Java properties file format. +text file that follows the Java properties file format. Configurations for one or more agents can be specified in the same configuration file. The configuration file includes properties of each source, sink and channel in an agent and how they are wired together to form data From 6dfe63cdcebaa5f8091b4789f4df5f679ccb3596 Mon Sep 17 00:00:00 2001 From: Hari Shreedharan Date: Wed, 30 Oct 2013 23:13:09 -0700 Subject: [PATCH 037/341] FLUME-2206. ElasticSearchSink ttl field modification to mimic Elasticsearch way of specifying TTL (Dib Ghosh via Hari Shreedharan) --- .gitignore | 1 + flume-ng-doc/sphinx/FlumeUserGuide.rst | 7 ++- .../sink/elasticsearch/ElasticSearchSink.java | 50 ++++++++++++++++++- .../ElasticSearchSinkConstants.java | 1 + .../elasticsearch/TestElasticSearchSink.java | 36 +++++++++++++ 5 files changed, 91 insertions(+), 4 deletions(-) diff --git a/.gitignore b/.gitignore index ef0a495c7e..b38739117c 100644 --- a/.gitignore +++ b/.gitignore @@ -17,3 +17,4 @@ derby.log .idea *.iml nb-configuration.xml +.DS_Store diff --git a/flume-ng-doc/sphinx/FlumeUserGuide.rst b/flume-ng-doc/sphinx/FlumeUserGuide.rst index a768383a1e..3a3038c4aa 100644 --- a/flume-ng-doc/sphinx/FlumeUserGuide.rst +++ b/flume-ng-doc/sphinx/FlumeUserGuide.rst @@ -1985,7 +1985,10 @@ indexType logs clusterName elasticsearch Name of the ElasticSearch cluster to connect to batchSize 100 Number of events to be written per txn. ttl -- TTL in days, when set will cause the expired documents to be deleted automatically, - if not set documents will never be automatically deleted + if not set documents will never be automatically deleted. TTL is accepted both in the earlier form of + integer only e.g. a1.sinks.k1.ttl = 5 and also with a qualifier ms (millisecond), s (second), m (minute), + h (hour), d (day) and w (week). Example a1.sinks.k1.ttl = 5d will set TTL to 5 days. Follow + http://www.elasticsearch.org/guide/reference/mapping/ttl-field/ for more information. serializer org.apache.flume.sink.elasticsearch.ElasticSearchLogStashEventSerializer The ElasticSearchIndexRequestBuilderFactory or ElasticSearchEventSerializer to use. Implementations of either class are accepted but ElasticSearchIndexRequestBuilderFactory is preferred. serializer.* -- Properties to be passed to the serializer. @@ -2003,7 +2006,7 @@ Example for agent named a1: a1.sinks.k1.indexType = bar_type a1.sinks.k1.clusterName = foobar_cluster a1.sinks.k1.batchSize = 500 - a1.sinks.k1.ttl = 5 + a1.sinks.k1.ttl = 5d a1.sinks.k1.serializer = org.apache.flume.sink.elasticsearch.ElasticSearchDynamicSerializer a1.sinks.k1.channel = c1 diff --git a/flume-ng-sinks/flume-ng-elasticsearch-sink/src/main/java/org/apache/flume/sink/elasticsearch/ElasticSearchSink.java b/flume-ng-sinks/flume-ng-elasticsearch-sink/src/main/java/org/apache/flume/sink/elasticsearch/ElasticSearchSink.java index 3d01173274..e38ab19b2f 100644 --- a/flume-ng-sinks/flume-ng-elasticsearch-sink/src/main/java/org/apache/flume/sink/elasticsearch/ElasticSearchSink.java +++ b/flume-ng-sinks/flume-ng-elasticsearch-sink/src/main/java/org/apache/flume/sink/elasticsearch/ElasticSearchSink.java @@ -31,9 +31,12 @@ import static org.apache.flume.sink.elasticsearch.ElasticSearchSinkConstants.SERIALIZER; import static org.apache.flume.sink.elasticsearch.ElasticSearchSinkConstants.SERIALIZER_PREFIX; import static org.apache.flume.sink.elasticsearch.ElasticSearchSinkConstants.TTL; +import static org.apache.flume.sink.elasticsearch.ElasticSearchSinkConstants.TTL_REGEX; import java.util.Arrays; import java.util.concurrent.TimeUnit; +import java.util.regex.Matcher; +import java.util.regex.Pattern; import org.apache.commons.lang.StringUtils; import org.apache.flume.Channel; @@ -98,6 +101,9 @@ public class ElasticSearchSink extends AbstractSink implements Configurable { private String clusterName = DEFAULT_CLUSTER_NAME; private String indexName = DEFAULT_INDEX_NAME; private String indexType = DEFAULT_INDEX_TYPE; + private final Pattern pattern + = Pattern.compile(TTL_REGEX, Pattern.CASE_INSENSITIVE); + private Matcher matcher = pattern.matcher(""); private InetSocketTransportAddress[] serverAddresses; @@ -269,8 +275,7 @@ public void configure(Context context) { } if (StringUtils.isNotBlank(context.getString(TTL))) { - this.ttlMs = TimeUnit.DAYS.toMillis(Integer.parseInt(context - .getString(TTL))); + this.ttlMs = parseTTL(context.getString(TTL)); Preconditions.checkState(ttlMs > 0, TTL + " must be greater than 0 or not set."); } @@ -353,6 +358,47 @@ private void openConnection() { sinkCounter.incrementConnectionCreatedCount(); } + /* + * Returns TTL value of ElasticSearch index in milliseconds + * when TTL specifier is "ms" / "s" / "m" / "h" / "d" / "w". + * In case of unknown specifier TTL is not set. When specifier + * is not provided it defaults to days in milliseconds where the number + * of days is parsed integer from TTL string provided by user. + *

+ * Elasticsearch supports ttl values being provided in the format: 1d / 1w / 1ms / 1s / 1h / 1m + * specify a time unit like d (days), m (minutes), h (hours), ms (milliseconds) or w (weeks), + * milliseconds is used as default unit. + * http://www.elasticsearch.org/guide/reference/mapping/ttl-field/. + * @param ttl TTL value provided by user in flume configuration file for the sink + * @return the ttl value in milliseconds + */ + private long parseTTL(String ttl){ + matcher = matcher.reset(ttl); + while (matcher.find()) { + if (matcher.group(2).equals("ms")) { + return Long.parseLong(matcher.group(1)); + } else if (matcher.group(2).equals("s")) { + return TimeUnit.SECONDS.toMillis(Integer.parseInt(matcher.group(1))); + } else if (matcher.group(2).equals("m")) { + return TimeUnit.MINUTES.toMillis(Integer.parseInt(matcher.group(1))); + } else if (matcher.group(2).equals("h")) { + return TimeUnit.HOURS.toMillis(Integer.parseInt(matcher.group(1))); + } else if (matcher.group(2).equals("d")) { + return TimeUnit.DAYS.toMillis(Integer.parseInt(matcher.group(1))); + } else if (matcher.group(2).equals("w")) { + return TimeUnit.DAYS.toMillis(7 * Integer.parseInt(matcher.group(1))); + } else if (matcher.group(2).equals("")) { + logger.info("TTL qualifier is empty. Defaulting to day qualifier."); + return TimeUnit.DAYS.toMillis(Integer.parseInt(matcher.group(1))); + } else { + logger.debug("Unknown TTL qualifier provided. Setting TTL to 0."); + return 0; + } + } + logger.info("TTL not provided. Skipping the TTL config by returning 0."); + return 0; + } + /* * FOR TESTING ONLY... * diff --git a/flume-ng-sinks/flume-ng-elasticsearch-sink/src/main/java/org/apache/flume/sink/elasticsearch/ElasticSearchSinkConstants.java b/flume-ng-sinks/flume-ng-elasticsearch-sink/src/main/java/org/apache/flume/sink/elasticsearch/ElasticSearchSinkConstants.java index 7f75e22212..dd0c59d1d9 100644 --- a/flume-ng-sinks/flume-ng-elasticsearch-sink/src/main/java/org/apache/flume/sink/elasticsearch/ElasticSearchSinkConstants.java +++ b/flume-ng-sinks/flume-ng-elasticsearch-sink/src/main/java/org/apache/flume/sink/elasticsearch/ElasticSearchSinkConstants.java @@ -78,4 +78,5 @@ public class ElasticSearchSinkConstants { public static final String DEFAULT_INDEX_NAME = "flume"; public static final String DEFAULT_INDEX_TYPE = "log"; public static final String DEFAULT_CLUSTER_NAME = "elasticsearch"; + public static final String TTL_REGEX = "^(\\d+)(\\D*)"; } diff --git a/flume-ng-sinks/flume-ng-elasticsearch-sink/src/test/java/org/apache/flume/sink/elasticsearch/TestElasticSearchSink.java b/flume-ng-sinks/flume-ng-elasticsearch-sink/src/test/java/org/apache/flume/sink/elasticsearch/TestElasticSearchSink.java index 3f2ec6ec03..71789e80b2 100644 --- a/flume-ng-sinks/flume-ng-elasticsearch-sink/src/test/java/org/apache/flume/sink/elasticsearch/TestElasticSearchSink.java +++ b/flume-ng-sinks/flume-ng-elasticsearch-sink/src/test/java/org/apache/flume/sink/elasticsearch/TestElasticSearchSink.java @@ -31,6 +31,8 @@ import static org.junit.Assert.assertTrue; import java.io.IOException; +import java.util.Map; +import java.util.HashMap; import java.util.TimeZone; import java.util.concurrent.TimeUnit; @@ -286,6 +288,40 @@ public void shouldAllowCustomElasticSearchIndexRequestBuilderFactory() assertTrue(CustomElasticSearchIndexRequestBuilderFactory.hasContext); } + @Test + public void shouldParseFullyQualifiedTTLs(){ + Map testTTLMap = new HashMap(); + testTTLMap.put("1ms", Long.valueOf(1)); + testTTLMap.put("1s", Long.valueOf(1000)); + testTTLMap.put("1m", Long.valueOf(60000)); + testTTLMap.put("1h", Long.valueOf(3600000)); + testTTLMap.put("1d", Long.valueOf(86400000)); + testTTLMap.put("1w", Long.valueOf(604800000)); + testTTLMap.put("1", Long.valueOf(86400000)); + + parameters.put(HOSTNAMES, "10.5.5.27"); + parameters.put(CLUSTER_NAME, "testing-cluster-name"); + parameters.put(INDEX_NAME, "testing-index-name"); + parameters.put(INDEX_TYPE, "testing-index-type"); + + for (String ttl : testTTLMap.keySet()) { + parameters.put(TTL, ttl); + fixture = new ElasticSearchSink(); + fixture.configure(new Context(parameters)); + + InetSocketTransportAddress[] expected = {new InetSocketTransportAddress( + "10.5.5.27", DEFAULT_PORT)}; + + assertEquals("testing-cluster-name", fixture.getClusterName()); + assertEquals("testing-index-name", fixture.getIndexName()); + assertEquals("testing-index-type", fixture.getIndexType()); + System.out.println("TTL MS" + Long.toString(testTTLMap.get(ttl))); + assertEquals((long) testTTLMap.get(ttl), fixture.getTTLMs()); + assertArrayEquals(expected, fixture.getServerAddresses()); + + } + } + public static final class CustomElasticSearchIndexRequestBuilderFactory extends AbstractElasticSearchIndexRequestBuilderFactory { From a89897bec4e7d6f3342ed966c61668e8a8139af5 Mon Sep 17 00:00:00 2001 From: Jarek Jarcec Cecho Date: Thu, 31 Oct 2013 12:54:10 -0700 Subject: [PATCH 038/341] FLUME-2229. Backoff period gets reset too often in OrderSelector (Hari Shreedharan via Jarek Jarcec Cecho) --- .../org/apache/flume/util/OrderSelector.java | 18 +++++++++++------- 1 file changed, 11 insertions(+), 7 deletions(-) diff --git a/flume-ng-sdk/src/main/java/org/apache/flume/util/OrderSelector.java b/flume-ng-sdk/src/main/java/org/apache/flume/util/OrderSelector.java index e869930235..fd9e81fddd 100644 --- a/flume-ng-sdk/src/main/java/org/apache/flume/util/OrderSelector.java +++ b/flume-ng-sdk/src/main/java/org/apache/flume/util/OrderSelector.java @@ -22,6 +22,7 @@ import java.util.LinkedHashMap; import java.util.List; import java.util.Map; +import java.util.concurrent.TimeUnit; /** * A basic implementation of an order selector that implements a simple @@ -39,7 +40,8 @@ public abstract class OrderSelector { private static final int EXP_BACKOFF_COUNTER_LIMIT = 16; - private static final long CONSIDER_SEQUENTIAL_RANGE = 2000l; + private static final long CONSIDER_SEQUENTIAL_RANGE = TimeUnit.HOURS + .toMillis(1); private static final long MAX_TIMEOUT = 30000l; private final Map stateMap = new LinkedHashMap(); @@ -92,12 +94,14 @@ public void informFailure(T failedObject) { long now = System.currentTimeMillis(); long delta = now - state.lastFail; - //Should we consider this as a new failure? If the failure happened - //within backoff length + a grace period (failed within - //grace period after the component started up again, don't consider this - //a new sequential failure - the component might have failed again while - //trying to recover. If the failure is outside backedoff time + grace period - //consider it a new failure and increase the backoff length. + /* + * When do we increase the backoff period? + * We basically calculate the time difference between the last failure + * and the current one. If this failure happened within one hour of the + * last backoff period getting over, then we increase the timeout, + * since the object did not recover yet. Else we assume this is a fresh + * failure and reset the count. + */ long lastBackoffLength = Math.min(maxTimeout, 1000 * (1 << state.sequentialFails)); long allowableDiff = lastBackoffLength + CONSIDER_SEQUENTIAL_RANGE; if (allowableDiff > delta) { From e026545183f577d21850162257152ba38a3f6f9f Mon Sep 17 00:00:00 2001 From: Roshan Naik Date: Wed, 6 Nov 2013 19:30:42 -0800 Subject: [PATCH 039/341] FLUME-2065. Regex Extractor Interceptor config agent name inconsistent with rest of docs (Ashish Paliwal via Roshan Naik) --- flume-ng-doc/sphinx/FlumeUserGuide.rst | 20 ++++++++++---------- 1 file changed, 10 insertions(+), 10 deletions(-) diff --git a/flume-ng-doc/sphinx/FlumeUserGuide.rst b/flume-ng-doc/sphinx/FlumeUserGuide.rst index 3a3038c4aa..aae63b7e61 100644 --- a/flume-ng-doc/sphinx/FlumeUserGuide.rst +++ b/flume-ng-doc/sphinx/FlumeUserGuide.rst @@ -2744,11 +2744,11 @@ If the Flume event body contained ``1:2:3.4foobar5`` and the following configura .. code-block:: properties - agent.sources.r1.interceptors.i1.regex = (\\d):(\\d):(\\d) - agent.sources.r1.interceptors.i1.serializers = s1 s2 s3 - agent.sources.r1.interceptors.i1.serializers.s1.name = one - agent.sources.r1.interceptors.i1.serializers.s2.name = two - agent.sources.r1.interceptors.i1.serializers.s3.name = three + a1.sources.r1.interceptors.i1.regex = (\\d):(\\d):(\\d) + a1.sources.r1.interceptors.i1.serializers = s1 s2 s3 + a1.sources.r1.interceptors.i1.serializers.s1.name = one + a1.sources.r1.interceptors.i1.serializers.s2.name = two + a1.sources.r1.interceptors.i1.serializers.s3.name = three The extracted event will contain the same body but the following headers will have been added ``one=>1, two=>2, three=>3`` @@ -2759,11 +2759,11 @@ If the Flume event body contained ``2012-10-18 18:47:57,614 some log line`` and .. code-block:: properties - agent.sources.r1.interceptors.i1.regex = ^(?:\\n)?(\\d\\d\\d\\d-\\d\\d-\\d\\d\\s\\d\\d:\\d\\d) - agent.sources.r1.interceptors.i1.serializers = s1 - agent.sources.r1.interceptors.i1.serializers.s1.type = org.apache.flume.interceptor.RegexExtractorInterceptorMillisSerializer - agent.sources.r1.interceptors.i1.serializers.s1.name = timestamp - agent.sources.r1.interceptors.i1.serializers.s1.pattern = yyyy-MM-dd HH:mm + a1.sources.r1.interceptors.i1.regex = ^(?:\\n)?(\\d\\d\\d\\d-\\d\\d-\\d\\d\\s\\d\\d:\\d\\d) + a1.sources.r1.interceptors.i1.serializers = s1 + a1.sources.r1.interceptors.i1.serializers.s1.type = org.apache.flume.interceptor.RegexExtractorInterceptorMillisSerializer + a1.sources.r1.interceptors.i1.serializers.s1.name = timestamp + a1.sources.r1.interceptors.i1.serializers.s1.pattern = yyyy-MM-dd HH:mm the extracted event will contain the same body but the following headers will have been added ``timestamp=>1350611220000`` From d3f5123c4d6cdbe4e5cca6e7e141e507bb1103a7 Mon Sep 17 00:00:00 2001 From: Roshan Naik Date: Thu, 7 Nov 2013 11:42:05 -0800 Subject: [PATCH 040/341] FLUME-2233. MemoryChannel lock contention on every put due to bytesRemaining Semaphore (Hari Shreedharan via Roshan Naik) --- .../apache/flume/channel/MemoryChannel.java | 25 +++---- .../flume/channel/TestMemoryChannel.java | 65 +++++++++++++++---- 2 files changed, 64 insertions(+), 26 deletions(-) diff --git a/flume-ng-core/src/main/java/org/apache/flume/channel/MemoryChannel.java b/flume-ng-core/src/main/java/org/apache/flume/channel/MemoryChannel.java index 688323db13..f10a79f24a 100644 --- a/flume-ng-core/src/main/java/org/apache/flume/channel/MemoryChannel.java +++ b/flume-ng-core/src/main/java/org/apache/flume/channel/MemoryChannel.java @@ -79,18 +79,11 @@ protected void doPut(Event event) throws InterruptedException { channelCounter.incrementEventPutAttemptCount(); int eventByteSize = (int)Math.ceil(estimateEventSize(event)/byteCapacitySlotSize); - if (bytesRemaining.tryAcquire(eventByteSize, keepAlive, TimeUnit.SECONDS)) { - if(!putList.offer(event)) { - throw new ChannelException("Put queue for MemoryTransaction of capacity " + - putList.size() + " full, consider committing more frequently, " + - "increasing capacity or increasing thread count"); - } - } else { - throw new ChannelException("Put queue for MemoryTransaction of byteCapacity " + - (lastByteCapacity * (int)byteCapacitySlotSize) + " bytes cannot add an " + - " event of size " + estimateEventSize(event) + " bytes because " + - (bytesRemaining.availablePermits() * (int)byteCapacitySlotSize) + " bytes are already used." + - " Try consider comitting more frequently, increasing byteCapacity or increasing thread count"); + if (!putList.offer(event)) { + throw new ChannelException( + "Put queue for MemoryTransaction of capacity " + + putList.size() + " full, consider committing more frequently, " + + "increasing capacity or increasing thread count"); } putByteCounter += eventByteSize; } @@ -124,7 +117,15 @@ protected Event doTake() throws InterruptedException { protected void doCommit() throws InterruptedException { int remainingChange = takeList.size() - putList.size(); if(remainingChange < 0) { + if(!bytesRemaining.tryAcquire(putByteCounter, keepAlive, + TimeUnit.SECONDS)) { + throw new ChannelException("Cannot commit transaction. Heap space " + + "limit of " + byteCapacity + "reached. Please increase heap space" + + " allocated to the channel as the sinks may not be keeping up " + + "with the sources"); + } if(!queueRemaining.tryAcquire(-remainingChange, keepAlive, TimeUnit.SECONDS)) { + bytesRemaining.release(putByteCounter); throw new ChannelException("Space for commit to queue couldn't be acquired" + " Sinks are likely not keeping up with sources, or the buffer size is too tight"); } diff --git a/flume-ng-core/src/test/java/org/apache/flume/channel/TestMemoryChannel.java b/flume-ng-core/src/test/java/org/apache/flume/channel/TestMemoryChannel.java index a78581aa2a..7851536954 100644 --- a/flume-ng-core/src/test/java/org/apache/flume/channel/TestMemoryChannel.java +++ b/flume-ng-core/src/test/java/org/apache/flume/channel/TestMemoryChannel.java @@ -356,63 +356,100 @@ public void testByteCapacityBufferChangeConfig() { Transaction tx = channel.getTransaction(); tx.begin(); channel.put(EventBuilder.withBody(eventBody)); - + tx.commit(); + tx.close(); + channel.stop(); parms.put("byteCapacity", "1500"); context.putAll(parms); Configurables.configure(channel, context); - + channel.start(); + tx = channel.getTransaction(); + tx.begin(); channel.put(EventBuilder.withBody(eventBody)); try { channel.put(EventBuilder.withBody(eventBody)); + tx.commit(); Assert.fail(); } catch ( ChannelException e ) { //success + tx.rollback(); + } finally { + tx.close(); } - parms.put("byteCapacity", "2500"); + channel.stop(); + parms.put("byteCapacity", "250"); parms.put("byteCapacityBufferPercentage", "20"); context.putAll(parms); Configurables.configure(channel, context); - + channel.start(); + tx = channel.getTransaction(); + tx.begin(); channel.put(EventBuilder.withBody(eventBody)); + tx.commit(); + tx.close(); + channel.stop(); parms.put("byteCapacity", "300"); context.putAll(parms); Configurables.configure(channel, context); - - channel.put(EventBuilder.withBody(eventBody)); + channel.start(); + tx = channel.getTransaction(); + tx.begin(); try { - channel.put(EventBuilder.withBody(eventBody)); + for(int i = 0; i < 2; i++) { + channel.put(EventBuilder.withBody(eventBody)); + } + tx.commit(); Assert.fail(); } catch ( ChannelException e ) { //success + tx.rollback(); + } finally { + tx.close(); } + channel.stop(); parms.put("byteCapacity", "3300"); context.putAll(parms); Configurables.configure(channel, context); - - channel.put(EventBuilder.withBody(eventBody)); + channel.start(); + tx = channel.getTransaction(); + tx.begin(); try { - channel.put(EventBuilder.withBody(eventBody)); + for(int i = 0; i < 15; i++) { + channel.put(EventBuilder.withBody(eventBody)); + } + tx.commit(); Assert.fail(); } catch ( ChannelException e ) { //success + tx.rollback(); + } finally { + tx.close(); } - + channel.stop(); parms.put("byteCapacity", "4000"); context.putAll(parms); Configurables.configure(channel, context); - - channel.put(EventBuilder.withBody(eventBody)); + channel.start(); + tx = channel.getTransaction(); + tx.begin(); try { - channel.put(EventBuilder.withBody(eventBody)); + for(int i = 0; i < 25; i++) { + channel.put(EventBuilder.withBody(eventBody)); + } + tx.commit(); Assert.fail(); } catch ( ChannelException e ) { //success + tx.rollback(); + } finally { + tx.close(); } + channel.stop(); } /* From e27ae5fdce48100a85e353f97ed8a150afe5a4aa Mon Sep 17 00:00:00 2001 From: Roshan Naik Date: Thu, 7 Nov 2013 13:07:24 -0800 Subject: [PATCH 041/341] FLUME-2231. Add details in Flume Ganglia config in User Guide (Ashish Paliwal via Roshan Naik) --- flume-ng-doc/sphinx/FlumeUserGuide.rst | 6 +++--- 1 file changed, 3 insertions(+), 3 deletions(-) diff --git a/flume-ng-doc/sphinx/FlumeUserGuide.rst b/flume-ng-doc/sphinx/FlumeUserGuide.rst index aae63b7e61..0f12427932 100644 --- a/flume-ng-doc/sphinx/FlumeUserGuide.rst +++ b/flume-ng-doc/sphinx/FlumeUserGuide.rst @@ -2956,9 +2956,9 @@ and can be specified in the flume-env.sh: Property Name Default Description ======================= ======= ===================================================================================== **type** -- The component type name, has to be ``ganglia`` -**hosts** -- Comma-separated list of ``hostname:port`` -pollInterval 60 Time, in seconds, between consecutive reporting to ganglia server -isGanglia3 false Ganglia server version is 3. By default, Flume sends in ganglia 3.1 format +**hosts** -- Comma-separated list of ``hostname:port`` of Ganglia servers +pollInterval 60 Time, in seconds, between consecutive reporting to Ganglia server +isGanglia3 false Ganglia server version is 3. By default, Flume sends in Ganglia 3.1 format ======================= ======= ===================================================================================== We can start Flume with Ganglia support as follows:: From 705abaf00fbf8ee69ac88cbccae47c1a33f4b4b2 Mon Sep 17 00:00:00 2001 From: Jarek Jarcec Cecho Date: Thu, 7 Nov 2013 14:53:04 -0800 Subject: [PATCH 042/341] FLUME-2235. idleFuture should be cancelled at the start of append (Hari Shreedharan via Jarek Jarcec Cecho) --- .../apache/flume/sink/hdfs/BucketWriter.java | 21 +++++++++++++++++++ 1 file changed, 21 insertions(+) diff --git a/flume-ng-sinks/flume-hdfs-sink/src/main/java/org/apache/flume/sink/hdfs/BucketWriter.java b/flume-ng-sinks/flume-hdfs-sink/src/main/java/org/apache/flume/sink/hdfs/BucketWriter.java index 65f4d2c299..200d457229 100644 --- a/flume-ng-sinks/flume-hdfs-sink/src/main/java/org/apache/flume/sink/hdfs/BucketWriter.java +++ b/flume-ng-sinks/flume-hdfs-sink/src/main/java/org/apache/flume/sink/hdfs/BucketWriter.java @@ -375,6 +375,27 @@ public Void call() throws Exception { public synchronized void append(final Event event) throws IOException, InterruptedException { checkAndThrowInterruptedException(); + // If idleFuture is not null, cancel it before we move forward to avoid a + // close call in the middle of the append. + if(idleFuture != null) { + idleFuture.cancel(false); + // There is still a small race condition - if the idleFuture is already + // running, interrupting it can cause HDFS close operation to throw - + // so we cannot interrupt it while running. If the future could not be + // cancelled, it is already running - wait for it to finish before + // attempting to write. + if(!idleFuture.isDone()) { + try { + idleFuture.get(callTimeout, TimeUnit.MILLISECONDS); + } catch (TimeoutException ex) { + LOG.warn("Timeout while trying to cancel closing of idle file. Idle" + + " file close may have failed", ex); + } catch (Exception ex) { + LOG.warn("Error while trying to cancel closing of idle file. ", ex); + } + } + idleFuture = null; + } if (!isOpen) { if(idleClosed) { throw new IOException("This bucket writer was closed due to idling and this handle " + From c23448fc959844eece5a8ab2dbf091c2c4973a26 Mon Sep 17 00:00:00 2001 From: Mike Percy Date: Thu, 5 Dec 2013 12:58:03 -0800 Subject: [PATCH 043/341] FLUME-2255. Correctly handle ChannelExceptions in SpoolingDirectorySource (Hari Shreedharan via Mike Percy) --- .../flume/source/SpoolDirectorySource.java | 47 ++++++++++++++- ...DirectorySourceConfigurationConstants.java | 4 ++ .../source/TestSpoolDirectorySource.java | 58 +++++++++++++++++++ flume-ng-doc/sphinx/FlumeUserGuide.rst | 1 + 4 files changed, 108 insertions(+), 2 deletions(-) diff --git a/flume-ng-core/src/main/java/org/apache/flume/source/SpoolDirectorySource.java b/flume-ng-core/src/main/java/org/apache/flume/source/SpoolDirectorySource.java index 72c405908f..016021590e 100644 --- a/flume-ng-core/src/main/java/org/apache/flume/source/SpoolDirectorySource.java +++ b/flume-ng-core/src/main/java/org/apache/flume/source/SpoolDirectorySource.java @@ -67,6 +67,9 @@ public class SpoolDirectorySource extends AbstractSource implements private SourceCounter sourceCounter; ReliableSpoolingFileEventReader reader; private ScheduledExecutorService executor; + private boolean backoff = true; + private boolean hitChannelException = false; + private int maxBackoff; @Override public synchronized void start() { @@ -161,6 +164,8 @@ public synchronized void configure(Context context) { deserializerContext.put(LineDeserializer.MAXLINE_KEY, bufferMaxLineLength.toString()); } + + maxBackoff = context.getInteger(MAX_BACKOFF, DEFAULT_MAX_BACKOFF); if (sourceCounter == null) { sourceCounter = new SourceCounter(getName()); } @@ -171,6 +176,28 @@ protected boolean hasFatalError() { return hasFatalError; } + + + /** + * The class always backs off, this exists only so that we can test without + * taking a really long time. + * @param backoff - whether the source should backoff if the channel is full + */ + @VisibleForTesting + protected void setBackOff(boolean backoff) { + this.backoff = backoff; + } + + @VisibleForTesting + protected boolean hitChannelException() { + return hitChannelException; + } + + @VisibleForTesting + protected SourceCounter getSourceCounter() { + return sourceCounter; + } + private class SpoolDirectoryRunnable implements Runnable { private ReliableSpoolingFileEventReader reader; private SourceCounter sourceCounter; @@ -183,6 +210,7 @@ public SpoolDirectoryRunnable(ReliableSpoolingFileEventReader reader, @Override public void run() { + int backoffInterval = 250; try { while (true) { List events = reader.readEvents(batchSize); @@ -192,8 +220,23 @@ public void run() { sourceCounter.addToEventReceivedCount(events.size()); sourceCounter.incrementAppendBatchReceivedCount(); - getChannelProcessor().processEventBatch(events); - reader.commit(); + try { + getChannelProcessor().processEventBatch(events); + reader.commit(); + } catch (ChannelException ex) { + logger.warn("The channel is full, and cannot write data now. The " + + "source will try again after " + String.valueOf(backoffInterval) + + " milliseconds"); + hitChannelException = true; + if (backoff) { + TimeUnit.MILLISECONDS.sleep(backoffInterval); + backoffInterval = backoffInterval << 1; + backoffInterval = backoffInterval >= maxBackoff ? maxBackoff : + backoffInterval; + } + continue; + } + backoffInterval = 250; sourceCounter.addToEventAcceptedCount(events.size()); sourceCounter.incrementAppendBatchAcceptedCount(); } diff --git a/flume-ng-core/src/main/java/org/apache/flume/source/SpoolDirectorySourceConfigurationConstants.java b/flume-ng-core/src/main/java/org/apache/flume/source/SpoolDirectorySourceConfigurationConstants.java index 7bfb0ee924..a2befe83be 100644 --- a/flume-ng-core/src/main/java/org/apache/flume/source/SpoolDirectorySourceConfigurationConstants.java +++ b/flume-ng-core/src/main/java/org/apache/flume/source/SpoolDirectorySourceConfigurationConstants.java @@ -74,4 +74,8 @@ public class SpoolDirectorySourceConfigurationConstants { public static final String DECODE_ERROR_POLICY = "decodeErrorPolicy"; public static final String DEFAULT_DECODE_ERROR_POLICY = DecodeErrorPolicy.FAIL.name(); + + public static final String MAX_BACKOFF = "maxBackoff"; + + public static final Integer DEFAULT_MAX_BACKOFF = 4000; } diff --git a/flume-ng-core/src/test/java/org/apache/flume/source/TestSpoolDirectorySource.java b/flume-ng-core/src/test/java/org/apache/flume/source/TestSpoolDirectorySource.java index 837cf152ba..9a546a57b4 100644 --- a/flume-ng-core/src/test/java/org/apache/flume/source/TestSpoolDirectorySource.java +++ b/flume-ng-core/src/test/java/org/apache/flume/source/TestSpoolDirectorySource.java @@ -23,6 +23,7 @@ import java.util.List; import java.util.concurrent.TimeUnit; +import com.google.common.collect.Lists; import org.apache.flume.Channel; import org.apache.flume.ChannelException; import org.apache.flume.ChannelSelector; @@ -33,6 +34,7 @@ import org.apache.flume.channel.MemoryChannel; import org.apache.flume.channel.ReplicatingChannelSelector; import org.apache.flume.conf.Configurables; +import org.apache.flume.instrumentation.SourceCounter; import org.apache.flume.lifecycle.LifecycleController; import org.apache.flume.lifecycle.LifecycleState; import org.junit.After; @@ -163,4 +165,60 @@ public void testReconfigure() throws InterruptedException, IOException { Assert.assertFalse("Fatal error on iteration " + i, source.hasFatalError()); } } + + @Test + public void testSourceDoesNotDieOnFullChannel() throws Exception { + + Context chContext = new Context(); + chContext.put("capacity", "2"); + chContext.put("transactionCapacity", "2"); + chContext.put("keep-alive", "0"); + channel.stop(); + Configurables.configure(channel, chContext); + + channel.start(); + Context context = new Context(); + File f1 = new File(tmpDir.getAbsolutePath() + "/file1"); + + Files.write("file1line1\nfile1line2\nfile1line3\nfile1line4\n" + + "file1line5\nfile1line6\nfile1line7\nfile1line8\n", + f1, Charsets.UTF_8); + + + context.put(SpoolDirectorySourceConfigurationConstants.SPOOL_DIRECTORY, + tmpDir.getAbsolutePath()); + + context.put(SpoolDirectorySourceConfigurationConstants.BATCH_SIZE, "2"); + Configurables.configure(source, context); + source.setBackOff(false); + source.start(); + + // Wait for the source to read enough events to fill up the channel. + while(!source.hitChannelException()) { + Thread.sleep(50); + } + + List dataOut = Lists.newArrayList(); + + for (int i = 0; i < 8; ) { + Transaction tx = channel.getTransaction(); + tx.begin(); + Event e = channel.take(); + if (e != null) { + dataOut.add(new String(e.getBody(), "UTF-8")); + i++; + } + e = channel.take(); + if (e != null) { + dataOut.add(new String(e.getBody(), "UTF-8")); + i++; + } + tx.commit(); + tx.close(); + } + Assert.assertTrue("Expected to hit ChannelException, but did not!", + source.hitChannelException()); + Assert.assertEquals(8, dataOut.size()); + source.stop(); + } } diff --git a/flume-ng-doc/sphinx/FlumeUserGuide.rst b/flume-ng-doc/sphinx/FlumeUserGuide.rst index 0f12427932..8687cb734a 100644 --- a/flume-ng-doc/sphinx/FlumeUserGuide.rst +++ b/flume-ng-doc/sphinx/FlumeUserGuide.rst @@ -951,6 +951,7 @@ fileHeaderKey file Header key to use when appending filename ignorePattern ^$ Regular expression specifying which files to ignore (skip) trackerDir .flumespool Directory to store metadata related to processing of files. If this path is not an absolute path, then it is interpreted as relative to the spoolDir. +maxBackoff 4000 The maximum time (in millis) to wait between consecutive attempts to write to the channel(s) if the channel is full. The source will start at a low backoff and increase it exponentially each time the channel throws a ChannelException, upto the value specified by this parameter. batchSize 100 Granularity at which to batch transfer to the channel inputCharset UTF-8 Character set used by deserializers that treat the input file as text. decodeErrorPolicy ``FAIL`` What to do when we see a non-decodable character in the input file. From 2ea4922025e6db25d8627b522146b8b29c40a62b Mon Sep 17 00:00:00 2001 From: Hari Shreedharan Date: Mon, 9 Dec 2013 15:12:47 -0800 Subject: [PATCH 044/341] FLUME-2262. Log4j Appender should use timeStamp field not getTimestamp, which was not available in older log4j versions. (Brock Noland via Hari Shreedharan) --- .../org/apache/flume/clients/log4jappender/Log4jAppender.java | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/flume-ng-clients/flume-ng-log4jappender/src/main/java/org/apache/flume/clients/log4jappender/Log4jAppender.java b/flume-ng-clients/flume-ng-log4jappender/src/main/java/org/apache/flume/clients/log4jappender/Log4jAppender.java index b07b189701..7c483db5db 100644 --- a/flume-ng-clients/flume-ng-log4jappender/src/main/java/org/apache/flume/clients/log4jappender/Log4jAppender.java +++ b/flume-ng-clients/flume-ng-log4jappender/src/main/java/org/apache/flume/clients/log4jappender/Log4jAppender.java @@ -135,7 +135,7 @@ public synchronized void append(LoggingEvent event) throws FlumeException{ Map hdrs = new HashMap(); hdrs.put(Log4jAvroHeaders.LOGGER_NAME.toString(), event.getLoggerName()); hdrs.put(Log4jAvroHeaders.TIMESTAMP.toString(), - String.valueOf(event.getTimeStamp())); + String.valueOf(event.timeStamp)); //To get the level back simply use //LoggerEvent.toLevel(hdrs.get(Integer.parseInt( From 753e4137918b5bdf559dd50a21db2a832aa1dce3 Mon Sep 17 00:00:00 2001 From: Hari Shreedharan Date: Mon, 9 Dec 2013 16:35:33 -0800 Subject: [PATCH 045/341] FLUME-2238. Provide option to configure worker threads in NettyAvroRpcClient (Cameron Gandevia via Hari Shreedharan) --- flume-ng-doc/sphinx/FlumeUserGuide.rst | 35 ++++---- .../apache/flume/api/NettyAvroRpcClient.java | 86 +++++++++++++++---- .../api/RpcClientConfigurationConstants.java | 6 ++ .../flume/api/TestNettyAvroRpcClient.java | 79 +++++++++++++++-- 4 files changed, 162 insertions(+), 44 deletions(-) diff --git a/flume-ng-doc/sphinx/FlumeUserGuide.rst b/flume-ng-doc/sphinx/FlumeUserGuide.rst index 8687cb734a..0737c441de 100644 --- a/flume-ng-doc/sphinx/FlumeUserGuide.rst +++ b/flume-ng-doc/sphinx/FlumeUserGuide.rst @@ -1638,25 +1638,26 @@ hostname / port pair. The events are taken from the configured Channel in batches of the configured batch size. Required properties are in **bold**. -========================== ======= ============================================== +========================== ===================================================== =========================================================================================== Property Name Default Description -========================== ======= ============================================== +========================== ===================================================== =========================================================================================== **channel** -- -**type** -- The component type name, needs to be ``avro``. -**hostname** -- The hostname or IP address to bind to. -**port** -- The port # to listen on. -batch-size 100 number of event to batch together for send. -connect-timeout 20000 Amount of time (ms) to allow for the first (handshake) request. -request-timeout 20000 Amount of time (ms) to allow for requests after the first. -reset-connection-interval none Amount of time (s) before the connection to the next hop is reset. This will force the Avro Sink to reconnect to the next hop. This will allow the sink to connect to hosts behind a hardware load-balancer when news hosts are added without having to restart the agent. -compression-type none This can be "none" or "deflate". The compression-type must match the compression-type of matching AvroSource -compression-level 6 The level of compression to compress event. 0 = no compression and 1-9 is compression. The higher the number the more compression -ssl false Set to true to enable SSL for this AvroSink. When configuring SSL, you can optionally set a "truststore", "truststore-password", "truststore-type", and specify whether to "trust-all-certs". -trust-all-certs false If this is set to true, SSL server certificates for remote servers (Avro Sources) will not be checked. This should NOT be used in production because it makes it easier for an attacker to execute a man-in-the-middle attack and "listen in" on the encrypted connection. -truststore -- The path to a custom Java truststore file. Flume uses the certificate authority information in this file to determine whether the remote Avro Source's SSL authentication credentials should be trusted. If not specified, the default Java JSSE certificate authority files (typically "jssecacerts" or "cacerts" in the Oracle JRE) will be used. -truststore-password -- The password for the specified truststore. -truststore-type JKS The type of the Java truststore. This can be "JKS" or other supported Java truststore type. -========================== ======= ============================================== +**type** -- The component type name, needs to be ``avro``. +**hostname** -- The hostname or IP address to bind to. +**port** -- The port # to listen on. +batch-size 100 number of event to batch together for send. +connect-timeout 20000 Amount of time (ms) to allow for the first (handshake) request. +request-timeout 20000 Amount of time (ms) to allow for requests after the first. +reset-connection-interval none Amount of time (s) before the connection to the next hop is reset. This will force the Avro Sink to reconnect to the next hop. This will allow the sink to connect to hosts behind a hardware load-balancer when news hosts are added without having to restart the agent. +compression-type none This can be "none" or "deflate". The compression-type must match the compression-type of matching AvroSource +compression-level 6 The level of compression to compress event. 0 = no compression and 1-9 is compression. The higher the number the more compression +ssl false Set to true to enable SSL for this AvroSink. When configuring SSL, you can optionally set a "truststore", "truststore-password", "truststore-type", and specify whether to "trust-all-certs". +trust-all-certs false If this is set to true, SSL server certificates for remote servers (Avro Sources) will not be checked. This should NOT be used in production because it makes it easier for an attacker to execute a man-in-the-middle attack and "listen in" on the encrypted connection. +truststore -- The path to a custom Java truststore file. Flume uses the certificate authority information in this file to determine whether the remote Avro Source's SSL authentication credentials should be trusted. If not specified, the default Java JSSE certificate authority files (typically "jssecacerts" or "cacerts" in the Oracle JRE) will be used. +truststore-password -- The password for the specified truststore. +truststore-type JKS The type of the Java truststore. This can be "JKS" or other supported Java truststore type. +maxIoWorkers 2 * the number of available processors in the machine The maximum number of I/O worker threads. This is configured on the NettyAvroRpcClient NioClientSocketChannelFactory. +========================== ===================================================== =========================================================================================== Example for agent named a1: diff --git a/flume-ng-sdk/src/main/java/org/apache/flume/api/NettyAvroRpcClient.java b/flume-ng-sdk/src/main/java/org/apache/flume/api/NettyAvroRpcClient.java index 9aabdd40ae..a2eb264fc1 100644 --- a/flume-ng-sdk/src/main/java/org/apache/flume/api/NettyAvroRpcClient.java +++ b/flume-ng-sdk/src/main/java/org/apache/flume/api/NettyAvroRpcClient.java @@ -45,6 +45,7 @@ import java.util.concurrent.atomic.AtomicInteger; import java.util.concurrent.locks.Condition; import java.util.concurrent.locks.ReentrantLock; + import javax.net.ssl.SSLContext; import javax.net.ssl.SSLEngine; import javax.net.ssl.TrustManager; @@ -55,6 +56,7 @@ import org.apache.avro.ipc.NettyTransceiver; import org.apache.avro.ipc.Transceiver; import org.apache.avro.ipc.specific.SpecificRequestor; +import org.apache.commons.lang.StringUtils; import org.apache.flume.Event; import org.apache.flume.EventDeliveryException; import org.apache.flume.FlumeException; @@ -99,6 +101,7 @@ public class NettyAvroRpcClient extends AbstractRpcClient .getLogger(NettyAvroRpcClient.class); private boolean enableDeflateCompression; private int compressionLevel; + private int maxIoWorkers; /** * This constructor is intended to be called from {@link RpcClientFactory}. @@ -128,20 +131,34 @@ private void connect(long timeout, TimeUnit tu) throws FlumeException { try { + ExecutorService bossExecutor = + Executors.newCachedThreadPool(new TransceiverThreadFactory( + "Avro " + NettyTransceiver.class.getSimpleName() + " Boss")); + ExecutorService workerExecutor = + Executors.newCachedThreadPool(new TransceiverThreadFactory( + "Avro " + NettyTransceiver.class.getSimpleName() + " I/O Worker")); + if (enableDeflateCompression || enableSsl) { - socketChannelFactory = new SSLCompressionChannelFactory( - Executors.newCachedThreadPool(new TransceiverThreadFactory( - "Avro " + NettyTransceiver.class.getSimpleName() + " Boss")), - Executors.newCachedThreadPool(new TransceiverThreadFactory( - "Avro " + NettyTransceiver.class.getSimpleName() + " I/O Worker")), - enableDeflateCompression, enableSsl, trustAllCerts, compressionLevel, - truststore, truststorePassword, truststoreType); + if (maxIoWorkers >= 1) { + socketChannelFactory = new SSLCompressionChannelFactory( + bossExecutor, workerExecutor, + enableDeflateCompression, enableSsl, trustAllCerts, + compressionLevel, truststore, truststorePassword, truststoreType, + maxIoWorkers); + } else { + socketChannelFactory = new SSLCompressionChannelFactory( + bossExecutor, workerExecutor, + enableDeflateCompression, enableSsl, trustAllCerts, + compressionLevel, truststore, truststorePassword, truststoreType); + } } else { - socketChannelFactory = new NioClientSocketChannelFactory( - Executors.newCachedThreadPool(new TransceiverThreadFactory( - "Avro " + NettyTransceiver.class.getSimpleName() + " Boss")), - Executors.newCachedThreadPool(new TransceiverThreadFactory( - "Avro " + NettyTransceiver.class.getSimpleName() + " I/O Worker"))); + if (maxIoWorkers >= 1) { + socketChannelFactory = new NioClientSocketChannelFactory( + bossExecutor, workerExecutor, maxIoWorkers); + } else { + socketChannelFactory = new NioClientSocketChannelFactory( + bossExecutor, workerExecutor); + } } transceiver = new NettyTransceiver(this.address, @@ -587,6 +604,23 @@ public synchronized void configure(Properties properties) truststoreType = properties.getProperty( RpcClientConfigurationConstants.CONFIG_TRUSTSTORE_TYPE, "JKS"); + String maxIoWorkersStr = properties.getProperty( + RpcClientConfigurationConstants.MAX_IO_WORKERS); + if (!StringUtils.isEmpty(maxIoWorkersStr)) { + try { + maxIoWorkers = Integer.parseInt(maxIoWorkersStr); + } catch (NumberFormatException ex) { + logger.warn ("Invalid maxIOWorkers:" + maxIoWorkersStr + " Using " + + "default maxIOWorkers."); + maxIoWorkers = -1; + } + } + + if (maxIoWorkers < 1) { + logger.warn("Using default maxIOWorkers"); + maxIoWorkers = -1; + } + this.connect(); } @@ -628,13 +662,13 @@ public Thread newThread(Runnable r) { */ private static class SSLCompressionChannelFactory extends NioClientSocketChannelFactory { - private boolean enableCompression; - private int compressionLevel; - private boolean enableSsl; - private boolean trustAllCerts; - private String truststore; - private String truststorePassword; - private String truststoreType; + private final boolean enableCompression; + private final int compressionLevel; + private final boolean enableSsl; + private final boolean trustAllCerts; + private final String truststore; + private final String truststorePassword; + private final String truststoreType; public SSLCompressionChannelFactory(Executor bossExecutor, Executor workerExecutor, boolean enableCompression, boolean enableSsl, boolean trustAllCerts, @@ -650,6 +684,20 @@ public SSLCompressionChannelFactory(Executor bossExecutor, Executor workerExecut this.truststoreType = truststoreType; } + public SSLCompressionChannelFactory(Executor bossExecutor, Executor workerExecutor, + boolean enableCompression, boolean enableSsl, boolean trustAllCerts, + int compressionLevel, String truststore, String truststorePassword, + String truststoreType, int maxIOWorkers) { + super(bossExecutor, workerExecutor, maxIOWorkers); + this.enableCompression = enableCompression; + this.enableSsl = enableSsl; + this.compressionLevel = compressionLevel; + this.trustAllCerts = trustAllCerts; + this.truststore = truststore; + this.truststorePassword = truststorePassword; + this.truststoreType = truststoreType; + } + @Override public SocketChannel newChannel(ChannelPipeline pipeline) { TrustManager[] managers; diff --git a/flume-ng-sdk/src/main/java/org/apache/flume/api/RpcClientConfigurationConstants.java b/flume-ng-sdk/src/main/java/org/apache/flume/api/RpcClientConfigurationConstants.java index 7aa70cbf7d..136c50496a 100644 --- a/flume-ng-sdk/src/main/java/org/apache/flume/api/RpcClientConfigurationConstants.java +++ b/flume-ng-sdk/src/main/java/org/apache/flume/api/RpcClientConfigurationConstants.java @@ -144,6 +144,12 @@ public final class RpcClientConfigurationConstants { public static final String CONFIG_TRUSTSTORE_PASSWORD = "truststore-password"; public static final String CONFIG_TRUSTSTORE_TYPE = "truststore-type"; + /** + * Configuration constants for the NettyAvroRpcClient + * NioClientSocketChannelFactory + */ + public static final String MAX_IO_WORKERS = "maxIoWorkers"; + private RpcClientConfigurationConstants() { // disable explicit object creation } diff --git a/flume-ng-sdk/src/test/java/org/apache/flume/api/TestNettyAvroRpcClient.java b/flume-ng-sdk/src/test/java/org/apache/flume/api/TestNettyAvroRpcClient.java index bfb1fa60e9..cf4f41532e 100644 --- a/flume-ng-sdk/src/test/java/org/apache/flume/api/TestNettyAvroRpcClient.java +++ b/flume-ng-sdk/src/test/java/org/apache/flume/api/TestNettyAvroRpcClient.java @@ -18,28 +18,22 @@ */ package org.apache.flume.api; -import java.io.IOException; -import java.lang.management.ManagementFactory; -import java.net.InetSocketAddress; -import java.net.ServerSocket; import java.nio.charset.Charset; import java.util.ArrayList; import java.util.List; import java.util.Properties; -import org.junit.Test; - import org.apache.avro.ipc.Server; import org.apache.flume.Event; import org.apache.flume.EventDeliveryException; import org.apache.flume.FlumeException; -import org.apache.flume.event.EventBuilder; - import org.apache.flume.api.RpcTestUtils.FailedAvroHandler; import org.apache.flume.api.RpcTestUtils.OKAvroHandler; import org.apache.flume.api.RpcTestUtils.ThrowingAvroHandler; import org.apache.flume.api.RpcTestUtils.UnknownAvroHandler; +import org.apache.flume.event.EventBuilder; import org.junit.Assert; +import org.junit.Test; import org.slf4j.Logger; import org.slf4j.LoggerFactory; @@ -333,4 +327,73 @@ public void testThrowingServerBatch() throws FlumeException, RpcTestUtils.handlerBatchAppendTest(new ThrowingAvroHandler()); logger.error("Throwing: I should never have gotten here!"); } + + /** + * configure the NettyAvroRpcClient with a non-default + * NioClientSocketChannelFactory number of io worker threads + * + * @throws FlumeException + * @throws EventDeliveryException + */ + @Test + public void testAppendWithMaxIOWorkers() throws FlumeException, EventDeliveryException { + NettyAvroRpcClient client = null; + Server server = RpcTestUtils.startServer(new OKAvroHandler()); + Properties props = new Properties(); + props.setProperty(RpcClientConfigurationConstants.CONFIG_HOSTS, "localhost"); + props.setProperty(RpcClientConfigurationConstants.CONFIG_HOSTS_PREFIX + "localhost", localhost + + ":" + server.getPort()); + props.setProperty(RpcClientConfigurationConstants.MAX_IO_WORKERS, Integer.toString(2)); + try { + client = new NettyAvroRpcClient(); + client.configure(props); + for (int i = 0; i < 5; i++) { + client.append(EventBuilder.withBody("evt:" + i, Charset.forName("UTF8"))); + } + } finally { + RpcTestUtils.stopServer(server); + if (client != null) { + client.close(); + } + } + } + + /** + * Simple request with compression on the server and client with compression + * level 0 + * + * configure the NettyAvroRpcClient with a non-default + * NioClientSocketChannelFactory number of io worker threads + * + * Compression level 0 = no compression + * + * @throws FlumeException + * @throws EventDeliveryException + */ + @Test + public void testAppendWithMaxIOWorkersSimpleCompressionLevel0() throws FlumeException, + EventDeliveryException { + NettyAvroRpcClient client = null; + Server server = RpcTestUtils.startServer(new OKAvroHandler(), 0, true); + Properties props = new Properties(); + props.setProperty(RpcClientConfigurationConstants.CONFIG_HOSTS, "localhost"); + props.setProperty(RpcClientConfigurationConstants.CONFIG_HOSTS_PREFIX + "localhost", localhost + + ":" + server.getPort()); + props.setProperty(RpcClientConfigurationConstants.MAX_IO_WORKERS, Integer.toString(2)); + props.setProperty(RpcClientConfigurationConstants.CONFIG_COMPRESSION_TYPE, "deflate"); + props.setProperty(RpcClientConfigurationConstants.CONFIG_COMPRESSION_LEVEL, "" + 0); + + try { + client = new NettyAvroRpcClient(); + client.configure(props); + for (int i = 0; i < 5; i++) { + client.append(EventBuilder.withBody("evt:" + i, Charset.forName("UTF8"))); + } + } finally { + RpcTestUtils.stopServer(server); + if (client != null) { + client.close(); + } + } + } } From 67454a71a3aba308ff0d1b29ad3f184e5c37fee2 Mon Sep 17 00:00:00 2001 From: Hari Shreedharan Date: Mon, 9 Dec 2013 20:02:01 -0800 Subject: [PATCH 046/341] FLUME-2209. AsyncHBaseSink will never recover if the column family does not exists for the first start. (Ashish Paliwal via Hari Shreedharan) --- .../main/java/org/apache/flume/sink/hbase/AsyncHBaseSink.java | 2 ++ 1 file changed, 2 insertions(+) diff --git a/flume-ng-sinks/flume-ng-hbase-sink/src/main/java/org/apache/flume/sink/hbase/AsyncHBaseSink.java b/flume-ng-sinks/flume-ng-hbase-sink/src/main/java/org/apache/flume/sink/hbase/AsyncHBaseSink.java index 0545554c08..5316cd6252 100644 --- a/flume-ng-sinks/flume-ng-hbase-sink/src/main/java/org/apache/flume/sink/hbase/AsyncHBaseSink.java +++ b/flume-ng-sinks/flume-ng-hbase-sink/src/main/java/org/apache/flume/sink/hbase/AsyncHBaseSink.java @@ -445,6 +445,8 @@ public Object call(Object arg) throws Exception { } if(fail.get()){ sinkCounter.incrementConnectionFailedCount(); + client.shutdown(); + client = null; throw new FlumeException( "Could not start sink. " + "Table or column family does not exist in Hbase."); From 9790ca7587060285efa4ae64591cea17dd3f00cf Mon Sep 17 00:00:00 2001 From: Mike Percy Date: Tue, 10 Dec 2013 14:38:06 -0800 Subject: [PATCH 047/341] FLUME-2217. Add option to preserve all Syslog headers in syslog sources (Jeff Lord via Mike Percy) --- .../source/MultiportSyslogTCPSource.java | 15 ++- .../org/apache/flume/source/SyslogParser.java | 11 +- .../apache/flume/source/SyslogTcpSource.java | 4 +- .../apache/flume/source/SyslogUDPSource.java | 19 +++ .../org/apache/flume/source/SyslogUtils.java | 40 ++++--- .../source/TestMultiportSyslogTCPSource.java | 9 +- .../apache/flume/source/TestSyslogParser.java | 16 ++- .../flume/source/TestSyslogTcpSource.java | 14 +-- .../flume/source/TestSyslogUdpSource.java | 113 +++++++++++------- .../apache/flume/source/TestSyslogUtils.java | 25 ++-- flume-ng-doc/sphinx/FlumeUserGuide.rst | 6 +- 11 files changed, 176 insertions(+), 96 deletions(-) diff --git a/flume-ng-core/src/main/java/org/apache/flume/source/MultiportSyslogTCPSource.java b/flume-ng-core/src/main/java/org/apache/flume/source/MultiportSyslogTCPSource.java index 884fd6241c..427e0e3748 100644 --- a/flume-ng-core/src/main/java/org/apache/flume/source/MultiportSyslogTCPSource.java +++ b/flume-ng-core/src/main/java/org/apache/flume/source/MultiportSyslogTCPSource.java @@ -67,6 +67,7 @@ public class MultiportSyslogTCPSource extends AbstractSource implements private SourceCounter sourceCounter = null; private Charset defaultCharset; private ThreadSafeDecoder defaultDecoder; + private boolean keepFields; public MultiportSyslogTCPSource() { portCharsets = new ConcurrentHashMap(); @@ -138,6 +139,10 @@ public void configure(Context context) { SyslogSourceConfigurationConstants.CONFIG_READBUF_SIZE, SyslogSourceConfigurationConstants.DEFAULT_READBUF_SIZE); + keepFields = context.getBoolean( + SyslogSourceConfigurationConstants.CONFIG_KEEP_FIELDS, + SyslogSourceConfigurationConstants.DEFAULT_KEEP_FIELDS); + if (sourceCounter == null) { sourceCounter = new SourceCounter(getName()); } @@ -159,7 +164,7 @@ public void start() { acceptor.setHandler(new MultiportSyslogHandler(maxEventSize, batchSize, getChannelProcessor(), sourceCounter, portHeader, defaultDecoder, - portCharsets)); + portCharsets, keepFields)); for (int port : ports) { InetSocketAddress addr; @@ -213,11 +218,12 @@ static class MultiportSyslogHandler extends IoHandlerAdapter { private final LineSplitter lineSplitter; private final ThreadSafeDecoder defaultDecoder; private final ConcurrentMap portCharsets; + private final boolean keepFields; public MultiportSyslogHandler(int maxEventSize, int batchSize, ChannelProcessor cp, SourceCounter ctr, String portHeader, ThreadSafeDecoder defaultDecoder, - ConcurrentMap portCharsets) { + ConcurrentMap portCharsets, boolean keepFields) { channelProcessor = cp; sourceCounter = ctr; this.maxEventSize = maxEventSize; @@ -225,6 +231,7 @@ public MultiportSyslogHandler(int maxEventSize, int batchSize, this.portHeader = portHeader; this.defaultDecoder = defaultDecoder; this.portCharsets = portCharsets; + this.keepFields = keepFields; syslogParser = new SyslogParser(); lineSplitter = new LineSplitter(maxEventSize); } @@ -321,7 +328,7 @@ public void messageReceived(IoSession session, Object message) { /** * Decodes a syslog-formatted ParsedLine into a Flume Event. * @param parsedBuf Buffer containing characters to be parsed - * @param port Incoming port + * @param decoder Character set is configurable on a per-port basis. * @return */ Event parseEvent(ParsedBuffer parsedBuf, CharsetDecoder decoder) { @@ -351,7 +358,7 @@ Event parseEvent(ParsedBuffer parsedBuf, CharsetDecoder decoder) { Event event; try { - event = syslogParser.parseMessage(msg, decoder.charset()); + event = syslogParser.parseMessage(msg, decoder.charset(), keepFields); if (parsedBuf.incomplete) { event.getHeaders().put(SyslogUtils.EVENT_STATUS, SyslogUtils.SyslogStatus.INCOMPLETE.getSyslogStatus()); diff --git a/flume-ng-core/src/main/java/org/apache/flume/source/SyslogParser.java b/flume-ng-core/src/main/java/org/apache/flume/source/SyslogParser.java index bf3305c957..557d1213ef 100644 --- a/flume-ng-core/src/main/java/org/apache/flume/source/SyslogParser.java +++ b/flume-ng-core/src/main/java/org/apache/flume/source/SyslogParser.java @@ -33,6 +33,8 @@ import java.util.concurrent.ExecutionException; import java.util.regex.Pattern; import org.apache.flume.Event; +import org.apache.flume.annotations.InterfaceAudience; +import org.apache.flume.annotations.InterfaceStability; import org.apache.flume.event.EventBuilder; import org.joda.time.DateTime; import org.joda.time.format.DateTimeFormat; @@ -40,6 +42,8 @@ import org.slf4j.Logger; import org.slf4j.LoggerFactory; +@InterfaceAudience.Private +@InterfaceStability.Evolving public class SyslogParser { private static final Logger logger = @@ -53,7 +57,6 @@ public class SyslogParser { private static final String timePat = "yyyy-MM-dd'T'HH:mm:ss"; private static final int RFC3164_LEN = 15; private static final int RFC5424_PREFIX_LEN = 19; - private final DateTimeFormatter timeParser; private Cache timestampCache; @@ -76,7 +79,7 @@ public Long load(String key) throws Exception { * @return Parsed Flume Event * @throws IllegalArgumentException if unable to successfully parse message */ - public Event parseMessage(String msg, Charset charset) { + public Event parseMessage(String msg, Charset charset, boolean keepFields) { Map headers = Maps.newHashMap(); int msgLen = msg.length(); @@ -164,9 +167,11 @@ public Event parseMessage(String msg, Charset charset) { // EventBuilder will do a copy of its own, so no defensive copy of the body String data = ""; - if (msgLen > nextSpace + 1) { + if (msgLen > nextSpace + 1 && !keepFields) { curPos = nextSpace + 1; data = msg.substring(curPos); + } else { + data = msg; } Event event = EventBuilder.withBody(data, charset, headers); diff --git a/flume-ng-core/src/main/java/org/apache/flume/source/SyslogTcpSource.java b/flume-ng-core/src/main/java/org/apache/flume/source/SyslogTcpSource.java index 7a12d275c5..e84e4b63d5 100644 --- a/flume-ng-core/src/main/java/org/apache/flume/source/SyslogTcpSource.java +++ b/flume-ng-core/src/main/java/org/apache/flume/source/SyslogTcpSource.java @@ -68,8 +68,8 @@ public void setEventSize(int eventSize){ syslogUtils.setEventSize(eventSize); } - public void setKeepFields(boolean removeFields){ - syslogUtils.setKeepFields(removeFields); + public void setKeepFields(boolean keepFields){ + syslogUtils.setKeepFields(keepFields); } public void setFormater(Map prop) { diff --git a/flume-ng-core/src/main/java/org/apache/flume/source/SyslogUDPSource.java b/flume-ng-core/src/main/java/org/apache/flume/source/SyslogUDPSource.java index 96a9e85781..8fb251bf0e 100644 --- a/flume-ng-core/src/main/java/org/apache/flume/source/SyslogUDPSource.java +++ b/flume-ng-core/src/main/java/org/apache/flume/source/SyslogUDPSource.java @@ -19,10 +19,12 @@ package org.apache.flume.source; import java.net.InetSocketAddress; +import java.net.SocketAddress; import java.util.Map; import java.util.concurrent.Executors; import java.util.concurrent.TimeUnit; +import com.google.common.annotations.VisibleForTesting; import org.apache.flume.ChannelException; import org.apache.flume.Context; import org.apache.flume.CounterGroup; @@ -53,6 +55,7 @@ public class SyslogUDPSource extends AbstractSource private String host = null; private Channel nettyChannel; private Map formaterProp; + private boolean keepFields; private static final Logger logger = LoggerFactory .getLogger(SyslogUDPSource.class); @@ -65,6 +68,10 @@ public void setFormater(Map prop) { syslogUtils.addFormats(prop); } + public void setKeepFields(boolean keepFields) { + syslogUtils.setKeepFields(keepFields); + } + @Override public void messageReceived(ChannelHandlerContext ctx, MessageEvent mEvent) { try { @@ -90,6 +97,7 @@ public void start() { (new OioDatagramChannelFactory(Executors.newCachedThreadPool())); final syslogHandler handler = new syslogHandler(); handler.setFormater(formaterProp); + handler.setKeepFields(keepFields); serverBootstrap.setPipelineFactory(new ChannelPipelineFactory() { @Override public ChannelPipeline getPipeline() { @@ -132,6 +140,17 @@ public void configure(Context context) { host = context.getString(SyslogSourceConfigurationConstants.CONFIG_HOST); formaterProp = context.getSubProperties( SyslogSourceConfigurationConstants.CONFIG_FORMAT_PREFIX); + keepFields = context.getBoolean(SyslogSourceConfigurationConstants.CONFIG_KEEP_FIELDS, + SyslogSourceConfigurationConstants.DEFAULT_KEEP_FIELDS); } + @VisibleForTesting + public int getSourcePort() { + SocketAddress localAddress = nettyChannel.getLocalAddress(); + if (localAddress instanceof InetSocketAddress) { + InetSocketAddress addr = (InetSocketAddress) localAddress; + return addr.getPort(); + } + return 0; + } } diff --git a/flume-ng-core/src/main/java/org/apache/flume/source/SyslogUtils.java b/flume-ng-core/src/main/java/org/apache/flume/source/SyslogUtils.java index f2ea932dbc..a77bfc99cd 100644 --- a/flume-ng-core/src/main/java/org/apache/flume/source/SyslogUtils.java +++ b/flume-ng-core/src/main/java/org/apache/flume/source/SyslogUtils.java @@ -28,7 +28,6 @@ import org.slf4j.LoggerFactory; import java.io.ByteArrayOutputStream; -import java.io.IOException; import java.text.ParseException; import java.text.SimpleDateFormat; import java.util.ArrayList; @@ -49,15 +48,18 @@ public class SyslogUtils { final public static String SYSLOG_TIMESTAMP_FORMAT_RFC3164_1 = "yyyyMMM d HH:mm:ss"; final public static String SYSLOG_MSG_RFC5424_0 = - "(?:\\d\\s)?" +// version - // yyyy-MM-dd'T'HH:mm:ss.SZ or yyyy-MM-dd'T'HH:mm:ss.S+hh:mm or - (null stamp) - "(?:(\\d{4}[-]\\d{2}[-]\\d{2}[T]\\d{2}[:]\\d{2}[:]\\d{2}(?:\\.\\d{1,6})?(?:[+-]\\d{2}[:]\\d{2}|Z)?)|-)" + // stamp + "(?:\\<\\d{1,3}\\>\\d?\\s?)" + // priority + /* yyyy-MM-dd'T'HH:mm:ss.SZ or yyyy-MM-dd'T'HH:mm:ss.S+hh:mm or - (null stamp) */ + "(?:" + + "(\\d{4}[-]\\d{2}[-]\\d{2}[T]\\d{2}[:]\\d{2}[:]\\d{2}" + + "(?:\\.\\d{1,6})?(?:[+-]\\d{2}[:]\\d{2}|Z)?)|-)" + // stamp "\\s" + // separator "(?:([\\w][\\w\\d\\.@-]*)|-)" + // host name or - (null) "\\s" + // separator "(.*)$"; // body final public static String SYSLOG_MSG_RFC3164_0 = + "(?:\\<\\d{1,3}\\>\\d?\\s?)" + // stamp MMM d HH:mm:ss, single digit date has two spaces "([A-Z][a-z][a-z]\\s{1,2}\\d{1,2}\\s\\d{2}[:]\\d{2}[:]\\d{2})" + "\\s" + // separator @@ -225,8 +227,13 @@ Event buildEvent() { headers.put(EVENT_STATUS, SyslogStatus.INCOMPLETE.getSyslogStatus()); } - if ((msgBody != null) && (msgBody.length() > 0) && !keepFields) { - body = msgBody.getBytes(); + if (!keepFields) { + if ((msgBody != null) && (msgBody.length() > 0)) { + body = msgBody.getBytes(); + } else { + // Parse failed. + body = baos.toByteArray(); + } } else { body = baos.toByteArray(); } @@ -311,14 +318,15 @@ public Event extractEvent(ChannelBuffer in){ switch (m) { case START: if (b == '<') { + baos.write(b); m = Mode.PRIO; } else if(b == '\n'){ - //If the character is \n, it was because the last event was exactly - //as long as the maximum size allowed and - //the only remaining character was the delimiter - '\n', or - //multiple delimiters were sent in a row. - //Just ignore it, and move forward, don't change the mode. - //This is a no-op, just ignore it. + //If the character is \n, it was because the last event was exactly + //as long as the maximum size allowed and + //the only remaining character was the delimiter - '\n', or + //multiple delimiters were sent in a row. + //Just ignore it, and move forward, don't change the mode. + //This is a no-op, just ignore it. logger.debug("Delimiter found while in START mode, ignoring.."); } else { @@ -329,6 +337,7 @@ public Event extractEvent(ChannelBuffer in){ } break; case PRIO: + baos.write(b); if (b == '>') { m = Mode.DATA; } else { @@ -336,9 +345,6 @@ public Event extractEvent(ChannelBuffer in){ prio.append(ch); if (!Character.isDigit(ch)) { isBadEvent = true; - //Append the priority to baos: - String badPrio = "<"+ prio; - baos.write(badPrio.getBytes()); //If we hit a bad priority, just write as if everything is data. m = Mode.DATA; } @@ -367,10 +373,6 @@ public Event extractEvent(ChannelBuffer in){ doneReading = true; e = buildEvent(); } - //} catch (IndexOutOfBoundsException eF) { - // e = buildEvent(prio, baos); - } catch (IOException e1) { - //no op } finally { // no-op } diff --git a/flume-ng-core/src/test/java/org/apache/flume/source/TestMultiportSyslogTCPSource.java b/flume-ng-core/src/test/java/org/apache/flume/source/TestMultiportSyslogTCPSource.java index 680e592dc4..9b97c8c730 100644 --- a/flume-ng-core/src/test/java/org/apache/flume/source/TestMultiportSyslogTCPSource.java +++ b/flume-ng-core/src/test/java/org/apache/flume/source/TestMultiportSyslogTCPSource.java @@ -70,6 +70,7 @@ public class TestMultiportSyslogTCPSource { private final String stamp1 = time.toString(); private final String host1 = "localhost.localdomain"; private final String data1 = "proc1 - some msg"; + private final static boolean KEEP_FIELDS = false; /** * Helper function to generate a syslog message. @@ -205,7 +206,8 @@ public void testFragmented() throws CharacterCodingException { new MultiportSyslogTCPSource.MultiportSyslogHandler(maxLen, 100, null, null, SyslogSourceConfigurationConstants.DEFAULT_PORT_HEADER, new ThreadSafeDecoder(Charsets.UTF_8), - new ConcurrentHashMap()); + new ConcurrentHashMap(), + KEEP_FIELDS); Event event = handler.parseEvent(parsedLine, Charsets.UTF_8.newDecoder()); String body = new String(event.getBody(), Charsets.UTF_8); @@ -231,7 +233,8 @@ public void testCharsetParsing() throws FileNotFoundException, IOException { 1000, 10, new ChannelProcessor(new ReplicatingChannelSelector()), new SourceCounter("test"), "port", new ThreadSafeDecoder(Charsets.UTF_8), - new ConcurrentHashMap()); + new ConcurrentHashMap(), + KEEP_FIELDS); ParsedBuffer parsedBuf = new ParsedBuffer(); parsedBuf.incomplete = false; @@ -331,7 +334,7 @@ public void testPortCharsetHandling() throws UnknownHostException, Exception { // defaults to UTF-8 MultiportSyslogHandler handler = new MultiportSyslogHandler( 1000, 10, chanProc, new SourceCounter("test"), "port", - new ThreadSafeDecoder(Charsets.UTF_8), portCharsets); + new ThreadSafeDecoder(Charsets.UTF_8), portCharsets, KEEP_FIELDS); // initialize buffers handler.sessionCreated(session1); diff --git a/flume-ng-core/src/test/java/org/apache/flume/source/TestSyslogParser.java b/flume-ng-core/src/test/java/org/apache/flume/source/TestSyslogParser.java index 258c2f12a3..2809163e16 100644 --- a/flume-ng-core/src/test/java/org/apache/flume/source/TestSyslogParser.java +++ b/flume-ng-core/src/test/java/org/apache/flume/source/TestSyslogParser.java @@ -29,7 +29,6 @@ import org.junit.Test; public class TestSyslogParser { - @Test public void testRfc5424DateParsing() { final String[] examples = { @@ -55,7 +54,7 @@ public void testMessageParsing() { Charset charset = Charsets.UTF_8; List messages = Lists.newArrayList(); - // supported examples from RFC 3161 + // supported examples from RFC 3164 messages.add("<34>Oct 11 22:14:15 mymachine su: 'su root' failed for " + "lonvick on /dev/pts/8"); messages.add("<13>Feb 5 17:32:18 10.0.0.99 Use the BFG!"); @@ -76,8 +75,19 @@ public void testMessageParsing() { messages.add("<13>2003-08-24T05:14:15Z localhost snarf?"); messages.add("<13>2012-08-16T14:34:03-08:00 127.0.0.1 test shnap!"); + // test with default keepFields = false + for (String msg : messages) { + boolean keepFields = false; + Event event = parser.parseMessage(msg, charset, keepFields); + Assert.assertNull("Failure to parse known-good syslog message", + event.getHeaders().get(SyslogUtils.EVENT_STATUS)); + } + + // test that priority, timestamp and hostname are preserved in event body for (String msg : messages) { - Event event = parser.parseMessage(msg, charset); + boolean keepFields = true; + Event event = parser.parseMessage(msg, charset, keepFields); + Assert.assertArrayEquals(event.getBody(), msg.getBytes()); Assert.assertNull("Failure to parse known-good syslog message", event.getHeaders().get(SyslogUtils.EVENT_STATUS)); } diff --git a/flume-ng-core/src/test/java/org/apache/flume/source/TestSyslogTcpSource.java b/flume-ng-core/src/test/java/org/apache/flume/source/TestSyslogTcpSource.java index a6a1d5bba4..22fa200865 100644 --- a/flume-ng-core/src/test/java/org/apache/flume/source/TestSyslogTcpSource.java +++ b/flume-ng-core/src/test/java/org/apache/flume/source/TestSyslogTcpSource.java @@ -49,13 +49,8 @@ public class TestSyslogTcpSource { private final String stamp1 = time.toString(); private final String host1 = "localhost.localdomain"; private final String data1 = "test syslog data"; - private final String bodyWithTandH = stamp1 + " " + host1 + " " + data1; - // Helper function to generate a syslog message. - private byte[] getEvent() { - // timestamp with 'Z' appended, translates to UTC - final String msg1 = "<10>" + stamp1 + " " + host1 + " " + data1 + "\n"; - return msg1.getBytes(); - } + private final String bodyWithTandH = "<10>" + stamp1 + " " + host1 + " " + + data1 + "\n"; private void init(boolean keepFields){ source = new SyslogTcpSource(); @@ -87,7 +82,7 @@ private void runKeepFieldsTest(boolean keepFields) throws IOException { for (int i = 0; i < 10 ; i++) { syslogSocket = new Socket( InetAddress.getLocalHost(), source.getSourcePort()); - syslogSocket.getOutputStream().write(getEvent()); + syslogSocket.getOutputStream().write(bodyWithTandH.getBytes()); syslogSocket.close(); } @@ -116,7 +111,8 @@ private void runKeepFieldsTest(boolean keepFields) throws IOException { String str = new String(e.getBody(), Charsets.UTF_8); logger.info(str); if (keepFields) { - Assert.assertArrayEquals(bodyWithTandH.getBytes(), e.getBody()); + Assert.assertArrayEquals(bodyWithTandH.trim().getBytes(), + e.getBody()); } else if (!keepFields) { Assert.assertArrayEquals(data1.getBytes(), e.getBody()); } diff --git a/flume-ng-core/src/test/java/org/apache/flume/source/TestSyslogUdpSource.java b/flume-ng-core/src/test/java/org/apache/flume/source/TestSyslogUdpSource.java index eae26edcdb..36f6479cf9 100644 --- a/flume-ng-core/src/test/java/org/apache/flume/source/TestSyslogUdpSource.java +++ b/flume-ng-core/src/test/java/org/apache/flume/source/TestSyslogUdpSource.java @@ -20,15 +20,11 @@ import java.util.ArrayList; import java.util.List; - +import java.io.IOException; +import java.net.DatagramPacket; +import java.net.InetAddress; +import java.net.DatagramSocket; import com.google.common.base.Charsets; -import org.apache.log4j.Logger; -import org.apache.log4j.net.SyslogAppender; - -import org.junit.Assert; -import org.junit.Before; -import org.junit.Test; - import org.apache.flume.Channel; import org.apache.flume.ChannelSelector; import org.apache.flume.Context; @@ -38,16 +34,27 @@ import org.apache.flume.channel.MemoryChannel; import org.apache.flume.channel.ReplicatingChannelSelector; import org.apache.flume.conf.Configurables; -import org.apache.flume.source.SyslogUtils; +import org.joda.time.DateTime; +import org.junit.Assert; +import org.junit.Test; +import org.slf4j.LoggerFactory; + public class TestSyslogUdpSource { + private static final org.slf4j.Logger logger = + LoggerFactory.getLogger(TestSyslogUdpSource.class); private SyslogUDPSource source; private Channel channel; - private static final int TEST_SYSLOG_PORT = 14455; - - @Before - public void setUp() { - source = new SyslogUDPSource(); //SyslogTcpSource(); + private static final int TEST_SYSLOG_PORT = 0; + private final DateTime time = new DateTime(); + private final String stamp1 = time.toString(); + private final String host1 = "localhost.localdomain"; + private final String data1 = "test UDP syslog data"; + private final String bodyWithTandH = "<10>" + stamp1 + " " + host1 + " " + + data1; + + private void init(boolean keepFields) { + source = new SyslogUDPSource(); channel = new MemoryChannel(); Configurables.configure(channel, new Context()); @@ -61,49 +68,69 @@ public void setUp() { source.setChannelProcessor(new ChannelProcessor(rcs)); Context context = new Context(); context.put("port", String.valueOf(TEST_SYSLOG_PORT)); + context.put("keepFields", String.valueOf(keepFields)); + source.configure(context); + } - @Test - public void testAppend() throws InterruptedException { - Logger logger = Logger.getLogger(getClass()); - // use the Apache syslog appender to write to syslog source - SyslogAppender appender = new SyslogAppender(null, - "localhost:"+TEST_SYSLOG_PORT, SyslogAppender.LOG_FTP); - logger.addAppender(appender); - Event e = null; - Event e2 = null; + /** Tests the keepFields configuration parameter (enabled or disabled) + using SyslogUDPSource.*/ + private void runKeepFieldsTest(boolean keepFields) throws IOException { + init(keepFields); source.start(); + // Write some message to the syslog port + DatagramSocket syslogSocket; + DatagramPacket datagramPacket; + datagramPacket = new DatagramPacket(bodyWithTandH.getBytes(), + bodyWithTandH.getBytes().length, + InetAddress.getLocalHost(), source.getSourcePort()); + for (int i = 0; i < 10 ; i++) { + syslogSocket = new DatagramSocket(); + syslogSocket.send(datagramPacket); + syslogSocket.close(); + } - // write to syslog - logger.info("test flume syslog"); - logger.info(""); - + List channelEvents = new ArrayList(); Transaction txn = channel.getTransaction(); + txn.begin(); + for (int i = 0; i < 10; i++) { + Event e = channel.take(); + Assert.assertNotNull(e); + channelEvents.add(e); + } + try { - txn.begin(); - e = channel.take(); - e2 = channel.take(); txn.commit(); + } catch (Throwable t) { + txn.rollback(); } finally { txn.close(); } source.stop(); - logger.removeAppender(appender); - - String str = new String(e.getBody(), Charsets.UTF_8); - logger.info(str); - Assert.assertNotNull(e); - Assert.assertEquals(String.valueOf(SyslogAppender.LOG_FTP / 8), - e.getHeaders().get(SyslogUtils.SYSLOG_FACILITY)); - Assert.assertArrayEquals(e.getBody(), "test flume syslog".getBytes()); - - Assert.assertNotNull(e2); - Assert.assertEquals(String.valueOf(SyslogAppender.LOG_FTP / 8), - e2.getHeaders().get(SyslogUtils.SYSLOG_FACILITY)); - Assert.assertArrayEquals(e2.getBody(), "".getBytes()); + for (Event e : channelEvents) { + Assert.assertNotNull(e); + String str = new String(e.getBody(), Charsets.UTF_8); + logger.info(str); + if (keepFields) { + Assert.assertArrayEquals(bodyWithTandH.getBytes(), + e.getBody()); + } else if (!keepFields) { + Assert.assertArrayEquals(data1.getBytes(), e.getBody()); + } + } } + @Test + public void testKeepFields() throws IOException { + runKeepFieldsTest(true); + } + + @Test + public void testRemoveFields() throws IOException { + runKeepFieldsTest(false); + } } + diff --git a/flume-ng-core/src/test/java/org/apache/flume/source/TestSyslogUtils.java b/flume-ng-core/src/test/java/org/apache/flume/source/TestSyslogUtils.java index 898096b1e8..82b7dd0a35 100644 --- a/flume-ng-core/src/test/java/org/apache/flume/source/TestSyslogUtils.java +++ b/flume-ng-core/src/test/java/org/apache/flume/source/TestSyslogUtils.java @@ -251,7 +251,8 @@ public void testExtractGoodEvent() { Assert.assertEquals("1", headers.get(SyslogUtils.SYSLOG_FACILITY)); Assert.assertEquals("2", headers.get(SyslogUtils.SYSLOG_SEVERITY)); Assert.assertEquals(null, headers.get(SyslogUtils.EVENT_STATUS)); - Assert.assertEquals(goodData1.trim(), new String(e.getBody()).trim()); + Assert.assertEquals(priority + goodData1.trim(), + new String(e.getBody()).trim()); } @@ -277,7 +278,8 @@ public void testBadEventGoodEvent(){ Assert.assertEquals("0", headers.get(SyslogUtils.SYSLOG_SEVERITY)); Assert.assertEquals(SyslogUtils.SyslogStatus.INVALID.getSyslogStatus(), headers.get(SyslogUtils.EVENT_STATUS)); - Assert.assertEquals(badData1.trim(), new String(e.getBody()).trim()); + Assert.assertEquals(badData1.trim(), new String(e.getBody()) + .trim()); Event e2 = util.extractEvent(buff); if(e2 == null){ @@ -288,7 +290,8 @@ public void testBadEventGoodEvent(){ Assert.assertEquals("2", headers2.get(SyslogUtils.SYSLOG_SEVERITY)); Assert.assertEquals(null, headers2.get(SyslogUtils.EVENT_STATUS)); - Assert.assertEquals(goodData1.trim(), new String(e2.getBody()).trim()); + Assert.assertEquals(priority + goodData1.trim(), + new String(e2.getBody()).trim()); } @Test @@ -310,7 +313,8 @@ public void testGoodEventBadEvent(){ Assert.assertEquals("2", headers2.get(SyslogUtils.SYSLOG_SEVERITY)); Assert.assertEquals(null, headers2.get(SyslogUtils.EVENT_STATUS)); - Assert.assertEquals(goodData1.trim(), new String(e2.getBody()).trim()); + Assert.assertEquals(priority + goodData1.trim(), + new String(e2.getBody()).trim()); Event e = util.extractEvent(buff); @@ -379,7 +383,8 @@ public void testGoodEventGoodEvent() { Assert.assertEquals("2", headers.get(SyslogUtils.SYSLOG_SEVERITY)); Assert.assertEquals(null, headers.get(SyslogUtils.EVENT_STATUS)); - Assert.assertEquals(goodData1.trim(), new String(e.getBody()).trim()); + Assert.assertEquals(priority + goodData1.trim(), + new String(e.getBody()).trim()); Event e2 = util.extractEvent(buff); @@ -391,14 +396,16 @@ public void testGoodEventGoodEvent() { Assert.assertEquals("4", headers2.get(SyslogUtils.SYSLOG_SEVERITY)); Assert.assertEquals(null, headers.get(SyslogUtils.EVENT_STATUS)); - Assert.assertEquals(goodData2.trim(), new String(e2.getBody()).trim()); + Assert.assertEquals(priority2 + goodData2.trim(), + new String(e2.getBody()).trim()); } @Test public void testExtractBadEventLarge() { String badData1 = "<10> bad bad data bad bad\n"; - SyslogUtils util = new SyslogUtils(5, true, false); + // The minimum size (which is 10) overrides the 5 specified here. + SyslogUtils util = new SyslogUtils(5, false, false); ChannelBuffer buff = ChannelBuffers.buffer(100); buff.writeBytes(badData1.getBytes()); Event e = util.extractEvent(buff); @@ -410,7 +417,7 @@ public void testExtractBadEventLarge() { Assert.assertEquals("2", headers.get(SyslogUtils.SYSLOG_SEVERITY)); Assert.assertEquals(SyslogUtils.SyslogStatus.INCOMPLETE.getSyslogStatus(), headers.get(SyslogUtils.EVENT_STATUS)); - Assert.assertEquals("bad bad d".trim(), new String(e.getBody()).trim()); + Assert.assertEquals("<10> bad b".trim(), new String(e.getBody()).trim()); Event e2 = util.extractEvent(buff); @@ -422,7 +429,7 @@ public void testExtractBadEventLarge() { Assert.assertEquals("0", headers2.get(SyslogUtils.SYSLOG_SEVERITY)); Assert.assertEquals(SyslogUtils.SyslogStatus.INVALID.getSyslogStatus(), headers2.get(SyslogUtils.EVENT_STATUS)); - Assert.assertEquals("ata bad ba".trim(), new String(e2.getBody()).trim()); + Assert.assertEquals("ad data ba".trim(), new String(e2.getBody()).trim()); } diff --git a/flume-ng-doc/sphinx/FlumeUserGuide.rst b/flume-ng-doc/sphinx/FlumeUserGuide.rst index 0737c441de..ae66f89e55 100644 --- a/flume-ng-doc/sphinx/FlumeUserGuide.rst +++ b/flume-ng-doc/sphinx/FlumeUserGuide.rst @@ -1171,7 +1171,7 @@ Property Name Default Description **host** -- Host name or IP address to bind to **port** -- Port # to bind to eventSize 2500 Maximum size of a single event line, in bytes -keepFields false Setting this to true will preserve the +keepFields false Setting this to true will preserve the Priority, Timestamp and Hostname in the body of the event. selector.type replicating or multiplexing selector.* replicating Depends on the selector.type value @@ -1209,6 +1209,8 @@ Property Name Default Description **host** -- Host name or IP address to bind to. **ports** -- Space-separated list (one or more) of ports to bind to. eventSize 2500 Maximum size of a single event line, in bytes. +keepFields false Setting this to true will preserve the + Priority, Timestamp and Hostname in the body of the event. portHeader -- If specified, the port number will be stored in the header of each event using the header name specified here. This allows for interceptors and channel selectors to customize routing logic based on the incoming port. charset.default UTF-8 Default character set used while parsing syslog events into strings. charset.port. -- Character set is configurable on a per-port basis. @@ -1243,6 +1245,8 @@ Property Name Default Description **type** -- The component type name, needs to be ``syslogudp`` **host** -- Host name or IP address to bind to **port** -- Port # to bind to +keepFields false Setting this to true will preserve the Priority, + Timestamp and Hostname in the body of the event. selector.type replicating or multiplexing selector.* replicating Depends on the selector.type value interceptors -- Space-separated list of interceptors From d76118d729d2fe0888b934b0dc743f5f068f63dd Mon Sep 17 00:00:00 2001 From: Hari Shreedharan Date: Thu, 12 Dec 2013 15:11:00 -0800 Subject: [PATCH 048/341] FLUME-2266. Update Morphline Sink to kite-0.10.0. (Wolfgang Hoschek via Hari Shreedharan) --- .../flume-ng-morphline-solr-sink/pom.xml | 14 +++++++------- .../solr/morphline/MorphlineHandlerImpl.java | 18 +++++++++--------- .../solr/morphline/MorphlineInterceptor.java | 6 +++--- .../sink/solr/morphline/MorphlineSink.java | 2 +- .../sink/solr/morphline/MorphlineSolrSink.java | 4 ++-- .../sink/solr/morphline/TestEnvironment.java | 2 +- .../morphline/TestMorphlineInterceptor.java | 2 +- .../solr/morphline/TestMorphlineSolrSink.java | 18 +++++++++--------- .../src/test/resources/log4j.properties | 2 +- .../grokIfNotMatchDropRecord.conf | 2 +- .../test-morphlines/ifDetectMimeType.conf | 2 +- .../resources/test-morphlines/noOperation.conf | 2 +- .../resources/test-morphlines/readClob.conf | 2 +- .../test-morphlines/solrCellDocumentTypes.conf | 2 +- 14 files changed, 39 insertions(+), 39 deletions(-) diff --git a/flume-ng-sinks/flume-ng-morphline-solr-sink/pom.xml b/flume-ng-sinks/flume-ng-morphline-solr-sink/pom.xml index c78788df57..7d9065199b 100644 --- a/flume-ng-sinks/flume-ng-morphline-solr-sink/pom.xml +++ b/flume-ng-sinks/flume-ng-morphline-solr-sink/pom.xml @@ -33,7 +33,7 @@ limitations under the License. UTF-8 4.3.0 4.3.0 - 0.8.0 + 0.10.0 1.6.1 2.12.4 @@ -74,9 +74,9 @@ limitations under the License. - com.cloudera.cdk - cdk-morphlines-all - ${cdk.version} + org.kitesdk + kite-morphlines-all + ${kite.version} org.apache.hadoop @@ -108,9 +108,9 @@ limitations under the License. - com.cloudera.cdk - cdk-morphlines-solr-core - ${cdk.version} + org.kitesdk + kite-morphlines-solr-core + ${kite.version} test-jar test diff --git a/flume-ng-sinks/flume-ng-morphline-solr-sink/src/main/java/org/apache/flume/sink/solr/morphline/MorphlineHandlerImpl.java b/flume-ng-sinks/flume-ng-morphline-solr-sink/src/main/java/org/apache/flume/sink/solr/morphline/MorphlineHandlerImpl.java index cb88dc28e8..d3154af261 100644 --- a/flume-ng-sinks/flume-ng-morphline-solr-sink/src/main/java/org/apache/flume/sink/solr/morphline/MorphlineHandlerImpl.java +++ b/flume-ng-sinks/flume-ng-morphline-solr-sink/src/main/java/org/apache/flume/sink/solr/morphline/MorphlineHandlerImpl.java @@ -24,15 +24,15 @@ import org.slf4j.Logger; import org.slf4j.LoggerFactory; -import com.cloudera.cdk.morphline.api.Command; -import com.cloudera.cdk.morphline.api.MorphlineCompilationException; -import com.cloudera.cdk.morphline.api.MorphlineContext; -import com.cloudera.cdk.morphline.api.Record; -import com.cloudera.cdk.morphline.base.Compiler; -import com.cloudera.cdk.morphline.base.FaultTolerance; -import com.cloudera.cdk.morphline.base.Fields; -import com.cloudera.cdk.morphline.base.Metrics; -import com.cloudera.cdk.morphline.base.Notifications; +import org.kitesdk.morphline.api.Command; +import org.kitesdk.morphline.api.MorphlineCompilationException; +import org.kitesdk.morphline.api.MorphlineContext; +import org.kitesdk.morphline.api.Record; +import org.kitesdk.morphline.base.Compiler; +import org.kitesdk.morphline.base.FaultTolerance; +import org.kitesdk.morphline.base.Fields; +import org.kitesdk.morphline.base.Metrics; +import org.kitesdk.morphline.base.Notifications; import com.codahale.metrics.Meter; import com.codahale.metrics.MetricRegistry; import com.codahale.metrics.SharedMetricRegistries; diff --git a/flume-ng-sinks/flume-ng-morphline-solr-sink/src/main/java/org/apache/flume/sink/solr/morphline/MorphlineInterceptor.java b/flume-ng-sinks/flume-ng-morphline-solr-sink/src/main/java/org/apache/flume/sink/solr/morphline/MorphlineInterceptor.java index ac0ccb6ae8..8e5e4b3294 100644 --- a/flume-ng-sinks/flume-ng-morphline-solr-sink/src/main/java/org/apache/flume/sink/solr/morphline/MorphlineInterceptor.java +++ b/flume-ng-sinks/flume-ng-morphline-solr-sink/src/main/java/org/apache/flume/sink/solr/morphline/MorphlineInterceptor.java @@ -32,9 +32,9 @@ import org.apache.flume.event.EventBuilder; import org.apache.flume.interceptor.Interceptor; -import com.cloudera.cdk.morphline.api.Command; -import com.cloudera.cdk.morphline.api.Record; -import com.cloudera.cdk.morphline.base.Fields; +import org.kitesdk.morphline.api.Command; +import org.kitesdk.morphline.api.Record; +import org.kitesdk.morphline.base.Fields; import com.google.common.base.Preconditions; import com.google.common.io.ByteStreams; diff --git a/flume-ng-sinks/flume-ng-morphline-solr-sink/src/main/java/org/apache/flume/sink/solr/morphline/MorphlineSink.java b/flume-ng-sinks/flume-ng-morphline-solr-sink/src/main/java/org/apache/flume/sink/solr/morphline/MorphlineSink.java index 1e52a66a3a..7c784c460c 100644 --- a/flume-ng-sinks/flume-ng-morphline-solr-sink/src/main/java/org/apache/flume/sink/solr/morphline/MorphlineSink.java +++ b/flume-ng-sinks/flume-ng-morphline-solr-sink/src/main/java/org/apache/flume/sink/solr/morphline/MorphlineSink.java @@ -29,7 +29,7 @@ import org.slf4j.Logger; import org.slf4j.LoggerFactory; -import com.cloudera.cdk.morphline.api.Command; +import org.kitesdk.morphline.api.Command; /** * Flume sink that extracts search documents from Flume events and processes them using a morphline diff --git a/flume-ng-sinks/flume-ng-morphline-solr-sink/src/main/java/org/apache/flume/sink/solr/morphline/MorphlineSolrSink.java b/flume-ng-sinks/flume-ng-morphline-solr-sink/src/main/java/org/apache/flume/sink/solr/morphline/MorphlineSolrSink.java index bc07a2e233..e403b104db 100644 --- a/flume-ng-sinks/flume-ng-morphline-solr-sink/src/main/java/org/apache/flume/sink/solr/morphline/MorphlineSolrSink.java +++ b/flume-ng-sinks/flume-ng-morphline-solr-sink/src/main/java/org/apache/flume/sink/solr/morphline/MorphlineSolrSink.java @@ -18,8 +18,8 @@ import org.apache.flume.Context; -import com.cloudera.cdk.morphline.api.Command; -import com.cloudera.cdk.morphline.base.FaultTolerance; +import org.kitesdk.morphline.api.Command; +import org.kitesdk.morphline.base.FaultTolerance; /** diff --git a/flume-ng-sinks/flume-ng-morphline-solr-sink/src/test/java/org/apache/flume/sink/solr/morphline/TestEnvironment.java b/flume-ng-sinks/flume-ng-morphline-solr-sink/src/test/java/org/apache/flume/sink/solr/morphline/TestEnvironment.java index 5576d1d7c8..933a6b1df9 100644 --- a/flume-ng-sinks/flume-ng-morphline-solr-sink/src/test/java/org/apache/flume/sink/solr/morphline/TestEnvironment.java +++ b/flume-ng-sinks/flume-ng-morphline-solr-sink/src/test/java/org/apache/flume/sink/solr/morphline/TestEnvironment.java @@ -20,7 +20,7 @@ import org.junit.Test; -import com.cloudera.cdk.morphline.solr.EnvironmentTest; +import org.kitesdk.morphline.solr.EnvironmentTest; /** Print and verify some info about the environment in which the unit tests are running */ public class TestEnvironment extends EnvironmentTest { diff --git a/flume-ng-sinks/flume-ng-morphline-solr-sink/src/test/java/org/apache/flume/sink/solr/morphline/TestMorphlineInterceptor.java b/flume-ng-sinks/flume-ng-morphline-solr-sink/src/test/java/org/apache/flume/sink/solr/morphline/TestMorphlineInterceptor.java index f330ecbc98..22cfe96a2d 100644 --- a/flume-ng-sinks/flume-ng-morphline-solr-sink/src/test/java/org/apache/flume/sink/solr/morphline/TestMorphlineInterceptor.java +++ b/flume-ng-sinks/flume-ng-morphline-solr-sink/src/test/java/org/apache/flume/sink/solr/morphline/TestMorphlineInterceptor.java @@ -28,7 +28,7 @@ import org.junit.Assert; import org.junit.Test; -import com.cloudera.cdk.morphline.base.Fields; +import org.kitesdk.morphline.base.Fields; import com.google.common.base.Charsets; import com.google.common.collect.ImmutableMap; import com.google.common.io.Files; diff --git a/flume-ng-sinks/flume-ng-morphline-solr-sink/src/test/java/org/apache/flume/sink/solr/morphline/TestMorphlineSolrSink.java b/flume-ng-sinks/flume-ng-morphline-solr-sink/src/test/java/org/apache/flume/sink/solr/morphline/TestMorphlineSolrSink.java index 5fbb22b5e6..232c092883 100644 --- a/flume-ng-sinks/flume-ng-morphline-solr-sink/src/test/java/org/apache/flume/sink/solr/morphline/TestMorphlineSolrSink.java +++ b/flume-ng-sinks/flume-ng-morphline-solr-sink/src/test/java/org/apache/flume/sink/solr/morphline/TestMorphlineSolrSink.java @@ -50,15 +50,15 @@ import org.slf4j.Logger; import org.slf4j.LoggerFactory; -import com.cloudera.cdk.morphline.api.MorphlineContext; -import com.cloudera.cdk.morphline.api.Record; -import com.cloudera.cdk.morphline.base.FaultTolerance; -import com.cloudera.cdk.morphline.base.Fields; -import com.cloudera.cdk.morphline.solr.DocumentLoader; -import com.cloudera.cdk.morphline.solr.SolrLocator; -import com.cloudera.cdk.morphline.solr.SolrMorphlineContext; -import com.cloudera.cdk.morphline.solr.SolrServerDocumentLoader; -import com.cloudera.cdk.morphline.solr.TestEmbeddedSolrServer; +import org.kitesdk.morphline.api.MorphlineContext; +import org.kitesdk.morphline.api.Record; +import org.kitesdk.morphline.base.FaultTolerance; +import org.kitesdk.morphline.base.Fields; +import org.kitesdk.morphline.solr.DocumentLoader; +import org.kitesdk.morphline.solr.SolrLocator; +import org.kitesdk.morphline.solr.SolrMorphlineContext; +import org.kitesdk.morphline.solr.SolrServerDocumentLoader; +import org.kitesdk.morphline.solr.TestEmbeddedSolrServer; import com.codahale.metrics.MetricRegistry; import com.google.common.base.Charsets; import com.google.common.collect.ImmutableListMultimap; diff --git a/flume-ng-sinks/flume-ng-morphline-solr-sink/src/test/resources/log4j.properties b/flume-ng-sinks/flume-ng-morphline-solr-sink/src/test/resources/log4j.properties index 1763bfea7b..4bfd3fc16b 100644 --- a/flume-ng-sinks/flume-ng-morphline-solr-sink/src/test/resources/log4j.properties +++ b/flume-ng-sinks/flume-ng-morphline-solr-sink/src/test/resources/log4j.properties @@ -20,7 +20,7 @@ log4j.logger.org.apache.flume.sink=INFO #log4j.logger.org.apache.flume.sink.solr=DEBUG log4j.logger.org.apache.solr=INFO #log4j.logger.org.apache.solr.hadoop=DEBUG -log4j.logger.com.cloudera.cdk.morphline=DEBUG +log4j.logger.org.kitesdk.morphline=DEBUG log4j.logger.org.apache.solr.morphline=DEBUG log4j.logger.org.apache.solr.update.processor.LogUpdateProcessor=WARN log4j.logger.org.apache.solr.core.SolrCore=WARN diff --git a/flume-ng-sinks/flume-ng-morphline-solr-sink/src/test/resources/test-morphlines/grokIfNotMatchDropRecord.conf b/flume-ng-sinks/flume-ng-morphline-solr-sink/src/test/resources/test-morphlines/grokIfNotMatchDropRecord.conf index 6578219355..bdd56eee3d 100644 --- a/flume-ng-sinks/flume-ng-morphline-solr-sink/src/test/resources/test-morphlines/grokIfNotMatchDropRecord.conf +++ b/flume-ng-sinks/flume-ng-morphline-solr-sink/src/test/resources/test-morphlines/grokIfNotMatchDropRecord.conf @@ -18,7 +18,7 @@ morphlines : [ { id : morphline1 - importCommands : ["com.cloudera.**"] + importCommands : ["org.kitesdk.**"] commands : [ { diff --git a/flume-ng-sinks/flume-ng-morphline-solr-sink/src/test/resources/test-morphlines/ifDetectMimeType.conf b/flume-ng-sinks/flume-ng-morphline-solr-sink/src/test/resources/test-morphlines/ifDetectMimeType.conf index 92fa33f1ef..cfe0893d2a 100644 --- a/flume-ng-sinks/flume-ng-morphline-solr-sink/src/test/resources/test-morphlines/ifDetectMimeType.conf +++ b/flume-ng-sinks/flume-ng-morphline-solr-sink/src/test/resources/test-morphlines/ifDetectMimeType.conf @@ -20,7 +20,7 @@ MY.MIME_TYPE : myDefault morphlines : [ { id : morphline1 - importCommands : ["com.cloudera.**", "org.apache.solr.**"] + importCommands : ["org.kitesdk.**", "org.apache.solr.**"] commands : [ { diff --git a/flume-ng-sinks/flume-ng-morphline-solr-sink/src/test/resources/test-morphlines/noOperation.conf b/flume-ng-sinks/flume-ng-morphline-solr-sink/src/test/resources/test-morphlines/noOperation.conf index 4e4389b149..f5b493cce3 100644 --- a/flume-ng-sinks/flume-ng-morphline-solr-sink/src/test/resources/test-morphlines/noOperation.conf +++ b/flume-ng-sinks/flume-ng-morphline-solr-sink/src/test/resources/test-morphlines/noOperation.conf @@ -18,7 +18,7 @@ morphlines : [ { id : morphline1 - importCommands : ["com.cloudera.**"] + importCommands : ["org.kitesdk.**"] commands : [ { logDebug { format : "output record: {}", args : ["@{}"] } } diff --git a/flume-ng-sinks/flume-ng-morphline-solr-sink/src/test/resources/test-morphlines/readClob.conf b/flume-ng-sinks/flume-ng-morphline-solr-sink/src/test/resources/test-morphlines/readClob.conf index 234377f20c..ac9df9b096 100644 --- a/flume-ng-sinks/flume-ng-morphline-solr-sink/src/test/resources/test-morphlines/readClob.conf +++ b/flume-ng-sinks/flume-ng-morphline-solr-sink/src/test/resources/test-morphlines/readClob.conf @@ -18,7 +18,7 @@ morphlines : [ { id : morphline1 - importCommands : ["com.cloudera.**", "org.apache.solr.**"] + importCommands : ["org.kitesdk.**", "org.apache.solr.**"] commands : [ { diff --git a/flume-ng-sinks/flume-ng-morphline-solr-sink/src/test/resources/test-morphlines/solrCellDocumentTypes.conf b/flume-ng-sinks/flume-ng-morphline-solr-sink/src/test/resources/test-morphlines/solrCellDocumentTypes.conf index 2574144792..88e6345065 100644 --- a/flume-ng-sinks/flume-ng-morphline-solr-sink/src/test/resources/test-morphlines/solrCellDocumentTypes.conf +++ b/flume-ng-sinks/flume-ng-morphline-solr-sink/src/test/resources/test-morphlines/solrCellDocumentTypes.conf @@ -27,7 +27,7 @@ morphlines : [ { id : morphline1 - importCommands : ["com.cloudera.**", "org.apache.solr.**"] + importCommands : ["org.kitesdk.**", "org.apache.solr.**"] commands : [ { separateAttachments {} } From 6373032a620bdc687b6d03b12726713d08c71a10 Mon Sep 17 00:00:00 2001 From: Hari Shreedharan Date: Fri, 13 Dec 2013 12:35:43 -0800 Subject: [PATCH 049/341] FLUME-2155. Index the Flume Event Queue during replay to improve replay time. (Brock Noland via Hari Shreedharan) --- flume-ng-channels/flume-file-channel/pom.xml | 5 + .../channel/file/CheckpointRebuilder.java | 4 +- .../file/EventQueueBackingStoreFile.java | 6 +- .../flume/channel/file/FileChannel.java | 1 - .../flume/channel/file/FlumeEventQueue.java | 106 +++++++++++++++++- .../org/apache/flume/channel/file/Log.java | 9 +- .../apache/flume/channel/file/LogFile.java | 1 - .../flume/channel/file/ReplayHandler.java | 18 ++- .../flume/channel/file/Serialization.java | 2 +- .../flume/channel/file/TestCheckpoint.java | 8 +- .../channel/file/TestCheckpointRebuilder.java | 3 +- .../TestEventQueueBackingStoreFactory.java | 4 +- .../channel/file/TestFlumeEventQueue.java | 103 +++++++++++++---- pom.xml | 6 + 14 files changed, 221 insertions(+), 55 deletions(-) diff --git a/flume-ng-channels/flume-file-channel/pom.xml b/flume-ng-channels/flume-file-channel/pom.xml index e055d8a977..eacd329969 100644 --- a/flume-ng-channels/flume-file-channel/pom.xml +++ b/flume-ng-channels/flume-file-channel/pom.xml @@ -102,6 +102,11 @@ compile + + org.mapdb + mapdb + + diff --git a/flume-ng-channels/flume-file-channel/src/main/java/org/apache/flume/channel/file/CheckpointRebuilder.java b/flume-ng-channels/flume-file-channel/src/main/java/org/apache/flume/channel/file/CheckpointRebuilder.java index 7883d0e924..43881816c6 100644 --- a/flume-ng-channels/flume-file-channel/src/main/java/org/apache/flume/channel/file/CheckpointRebuilder.java +++ b/flume-ng-channels/flume-file-channel/src/main/java/org/apache/flume/channel/file/CheckpointRebuilder.java @@ -27,7 +27,6 @@ import java.io.EOFException; import java.io.File; import java.io.IOException; -import java.util.Arrays; import java.util.List; import java.util.Set; import org.apache.commons.cli.CommandLine; @@ -251,7 +250,8 @@ public static void main(String[] args) throws Exception { capacity, "channel"); FlumeEventQueue queue = new FlumeEventQueue(backingStore, new File(checkpointDir, "inflighttakes"), - new File(checkpointDir, "inflightputs")); + new File(checkpointDir, "inflightputs"), + new File(checkpointDir, Log.QUEUE_SET)); CheckpointRebuilder rebuilder = new CheckpointRebuilder(logFiles, queue); if(rebuilder.rebuild()) { rebuilder.writeCheckpoint(); diff --git a/flume-ng-channels/flume-file-channel/src/main/java/org/apache/flume/channel/file/EventQueueBackingStoreFile.java b/flume-ng-channels/flume-file-channel/src/main/java/org/apache/flume/channel/file/EventQueueBackingStoreFile.java index 2366cbc19d..8a9fdae50c 100644 --- a/flume-ng-channels/flume-file-channel/src/main/java/org/apache/flume/channel/file/EventQueueBackingStoreFile.java +++ b/flume-ng-channels/flume-file-channel/src/main/java/org/apache/flume/channel/file/EventQueueBackingStoreFile.java @@ -166,7 +166,7 @@ protected void backupCheckpoint(File backupDirectory) throws IOException { "from the checkpoint directory. Cannot complete backup of the " + "checkpoint."); for (File origFile : checkpointFiles) { - if(origFile.getName().equals(Log.FILE_LOCK)) { + if(Log.EXCLUDES.contains(origFile.getName())) { continue; } Serialization.copyFile(origFile, new File(backupDirectory, @@ -399,6 +399,7 @@ public static void main(String[] args) throws Exception { File file = new File(args[0]); File inflightTakesFile = new File(args[1]); File inflightPutsFile = new File(args[2]); + File queueSetDir = new File(args[3]); if (!file.exists()) { throw new IOException("File " + file + " does not exist"); } @@ -421,7 +422,8 @@ public static void main(String[] args) throws Exception { + fileID + ", offset = " + offset); } FlumeEventQueue queue = - new FlumeEventQueue(backingStore, inflightTakesFile, inflightPutsFile); + new FlumeEventQueue(backingStore, inflightTakesFile, inflightPutsFile, + queueSetDir); SetMultimap putMap = queue.deserializeInflightPuts(); System.out.println("Inflight Puts:"); diff --git a/flume-ng-channels/flume-file-channel/src/main/java/org/apache/flume/channel/file/FileChannel.java b/flume-ng-channels/flume-file-channel/src/main/java/org/apache/flume/channel/file/FileChannel.java index 36f150bff9..2cd7f03b36 100644 --- a/flume-ng-channels/flume-file-channel/src/main/java/org/apache/flume/channel/file/FileChannel.java +++ b/flume-ng-channels/flume-file-channel/src/main/java/org/apache/flume/channel/file/FileChannel.java @@ -100,7 +100,6 @@ public class FileChannel extends BasicChannelSemantics { private String encryptionActiveKey; private String encryptionCipherProvider; private boolean useDualCheckpoints; - private boolean isTest = false; @Override public synchronized void setName(String name) { diff --git a/flume-ng-channels/flume-file-channel/src/main/java/org/apache/flume/channel/file/FlumeEventQueue.java b/flume-ng-channels/flume-file-channel/src/main/java/org/apache/flume/channel/file/FlumeEventQueue.java index ac03fb4b5b..7888b419a5 100644 --- a/flume-ng-channels/flume-file-channel/src/main/java/org/apache/flume/channel/file/FlumeEventQueue.java +++ b/flume-ng-channels/flume-file-channel/src/main/java/org/apache/flume/channel/file/FlumeEventQueue.java @@ -30,11 +30,15 @@ import java.util.Set; import java.util.SortedSet; import java.util.TreeSet; -import java.util.concurrent.Future; + +import org.apache.commons.io.FileUtils; import org.apache.commons.lang.ArrayUtils; +import org.mapdb.DB; +import org.mapdb.DBMaker; import org.slf4j.Logger; import org.slf4j.LoggerFactory; +import com.google.common.annotations.VisibleForTesting; import com.google.common.base.Preconditions; import com.google.common.collect.HashMultimap; import com.google.common.collect.SetMultimap; @@ -56,16 +60,26 @@ final class FlumeEventQueue { private final String channelNameDescriptor; private final InflightEventWrapper inflightTakes; private final InflightEventWrapper inflightPuts; + private long searchTime = 0; + private long searchCount = 0; + private long copyTime = 0; + private long copyCount = 0; + private DB db; + private Set queueSet; /** * @param capacity max event capacity of queue * @throws IOException */ FlumeEventQueue(EventQueueBackingStore backingStore, File inflightTakesFile, - File inflightPutsFile) throws Exception { + File inflightPutsFile, File queueSetDBDir) throws Exception { Preconditions.checkArgument(backingStore.getCapacity() > 0, "Capacity must be greater than zero"); + Preconditions.checkNotNull(backingStore, "backingStore"); this.channelNameDescriptor = "[channel=" + backingStore.getName() + "]"; + Preconditions.checkNotNull(inflightTakesFile, "inflightTakesFile"); + Preconditions.checkNotNull(inflightPutsFile, "inflightPutsFile"); + Preconditions.checkNotNull(queueSetDBDir, "queueSetDBDir"); this.backingStore = backingStore; try { inflightPuts = new InflightEventWrapper(inflightPutsFile); @@ -74,6 +88,32 @@ final class FlumeEventQueue { LOG.error("Could not read checkpoint.", e); throw e; } + if(queueSetDBDir.isDirectory()) { + FileUtils.deleteDirectory(queueSetDBDir); + } else if(queueSetDBDir.isFile() && !queueSetDBDir.delete()) { + throw new IOException("QueueSetDir " + queueSetDBDir + " is a file and" + + " could not be deleted"); + } + if(!queueSetDBDir.mkdirs()) { + throw new IllegalStateException("Could not create QueueSet Dir " + + queueSetDBDir); + } + File dbFile = new File(queueSetDBDir, "db"); + db = DBMaker.newFileDB(dbFile) + .closeOnJvmShutdown() + .transactionDisable() + .syncOnCommitDisable() + .deleteFilesAfterClose() + .cacheDisable() + .randomAccessFileEnableIfNeeded() + .make(); + queueSet = db.createTreeSet("QueueSet").make(); + long start = System.currentTimeMillis(); + for (int i = 0; i < backingStore.getSize(); i++) { + queueSet.add(get(i)); + } + LOG.info("QueueSet population inserting " + backingStore.getSize() + + " took " + (System.currentTimeMillis() - start)); } SetMultimap deserializeInflightPuts() @@ -182,8 +222,10 @@ synchronized void addWithoutCommit(FlumeEventPointer e, long transactionID) { } /** - * Remove FlumeEventPointer from queue, will normally - * only be used when recovering from a crash + * Remove FlumeEventPointer from queue, will + * only be used when recovering from a crash. It is not + * legal to call this method after replayComplete has been + * called. * @param FlumeEventPointer to be removed * @return true if the FlumeEventPointer was found * and removed @@ -191,14 +233,25 @@ synchronized void addWithoutCommit(FlumeEventPointer e, long transactionID) { synchronized boolean remove(FlumeEventPointer e) { long value = e.toLong(); Preconditions.checkArgument(value != EMPTY); + if (queueSet == null) { + throw new IllegalStateException("QueueSet is null, thus replayComplete" + + " has been called which is illegal"); + } + if (!queueSet.contains(value)) { + return false; + } + searchCount++; + long start = System.currentTimeMillis(); for (int i = 0; i < backingStore.getSize(); i++) { if(get(i) == value) { remove(i, 0); FlumeEventPointer ptr = FlumeEventPointer.fromLong(value); backingStore.decrementFileID(ptr.getFileID()); + searchTime += System.currentTimeMillis() - start; return true; } } + searchTime += System.currentTimeMillis() - start; return false; } /** @@ -261,6 +314,9 @@ protected boolean add(int index, long value) { } } set(index, value); + if (queueSet != null) { + queueSet.add(value); + } return true; } @@ -279,7 +335,12 @@ protected synchronized long remove(int index, long transactionID) { throw new IndexOutOfBoundsException("index = " + index + ", queueSize " + backingStore.getSize() +" " + channelNameDescriptor); } + copyCount++; + long start = System.currentTimeMillis(); long value = get(index); + if (queueSet != null) { + queueSet.remove(value); + } //if txn id = 0, we are recovering from a crash. if(transactionID != 0) { inflightTakes.addEvent(transactionID, value); @@ -304,10 +365,10 @@ protected synchronized long remove(int index, long transactionID) { } } backingStore.setSize(backingStore.getSize() - 1); + copyTime += System.currentTimeMillis() - start; return value; } - protected synchronized int getSize() { return backingStore.getSize() + inflightTakes.getSize(); } @@ -320,6 +381,13 @@ public int getCapacity() { } synchronized void close() throws IOException { + try { + if (db != null) { + db.close(); + } + } catch(Exception ex) { + LOG.warn("Error closing db", ex); + } try { backingStore.close(); inflightPuts.close(); @@ -328,6 +396,33 @@ synchronized void close() throws IOException { LOG.warn("Error closing backing store", e); } } + + /** + * Called when ReplayHandler has completed and thus remove(FlumeEventPointer) + * will no longer be called. + */ + synchronized void replayComplete() { + String msg = "Search Count = " + searchCount + ", Search Time = " + + searchTime + ", Copy Count = " + copyCount + ", Copy Time = " + + copyTime; + LOG.info(msg); + if(db != null) { + db.close(); + } + queueSet = null; + db = null; + } + + @VisibleForTesting + long getSearchCount() { + return searchCount; + } + + @VisibleForTesting + long getCopyCount() { + return copyCount; + } + /** * A representation of in flight events which have not yet been committed. * None of the methods are thread safe, and should be called from thread @@ -340,7 +435,6 @@ class InflightEventWrapper { private volatile RandomAccessFile file; private volatile java.nio.channels.FileChannel fileChannel; private final MessageDigest digest; - private volatile Future future; private final File inflightEventsFile; private volatile boolean syncRequired = false; private SetMultimap inflightFileIDs = HashMultimap.create(); diff --git a/flume-ng-channels/flume-file-channel/src/main/java/org/apache/flume/channel/file/Log.java b/flume-ng-channels/flume-file-channel/src/main/java/org/apache/flume/channel/file/Log.java index 8a8cb7f64b..70106cbd39 100644 --- a/flume-ng-channels/flume-file-channel/src/main/java/org/apache/flume/channel/file/Log.java +++ b/flume-ng-channels/flume-file-channel/src/main/java/org/apache/flume/channel/file/Log.java @@ -78,6 +78,7 @@ public class Log { private static final Logger LOGGER = LoggerFactory.getLogger(Log.class); private static final int MIN_NUM_LOGS = 2; public static final String FILE_LOCK = "in_use.lock"; + public static final String QUEUE_SET = "queueset"; // for reader private final Map idLogFileMap = Collections .synchronizedMap(new HashMap()); @@ -103,7 +104,8 @@ public class Log { /** * Set of files that should be excluded from backup and restores. */ - public static final Set EXCLUDES = Sets.newHashSet(FILE_LOCK); + public static final Set EXCLUDES = Sets.newHashSet(FILE_LOCK, + QUEUE_SET); /** * Shared lock */ @@ -405,6 +407,7 @@ void replay() throws IOException { } File inflightTakesFile = new File(checkpointDir, "inflighttakes"); File inflightPutsFile = new File(checkpointDir, "inflightputs"); + File queueSetDir = new File(checkpointDir, QUEUE_SET); EventQueueBackingStore backingStore = null; @@ -414,7 +417,7 @@ void replay() throws IOException { backupCheckpointDir, queueCapacity, channelNameDescriptor, true, this.useDualCheckpoints); queue = new FlumeEventQueue(backingStore, inflightTakesFile, - inflightPutsFile); + inflightPutsFile, queueSetDir); LOGGER.info("Last Checkpoint " + new Date(checkpointFile.lastModified()) + ", queue depth = " + queue.getSize()); @@ -450,7 +453,7 @@ void replay() throws IOException { backupCheckpointDir, queueCapacity, channelNameDescriptor, true, useDualCheckpoints); queue = new FlumeEventQueue(backingStore, inflightTakesFile, - inflightPutsFile); + inflightPutsFile, queueSetDir); // If the checkpoint was deleted due to BadCheckpointException, then // trigger fast replay if the channel is configured to. shouldFastReplay = this.useFastReplay; diff --git a/flume-ng-channels/flume-file-channel/src/main/java/org/apache/flume/channel/file/LogFile.java b/flume-ng-channels/flume-file-channel/src/main/java/org/apache/flume/channel/file/LogFile.java index 62f68c6c74..26a24b1814 100644 --- a/flume-ng-channels/flume-file-channel/src/main/java/org/apache/flume/channel/file/LogFile.java +++ b/flume-ng-channels/flume-file-channel/src/main/java/org/apache/flume/channel/file/LogFile.java @@ -22,7 +22,6 @@ import com.google.common.base.Joiner; import com.google.common.base.Preconditions; import com.google.common.collect.Lists; -import org.apache.flume.ChannelException; import org.apache.flume.annotations.InterfaceAudience; import org.apache.flume.annotations.InterfaceStability; import org.apache.flume.channel.file.encryption.CipherProvider; diff --git a/flume-ng-channels/flume-file-channel/src/main/java/org/apache/flume/channel/file/ReplayHandler.java b/flume-ng-channels/flume-file-channel/src/main/java/org/apache/flume/channel/file/ReplayHandler.java index c8f5fdd417..e668c2e39e 100644 --- a/flume-ng-channels/flume-file-channel/src/main/java/org/apache/flume/channel/file/ReplayHandler.java +++ b/flume-ng-channels/flume-file-channel/src/main/java/org/apache/flume/channel/file/ReplayHandler.java @@ -26,7 +26,6 @@ import com.google.common.collect.Sets; import org.apache.commons.collections.MultiMap; import org.apache.commons.collections.map.MultiValueMap; -import org.apache.flume.ChannelException; import org.apache.flume.channel.file.encryption.KeyProvider; import org.slf4j.Logger; import org.slf4j.LoggerFactory; @@ -290,7 +289,9 @@ void replayLog(List logs) throws Exception { record.getLogWriteOrderID()); readCount++; if(readCount % 10000 == 0 && readCount > 0) { - LOG.info("Read " + readCount + " records"); + LOG.info("read: " + readCount + ", put: " + putCount + ", take: " + + takeCount + ", rollback: " + rollbackCount + ", commit: " + + commitCount + ", skip: " + skipCount + ", eventCount:" + count); } if (record.getLogWriteOrderID() > lastCheckpoint) { if (type == TransactionEventRecord.Type.PUT.get()) { @@ -339,6 +340,7 @@ void replayLog(List logs) throws Exception { LOG.info("read: " + readCount + ", put: " + putCount + ", take: " + takeCount + ", rollback: " + rollbackCount + ", commit: " + commitCount + ", skip: " + skipCount + ", eventCount:" + count); + queue.replayComplete(); } finally { TransactionIDOracle.setSeed(transactionIDSeed); WriteOrderOracle.setSeed(writeOrderIDSeed); @@ -363,15 +365,9 @@ void replayLog(List logs) throws Exception { count += uncommittedTakes; int pendingTakesSize = pendingTakes.size(); if (pendingTakesSize > 0) { - String msg = "Pending takes " + pendingTakesSize - + " exist after the end of replay"; - if (LOG.isDebugEnabled()) { - for (Long pointer : pendingTakes) { - LOG.debug("Pending take " + FlumeEventPointer.fromLong(pointer)); - } - } else { - LOG.error(msg + ". Duplicate messages will exist in destination."); - } + LOG.info("Pending takes " + pendingTakesSize + " exist after the" + + " end of replay. Duplicate messages will exist in" + + " destination."); } } private LogRecord next() throws IOException, CorruptEventException { diff --git a/flume-ng-channels/flume-file-channel/src/main/java/org/apache/flume/channel/file/Serialization.java b/flume-ng-channels/flume-file-channel/src/main/java/org/apache/flume/channel/file/Serialization.java index f8160d988e..d55660d186 100644 --- a/flume-ng-channels/flume-file-channel/src/main/java/org/apache/flume/channel/file/Serialization.java +++ b/flume-ng-channels/flume-file-channel/src/main/java/org/apache/flume/channel/file/Serialization.java @@ -98,7 +98,7 @@ static boolean deleteAllFiles(File checkpointDir, builder = new StringBuilder("Deleted the following files: "); } if(excludes == null) { - excludes = Collections.EMPTY_SET; + excludes = Collections.emptySet(); } for (File file : files) { if(excludes.contains(file.getName())) { diff --git a/flume-ng-channels/flume-file-channel/src/test/java/org/apache/flume/channel/file/TestCheckpoint.java b/flume-ng-channels/flume-file-channel/src/test/java/org/apache/flume/channel/file/TestCheckpoint.java index 1e0230d459..c1de12e350 100644 --- a/flume-ng-channels/flume-file-channel/src/test/java/org/apache/flume/channel/file/TestCheckpoint.java +++ b/flume-ng-channels/flume-file-channel/src/test/java/org/apache/flume/channel/file/TestCheckpoint.java @@ -32,11 +32,13 @@ public class TestCheckpoint { File file; File inflightPuts; File inflightTakes; + File queueSet; @Before public void setup() throws IOException { file = File.createTempFile("Checkpoint", ""); inflightPuts = File.createTempFile("inflightPuts", ""); inflightTakes = File.createTempFile("inflightTakes", ""); + queueSet = File.createTempFile("queueset", ""); Assert.assertTrue(file.isFile()); Assert.assertTrue(file.canWrite()); } @@ -50,14 +52,14 @@ public void testSerialization() throws Exception { new EventQueueBackingStoreFileV2(file, 1, "test"); FlumeEventPointer ptrIn = new FlumeEventPointer(10, 20); FlumeEventQueue queueIn = new FlumeEventQueue(backingStore, - inflightTakes, inflightPuts); + inflightTakes, inflightPuts, queueSet); queueIn.addHead(ptrIn); FlumeEventQueue queueOut = new FlumeEventQueue(backingStore, - inflightTakes, inflightPuts); + inflightTakes, inflightPuts, queueSet); Assert.assertEquals(0, queueOut.getLogWriteOrderID()); queueIn.checkpoint(false); FlumeEventQueue queueOut2 = new FlumeEventQueue(backingStore, - inflightTakes, inflightPuts); + inflightTakes, inflightPuts, queueSet); FlumeEventPointer ptrOut = queueOut2.removeHead(0L); Assert.assertEquals(ptrIn, ptrOut); Assert.assertTrue(queueOut2.getLogWriteOrderID() > 0); diff --git a/flume-ng-channels/flume-file-channel/src/test/java/org/apache/flume/channel/file/TestCheckpointRebuilder.java b/flume-ng-channels/flume-file-channel/src/test/java/org/apache/flume/channel/file/TestCheckpointRebuilder.java index 536af54e09..621d445ffa 100644 --- a/flume-ng-channels/flume-file-channel/src/test/java/org/apache/flume/channel/file/TestCheckpointRebuilder.java +++ b/flume-ng-channels/flume-file-channel/src/test/java/org/apache/flume/channel/file/TestCheckpointRebuilder.java @@ -63,6 +63,7 @@ public void testFastReplay() throws Exception { File metaDataFile = Serialization.getMetaDataFile(checkpointFile); File inflightTakesFile = new File(checkpointDir, "inflighttakes"); File inflightPutsFile = new File(checkpointDir, "inflightputs"); + File queueSetDir = new File(checkpointDir, "queueset"); Assert.assertTrue(checkpointFile.delete()); Assert.assertTrue(metaDataFile.delete()); Assert.assertTrue(inflightTakesFile.delete()); @@ -71,7 +72,7 @@ public void testFastReplay() throws Exception { EventQueueBackingStoreFactory.get(checkpointFile, 50, "test"); FlumeEventQueue queue = new FlumeEventQueue(backingStore, inflightTakesFile, - inflightPutsFile); + inflightPutsFile, queueSetDir); CheckpointRebuilder checkpointRebuilder = new CheckpointRebuilder(getAllLogs(dataDirs), queue); Assert.assertTrue(checkpointRebuilder.rebuild()); diff --git a/flume-ng-channels/flume-file-channel/src/test/java/org/apache/flume/channel/file/TestEventQueueBackingStoreFactory.java b/flume-ng-channels/flume-file-channel/src/test/java/org/apache/flume/channel/file/TestEventQueueBackingStoreFactory.java index dfb3bf983b..52c706d59a 100644 --- a/flume-ng-channels/flume-file-channel/src/test/java/org/apache/flume/channel/file/TestEventQueueBackingStoreFactory.java +++ b/flume-ng-channels/flume-file-channel/src/test/java/org/apache/flume/channel/file/TestEventQueueBackingStoreFactory.java @@ -58,12 +58,14 @@ public class TestEventQueueBackingStoreFactory { File checkpoint; File inflightTakes; File inflightPuts; + File queueSetDir; @Before public void setup() throws IOException { baseDir = Files.createTempDir(); checkpoint = new File(baseDir, "checkpoint"); inflightTakes = new File(baseDir, "takes"); inflightPuts = new File(baseDir, "puts"); + queueSetDir = new File(baseDir, "queueset"); TestUtils.copyDecompressed("fileformat-v2-checkpoint.gz", checkpoint); } @@ -275,7 +277,7 @@ private void verify(EventQueueBackingStore backingStore, long expectedVersion, List expectedPointers) throws Exception { FlumeEventQueue queue = new FlumeEventQueue(backingStore, inflightTakes, - inflightPuts); + inflightPuts, queueSetDir); List actualPointers = Lists.newArrayList(); FlumeEventPointer ptr; while((ptr = queue.removeHead(0L)) != null) { diff --git a/flume-ng-channels/flume-file-channel/src/test/java/org/apache/flume/channel/file/TestFlumeEventQueue.java b/flume-ng-channels/flume-file-channel/src/test/java/org/apache/flume/channel/file/TestFlumeEventQueue.java index 203cbf2c77..1adb21a63f 100644 --- a/flume-ng-channels/flume-file-channel/src/test/java/org/apache/flume/channel/file/TestFlumeEventQueue.java +++ b/flume-ng-channels/flume-file-channel/src/test/java/org/apache/flume/channel/file/TestFlumeEventQueue.java @@ -44,6 +44,7 @@ public class TestFlumeEventQueue { FlumeEventPointer pointer1 = new FlumeEventPointer(1, 1); FlumeEventPointer pointer2 = new FlumeEventPointer(2, 2); + FlumeEventPointer pointer3 = new FlumeEventPointer(3, 3); FlumeEventQueue queue; EventQueueBackingStoreSupplier backingStoreSupplier; EventQueueBackingStore backingStore; @@ -53,11 +54,13 @@ private abstract static class EventQueueBackingStoreSupplier { File checkpoint; File inflightTakes; File inflightPuts; + File queueSetDir; EventQueueBackingStoreSupplier() { baseDir = Files.createTempDir(); checkpoint = new File(baseDir, "checkpoint"); inflightTakes = new File(baseDir, "inflightputs"); inflightPuts = new File(baseDir, "inflighttakes"); + queueSetDir = new File(baseDir, "queueset"); } File getCheckpoint() { return checkpoint; @@ -68,6 +71,9 @@ File getInflightPuts() { File getInflightTakes() { return inflightTakes; } + File getQueueSetDir() { + return queueSetDir; + } void delete() { FileUtils.deleteQuietly(baseDir); } @@ -120,7 +126,8 @@ public void testCapacity() throws Exception { backingStore = new EventQueueBackingStoreFileV2(checkpoint, 1, "test"); queue = new FlumeEventQueue(backingStore, backingStoreSupplier.getInflightTakes(), - backingStoreSupplier.getInflightPuts()); + backingStoreSupplier.getInflightPuts(), + backingStoreSupplier.getQueueSetDir()); Assert.assertTrue(queue.addTail(pointer1)); Assert.assertFalse(queue.addTail(pointer2)); } @@ -132,7 +139,8 @@ public void testInvalidCapacityZero() throws Exception { backingStore = new EventQueueBackingStoreFileV2(checkpoint, 0, "test"); queue = new FlumeEventQueue(backingStore, backingStoreSupplier.getInflightTakes(), - backingStoreSupplier.getInflightPuts()); + backingStoreSupplier.getInflightPuts(), + backingStoreSupplier.getQueueSetDir()); } @Test(expected=IllegalArgumentException.class) public void testInvalidCapacityNegative() throws Exception { @@ -142,20 +150,23 @@ public void testInvalidCapacityNegative() throws Exception { backingStore = new EventQueueBackingStoreFileV2(checkpoint, -1, "test"); queue = new FlumeEventQueue(backingStore, backingStoreSupplier.getInflightTakes(), - backingStoreSupplier.getInflightPuts()); + backingStoreSupplier.getInflightPuts(), + backingStoreSupplier.getQueueSetDir()); } @Test public void testQueueIsEmptyAfterCreation() throws Exception { queue = new FlumeEventQueue(backingStore, backingStoreSupplier.getInflightTakes(), - backingStoreSupplier.getInflightPuts()); + backingStoreSupplier.getInflightPuts(), + backingStoreSupplier.getQueueSetDir()); Assert.assertNull(queue.removeHead(0L)); } @Test public void addTail1() throws Exception { queue = new FlumeEventQueue(backingStore, backingStoreSupplier.getInflightTakes(), - backingStoreSupplier.getInflightPuts()); + backingStoreSupplier.getInflightPuts(), + backingStoreSupplier.getQueueSetDir()); Assert.assertTrue(queue.addTail(pointer1)); Assert.assertEquals(pointer1, queue.removeHead(0)); Assert.assertEquals(Sets.newHashSet(), queue.getFileIDs()); @@ -164,7 +175,8 @@ public void addTail1() throws Exception { public void addTail2() throws Exception { queue = new FlumeEventQueue(backingStore, backingStoreSupplier.getInflightTakes(), - backingStoreSupplier.getInflightPuts()); + backingStoreSupplier.getInflightPuts(), + backingStoreSupplier.getQueueSetDir()); Assert.assertTrue(queue.addTail(pointer1)); Assert.assertTrue(queue.addTail(pointer2)); Assert.assertEquals(Sets.newHashSet(1, 2), queue.getFileIDs()); @@ -175,7 +187,8 @@ public void addTail2() throws Exception { public void addTailLarge() throws Exception { queue = new FlumeEventQueue(backingStore, backingStoreSupplier.getInflightTakes(), - backingStoreSupplier.getInflightPuts()); + backingStoreSupplier.getInflightPuts(), + backingStoreSupplier.getQueueSetDir()); int size = 500; Set fileIDs = Sets.newHashSet(); for (int i = 1; i <= size; i++) { @@ -194,7 +207,8 @@ public void addTailLarge() throws Exception { public void addHead1() throws Exception { queue = new FlumeEventQueue(backingStore, backingStoreSupplier.getInflightTakes(), - backingStoreSupplier.getInflightPuts()); + backingStoreSupplier.getInflightPuts(), + backingStoreSupplier.getQueueSetDir()); Assert.assertTrue(queue.addHead(pointer1)); Assert.assertEquals(Sets.newHashSet(1), queue.getFileIDs()); Assert.assertEquals(pointer1, queue.removeHead(0)); @@ -204,7 +218,9 @@ public void addHead1() throws Exception { public void addHead2() throws Exception { queue = new FlumeEventQueue(backingStore, backingStoreSupplier.getInflightTakes(), - backingStoreSupplier.getInflightPuts()); + backingStoreSupplier.getInflightPuts(), + backingStoreSupplier.getQueueSetDir()); + queue.replayComplete(); Assert.assertTrue(queue.addHead(pointer1)); Assert.assertTrue(queue.addHead(pointer2)); Assert.assertEquals(Sets.newHashSet(1, 2), queue.getFileIDs()); @@ -215,7 +231,9 @@ public void addHead2() throws Exception { public void addHeadLarge() throws Exception { queue = new FlumeEventQueue(backingStore, backingStoreSupplier.getInflightTakes(), - backingStoreSupplier.getInflightPuts()); + backingStoreSupplier.getInflightPuts(), + backingStoreSupplier.getQueueSetDir()); + queue.replayComplete(); int size = 500; Set fileIDs = Sets.newHashSet(); for (int i = 1; i <= size; i++) { @@ -234,10 +252,12 @@ public void addHeadLarge() throws Exception { public void addTailRemove1() throws Exception { queue = new FlumeEventQueue(backingStore, backingStoreSupplier.getInflightTakes(), - backingStoreSupplier.getInflightPuts()); + backingStoreSupplier.getInflightPuts(), + backingStoreSupplier.getQueueSetDir()); Assert.assertTrue(queue.addTail(pointer1)); Assert.assertEquals(Sets.newHashSet(1), queue.getFileIDs()); Assert.assertTrue(queue.remove(pointer1)); + queue.replayComplete(); Assert.assertEquals(Sets.newHashSet(), queue.getFileIDs()); Assert.assertNull(queue.removeHead(0)); Assert.assertEquals(Sets.newHashSet(), queue.getFileIDs()); @@ -247,10 +267,12 @@ public void addTailRemove1() throws Exception { public void addTailRemove2() throws Exception { queue = new FlumeEventQueue(backingStore, backingStoreSupplier.getInflightTakes(), - backingStoreSupplier.getInflightPuts()); + backingStoreSupplier.getInflightPuts(), + backingStoreSupplier.getQueueSetDir()); Assert.assertTrue(queue.addTail(pointer1)); Assert.assertTrue(queue.addTail(pointer2)); Assert.assertTrue(queue.remove(pointer1)); + queue.replayComplete(); Assert.assertEquals(pointer2, queue.removeHead(0)); } @@ -258,7 +280,8 @@ public void addTailRemove2() throws Exception { public void addHeadRemove1() throws Exception { queue = new FlumeEventQueue(backingStore, backingStoreSupplier.getInflightTakes(), - backingStoreSupplier.getInflightPuts()); + backingStoreSupplier.getInflightPuts(), + backingStoreSupplier.getQueueSetDir()); queue.addHead(pointer1); Assert.assertTrue(queue.remove(pointer1)); Assert.assertNull(queue.removeHead(0)); @@ -267,17 +290,43 @@ public void addHeadRemove1() throws Exception { public void addHeadRemove2() throws Exception { queue = new FlumeEventQueue(backingStore, backingStoreSupplier.getInflightTakes(), - backingStoreSupplier.getInflightPuts()); + backingStoreSupplier.getInflightPuts(), + backingStoreSupplier.getQueueSetDir()); Assert.assertTrue(queue.addHead(pointer1)); Assert.assertTrue(queue.addHead(pointer2)); Assert.assertTrue(queue.remove(pointer1)); + queue.replayComplete(); Assert.assertEquals(pointer2, queue.removeHead(0)); } @Test + public void testUnknownPointerDoesNotCauseSearch() throws Exception { + queue = new FlumeEventQueue(backingStore, + backingStoreSupplier.getInflightTakes(), + backingStoreSupplier.getInflightPuts(), + backingStoreSupplier.getQueueSetDir()); + Assert.assertTrue(queue.addHead(pointer1)); + Assert.assertTrue(queue.addHead(pointer2)); + Assert.assertFalse(queue.remove(pointer3)); // does search + Assert.assertTrue(queue.remove(pointer1)); + Assert.assertTrue(queue.remove(pointer2)); + queue.replayComplete(); + Assert.assertEquals(2, queue.getSearchCount()); + } + @Test(expected=IllegalStateException.class) + public void testRemoveAfterReplayComplete() throws Exception { + queue = new FlumeEventQueue(backingStore, + backingStoreSupplier.getInflightTakes(), + backingStoreSupplier.getInflightPuts(), + backingStoreSupplier.getQueueSetDir()); + queue.replayComplete(); + queue.remove(pointer1); + } + @Test public void testWrappingCorrectly() throws Exception { queue = new FlumeEventQueue(backingStore, backingStoreSupplier.getInflightTakes(), - backingStoreSupplier.getInflightPuts()); + backingStoreSupplier.getInflightPuts(), + backingStoreSupplier.getQueueSetDir()); int size = Integer.MAX_VALUE; for (int i = 1; i <= size; i++) { if(!queue.addHead(new FlumeEventPointer(i, i))) { @@ -299,7 +348,8 @@ public void testWrappingCorrectly() throws Exception { public void testInflightPuts() throws Exception{ queue = new FlumeEventQueue(backingStore, backingStoreSupplier.getInflightTakes(), - backingStoreSupplier.getInflightPuts()); + backingStoreSupplier.getInflightPuts(), + backingStoreSupplier.getQueueSetDir()); long txnID1 = new Random().nextInt(Integer.MAX_VALUE - 1); long txnID2 = txnID1 + 1; queue.addWithoutCommit(new FlumeEventPointer(1, 1), txnID1); @@ -309,7 +359,8 @@ public void testInflightPuts() throws Exception{ TimeUnit.SECONDS.sleep(3L); queue = new FlumeEventQueue(backingStore, backingStoreSupplier.getInflightTakes(), - backingStoreSupplier.getInflightPuts()); + backingStoreSupplier.getInflightPuts(), + backingStoreSupplier.getQueueSetDir()); SetMultimap deserializedMap = queue.deserializeInflightPuts(); Assert.assertTrue(deserializedMap.get( txnID1).contains(new FlumeEventPointer(1, 1).toLong())); @@ -323,7 +374,8 @@ public void testInflightPuts() throws Exception{ public void testInflightTakes() throws Exception { queue = new FlumeEventQueue(backingStore, backingStoreSupplier.getInflightTakes(), - backingStoreSupplier.getInflightPuts()); + backingStoreSupplier.getInflightPuts(), + backingStoreSupplier.getQueueSetDir()); long txnID1 = new Random().nextInt(Integer.MAX_VALUE - 1); long txnID2 = txnID1 + 1; queue.addTail(new FlumeEventPointer(1, 1)); @@ -336,7 +388,8 @@ public void testInflightTakes() throws Exception { TimeUnit.SECONDS.sleep(3L); queue = new FlumeEventQueue(backingStore, backingStoreSupplier.getInflightTakes(), - backingStoreSupplier.getInflightPuts()); + backingStoreSupplier.getInflightPuts(), + backingStoreSupplier.getQueueSetDir()); SetMultimap deserializedMap = queue.deserializeInflightTakes(); Assert.assertTrue(deserializedMap.get( txnID1).contains(new FlumeEventPointer(1, 1).toLong())); @@ -353,7 +406,8 @@ public void testCorruptInflightPuts() throws Exception { try { queue = new FlumeEventQueue(backingStore, backingStoreSupplier.getInflightTakes(), - backingStoreSupplier.getInflightPuts()); + backingStoreSupplier.getInflightPuts(), + backingStoreSupplier.getQueueSetDir()); long txnID1 = new Random().nextInt(Integer.MAX_VALUE - 1); long txnID2 = txnID1 + 1; queue.addWithoutCommit(new FlumeEventPointer(1, 1), txnID1); @@ -367,7 +421,8 @@ public void testCorruptInflightPuts() throws Exception { inflight.writeInt(new Random().nextInt()); queue = new FlumeEventQueue(backingStore, backingStoreSupplier.getInflightTakes(), - backingStoreSupplier.getInflightPuts()); + backingStoreSupplier.getInflightPuts(), + backingStoreSupplier.getQueueSetDir()); SetMultimap deserializedMap = queue.deserializeInflightPuts(); Assert.assertTrue(deserializedMap.get( txnID1).contains(new FlumeEventPointer(1, 1).toLong())); @@ -386,7 +441,8 @@ public void testCorruptInflightTakes() throws Exception { try { queue = new FlumeEventQueue(backingStore, backingStoreSupplier.getInflightTakes(), - backingStoreSupplier.getInflightPuts()); + backingStoreSupplier.getInflightPuts(), + backingStoreSupplier.getQueueSetDir()); long txnID1 = new Random().nextInt(Integer.MAX_VALUE - 1); long txnID2 = txnID1 + 1; queue.addWithoutCommit(new FlumeEventPointer(1, 1), txnID1); @@ -400,7 +456,8 @@ public void testCorruptInflightTakes() throws Exception { inflight.writeInt(new Random().nextInt()); queue = new FlumeEventQueue(backingStore, backingStoreSupplier.getInflightTakes(), - backingStoreSupplier.getInflightPuts()); + backingStoreSupplier.getInflightPuts(), + backingStoreSupplier.getQueueSetDir()); SetMultimap deserializedMap = queue.deserializeInflightTakes(); Assert.assertTrue(deserializedMap.get( txnID1).contains(new FlumeEventPointer(1, 1).toLong())); diff --git a/pom.xml b/pom.xml index 267925f70b..77b7f226e7 100644 --- a/pom.xml +++ b/pom.xml @@ -774,6 +774,12 @@ limitations under the License. 4.2.1 + + org.mapdb + mapdb + 0.9.7 + + com.google.code.gson From 58f3f6fb18e18fbf67fbd1ae0044c337845eba8d Mon Sep 17 00:00:00 2001 From: Hari Shreedharan Date: Fri, 13 Dec 2013 13:23:29 -0800 Subject: [PATCH 050/341] FLUME-1679. Add dependency on Guava to flume-ng-elasticsearch-sink POM (Andrew Purtell via Hari Shreedharan) --- flume-ng-sinks/flume-ng-elasticsearch-sink/pom.xml | 5 +++++ 1 file changed, 5 insertions(+) diff --git a/flume-ng-sinks/flume-ng-elasticsearch-sink/pom.xml b/flume-ng-sinks/flume-ng-elasticsearch-sink/pom.xml index 7d5b0a8088..bdc21d1eb8 100644 --- a/flume-ng-sinks/flume-ng-elasticsearch-sink/pom.xml +++ b/flume-ng-sinks/flume-ng-elasticsearch-sink/pom.xml @@ -73,5 +73,10 @@ commons-lang + + com.google.guava + guava + + From 79dc97bddbf6602c5f375337b3261f33d5555775 Mon Sep 17 00:00:00 2001 From: Hari Shreedharan Date: Fri, 13 Dec 2013 14:14:57 -0800 Subject: [PATCH 051/341] FLUME-2264. Log4j Appender + Avro Reflection on string results in an invalid avro schema (Brock Noland via Hari Shreedharan) --- flume-ng-doc/sphinx/FlumeUserGuide.rst | 8 ++++---- 1 file changed, 4 insertions(+), 4 deletions(-) diff --git a/flume-ng-doc/sphinx/FlumeUserGuide.rst b/flume-ng-doc/sphinx/FlumeUserGuide.rst index ae66f89e55..026907424b 100644 --- a/flume-ng-doc/sphinx/FlumeUserGuide.rst +++ b/flume-ng-doc/sphinx/FlumeUserGuide.rst @@ -2813,18 +2813,18 @@ appender must have the flume-ng-sdk in the classpath (eg, flume-ng-sdk-1.5.0-SNAPSHOT.jar). Required properties are in **bold**. -===================== ======= ============================================================== +===================== ======= ================================================================================== Property Name Default Description -===================== ======= ============================================================== +===================== ======= ================================================================================== **Hostname** -- The hostname on which a remote Flume agent is running with an avro source. **Port** -- The port at which the remote Flume agent's avro source is listening. UnsafeMode false If true, the appender will not throw exceptions on failure to send the events. -AvroReflectionEnabled false Use Avro Reflection to serialize Log4j events. +AvroReflectionEnabled false Use Avro Reflection to serialize Log4j events. (Do not use when users log strings) AvroSchemaUrl -- A URL from which the Avro schema can be retrieved. -===================== ======= ============================================================== +===================== ======= ================================================================================== Sample log4j.properties file: From 90bb15383c9a6d0b376c3ff5c83adade5092f8c4 Mon Sep 17 00:00:00 2001 From: Hari Shreedharan Date: Fri, 13 Dec 2013 14:47:27 -0800 Subject: [PATCH 052/341] FLUME-2239. Clarify File Channel's dataDirs setting in User Guide (Roshan Naik via Hari Shreedharan) --- flume-ng-doc/sphinx/FlumeUserGuide.rst | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/flume-ng-doc/sphinx/FlumeUserGuide.rst b/flume-ng-doc/sphinx/FlumeUserGuide.rst index 026907424b..335a991d01 100644 --- a/flume-ng-doc/sphinx/FlumeUserGuide.rst +++ b/flume-ng-doc/sphinx/FlumeUserGuide.rst @@ -2138,7 +2138,7 @@ Property Name Default Description checkpointDir ~/.flume/file-channel/checkpoint The directory where checkpoint file will be stored useDualCheckpoints false Backup the checkpoint. If this is set to ``true``, ``backupCheckpointDir`` **must** be set backupCheckpointDir -- The directory where the checkpoint is backed up to. This directory **must not** be the same as the data directories or the checkpoint directory -dataDirs ~/.flume/file-channel/data The directory where log files will be stored +dataDirs ~/.flume/file-channel/data Comma separated list of directories for storing log files. Using multiple directories on separate disks can improve file channel peformance transactionCapacity 1000 The maximum size of transaction supported by the channel checkpointInterval 30000 Amount of time (in millis) between checkpoints maxFileSize 2146435071 Max size (in bytes) of a single log file From 0d04373473e302a3155b12945e3350cee760c151 Mon Sep 17 00:00:00 2001 From: Mike Percy Date: Sat, 14 Dec 2013 00:51:57 -0800 Subject: [PATCH 053/341] FLUME-2267. Increase default transactionCapacity for FileChannel from 1000 to 10000 (Udai Kiran Potluri via Mike Percy) --- .../org/apache/flume/channel/file/FileChannelConfiguration.java | 2 +- flume-ng-doc/sphinx/FlumeUserGuide.rst | 2 +- 2 files changed, 2 insertions(+), 2 deletions(-) diff --git a/flume-ng-channels/flume-file-channel/src/main/java/org/apache/flume/channel/file/FileChannelConfiguration.java b/flume-ng-channels/flume-file-channel/src/main/java/org/apache/flume/channel/file/FileChannelConfiguration.java index c2dcffc7ec..10ca11f1eb 100644 --- a/flume-ng-channels/flume-file-channel/src/main/java/org/apache/flume/channel/file/FileChannelConfiguration.java +++ b/flume-ng-channels/flume-file-channel/src/main/java/org/apache/flume/channel/file/FileChannelConfiguration.java @@ -39,7 +39,7 @@ public class FileChannelConfiguration { * Maximum number of put/take events in a transaction. Default: 1000 */ public static final String TRANSACTION_CAPACITY = "transactionCapacity"; - public static final int DEFAULT_TRANSACTION_CAPACITY = 1000; + public static final int DEFAULT_TRANSACTION_CAPACITY = 10000; /** * Interval at which checkpoints should be taken. Default 30s (ms) */ diff --git a/flume-ng-doc/sphinx/FlumeUserGuide.rst b/flume-ng-doc/sphinx/FlumeUserGuide.rst index 335a991d01..7a41efb5c4 100644 --- a/flume-ng-doc/sphinx/FlumeUserGuide.rst +++ b/flume-ng-doc/sphinx/FlumeUserGuide.rst @@ -2139,7 +2139,7 @@ checkpointDir ~/.flume/file-channel/checkpoi useDualCheckpoints false Backup the checkpoint. If this is set to ``true``, ``backupCheckpointDir`` **must** be set backupCheckpointDir -- The directory where the checkpoint is backed up to. This directory **must not** be the same as the data directories or the checkpoint directory dataDirs ~/.flume/file-channel/data Comma separated list of directories for storing log files. Using multiple directories on separate disks can improve file channel peformance -transactionCapacity 1000 The maximum size of transaction supported by the channel +transactionCapacity 10000 The maximum size of transaction supported by the channel checkpointInterval 30000 Amount of time (in millis) between checkpoints maxFileSize 2146435071 Max size (in bytes) of a single log file minimumRequiredSpace 524288000 Minimum Required free space (in bytes). To avoid data corruption, File Channel stops accepting take/put requests when free space drops below this value From 72b3b7889f1dcf6736f099b77c878efe891c6585 Mon Sep 17 00:00:00 2001 From: Mike Percy Date: Mon, 16 Dec 2013 14:59:14 -0800 Subject: [PATCH 054/341] FLUME-2056. Allow SpoolDir to pass just the filename that is the source of an event (Jeff Lord via Mike Percy) --- .../avro/ReliableSpoolingFileEventReader.java | 30 ++++++++++++++- .../flume/source/SpoolDirectorySource.java | 8 ++++ ...DirectorySourceConfigurationConstants.java | 12 +++++- .../source/TestSpoolDirectorySource.java | 38 ++++++++++++++++++- flume-ng-doc/sphinx/FlumeUserGuide.rst | 6 ++- 5 files changed, 87 insertions(+), 7 deletions(-) diff --git a/flume-ng-core/src/main/java/org/apache/flume/client/avro/ReliableSpoolingFileEventReader.java b/flume-ng-core/src/main/java/org/apache/flume/client/avro/ReliableSpoolingFileEventReader.java index bd684ed848..a88ed6e453 100644 --- a/flume-ng-core/src/main/java/org/apache/flume/client/avro/ReliableSpoolingFileEventReader.java +++ b/flume-ng-core/src/main/java/org/apache/flume/client/avro/ReliableSpoolingFileEventReader.java @@ -92,7 +92,9 @@ public class ReliableSpoolingFileEventReader implements ReliableEventReader { private final Pattern ignorePattern; private final File metaFile; private final boolean annotateFileName; + private final boolean annotateBaseName; private final String fileNameHeader; + private final String baseNameHeader; private final String deletePolicy; private final Charset inputCharset; private final DecodeErrorPolicy decodeErrorPolicy; @@ -108,6 +110,7 @@ public class ReliableSpoolingFileEventReader implements ReliableEventReader { private ReliableSpoolingFileEventReader(File spoolDirectory, String completedSuffix, String ignorePattern, String trackerDirPath, boolean annotateFileName, String fileNameHeader, + boolean annotateBaseName, String baseNameHeader, String deserializerType, Context deserializerContext, String deletePolicy, String inputCharset, DecodeErrorPolicy decodeErrorPolicy) throws IOException { @@ -164,6 +167,8 @@ private ReliableSpoolingFileEventReader(File spoolDirectory, this.deserializerContext = deserializerContext; this.annotateFileName = annotateFileName; this.fileNameHeader = fileNameHeader; + this.annotateBaseName = annotateBaseName; + this.baseNameHeader = baseNameHeader; this.ignorePattern = Pattern.compile(ignorePattern); this.deletePolicy = deletePolicy; this.inputCharset = Charset.forName(inputCharset); @@ -253,6 +258,13 @@ public List readEvents(int numEvents) throws IOException { } } + if (annotateBaseName) { + String basename = currentFile.get().getFile().getName(); + for (Event event : events) { + event.getHeaders().put(baseNameHeader, basename); + } + } + committed = false; lastFileRead = currentFile; return events; @@ -510,6 +522,10 @@ public static class Builder { SpoolDirectorySourceConfigurationConstants.DEFAULT_FILE_HEADER; private String fileNameHeader = SpoolDirectorySourceConfigurationConstants.DEFAULT_FILENAME_HEADER_KEY; + private Boolean annotateBaseName = + SpoolDirectorySourceConfigurationConstants.DEFAULT_BASENAME_HEADER; + private String baseNameHeader = + SpoolDirectorySourceConfigurationConstants.DEFAULT_BASENAME_HEADER_KEY; private String deserializerType = SpoolDirectorySourceConfigurationConstants.DEFAULT_DESERIALIZER; private Context deserializerContext = new Context(); @@ -551,6 +567,16 @@ public Builder fileNameHeader(String fileNameHeader) { return this; } + public Builder annotateBaseName(Boolean annotateBaseName) { + this.annotateBaseName = annotateBaseName; + return this; + } + + public Builder baseNameHeader(String baseNameHeader) { + this.baseNameHeader = baseNameHeader; + return this; + } + public Builder deserializerType(String deserializerType) { this.deserializerType = deserializerType; return this; @@ -579,8 +605,8 @@ public Builder decodeErrorPolicy(DecodeErrorPolicy decodeErrorPolicy) { public ReliableSpoolingFileEventReader build() throws IOException { return new ReliableSpoolingFileEventReader(spoolDirectory, completedSuffix, ignorePattern, trackerDirPath, annotateFileName, fileNameHeader, - deserializerType, deserializerContext, deletePolicy, inputCharset, - decodeErrorPolicy); + annotateBaseName, baseNameHeader, deserializerType, + deserializerContext, deletePolicy, inputCharset, decodeErrorPolicy); } } diff --git a/flume-ng-core/src/main/java/org/apache/flume/source/SpoolDirectorySource.java b/flume-ng-core/src/main/java/org/apache/flume/source/SpoolDirectorySource.java index 016021590e..f42ed2d859 100644 --- a/flume-ng-core/src/main/java/org/apache/flume/source/SpoolDirectorySource.java +++ b/flume-ng-core/src/main/java/org/apache/flume/source/SpoolDirectorySource.java @@ -54,6 +54,8 @@ public class SpoolDirectorySource extends AbstractSource implements private String spoolDirectory; private boolean fileHeader; private String fileHeaderKey; + private boolean basenameHeader; + private String basenameHeaderKey; private int batchSize; private String ignorePattern; private String trackerDirPath; @@ -87,6 +89,8 @@ public synchronized void start() { .trackerDirPath(trackerDirPath) .annotateFileName(fileHeader) .fileNameHeader(fileHeaderKey) + .annotateBaseName(basenameHeader) + .baseNameHeader(basenameHeaderKey) .deserializerType(deserializerType) .deserializerContext(deserializerContext) .deletePolicy(deletePolicy) @@ -142,6 +146,10 @@ public synchronized void configure(Context context) { DEFAULT_FILE_HEADER); fileHeaderKey = context.getString(FILENAME_HEADER_KEY, DEFAULT_FILENAME_HEADER_KEY); + basenameHeader = context.getBoolean(BASENAME_HEADER, + DEFAULT_BASENAME_HEADER); + basenameHeaderKey = context.getString(BASENAME_HEADER_KEY, + DEFAULT_BASENAME_HEADER_KEY); batchSize = context.getInteger(BATCH_SIZE, DEFAULT_BATCH_SIZE); inputCharset = context.getString(INPUT_CHARSET, DEFAULT_INPUT_CHARSET); diff --git a/flume-ng-core/src/main/java/org/apache/flume/source/SpoolDirectorySourceConfigurationConstants.java b/flume-ng-core/src/main/java/org/apache/flume/source/SpoolDirectorySourceConfigurationConstants.java index a2befe83be..83522c0d0a 100644 --- a/flume-ng-core/src/main/java/org/apache/flume/source/SpoolDirectorySourceConfigurationConstants.java +++ b/flume-ng-core/src/main/java/org/apache/flume/source/SpoolDirectorySourceConfigurationConstants.java @@ -27,14 +27,22 @@ public class SpoolDirectorySourceConfigurationConstants { public static final String SPOOLED_FILE_SUFFIX = "fileSuffix"; public static final String DEFAULT_SPOOLED_FILE_SUFFIX = ".COMPLETED"; - /** Header in which to put filename. */ + /** Header in which to put absolute path filename. */ public static final String FILENAME_HEADER_KEY = "fileHeaderKey"; public static final String DEFAULT_FILENAME_HEADER_KEY = "file"; - /** Whether to include filename in a header. */ + /** Whether to include absolute path filename in a header. */ public static final String FILENAME_HEADER = "fileHeader"; public static final boolean DEFAULT_FILE_HEADER = false; + /** Header in which to put the basename of file. */ + public static final String BASENAME_HEADER_KEY = "basenameHeaderKey"; + public static final String DEFAULT_BASENAME_HEADER_KEY = "basename"; + + /** Whether to include the basename of a file in a header. */ + public static final String BASENAME_HEADER = "basenameHeader"; + public static final boolean DEFAULT_BASENAME_HEADER = false; + /** What size to batch with before sending to ChannelProcessor. */ public static final String BATCH_SIZE = "batchSize"; public static final int DEFAULT_BATCH_SIZE = 100; diff --git a/flume-ng-core/src/test/java/org/apache/flume/source/TestSpoolDirectorySource.java b/flume-ng-core/src/test/java/org/apache/flume/source/TestSpoolDirectorySource.java index 9a546a57b4..503ab4d24c 100644 --- a/flume-ng-core/src/test/java/org/apache/flume/source/TestSpoolDirectorySource.java +++ b/flume-ng-core/src/test/java/org/apache/flume/source/TestSpoolDirectorySource.java @@ -93,7 +93,9 @@ public void testPutFilenameHeader() throws IOException, InterruptedException { Configurables.configure(source, context); source.start(); - Thread.sleep(500); + while (source.getSourceCounter().getEventAcceptedCount() < 8) { + Thread.sleep(10); + } Transaction txn = channel.getTransaction(); txn.begin(); Event e = channel.take(); @@ -106,6 +108,40 @@ public void testPutFilenameHeader() throws IOException, InterruptedException { txn.close(); } + @Test + public void testPutBasenameHeader() throws IOException, + InterruptedException { + Context context = new Context(); + File f1 = new File(tmpDir.getAbsolutePath() + "/file1"); + + Files.write("file1line1\nfile1line2\nfile1line3\nfile1line4\n" + + "file1line5\nfile1line6\nfile1line7\nfile1line8\n", + f1, Charsets.UTF_8); + + context.put(SpoolDirectorySourceConfigurationConstants.SPOOL_DIRECTORY, + tmpDir.getAbsolutePath()); + context.put(SpoolDirectorySourceConfigurationConstants.BASENAME_HEADER, + "true"); + context.put(SpoolDirectorySourceConfigurationConstants.BASENAME_HEADER_KEY, + "basenameHeaderKeyTest"); + + Configurables.configure(source, context); + source.start(); + while (source.getSourceCounter().getEventAcceptedCount() < 8) { + Thread.sleep(10); + } + Transaction txn = channel.getTransaction(); + txn.begin(); + Event e = channel.take(); + Assert.assertNotNull("Event must not be null", e); + Assert.assertNotNull("Event headers must not be null", e.getHeaders()); + Assert.assertNotNull(e.getHeaders().get("basenameHeaderKeyTest")); + Assert.assertEquals(f1.getName(), + e.getHeaders().get("basenameHeaderKeyTest")); + txn.commit(); + txn.close(); + } + @Test public void testLifecycle() throws IOException, InterruptedException { Context context = new Context(); diff --git a/flume-ng-doc/sphinx/FlumeUserGuide.rst b/flume-ng-doc/sphinx/FlumeUserGuide.rst index 7a41efb5c4..08c77400f9 100644 --- a/flume-ng-doc/sphinx/FlumeUserGuide.rst +++ b/flume-ng-doc/sphinx/FlumeUserGuide.rst @@ -946,8 +946,10 @@ Property Name Default Description **spoolDir** -- The directory from which to read files from. fileSuffix .COMPLETED Suffix to append to completely ingested files deletePolicy never When to delete completed files: ``never`` or ``immediate`` -fileHeader false Whether to add a header storing the filename -fileHeaderKey file Header key to use when appending filename to header +fileHeader false Whether to add a header storing the absolute path filename. +fileHeaderKey file Header key to use when appending absolute path filename to event header. +basenameHeader false Whether to add a header storing the basename of the file. +basenameHeaderKey basename Header Key to use when appending basename of file to event header. ignorePattern ^$ Regular expression specifying which files to ignore (skip) trackerDir .flumespool Directory to store metadata related to processing of files. If this path is not an absolute path, then it is interpreted as relative to the spoolDir. From c10c7beced9cb39a675e82f6d86e8cd5b84485c9 Mon Sep 17 00:00:00 2001 From: Hari Shreedharan Date: Tue, 17 Dec 2013 14:57:56 -0800 Subject: [PATCH 055/341] FLUME-2270. Fix Twitter Source documentation. (Jeff Lord via Hari Shreedharan) --- flume-ng-doc/sphinx/FlumeUserGuide.rst | 6 +++--- pom.xml | 2 +- 2 files changed, 4 insertions(+), 4 deletions(-) diff --git a/flume-ng-doc/sphinx/FlumeUserGuide.rst b/flume-ng-doc/sphinx/FlumeUserGuide.rst index 08c77400f9..3088400ba0 100644 --- a/flume-ng-doc/sphinx/FlumeUserGuide.rst +++ b/flume-ng-doc/sphinx/FlumeUserGuide.rst @@ -998,9 +998,9 @@ sends Avro events to a downstream Flume sink. Requires the consumer and access tokens and secrets of a Twitter developer account. Required properties are in **bold**. -================== =========== =================================================== +====================== =========== =================================================== Property Name Default Description -================== =========== =================================================== +====================== =========== =================================================== **channels** -- **type** -- The component type name, needs to be ``org.apache.flume.source.twitter.TwitterSource`` **consumerKey** -- OAuth consumer key @@ -1009,7 +1009,7 @@ Property Name Default Description **accessTokenSecret** -- OAuth toekn secret maxBatchSize 1000 Maximum number of twitter messages to put in a single batch maxBatchDurationMillis 1000 Maximum number of milliseconds to wait before closing a batch -================== =========== =================================================== +====================== =========== =================================================== Example for agent named a1: diff --git a/pom.xml b/pom.xml index 77b7f226e7..3c741c3126 100644 --- a/pom.xml +++ b/pom.xml @@ -599,7 +599,7 @@ limitations under the License. org.apache.maven.plugins maven-site-plugin - 3.1 + 3.3 ${project.basedir}/flume-ng-doc From 8acd54bb1089cc7b620dfd0c9ef9c828de1cc89c Mon Sep 17 00:00:00 2001 From: Roshan Naik Date: Fri, 20 Dec 2013 01:44:21 -0800 Subject: [PATCH 056/341] FLUME-2278. Incorrect documentation for write-timeout of File Channel (Satoshi Iijima via Roshan Naik) --- flume-ng-doc/sphinx/FlumeUserGuide.rst | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/flume-ng-doc/sphinx/FlumeUserGuide.rst b/flume-ng-doc/sphinx/FlumeUserGuide.rst index 3088400ba0..a2790d97cc 100644 --- a/flume-ng-doc/sphinx/FlumeUserGuide.rst +++ b/flume-ng-doc/sphinx/FlumeUserGuide.rst @@ -2147,7 +2147,7 @@ maxFileSize 2146435071 minimumRequiredSpace 524288000 Minimum Required free space (in bytes). To avoid data corruption, File Channel stops accepting take/put requests when free space drops below this value capacity 1000000 Maximum capacity of the channel keep-alive 3 Amount of time (in sec) to wait for a put operation -write-timeout 3 Amount of time (in sec) to wait for a write operation +write-timeout 10 Amount of time (in sec) to wait for a write operation checkpoint-timeout 600 Expert: Amount of time (in sec) to wait for a checkpoint use-log-replay-v1 false Expert: Use old replay logic use-fast-replay false Expert: Replay without using queue From 3b1034e8229eb9ad3e27ed0faab77c3f68f708c6 Mon Sep 17 00:00:00 2001 From: Jarek Jarcec Cecho Date: Mon, 23 Dec 2013 04:41:29 -0800 Subject: [PATCH 057/341] FLUME-2265. Closed bucket writers should be removed from sfwriters map (Hari Shreedharan via Jarek Jarcec Cecho) --- .../sink/hdfs/BucketClosedException.java | 30 +++ .../apache/flume/sink/hdfs/BucketWriter.java | 73 ++++--- .../apache/flume/sink/hdfs/HDFSEventSink.java | 74 +++++-- ...dSeqWriter.java => HDFSTestSeqWriter.java} | 10 +- ...actory.java => HDFSTestWriterFactory.java} | 16 +- .../flume/sink/hdfs/TestBucketWriter.java | 47 ++++- .../flume/sink/hdfs/TestHDFSEventSink.java | 195 ++++++++++++++++-- 7 files changed, 372 insertions(+), 73 deletions(-) create mode 100644 flume-ng-sinks/flume-hdfs-sink/src/main/java/org/apache/flume/sink/hdfs/BucketClosedException.java rename flume-ng-sinks/flume-hdfs-sink/src/test/java/org/apache/flume/sink/hdfs/{HDFSBadSeqWriter.java => HDFSTestSeqWriter.java} (90%) rename flume-ng-sinks/flume-hdfs-sink/src/test/java/org/apache/flume/sink/hdfs/{HDFSBadWriterFactory.java => HDFSTestWriterFactory.java} (75%) diff --git a/flume-ng-sinks/flume-hdfs-sink/src/main/java/org/apache/flume/sink/hdfs/BucketClosedException.java b/flume-ng-sinks/flume-hdfs-sink/src/main/java/org/apache/flume/sink/hdfs/BucketClosedException.java new file mode 100644 index 0000000000..1aca58f630 --- /dev/null +++ b/flume-ng-sinks/flume-hdfs-sink/src/main/java/org/apache/flume/sink/hdfs/BucketClosedException.java @@ -0,0 +1,30 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, + * software distributed under the License is distributed on an + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + * KIND, either express or implied. See the License for the + * specific language governing permissions and limitations + * under the License. + */ +package org.apache.flume.sink.hdfs; + +import org.apache.flume.FlumeException; + +public class BucketClosedException extends FlumeException{ + + private static final long serialVersionUID = -4216667125119540357L; + + public BucketClosedException(String msg) { + super(msg); + } +} diff --git a/flume-ng-sinks/flume-hdfs-sink/src/main/java/org/apache/flume/sink/hdfs/BucketWriter.java b/flume-ng-sinks/flume-hdfs-sink/src/main/java/org/apache/flume/sink/hdfs/BucketWriter.java index 200d457229..62e47de891 100644 --- a/flume-ng-sinks/flume-hdfs-sink/src/main/java/org/apache/flume/sink/hdfs/BucketWriter.java +++ b/flume-ng-sinks/flume-hdfs-sink/src/main/java/org/apache/flume/sink/hdfs/BucketWriter.java @@ -31,6 +31,7 @@ import java.util.concurrent.TimeoutException; import java.util.concurrent.atomic.AtomicLong; +import com.google.common.annotations.VisibleForTesting; import org.apache.flume.Clock; import org.apache.flume.Context; import org.apache.flume.Event; @@ -95,8 +96,8 @@ class BucketWriter { private SinkCounter sinkCounter; private final int idleTimeout; private volatile ScheduledFuture idleFuture; - private final WriterCallback onIdleCallback; - private final String onIdleCallbackPath; + private final WriterCallback onCloseCallback; + private final String onCloseCallbackPath; private final long callTimeout; private final ExecutorService callTimeoutPool; private final int maxConsecUnderReplRotations = 30; // make this config'able? @@ -105,15 +106,15 @@ class BucketWriter { // flag that the bucket writer was closed due to idling and thus shouldn't be // reopened. Not ideal, but avoids internals of owners - protected boolean idleClosed = false; + protected boolean closed = false; BucketWriter(long rollInterval, long rollSize, long rollCount, long batchSize, Context context, String filePath, String fileName, String inUsePrefix, String inUseSuffix, String fileSuffix, CompressionCodec codeC, CompressionType compType, HDFSWriter writer, ScheduledExecutorService timedRollerPool, UserGroupInformation user, - SinkCounter sinkCounter, int idleTimeout, WriterCallback onIdleCallback, - String onIdleCallbackPath, long callTimeout, + SinkCounter sinkCounter, int idleTimeout, WriterCallback onCloseCallback, + String onCloseCallbackPath, long callTimeout, ExecutorService callTimeoutPool) { this.rollInterval = rollInterval; this.rollSize = rollSize; @@ -131,8 +132,8 @@ class BucketWriter { this.user = user; this.sinkCounter = sinkCounter; this.idleTimeout = idleTimeout; - this.onIdleCallback = onIdleCallback; - this.onIdleCallbackPath = onIdleCallbackPath; + this.onCloseCallback = onCloseCallback; + this.onCloseCallbackPath = onCloseCallbackPath; this.callTimeout = callTimeout; this.callTimeoutPool = callTimeoutPool; fileExtensionCounter = new AtomicLong(clock.currentTimeMillis()); @@ -252,7 +253,8 @@ public Void call() throws Exception { LOG.debug("Rolling file ({}): Roll scheduled after {} sec elapsed.", bucketPath, rollInterval); try { - close(); + // Roll the file and remove reference from sfWriters map. + close(true); } catch(Throwable t) { LOG.error("Unexpected error", t); } @@ -268,11 +270,24 @@ public Void call() throws Exception { /** * Close the file handle and rename the temp file to the permanent filename. - * Safe to call multiple times. Logs HDFSWriter.close() exceptions. + * Safe to call multiple times. Logs HDFSWriter.close() exceptions. This + * method will not cause the bucket writer to be dereferenced from the HDFS + * sink that owns it. This method should be used only when size or count + * based rolling closes this file. * @throws IOException On failure to rename if temp file exists. * @throws InterruptedException */ public synchronized void close() throws IOException, InterruptedException { + close(false); + } + /** + * Close the file handle and rename the temp file to the permanent filename. + * Safe to call multiple times. Logs HDFSWriter.close() exceptions. + * @throws IOException On failure to rename if temp file exists. + * @throws InterruptedException + */ + public synchronized void close(boolean callCloseCallback) + throws IOException, InterruptedException { checkAndThrowInterruptedException(); flush(); LOG.debug("Closing {}", bucketPath); @@ -306,6 +321,10 @@ public Void call() throws Exception { renameBucket(); // could block or throw IOException fileSystem = null; } + if (callCloseCallback) { + runCloseAction(); + } + closed = true; } /** @@ -324,16 +343,10 @@ public synchronized void flush() throws IOException, InterruptedException { if(idleFuture == null || idleFuture.cancel(false)) { Callable idleAction = new Callable() { public Void call() throws Exception { - try { - if(isOpen) { - LOG.info("Closing idle bucketWriter {}", bucketPath); - idleClosed = true; - close(); - } - if(onIdleCallback != null) - onIdleCallback.run(onIdleCallbackPath); - } catch(Throwable t) { - LOG.error("Unexpected error", t); + LOG.info("Closing idle bucketWriter {} at {}", bucketPath, + System.currentTimeMillis()); + if (isOpen) { + close(true); } return null; } @@ -345,6 +358,16 @@ public Void call() throws Exception { } } + private void runCloseAction() { + try { + if(onCloseCallback != null) { + onCloseCallback.run(onCloseCallbackPath); + } + } catch(Throwable t) { + LOG.error("Unexpected error", t); + } + } + /** * doFlush() must only be called by flush() * @throws IOException @@ -396,10 +419,14 @@ public synchronized void append(final Event event) } idleFuture = null; } + + // If the bucket writer was closed due to roll timeout or idle timeout, + // force a new bucket writer to be created. Roll count and roll size will + // just reuse this one if (!isOpen) { - if(idleClosed) { - throw new IOException("This bucket writer was closed due to idling and this handle " + - "is thus no longer valid"); + if (closed) { + throw new BucketClosedException("This bucket writer was closed and " + + "this handle is thus no longer valid"); } open(); } @@ -446,7 +473,7 @@ public Void call() throws Exception { bucketPath + ") and rethrowing exception.", e.getMessage()); try { - close(); + close(true); } catch (IOException e2) { LOG.warn("Caught IOException while closing file (" + bucketPath + "). Exception follows.", e2); diff --git a/flume-ng-sinks/flume-hdfs-sink/src/main/java/org/apache/flume/sink/hdfs/HDFSEventSink.java b/flume-ng-sinks/flume-hdfs-sink/src/main/java/org/apache/flume/sink/hdfs/HDFSEventSink.java index f0a6e4bf1e..4ea78c1d4b 100644 --- a/flume-ng-sinks/flume-hdfs-sink/src/main/java/org/apache/flume/sink/hdfs/HDFSEventSink.java +++ b/flume-ng-sinks/flume-hdfs-sink/src/main/java/org/apache/flume/sink/hdfs/HDFSEventSink.java @@ -24,6 +24,7 @@ import java.util.Calendar; import java.util.LinkedHashMap; import java.util.List; +import java.util.Map; import java.util.TimeZone; import java.util.Map.Entry; import java.util.concurrent.ExecutorService; @@ -139,6 +140,7 @@ public interface WriterCallback { private volatile int idleTimeout; private Clock clock; + private final Object sfWritersLock = new Object(); /* * Extended Java LinkedHashMap for open file handle LRU queue. @@ -182,6 +184,11 @@ public HDFSEventSink(HDFSWriterFactory writerFactory) { this.writerFactory = writerFactory; } + @VisibleForTesting + Map getSfWriters() { + return sfWriters; + } + // read configuration and setup thresholds @Override public void configure(Context context) { @@ -359,28 +366,29 @@ public Status process() throws EventDeliveryException { timeZone, needRounding, roundUnit, roundValue, useLocalTime); String lookupPath = realPath + DIRECTORY_DELIMITER + realName; - BucketWriter bucketWriter = sfWriters.get(lookupPath); - - // we haven't seen this file yet, so open it and cache the handle - if (bucketWriter == null) { - HDFSWriter hdfsWriter = writerFactory.getWriter(fileType); - - WriterCallback idleCallback = null; - if(idleTimeout != 0) { - idleCallback = new WriterCallback() { - @Override - public void run(String bucketPath) { - sfWriters.remove(bucketPath); - } - }; + BucketWriter bucketWriter; + HDFSWriter hdfsWriter = null; + // Callback to remove the reference to the bucket writer from the + // sfWriters map so that all buffers used by the HDFS file + // handles are garbage collected. + WriterCallback closeCallback = new WriterCallback() { + @Override + public void run(String bucketPath) { + LOG.info("Writer callback called."); + synchronized (sfWritersLock) { + sfWriters.remove(bucketPath); + } + } + }; + synchronized (sfWritersLock) { + bucketWriter = sfWriters.get(lookupPath); + // we haven't seen this file yet, so open it and cache the handle + if (bucketWriter == null) { + hdfsWriter = writerFactory.getWriter(fileType); + bucketWriter = initializeBucketWriter(realPath, realName, + lookupPath, hdfsWriter, closeCallback); + sfWriters.put(lookupPath, bucketWriter); } - bucketWriter = new BucketWriter(rollInterval, rollSize, rollCount, - batchSize, context, realPath, realName, inUsePrefix, inUseSuffix, - suffix, codeC, compType, hdfsWriter, timedRollerPool, - proxyTicket, sinkCounter, idleTimeout, idleCallback, - lookupPath, callTimeout, callTimeoutPool); - - sfWriters.put(lookupPath, bucketWriter); } // track the buckets getting written in this transaction @@ -389,7 +397,19 @@ public void run(String bucketPath) { } // Write the data to HDFS - bucketWriter.append(event); + try { + bucketWriter.append(event); + } catch (BucketClosedException ex) { + LOG.info("Bucket was closed while trying to append, " + + "reinitializing bucket and writing event."); + hdfsWriter = writerFactory.getWriter(fileType); + bucketWriter = initializeBucketWriter(realPath, realName, + lookupPath, hdfsWriter, closeCallback); + synchronized (sfWritersLock) { + sfWriters.put(lookupPath, bucketWriter); + } + bucketWriter.append(event); + } } if (txnEventCount == 0) { @@ -430,6 +450,16 @@ public void run(String bucketPath) { } } + private BucketWriter initializeBucketWriter(String realPath, + String realName, String lookupPath, HDFSWriter hdfsWriter, + WriterCallback closeCallback) { + return new BucketWriter(rollInterval, rollSize, rollCount, + batchSize, context, realPath, realName, inUsePrefix, inUseSuffix, + suffix, codeC, compType, hdfsWriter, timedRollerPool, + proxyTicket, sinkCounter, idleTimeout, closeCallback, + lookupPath, callTimeout, callTimeoutPool); + } + @Override public void stop() { // do not constrain close() calls with a timeout diff --git a/flume-ng-sinks/flume-hdfs-sink/src/test/java/org/apache/flume/sink/hdfs/HDFSBadSeqWriter.java b/flume-ng-sinks/flume-hdfs-sink/src/test/java/org/apache/flume/sink/hdfs/HDFSTestSeqWriter.java similarity index 90% rename from flume-ng-sinks/flume-hdfs-sink/src/test/java/org/apache/flume/sink/hdfs/HDFSBadSeqWriter.java rename to flume-ng-sinks/flume-hdfs-sink/src/test/java/org/apache/flume/sink/hdfs/HDFSTestSeqWriter.java index 63ab5af5eb..9c1cd0998d 100644 --- a/flume-ng-sinks/flume-hdfs-sink/src/test/java/org/apache/flume/sink/hdfs/HDFSBadSeqWriter.java +++ b/flume-ng-sinks/flume-hdfs-sink/src/test/java/org/apache/flume/sink/hdfs/HDFSTestSeqWriter.java @@ -25,9 +25,14 @@ import org.apache.hadoop.io.SequenceFile.CompressionType; import org.apache.hadoop.io.compress.CompressionCodec; -public class HDFSBadSeqWriter extends HDFSSequenceFile { +public class HDFSTestSeqWriter extends HDFSSequenceFile { protected volatile boolean closed, opened; + private int openCount = 0; + HDFSTestSeqWriter(int openCount) { + this.openCount = openCount; + } + @Override public void open(String filePath, CompressionCodec codeC, CompressionType compType) throws IOException { @@ -46,7 +51,8 @@ public void append(Event e) throws IOException { e.getHeaders().remove("fault-once"); throw new IOException("Injected fault"); } else if (e.getHeaders().containsKey("fault-until-reopen")) { - if(!(closed && opened)) { + // opening first time. + if(openCount == 1) { throw new IOException("Injected fault-until-reopen"); } } else if (e.getHeaders().containsKey("slow")) { diff --git a/flume-ng-sinks/flume-hdfs-sink/src/test/java/org/apache/flume/sink/hdfs/HDFSBadWriterFactory.java b/flume-ng-sinks/flume-hdfs-sink/src/test/java/org/apache/flume/sink/hdfs/HDFSTestWriterFactory.java similarity index 75% rename from flume-ng-sinks/flume-hdfs-sink/src/test/java/org/apache/flume/sink/hdfs/HDFSBadWriterFactory.java rename to flume-ng-sinks/flume-hdfs-sink/src/test/java/org/apache/flume/sink/hdfs/HDFSTestWriterFactory.java index f5d0808610..70bd9e643c 100644 --- a/flume-ng-sinks/flume-hdfs-sink/src/test/java/org/apache/flume/sink/hdfs/HDFSBadWriterFactory.java +++ b/flume-ng-sinks/flume-hdfs-sink/src/test/java/org/apache/flume/sink/hdfs/HDFSTestWriterFactory.java @@ -20,19 +20,19 @@ package org.apache.flume.sink.hdfs; import java.io.IOException; +import java.util.concurrent.atomic.AtomicInteger; -import org.apache.flume.sink.hdfs.HDFSBadSeqWriter; -import org.apache.flume.sink.hdfs.HDFSBadDataStream; - -public class HDFSBadWriterFactory extends HDFSWriterFactory { - static final String BadSequenceFileType = "SequenceFile"; +public class HDFSTestWriterFactory extends HDFSWriterFactory { + static final String TestSequenceFileType = "SequenceFile"; static final String BadDataStreamType = "DataStream"; - static final String BadCompStreamType = "CompressedStream"; + + // so we can get a handle to this one in our test. + AtomicInteger openCount = new AtomicInteger(0); @Override public HDFSWriter getWriter(String fileType) throws IOException { - if (fileType == BadSequenceFileType) { - return new HDFSBadSeqWriter(); + if (fileType == TestSequenceFileType) { + return new HDFSTestSeqWriter(openCount.incrementAndGet()); } else if (fileType == BadDataStreamType) { return new HDFSBadDataStream(); } else { diff --git a/flume-ng-sinks/flume-hdfs-sink/src/test/java/org/apache/flume/sink/hdfs/TestBucketWriter.java b/flume-ng-sinks/flume-hdfs-sink/src/test/java/org/apache/flume/sink/hdfs/TestBucketWriter.java index f741e03a3e..b7cc586cd8 100644 --- a/flume-ng-sinks/flume-hdfs-sink/src/test/java/org/apache/flume/sink/hdfs/TestBucketWriter.java +++ b/flume-ng-sinks/flume-hdfs-sink/src/test/java/org/apache/flume/sink/hdfs/TestBucketWriter.java @@ -23,6 +23,7 @@ import java.util.concurrent.Executors; import java.util.concurrent.ScheduledExecutorService; import java.util.concurrent.TimeUnit; +import java.util.concurrent.atomic.AtomicBoolean; import org.apache.flume.Clock; import org.apache.flume.Context; @@ -113,13 +114,19 @@ public void testSizeRoller() throws IOException, InterruptedException { public void testIntervalRoller() throws IOException, InterruptedException { final int ROLL_INTERVAL = 1; // seconds final int NUM_EVENTS = 10; + final AtomicBoolean calledBack = new AtomicBoolean(false); MockHDFSWriter hdfsWriter = new MockHDFSWriter(); BucketWriter bucketWriter = new BucketWriter(ROLL_INTERVAL, 0, 0, 0, ctx, - "/tmp", "file", "", ".tmp", null, null, SequenceFile.CompressionType.NONE, - hdfsWriter, timedRollerPool, null, - new SinkCounter("test-bucket-writer-" + System.currentTimeMillis()), - 0, null, null, 30000, Executors.newSingleThreadExecutor()); + "/tmp", "file", "", ".tmp", null, null, SequenceFile.CompressionType.NONE, + hdfsWriter, timedRollerPool, null, + new SinkCounter("test-bucket-writer-" + System.currentTimeMillis()), + 0, new HDFSEventSink.WriterCallback() { + @Override + public void run(String filePath) { + calledBack.set(true); + } + }, null, 30000, Executors.newSingleThreadExecutor()); Event e = EventBuilder.withBody("foo", Charsets.UTF_8); long startNanos = System.nanoTime(); @@ -130,6 +137,14 @@ public void testIntervalRoller() throws IOException, InterruptedException { // sleep to force a roll... wait 2x interval just to be sure Thread.sleep(2 * ROLL_INTERVAL * 1000L); + Assert.assertTrue(bucketWriter.closed); + Assert.assertTrue(calledBack.get()); + + bucketWriter = new BucketWriter(ROLL_INTERVAL, 0, 0, 0, ctx, + "/tmp", "file", "", ".tmp", null, null, SequenceFile.CompressionType.NONE, + hdfsWriter, timedRollerPool, null, + new SinkCounter("test-bucket-writer-" + System.currentTimeMillis()), + 0, null, null, 30000, Executors.newSingleThreadExecutor()); // write one more event (to reopen a new file so we will roll again later) bucketWriter.append(e); @@ -348,4 +363,28 @@ public void testInUseSuffix() throws IOException, InterruptedException { Assert.assertTrue("Incorrect in use suffix", hdfsWriter.getOpenedFilePath().contains(SUFFIX)); } + @Test + public void testCallbackOnClose() throws IOException, InterruptedException { + final int ROLL_INTERVAL = 1000; // seconds. Make sure it doesn't change in course of test + final String SUFFIX = "WELCOME_TO_THE_EREBOR"; + final AtomicBoolean callbackCalled = new AtomicBoolean(false); + + MockHDFSWriter hdfsWriter = new MockHDFSWriter(); + BucketWriter bucketWriter = new BucketWriter(ROLL_INTERVAL, 0, 0, 0, ctx, + "/tmp", "file", "", SUFFIX, null, null, SequenceFile.CompressionType.NONE, + hdfsWriter, timedRollerPool, null, + new SinkCounter("test-bucket-writer-" + System.currentTimeMillis()), 0, + new HDFSEventSink.WriterCallback() { + @Override + public void run(String filePath) { + callbackCalled.set(true); + } + }, "blah", 30000, Executors.newSingleThreadExecutor()); + + Event e = EventBuilder.withBody("foo", Charsets.UTF_8); + bucketWriter.append(e); + bucketWriter.close(true); + + Assert.assertTrue(callbackCalled.get()); + } } diff --git a/flume-ng-sinks/flume-hdfs-sink/src/test/java/org/apache/flume/sink/hdfs/TestHDFSEventSink.java b/flume-ng-sinks/flume-hdfs-sink/src/test/java/org/apache/flume/sink/hdfs/TestHDFSEventSink.java index 5b7cec94c1..4337ef4db7 100644 --- a/flume-ng-sinks/flume-hdfs-sink/src/test/java/org/apache/flume/sink/hdfs/TestHDFSEventSink.java +++ b/flume-ng-sinks/flume-hdfs-sink/src/test/java/org/apache/flume/sink/hdfs/TestHDFSEventSink.java @@ -23,7 +23,6 @@ import java.io.InputStreamReader; import java.nio.ByteBuffer; import java.nio.charset.CharsetDecoder; -import java.util.Arrays; import java.util.Calendar; import java.util.List; import java.util.UUID; @@ -673,7 +672,7 @@ public void testBadSimpleAppend() throws InterruptedException, int totalEvents = 0; int i = 1, j = 1; - HDFSBadWriterFactory badWriterFactory = new HDFSBadWriterFactory(); + HDFSTestWriterFactory badWriterFactory = new HDFSTestWriterFactory(); sink = new HDFSEventSink(badWriterFactory); // clear the test directory @@ -689,7 +688,7 @@ public void testBadSimpleAppend() throws InterruptedException, context.put("hdfs.filePrefix", fileName); context.put("hdfs.rollCount", String.valueOf(rollCount)); context.put("hdfs.batchSize", String.valueOf(batchSize)); - context.put("hdfs.fileType", HDFSBadWriterFactory.BadSequenceFileType); + context.put("hdfs.fileType", HDFSTestWriterFactory.TestSequenceFileType); Configurables.configure(sink, context); @@ -840,7 +839,7 @@ private void verifyOutputAvroFiles(FileSystem fs, Configuration conf, String dir * This relies on Transactional rollback semantics for durability and * the behavior of the BucketWriter class of close()ing upon IOException. */ - @Test + @Test public void testCloseReopen() throws InterruptedException, LifecycleException, EventDeliveryException, IOException { @@ -852,7 +851,7 @@ public void testCloseReopen() throws InterruptedException, String newPath = testPath + "/singleBucket"; int i = 1, j = 1; - HDFSBadWriterFactory badWriterFactory = new HDFSBadWriterFactory(); + HDFSTestWriterFactory badWriterFactory = new HDFSTestWriterFactory(); sink = new HDFSEventSink(badWriterFactory); // clear the test directory @@ -868,7 +867,7 @@ public void testCloseReopen() throws InterruptedException, context.put("hdfs.filePrefix", fileName); context.put("hdfs.rollCount", String.valueOf(rollCount)); context.put("hdfs.batchSize", String.valueOf(batchSize)); - context.put("hdfs.fileType", HDFSBadWriterFactory.BadSequenceFileType); + context.put("hdfs.fileType", HDFSTestWriterFactory.TestSequenceFileType); Configurables.configure(sink, context); @@ -910,6 +909,174 @@ public void testCloseReopen() throws InterruptedException, verifyOutputSequenceFiles(fs, conf, dirPath.toUri().getPath(), fileName, bodies); } + /** + * Test that the old bucket writer is closed at the end of rollInterval and + * a new one is used for the next set of events. + */ + @Test + public void testCloseReopenOnRollTime() throws InterruptedException, + LifecycleException, EventDeliveryException, IOException { + + LOG.debug("Starting..."); + final int numBatches = 4; + final String fileName = "FlumeData"; + final long batchSize = 2; + String newPath = testPath + "/singleBucket"; + int i = 1, j = 1; + + HDFSTestWriterFactory badWriterFactory = new HDFSTestWriterFactory(); + sink = new HDFSEventSink(badWriterFactory); + + // clear the test directory + Configuration conf = new Configuration(); + FileSystem fs = FileSystem.get(conf); + Path dirPath = new Path(newPath); + fs.delete(dirPath, true); + fs.mkdirs(dirPath); + + Context context = new Context(); + + context.put("hdfs.path", newPath); + context.put("hdfs.filePrefix", fileName); + context.put("hdfs.rollCount", String.valueOf(0)); + context.put("hdfs.rollSize", String.valueOf(0)); + context.put("hdfs.rollInterval", String.valueOf(2)); + context.put("hdfs.batchSize", String.valueOf(batchSize)); + context.put("hdfs.fileType", HDFSTestWriterFactory.TestSequenceFileType); + + Configurables.configure(sink, context); + + MemoryChannel channel = new MemoryChannel(); + Configurables.configure(channel, new Context()); + + sink.setChannel(channel); + sink.start(); + + Calendar eventDate = Calendar.getInstance(); + List bodies = Lists.newArrayList(); + // push the event batches into channel + for (i = 1; i < numBatches; i++) { + channel.getTransaction().begin(); + try { + for (j = 1; j <= batchSize; j++) { + Event event = new SimpleEvent(); + eventDate.clear(); + eventDate.set(2011, i, i, i, 0); // yy mm dd + event.getHeaders().put("timestamp", + String.valueOf(eventDate.getTimeInMillis())); + event.getHeaders().put("hostname", "Host" + i); + String body = "Test." + i + "." + j; + event.setBody(body.getBytes()); + bodies.add(body); + // inject fault + event.getHeaders().put("count-check", ""); + channel.put(event); + } + channel.getTransaction().commit(); + } finally { + channel.getTransaction().close(); + } + LOG.info("execute sink to process the events: " + sink.process()); + // Make sure the first file gets rolled due to rollTimeout. + if (i == 1) { + Thread.sleep(2001); + } + } + LOG.info("clear any events pending due to errors: " + sink.process()); + sink.stop(); + + Assert.assertTrue(badWriterFactory.openCount.get() >= 2); + LOG.info("Total number of bucket writers opened: {}", + badWriterFactory.openCount.get()); + verifyOutputSequenceFiles(fs, conf, dirPath.toUri().getPath(), fileName, + bodies); + } + + /** + * Test that a close due to roll interval removes the bucketwriter from + * sfWriters map. + */ + @Test + public void testCloseRemovesFromSFWriters() throws InterruptedException, + LifecycleException, EventDeliveryException, IOException { + + LOG.debug("Starting..."); + final String fileName = "FlumeData"; + final long batchSize = 2; + String newPath = testPath + "/singleBucket"; + int i = 1, j = 1; + + HDFSTestWriterFactory badWriterFactory = new HDFSTestWriterFactory(); + sink = new HDFSEventSink(badWriterFactory); + + // clear the test directory + Configuration conf = new Configuration(); + FileSystem fs = FileSystem.get(conf); + Path dirPath = new Path(newPath); + fs.delete(dirPath, true); + fs.mkdirs(dirPath); + + Context context = new Context(); + + context.put("hdfs.path", newPath); + context.put("hdfs.filePrefix", fileName); + context.put("hdfs.rollCount", String.valueOf(0)); + context.put("hdfs.rollSize", String.valueOf(0)); + context.put("hdfs.rollInterval", String.valueOf(1)); + context.put("hdfs.batchSize", String.valueOf(batchSize)); + context.put("hdfs.fileType", HDFSTestWriterFactory.TestSequenceFileType); + String expectedLookupPath = newPath + "/FlumeData"; + + Configurables.configure(sink, context); + + MemoryChannel channel = new MemoryChannel(); + Configurables.configure(channel, new Context()); + + sink.setChannel(channel); + sink.start(); + + Calendar eventDate = Calendar.getInstance(); + List bodies = Lists.newArrayList(); + // push the event batches into channel + channel.getTransaction().begin(); + try { + for (j = 1; j <= 2 * batchSize; j++) { + Event event = new SimpleEvent(); + eventDate.clear(); + eventDate.set(2011, i, i, i, 0); // yy mm dd + event.getHeaders().put("timestamp", + String.valueOf(eventDate.getTimeInMillis())); + event.getHeaders().put("hostname", "Host" + i); + String body = "Test." + i + "." + j; + event.setBody(body.getBytes()); + bodies.add(body); + // inject fault + event.getHeaders().put("count-check", ""); + channel.put(event); + } + channel.getTransaction().commit(); + } finally { + channel.getTransaction().close(); + } + LOG.info("execute sink to process the events: " + sink.process()); + Assert.assertTrue(sink.getSfWriters().containsKey(expectedLookupPath)); + // Make sure the first file gets rolled due to rollTimeout. + Thread.sleep(2001); + Assert.assertFalse(sink.getSfWriters().containsKey(expectedLookupPath)); + LOG.info("execute sink to process the events: " + sink.process()); + // A new bucket writer should have been created for this bucket. So + // sfWriters map should not have the same key again. + Assert.assertTrue(sink.getSfWriters().containsKey(expectedLookupPath)); + sink.stop(); + + LOG.info("Total number of bucket writers opened: {}", + badWriterFactory.openCount.get()); + verifyOutputSequenceFiles(fs, conf, dirPath.toUri().getPath(), fileName, + bodies); + } + + + /* * append using slow sink writer. * verify that the process returns backoff due to timeout @@ -934,7 +1101,7 @@ public void testSlowAppendFailure() throws InterruptedException, fs.mkdirs(dirPath); // create HDFS sink with slow writer - HDFSBadWriterFactory badWriterFactory = new HDFSBadWriterFactory(); + HDFSTestWriterFactory badWriterFactory = new HDFSTestWriterFactory(); sink = new HDFSEventSink(badWriterFactory); Context context = new Context(); @@ -942,7 +1109,7 @@ public void testSlowAppendFailure() throws InterruptedException, context.put("hdfs.filePrefix", fileName); context.put("hdfs.rollCount", String.valueOf(rollCount)); context.put("hdfs.batchSize", String.valueOf(batchSize)); - context.put("hdfs.fileType", HDFSBadWriterFactory.BadSequenceFileType); + context.put("hdfs.fileType", HDFSTestWriterFactory.TestSequenceFileType); context.put("hdfs.callTimeout", Long.toString(1000)); Configurables.configure(sink, context); @@ -1004,7 +1171,7 @@ private void slowAppendTestHelper (long appendTimeout) throws InterruptedExcept fs.mkdirs(dirPath); // create HDFS sink with slow writer - HDFSBadWriterFactory badWriterFactory = new HDFSBadWriterFactory(); + HDFSTestWriterFactory badWriterFactory = new HDFSTestWriterFactory(); sink = new HDFSEventSink(badWriterFactory); Context context = new Context(); @@ -1012,7 +1179,7 @@ private void slowAppendTestHelper (long appendTimeout) throws InterruptedExcept context.put("hdfs.filePrefix", fileName); context.put("hdfs.rollCount", String.valueOf(rollCount)); context.put("hdfs.batchSize", String.valueOf(batchSize)); - context.put("hdfs.fileType", HDFSBadWriterFactory.BadSequenceFileType); + context.put("hdfs.fileType", HDFSTestWriterFactory.TestSequenceFileType); context.put("hdfs.appendTimeout", String.valueOf(appendTimeout)); Configurables.configure(sink, context); @@ -1127,10 +1294,10 @@ public void testCloseOnIdle() throws IOException, EventDeliveryException, Interr sink.process(); Thread.sleep(1001); // previous file should have timed out now - // this can throw an IOException(from the bucketWriter having idleClosed) - // this is not an issue as the sink will retry and get a fresh bucketWriter - // so long as the onIdleClose handler properly removes bucket writers that - // were closed due to idling + // this can throw BucketClosedException(from the bucketWriter having + // closed),this is not an issue as the sink will retry and get a fresh + // bucketWriter so long as the onClose handler properly removes + // bucket writers that were closed. sink.process(); sink.process(); Thread.sleep(500); // shouldn't be enough for a timeout to occur From c3a9c80ab431f6ba670142c7ce6813692422f764 Mon Sep 17 00:00:00 2001 From: Hari Shreedharan Date: Thu, 2 Jan 2014 18:22:31 -0800 Subject: [PATCH 058/341] FLUME-2275. Improve scalability of MorphlineInterceptor under contention (Wolfgang Hoschek via Hari Shreedharan) --- .../solr/morphline/MorphlineInterceptor.java | 19 +++++++------------ 1 file changed, 7 insertions(+), 12 deletions(-) diff --git a/flume-ng-sinks/flume-ng-morphline-solr-sink/src/main/java/org/apache/flume/sink/solr/morphline/MorphlineInterceptor.java b/flume-ng-sinks/flume-ng-morphline-solr-sink/src/main/java/org/apache/flume/sink/solr/morphline/MorphlineInterceptor.java index 8e5e4b3294..ef8f716517 100644 --- a/flume-ng-sinks/flume-ng-morphline-solr-sink/src/main/java/org/apache/flume/sink/solr/morphline/MorphlineInterceptor.java +++ b/flume-ng-sinks/flume-ng-morphline-solr-sink/src/main/java/org/apache/flume/sink/solr/morphline/MorphlineInterceptor.java @@ -23,18 +23,18 @@ import java.util.HashMap; import java.util.List; import java.util.Map; -import java.util.concurrent.BlockingQueue; -import java.util.concurrent.LinkedBlockingQueue; +import java.util.Queue; +import java.util.concurrent.ConcurrentLinkedQueue; import org.apache.flume.Context; import org.apache.flume.Event; import org.apache.flume.FlumeException; import org.apache.flume.event.EventBuilder; import org.apache.flume.interceptor.Interceptor; - import org.kitesdk.morphline.api.Command; import org.kitesdk.morphline.api.Record; import org.kitesdk.morphline.base.Fields; + import com.google.common.base.Preconditions; import com.google.common.io.ByteStreams; @@ -47,7 +47,7 @@ public class MorphlineInterceptor implements Interceptor { private final Context context; - private final BlockingQueue pool = new LinkedBlockingQueue(); + private final Queue pool = new ConcurrentLinkedQueue(); protected MorphlineInterceptor(Context context) { Preconditions.checkNotNull(context); @@ -61,9 +61,8 @@ public void initialize() { @Override public void close() { - List interceptors = new ArrayList(); - pool.drainTo(interceptors); - for (LocalMorphlineInterceptor interceptor : interceptors) { + LocalMorphlineInterceptor interceptor; + while ((interceptor = pool.poll()) != null) { interceptor.close(); } } @@ -85,11 +84,7 @@ public Event intercept(Event event) { } private void returnToPool(LocalMorphlineInterceptor interceptor) { - try { - pool.put(interceptor); - } catch (InterruptedException e) { - throw new FlumeException(e); - } + pool.add(interceptor); } private LocalMorphlineInterceptor borrowFromPool() { From 56027c2264fe1e2f4158f89c9f25f9a9521aa254 Mon Sep 17 00:00:00 2001 From: Jarek Jarcec Cecho Date: Tue, 7 Jan 2014 02:48:49 -0800 Subject: [PATCH 059/341] FLUME-2289. Disable maxUnderReplication test which is extremely flakey (Hari Shreedharan via Jarek Jarcec Cecho) --- .../apache/flume/sink/hdfs/TestHDFSEventSinkOnMiniCluster.java | 1 + 1 file changed, 1 insertion(+) diff --git a/flume-ng-sinks/flume-hdfs-sink/src/test/java/org/apache/flume/sink/hdfs/TestHDFSEventSinkOnMiniCluster.java b/flume-ng-sinks/flume-hdfs-sink/src/test/java/org/apache/flume/sink/hdfs/TestHDFSEventSinkOnMiniCluster.java index 6e11624cc1..7c1caaaedd 100644 --- a/flume-ng-sinks/flume-hdfs-sink/src/test/java/org/apache/flume/sink/hdfs/TestHDFSEventSinkOnMiniCluster.java +++ b/flume-ng-sinks/flume-hdfs-sink/src/test/java/org/apache/flume/sink/hdfs/TestHDFSEventSinkOnMiniCluster.java @@ -368,6 +368,7 @@ public void underReplicationTest() throws EventDeliveryException, /** * This is a very basic test that writes one event to HDFS and reads it back. */ + @Ignore("This test is flakey and causes tests to fail pretty often.") @Test public void maxUnderReplicationTest() throws EventDeliveryException, IOException { From bfd8e508a7498cd70a76e2269b610d063d30c270 Mon Sep 17 00:00:00 2001 From: Hari Shreedharan Date: Mon, 13 Jan 2014 23:06:23 -0800 Subject: [PATCH 060/341] FLUME-2292. Upgrade mapdb to 0.9.8 (Ashish Paliwal via Hari Shreedharan) --- .../java/org/apache/flume/channel/file/FlumeEventQueue.java | 1 - pom.xml | 2 +- 2 files changed, 1 insertion(+), 2 deletions(-) diff --git a/flume-ng-channels/flume-file-channel/src/main/java/org/apache/flume/channel/file/FlumeEventQueue.java b/flume-ng-channels/flume-file-channel/src/main/java/org/apache/flume/channel/file/FlumeEventQueue.java index 7888b419a5..83309ac43f 100644 --- a/flume-ng-channels/flume-file-channel/src/main/java/org/apache/flume/channel/file/FlumeEventQueue.java +++ b/flume-ng-channels/flume-file-channel/src/main/java/org/apache/flume/channel/file/FlumeEventQueue.java @@ -105,7 +105,6 @@ final class FlumeEventQueue { .syncOnCommitDisable() .deleteFilesAfterClose() .cacheDisable() - .randomAccessFileEnableIfNeeded() .make(); queueSet = db.createTreeSet("QueueSet").make(); long start = System.currentTimeMillis(); diff --git a/pom.xml b/pom.xml index 3c741c3126..453be9585e 100644 --- a/pom.xml +++ b/pom.xml @@ -777,7 +777,7 @@ limitations under the License. org.mapdb mapdb - 0.9.7 + 0.9.8 From cf2ac371351812f333960604a3a8ad4a510307ae Mon Sep 17 00:00:00 2001 From: Hari Shreedharan Date: Wed, 15 Jan 2014 21:27:16 -0800 Subject: [PATCH 061/341] FLUME-2259. Transaction closure not happening for all the scenario in HBaseSink. (Gopinathan A via Hari Shreedharan) --- .../apache/flume/sink/hbase/HBaseSink.java | 112 +++++++++--------- .../hbase/MockSimpleHbaseEventSerializer.java | 38 ++++++ .../flume/sink/hbase/TestHBaseSink.java | 82 ++++++++++++- 3 files changed, 173 insertions(+), 59 deletions(-) create mode 100644 flume-ng-sinks/flume-ng-hbase-sink/src/test/java/org/apache/flume/sink/hbase/MockSimpleHbaseEventSerializer.java diff --git a/flume-ng-sinks/flume-ng-hbase-sink/src/main/java/org/apache/flume/sink/hbase/HBaseSink.java b/flume-ng-sinks/flume-ng-hbase-sink/src/main/java/org/apache/flume/sink/hbase/HBaseSink.java index d5996c3276..f5cb2297f6 100644 --- a/flume-ng-sinks/flume-ng-hbase-sink/src/main/java/org/apache/flume/sink/hbase/HBaseSink.java +++ b/flume-ng-sinks/flume-ng-hbase-sink/src/main/java/org/apache/flume/sink/hbase/HBaseSink.java @@ -231,66 +231,32 @@ public Status process() throws EventDeliveryException { Transaction txn = channel.getTransaction(); List actions = new LinkedList(); List incs = new LinkedList(); - txn.begin(); - long i = 0; - for(; i < batchSize; i++) { - Event event = channel.take(); - if(event == null){ - status = Status.BACKOFF; - if (i == 0) { - sinkCounter.incrementBatchEmptyCount(); + try { + txn.begin(); + long i = 0; + for (; i < batchSize; i++) { + Event event = channel.take(); + if (event == null) { + if (i == 0) { + status = Status.BACKOFF; + sinkCounter.incrementBatchEmptyCount(); + } else { + sinkCounter.incrementBatchUnderflowCount(); + } + break; } else { - sinkCounter.incrementBatchUnderflowCount(); + serializer.initialize(event, columnFamily); + actions.addAll(serializer.getActions()); + incs.addAll(serializer.getIncrements()); } - break; - } else { - serializer.initialize(event, columnFamily); - actions.addAll(serializer.getActions()); - incs.addAll(serializer.getIncrements()); } - } - if (i == batchSize) { - sinkCounter.incrementBatchCompleteCount(); - } - sinkCounter.addToEventDrainAttemptCount(i); - - putEventsAndCommit(actions, incs, txn); - return status; - } + if (i == batchSize) { + sinkCounter.incrementBatchCompleteCount(); + } + sinkCounter.addToEventDrainAttemptCount(i); - private void putEventsAndCommit(final List actions, final List incs, - Transaction txn) throws EventDeliveryException { - try { - runPrivileged(new PrivilegedExceptionAction() { - @Override - public Void run() throws Exception { - for(Row r : actions) { - if(r instanceof Put) { - ((Put)r).setWriteToWAL(enableWal); - } - // Newer versions of HBase - Increment implements Row. - if(r instanceof Increment) { - ((Increment)r).setWriteToWAL(enableWal); - } - } - table.batch(actions); - return null; - } - }); + putEventsAndCommit(actions, incs, txn); - runPrivileged(new PrivilegedExceptionAction() { - @Override - public Void run() throws Exception { - for (final Increment i : incs) { - i.setWriteToWAL(enableWal); - table.increment(i); - } - return null; - } - }); - - txn.commit(); - sinkCounter.addToEventDrainSuccessCount(actions.size()); } catch (Throwable e) { try{ txn.rollback(); @@ -313,6 +279,42 @@ public Void run() throws Exception { } finally { txn.close(); } + return status; + } + + private void putEventsAndCommit(final List actions, + final List incs, Transaction txn) throws Exception { + + runPrivileged(new PrivilegedExceptionAction() { + @Override + public Void run() throws Exception { + for (Row r : actions) { + if (r instanceof Put) { + ((Put) r).setWriteToWAL(enableWal); + } + // Newer versions of HBase - Increment implements Row. + if (r instanceof Increment) { + ((Increment) r).setWriteToWAL(enableWal); + } + } + table.batch(actions); + return null; + } + }); + + runPrivileged(new PrivilegedExceptionAction() { + @Override + public Void run() throws Exception { + for (final Increment i : incs) { + i.setWriteToWAL(enableWal); + table.increment(i); + } + return null; + } + }); + + txn.commit(); + sinkCounter.addToEventDrainSuccessCount(actions.size()); } private T runPrivileged(final PrivilegedExceptionAction action) throws Exception { diff --git a/flume-ng-sinks/flume-ng-hbase-sink/src/test/java/org/apache/flume/sink/hbase/MockSimpleHbaseEventSerializer.java b/flume-ng-sinks/flume-ng-hbase-sink/src/test/java/org/apache/flume/sink/hbase/MockSimpleHbaseEventSerializer.java new file mode 100644 index 0000000000..9b2a8506c5 --- /dev/null +++ b/flume-ng-sinks/flume-ng-hbase-sink/src/test/java/org/apache/flume/sink/hbase/MockSimpleHbaseEventSerializer.java @@ -0,0 +1,38 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, + * software distributed under the License is distributed on an + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + * KIND, either express or implied. See the License for the + * specific language governing permissions and limitations + * under the License. + */ + +package org.apache.flume.sink.hbase; + +import java.util.List; + +import org.apache.flume.FlumeException; +import org.apache.hadoop.hbase.client.Row; + +class MockSimpleHbaseEventSerializer extends SimpleHbaseEventSerializer { + + public static boolean throwException = false; + + @Override + public List getActions() throws FlumeException { + if (throwException) { + throw new FlumeException("Exception for testing"); + } + return super.getActions(); + } +} \ No newline at end of file diff --git a/flume-ng-sinks/flume-ng-hbase-sink/src/test/java/org/apache/flume/sink/hbase/TestHBaseSink.java b/flume-ng-sinks/flume-ng-hbase-sink/src/test/java/org/apache/flume/sink/hbase/TestHBaseSink.java index ab4128eac6..f41bf53286 100644 --- a/flume-ng-sinks/flume-ng-hbase-sink/src/test/java/org/apache/flume/sink/hbase/TestHBaseSink.java +++ b/flume-ng-sinks/flume-ng-hbase-sink/src/test/java/org/apache/flume/sink/hbase/TestHBaseSink.java @@ -18,14 +18,15 @@ */ package org.apache.flume.sink.hbase; +import static org.mockito.Mockito.*; import java.io.IOException; import java.util.Arrays; import java.util.HashMap; import java.util.Map; - import org.apache.flume.Channel; +import org.apache.flume.ChannelException; import org.apache.flume.Context; import org.apache.flume.Event; import org.apache.flume.EventDeliveryException; @@ -35,14 +36,12 @@ import org.apache.flume.channel.MemoryChannel; import org.apache.flume.conf.Configurables; import org.apache.flume.event.EventBuilder; -import org.apache.flume.sink.hbase.HBaseSink; import org.apache.hadoop.hbase.*; import org.apache.hadoop.hbase.client.HTable; import org.apache.hadoop.hbase.client.Result; import org.apache.hadoop.hbase.client.ResultScanner; import org.apache.hadoop.hbase.client.Scan; import org.apache.hadoop.hbase.util.Bytes; - import org.junit.AfterClass; import org.junit.BeforeClass; import org.junit.Ignore; @@ -60,7 +59,6 @@ public class TestHBaseSink { private static Context ctx = new Context(); private static String valBase = "testing hbase sink: jham"; - @BeforeClass public static void setUp() throws Exception { testUtility.startMiniCluster(); @@ -368,5 +366,81 @@ private byte[][] getResults(HTable table, int numEvents) throws IOException{ } return results; } + + @Test + public void testTransactionStateOnChannelException() throws Exception { + ctx.put("batchSize", "1"); + testUtility.createTable(tableName.getBytes(), columnFamily.getBytes()); + HBaseSink sink = new HBaseSink(testUtility.getConfiguration()); + Configurables.configure(sink, ctx); + // Reset the context to a higher batchSize + Channel channel = spy(new MemoryChannel()); + Configurables.configure(channel, new Context()); + sink.setChannel(channel); + sink.start(); + Transaction tx = channel.getTransaction(); + tx.begin(); + Event e = EventBuilder.withBody(Bytes.toBytes(valBase + "-" + 0)); + channel.put(e); + tx.commit(); + tx.close(); + doThrow(new ChannelException("Mock Exception")).when(channel).take(); + try { + sink.process(); + Assert.fail("take() method should throw exception"); + } catch (ChannelException ex) { + Assert.assertEquals("Mock Exception", ex.getMessage()); + } + doReturn(e).when(channel).take(); + sink.process(); + sink.stop(); + HTable table = new HTable(testUtility.getConfiguration(), tableName); + byte[][] results = getResults(table, 1); + byte[] out = results[0]; + Assert.assertArrayEquals(e.getBody(), out); + out = results[1]; + Assert.assertArrayEquals(Longs.toByteArray(1), out); + testUtility.deleteTable(tableName.getBytes()); + } + + @Test + public void testTransactionStateOnSerializationException() throws Exception { + ctx.put("batchSize", "1"); + ctx.put(HBaseSinkConfigurationConstants.CONFIG_SERIALIZER, + "org.apache.flume.sink.hbase.MockSimpleHbaseEventSerializer"); + testUtility.createTable(tableName.getBytes(), columnFamily.getBytes()); + HBaseSink sink = new HBaseSink(testUtility.getConfiguration()); + Configurables.configure(sink, ctx); + // Reset the context to a higher batchSize + Channel channel = new MemoryChannel(); + Configurables.configure(channel, new Context()); + sink.setChannel(channel); + sink.start(); + Transaction tx = channel.getTransaction(); + tx.begin(); + Event e = EventBuilder.withBody(Bytes.toBytes(valBase + "-" + 0)); + channel.put(e); + tx.commit(); + tx.close(); + try { + MockSimpleHbaseEventSerializer.throwException = true; + sink.process(); + Assert.fail("FlumeException expected from serilazer"); + } catch (FlumeException ex) { + Assert.assertEquals("Exception for testing", ex.getMessage()); + } + MockSimpleHbaseEventSerializer.throwException = false; + sink.process(); + sink.stop(); + HTable table = new HTable(testUtility.getConfiguration(), tableName); + byte[][] results = getResults(table, 1); + byte[] out = results[0]; + Assert.assertArrayEquals(e.getBody(), out); + out = results[1]; + Assert.assertArrayEquals(Longs.toByteArray(1), out); + testUtility.deleteTable(tableName.getBytes()); + } + + } From 9a4f047668eb56895fb4bf5c1ee3e5dd6add8601 Mon Sep 17 00:00:00 2001 From: Jarek Jarcec Cecho Date: Thu, 16 Jan 2014 11:58:39 -0800 Subject: [PATCH 062/341] FLUME-2301. Update HBaseSink tests to reflect sink returning backoff only on empty batches (Hari Shreedharan via Jarek Jarcec Cecho) --- .../test/java/org/apache/flume/sink/hbase/TestHBaseSink.java | 4 +--- 1 file changed, 1 insertion(+), 3 deletions(-) diff --git a/flume-ng-sinks/flume-ng-hbase-sink/src/test/java/org/apache/flume/sink/hbase/TestHBaseSink.java b/flume-ng-sinks/flume-ng-hbase-sink/src/test/java/org/apache/flume/sink/hbase/TestHBaseSink.java index f41bf53286..f2bba314e1 100644 --- a/flume-ng-sinks/flume-ng-hbase-sink/src/test/java/org/apache/flume/sink/hbase/TestHBaseSink.java +++ b/flume-ng-sinks/flume-ng-hbase-sink/src/test/java/org/apache/flume/sink/hbase/TestHBaseSink.java @@ -200,10 +200,8 @@ public void testMultipleBatches() throws Exception { tx.commit(); tx.close(); int count = 0; - Status status = Status.READY; - while(status != Status.BACKOFF){ + while(sink.process() != Status.BACKOFF){ count++; - status = sink.process(); } sink.stop(); Assert.assertEquals(2, count); From 68ba5cf7185f333ad8723c3af5bcefe868c783cd Mon Sep 17 00:00:00 2001 From: Hari Shreedharan Date: Sat, 18 Jan 2014 21:16:12 -0800 Subject: [PATCH 063/341] FLUME-2294. Add a sink for Kite Datasets. (Ryan Blue via Hari Shreedharan) --- flume-ng-dist/pom.xml | 18 + flume-ng-doc/sphinx/FlumeUserGuide.rst | 33 ++ flume-ng-sinks/flume-dataset-sink/pom.xml | 139 +++++ .../apache/flume/sink/kite/DatasetSink.java | 367 ++++++++++++++ .../flume/sink/kite/DatasetSinkConstants.java | 56 +++ .../flume/sink/kite/TestDatasetSink.java | 475 ++++++++++++++++++ flume-ng-sinks/pom.xml | 29 ++ pom.xml | 19 +- 8 files changed, 1134 insertions(+), 2 deletions(-) create mode 100644 flume-ng-sinks/flume-dataset-sink/pom.xml create mode 100644 flume-ng-sinks/flume-dataset-sink/src/main/java/org/apache/flume/sink/kite/DatasetSink.java create mode 100644 flume-ng-sinks/flume-dataset-sink/src/main/java/org/apache/flume/sink/kite/DatasetSinkConstants.java create mode 100644 flume-ng-sinks/flume-dataset-sink/src/test/java/org/apache/flume/sink/kite/TestDatasetSink.java diff --git a/flume-ng-dist/pom.xml b/flume-ng-dist/pom.xml index 2d0ee47718..8b814b7c97 100644 --- a/flume-ng-dist/pom.xml +++ b/flume-ng-dist/pom.xml @@ -63,6 +63,24 @@ + + + hadoop-2 + + + hadoop.profile + 2 + + + + + org.apache.flume.flume-ng-sinks + flume-dataset-sink + + + + + org.apache.flume diff --git a/flume-ng-doc/sphinx/FlumeUserGuide.rst b/flume-ng-doc/sphinx/FlumeUserGuide.rst index a2790d97cc..d120a7445b 100644 --- a/flume-ng-doc/sphinx/FlumeUserGuide.rst +++ b/flume-ng-doc/sphinx/FlumeUserGuide.rst @@ -2018,6 +2018,39 @@ Example for agent named a1: a1.sinks.k1.serializer = org.apache.flume.sink.elasticsearch.ElasticSearchDynamicSerializer a1.sinks.k1.channel = c1 +Kite Dataset Sink (experimental) +~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~ + +.. warning:: + This source is experimental and may change between minor versions of Flume. + Use at your own risk. + +Experimental sink that writes events to a `Kite Dataset `_. +This sink will deserialize the body of each incoming event and store the +resulting record in a Kite Dataset. It determines target Dataset by opening a +repository URI, ``kite.repo.uri``, and loading a Dataset by name, +``kite.dataset.name``. + +The only supported serialization is avro, and the record schema must be passed +in the event headers, using either ``flume.avro.schema.literal`` with the JSON +schema representation or ``flume.avro.schema.url`` with a URL where the schema +may be found (``hdfs:/...`` URIs are supported). This is compatible with the +Log4jAppender flume client and the spooling directory source's Avro +deserializer using ``deserializer.schemaType = LITERAL``. + +Note: The ``flume.avro.schema.hash`` header is **not supported**. + +===================== ======= =========================================================== +Property Name Default Description +===================== ======= =========================================================== +**channel** -- +**type** -- Must be org.apache.flume.sink.kite.DatasetSink +**kite.repo.uri** -- URI of the repository to open +**kite.dataset.name** -- Name of the Dataset where records will be written +kite.batchSize 100 Number of records to process in each batch +kite.rollInterval 30 Maximum wait time (seconds) before data files are released +===================== ======= =========================================================== + Custom Sink ~~~~~~~~~~~ diff --git a/flume-ng-sinks/flume-dataset-sink/pom.xml b/flume-ng-sinks/flume-dataset-sink/pom.xml new file mode 100644 index 0000000000..57fd0e4a67 --- /dev/null +++ b/flume-ng-sinks/flume-dataset-sink/pom.xml @@ -0,0 +1,139 @@ + + + + + 4.0.0 + + + flume-ng-sinks + org.apache.flume + 1.5.0-SNAPSHOT + + + org.apache.flume.flume-ng-sinks + flume-dataset-sink + Flume NG Kite Dataset Sink + + + + + org.apache.rat + apache-rat-plugin + + + + + + + cdh.repo + https://repository.cloudera.com/artifactory/cloudera-repos + Cloudera Repositories + + false + + + + + cdh.snapshots.repo + https://repository.cloudera.com/artifactory/libs-snapshot-local + Cloudera Snapshots Repository + + true + + + false + + + + + + + + org.apache.flume + flume-ng-sdk + + + + org.apache.flume + flume-ng-configuration + + + + org.apache.flume + flume-ng-core + + + + org.kitesdk + kite-data-core + + + + org.apache.avro + avro + + + + + org.apache.hadoop + hadoop-common + ${hadoop2.version} + true + + + + org.slf4j + slf4j-api + + + + com.google.guava + guava + + + + junit + junit + test + + + + org.apache.hadoop + hadoop-minicluster + ${hadoop2.version} + test + + + + org.slf4j + slf4j-log4j12 + test + + + + org.mockito + mockito-all + test + + + + + diff --git a/flume-ng-sinks/flume-dataset-sink/src/main/java/org/apache/flume/sink/kite/DatasetSink.java b/flume-ng-sinks/flume-dataset-sink/src/main/java/org/apache/flume/sink/kite/DatasetSink.java new file mode 100644 index 0000000000..9a00fb1c8c --- /dev/null +++ b/flume-ng-sinks/flume-dataset-sink/src/main/java/org/apache/flume/sink/kite/DatasetSink.java @@ -0,0 +1,367 @@ +/** + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.flume.sink.kite; + +import com.google.common.base.Preconditions; +import com.google.common.base.Throwables; +import com.google.common.cache.CacheBuilder; +import com.google.common.cache.CacheLoader; +import com.google.common.cache.LoadingCache; +import com.google.common.collect.Lists; +import com.google.common.util.concurrent.ThreadFactoryBuilder; +import java.io.IOException; +import java.io.InputStream; +import java.net.URI; +import java.net.URL; +import java.util.List; +import java.util.Map; +import java.util.concurrent.ExecutionException; +import java.util.concurrent.Executors; +import java.util.concurrent.ScheduledExecutorService; +import java.util.concurrent.TimeUnit; +import java.util.concurrent.locks.ReentrantLock; +import org.apache.avro.Schema; +import org.apache.avro.io.BinaryDecoder; +import org.apache.avro.io.DecoderFactory; +import org.apache.avro.reflect.ReflectDatumReader; +import org.apache.flume.Channel; +import org.apache.flume.Context; +import org.apache.flume.Event; +import org.apache.flume.EventDeliveryException; +import org.apache.flume.Transaction; +import org.apache.flume.conf.Configurable; +import org.apache.flume.instrumentation.SinkCounter; +import org.apache.flume.sink.AbstractSink; +import org.apache.hadoop.conf.Configuration; +import org.apache.hadoop.fs.FileSystem; +import org.apache.hadoop.fs.Path; +import org.kitesdk.data.Dataset; +import org.kitesdk.data.DatasetRepositories; +import org.kitesdk.data.DatasetWriter; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +/** + * Experimental sink that writes events to a Kite Dataset. This sink will + * deserialize the body of each incoming event and store the resulting record + * in a Kite Dataset. It determines target Dataset by opening a repository URI, + * {@code kite.repo.uri}, and loading a Dataset by name, + * {@code kite.dataset.name}. + */ +public class DatasetSink extends AbstractSink implements Configurable { + + private static final Logger LOG = LoggerFactory.getLogger(DatasetSink.class); + + static Configuration conf = new Configuration(); + + /** + * Lock used to protect access to the current writer + */ + private final ReentrantLock writerLock = new ReentrantLock(true); + + private String repositoryURI = null; + private String datasetName = null; + private long batchSize = DatasetSinkConstants.DEFAULT_BATCH_SIZE; + private Dataset targetDataset = null; + private DatasetWriter writer = null; + private SinkCounter counter = null; + + // for rolling files at a given interval + private ScheduledExecutorService rollTimer; + private int rollInterval = DatasetSinkConstants.DEFAULT_ROLL_INTERVAL; + + // for working with avro serialized records + private Object datum = null; + private BinaryDecoder decoder = null; + private LoadingCache> readers = + CacheBuilder.newBuilder() + .build(new CacheLoader>() { + @Override + public ReflectDatumReader load(Schema schema) { + // must use the target dataset's schema for reading to ensure the + // records are able to be stored using it + return new ReflectDatumReader( + schema, targetDataset.getDescriptor().getSchema()); + } + }); + private static LoadingCache schemasFromLiteral = CacheBuilder + .newBuilder() + .build(new CacheLoader() { + @Override + public Schema load(String literal) { + Preconditions.checkNotNull(literal, + "Schema literal cannot be null without a Schema URL"); + return new Schema.Parser().parse(literal); + } + }); + private static LoadingCache schemasFromURL = CacheBuilder + .newBuilder() + .build(new CacheLoader() { + @Override + public Schema load(String url) throws IOException { + Schema.Parser parser = new Schema.Parser(); + InputStream is = null; + try { + FileSystem fs = FileSystem.get(URI.create(url), conf); + if (url.toLowerCase().startsWith("hdfs:/")) { + is = fs.open(new Path(url)); + } else { + is = new URL(url).openStream(); + } + return parser.parse(is); + } finally { + if (is != null) { + is.close(); + } + } + } + }); + + protected List allowedFormats() { + return Lists.newArrayList("avro"); + } + + @Override + public void configure(Context context) { + this.repositoryURI = context.getString( + DatasetSinkConstants.CONFIG_KITE_REPO_URI); + Preconditions.checkNotNull(repositoryURI, "Repository URI is missing"); + this.datasetName = context.getString( + DatasetSinkConstants.CONFIG_KITE_DATASET_NAME); + Preconditions.checkNotNull(datasetName, "Dataset name is missing"); + this.targetDataset = DatasetRepositories.open(repositoryURI) + .load(datasetName); + + String formatName = targetDataset.getDescriptor().getFormat().getName(); + Preconditions.checkArgument(allowedFormats().contains(formatName), + "Unsupported format: " + formatName); + + // other configuration + this.batchSize = context.getLong( + DatasetSinkConstants.CONFIG_KITE_BATCH_SIZE, + DatasetSinkConstants.DEFAULT_BATCH_SIZE); + this.rollInterval = context.getInteger( + DatasetSinkConstants.CONFIG_KITE_ROLL_INTERVAL, + DatasetSinkConstants.DEFAULT_ROLL_INTERVAL); + + this.counter = new SinkCounter(getName()); + } + + @Override + public synchronized void start() { + this.writer = openWriter(targetDataset); + if (rollInterval > 0) { + this.rollTimer = Executors.newSingleThreadScheduledExecutor( + new ThreadFactoryBuilder() + .setNameFormat(getName() + "-timed-roll-thread") + .build()); + rollTimer.scheduleWithFixedDelay(new Runnable() { + @Override + public void run() { + roll(); + } + }, rollInterval, rollInterval, TimeUnit.SECONDS); + } + counter.start(); + // signal that this sink is ready to process + LOG.info("Started DatasetSink " + getName()); + super.start(); + } + + void roll() { + // if the writer is null, nothing to do + if (writer == null) { + return; + } + + // no need to open/close while the lock is held, just replace the reference + DatasetWriter toClose = null; + DatasetWriter newWriter = openWriter(targetDataset); + + writerLock.lock(); + try { + toClose = writer; + this.writer = newWriter; + } finally { + writerLock.unlock(); + } + + LOG.info("Rolled writer for dataset: " + datasetName); + toClose.close(); + } + + @Override + public synchronized void stop() { + counter.stop(); + if (rollTimer != null) { + rollTimer.shutdown(); + try { + while (!rollTimer.isTerminated()) { + rollTimer.awaitTermination( + DatasetSinkConstants.DEFAULT_TERMINATION_INTERVAL, + TimeUnit.MILLISECONDS); + } + } catch (InterruptedException ex) { + LOG.warn("Interrupted while waiting for shutdown: " + rollTimer); + Thread.interrupted(); + } + } + + if (writer != null) { + // any write problems invalidate the writer, which is immediately closed + writer.close(); + this.writer = null; + } + + // signal that this sink has stopped + LOG.info("Stopped dataset sink: " + getName()); + super.stop(); + } + + @Override + public Status process() throws EventDeliveryException { + if (writer == null) { + throw new EventDeliveryException( + "Cannot recover after previous failure"); + } + + Channel channel = getChannel(); + Transaction transaction = null; + try { + long processedEvents = 0; + + // coarse locking to avoid waiting within the loop + writerLock.lock(); + transaction = channel.getTransaction(); + transaction.begin(); + try { + for (; processedEvents < batchSize; processedEvents += 1) { + Event event = channel.take(); + if (event == null) { + // no events available in the channel + break; + } + + this.datum = deserialize(event, datum); + + // writeEncoded would be an optimization in some cases, but HBase + // will not support it and partitioned Datasets need to get partition + // info from the entity Object. We may be able to avoid the + // serialization round-trip otherwise. + writer.write(datum); + } + // TODO: Add option to sync, depends on CDK-203 + writer.flush(); + } finally { + writerLock.unlock(); + } + + // commit after data has been written and flushed + transaction.commit(); + + if (processedEvents == 0) { + counter.incrementBatchEmptyCount(); + return Status.BACKOFF; + } else if (processedEvents < batchSize) { + counter.incrementBatchUnderflowCount(); + } else { + counter.incrementBatchCompleteCount(); + } + + counter.addToEventDrainSuccessCount(processedEvents); + + return Status.READY; + + } catch (Throwable th) { + // catch-all for any unhandled Throwable so that the transaction is + // correctly rolled back. + if (transaction != null) { + try { + transaction.rollback(); + } catch (Exception ex) { + LOG.error("Transaction rollback failed", ex); + throw Throwables.propagate(ex); + } + } + + // remove the writer's reference and close it + DatasetWriter toClose = null; + writerLock.lock(); + try { + toClose = writer; + this.writer = null; + } finally { + writerLock.unlock(); + } + toClose.close(); + + // handle the exception + Throwables.propagateIfInstanceOf(th, Error.class); + Throwables.propagateIfInstanceOf(th, EventDeliveryException.class); + throw new EventDeliveryException(th); + + } finally { + if (transaction != null) { + transaction.close(); + } + } + } + + /** + * Not thread-safe. + * + * @param event + * @param reuse + * @return + */ + private Object deserialize(Event event, Object reuse) + throws EventDeliveryException { + decoder = DecoderFactory.get().binaryDecoder(event.getBody(), decoder); + // no checked exception is thrown in the CacheLoader + ReflectDatumReader reader = readers.getUnchecked(schema(event)); + try { + return reader.read(reuse, decoder); + } catch (IOException ex) { + throw new EventDeliveryException("Cannot deserialize event", ex); + } + } + + private static Schema schema(Event event) throws EventDeliveryException { + Map headers = event.getHeaders(); + String schemaURL = headers.get( + DatasetSinkConstants.AVRO_SCHEMA_URL_HEADER); + try { + if (headers.get(DatasetSinkConstants.AVRO_SCHEMA_URL_HEADER) != null) { + return schemasFromURL.get(schemaURL); + } else { + return schemasFromLiteral.get( + headers.get(DatasetSinkConstants.AVRO_SCHEMA_LITERAL_HEADER)); + } + } catch (ExecutionException ex) { + throw new EventDeliveryException("Cannot get schema", ex.getCause()); + } + } + + private static DatasetWriter openWriter(Dataset target) { + DatasetWriter writer = target.newWriter(); + writer.open(); + return writer; + } + +} diff --git a/flume-ng-sinks/flume-dataset-sink/src/main/java/org/apache/flume/sink/kite/DatasetSinkConstants.java b/flume-ng-sinks/flume-dataset-sink/src/main/java/org/apache/flume/sink/kite/DatasetSinkConstants.java new file mode 100644 index 0000000000..5087352b42 --- /dev/null +++ b/flume-ng-sinks/flume-dataset-sink/src/main/java/org/apache/flume/sink/kite/DatasetSinkConstants.java @@ -0,0 +1,56 @@ +/** + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.flume.sink.kite; + +public class DatasetSinkConstants { + /** + * URI of the Kite DatasetRepository. + */ + public static final String CONFIG_KITE_REPO_URI = "kite.repo.uri"; + + /** + * Name of the Kite Dataset to write into. + */ + public static final String CONFIG_KITE_DATASET_NAME = "kite.dataset.name"; + + /** + * Number of records to process from the incoming channel per call to process. + */ + public static final String CONFIG_KITE_BATCH_SIZE = "kite.batchSize"; + public static long DEFAULT_BATCH_SIZE = 100; + + /** + * Maximum time to wait before finishing files. + */ + public static final String CONFIG_KITE_ROLL_INTERVAL = "kite.rollInterval"; + public static int DEFAULT_ROLL_INTERVAL = 30; // seconds + + /** + * Interval to wait for thread termination + */ + public static final int DEFAULT_TERMINATION_INTERVAL = 10000; // milliseconds + + /** + * Headers with avro schema information is expected. + */ + public static final String AVRO_SCHEMA_LITERAL_HEADER = + "flume.avro.schema.literal"; + public static final String AVRO_SCHEMA_URL_HEADER = "flume.avro.schema.url"; + +} diff --git a/flume-ng-sinks/flume-dataset-sink/src/test/java/org/apache/flume/sink/kite/TestDatasetSink.java b/flume-ng-sinks/flume-dataset-sink/src/test/java/org/apache/flume/sink/kite/TestDatasetSink.java new file mode 100644 index 0000000000..5708f0cb56 --- /dev/null +++ b/flume-ng-sinks/flume-dataset-sink/src/test/java/org/apache/flume/sink/kite/TestDatasetSink.java @@ -0,0 +1,475 @@ +/** + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.flume.sink.kite; + +import com.google.common.base.Function; +import com.google.common.base.Throwables; +import com.google.common.collect.Iterables; +import com.google.common.collect.Lists; +import com.google.common.collect.Maps; +import com.google.common.collect.Sets; +import java.io.ByteArrayOutputStream; +import java.io.File; +import java.io.FileWriter; +import java.io.IOException; +import java.net.URI; +import java.util.Arrays; +import java.util.HashSet; +import java.util.List; +import java.util.Map; +import java.util.Set; +import java.util.concurrent.Callable; +import javax.annotation.Nullable; +import org.apache.avro.Schema; +import org.apache.avro.generic.GenericData; +import org.apache.avro.generic.GenericRecordBuilder; +import org.apache.avro.io.Encoder; +import org.apache.avro.io.EncoderFactory; +import org.apache.avro.reflect.ReflectDatumWriter; +import org.apache.commons.io.FileUtils; +import org.apache.flume.Channel; +import org.apache.flume.Context; +import org.apache.flume.Event; +import org.apache.flume.EventDeliveryException; +import org.apache.flume.Transaction; +import org.apache.flume.channel.MemoryChannel; +import org.apache.flume.conf.Configurables; +import org.apache.flume.event.SimpleEvent; +import org.apache.hadoop.conf.Configuration; +import org.apache.hadoop.fs.FileSystem; +import org.apache.hadoop.hdfs.MiniDFSCluster; +import org.junit.After; +import org.junit.AfterClass; +import org.junit.Assert; +import org.junit.Before; +import org.junit.BeforeClass; +import org.junit.Test; +import org.kitesdk.data.Dataset; +import org.kitesdk.data.DatasetDescriptor; +import org.kitesdk.data.DatasetReader; +import org.kitesdk.data.DatasetRepositories; +import org.kitesdk.data.DatasetRepository; +import org.kitesdk.data.PartitionStrategy; + +public class TestDatasetSink { + + public static final String FILE_REPO_URI = "repo:file:target/test-repo"; + public static final String DATASET_NAME = "test"; + public static final DatasetRepository REPO = DatasetRepositories + .open(FILE_REPO_URI); + public static final File SCHEMA_FILE = new File("target/record-schema.avsc"); + public static final Schema RECORD_SCHEMA = new Schema.Parser().parse( + "{\"type\":\"record\",\"name\":\"rec\",\"fields\":[" + + "{\"name\":\"id\",\"type\":\"string\"}," + + "{\"name\":\"msg\",\"type\":[\"string\",\"null\"]," + + "\"default\":\"default\"}]}"); + public static final Schema COMPATIBLE_SCHEMA = new Schema.Parser().parse( + "{\"type\":\"record\",\"name\":\"rec\",\"fields\":[" + + "{\"name\":\"id\",\"type\":\"string\"}]}"); + public static final Schema INCOMPATIBLE_SCHEMA = new Schema.Parser().parse( + "{\"type\":\"record\",\"name\":\"user\",\"fields\":[" + + "{\"name\":\"username\",\"type\":\"string\"}]}"); + public static final DatasetDescriptor DESCRIPTOR = new DatasetDescriptor + .Builder() + .schema(RECORD_SCHEMA) + .build(); + + Context config = null; + Channel in = null; + List expected = null; + private static final String DFS_DIR = "target/test/dfs"; + private static final String TEST_BUILD_DATA_KEY = "test.build.data"; + private static String oldTestBuildDataProp = null; + + @BeforeClass + public static void saveSchema() throws IOException { + oldTestBuildDataProp = System.getProperty(TEST_BUILD_DATA_KEY); + System.setProperty(TEST_BUILD_DATA_KEY, DFS_DIR); + FileWriter schema = new FileWriter(SCHEMA_FILE); + schema.append(RECORD_SCHEMA.toString()); + schema.close(); + } + + @AfterClass + public static void tearDownClass() { + FileUtils.deleteQuietly(new File(DFS_DIR)); + if (oldTestBuildDataProp != null) { + System.setProperty(TEST_BUILD_DATA_KEY, oldTestBuildDataProp); + } + } + + @Before + public void setup() throws EventDeliveryException { + REPO.create(DATASET_NAME, DESCRIPTOR); + + this.config = new Context(); + config.put(DatasetSinkConstants.CONFIG_KITE_REPO_URI, FILE_REPO_URI); + config.put(DatasetSinkConstants.CONFIG_KITE_DATASET_NAME, DATASET_NAME); + + this.in = new MemoryChannel(); + Configurables.configure(in, config); + + GenericRecordBuilder builder = new GenericRecordBuilder(RECORD_SCHEMA); + expected = Lists.newArrayList( + builder.set("id", "1").set("msg", "msg1").build(), + builder.set("id", "2").set("msg", "msg2").build(), + builder.set("id", "3").set("msg", "msg3").build()); + + putToChannel(in, Iterables.transform(expected, + new Function() { + private int i = 0; + + @Override + public Event apply(@Nullable GenericData.Record rec) { + this.i += 1; + boolean useURI = (i % 2) == 0; + return event(rec, RECORD_SCHEMA, SCHEMA_FILE, useURI); + } + })); + } + + @After + public void teardown() { + REPO.delete(DATASET_NAME); + } + + @Test + public void testFileStore() throws EventDeliveryException { + DatasetSink sink = sink(in, config); + + // run the sink + sink.start(); + sink.process(); + sink.stop(); + + Assert.assertEquals( + Sets.newHashSet(expected), + read(REPO.load(DATASET_NAME))); + Assert.assertEquals("Should have committed", 0, remaining(in)); + } + + @Test + public void testPartitionedData() throws EventDeliveryException { + REPO.create("partitioned", new DatasetDescriptor.Builder(DESCRIPTOR) + .partitionStrategy(new PartitionStrategy.Builder() + .identity("id", String.class, 10) // partition by id + .build()) + .build()); + + try { + config.put(DatasetSinkConstants.CONFIG_KITE_DATASET_NAME, "partitioned"); + DatasetSink sink = sink(in, config); + + // run the sink + sink.start(); + sink.process(); + sink.stop(); + + Assert.assertEquals( + Sets.newHashSet(expected), + read(REPO.load("partitioned"))); + Assert.assertEquals("Should have committed", 0, remaining(in)); + } finally { + if (REPO.exists("partitioned")) { + REPO.delete("partitioned"); + } + } + } + + @Test + public void testMiniClusterStore() + throws EventDeliveryException, IOException { + // setup a minicluster + MiniDFSCluster cluster = new MiniDFSCluster + .Builder(new Configuration()) + .build(); + DatasetRepository hdfsRepo = null; + try { + FileSystem dfs = cluster.getFileSystem(); + Configuration conf = dfs.getConf(); + String repoURI = "repo:" + conf.get("fs.defaultFS") + "/tmp/repo"; + + // create a repository and dataset in HDFS + hdfsRepo = DatasetRepositories.open(repoURI); + hdfsRepo.create(DATASET_NAME, DESCRIPTOR); + + // update the config to use the HDFS repository + config.put(DatasetSinkConstants.CONFIG_KITE_REPO_URI, repoURI); + + DatasetSink sink = sink(in, config); + + // run the sink + sink.start(); + sink.process(); + sink.stop(); + + Assert.assertEquals( + Sets.newHashSet(expected), + read(hdfsRepo.load(DATASET_NAME))); + Assert.assertEquals("Should have committed", 0, remaining(in)); + + } finally { + if (hdfsRepo != null && hdfsRepo.exists(DATASET_NAME)) { + hdfsRepo.delete(DATASET_NAME); + } + cluster.shutdown(); + } + } + + @Test + public void testBatchSize() throws EventDeliveryException { + DatasetSink sink = sink(in, config); + + // release one record per process call + config.put("kite.batchSize", "2"); + Configurables.configure(sink, config); + + sink.start(); + sink.process(); // process the first and second + sink.roll(); // roll at the next process call + sink.process(); // roll and process the third + Assert.assertEquals( + Sets.newHashSet(expected.subList(0, 2)), + read(REPO.load(DATASET_NAME))); + Assert.assertEquals("Should have committed", 0, remaining(in)); + sink.roll(); // roll at the next process call + sink.process(); // roll, the channel is empty + Assert.assertEquals( + Sets.newHashSet(expected), + read(REPO.load(DATASET_NAME))); + sink.stop(); + } + + @Test + public void testTimedFileRolling() + throws EventDeliveryException, InterruptedException { + // use a new roll interval + config.put("kite.rollInterval", "1"); // in seconds + + DatasetSink sink = sink(in, config); + + Dataset records = REPO.load(DATASET_NAME); + + // run the sink + sink.start(); + sink.process(); + + Assert.assertEquals(Sets.newHashSet(), read(records)); + Assert.assertEquals("Should have committed", 0, remaining(in)); + + Thread.sleep(1100); // sleep longer than the roll interval + sink.process(); // rolling happens in the process method + + Assert.assertEquals(Sets.newHashSet(expected), read(records)); + + // wait until the end to stop because it would close the files + sink.stop(); + } + + @Test + public void testCompatibleSchemas() throws EventDeliveryException { + DatasetSink sink = sink(in, config); + + // add a compatible record that is missing the msg field + GenericRecordBuilder compatBuilder = new GenericRecordBuilder( + COMPATIBLE_SCHEMA); + GenericData.Record compatibleRecord = compatBuilder.set("id", "0").build(); + + // add the record to the incoming channel + putToChannel(in, event(compatibleRecord, COMPATIBLE_SCHEMA, null, false)); + + // the record will be read using the real schema, so create the expected + // record using it, but without any data + + GenericRecordBuilder builder = new GenericRecordBuilder(RECORD_SCHEMA); + GenericData.Record expectedRecord = builder.set("id", "0").build(); + expected.add(expectedRecord); + + // run the sink + sink.start(); + sink.process(); + sink.stop(); + + Assert.assertEquals( + Sets.newHashSet(expected), + read(REPO.load(DATASET_NAME))); + Assert.assertEquals("Should have committed", 0, remaining(in)); + } + + @Test + public void testIncompatibleSchemas() throws EventDeliveryException { + final DatasetSink sink = sink(in, config); + + GenericRecordBuilder builder = new GenericRecordBuilder( + INCOMPATIBLE_SCHEMA); + GenericData.Record rec = builder.set("username", "koala").build(); + putToChannel(in, event(rec, INCOMPATIBLE_SCHEMA, null, false)); + + // run the sink + sink.start(); + assertThrows("Should fail", EventDeliveryException.class, + new Callable() { + @Override + public Object call() throws EventDeliveryException { + sink.process(); + return null; + } + }); + sink.stop(); + + Assert.assertEquals("Should have rolled back", + expected.size() + 1, remaining(in)); + } + + @Test + public void testMissingSchema() throws EventDeliveryException { + final DatasetSink sink = sink(in, config); + + Event badEvent = new SimpleEvent(); + badEvent.setHeaders(Maps.newHashMap()); + badEvent.setBody(serialize(expected.get(0), RECORD_SCHEMA)); + putToChannel(in, badEvent); + + // run the sink + sink.start(); + assertThrows("Should fail", EventDeliveryException.class, + new Callable() { + @Override + public Object call() throws EventDeliveryException { + sink.process(); + return null; + } + }); + sink.stop(); + + Assert.assertEquals("Should have rolled back", + expected.size() + 1, remaining(in)); + } + + public static DatasetSink sink(Channel in, Context config) { + DatasetSink sink = new DatasetSink(); + sink.setChannel(in); + Configurables.configure(sink, config); + return sink; + } + + public static HashSet read(Dataset dataset) { + DatasetReader reader = dataset.newReader(); + try { + reader.open(); + return Sets.newHashSet(reader.iterator()); + } finally { + reader.close(); + } + } + + public static int remaining(Channel ch) throws EventDeliveryException { + Transaction t = ch.getTransaction(); + try { + t.begin(); + int count = 0; + while (ch.take() != null) { + count += 1; + } + t.commit(); + return count; + } catch (Throwable th) { + t.rollback(); + Throwables.propagateIfInstanceOf(th, Error.class); + Throwables.propagateIfInstanceOf(th, EventDeliveryException.class); + throw new EventDeliveryException(th); + } finally { + t.close(); + } + } + + public static void putToChannel(Channel in, Event... records) + throws EventDeliveryException { + putToChannel(in, Arrays.asList(records)); + } + + public static void putToChannel(Channel in, Iterable records) + throws EventDeliveryException { + Transaction t = in.getTransaction(); + try { + t.begin(); + for (Event record : records) { + in.put(record); + } + t.commit(); + } catch (Throwable th) { + t.rollback(); + Throwables.propagateIfInstanceOf(th, Error.class); + Throwables.propagateIfInstanceOf(th, EventDeliveryException.class); + throw new EventDeliveryException(th); + } finally { + t.close(); + } + } + + public static Event event( + Object datum, Schema schema, File file, boolean useURI) { + Map headers = Maps.newHashMap(); + if (useURI) { + headers.put(DatasetSinkConstants.AVRO_SCHEMA_URL_HEADER, + file.getAbsoluteFile().toURI().toString()); + } else { + headers.put(DatasetSinkConstants.AVRO_SCHEMA_LITERAL_HEADER, + schema.toString()); + } + Event e = new SimpleEvent(); + e.setBody(serialize(datum, schema)); + e.setHeaders(headers); + return e; + } + + @SuppressWarnings("unchecked") + public static byte[] serialize(Object datum, Schema schema) { + ByteArrayOutputStream out = new ByteArrayOutputStream(); + Encoder encoder = EncoderFactory.get().binaryEncoder(out, null); + ReflectDatumWriter writer = new ReflectDatumWriter(schema); + try { + writer.write(datum, encoder); + encoder.flush(); + } catch (IOException ex) { + Throwables.propagate(ex); + } + return out.toByteArray(); + } + + /** + * A convenience method to avoid a large number of @Test(expected=...) tests. + * + * This variant uses a Callable, which is allowed to throw checked Exceptions. + * + * @param message A String message to describe this assertion + * @param expected An Exception class that the Runnable should throw + * @param callable A Callable that is expected to throw the exception + */ + public static void assertThrows( + String message, Class expected, Callable callable) { + try { + callable.call(); + Assert.fail("No exception was thrown (" + message + "), expected: " + + expected.getName()); + } catch (Exception actual) { + Assert.assertEquals(message, expected, actual.getClass()); + } + } +} diff --git a/flume-ng-sinks/pom.xml b/flume-ng-sinks/pom.xml index d03576b5a6..6ac2b4d0be 100644 --- a/flume-ng-sinks/pom.xml +++ b/flume-ng-sinks/pom.xml @@ -47,4 +47,33 @@ limitations under the License. flume-ng-elasticsearch-sink flume-ng-morphline-solr-sink + + + + + hadoop-1.0 + + + !hadoop.profile + + + + + + hadoop-2 + + + hadoop.profile + 2 + + + + + flume-dataset-sink + + + + + diff --git a/pom.xml b/pom.xml index 453be9585e..d71239ad1a 100644 --- a/pom.xml +++ b/pom.xml @@ -48,6 +48,8 @@ limitations under the License. 1.7.3 0.90.1 + + 2.1.0-beta @@ -106,7 +108,7 @@ limitations under the License. - 2.1.0-beta + ${hadoop2.version} 0.94.2 hadoop-common 0.8.0 @@ -143,6 +145,13 @@ limitations under the License. hadoop-auth ${hadoop.version} + + + + org.apache.flume.flume-ng-sinks + flume-dataset-sink + 1.5.0-SNAPSHOT + @@ -538,7 +547,7 @@ limitations under the License. - .idea/ + **/.idea/ **/*.iml **/nb-configuration.xml .git/ @@ -1087,6 +1096,12 @@ limitations under the License. 3.0.3 + + org.kitesdk + kite-data-core + 0.10.1 + + From b8c5501683244df156ff6d70a72a12bac131ed8f Mon Sep 17 00:00:00 2001 From: Jarek Jarcec Cecho Date: Sat, 18 Jan 2014 21:38:18 -0800 Subject: [PATCH 064/341] FLUME-2303. HBaseSink tests can fail based on order of execution (Hari Shreedharan via Jarek Jarcec Cecho) --- .../src/test/java/org/apache/flume/sink/hbase/TestHBaseSink.java | 1 + 1 file changed, 1 insertion(+) diff --git a/flume-ng-sinks/flume-ng-hbase-sink/src/test/java/org/apache/flume/sink/hbase/TestHBaseSink.java b/flume-ng-sinks/flume-ng-hbase-sink/src/test/java/org/apache/flume/sink/hbase/TestHBaseSink.java index f2bba314e1..068f543b1d 100644 --- a/flume-ng-sinks/flume-ng-hbase-sink/src/test/java/org/apache/flume/sink/hbase/TestHBaseSink.java +++ b/flume-ng-sinks/flume-ng-hbase-sink/src/test/java/org/apache/flume/sink/hbase/TestHBaseSink.java @@ -145,6 +145,7 @@ public void testOneEvent() throws Exception { @Test public void testThreeEvents() throws Exception { testUtility.createTable(tableName.getBytes(), columnFamily.getBytes()); + ctx.put("batchSize", "3"); HBaseSink sink = new HBaseSink(testUtility.getConfiguration()); Configurables.configure(sink, ctx); Channel channel = new MemoryChannel(); From 7951c4e9b2f10fe25cac51c09f041d75f9122568 Mon Sep 17 00:00:00 2001 From: Jarek Jarcec Cecho Date: Mon, 20 Jan 2014 14:12:29 -0800 Subject: [PATCH 065/341] FLUME-2302. TestHDFS Sink fails with Can't get Kerberos realm' (Hari Shreedharan via Jarek Jarcec Cecho) --- .../java/org/apache/flume/sink/hdfs/TestHDFSEventSink.java | 5 +++++ 1 file changed, 5 insertions(+) diff --git a/flume-ng-sinks/flume-hdfs-sink/src/test/java/org/apache/flume/sink/hdfs/TestHDFSEventSink.java b/flume-ng-sinks/flume-hdfs-sink/src/test/java/org/apache/flume/sink/hdfs/TestHDFSEventSink.java index 4337ef4db7..87918d118e 100644 --- a/flume-ng-sinks/flume-hdfs-sink/src/test/java/org/apache/flume/sink/hdfs/TestHDFSEventSink.java +++ b/flume-ng-sinks/flume-hdfs-sink/src/test/java/org/apache/flume/sink/hdfs/TestHDFSEventSink.java @@ -74,6 +74,11 @@ public class TestHDFSEventSink { private static final Logger LOG = LoggerFactory .getLogger(HDFSEventSink.class); + static { + System.setProperty("java.security.krb5.realm", "flume"); + System.setProperty("java.security.krb5.kdc", "blah"); + } + private void dirCleanup() { Configuration conf = new Configuration(); try { From 492cd8d08cab347e02e43805b47435413d16937c Mon Sep 17 00:00:00 2001 From: Hari Shreedharan Date: Mon, 20 Jan 2014 14:15:14 -0800 Subject: [PATCH 066/341] FLUME-2304. DatasetSink test fails unexpectedly. (Ryan Blue via Hari Shreedharan) --- .../test/java/org/apache/flume/sink/kite/TestDatasetSink.java | 1 - 1 file changed, 1 deletion(-) diff --git a/flume-ng-sinks/flume-dataset-sink/src/test/java/org/apache/flume/sink/kite/TestDatasetSink.java b/flume-ng-sinks/flume-dataset-sink/src/test/java/org/apache/flume/sink/kite/TestDatasetSink.java index 5708f0cb56..bd0e1dcfe9 100644 --- a/flume-ng-sinks/flume-dataset-sink/src/test/java/org/apache/flume/sink/kite/TestDatasetSink.java +++ b/flume-ng-sinks/flume-dataset-sink/src/test/java/org/apache/flume/sink/kite/TestDatasetSink.java @@ -270,7 +270,6 @@ public void testTimedFileRolling() sink.start(); sink.process(); - Assert.assertEquals(Sets.newHashSet(), read(records)); Assert.assertEquals("Should have committed", 0, remaining(in)); Thread.sleep(1100); // sleep longer than the roll interval From fa3fb3deaa1045d0c6ebbd18630fa268f5db7fc1 Mon Sep 17 00:00:00 2001 From: Hari Shreedharan Date: Mon, 27 Jan 2014 21:13:22 -0800 Subject: [PATCH 067/341] FLUME-1892. Fix NullPointerException in IRC Sink. (Ashish Paliwal via Hari Shreedharan) --- .../org/apache/flume/sink/irc/IRCSink.java | 2 +- .../apache/flume/sink/irc/TestIRCSink.java | 159 ++++++++++++++++++ 2 files changed, 160 insertions(+), 1 deletion(-) create mode 100644 flume-ng-sinks/flume-irc-sink/src/test/java/org/apache/flume/sink/irc/TestIRCSink.java diff --git a/flume-ng-sinks/flume-irc-sink/src/main/java/org/apache/flume/sink/irc/IRCSink.java b/flume-ng-sinks/flume-irc-sink/src/main/java/org/apache/flume/sink/irc/IRCSink.java index 8e77218e89..40657b40b3 100644 --- a/flume-ng-sinks/flume-irc-sink/src/main/java/org/apache/flume/sink/irc/IRCSink.java +++ b/flume-ng-sinks/flume-irc-sink/src/main/java/org/apache/flume/sink/irc/IRCSink.java @@ -132,7 +132,7 @@ public void configure(Context context) { user = context.getString("user"); name = context.getString("name"); chan = context.getString("chan"); - splitLines = context.getBoolean("splitlines"); + splitLines = context.getBoolean("splitlines", false); splitChars = context.getString("splitchars"); if (portStr != null) { diff --git a/flume-ng-sinks/flume-irc-sink/src/test/java/org/apache/flume/sink/irc/TestIRCSink.java b/flume-ng-sinks/flume-irc-sink/src/test/java/org/apache/flume/sink/irc/TestIRCSink.java new file mode 100644 index 0000000000..e6c065ee0d --- /dev/null +++ b/flume-ng-sinks/flume-irc-sink/src/test/java/org/apache/flume/sink/irc/TestIRCSink.java @@ -0,0 +1,159 @@ +/** + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package org.apache.flume.sink.irc; + +import org.apache.commons.io.FileUtils; +import org.apache.commons.io.IOUtils; +import org.apache.flume.*; +import org.apache.flume.channel.MemoryChannel; +import org.apache.flume.conf.Configurables; +import org.apache.flume.event.EventBuilder; +import org.junit.After; +import org.junit.Before; +import org.junit.Rule; +import org.junit.Test; +import org.junit.rules.TemporaryFolder; + +import java.io.File; +import java.io.FileOutputStream; +import java.io.IOException; +import java.net.ServerSocket; +import java.net.Socket; +import java.util.List; +import java.util.UUID; + +import static org.junit.Assert.fail; + +public class TestIRCSink { + + private File eventFile; + int ircServerPort; + DumbIRCServer dumbIRCServer; + @Rule + public TemporaryFolder folder = new TemporaryFolder(); + + private static int findFreePort() throws IOException { + ServerSocket socket = new ServerSocket(0); + int port = socket.getLocalPort(); + socket.close(); + return port; + } + + @Before + public void setUp() throws IOException { + ircServerPort = findFreePort(); + dumbIRCServer = new DumbIRCServer(ircServerPort); + dumbIRCServer.start(); + eventFile = folder.newFile("eventFile.txt"); + } + + @After + public void tearDown() throws Exception { + dumbIRCServer.shutdownServer(); + } + + @Test + public void testIRCSinkMissingSplitLineProperty() { + Sink ircSink = new IRCSink(); + ircSink.setName("IRC Sink - " + UUID.randomUUID().toString()); + Context context = new Context(); + context.put("hostname", "localhost"); + context.put("port", String.valueOf(ircServerPort)); + context.put("nick", "flume"); + context.put("password", "flume"); + context.put("user", "flume"); + context.put("name", "flume-dev"); + context.put("chan", "flume"); + context.put("splitchars", "false"); + Configurables.configure(ircSink, context); + Channel memoryChannel = new MemoryChannel(); + Configurables.configure(memoryChannel, context); + ircSink.setChannel(memoryChannel); + ircSink.start(); + Transaction txn = memoryChannel.getTransaction(); + txn.begin(); + Event event = EventBuilder.withBody("Dummy Event".getBytes()); + memoryChannel.put(event); + txn.commit(); + txn.close(); + try { + Sink.Status status = ircSink.process(); + if (status == Sink.Status.BACKOFF) { + fail("Error occured"); + } + } catch (EventDeliveryException eDelExcp) { + // noop + } + } + + class DumbIRCServer extends Thread { + int port; + ServerSocket ss; + + public DumbIRCServer(int port) { + this.port = port; + } + + public void run() { + try { + ss = new ServerSocket(port); + while (true) { + try { + Socket socket = ss.accept(); + process(socket); + } catch (Exception ex) {/* noop */ } + } + } catch (IOException e) { + // noop + } + } + + public void shutdownServer() throws Exception { + ss.close(); + } + + /** + * Process the incoming request from IRC client + * + * @param socket IRC client connection socket + * @throws IOException + */ + private void process(Socket socket) throws IOException { + FileOutputStream fileOutputStream = FileUtils.openOutputStream(eventFile); + List input = IOUtils.readLines(socket.getInputStream()); + for (String next : input) { + if (isPrivMessage(next)) { + fileOutputStream.write(next.getBytes()); + fileOutputStream.write("\n".getBytes()); + } + } + fileOutputStream.close(); + socket.close(); + } + + /** + * Checks if the message is Priv message + * + * @param input command received from IRC client + * @return true, if command received is PrivMessage + */ + private boolean isPrivMessage(String input) { + return input.startsWith("PRIVMSG"); + } + } +} \ No newline at end of file From 044ba300311a4a3e682a946bd0821c2b52acc190 Mon Sep 17 00:00:00 2001 From: Jarek Jarcec Cecho Date: Tue, 28 Jan 2014 15:49:44 -0800 Subject: [PATCH 068/341] FLUME-2305. BucketWriter#close must cancel idleFuture (Hari Shreedharan via Jarek Jarcec Cecho) --- .../main/java/org/apache/flume/sink/hdfs/BucketWriter.java | 5 +++++ 1 file changed, 5 insertions(+) diff --git a/flume-ng-sinks/flume-hdfs-sink/src/main/java/org/apache/flume/sink/hdfs/BucketWriter.java b/flume-ng-sinks/flume-hdfs-sink/src/main/java/org/apache/flume/sink/hdfs/BucketWriter.java index 62e47de891..7fb3bdd760 100644 --- a/flume-ng-sinks/flume-hdfs-sink/src/main/java/org/apache/flume/sink/hdfs/BucketWriter.java +++ b/flume-ng-sinks/flume-hdfs-sink/src/main/java/org/apache/flume/sink/hdfs/BucketWriter.java @@ -317,6 +317,11 @@ public Void call() throws Exception { timedRollFuture = null; } + if (idleFuture != null && !idleFuture.isDone()) { + idleFuture.cancel(false); // do not cancel myself if running! + idleFuture = null; + } + if (bucketPath != null && fileSystem != null) { renameBucket(); // could block or throw IOException fileSystem = null; From d9061a037662dd0a37c44ba5f9cf705d21c84987 Mon Sep 17 00:00:00 2001 From: Jarek Jarcec Cecho Date: Wed, 5 Feb 2014 18:02:12 -0800 Subject: [PATCH 069/341] FLUME-2312. Add utility for adorning HTTP contexts in Jetty (Hari Shreedharan via Jarek Jarcec Cecho) --- .../http/HTTPMetricsServer.java | 7 +++ .../apache/flume/source/http/HTTPSource.java | 2 + .../flume/tools/HTTPServerConstraintUtil.java | 62 +++++++++++++++++++ .../http/TestHTTPMetricsServer.java | 31 ++++++++++ .../flume/source/http/TestHTTPSource.java | 20 ++++++ 5 files changed, 122 insertions(+) create mode 100644 flume-ng-core/src/main/java/org/apache/flume/tools/HTTPServerConstraintUtil.java diff --git a/flume-ng-core/src/main/java/org/apache/flume/instrumentation/http/HTTPMetricsServer.java b/flume-ng-core/src/main/java/org/apache/flume/instrumentation/http/HTTPMetricsServer.java index 2c2c6f3f78..7c0afb046b 100644 --- a/flume-ng-core/src/main/java/org/apache/flume/instrumentation/http/HTTPMetricsServer.java +++ b/flume-ng-core/src/main/java/org/apache/flume/instrumentation/http/HTTPMetricsServer.java @@ -108,6 +108,13 @@ public void handle(String target, //If we want to use any other url for something else, we should make sure //that for metrics only /metrics is used to prevent backward //compatibility issues. + if(request.getMethod().equalsIgnoreCase("TRACE") || request.getMethod() + .equalsIgnoreCase("OPTIONS")) { + response.sendError(HttpServletResponse.SC_FORBIDDEN); + response.flushBuffer(); + ((Request) request).setHandled(true); + return; + } if (target.equals("/")) { response.setContentType("text/html;charset=utf-8"); response.setStatus(HttpServletResponse.SC_OK); diff --git a/flume-ng-core/src/main/java/org/apache/flume/source/http/HTTPSource.java b/flume-ng-core/src/main/java/org/apache/flume/source/http/HTTPSource.java index 48c64922b8..115b34f584 100644 --- a/flume-ng-core/src/main/java/org/apache/flume/source/http/HTTPSource.java +++ b/flume-ng-core/src/main/java/org/apache/flume/source/http/HTTPSource.java @@ -26,6 +26,7 @@ import org.apache.flume.conf.Configurable; import org.apache.flume.instrumentation.SourceCounter; import org.apache.flume.source.AbstractSource; +import org.apache.flume.tools.HTTPServerConstraintUtil; import org.mortbay.jetty.Connector; import org.mortbay.jetty.Server; import org.mortbay.jetty.nio.SelectChannelConnector; @@ -190,6 +191,7 @@ public void start() { new org.mortbay.jetty.servlet.Context( srv, "/", org.mortbay.jetty.servlet.Context.SESSIONS); root.addServlet(new ServletHolder(new FlumeHTTPServlet()), "/"); + HTTPServerConstraintUtil.enforceConstraints(root); srv.start(); Preconditions.checkArgument(srv.getHandler().equals(root)); } catch (Exception ex) { diff --git a/flume-ng-core/src/main/java/org/apache/flume/tools/HTTPServerConstraintUtil.java b/flume-ng-core/src/main/java/org/apache/flume/tools/HTTPServerConstraintUtil.java new file mode 100644 index 0000000000..479cfc46bf --- /dev/null +++ b/flume-ng-core/src/main/java/org/apache/flume/tools/HTTPServerConstraintUtil.java @@ -0,0 +1,62 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, + * software distributed under the License is distributed on an + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + * KIND, either express or implied. See the License for the + * specific language governing permissions and limitations + * under the License. + */ +package org.apache.flume.tools; + +import org.mortbay.jetty.security.Constraint; +import org.mortbay.jetty.security.ConstraintMapping; +import org.mortbay.jetty.security.SecurityHandler; +import org.mortbay.jetty.servlet.Context; + +// Most of the code in this class is copied from HBASE-10473 + +/** + * Utility class to impose constraints on Jetty HTTP servers + */ + +public class HTTPServerConstraintUtil { + + private HTTPServerConstraintUtil() { + + } + + /** + * Impose constraints on the {@linkplain org.mortbay.jetty.servlet.Context} + * passed in. + * @param ctx - {@linkplain org.mortbay.jetty.servlet.Context} to impose + * constraints on. + */ + public static void enforceConstraints(Context ctx) { + Constraint c = new Constraint(); + c.setAuthenticate(true); + + ConstraintMapping cmt = new ConstraintMapping(); + cmt.setConstraint(c); + cmt.setMethod("TRACE"); + cmt.setPathSpec("/*"); + + ConstraintMapping cmo = new ConstraintMapping(); + cmo.setConstraint(c); + cmo.setMethod("OPTIONS"); + cmo.setPathSpec("/*"); + + SecurityHandler sh = new SecurityHandler(); + sh.setConstraintMappings(new ConstraintMapping[]{cmt, cmo}); + ctx.addHandler(sh); + } +} diff --git a/flume-ng-core/src/test/java/org/apache/flume/instrumentation/http/TestHTTPMetricsServer.java b/flume-ng-core/src/test/java/org/apache/flume/instrumentation/http/TestHTTPMetricsServer.java index a2a1c30adc..eb2d02d891 100644 --- a/flume-ng-core/src/test/java/org/apache/flume/instrumentation/http/TestHTTPMetricsServer.java +++ b/flume-ng-core/src/test/java/org/apache/flume/instrumentation/http/TestHTTPMetricsServer.java @@ -38,6 +38,8 @@ import org.junit.Assert; import org.junit.Test; +import javax.servlet.http.HttpServletResponse; + /** * */ @@ -127,4 +129,33 @@ private void testWithPort(int port) throws Exception { srv.stop(); System.out.println(String.valueOf(port) + "test success!"); } + + @Test + public void testTrace() throws Exception { + doTestForbiddenMethods(4543,"TRACE"); + } + @Test + public void testOptions() throws Exception { + doTestForbiddenMethods(4432,"OPTIONS"); + } + + public void doTestForbiddenMethods(int port, String method) + throws Exception { + MonitorService srv = new HTTPMetricsServer(); + Context context = new Context(); + if (port > 1024) { + context.put(HTTPMetricsServer.CONFIG_PORT, String.valueOf(port)); + } else { + port = HTTPMetricsServer.DEFAULT_PORT; + } + srv.configure(context); + srv.start(); + Thread.sleep(1000); + URL url = new URL("http://0.0.0.0:" + String.valueOf(port) + "/metrics"); + HttpURLConnection conn = (HttpURLConnection) url.openConnection(); + conn.setRequestMethod(method); + Assert.assertEquals(HttpServletResponse.SC_FORBIDDEN, + conn.getResponseCode()); + srv.stop(); + } } diff --git a/flume-ng-core/src/test/java/org/apache/flume/source/http/TestHTTPSource.java b/flume-ng-core/src/test/java/org/apache/flume/source/http/TestHTTPSource.java index ab8ec094a3..5b07a6ea38 100644 --- a/flume-ng-core/src/test/java/org/apache/flume/source/http/TestHTTPSource.java +++ b/flume-ng-core/src/test/java/org/apache/flume/source/http/TestHTTPSource.java @@ -29,7 +29,10 @@ import org.apache.flume.conf.Configurables; import org.apache.flume.event.JSONEvent; import org.apache.http.HttpResponse; +import org.apache.http.client.methods.HttpOptions; import org.apache.http.client.methods.HttpPost; +import org.apache.http.client.methods.HttpRequestBase; +import org.apache.http.client.methods.HttpTrace; import org.apache.http.conn.ssl.SSLSocketFactory; import org.apache.http.entity.StringEntity; import org.apache.http.impl.client.DefaultHttpClient; @@ -165,6 +168,23 @@ public void testSimple() throws IOException, InterruptedException { tx.close(); } + @Test + public void testTrace() throws Exception { + doTestForbidden(new HttpTrace("http://0.0.0.0:" + selectedPort)); + } + + @Test + public void testOptions() throws Exception { + doTestForbidden(new HttpOptions("http://0.0.0.0:" + selectedPort)); + } + + + private void doTestForbidden(HttpRequestBase request) throws Exception { + HttpResponse response = httpClient.execute(request); + Assert.assertEquals(HttpServletResponse.SC_FORBIDDEN, + response.getStatusLine().getStatusCode()); + } + @Test public void testSimpleUTF16() throws IOException, InterruptedException { From e30cbd544e6ca0b8a19d2e9db491c679e9ca4ee8 Mon Sep 17 00:00:00 2001 From: Brock Noland Date: Fri, 7 Feb 2014 10:45:52 -0600 Subject: [PATCH 070/341] =?UTF-8?q?FLUME-2311=20-=20Use=20standard=20way?= =?UTF-8?q?=20of=20finding=20queue/topic=20(Hugo=20Lassi=C3=A8ge=20via=20B?= =?UTF-8?q?rock=20Noland)?= MIME-Version: 1.0 Content-Type: text/plain; charset=UTF-8 Content-Transfer-Encoding: 8bit --- .../source/jms/JMSDestinationLocator.java | 23 +++++++++++++++ .../flume/source/jms/JMSMessageConsumer.java | 28 ++++++++++++------- .../source/jms/JMSMessageConsumerFactory.java | 14 +++++----- .../apache/flume/source/jms/JMSSource.java | 24 ++++++++++++---- .../source/jms/JMSSourceConfiguration.java | 3 ++ .../jms/JMSMessageConsumerTestBase.java | 10 +++++-- .../source/jms/TestIntegrationActiveMQ.java | 12 ++++++++ .../flume/source/jms/TestJMSSource.java | 18 ++++++------ 8 files changed, 97 insertions(+), 35 deletions(-) create mode 100644 flume-ng-sources/flume-jms-source/src/main/java/org/apache/flume/source/jms/JMSDestinationLocator.java diff --git a/flume-ng-sources/flume-jms-source/src/main/java/org/apache/flume/source/jms/JMSDestinationLocator.java b/flume-ng-sources/flume-jms-source/src/main/java/org/apache/flume/source/jms/JMSDestinationLocator.java new file mode 100644 index 0000000000..c590c8e31b --- /dev/null +++ b/flume-ng-sources/flume-jms-source/src/main/java/org/apache/flume/source/jms/JMSDestinationLocator.java @@ -0,0 +1,23 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, + * software distributed under the License is distributed on an + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + * KIND, either express or implied. See the License for the + * specific language governing permissions and limitations + * under the License. + */ +package org.apache.flume.source.jms; + +public enum JMSDestinationLocator { + JNDI, CDI +} \ No newline at end of file diff --git a/flume-ng-sources/flume-jms-source/src/main/java/org/apache/flume/source/jms/JMSMessageConsumer.java b/flume-ng-sources/flume-jms-source/src/main/java/org/apache/flume/source/jms/JMSMessageConsumer.java index 9463e9a83e..7a9461ba1c 100644 --- a/flume-ng-sources/flume-jms-source/src/main/java/org/apache/flume/source/jms/JMSMessageConsumer.java +++ b/flume-ng-sources/flume-jms-source/src/main/java/org/apache/flume/source/jms/JMSMessageConsumer.java @@ -28,6 +28,8 @@ import javax.jms.Message; import javax.jms.MessageConsumer; import javax.jms.Session; +import javax.naming.InitialContext; +import javax.naming.NamingException; import org.apache.flume.Event; import org.apache.flume.FlumeException; @@ -50,11 +52,11 @@ class JMSMessageConsumer { private final Destination destination; private final MessageConsumer messageConsumer; - JMSMessageConsumer(ConnectionFactory connectionFactory, - String destinationName, JMSDestinationType destinationType, - String messageSelector, int batchSize, long pollTimeout, - JMSMessageConverter messageConverter, Optional userName, - Optional password) { + JMSMessageConsumer(InitialContext initialContext, ConnectionFactory connectionFactory, String destinationName, + JMSDestinationLocator destinationLocator, JMSDestinationType destinationType, + String messageSelector, int batchSize, long pollTimeout, + JMSMessageConverter messageConverter, + Optional userName, Optional password) { this.batchSize = batchSize; this.pollTimeout = pollTimeout; this.messageConverter = messageConverter; @@ -79,7 +81,9 @@ class JMSMessageConsumer { } catch (JMSException e) { throw new FlumeException("Could not create session", e); } - try { + + try { + if (destinationLocator.equals(JMSDestinationLocator.CDI)) { switch (destinationType) { case QUEUE: destination = session.createQueue(destinationName); @@ -90,12 +94,16 @@ class JMSMessageConsumer { default: throw new IllegalStateException(String.valueOf(destinationType)); } - } catch (JMSException e) { - throw new FlumeException("Could not create destination " - + destinationName, e); + } else { + destination = (Destination) initialContext.lookup(destinationName); } + } catch (JMSException e) { + throw new FlumeException("Could not create destination " + destinationName, e); + } catch (NamingException e) { + throw new FlumeException("Could not find destination " + destinationName, e); + } - try { + try { messageConsumer = session.createConsumer(destination, messageSelector.isEmpty() ? null: messageSelector); } catch (JMSException e) { diff --git a/flume-ng-sources/flume-jms-source/src/main/java/org/apache/flume/source/jms/JMSMessageConsumerFactory.java b/flume-ng-sources/flume-jms-source/src/main/java/org/apache/flume/source/jms/JMSMessageConsumerFactory.java index af2a68a8b7..af74bf4515 100644 --- a/flume-ng-sources/flume-jms-source/src/main/java/org/apache/flume/source/jms/JMSMessageConsumerFactory.java +++ b/flume-ng-sources/flume-jms-source/src/main/java/org/apache/flume/source/jms/JMSMessageConsumerFactory.java @@ -18,19 +18,19 @@ package org.apache.flume.source.jms; import javax.jms.ConnectionFactory; +import javax.naming.InitialContext; import com.google.common.base.Optional; public class JMSMessageConsumerFactory { - JMSMessageConsumer create(ConnectionFactory connectionFactory, - String destinationName, JMSDestinationType destinationType, - String messageSelector, int batchSize, long pollTimeout, - JMSMessageConverter messageConverter, Optional userName, - Optional password) { - return new JMSMessageConsumer(connectionFactory, destinationName, - destinationType, messageSelector, batchSize, pollTimeout, + JMSMessageConsumer create(InitialContext initialContext, ConnectionFactory connectionFactory, + String destinationName, JMSDestinationType destinationType, JMSDestinationLocator destinationLocator, + String messageSelector, int batchSize, long pollTimeout, JMSMessageConverter messageConverter, + Optional userName, Optional password) { + return new JMSMessageConsumer(initialContext, connectionFactory, destinationName, + destinationLocator, destinationType, messageSelector, batchSize, pollTimeout, messageConverter, userName, password); } } diff --git a/flume-ng-sources/flume-jms-source/src/main/java/org/apache/flume/source/jms/JMSSource.java b/flume-ng-sources/flume-jms-source/src/main/java/org/apache/flume/source/jms/JMSSource.java index 6ebb2bb2bf..addd97a670 100644 --- a/flume-ng-sources/flume-jms-source/src/main/java/org/apache/flume/source/jms/JMSSource.java +++ b/flume-ng-sources/flume-jms-source/src/main/java/org/apache/flume/source/jms/JMSSource.java @@ -64,6 +64,7 @@ public class JMSSource extends AbstractPollableSource { private String providerUrl; private String destinationName; private JMSDestinationType destinationType; + private JMSDestinationLocator destinationLocator; private String messageSelector; private Optional userName; private Optional password; @@ -72,6 +73,7 @@ public class JMSSource extends AbstractPollableSource { private long pollTimeout; private int jmsExceptionCounter; + private InitialContext initialContext; public JMSSource() { @@ -101,6 +103,10 @@ protected void doConfigure(Context context) throws FlumeException { String destinationTypeName = context.getString(JMSSourceConfiguration. DESTINATION_TYPE, "").trim().toUpperCase(); + String destinationLocatorName = context.getString(JMSSourceConfiguration. + DESTINATION_LOCATOR, JMSSourceConfiguration.DESTINATION_LOCATOR_DEFAULT) + .trim().toUpperCase(); + messageSelector = context.getString(JMSSourceConfiguration. MESSAGE_SELECTOR, "").trim(); @@ -196,11 +202,16 @@ protected void doConfigure(Context context) throws FlumeException { "invalid.", destinationTypeName), e); } + try { + destinationLocator = JMSDestinationLocator.valueOf(destinationLocatorName); + } catch (IllegalArgumentException e) { + throw new FlumeException(String.format("Destination locator '%s' is " + + "invalid.", destinationLocatorName), e); + } + Preconditions.checkArgument(batchSize > 0, "Batch size must be greater " + "than 0"); - InitialContext initalContext; - try { Properties contextProperties = new Properties(); contextProperties.setProperty( @@ -208,7 +219,7 @@ protected void doConfigure(Context context) throws FlumeException { initialContextFactoryName); contextProperties.setProperty( javax.naming.Context.PROVIDER_URL, providerUrl); - initalContext = initialContextFactory.create(contextProperties); + initialContext = initialContextFactory.create(contextProperties); } catch (NamingException e) { throw new FlumeException(String.format( "Could not create initial context %s provider %s", @@ -216,7 +227,7 @@ protected void doConfigure(Context context) throws FlumeException { } try { - connectionFactory = (ConnectionFactory) initalContext. + connectionFactory = (ConnectionFactory) initialContext. lookup(connectionFactoryName); } catch (NamingException e) { throw new FlumeException("Could not lookup ConnectionFactory", e); @@ -302,8 +313,9 @@ protected synchronized void doStop() { } private JMSMessageConsumer createConsumer() throws JMSException { logger.info("Creating new consumer for " + destinationName); - JMSMessageConsumer consumer = consumerFactory.create(connectionFactory, - destinationName, destinationType, messageSelector, batchSize, + JMSMessageConsumer consumer = consumerFactory.create(initialContext, + connectionFactory, destinationName, destinationType, destinationLocator, + messageSelector, batchSize, pollTimeout, converter, userName, password); jmsExceptionCounter = 0; return consumer; diff --git a/flume-ng-sources/flume-jms-source/src/main/java/org/apache/flume/source/jms/JMSSourceConfiguration.java b/flume-ng-sources/flume-jms-source/src/main/java/org/apache/flume/source/jms/JMSSourceConfiguration.java index c0ec9b6bd7..98bf8ab01e 100644 --- a/flume-ng-sources/flume-jms-source/src/main/java/org/apache/flume/source/jms/JMSSourceConfiguration.java +++ b/flume-ng-sources/flume-jms-source/src/main/java/org/apache/flume/source/jms/JMSSourceConfiguration.java @@ -29,6 +29,9 @@ public class JMSSourceConfiguration { public static final String DESTINATION_NAME = "destinationName"; public static final String DESTINATION_TYPE = "destinationType"; + public static final String DESTINATION_LOCATOR = "destinationLocator"; + public static final String DESTINATION_LOCATOR_DEFAULT = "CDI"; + public static final String DESTINATION_TYPE_QUEUE = "queue"; public static final String DESTINATION_TYPE_TOPIC = "topic"; diff --git a/flume-ng-sources/flume-jms-source/src/test/java/org/apache/flume/source/jms/JMSMessageConsumerTestBase.java b/flume-ng-sources/flume-jms-source/src/test/java/org/apache/flume/source/jms/JMSMessageConsumerTestBase.java index e40e95a1be..68819677b3 100644 --- a/flume-ng-sources/flume-jms-source/src/test/java/org/apache/flume/source/jms/JMSMessageConsumerTestBase.java +++ b/flume-ng-sources/flume-jms-source/src/test/java/org/apache/flume/source/jms/JMSMessageConsumerTestBase.java @@ -32,6 +32,7 @@ import javax.jms.Session; import javax.jms.TextMessage; import javax.jms.Topic; +import javax.naming.InitialContext; import org.apache.flume.Context; import org.apache.flume.Event; @@ -47,12 +48,14 @@ public abstract class JMSMessageConsumerTestBase { static final String DESTINATION_NAME = "destinationName"; static final String SELECTOR = "selector"; static final String TEXT = "text"; + static final InitialContext WONT_USE = null; Context context; JMSMessageConsumer consumer; ConnectionFactory connectionFactory; String destinationName; JMSDestinationType destinationType; + JMSDestinationLocator destinationLocator; String messageSelector; int batchSize; long pollTimeout; @@ -100,6 +103,7 @@ public Object nextElement() { when(messageConsumer.receive(anyLong())).thenReturn(message); destinationName = DESTINATION_NAME; destinationType = JMSDestinationType.QUEUE; + destinationLocator = JMSDestinationLocator.CDI; messageSelector = SELECTOR; batchSize = 10; pollTimeout = 500L; @@ -129,9 +133,9 @@ void assertBodyIsExpected(List events) { } JMSMessageConsumer create() { - return new JMSMessageConsumer(connectionFactory, destinationName, - destinationType, messageSelector, batchSize, pollTimeout, converter, - userName, password); + return new JMSMessageConsumer(WONT_USE, connectionFactory, destinationName, + destinationLocator, destinationType, messageSelector, batchSize, + pollTimeout, converter, userName, password); } @After public void tearDown() throws Exception { diff --git a/flume-ng-sources/flume-jms-source/src/test/java/org/apache/flume/source/jms/TestIntegrationActiveMQ.java b/flume-ng-sources/flume-jms-source/src/test/java/org/apache/flume/source/jms/TestIntegrationActiveMQ.java index 20c0d2e644..e28e02ab4f 100644 --- a/flume-ng-sources/flume-jms-source/src/test/java/org/apache/flume/source/jms/TestIntegrationActiveMQ.java +++ b/flume-ng-sources/flume-jms-source/src/test/java/org/apache/flume/source/jms/TestIntegrationActiveMQ.java @@ -55,11 +55,14 @@ import com.google.common.io.Files; public class TestIntegrationActiveMQ { + private final static String INITIAL_CONTEXT_FACTORY = "org.apache.activemq.jndi.ActiveMQInitialContextFactory"; public static final String BROKER_BIND_URL = "tcp://localhost:61516"; private final static String DESTINATION_NAME = "test"; private static final String USERNAME = "user"; private static final String PASSWORD = "pass"; + // specific for dynamic queues on ActiveMq + public static final String JNDI_PREFIX = "dynamicQueues/"; private File baseDir; private File tmpDir; @@ -170,6 +173,15 @@ private void putTopic(List events) throws Exception { connection.close(); } + @Test + public void testQueueLocatedWithJndi() throws Exception { + context.put(JMSSourceConfiguration.DESTINATION_NAME, + JNDI_PREFIX + DESTINATION_NAME); + context.put(JMSSourceConfiguration.DESTINATION_LOCATOR, + JMSDestinationLocator.JNDI.name()); + testQueue(); + } + @Test public void testQueue() throws Exception { context.put(JMSSourceConfiguration.DESTINATION_TYPE, diff --git a/flume-ng-sources/flume-jms-source/src/test/java/org/apache/flume/source/jms/TestJMSSource.java b/flume-ng-sources/flume-jms-source/src/test/java/org/apache/flume/source/jms/TestJMSSource.java index ddfd76795c..5423f8f369 100644 --- a/flume-ng-sources/flume-jms-source/src/test/java/org/apache/flume/source/jms/TestJMSSource.java +++ b/flume-ng-sources/flume-jms-source/src/test/java/org/apache/flume/source/jms/TestJMSSource.java @@ -54,7 +54,7 @@ public class TestJMSSource extends JMSMessageConsumerTestBase { private JMSSource source; private Context context; - private InitialContext initialConext; + private InitialContext initialContext; private ChannelProcessor channelProcessor; private List events; private JMSMessageConsumerFactory consumerFactory; @@ -67,7 +67,7 @@ void afterSetup() throws Exception { baseDir = Files.createTempDir(); passwordFile = new File(baseDir, "password"); Assert.assertTrue(passwordFile.createNewFile()); - initialConext = mock(InitialContext.class); + initialContext = mock(InitialContext.class); channelProcessor = mock(ChannelProcessor.class); events = Lists.newArrayList(); doAnswer(new Answer() { @@ -79,13 +79,13 @@ public Void answer(InvocationOnMock invocation) throws Throwable { }).when(channelProcessor).processEventBatch(any(List.class)); consumerFactory = mock(JMSMessageConsumerFactory.class); consumer = spy(create()); - when(consumerFactory.create(any(ConnectionFactory.class), anyString(), - any(JMSDestinationType.class), anyString(), anyInt(), anyLong(), + when(consumerFactory.create(any(InitialContext.class), any(ConnectionFactory.class), anyString(), + any(JMSDestinationType.class), any(JMSDestinationLocator.class), anyString(), anyInt(), anyLong(), any(JMSMessageConverter.class), any(Optional.class), any(Optional.class))).thenReturn(consumer); - when(initialConext.lookup(anyString())).thenReturn(connectionFactory); + when(initialContext.lookup(anyString())).thenReturn(connectionFactory); contextFactory = mock(InitialContextFactory.class); - when(contextFactory.create(any(Properties.class))).thenReturn(initialConext); + when(contextFactory.create(any(Properties.class))).thenReturn(initialContext); source = new JMSSource(consumerFactory, contextFactory); source.setName("JMSSource-" + UUID.randomUUID()); source.setChannelProcessor(channelProcessor); @@ -136,8 +136,8 @@ public void testConfigureWithEmptyDestinationType() throws Exception { @SuppressWarnings("unchecked") @Test public void testStartConsumerCreateThrowsException() throws Exception { - when(consumerFactory.create(any(ConnectionFactory.class), anyString(), - any(JMSDestinationType.class), anyString(), anyInt(), anyLong(), + when(consumerFactory.create(any(InitialContext.class), any(ConnectionFactory.class), anyString(), + any(JMSDestinationType.class), any(JMSDestinationLocator.class), anyString(), anyInt(), anyLong(), any(JMSMessageConverter.class), any(Optional.class), any(Optional.class))).thenThrow(new RuntimeException()); source.configure(context); @@ -151,7 +151,7 @@ public void testStartConsumerCreateThrowsException() throws Exception { } @Test(expected = FlumeException.class) public void testConfigureWithContextLookupThrowsException() throws Exception { - when(initialConext.lookup(anyString())).thenThrow(new NamingException()); + when(initialContext.lookup(anyString())).thenThrow(new NamingException()); source.configure(context); } @Test(expected = FlumeException.class) From e07a0a6883c84836e618d187c1381d47a26bfc71 Mon Sep 17 00:00:00 2001 From: Hari Shreedharan Date: Fri, 7 Feb 2014 12:32:56 -0800 Subject: [PATCH 071/341] FLUME-2130. Handle larger payloads via SyslogUDPSource (Ashish Paliwal via Hari Shreedharan) --- .../apache/flume/source/SyslogUDPSource.java | 19 ++++--- .../flume/source/TestSyslogUdpSource.java | 52 +++++++++++++++++++ 2 files changed, 63 insertions(+), 8 deletions(-) diff --git a/flume-ng-core/src/main/java/org/apache/flume/source/SyslogUDPSource.java b/flume-ng-core/src/main/java/org/apache/flume/source/SyslogUDPSource.java index 8fb251bf0e..01b8905513 100644 --- a/flume-ng-core/src/main/java/org/apache/flume/source/SyslogUDPSource.java +++ b/flume-ng-core/src/main/java/org/apache/flume/source/SyslogUDPSource.java @@ -35,13 +35,7 @@ import org.apache.flume.source.SyslogUtils; import org.jboss.netty.bootstrap.ConnectionlessBootstrap; import org.jboss.netty.buffer.ChannelBuffer; -import org.jboss.netty.channel.Channel; -import org.jboss.netty.channel.ChannelHandlerContext; -import org.jboss.netty.channel.ChannelPipeline; -import org.jboss.netty.channel.ChannelPipelineFactory; -import org.jboss.netty.channel.Channels; -import org.jboss.netty.channel.MessageEvent; -import org.jboss.netty.channel.SimpleChannelHandler; +import org.jboss.netty.channel.*; import org.jboss.netty.channel.socket.oio.OioDatagramChannelFactory; import org.slf4j.Logger; @@ -61,8 +55,14 @@ public class SyslogUDPSource extends AbstractSource .getLogger(SyslogUDPSource.class); private CounterGroup counterGroup = new CounterGroup(); + + // Default Min size + public static final int DEFAULT_MIN_SIZE = 2048; + public static final int DEFAULT_INITIAL_SIZE = DEFAULT_MIN_SIZE; + public class syslogHandler extends SimpleChannelHandler { - private SyslogUtils syslogUtils = new SyslogUtils(true); + private SyslogUtils syslogUtils = new SyslogUtils(DEFAULT_INITIAL_SIZE, + SyslogSourceConfigurationConstants.DEFAULT_KEEP_FIELDS, true); public void setFormater(Map prop) { syslogUtils.addFormats(prop); @@ -98,6 +98,9 @@ public void start() { final syslogHandler handler = new syslogHandler(); handler.setFormater(formaterProp); handler.setKeepFields(keepFields); + serverBootstrap.setOption("receiveBufferSizePredictorFactory", + new AdaptiveReceiveBufferSizePredictorFactory(DEFAULT_MIN_SIZE, + DEFAULT_INITIAL_SIZE, maxsize)); serverBootstrap.setPipelineFactory(new ChannelPipelineFactory() { @Override public ChannelPipeline getPipeline() { diff --git a/flume-ng-core/src/test/java/org/apache/flume/source/TestSyslogUdpSource.java b/flume-ng-core/src/test/java/org/apache/flume/source/TestSyslogUdpSource.java index 36f6479cf9..95ee48c4cb 100644 --- a/flume-ng-core/src/test/java/org/apache/flume/source/TestSyslogUdpSource.java +++ b/flume-ng-core/src/test/java/org/apache/flume/source/TestSyslogUdpSource.java @@ -25,6 +25,7 @@ import java.net.InetAddress; import java.net.DatagramSocket; import com.google.common.base.Charsets; +import com.google.common.base.Strings; import org.apache.flume.Channel; import org.apache.flume.ChannelSelector; import org.apache.flume.Context; @@ -123,6 +124,49 @@ private void runKeepFieldsTest(boolean keepFields) throws IOException { } } + @Test + public void testLargePayload() throws Exception { + init(true); + source.start(); + // Write some message to the syslog port + + byte[] largePayload = getPayload(1000).getBytes(); + + DatagramSocket syslogSocket; + DatagramPacket datagramPacket; + datagramPacket = new DatagramPacket(largePayload, + 1000, + InetAddress.getLocalHost(), source.getSourcePort()); + for (int i = 0; i < 10 ; i++) { + syslogSocket = new DatagramSocket(); + syslogSocket.send(datagramPacket); + syslogSocket.close(); + } + + List channelEvents = new ArrayList(); + Transaction txn = channel.getTransaction(); + txn.begin(); + for (int i = 0; i < 10; i++) { + Event e = channel.take(); + Assert.assertNotNull(e); + channelEvents.add(e); + } + + try { + txn.commit(); + } catch (Throwable t) { + txn.rollback(); + } finally { + txn.close(); + } + + source.stop(); + for (Event e : channelEvents) { + Assert.assertNotNull(e); + Assert.assertArrayEquals(largePayload, e.getBody()); + } + } + @Test public void testKeepFields() throws IOException { runKeepFieldsTest(true); @@ -132,5 +176,13 @@ public void testKeepFields() throws IOException { public void testRemoveFields() throws IOException { runKeepFieldsTest(false); } + + private String getPayload(int length) { + StringBuilder payload = new StringBuilder(length); + for (int n = 0; n < length; ++n) { + payload.append("x"); + } + return payload.toString(); + } } From ba0b2685b96747a60a3e318b1e2fc2c9c02bcea4 Mon Sep 17 00:00:00 2001 From: Jarek Jarcec Cecho Date: Sun, 9 Feb 2014 09:34:50 -0800 Subject: [PATCH 072/341] FLUME-2314. Upgrade to Mapdb 0.9.9 (Hari Shreedharan via Jarek Jarcec Cecho) --- pom.xml | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/pom.xml b/pom.xml index d71239ad1a..362fb4555d 100644 --- a/pom.xml +++ b/pom.xml @@ -786,7 +786,7 @@ limitations under the License. org.mapdb mapdb - 0.9.8 + 0.9.9 From b4ddd5829897f758f869a5fc3b08dcbf4b55156a Mon Sep 17 00:00:00 2001 From: Jarek Jarcec Cecho Date: Mon, 10 Feb 2014 13:23:49 -0800 Subject: [PATCH 073/341] FLUME-2307. Remove Log writetimeout (Hari Shreedharan via Jarek Jarcec Cecho) --- .../flume/channel/file/FileChannel.java | 82 +++------------- .../file/FileChannelConfiguration.java | 13 --- .../org/apache/flume/channel/file/Log.java | 96 +++++-------------- flume-ng-doc/sphinx/FlumeUserGuide.rst | 2 - 4 files changed, 38 insertions(+), 155 deletions(-) diff --git a/flume-ng-channels/flume-file-channel/src/main/java/org/apache/flume/channel/file/FileChannel.java b/flume-ng-channels/flume-file-channel/src/main/java/org/apache/flume/channel/file/FileChannel.java index 2cd7f03b36..71b26f77fa 100644 --- a/flume-ng-channels/flume-file-channel/src/main/java/org/apache/flume/channel/file/FileChannel.java +++ b/flume-ng-channels/flume-file-channel/src/main/java/org/apache/flume/channel/file/FileChannel.java @@ -90,8 +90,6 @@ public class FileChannel extends BasicChannelSemantics { private Semaphore queueRemaining; private final ThreadLocal transactions = new ThreadLocal(); - private int logWriteTimeout; - private int checkpointWriteTimeout; private String channelNameDescriptor = "[channel=unknown]"; private ChannelCounter channelCounter; private boolean useLogReplayV1; @@ -190,39 +188,14 @@ public void configure(Context context) { // cannot be over FileChannelConfiguration.DEFAULT_MAX_FILE_SIZE maxFileSize = Math.min( - context.getLong(FileChannelConfiguration.MAX_FILE_SIZE, - FileChannelConfiguration.DEFAULT_MAX_FILE_SIZE), - FileChannelConfiguration.DEFAULT_MAX_FILE_SIZE); + context.getLong(FileChannelConfiguration.MAX_FILE_SIZE, + FileChannelConfiguration.DEFAULT_MAX_FILE_SIZE), + FileChannelConfiguration.DEFAULT_MAX_FILE_SIZE); minimumRequiredSpace = Math.max( - context.getLong(FileChannelConfiguration.MINIMUM_REQUIRED_SPACE, - FileChannelConfiguration.DEFAULT_MINIMUM_REQUIRED_SPACE), - FileChannelConfiguration.FLOOR_MINIMUM_REQUIRED_SPACE); - - logWriteTimeout = context.getInteger( - FileChannelConfiguration.LOG_WRITE_TIMEOUT, - FileChannelConfiguration.DEFAULT_WRITE_TIMEOUT); - - if (logWriteTimeout < 0) { - LOG.warn("Log write time out is invalid: " + logWriteTimeout - + ", using default: " - + FileChannelConfiguration.DEFAULT_WRITE_TIMEOUT); - - logWriteTimeout = FileChannelConfiguration.DEFAULT_WRITE_TIMEOUT; - } - - checkpointWriteTimeout = context.getInteger( - FileChannelConfiguration.CHECKPOINT_WRITE_TIMEOUT, - FileChannelConfiguration.DEFAULT_CHECKPOINT_WRITE_TIMEOUT); - - if (checkpointWriteTimeout < 0) { - LOG.warn("Checkpoint write time out is invalid: " + checkpointWriteTimeout - + ", using default: " - + FileChannelConfiguration.DEFAULT_CHECKPOINT_WRITE_TIMEOUT); - - checkpointWriteTimeout = - FileChannelConfiguration.DEFAULT_CHECKPOINT_WRITE_TIMEOUT; - } + context.getLong(FileChannelConfiguration.MINIMUM_REQUIRED_SPACE, + FileChannelConfiguration.DEFAULT_MINIMUM_REQUIRED_SPACE), + FileChannelConfiguration.FLOOR_MINIMUM_REQUIRED_SPACE); useLogReplayV1 = context.getBoolean( FileChannelConfiguration.USE_LOG_REPLAY_V1, @@ -285,11 +258,9 @@ public synchronized void start() { builder.setMaxFileSize(maxFileSize); builder.setMinimumRequiredSpace(minimumRequiredSpace); builder.setQueueSize(capacity); - builder.setLogWriteTimeout(logWriteTimeout); builder.setCheckpointDir(checkpointDir); builder.setLogDirs(dataDirs); builder.setChannelName(getName()); - builder.setCheckpointWriteTimeout(checkpointWriteTimeout); builder.setUseLogReplayV1(useLogReplayV1); builder.setUseFastReplay(useFastReplay); builder.setEncryptionKeyProvider(encryptionKeyProvider); @@ -471,13 +442,8 @@ protected void doPut(Event event) throws InterruptedException { + channelNameDescriptor); } boolean success = false; - boolean lockAcquired = log.tryLockShared(); + log.lockShared(); try { - if(!lockAcquired) { - throw new ChannelException("Failed to obtain lock for writing to the " - + "log. Try increasing the log write timeout value. " + - channelNameDescriptor); - } FlumeEventPointer ptr = log.put(transactionID, event); Preconditions.checkState(putList.offer(ptr), "putList offer failed " + channelNameDescriptor); @@ -487,9 +453,7 @@ protected void doPut(Event event) throws InterruptedException { throw new ChannelException("Put failed due to IO error " + channelNameDescriptor, e); } finally { - if(lockAcquired) { - log.unlockShared(); - } + log.unlockShared(); if(!success) { // release slot obtained in the case // the put fails for any reason @@ -507,12 +471,7 @@ protected Event doTake() throws InterruptedException { "increasing capacity, or increasing thread count. " + channelNameDescriptor); } - if(!log.tryLockShared()) { - throw new ChannelException("Failed to obtain lock for writing to the " - + "log. Try increasing the log write timeout value. " + - channelNameDescriptor); - } - + log.lockShared(); /* * 1. Take an event which is in the queue. * 2. If getting that event does not throw NoopRecordException, @@ -557,11 +516,7 @@ protected void doCommit() throws InterruptedException { if(puts > 0) { Preconditions.checkState(takes == 0, "nonzero puts and takes " + channelNameDescriptor); - if(!log.tryLockShared()) { - throw new ChannelException("Failed to obtain lock for writing to the " - + "log. Try increasing the log write timeout value. " + - channelNameDescriptor); - } + log.lockShared(); try { log.commitPut(transactionID); channelCounter.addToEventPutSuccessCount(puts); @@ -589,11 +544,7 @@ protected void doCommit() throws InterruptedException { } } else if (takes > 0) { - if(!log.tryLockShared()) { - throw new ChannelException("Failed to obtain lock for writing to the " - + "log. Try increasing the log write timeout value. " + - channelNameDescriptor); - } + log.lockShared(); try { log.commitTake(transactionID); queue.completeTransaction(transactionID); @@ -614,13 +565,8 @@ protected void doCommit() throws InterruptedException { protected void doRollback() throws InterruptedException { int puts = putList.size(); int takes = takeList.size(); - boolean lockAcquired = log.tryLockShared(); + log.lockShared(); try { - if(!lockAcquired) { - throw new ChannelException("Failed to obtain lock for writing to the " - + "log. Try increasing the log write timeout value. " + - channelNameDescriptor); - } if(takes > 0) { Preconditions.checkState(puts == 0, "nonzero puts and takes " + channelNameDescriptor); @@ -641,9 +587,7 @@ protected void doRollback() throws InterruptedException { throw new ChannelException("Commit failed due to IO error " + channelNameDescriptor, e); } finally { - if(lockAcquired) { - log.unlockShared(); - } + log.unlockShared(); // since rollback is being called, puts will never make it on // to the queue and we need to be sure to release the resources queueRemaining.release(puts); diff --git a/flume-ng-channels/flume-file-channel/src/main/java/org/apache/flume/channel/file/FileChannelConfiguration.java b/flume-ng-channels/flume-file-channel/src/main/java/org/apache/flume/channel/file/FileChannelConfiguration.java index 10ca11f1eb..e4bc879a24 100644 --- a/flume-ng-channels/flume-file-channel/src/main/java/org/apache/flume/channel/file/FileChannelConfiguration.java +++ b/flume-ng-channels/flume-file-channel/src/main/java/org/apache/flume/channel/file/FileChannelConfiguration.java @@ -75,19 +75,6 @@ public class FileChannelConfiguration { public static final String KEEP_ALIVE = "keep-alive"; public static final int DEFAULT_KEEP_ALIVE = 3; - /** - * The amount of time in seconds a writer will wait before failing when - * checkpoint is enqueued or in progress. - */ - public static final String LOG_WRITE_TIMEOUT = "write-timeout"; - public static final int DEFAULT_WRITE_TIMEOUT = 10; - - /** - * The amount of time in seconds the channel should wait to write the - * checkpoint when some other operation(s) are enqueued or in progress. - */ - public static final String CHECKPOINT_WRITE_TIMEOUT = "checkpoint-timeout"; - public static final int DEFAULT_CHECKPOINT_WRITE_TIMEOUT = 600; /** * Turn on Flume 1.2 log replay logic */ diff --git a/flume-ng-channels/flume-file-channel/src/main/java/org/apache/flume/channel/file/Log.java b/flume-ng-channels/flume-file-channel/src/main/java/org/apache/flume/channel/file/Log.java index 70106cbd39..579ee35a8b 100644 --- a/flume-ng-channels/flume-file-channel/src/main/java/org/apache/flume/channel/file/Log.java +++ b/flume-ng-channels/flume-file-channel/src/main/java/org/apache/flume/channel/file/Log.java @@ -26,7 +26,6 @@ import com.google.common.collect.Sets; import com.google.common.util.concurrent.ThreadFactoryBuilder; import org.apache.commons.io.FileUtils; -import org.apache.flume.ChannelException; import org.apache.flume.Event; import org.apache.flume.annotations.InterfaceAudience; import org.apache.flume.annotations.InterfaceStability; @@ -66,8 +65,8 @@ * the on disk write ahead log with the last checkpoint of the queue. * * Before calling any of commitPut/commitTake/get/put/rollback/take - * Log.tryLockShared should be called and the above operations - * should only be called if tryLockShared returns true. After + * {@linkplain org.apache.flume.channel.file.Log#lockShared()} + * should be called. After * the operation and any additional modifications of the * FlumeEventQueue, the Log.unlockShared method should be called. */ @@ -114,9 +113,7 @@ public class Log { * Exclusive lock */ private final WriteLock checkpointWriterLock = checkpointLock.writeLock(); - private int logWriteTimeout; private final String channelNameDescriptor; - private int checkpointWriteTimeout; private boolean useLogReplayV1; private KeyProvider encryptionKeyProvider; private String encryptionCipherProvider; @@ -143,11 +140,7 @@ static class Builder { private int bQueueCapacity; private File bCheckpointDir; private File[] bLogDirs; - private int bLogWriteTimeout = - FileChannelConfiguration.DEFAULT_WRITE_TIMEOUT; private String bName; - private int bCheckpointWriteTimeout = - FileChannelConfiguration.DEFAULT_CHECKPOINT_WRITE_TIMEOUT; private boolean useLogReplayV1; private boolean useFastReplay; private KeyProvider bEncryptionKeyProvider; @@ -187,11 +180,6 @@ Builder setLogDirs(File[] dirs) { return this; } - Builder setLogWriteTimeout(int timeout) { - bLogWriteTimeout = timeout; - return this; - } - Builder setChannelName(String name) { bName = name; return this; @@ -202,11 +190,6 @@ Builder setMinimumRequiredSpace(long minimumRequiredSpace) { return this; } - Builder setCheckpointWriteTimeout(int checkpointTimeout){ - bCheckpointWriteTimeout = checkpointTimeout; - return this; - } - Builder setUseLogReplayV1(boolean useLogReplayV1){ this.useLogReplayV1 = useLogReplayV1; return this; @@ -244,23 +227,21 @@ Builder setBackupCheckpointDir(File backupCheckpointDir) { Log build() throws IOException { return new Log(bCheckpointInterval, bMaxFileSize, bQueueCapacity, - bLogWriteTimeout, bCheckpointWriteTimeout, bUseDualCheckpoints, - bCheckpointDir, bBackupCheckpointDir, bName, - useLogReplayV1, useFastReplay, bMinimumRequiredSpace, - bEncryptionKeyProvider, bEncryptionKeyAlias, - bEncryptionCipherProvider, bUsableSpaceRefreshInterval, - bLogDirs); + bUseDualCheckpoints, bCheckpointDir, bBackupCheckpointDir, bName, + useLogReplayV1, useFastReplay, bMinimumRequiredSpace, + bEncryptionKeyProvider, bEncryptionKeyAlias, + bEncryptionCipherProvider, bUsableSpaceRefreshInterval, + bLogDirs); } } private Log(long checkpointInterval, long maxFileSize, int queueCapacity, - int logWriteTimeout, int checkpointWriteTimeout, - boolean useDualCheckpoints, File checkpointDir, File backupCheckpointDir, - String name, boolean useLogReplayV1, boolean useFastReplay, - long minimumRequiredSpace, @Nullable KeyProvider encryptionKeyProvider, - @Nullable String encryptionKeyAlias, - @Nullable String encryptionCipherProvider, - long usableSpaceRefreshInterval, File... logDirs) + boolean useDualCheckpoints, File checkpointDir, File backupCheckpointDir, + String name, boolean useLogReplayV1, boolean useFastReplay, + long minimumRequiredSpace, @Nullable KeyProvider encryptionKeyProvider, + @Nullable String encryptionKeyAlias, + @Nullable String encryptionCipherProvider, + long usableSpaceRefreshInterval, File... logDirs) throws IOException { Preconditions.checkArgument(checkpointInterval > 0, "checkpointInterval <= 0"); @@ -337,8 +318,6 @@ private Log(long checkpointInterval, long maxFileSize, int queueCapacity, this.checkpointDir = checkpointDir; this.backupCheckpointDir = backupCheckpointDir; this.logDirs = logDirs; - this.logWriteTimeout = logWriteTimeout; - this.checkpointWriteTimeout = checkpointWriteTimeout; logFiles = new AtomicReferenceArray(this.logDirs.length); workerExecutor = Executors.newSingleThreadScheduledExecutor(new ThreadFactoryBuilder().setNameFormat("Log-BackgroundWorker-" + name) @@ -356,9 +335,7 @@ private Log(long checkpointInterval, long maxFileSize, int queueCapacity, void replay() throws IOException { Preconditions.checkState(!open, "Cannot replay after Log has been opened"); - Preconditions.checkState(tryLockExclusive(), "Cannot obtain lock on " - + channelNameDescriptor); - + lockExclusive(); try { /* * First we are going to look through the data directories @@ -751,28 +728,12 @@ void commitTake(long transactionID) throws IOException, } - private boolean tryLockExclusive() { - try { - return checkpointWriterLock.tryLock(checkpointWriteTimeout, - TimeUnit.SECONDS); - } catch (InterruptedException ex) { - LOGGER.warn("Interrupted while waiting for log exclusive lock", ex); - Thread.currentThread().interrupt(); - } - return false; - } private void unlockExclusive() { checkpointWriterLock.unlock(); } - boolean tryLockShared() { - try { - return checkpointReadLock.tryLock(logWriteTimeout, TimeUnit.SECONDS); - } catch (InterruptedException ex) { - LOGGER.warn("Interrupted while waiting for log shared lock", ex); - Thread.currentThread().interrupt(); - } - return false; + void lockShared() { + checkpointReadLock.lock(); } void unlockShared() { @@ -929,29 +890,25 @@ private void roll(int index) throws IOException { * @param index * @throws IOException */ - private synchronized void roll(int index, ByteBuffer buffer) - throws IOException { - if (!tryLockShared()) { - throw new ChannelException("Failed to obtain lock for writing to the " - + "log. Try increasing the log write timeout value. " + - channelNameDescriptor); - } + private synchronized void roll(int index, ByteBuffer buffer) + throws IOException { + lockShared(); try { LogFile.Writer oldLogFile = logFiles.get(index); // check to make sure a roll is actually required due to // the possibility of multiple writes waiting on lock - if(oldLogFile == null || buffer == null || - oldLogFile.isRollRequired(buffer)) { + if (oldLogFile == null || buffer == null || + oldLogFile.isRollRequired(buffer)) { try { LOGGER.info("Roll start " + logDirs[index]); int fileID = nextFileID.incrementAndGet(); File file = new File(logDirs[index], PREFIX + fileID); LogFile.Writer writer = LogFileFactory.getWriter(file, fileID, - maxFileSize, encryptionKey, encryptionKeyAlias, - encryptionCipherProvider, usableSpaceRefreshInterval); + maxFileSize, encryptionKey, encryptionKeyAlias, + encryptionCipherProvider, usableSpaceRefreshInterval); idLogFileMap.put(fileID, LogFileFactory.getRandomReader(file, - encryptionKeyProvider)); + encryptionKeyProvider)); // writer from this point on will get new reference logFiles.set(index, writer); // close out old log @@ -988,10 +945,7 @@ private Boolean writeCheckpoint(Boolean force) throws Exception { throw new IOException("Usable space exhaused, only " + usableSpace + " bytes remaining, required " + minimumRequiredSpace + " bytes"); } - boolean lockAcquired = tryLockExclusive(); - if(!lockAcquired) { - return false; - } + lockExclusive(); SortedSet logFileRefCountsAll = null, logFileRefCountsActive = null; try { if (queue.checkpoint(force)) { diff --git a/flume-ng-doc/sphinx/FlumeUserGuide.rst b/flume-ng-doc/sphinx/FlumeUserGuide.rst index d120a7445b..1ec5a22b73 100644 --- a/flume-ng-doc/sphinx/FlumeUserGuide.rst +++ b/flume-ng-doc/sphinx/FlumeUserGuide.rst @@ -2180,8 +2180,6 @@ maxFileSize 2146435071 minimumRequiredSpace 524288000 Minimum Required free space (in bytes). To avoid data corruption, File Channel stops accepting take/put requests when free space drops below this value capacity 1000000 Maximum capacity of the channel keep-alive 3 Amount of time (in sec) to wait for a put operation -write-timeout 10 Amount of time (in sec) to wait for a write operation -checkpoint-timeout 600 Expert: Amount of time (in sec) to wait for a checkpoint use-log-replay-v1 false Expert: Use old replay logic use-fast-replay false Expert: Replay without using queue encryption.activeKey -- Key name used to encrypt new data From 89470c6a62b54b5c1411d519ebc952e557dc5a9f Mon Sep 17 00:00:00 2001 From: Juhani Connolly Date: Mon, 17 Feb 2014 13:54:13 +0900 Subject: [PATCH 074/341] FLUME-2319. Fix Ganglia documentation error (Satoshi Iijima via Juhani Connolly) --- flume-ng-doc/sphinx/FlumeUserGuide.rst | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/flume-ng-doc/sphinx/FlumeUserGuide.rst b/flume-ng-doc/sphinx/FlumeUserGuide.rst index 1ec5a22b73..8390cd21a3 100644 --- a/flume-ng-doc/sphinx/FlumeUserGuide.rst +++ b/flume-ng-doc/sphinx/FlumeUserGuide.rst @@ -2996,7 +2996,7 @@ Property Name Default Description ======================= ======= ===================================================================================== **type** -- The component type name, has to be ``ganglia`` **hosts** -- Comma-separated list of ``hostname:port`` of Ganglia servers -pollInterval 60 Time, in seconds, between consecutive reporting to Ganglia server +pollFrequency 60 Time, in seconds, between consecutive reporting to Ganglia server isGanglia3 false Ganglia server version is 3. By default, Flume sends in Ganglia 3.1 format ======================= ======= ===================================================================================== From f32fe79c465d15e813bd54cf1bf5ac59157c1dca Mon Sep 17 00:00:00 2001 From: Hari Shreedharan Date: Tue, 18 Feb 2014 17:16:00 -0800 Subject: [PATCH 075/341] FLUME-2325. Bucketwriter must be permanently closed only on idle and roll timeouts. (chenshangan via Hari Shreedharan) --- .../main/java/org/apache/flume/sink/hdfs/BucketWriter.java | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/flume-ng-sinks/flume-hdfs-sink/src/main/java/org/apache/flume/sink/hdfs/BucketWriter.java b/flume-ng-sinks/flume-hdfs-sink/src/main/java/org/apache/flume/sink/hdfs/BucketWriter.java index 7fb3bdd760..e82d13df05 100644 --- a/flume-ng-sinks/flume-hdfs-sink/src/main/java/org/apache/flume/sink/hdfs/BucketWriter.java +++ b/flume-ng-sinks/flume-hdfs-sink/src/main/java/org/apache/flume/sink/hdfs/BucketWriter.java @@ -328,8 +328,8 @@ public Void call() throws Exception { } if (callCloseCallback) { runCloseAction(); - } - closed = true; + closed = true; + } } /** From ec40929596b06b4ec608a8199f6be2f807a865c6 Mon Sep 17 00:00:00 2001 From: Hari Shreedharan Date: Thu, 27 Feb 2014 11:06:25 -0800 Subject: [PATCH 076/341] FLUME-2320. Fixed Deadlock in DatasetSink (Ryan Blue via Hari Shreedharan) --- flume-ng-doc/sphinx/FlumeUserGuide.rst | 5 +- .../apache/flume/sink/kite/DatasetSink.java | 128 ++++++------------ .../flume/sink/kite/DatasetSinkConstants.java | 5 - .../flume/sink/kite/TestDatasetSink.java | 2 - 4 files changed, 44 insertions(+), 96 deletions(-) diff --git a/flume-ng-doc/sphinx/FlumeUserGuide.rst b/flume-ng-doc/sphinx/FlumeUserGuide.rst index 8390cd21a3..cd43634a63 100644 --- a/flume-ng-doc/sphinx/FlumeUserGuide.rst +++ b/flume-ng-doc/sphinx/FlumeUserGuide.rst @@ -2038,7 +2038,10 @@ may be found (``hdfs:/...`` URIs are supported). This is compatible with the Log4jAppender flume client and the spooling directory source's Avro deserializer using ``deserializer.schemaType = LITERAL``. -Note: The ``flume.avro.schema.hash`` header is **not supported**. +Note 1: The ``flume.avro.schema.hash`` header is **not supported**. +Note 2: In some cases, file rolling may occur slightly after the roll interval +has been exceeded. However, this delay will not exceed 5 seconds. In most +cases, the delay is neglegible. ===================== ======= =========================================================== Property Name Default Description diff --git a/flume-ng-sinks/flume-dataset-sink/src/main/java/org/apache/flume/sink/kite/DatasetSink.java b/flume-ng-sinks/flume-dataset-sink/src/main/java/org/apache/flume/sink/kite/DatasetSink.java index 9a00fb1c8c..1ee0a1fe4e 100644 --- a/flume-ng-sinks/flume-dataset-sink/src/main/java/org/apache/flume/sink/kite/DatasetSink.java +++ b/flume-ng-sinks/flume-dataset-sink/src/main/java/org/apache/flume/sink/kite/DatasetSink.java @@ -18,13 +18,13 @@ package org.apache.flume.sink.kite; +import com.google.common.annotations.VisibleForTesting; import com.google.common.base.Preconditions; import com.google.common.base.Throwables; import com.google.common.cache.CacheBuilder; import com.google.common.cache.CacheLoader; import com.google.common.cache.LoadingCache; import com.google.common.collect.Lists; -import com.google.common.util.concurrent.ThreadFactoryBuilder; import java.io.IOException; import java.io.InputStream; import java.net.URI; @@ -32,10 +32,6 @@ import java.util.List; import java.util.Map; import java.util.concurrent.ExecutionException; -import java.util.concurrent.Executors; -import java.util.concurrent.ScheduledExecutorService; -import java.util.concurrent.TimeUnit; -import java.util.concurrent.locks.ReentrantLock; import org.apache.avro.Schema; import org.apache.avro.io.BinaryDecoder; import org.apache.avro.io.DecoderFactory; @@ -70,11 +66,6 @@ public class DatasetSink extends AbstractSink implements Configurable { static Configuration conf = new Configuration(); - /** - * Lock used to protect access to the current writer - */ - private final ReentrantLock writerLock = new ReentrantLock(true); - private String repositoryURI = null; private String datasetName = null; private long batchSize = DatasetSinkConstants.DEFAULT_BATCH_SIZE; @@ -83,8 +74,8 @@ public class DatasetSink extends AbstractSink implements Configurable { private SinkCounter counter = null; // for rolling files at a given interval - private ScheduledExecutorService rollTimer; - private int rollInterval = DatasetSinkConstants.DEFAULT_ROLL_INTERVAL; + private int rollIntervalS = DatasetSinkConstants.DEFAULT_ROLL_INTERVAL; + private long lastRolledMs = 0l; // for working with avro serialized records private Object datum = null; @@ -156,7 +147,7 @@ public void configure(Context context) { this.batchSize = context.getLong( DatasetSinkConstants.CONFIG_KITE_BATCH_SIZE, DatasetSinkConstants.DEFAULT_BATCH_SIZE); - this.rollInterval = context.getInteger( + this.rollIntervalS = context.getInteger( DatasetSinkConstants.CONFIG_KITE_ROLL_INTERVAL, DatasetSinkConstants.DEFAULT_ROLL_INTERVAL); @@ -166,67 +157,30 @@ public void configure(Context context) { @Override public synchronized void start() { this.writer = openWriter(targetDataset); - if (rollInterval > 0) { - this.rollTimer = Executors.newSingleThreadScheduledExecutor( - new ThreadFactoryBuilder() - .setNameFormat(getName() + "-timed-roll-thread") - .build()); - rollTimer.scheduleWithFixedDelay(new Runnable() { - @Override - public void run() { - roll(); - } - }, rollInterval, rollInterval, TimeUnit.SECONDS); - } + this.lastRolledMs = System.currentTimeMillis(); counter.start(); // signal that this sink is ready to process LOG.info("Started DatasetSink " + getName()); super.start(); } - void roll() { - // if the writer is null, nothing to do - if (writer == null) { - return; - } - - // no need to open/close while the lock is held, just replace the reference - DatasetWriter toClose = null; - DatasetWriter newWriter = openWriter(targetDataset); - - writerLock.lock(); - try { - toClose = writer; - this.writer = newWriter; - } finally { - writerLock.unlock(); - } - - LOG.info("Rolled writer for dataset: " + datasetName); - toClose.close(); + /** + * Causes the sink to roll at the next {@link #process()} call. + */ + @VisibleForTesting + public void roll() { + this.lastRolledMs = 0l; } @Override public synchronized void stop() { counter.stop(); - if (rollTimer != null) { - rollTimer.shutdown(); - try { - while (!rollTimer.isTerminated()) { - rollTimer.awaitTermination( - DatasetSinkConstants.DEFAULT_TERMINATION_INTERVAL, - TimeUnit.MILLISECONDS); - } - } catch (InterruptedException ex) { - LOG.warn("Interrupted while waiting for shutdown: " + rollTimer); - Thread.interrupted(); - } - } if (writer != null) { // any write problems invalidate the writer, which is immediately closed writer.close(); this.writer = null; + this.lastRolledMs = System.currentTimeMillis(); } // signal that this sink has stopped @@ -241,37 +195,41 @@ public Status process() throws EventDeliveryException { "Cannot recover after previous failure"); } + // handle file rolling + if ((System.currentTimeMillis() - lastRolledMs) / 1000 > rollIntervalS) { + // close the current writer and get a new one + writer.close(); + this.writer = openWriter(targetDataset); + this.lastRolledMs = System.currentTimeMillis(); + LOG.info("Rolled writer for dataset: " + datasetName); + } + Channel channel = getChannel(); Transaction transaction = null; try { long processedEvents = 0; - // coarse locking to avoid waiting within the loop - writerLock.lock(); transaction = channel.getTransaction(); transaction.begin(); - try { - for (; processedEvents < batchSize; processedEvents += 1) { - Event event = channel.take(); - if (event == null) { - // no events available in the channel - break; - } + for (; processedEvents < batchSize; processedEvents += 1) { + Event event = channel.take(); + if (event == null) { + // no events available in the channel + break; + } - this.datum = deserialize(event, datum); + this.datum = deserialize(event, datum); - // writeEncoded would be an optimization in some cases, but HBase - // will not support it and partitioned Datasets need to get partition - // info from the entity Object. We may be able to avoid the - // serialization round-trip otherwise. - writer.write(datum); - } - // TODO: Add option to sync, depends on CDK-203 - writer.flush(); - } finally { - writerLock.unlock(); + // writeEncoded would be an optimization in some cases, but HBase + // will not support it and partitioned Datasets need to get partition + // info from the entity Object. We may be able to avoid the + // serialization round-trip otherwise. + writer.write(datum); } + // TODO: Add option to sync, depends on CDK-203 + writer.flush(); + // commit after data has been written and flushed transaction.commit(); @@ -300,16 +258,10 @@ public Status process() throws EventDeliveryException { } } - // remove the writer's reference and close it - DatasetWriter toClose = null; - writerLock.lock(); - try { - toClose = writer; - this.writer = null; - } finally { - writerLock.unlock(); - } - toClose.close(); + // close the writer and remove the its reference + writer.close(); + this.writer = null; + this.lastRolledMs = System.currentTimeMillis(); // handle the exception Throwables.propagateIfInstanceOf(th, Error.class); diff --git a/flume-ng-sinks/flume-dataset-sink/src/main/java/org/apache/flume/sink/kite/DatasetSinkConstants.java b/flume-ng-sinks/flume-dataset-sink/src/main/java/org/apache/flume/sink/kite/DatasetSinkConstants.java index 5087352b42..13c776ef9a 100644 --- a/flume-ng-sinks/flume-dataset-sink/src/main/java/org/apache/flume/sink/kite/DatasetSinkConstants.java +++ b/flume-ng-sinks/flume-dataset-sink/src/main/java/org/apache/flume/sink/kite/DatasetSinkConstants.java @@ -41,11 +41,6 @@ public class DatasetSinkConstants { public static final String CONFIG_KITE_ROLL_INTERVAL = "kite.rollInterval"; public static int DEFAULT_ROLL_INTERVAL = 30; // seconds - /** - * Interval to wait for thread termination - */ - public static final int DEFAULT_TERMINATION_INTERVAL = 10000; // milliseconds - /** * Headers with avro schema information is expected. */ diff --git a/flume-ng-sinks/flume-dataset-sink/src/test/java/org/apache/flume/sink/kite/TestDatasetSink.java b/flume-ng-sinks/flume-dataset-sink/src/test/java/org/apache/flume/sink/kite/TestDatasetSink.java index bd0e1dcfe9..ac275db5da 100644 --- a/flume-ng-sinks/flume-dataset-sink/src/test/java/org/apache/flume/sink/kite/TestDatasetSink.java +++ b/flume-ng-sinks/flume-dataset-sink/src/test/java/org/apache/flume/sink/kite/TestDatasetSink.java @@ -28,12 +28,10 @@ import java.io.File; import java.io.FileWriter; import java.io.IOException; -import java.net.URI; import java.util.Arrays; import java.util.HashSet; import java.util.List; import java.util.Map; -import java.util.Set; import java.util.concurrent.Callable; import javax.annotation.Nullable; import org.apache.avro.Schema; From 6a50ec2ad33b8cbd057907c67030d855520c5f13 Mon Sep 17 00:00:00 2001 From: Hari Shreedharan Date: Thu, 27 Feb 2014 15:17:30 -0800 Subject: [PATCH 077/341] FLUME-1227. Introduce Spillable Channel. (Roshan Naik via Hari Shreedharan) --- .../flume/channel/file/FileChannel.java | 12 +- .../flume-spillable-memory-channel/pom.xml | 108 +++ .../flume/channel/SpillableMemoryChannel.java | 808 ++++++++++++++++ .../channel/TestSpillableMemoryChannel.java | 883 ++++++++++++++++++ flume-ng-channels/pom.xml | 1 + .../conf/channel/ChannelConfiguration.java | 7 +- .../flume/conf/channel/ChannelType.java | 8 +- .../apache/flume/ChannelFullException.java | 47 + .../apache/flume/channel/AbstractChannel.java | 2 + .../apache/flume/channel/MemoryChannel.java | 3 +- flume-ng-dist/pom.xml | 4 + flume-ng-doc/sphinx/FlumeUserGuide.rst | 74 ++ .../embedded/EmbeddedAgentConfiguration.java | 8 + flume-ng-node/pom.xml | 5 + pom.xml | 12 + 15 files changed, 1972 insertions(+), 10 deletions(-) create mode 100644 flume-ng-channels/flume-spillable-memory-channel/pom.xml create mode 100644 flume-ng-channels/flume-spillable-memory-channel/src/main/java/org/apache/flume/channel/SpillableMemoryChannel.java create mode 100644 flume-ng-channels/flume-spillable-memory-channel/src/test/java/org/apache/flume/channel/TestSpillableMemoryChannel.java create mode 100644 flume-ng-core/src/main/java/org/apache/flume/ChannelFullException.java diff --git a/flume-ng-channels/flume-file-channel/src/main/java/org/apache/flume/channel/file/FileChannel.java b/flume-ng-channels/flume-file-channel/src/main/java/org/apache/flume/channel/file/FileChannel.java index 71b26f77fa..5203ca108c 100644 --- a/flume-ng-channels/flume-file-channel/src/main/java/org/apache/flume/channel/file/FileChannel.java +++ b/flume-ng-channels/flume-file-channel/src/main/java/org/apache/flume/channel/file/FileChannel.java @@ -23,10 +23,7 @@ import com.google.common.base.Preconditions; import com.google.common.base.Strings; import com.google.common.base.Throwables; -import org.apache.flume.Channel; -import org.apache.flume.ChannelException; -import org.apache.flume.Context; -import org.apache.flume.Event; +import org.apache.flume.*; import org.apache.flume.annotations.Disposable; import org.apache.flume.annotations.InterfaceAudience; import org.apache.flume.annotations.InterfaceStability; @@ -77,7 +74,7 @@ public class FileChannel extends BasicChannelSemantics { private Integer capacity = 0; private int keepAlive; - private Integer transactionCapacity = 0; + protected Integer transactionCapacity = 0; private Long checkpointInterval = 0L; private long maxFileSize; private long minimumRequiredSpace; @@ -323,6 +320,7 @@ protected BasicTransactionSemantics createTransaction() { } throw new IllegalStateException(msg); } + FileBackedTransaction trans = transactions.get(); if(trans != null && !trans.isClosed()) { Preconditions.checkState(false, @@ -336,7 +334,7 @@ transactionCapacity, keepAlive, queueRemaining, getName(), return trans; } - int getDepth() { + protected int getDepth() { Preconditions.checkState(open, "Channel closed" + channelNameDescriptor); Preconditions.checkNotNull(log, "log"); FlumeEventQueue queue = log.getFlumeEventQueue(); @@ -435,7 +433,7 @@ protected void doPut(Event event) throws InterruptedException { // this does not need to be in the critical section as it does not // modify the structure of the log or queue. if(!queueRemaining.tryAcquire(keepAlive, TimeUnit.SECONDS)) { - throw new ChannelException("The channel has reached it's capacity. " + throw new ChannelFullException("The channel has reached it's capacity. " + "This might be the result of a sink on the channel having too " + "low of batch size, a downstream system running slower than " + "normal, or that the channel capacity is just too low. " diff --git a/flume-ng-channels/flume-spillable-memory-channel/pom.xml b/flume-ng-channels/flume-spillable-memory-channel/pom.xml new file mode 100644 index 0000000000..b4646e8992 --- /dev/null +++ b/flume-ng-channels/flume-spillable-memory-channel/pom.xml @@ -0,0 +1,108 @@ + + + + + 4.0.0 + + flume-ng-channels + org.apache.flume + 1.5.0-SNAPSHOT + + + org.apache.flume.flume-ng-channels + flume-spillable-memory-channel + Flume NG Spillable Memory channel + + + + + org.apache.rat + apache-rat-plugin + + + + + + + + org.apache.flume + flume-ng-sdk + + + + org.apache.flume + flume-ng-configuration + + + + org.apache.flume + flume-ng-core + + + + org.apache.flume.flume-ng-channels + flume-file-channel + + + + org.slf4j + slf4j-api + + + + commons-collections + commons-collections + + + + commons-lang + commons-lang + + + + junit + junit + test + + + + org.mockito + mockito-all + 1.9.0 + test + + + + org.easytesting + fest-reflect + 1.4 + test + + + + org.slf4j + slf4j-log4j12 + test + + + + + diff --git a/flume-ng-channels/flume-spillable-memory-channel/src/main/java/org/apache/flume/channel/SpillableMemoryChannel.java b/flume-ng-channels/flume-spillable-memory-channel/src/main/java/org/apache/flume/channel/SpillableMemoryChannel.java new file mode 100644 index 0000000000..bdf42cdcb1 --- /dev/null +++ b/flume-ng-channels/flume-spillable-memory-channel/src/main/java/org/apache/flume/channel/SpillableMemoryChannel.java @@ -0,0 +1,808 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, + * software distributed under the License is distributed on an + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + * KIND, either express or implied. See the License for the + * specific language governing permissions and limitations + * under the License. + */ +package org.apache.flume.channel; + +import java.util.ArrayDeque; +import java.util.concurrent.Semaphore; +import java.util.concurrent.TimeUnit; + +import javax.annotation.concurrent.GuardedBy; + +import com.google.common.annotations.VisibleForTesting; +import org.apache.flume.*; +import org.apache.flume.annotations.Recyclable; + +import org.apache.flume.annotations.InterfaceAudience; +import org.apache.flume.annotations.InterfaceStability; +import org.apache.flume.channel.file.FileChannel; +import org.apache.flume.instrumentation.ChannelCounter; + +import org.apache.flume.lifecycle.LifecycleState; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +import com.google.common.base.Preconditions; + +/** + *

+ * SpillableMemoryChannel will use main memory for buffering events until it has reached capacity. + * Thereafter file channel will be used as overflow. + *

+ */ +@InterfaceAudience.Private +@InterfaceStability.Evolving +@Recyclable +public class SpillableMemoryChannel extends FileChannel { + // config settings + /** Max number of events to be stored in memory */ + public static final String MEMORY_CAPACITY = "memoryCapacity"; + /** Seconds to wait before enabling disk overflow when memory fills up */ + public static final String OVERFLOW_TIMEOUT = "overflowTimeout"; + /** Internal use only. To remain undocumented in User guide. Determines the + * percent free space available in mem queue when we stop spilling to overflow + */ + public static final String OVERFLOW_DEACTIVATION_THRESHOLD + = "overflowDeactivationThreshold"; + /** percent of buffer between byteCapacity and the estimated event size. */ + public static final String BYTE_CAPACITY_BUFFER_PERCENTAGE + = "byteCapacityBufferPercentage"; + + /** max number of bytes used for all events in the queue. */ + public static final String BYTE_CAPACITY = "byteCapacity"; + /** max number of events in overflow. */ + public static final String OVERFLOW_CAPACITY = "overflowCapacity"; + /** file channel setting that is overriden by Spillable Channel */ + public static final String KEEP_ALIVE = "keep-alive"; + /** file channel capacity overridden by Spillable Channel */ + public static final String CAPACITY = "capacity"; + /** Estimated average size of events expected to be in the channel */ + public static final String AVG_EVENT_SIZE = "avgEventSize"; + + private static Logger LOGGER = LoggerFactory.getLogger(SpillableMemoryChannel.class); + public static final int defaultMemoryCapacity = 10000; + public static final int defaultOverflowCapacity = 100000000; + + public static final int defaultOverflowTimeout = 3; + public static final int defaultOverflowDeactivationThreshold = 5; // percent + + // memory consumption control + private static final int defaultAvgEventSize = 500; + private static final Long defaultByteCapacity + = (long)(Runtime.getRuntime().maxMemory() * .80); + private static final int defaultByteCapacityBufferPercentage = 20; + + private volatile int byteCapacity; + private volatile double avgEventSize = defaultAvgEventSize; + private volatile int lastByteCapacity; + private volatile int byteCapacityBufferPercentage; + private Semaphore bytesRemaining; + + // for synchronizing access to primary/overflow channels & drain order + final private Object queueLock = new Object(); + + @GuardedBy(value = "queueLock") + public ArrayDeque memQueue; + + // This semaphore tracks number of free slots in primary channel (includes + // all active put lists) .. used to determine if the puts + // should go into primary or overflow + private Semaphore memQueRemaining; + + // tracks number of events in both channels. Takes will block on this + private Semaphore totalStored; + + private int maxMemQueueSize = 0; // max sie of memory Queue + + private boolean overflowDisabled; // if true indicates the overflow should not be used at all. + private boolean overflowActivated=false; // indicates if overflow can be used. invariant: false if overflowDisabled is true. + + // if true overflow can be used. invariant: false if overflowDisabled is true. + private int memoryCapacity = -1; // max events that the channel can hold in memory + private int overflowCapacity; + + private int overflowTimeout; + + // mem full % at which we stop spill to overflow + private double overflowDeactivationThreshold + = defaultOverflowDeactivationThreshold / 100; + + public SpillableMemoryChannel() { + super(); + } + + protected int getTotalStored() { + return totalStored.availablePermits(); + } + + public int getMemoryCapacity() { + return memoryCapacity; + } + public int getOverflowTimeout() { + return overflowTimeout; + } + + public int getMaxMemQueueSize() { + return maxMemQueueSize; + } + + protected Integer getOverflowCapacity() { + return overflowCapacity; + } + + protected boolean isOverflowDisabled() { + return overflowDisabled; + } + + @VisibleForTesting + protected ChannelCounter channelCounter; + + public final DrainOrderQueue drainOrder = new DrainOrderQueue(); + + public int queueSize() { + synchronized (queueLock) { + return memQueue.size(); + } + } + + + private static class MutableInteger { + private int value; + + public MutableInteger(int val) { + value = val; + } + + public void add(int amount) { + value += amount; + } + + public int intValue() { + return value; + } + } + + // pop on a empty queue will throw NoSuchElementException + // invariant: 0 will never be left in the queue + + public static class DrainOrderQueue { + public ArrayDeque queue = new ArrayDeque(1000); + + public int totalPuts = 0; // for debugging only + private long overflowCounter = 0; // # of items in overflow channel + + public String dump() { + StringBuilder sb = new StringBuilder(); + + sb.append(" [ "); + for (MutableInteger i : queue) { + sb.append(i.intValue()); + sb.append(" "); + } + sb.append("]"); + return sb.toString(); + } + + public void putPrimary(Integer eventCount) { + totalPuts += eventCount; + if ( (queue.peekLast() == null) || queue.getLast().intValue() < 0) { + queue.addLast(new MutableInteger(eventCount)); + } else { + queue.getLast().add(eventCount); + } + } + + public void putFirstPrimary(Integer eventCount) { + if ( (queue.peekFirst() == null) || queue.getFirst().intValue() < 0) { + queue.addFirst(new MutableInteger(eventCount)); + } else { + queue.getFirst().add(eventCount); + } + } + + public void putOverflow(Integer eventCount) { + totalPuts += eventCount; + if ( (queue.peekLast() == null) || queue.getLast().intValue() > 0) { + queue.addLast(new MutableInteger(-eventCount)); + } else { + queue.getLast().add(-eventCount); + } + overflowCounter += eventCount; + } + + public void putFirstOverflow(Integer eventCount) { + if ( (queue.peekFirst() == null) || queue.getFirst().intValue() > 0) { + queue.addFirst(new MutableInteger(-eventCount)); + } else { + queue.getFirst().add(-eventCount); + } + overflowCounter += eventCount; + } + + public int front() { + return queue.getFirst().intValue(); + } + + public boolean isEmpty() { + return queue.isEmpty(); + } + + public void takePrimary(int takeCount) { + MutableInteger headValue = queue.getFirst(); + + // this condition is optimization to avoid redundant conversions of + // int -> Integer -> string in hot path + if (headValue.intValue() < takeCount) { + throw new IllegalStateException("Cannot take " + takeCount + + " from " + headValue.intValue() + " in DrainOrder Queue"); + } + + headValue.add(-takeCount); + if (headValue.intValue() == 0) { + queue.removeFirst(); + } + } + + public void takeOverflow(int takeCount) { + MutableInteger headValue = queue.getFirst(); + if(headValue.intValue() > -takeCount) { + throw new IllegalStateException("Cannot take " + takeCount + " from " + + headValue.intValue() + " in DrainOrder Queue head " ); + } + + headValue.add(takeCount); + if (headValue.intValue() == 0) { + queue.removeFirst(); + } + overflowCounter -= takeCount; + } + + } + + private class SpillableMemoryTransaction extends BasicTransactionSemantics { + BasicTransactionSemantics overflowTakeTx = null; // Take-Txn for overflow + BasicTransactionSemantics overflowPutTx = null; // Put-Txn for overflow + boolean useOverflow = false; + boolean putCalled = false; // set on first invocation to put + boolean takeCalled = false; // set on first invocation to take + int largestTakeTxSize = 5000; // not a constraint, just hint for allocation + int largestPutTxSize = 5000; // not a constraint, just hint for allocation + + Integer overflowPutCount = 0; // # of puts going to overflow in this Txn + + private int putListByteCount = 0; + private int takeListByteCount = 0; + private int takeCount = 0; + + ArrayDeque takeList; + ArrayDeque putList; + private final ChannelCounter channelCounter; + + + public SpillableMemoryTransaction(ChannelCounter counter) { + takeList = new ArrayDeque(largestTakeTxSize); + putList = new ArrayDeque(largestPutTxSize); + channelCounter = counter; + } + + @Override + public void begin() { + super.begin(); + } + + @Override + public void close() { + if (overflowTakeTx!=null) { + overflowTakeTx.close(); + } + if (overflowPutTx!=null) { + overflowPutTx.close(); + } + super.close(); + } + + + @Override + protected void doPut(Event event) throws InterruptedException { + channelCounter.incrementEventPutAttemptCount(); + + putCalled = true; + int eventByteSize = (int)Math.ceil(estimateEventSize(event)/ avgEventSize); + if (!putList.offer(event)) { + throw new ChannelFullException("Put queue in " + getName() + + " channel's Transaction having capacity " + putList.size() + + " full, consider reducing batch size of sources"); + } + putListByteCount += eventByteSize; + } + + + // Take will limit itself to a single channel within a transaction. + // This ensures commits/rollbacks are restricted to a single channel. + @Override + protected Event doTake() throws InterruptedException { + channelCounter.incrementEventTakeAttemptCount(); + if (!totalStored.tryAcquire(overflowTimeout, TimeUnit.SECONDS)) { + LOGGER.debug("Take is backing off as channel is empty."); + return null; + } + boolean takeSuceeded = false; + try { + Event event; + synchronized(queueLock) { + int drainOrderTop = drainOrder.front(); + + if (!takeCalled) { + takeCalled = true; + if (drainOrderTop < 0) { + useOverflow = true; + overflowTakeTx = getOverflowTx(); + overflowTakeTx.begin(); + } + } + + if (useOverflow) { + if (drainOrderTop > 0) { + LOGGER.debug("Take is switching to primary"); + return null; // takes should now occur from primary channel + } + + event = overflowTakeTx.take(); + ++takeCount; + drainOrder.takeOverflow(1); + } else { + if (drainOrderTop < 0) { + LOGGER.debug("Take is switching to overflow"); + return null; // takes should now occur from overflow channel + } + + event = memQueue.poll(); + ++takeCount; + drainOrder.takePrimary(1); + Preconditions.checkNotNull(event, "Queue.poll returned NULL despite" + + " semaphore signalling existence of entry"); + } + } + + int eventByteSize = (int)Math.ceil(estimateEventSize(event)/ avgEventSize); + if (!useOverflow) { + // takeList is thd pvt, so no need to do this in synchronized block + takeList.offer(event); + } + + takeListByteCount += eventByteSize; + takeSuceeded = true; + return event; + } finally { + if(!takeSuceeded) { + totalStored.release(); + } + } + } + + @Override + protected void doCommit() throws InterruptedException { + if (putCalled) { + putCommit(); + if (LOGGER.isDebugEnabled()) { + LOGGER.debug("Put Committed. Drain Order Queue state : " + + drainOrder.dump()); + } + } else if (takeCalled) { + takeCommit(); + if (LOGGER.isDebugEnabled()) { + LOGGER.debug("Take Committed. Drain Order Queue state : " + + drainOrder.dump()); + } + } + } + + private void takeCommit() { + if (takeCount > largestTakeTxSize) + largestTakeTxSize = takeCount; + + synchronized (queueLock) { + if (overflowTakeTx!=null) { + overflowTakeTx.commit(); + } + double memoryPercentFree = (memoryCapacity == 0) ? 0 + : (memoryCapacity - memQueue.size() + takeCount ) / (double)memoryCapacity ; + + if (overflowActivated + && memoryPercentFree >= overflowDeactivationThreshold) { + overflowActivated = false; + LOGGER.info("Overflow Deactivated"); + } + channelCounter.setChannelSize(getTotalStored()); + } + if (!useOverflow) { + memQueRemaining.release(takeCount); + bytesRemaining.release(takeListByteCount); + } + + channelCounter.addToEventTakeSuccessCount(takeCount); + } + + private void putCommit() throws InterruptedException { + // decide if overflow needs to be used + int timeout = overflowActivated ? 0 : overflowTimeout; + + if (memoryCapacity != 0) { + // check for enough event slots(memoryCapacity) for using memory queue + if (!memQueRemaining.tryAcquire(putList.size(), timeout, + TimeUnit.SECONDS)) { + if (overflowDisabled) { + throw new ChannelFullException("Spillable Memory Channel's " + + "memory capacity has been reached and overflow is " + + "disabled. Consider increasing memoryCapacity."); + } + overflowActivated = true; + useOverflow = true; + } + // check if we have enough byteCapacity for using memory queue + else if (!bytesRemaining.tryAcquire(putListByteCount, overflowTimeout + , TimeUnit.SECONDS)) { + memQueRemaining.release(putList.size()); + if (overflowDisabled) { + throw new ChannelFullException("Spillable Memory Channel's " + + "memory capacity has been reached. " + + (bytesRemaining.availablePermits() * (int) avgEventSize) + + " bytes are free and overflow is disabled. Consider " + + "increasing byteCapacity or capacity."); + } + overflowActivated = true; + useOverflow = true; + } + } else { + useOverflow = true; + } + + if (putList.size() > largestPutTxSize) { + largestPutTxSize = putList.size(); + } + + if (useOverflow) { + commitPutsToOverflow(); + } else { + commitPutsToPrimary(); + } + } + + private void commitPutsToOverflow() throws InterruptedException { + overflowPutTx = getOverflowTx(); + overflowPutTx.begin(); + for (Event event : putList) { + overflowPutTx.put(event); + } + commitPutsToOverflow_core(overflowPutTx); + totalStored.release(putList.size()); + overflowPutCount += putList.size(); + channelCounter.addToEventPutSuccessCount(putList.size()); + } + + private void commitPutsToOverflow_core(Transaction overflowPutTx) + throws InterruptedException { + // reattempt only once if overflow is full first time around + for (int i = 0; i < 2; ++i) { + try { + synchronized(queueLock) { + overflowPutTx.commit(); + drainOrder.putOverflow(putList.size()); + channelCounter.setChannelSize(memQueue.size() + + drainOrder.overflowCounter); + break; + } + } catch (ChannelFullException e) { // drop lock & reattempt + if (i==0) { + Thread.sleep(overflowTimeout *1000); + } + else { + throw e; + } + } + } + } + + private void commitPutsToPrimary() { + synchronized (queueLock) { + for (Event e : putList) { + if (!memQueue.offer(e)) { + throw new ChannelException("Unable to insert event into memory " + + "queue in spite of spare capacity, this is very unexpected"); + } + } + drainOrder.putPrimary(putList.size()); + maxMemQueueSize = (memQueue.size() > maxMemQueueSize) ? memQueue.size() + : maxMemQueueSize; + channelCounter.setChannelSize(memQueue.size() + + drainOrder.overflowCounter); + } + // update counters and semaphores + totalStored.release(putList.size()); + channelCounter.addToEventPutSuccessCount(putList.size()); + } + + @Override + protected void doRollback() { + LOGGER.debug("Rollback() of " + + (takeCalled ? " Take Tx" : (putCalled ? " Put Tx" : "Empty Tx"))); + + if (putCalled) { + if (overflowPutTx!=null) { + overflowPutTx.rollback(); + } + if (!useOverflow) { + bytesRemaining.release(putListByteCount); + putList.clear(); + } + putListByteCount = 0; + } else if (takeCalled) { + synchronized(queueLock) { + if (overflowTakeTx!=null) { + overflowTakeTx.rollback(); + } + if (useOverflow) { + drainOrder.putFirstOverflow(takeCount); + } else { + int remainingCapacity = memoryCapacity - memQueue.size(); + Preconditions.checkState(remainingCapacity >= takeCount, + "Not enough space in memory queue to rollback takes. This" + + " should never happen, please report"); + while (!takeList.isEmpty()) { + memQueue.addFirst(takeList.removeLast()); + } + drainOrder.putFirstPrimary(takeCount); + } + } + totalStored.release(takeCount); + } else { + overflowTakeTx.rollback(); + } + channelCounter.setChannelSize(memQueue.size() + drainOrder.overflowCounter); + } + } // Transaction + + /** + * Read parameters from context + *
  • memoryCapacity = total number of events allowed at one time in the memory queue. + *
  • overflowCapacity = total number of events allowed at one time in the overflow file channel. + *
  • byteCapacity = the max number of bytes used for events in the memory queue. + *
  • byteCapacityBufferPercentage = type int. Defines the percent of buffer between byteCapacity and the estimated event size. + *
  • overflowTimeout = type int. Number of seconds to wait on a full memory before deciding to enable overflow + */ + @Override + public void configure(Context context) { + + if (getLifecycleState() == LifecycleState.START // does not support reconfig when running + || getLifecycleState() == LifecycleState.ERROR) + stop(); + + if (totalStored == null) { + totalStored = new Semaphore(0); + } + + if (channelCounter == null) { + channelCounter = new ChannelCounter(getName()); + } + + // 1) Memory Capacity + Integer newMemoryCapacity; + try { + newMemoryCapacity = context.getInteger(MEMORY_CAPACITY + , defaultMemoryCapacity); + if (newMemoryCapacity == null) { + newMemoryCapacity = defaultMemoryCapacity; + } + if (newMemoryCapacity < 0) { + throw new NumberFormatException(MEMORY_CAPACITY + " must be >= 0"); + } + + } catch(NumberFormatException e) { + newMemoryCapacity = defaultMemoryCapacity; + LOGGER.warn("Invalid " + MEMORY_CAPACITY + " specified, initializing " + + getName() + " channel to default value of {}", defaultMemoryCapacity); + } + try { + resizePrimaryQueue(newMemoryCapacity); + } catch (InterruptedException e) { + Thread.currentThread().interrupt(); + } + + // overflowTimeout - wait time before switching to overflow when mem is full + try { + Integer newOverflowTimeout = + context.getInteger(OVERFLOW_TIMEOUT, defaultOverflowTimeout); + overflowTimeout = (newOverflowTimeout != null) ? newOverflowTimeout + : defaultOverflowTimeout; + } catch(NumberFormatException e) { + LOGGER.warn("Incorrect value for " + getName() + "'s " + OVERFLOW_TIMEOUT + + " setting. Using default value {}", defaultOverflowTimeout); + overflowTimeout = defaultOverflowTimeout; + } + + try { + Integer newThreshold = context.getInteger(OVERFLOW_DEACTIVATION_THRESHOLD); + overflowDeactivationThreshold = (newThreshold != null) ? + newThreshold/100.0 + : defaultOverflowDeactivationThreshold / 100.0; + } catch(NumberFormatException e) { + LOGGER.warn("Incorrect value for " + getName() + "'s " + + OVERFLOW_DEACTIVATION_THRESHOLD + ". Using default value {} %", + defaultOverflowDeactivationThreshold); + overflowDeactivationThreshold = defaultOverflowDeactivationThreshold / 100.0; + } + + // 3) Memory consumption control + try { + byteCapacityBufferPercentage = + context.getInteger(BYTE_CAPACITY_BUFFER_PERCENTAGE + , defaultByteCapacityBufferPercentage); + } catch(NumberFormatException e) { + LOGGER.warn("Error parsing " + BYTE_CAPACITY_BUFFER_PERCENTAGE + " for " + + getName() + ". Using default=" + + defaultByteCapacityBufferPercentage + ". " + e.getMessage()); + byteCapacityBufferPercentage = defaultByteCapacityBufferPercentage; + } + + try { + avgEventSize = context.getInteger(AVG_EVENT_SIZE, defaultAvgEventSize); + } catch ( NumberFormatException e) { + LOGGER.warn("Error parsing " + AVG_EVENT_SIZE + " for " + getName() + + ". Using default = " + defaultAvgEventSize + ". " + + e.getMessage()); + avgEventSize = defaultAvgEventSize; + } + + try { + byteCapacity = (int) ((context.getLong(BYTE_CAPACITY, defaultByteCapacity) * (1 - byteCapacityBufferPercentage * .01 )) / avgEventSize); + if (byteCapacity < 1) { + byteCapacity = Integer.MAX_VALUE; + } + } catch(NumberFormatException e) { + LOGGER.warn("Error parsing " + BYTE_CAPACITY + " setting for " + getName() + + ". Using default = " + defaultByteCapacity + ". " + + e.getMessage()); + byteCapacity = (int) + ( (defaultByteCapacity * (1 - byteCapacityBufferPercentage * .01 )) + / avgEventSize); + } + + + if (bytesRemaining == null) { + bytesRemaining = new Semaphore(byteCapacity); + lastByteCapacity = byteCapacity; + } else { + if (byteCapacity > lastByteCapacity) { + bytesRemaining.release(byteCapacity - lastByteCapacity); + lastByteCapacity = byteCapacity; + } else { + try { + if (!bytesRemaining.tryAcquire(lastByteCapacity - byteCapacity, overflowTimeout, TimeUnit.SECONDS)) { + LOGGER.warn("Couldn't acquire permits to downsize the byte capacity, resizing has been aborted"); + } else { + lastByteCapacity = byteCapacity; + } + } catch (InterruptedException e) { + Thread.currentThread().interrupt(); + } + } + } + + try { + overflowCapacity = context.getInteger(OVERFLOW_CAPACITY, defaultOverflowCapacity); // file channel capacity + // Determine if File channel needs to be disabled + if ( memoryCapacity < 1 && overflowCapacity < 1) { + LOGGER.warn("For channel " + getName() + OVERFLOW_CAPACITY + + " cannot be set to 0 if " + MEMORY_CAPACITY + " is also 0. " + + "Using default value " + OVERFLOW_CAPACITY + " = " + + defaultOverflowCapacity); + overflowCapacity = defaultOverflowCapacity; + } + overflowDisabled = (overflowCapacity < 1) ; + if (overflowDisabled) { + overflowActivated = false; + } + } catch(NumberFormatException e) { + overflowCapacity = defaultOverflowCapacity; + } + + // Configure File channel + context.put(KEEP_ALIVE,"0"); // override keep-alive for File channel + context.put(CAPACITY, Integer.toString(overflowCapacity) ); // file channel capacity + super.configure(context); + } + + + private void resizePrimaryQueue(int newMemoryCapacity) throws InterruptedException { + if (memQueue != null && memoryCapacity == newMemoryCapacity) { + return; + } + + if (memoryCapacity > newMemoryCapacity) { + int diff = memoryCapacity - newMemoryCapacity; + if (!memQueRemaining.tryAcquire(diff, overflowTimeout, TimeUnit.SECONDS)) { + LOGGER.warn("Memory buffer currently contains more events than the new size. Downsizing has been aborted."); + return; + } + synchronized(queueLock) { + ArrayDeque newQueue = new ArrayDeque(newMemoryCapacity); + newQueue.addAll(memQueue); + memQueue = newQueue; + memoryCapacity = newMemoryCapacity; + } + } else { // if (memoryCapacity <= newMemoryCapacity) + synchronized(queueLock) { + ArrayDeque newQueue = new ArrayDeque(newMemoryCapacity); + if (memQueue !=null) { + newQueue.addAll(memQueue); + } + memQueue = newQueue; + if (memQueRemaining == null) { + memQueRemaining = new Semaphore(newMemoryCapacity); + } else { + int diff = newMemoryCapacity - memoryCapacity; + memQueRemaining.release(diff); + } + memoryCapacity = newMemoryCapacity; + } + } + } + + @Override + public synchronized void start() { + super.start(); + int overFlowCount = super.getDepth(); + if (drainOrder.isEmpty()) { + drainOrder.putOverflow(overFlowCount); + totalStored.release(overFlowCount); + } + int totalCount = overFlowCount + memQueue.size(); + channelCounter.setChannelCapacity(memoryCapacity + getOverflowCapacity()); + channelCounter.setChannelSize(totalCount); + } + + @Override + public synchronized void stop() { + if (getLifecycleState()==LifecycleState.STOP) { + return; + } + channelCounter.setChannelSize(memQueue.size() + drainOrder.overflowCounter); + channelCounter.stop(); + super.stop(); + } + + @Override + protected BasicTransactionSemantics createTransaction() { + return new SpillableMemoryTransaction(channelCounter); + } + + + private BasicTransactionSemantics getOverflowTx() { + return super.createTransaction(); + } + + private long estimateEventSize(Event event) { + byte[] body = event.getBody(); + if (body != null && body.length != 0) { + return body.length; + } + //Each event occupies at least 1 slot, so return 1. + return 1; + } + +} diff --git a/flume-ng-channels/flume-spillable-memory-channel/src/test/java/org/apache/flume/channel/TestSpillableMemoryChannel.java b/flume-ng-channels/flume-spillable-memory-channel/src/test/java/org/apache/flume/channel/TestSpillableMemoryChannel.java new file mode 100644 index 0000000000..1e4e81999e --- /dev/null +++ b/flume-ng-channels/flume-spillable-memory-channel/src/test/java/org/apache/flume/channel/TestSpillableMemoryChannel.java @@ -0,0 +1,883 @@ +/* +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, + * software distributed under the License is distributed on an + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + * KIND, either express or implied. See the License for the + * specific language governing permissions and limitations + * under the License. + */ + +package org.apache.flume.channel; + +import java.io.File; +import java.util.ArrayList; +import java.util.HashMap; +import java.util.Map; + +import java.util.UUID; + +import org.apache.flume.*; +import org.apache.flume.conf.Configurables; +import org.apache.flume.event.EventBuilder; +import org.apache.flume.channel.file.FileChannelConfiguration; + +import org.junit.After; +import org.junit.Assert; +import org.junit.Before; +import org.junit.Test; +import org.junit.Rule; +import org.junit.rules.TemporaryFolder; + + +public class TestSpillableMemoryChannel { + + private SpillableMemoryChannel channel; + + @Rule + public TemporaryFolder fileChannelDir = new TemporaryFolder(); + + private void configureChannel(Map overrides) { + Context context = new Context(); + File checkPointDir = fileChannelDir.newFolder("checkpoint"); + File dataDir = fileChannelDir.newFolder("data"); + context.put(FileChannelConfiguration.CHECKPOINT_DIR + , checkPointDir.getAbsolutePath()); + context.put(FileChannelConfiguration.DATA_DIRS, dataDir.getAbsolutePath()); + // Set checkpoint for 5 seconds otherwise test will run out of memory + context.put(FileChannelConfiguration.CHECKPOINT_INTERVAL, "5000"); + + if (overrides != null) + context.putAll(overrides); + + Configurables.configure(channel, context); + } + + private void reconfigureChannel(Map overrides) { + configureChannel(overrides); + channel.stop(); + channel.start(); + } + + private void startChannel(Map params) { + configureChannel(params); + channel.start(); + } + + // performs a hard restart of the channel... creates a new channel object + private void restartChannel(Map params) { + channel.stop(); + setUp(); + startChannel(params); + } + + + static class NullFound extends RuntimeException { + public int expectedValue; + public NullFound(int expected) { + super("Expected " + expected + ", but null found"); + expectedValue = expected; + } + } + + static class TooManyNulls extends RuntimeException { + private int nullsFound; + public TooManyNulls(int count) { + super("Total nulls found in thread (" + + Thread.currentThread().getName() + ") : " + count); + nullsFound = count; + } + } + + @Before + public void setUp() { + channel = new SpillableMemoryChannel(); + channel.setName("spillChannel-" + UUID.randomUUID() ); + } + + @After + public void tearDown() { + channel.stop(); + } + + private static void putN(int first, int count, AbstractChannel channel) { + for (int i = 0; i < count; ++i) { + channel.put(EventBuilder.withBody(String.valueOf(first++).getBytes())); + } + } + + private static void takeNull(AbstractChannel channel) { + channel.take(); + } + + private static void takeN(int first, int count, AbstractChannel channel) { + int last = first + count; + for (int i = first; i < last; ++i) { + Event e = channel.take(); + if (e == null) { + throw new NullFound(i); + } + Event expected = EventBuilder.withBody( String.valueOf(i).getBytes() ); + Assert.assertArrayEquals(e.getBody(), expected.getBody()); + } + } + + // returns the number of non null events found + private static int takeN_NoCheck(int batchSize, AbstractChannel channel) { + int i = 0; + for (; i < batchSize; ++i) { + Event e = channel.take(); + if (e == null) { + try { + Thread.sleep(0); + } catch (InterruptedException ex) + { /* ignore */ } + return i; + } + } + return i; + } + + private static void transactionalPutN(int first, int count, + AbstractChannel channel) { + Transaction tx = channel.getTransaction(); + tx.begin(); + try { + putN(first, count, channel); + tx.commit(); + } catch (RuntimeException e) { + tx.rollback(); + throw e; + } finally { + tx.close(); + } + } + + private static void transactionalTakeN(int first, int count, + AbstractChannel channel) { + Transaction tx = channel.getTransaction(); + tx.begin(); + try { + takeN(first, count, channel); + tx.commit(); + } catch (NullFound e) { + tx.commit(); + throw e; + } catch (AssertionError e) { + tx.rollback(); + throw e; + } catch (RuntimeException e) { + tx.rollback(); + throw e; + } finally { + tx.close(); + } + } + + private static int transactionalTakeN_NoCheck(int count + , AbstractChannel channel) { + Transaction tx = channel.getTransaction(); + tx.begin(); + try { + int eventCount = takeN_NoCheck(count, channel); + tx.commit(); + return eventCount; + } catch (RuntimeException e) { + tx.rollback(); + throw e; + } finally { + tx.close(); + } + } + + private static void transactionalTakeNull(int count, AbstractChannel channel) { + Transaction tx = channel.getTransaction(); + tx.begin(); + try { + for (int i = 0; i < count; ++i) + takeNull(channel); + tx.commit(); + } catch (AssertionError e) { + tx.rollback(); + throw e; + } catch (RuntimeException e) { + tx.rollback(); + throw e; + } finally { + tx.close(); + } + } + + private Thread makePutThread(String threadName + , final int first, final int count, final int batchSize + , final AbstractChannel channel) { + return + new Thread(threadName) { + public void run() { + int maxdepth = 0; + StopWatch watch = new StopWatch(); + for (int i = first; i batchSz) ? batchSz : remaining; + int takenCount = transactionalTakeN_NoCheck(batchSz, channel); + if(takenCount < batchSz) { + try { + Thread.sleep(20); + } catch (InterruptedException ex) + { /* ignore */ } + } + i += takenCount; + ++attempts; + if(attempts > totalEvents * 3 ) { + throw new TooManyNulls(attempts); + } + } + watch.elapsed(" items = " + i + ", attempts = " + attempts); + } + }; + } + + @Test + public void testPutTake() { + Map params = new HashMap(); + params.put("memoryCapacity", "5"); + params.put("overflowCapacity", "5"); + params.put(FileChannelConfiguration.TRANSACTION_CAPACITY, "5"); + startChannel(params); + + Transaction tx = channel.getTransaction(); + tx.begin(); + putN(0,2,channel); + tx.commit(); + tx.close(); + + tx = channel.getTransaction(); + tx.begin(); + takeN(0,2,channel); + tx.commit(); + tx.close(); + } + + + @Test + public void testCapacityDisableOverflow() { + Map params = new HashMap(); + params.put("memoryCapacity", "2"); + params.put("overflowCapacity", "0"); // overflow is disabled effectively + params.put("overflowTimeout", "0" ); + startChannel(params); + + transactionalPutN(0,2,channel); + + boolean threw = false; + try { + transactionalPutN(2,1,channel); + } catch (ChannelException e) { + threw = true; + } + Assert.assertTrue("Expecting ChannelFullException to be thrown", threw); + + transactionalTakeN(0,2, channel); + + Transaction tx = channel.getTransaction(); + tx.begin(); + Assert.assertNull(channel.take()); + tx.commit(); + tx.close(); + } + + @Test + public void testCapacityWithOverflow() { + Map params = new HashMap(); + params.put("memoryCapacity", "2"); + params.put("overflowCapacity", "4"); + params.put(FileChannelConfiguration.TRANSACTION_CAPACITY, "3"); + params.put("overflowTimeout", "0"); + startChannel(params); + + transactionalPutN(1, 2, channel); + transactionalPutN(3, 2, channel); + transactionalPutN(5, 2, channel); + + boolean threw = false; + try { + transactionalPutN(7,2,channel); // cannot fit in channel + } catch (ChannelFullException e) { + threw = true; + } + Assert.assertTrue("Expecting ChannelFullException to be thrown", threw); + + transactionalTakeN(1,2, channel); + transactionalTakeN(3,2, channel); + transactionalTakeN(5,2, channel); + } + + @Test + public void testRestart() { + Map params = new HashMap(); + params.put("memoryCapacity", "2"); + params.put("overflowCapacity", "10"); + params.put(FileChannelConfiguration.TRANSACTION_CAPACITY, "4"); + params.put("overflowTimeout", "0"); + startChannel(params); + + transactionalPutN(1, 2, channel); + transactionalPutN(3, 2, channel); // goes in overflow + + restartChannel(params); + + // from overflow, as in memory stuff should be lost + transactionalTakeN(3,2, channel); + + } + + @Test + public void testBasicStart() { + Map params = new HashMap(); + params.put("memoryCapacity", "10000000"); + params.put("overflowCapacity", "20000000"); + params.put(FileChannelConfiguration.TRANSACTION_CAPACITY, "10"); + params.put("overflowTimeout", "1" ); + + startChannel(params); + + transactionalPutN( 1,5,channel); + transactionalPutN( 6,5,channel); + transactionalPutN(11,5,channel); // these should go to overflow + + transactionalTakeN(1,10, channel); + transactionalTakeN(11,5, channel); + } + + @Test + public void testOverflow() { + + Map params = new HashMap(); + params.put("memoryCapacity", "10"); + params.put("overflowCapacity", "20"); + params.put(FileChannelConfiguration.TRANSACTION_CAPACITY, "10"); + params.put("overflowTimeout", "1" ); + + startChannel(params); + + transactionalPutN( 1,5,channel); + transactionalPutN( 6,5,channel); + transactionalPutN(11,5,channel); // these should go to overflow + + transactionalTakeN(1,10, channel); + transactionalTakeN(11,5, channel); + } + + @Test + public void testDrainOrder() { + Map params = new HashMap(); + params.put("memoryCapacity", "10"); + params.put("overflowCapacity", "10"); + params.put(FileChannelConfiguration.TRANSACTION_CAPACITY, "5"); + params.put("overflowTimeout", "1" ); + + startChannel(params); + + transactionalPutN( 1,5,channel); + transactionalPutN( 6,5,channel); + transactionalPutN(11,5,channel); // into overflow + transactionalPutN(16,5,channel); // into overflow + + transactionalTakeN(1, 1, channel); + transactionalTakeN(2, 5,channel); + transactionalTakeN(7, 4,channel); + + transactionalPutN( 20,2,channel); + transactionalPutN( 22,3,channel); + + transactionalTakeN( 11,3,channel); // from overflow + transactionalTakeN( 14,5,channel); // from overflow + transactionalTakeN( 19,2,channel); // from overflow + } + + @Test + public void testByteCapacity() { + Map params = new HashMap(); + params.put("memoryCapacity", "1000"); + // configure to hold 8 events of 10 bytes each (plus 20% event header space) + params.put("byteCapacity", "100"); + params.put("avgEventSize", "10"); + params.put("overflowCapacity", "20"); + params.put(FileChannelConfiguration.TRANSACTION_CAPACITY, "10"); + params.put("overflowTimeout", "1" ); + startChannel(params); + + transactionalPutN(1, 8, channel); // this wil max the byteCapacity + transactionalPutN(9, 10, channel); + transactionalPutN(19,10, channel); // this will fill up the overflow + + boolean threw = false; + try { + transactionalPutN(11, 1, channel); // into overflow + } catch (ChannelFullException e) { + threw = true; + } + Assert.assertTrue("byteCapacity did not throw as expected", threw); + + } + + @Test + public void testDrainingOnChannelBoundary() { + + Map params = new HashMap(); + params.put("memoryCapacity", "5"); + params.put("overflowCapacity", "15"); + params.put(FileChannelConfiguration.TRANSACTION_CAPACITY, "10"); + params.put("overflowTimeout", "1" ); + startChannel(params); + + transactionalPutN(1, 5, channel); + transactionalPutN(6, 5, channel); // into overflow + transactionalPutN(11, 5, channel); // into overflow + transactionalPutN(16, 5, channel); // into overflow + + transactionalTakeN(1, 3, channel); + + Transaction tx = channel.getTransaction(); + tx.begin(); + takeN(4, 2, channel); + takeNull(channel); // expect null since next event is in overflow + tx.commit(); + tx.close(); + + transactionalTakeN(6, 5, channel); // from overflow + + transactionalTakeN(11, 5, channel); // from overflow + transactionalTakeN(16, 2,channel); // from overflow + + transactionalPutN(21, 5, channel); + + tx = channel.getTransaction(); + tx.begin(); + takeN(18,3, channel); // from overflow + takeNull(channel); // expect null since next event is in primary + tx.commit(); + tx.close(); + + transactionalTakeN(21, 5, channel); + } + + @Test + public void testRollBack() { + Map params = new HashMap(); + params.put("memoryCapacity", "100"); + params.put("overflowCapacity", "900"); + params.put(FileChannelConfiguration.TRANSACTION_CAPACITY, "900"); + params.put("overflowTimeout", "0"); + startChannel(params); + + + //1 Rollback for Puts + transactionalPutN(1,5, channel); + Transaction tx = channel.getTransaction(); + tx.begin(); + putN(6, 5, channel); + tx.rollback(); + tx.close(); + + transactionalTakeN(1, 5, channel); + transactionalTakeNull(2, channel); + + //2. verify things back to normal after put rollback + transactionalPutN(11, 5, channel); + transactionalTakeN(11,5,channel); + + + //3 Rollback for Takes + transactionalPutN(16, 5, channel); + tx = channel.getTransaction(); + tx.begin(); + takeN(16, 5, channel); + takeNull(channel); + tx.rollback(); + tx.close(); + + transactionalTakeN_NoCheck(5, channel); + + //4. verify things back to normal after take rollback + transactionalPutN(21,5, channel); + transactionalTakeN(21,5,channel); + } + + + @Test + public void testReconfigure() { + //1) bring up with small capacity + Map params = new HashMap(); + params.put("memoryCapacity", "10"); + params.put("overflowCapacity", "0"); + params.put("overflowTimeout", "0"); + startChannel(params); + + Assert.assertTrue("overflowTimeout setting did not reconfigure correctly" + , channel.getOverflowTimeout() == 0); + Assert.assertTrue("memoryCapacity did not reconfigure correctly" + , channel.getMemoryCapacity() == 10); + Assert.assertTrue("overflowCapacity did not reconfigure correctly" + , channel.isOverflowDisabled() ); + + transactionalPutN(1, 10, channel); + boolean threw = false; + try { + transactionalPutN(11, 10, channel); // should throw an error + } catch (ChannelException e) { + threw = true; + } + Assert.assertTrue("Expected the channel to fill up and throw an exception, " + + "but it did not throw", threw); + + + //2) Resize and verify + params = new HashMap(); + params.put("memoryCapacity", "20"); + params.put("overflowCapacity", "0"); + reconfigureChannel(params); + + Assert.assertTrue("overflowTimeout setting did not reconfigure correctly" + , channel.getOverflowTimeout() == SpillableMemoryChannel.defaultOverflowTimeout); + Assert.assertTrue("memoryCapacity did not reconfigure correctly" + , channel.getMemoryCapacity() == 20); + Assert.assertTrue("overflowCapacity did not reconfigure correctly" + , channel.isOverflowDisabled() ); + + // pull out the values inserted prior to reconfiguration + transactionalTakeN(1, 10, channel); + + transactionalPutN(11, 10, channel); + transactionalPutN(21, 10, channel); + + threw = false; + try { + transactionalPutN(31, 10, channel); // should throw an error + } catch (ChannelException e) { + threw = true; + } + Assert.assertTrue("Expected the channel to fill up and throw an exception, " + + "but it did not throw", threw); + + transactionalTakeN(11, 10, channel); + transactionalTakeN(21, 10, channel); + + + // 3) Reconfigure with empty config and verify settings revert to default + params = new HashMap(); + reconfigureChannel(params); + + Assert.assertTrue("overflowTimeout setting did not reconfigure correctly" + , channel.getOverflowTimeout() == SpillableMemoryChannel.defaultOverflowTimeout); + Assert.assertTrue("memoryCapacity did not reconfigure correctly" + , channel.getMemoryCapacity() == SpillableMemoryChannel.defaultMemoryCapacity); + Assert.assertTrue("overflowCapacity did not reconfigure correctly" + , channel.getOverflowCapacity() == SpillableMemoryChannel.defaultOverflowCapacity); + Assert.assertFalse("overflowCapacity did not reconfigure correctly" + , channel.isOverflowDisabled()); + + + // 4) Reconfiguring of overflow + params = new HashMap(); + params.put("memoryCapacity", "10"); + params.put("overflowCapacity", "10"); + params.put("transactionCapacity", "5"); + params.put("overflowTimeout", "1"); + reconfigureChannel(params); + + transactionalPutN( 1,5, channel); + transactionalPutN( 6,5, channel); + transactionalPutN(11,5, channel); + transactionalPutN(16,5, channel); + threw=false; + try { + // should error out as both primary & overflow are full + transactionalPutN(21,5, channel); + } catch (ChannelException e) { + threw = true; + } + Assert.assertTrue("Expected the last insertion to fail, but it didn't." + , threw); + + // reconfig the overflow + params = new HashMap(); + params.put("memoryCapacity", "10"); + params.put("overflowCapacity", "20"); + params.put("transactionCapacity", "10"); + params.put("overflowTimeout", "1"); + reconfigureChannel(params); + + // should succeed now as we have made room in the overflow + transactionalPutN(21,5, channel); + + transactionalTakeN(1,10, channel); + transactionalTakeN(11,5, channel); + transactionalTakeN(16, 5, channel); + transactionalTakeN(21, 5, channel); + } + + @Test + public void testParallelSingleSourceAndSink() throws InterruptedException { + Map params = new HashMap(); + params.put("memoryCapacity", "1000020"); + params.put("overflowCapacity", "0"); + params.put("overflowTimeout", "3"); + startChannel(params); + + // run source and sink concurrently + Thread sourceThd = makePutThread("src", 1, 500000, 100, channel); + Thread sinkThd = makeTakeThread("sink", 1, 500000, 100, channel); + + StopWatch watch = new StopWatch(); + + sinkThd.start(); + sourceThd.start(); + + sourceThd.join(); + sinkThd.join(); + + watch.elapsed(); + System.out.println("Max Queue size " + channel.getMaxMemQueueSize() ); + } + + @Test + public void testCounters() throws InterruptedException { + Map params = new HashMap(); + params.put("memoryCapacity", "5000"); + params.put("overflowCapacity","5000"); + params.put("transactionCapacity","5000"); + params.put("overflowTimeout", "0"); + startChannel(params); + + //1. fill up mem queue + Thread sourceThd = makePutThread("src", 1, 5000, 2500, channel); + sourceThd.start(); + sourceThd.join(); + + Assert.assertEquals(5000, channel.getTotalStored()); + Assert.assertEquals(5000, channel.channelCounter.getChannelSize()); + Assert.assertEquals(5000, channel.channelCounter.getEventPutAttemptCount()); + Assert.assertEquals(5000, channel.channelCounter.getEventPutSuccessCount()); + + //2. empty mem queue + Thread sinkThd = makeTakeThread("sink", 1, 5000, 1000, channel); + sinkThd.start(); + sinkThd.join(); + Assert.assertEquals(0, channel.getTotalStored()); + Assert.assertEquals(0, channel.channelCounter.getChannelSize()); + Assert.assertEquals(5000, channel.channelCounter.getEventTakeAttemptCount()); + Assert.assertEquals(5000, channel.channelCounter.getEventTakeSuccessCount()); + + + //3. fill up mem & overflow + sourceThd = makePutThread("src", 1, 10000, 1000, channel); + sourceThd.start(); + sourceThd.join(); + Assert.assertEquals(10000, channel.getTotalStored()); + Assert.assertEquals(10000, channel.channelCounter.getChannelSize()); + Assert.assertEquals(15000, channel.channelCounter.getEventPutAttemptCount()); + Assert.assertEquals(15000, channel.channelCounter.getEventPutSuccessCount()); + + + //4. empty memory + sinkThd = makeTakeThread("sink", 1, 5000, 1000, channel); + sinkThd.start(); + sinkThd.join(); + Assert.assertEquals(5000, channel.getTotalStored()); + Assert.assertEquals(5000, channel.channelCounter.getChannelSize()); + Assert.assertEquals(10000, channel.channelCounter.getEventTakeAttemptCount()); + Assert.assertEquals(10000, channel.channelCounter.getEventTakeSuccessCount()); + + //5. empty overflow + transactionalTakeN(5001, 1000, channel); + transactionalTakeN(6001, 1000, channel); + transactionalTakeN(7001, 1000, channel); + transactionalTakeN(8001, 1000, channel); + transactionalTakeN(9001, 1000, channel); + Assert.assertEquals(0, channel.getTotalStored()); + Assert.assertEquals(0, channel.channelCounter.getChannelSize()); + Assert.assertEquals(15000, channel.channelCounter.getEventTakeAttemptCount()); + Assert.assertEquals(15000, channel.channelCounter.getEventTakeSuccessCount()); + + + + //6. now do it concurrently + sourceThd = makePutThread("src1", 1, 5000, 1000, channel); + Thread sourceThd2 = makePutThread("src2", 1, 5000, 500, channel); + sinkThd = makeTakeThread_noCheck("sink1", 5000, 1000, channel); + sourceThd.start(); + sourceThd2.start(); + sinkThd.start(); + sourceThd.join(); + sourceThd2.join(); + sinkThd.join(); + Assert.assertEquals(5000, channel.getTotalStored()); + Assert.assertEquals(5000, channel.channelCounter.getChannelSize()); + Thread sinkThd2 = makeTakeThread_noCheck("sink2", 2500, 500, channel); + Thread sinkThd3 = makeTakeThread_noCheck("sink3", 2500, 1000, channel); + sinkThd2.start(); + sinkThd3.start(); + sinkThd2.join(); + sinkThd3.join(); + Assert.assertEquals(0, channel.getTotalStored()); + Assert.assertEquals(0, channel.channelCounter.getChannelSize()); + Assert.assertEquals(25000, channel.channelCounter.getEventTakeSuccessCount()); + Assert.assertEquals(25000, channel.channelCounter.getEventPutSuccessCount()); + Assert.assertTrue("TakeAttempt channel counter value larger than expected" , + 25000 <= channel.channelCounter.getEventTakeAttemptCount()); + Assert.assertTrue("PutAttempt channel counter value larger than expected", + 25000 <= channel.channelCounter.getEventPutAttemptCount()); + } + + public ArrayList createSourceThreads(int count, int totalEvents + , int batchSize) { + ArrayList sourceThds = new ArrayList(); + + for (int i = 0; i < count; ++i) { + sourceThds.add( makePutThread("src" + i, 1, totalEvents/count + , batchSize, channel) ); + } + return sourceThds; + } + + public ArrayList createSinkThreads(int count, int totalEvents + , int batchSize) { + ArrayList sinkThreads = new ArrayList(count); + + for (int i = 0; i < count; ++i) { + sinkThreads.add( makeTakeThread_noCheck("sink"+i, totalEvents/count + , batchSize, channel) ); + } + return sinkThreads; + } + + public void startThreads(ArrayList threads) { + for (Thread thread : threads) { + thread.start(); + } + } + + public void joinThreads(ArrayList threads) + throws InterruptedException { + for (Thread thread : threads) { + try { + thread.join(); + } catch (InterruptedException e) { + System.out.println("Interrupted while waiting on " + thread.getName() ); + throw e; + } + } + } + + @Test + public void testParallelMultipleSourcesAndSinks() throws InterruptedException { + int sourceCount = 8; + int sinkCount = 8; + int eventCount = 1000000; + int batchSize = 100; + + Map params = new HashMap(); + params.put("memoryCapacity", "0"); + params.put("overflowCapacity", "500020"); + params.put("overflowTimeout", "3"); + startChannel(params); + + ArrayList sinks = createSinkThreads(sinkCount, eventCount, batchSize); + + ArrayList sources = createSourceThreads(sourceCount + , eventCount, batchSize); + + + StopWatch watch = new StopWatch(); + startThreads(sinks); + startThreads(sources); + + joinThreads(sources); + joinThreads(sinks); + + watch.elapsed(); + + System.out.println("Total puts " + channel.drainOrder.totalPuts); + + System.out.println("Max Queue size " + channel.getMaxMemQueueSize() ); + System.out.println(channel.memQueue.size()); + + System.out.println("done"); + } + + static class StopWatch { + long startTime; + + public StopWatch() { + startTime = System.currentTimeMillis(); + } + + public void elapsed() { + elapsed(null); + } + + public void elapsed(String suffix) { + long elapsed = System.currentTimeMillis() - startTime; + if (suffix == null) { + suffix = ""; + } else { + suffix = "{ " + suffix + " }"; + } + + if (elapsed < 10000) { + System.out.println(Thread.currentThread().getName() + + " : [ " + elapsed + " ms ]. " + suffix); + } else { + System.out.println(Thread.currentThread().getName() + + " : [ " + elapsed / 1000 + " sec ]. " + suffix); + } + } + } + +} diff --git a/flume-ng-channels/pom.xml b/flume-ng-channels/pom.xml index 5832ab4459..0da3be8be7 100644 --- a/flume-ng-channels/pom.xml +++ b/flume-ng-channels/pom.xml @@ -43,5 +43,6 @@ limitations under the License. flume-jdbc-channel flume-file-channel + flume-spillable-memory-channel diff --git a/flume-ng-configuration/src/main/java/org/apache/flume/conf/channel/ChannelConfiguration.java b/flume-ng-configuration/src/main/java/org/apache/flume/conf/channel/ChannelConfiguration.java index 26f4dd76d5..b3b477fe61 100644 --- a/flume-ng-configuration/src/main/java/org/apache/flume/conf/channel/ChannelConfiguration.java +++ b/flume-ng-configuration/src/main/java/org/apache/flume/conf/channel/ChannelConfiguration.java @@ -44,7 +44,12 @@ public enum ChannelConfigurationType { /** * JDBC channel provided by org.apache.flume.channel.jdbc.JdbcChannel */ - JDBC("org.apache.flume.conf.channel.JdbcChannelConfiguration"); + JDBC("org.apache.flume.conf.channel.JdbcChannelConfiguration"), + + /** + * Spillable Memory channel + */ + SPILLABLEMEMORY("org.apache.flume.conf.channel.SpillableMemoryChannelConfiguration"); private String channelConfigurationType; diff --git a/flume-ng-configuration/src/main/java/org/apache/flume/conf/channel/ChannelType.java b/flume-ng-configuration/src/main/java/org/apache/flume/conf/channel/ChannelType.java index 15b8cc3804..2e3cb34397 100644 --- a/flume-ng-configuration/src/main/java/org/apache/flume/conf/channel/ChannelType.java +++ b/flume-ng-configuration/src/main/java/org/apache/flume/conf/channel/ChannelType.java @@ -44,8 +44,14 @@ public enum ChannelType { /** * JDBC channel provided by org.apache.flume.channel.jdbc.JdbcChannel */ - JDBC("org.apache.flume.channel.jdbc.JdbcChannel"); + JDBC("org.apache.flume.channel.jdbc.JdbcChannel"), + /** + * Spillable Memory channel + * + * @see SpillableMemoryChannel + */ + SPILLABLEMEMORY("org.apache.flume.channel.SpillableMemoryChannel"); private final String channelClassName; diff --git a/flume-ng-core/src/main/java/org/apache/flume/ChannelFullException.java b/flume-ng-core/src/main/java/org/apache/flume/ChannelFullException.java new file mode 100644 index 0000000000..675ef13f73 --- /dev/null +++ b/flume-ng-core/src/main/java/org/apache/flume/ChannelFullException.java @@ -0,0 +1,47 @@ +/** + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.flume; + +public class ChannelFullException extends ChannelException { + + private static final long serialVersionUID = -8098141359417449525L; + + /** + * @param message the exception message + */ + public ChannelFullException(String message) { + super(message); + } + + /** + * @param ex the causal exception + */ + public ChannelFullException(Throwable ex) { + super(ex); + } + + /** + * @param message the exception message + * @param ex the causal exception + */ + public ChannelFullException(String message, Throwable ex) { + super(message, ex); + } + +} diff --git a/flume-ng-core/src/main/java/org/apache/flume/channel/AbstractChannel.java b/flume-ng-core/src/main/java/org/apache/flume/channel/AbstractChannel.java index 1370e6638d..b991650b09 100644 --- a/flume-ng-core/src/main/java/org/apache/flume/channel/AbstractChannel.java +++ b/flume-ng-core/src/main/java/org/apache/flume/channel/AbstractChannel.java @@ -26,6 +26,8 @@ import org.apache.flume.lifecycle.LifecycleAware; import org.apache.flume.lifecycle.LifecycleState; +import java.util.Map; + @InterfaceAudience.Public @InterfaceStability.Stable public abstract class AbstractChannel diff --git a/flume-ng-core/src/main/java/org/apache/flume/channel/MemoryChannel.java b/flume-ng-core/src/main/java/org/apache/flume/channel/MemoryChannel.java index f10a79f24a..b84a3e0a4d 100644 --- a/flume-ng-core/src/main/java/org/apache/flume/channel/MemoryChannel.java +++ b/flume-ng-core/src/main/java/org/apache/flume/channel/MemoryChannel.java @@ -25,6 +25,7 @@ import javax.annotation.concurrent.GuardedBy; import org.apache.flume.ChannelException; +import org.apache.flume.ChannelFullException; import org.apache.flume.Context; import org.apache.flume.Event; import org.apache.flume.annotations.InterfaceAudience; @@ -126,7 +127,7 @@ protected void doCommit() throws InterruptedException { } if(!queueRemaining.tryAcquire(-remainingChange, keepAlive, TimeUnit.SECONDS)) { bytesRemaining.release(putByteCounter); - throw new ChannelException("Space for commit to queue couldn't be acquired" + + throw new ChannelFullException("Space for commit to queue couldn't be acquired." + " Sinks are likely not keeping up with sources, or the buffer size is too tight"); } } diff --git a/flume-ng-dist/pom.xml b/flume-ng-dist/pom.xml index 8b814b7c97..338e108813 100644 --- a/flume-ng-dist/pom.xml +++ b/flume-ng-dist/pom.xml @@ -106,6 +106,10 @@ org.apache.flume.flume-ng-channels flume-file-channel + + org.apache.flume.flume-ng-channels + flume-spillable-memory-channel + org.apache.flume.flume-ng-channels flume-jdbc-channel diff --git a/flume-ng-doc/sphinx/FlumeUserGuide.rst b/flume-ng-doc/sphinx/FlumeUserGuide.rst index cd43634a63..2cd0996b27 100644 --- a/flume-ng-doc/sphinx/FlumeUserGuide.rst +++ b/flume-ng-doc/sphinx/FlumeUserGuide.rst @@ -2270,6 +2270,80 @@ The same scenerio as above, however key-0 has its own password: a1.channels.c1.encryption.keyProvider.keys.key-0.passwordFile = /path/to/key-0.password +Spillable Memory Channel +~~~~~~~~~~~~~~~~~~~~~~~~ + +The events are stored in an in-memory queue and on disk. The in-memory queue serves as the primary store and the disk as overflow. +The disk store is managed using an embedded File channel. When the in-memory queue is full, additional incoming events are stored in +the file channel. This channel is ideal for flows that need high throughput of memory channel during normal operation, but at the +same time need the larger capacity of the file channel for better tolerance of intermittent sink side outages or drop in drain rates. +The throughput will reduce approximately to file channel speeds during such abnormal situations. In case of an agent crash or restart, +only the events stored on disk are recovered when the agent comes online. **This channel is currently experimental and +not recommended for use in production.** + +Required properties are in **bold**. Please refer to file channel for additional required properties. + +============================ ================ ============================================================================================= +Property Name Default Description +============================ ================ ============================================================================================= +**type** -- The component type name, needs to be ``SPILLABLEMEMORY`` +memoryCapacity 10000 Maximum number of events stored in memory queue. To disable use of in-memory queue, set this to zero. +overflowCapacity 100000000 Maximum number of events stored in overflow disk (i.e File channel). To disable use of overflow, set this to zero. +overflowTimeout 3 The number of seconds to wait before enabling disk overflow when memory fills up. +byteCapacityBufferPercentage 20 Defines the percent of buffer between byteCapacity and the estimated total size + of all events in the channel, to account for data in headers. See below. +byteCapacity see description Maximum **bytes** of memory allowed as a sum of all events in the memory queue. + The implementation only counts the Event ``body``, which is the reason for + providing the ``byteCapacityBufferPercentage`` configuration parameter as well. + Defaults to a computed value equal to 80% of the maximum memory available to + the JVM (i.e. 80% of the -Xmx value passed on the command line). + Note that if you have multiple memory channels on a single JVM, and they happen + to hold the same physical events (i.e. if you are using a replicating channel + selector from a single source) then those event sizes may be double-counted for + channel byteCapacity purposes. + Setting this value to ``0`` will cause this value to fall back to a hard + internal limit of about 200 GB. +avgEventSize 500 Estimated average size of events, in bytes, going into the channel + see file channel Any file channel property with the exception of 'keep-alive' and 'capacity' can be used. + The keep-alive of file channel is managed by Spillable Memory Channel. Use 'overflowCapacity' + to set the File channel's capacity. +============================ ================ ============================================================================================= + +In-memory queue is considered full if either memoryCapacity or byteCapacity limit is reached. + +Example for agent named a1: + +.. code-block:: properties + + a1.channels = c1 + a1.channels.c1.type = SPILLABLEMEMORY + a1.channels.c1.memoryCapacity = 10000 + a1.channels.c1.overflowCapacity = 1000000 + a1.channels.c1.byteCapacity = 800000 + a1.channels.c1.checkpointDir = /mnt/flume/checkpoint + a1.channels.c1.dataDirs = /mnt/flume/data + +To disable the use of the in-memory queue and function like a file channel: + +.. code-block:: properties + + a1.channels = c1 + a1.channels.c1.type = SPILLABLEMEMORY + a1.channels.c1.memoryCapacity = 0 + a1.channels.c1.overflowCapacity = 1000000 + a1.channels.c1.checkpointDir = /mnt/flume/checkpoint + a1.channels.c1.dataDirs = /mnt/flume/data + + +To disable the use of overflow disk and function purely as a in-memory channel: + +.. code-block:: properties + + a1.channels = c1 + a1.channels.c1.type = SPILLABLEMEMORY + a1.channels.c1.memoryCapacity = 100000 + + Pseudo Transaction Channel ~~~~~~~~~~~~~~~~~~~~~~~~~~ diff --git a/flume-ng-embedded-agent/src/main/java/org/apache/flume/agent/embedded/EmbeddedAgentConfiguration.java b/flume-ng-embedded-agent/src/main/java/org/apache/flume/agent/embedded/EmbeddedAgentConfiguration.java index 6204bc551b..06ea7c042e 100644 --- a/flume-ng-embedded-agent/src/main/java/org/apache/flume/agent/embedded/EmbeddedAgentConfiguration.java +++ b/flume-ng-embedded-agent/src/main/java/org/apache/flume/agent/embedded/EmbeddedAgentConfiguration.java @@ -102,6 +102,14 @@ public class EmbeddedAgentConfiguration { * Embedded Agents. */ public static final String CHANNEL_TYPE_MEMORY = ChannelType.MEMORY.name(); + + /** + * Spillable Memory channel which stores events in heap. See Flume User Guide for + * configuration information. This is the recommended channel to use for + * Embedded Agents. + */ + public static final String CHANNEL_TYPE_SPILLABLEMEMORY = ChannelType.SPILLABLEMEMORY.name(); + /** * File based channel which stores events in on local disk. See Flume User * Guide for configuration information. diff --git a/flume-ng-node/pom.xml b/flume-ng-node/pom.xml index f1b0c652c0..7e9c9755f5 100644 --- a/flume-ng-node/pom.xml +++ b/flume-ng-node/pom.xml @@ -93,6 +93,11 @@ + org.apache.flume.flume-ng-channels + flume-spillable-memory-channel + + + org.slf4j slf4j-api diff --git a/pom.xml b/pom.xml index 362fb4555d..815cecd2a3 100644 --- a/pom.xml +++ b/pom.xml @@ -940,6 +940,18 @@ limitations under the License. 1.5.0-SNAPSHOT + + org.apache.flume.flume-ng-channels + flume-spillable-memory-channel + 1.5.0-SNAPSHOT + + + + org.apache.flume.flume-ng-channels + flume-recoverable-memory-channel + 1.5.0-SNAPSHOT + + org.apache.flume.flume-ng-sinks flume-hdfs-sink From 1fbb7c929ef33a4597de31ce271ae06aa8dbb9d2 Mon Sep 17 00:00:00 2001 From: Jarek Jarcec Cecho Date: Fri, 28 Feb 2014 14:08:36 -0800 Subject: [PATCH 078/341] FLUME-2334: Upgrade Asynchbase to 1.5.0 as this include hbase 0.96.x support --- pom.xml | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/pom.xml b/pom.xml index 815cecd2a3..08fffc46ab 100644 --- a/pom.xml +++ b/pom.xml @@ -1066,7 +1066,7 @@ limitations under the License. org.hbase asynchbase - 1.4.1 + 1.5.0 org.slf4j From c9b531e70a655e331f18012d3c24614ac2c66b75 Mon Sep 17 00:00:00 2001 From: Jarek Jarcec Cecho Date: Fri, 28 Feb 2014 15:09:11 -0800 Subject: [PATCH 079/341] FLUME-2328: FileChannel Dual Checkpoint Backup Thread not released on Application stop (Hari Shreedharan via Jarek Jarcec Cecho) --- .../channel/file/EventQueueBackingStoreFile.java | 12 ++++++++++++ 1 file changed, 12 insertions(+) diff --git a/flume-ng-channels/flume-file-channel/src/main/java/org/apache/flume/channel/file/EventQueueBackingStoreFile.java b/flume-ng-channels/flume-file-channel/src/main/java/org/apache/flume/channel/file/EventQueueBackingStoreFile.java index 8a9fdae50c..113dcd22c2 100644 --- a/flume-ng-channels/flume-file-channel/src/main/java/org/apache/flume/channel/file/EventQueueBackingStoreFile.java +++ b/flume-ng-channels/flume-file-channel/src/main/java/org/apache/flume/channel/file/EventQueueBackingStoreFile.java @@ -301,6 +301,18 @@ void close() { } catch (IOException e) { LOG.info("Error closing " + checkpointFile, e); } + if(checkpointBackUpExecutor != null && !checkpointBackUpExecutor + .isShutdown()) { + checkpointBackUpExecutor.shutdown(); + try { + // Wait till the executor dies. + while (!checkpointBackUpExecutor.awaitTermination(1, + TimeUnit.SECONDS)); + } catch (InterruptedException ex) { + LOG.warn("Interrupted while waiting for checkpoint backup to " + + "complete"); + } + } } @Override From e1ec22e4f24600006c20812f7e7362ac80a38656 Mon Sep 17 00:00:00 2001 From: Jarek Jarcec Cecho Date: Fri, 28 Feb 2014 15:37:25 -0800 Subject: [PATCH 080/341] FLUME-2323: Morphline sink must increment eventDrainAttemptCount when it takes event from channel (Hari Shreedharan via Jarek Jarcec Cecho) --- .../apache/flume/sink/solr/morphline/MorphlineSink.java | 7 +++---- 1 file changed, 3 insertions(+), 4 deletions(-) diff --git a/flume-ng-sinks/flume-ng-morphline-solr-sink/src/main/java/org/apache/flume/sink/solr/morphline/MorphlineSink.java b/flume-ng-sinks/flume-ng-morphline-solr-sink/src/main/java/org/apache/flume/sink/solr/morphline/MorphlineSink.java index 7c784c460c..9c4dc25189 100644 --- a/flume-ng-sinks/flume-ng-morphline-solr-sink/src/main/java/org/apache/flume/sink/solr/morphline/MorphlineSink.java +++ b/flume-ng-sinks/flume-ng-morphline-solr-sink/src/main/java/org/apache/flume/sink/solr/morphline/MorphlineSink.java @@ -134,8 +134,9 @@ public Status process() throws EventDeliveryException { if (event == null) { break; } + sinkCounter.incrementEventDrainAttemptCount(); numEventsTaken++; - LOGGER.debug("Flume event: {}", event); + LOGGER.debug("Flume event: {}", event); //StreamEvent streamEvent = createStreamEvent(event); handler.process(event); if (System.currentTimeMillis() >= batchEndTime) { @@ -152,12 +153,10 @@ public Status process() throws EventDeliveryException { } else { sinkCounter.incrementBatchCompleteCount(); } - sinkCounter.addToEventDrainAttemptCount(numEventsTaken); - sinkCounter.addToEventDrainSuccessCount(numEventsTaken); - handler.commitTransaction(); isMorphlineTransactionCommitted = true; txn.commit(); + sinkCounter.addToEventDrainSuccessCount(numEventsTaken); return numEventsTaken == 0 ? Status.BACKOFF : Status.READY; } catch (Throwable t) { // Ooops - need to rollback and back off From 7296e1a8b98706c990c4e6aa0deda1883c422be9 Mon Sep 17 00:00:00 2001 From: Jarek Jarcec Cecho Date: Fri, 28 Feb 2014 15:41:11 -0800 Subject: [PATCH 081/341] FLUME-2283: Spool Dir source must check interrupt flag before writing to channel (Hari Shreedharan via Jarek Jarcec Cecho) --- .../java/org/apache/flume/source/SpoolDirectorySource.java | 3 ++- 1 file changed, 2 insertions(+), 1 deletion(-) diff --git a/flume-ng-core/src/main/java/org/apache/flume/source/SpoolDirectorySource.java b/flume-ng-core/src/main/java/org/apache/flume/source/SpoolDirectorySource.java index f42ed2d859..49d8826442 100644 --- a/flume-ng-core/src/main/java/org/apache/flume/source/SpoolDirectorySource.java +++ b/flume-ng-core/src/main/java/org/apache/flume/source/SpoolDirectorySource.java @@ -220,7 +220,7 @@ public SpoolDirectoryRunnable(ReliableSpoolingFileEventReader reader, public void run() { int backoffInterval = 250; try { - while (true) { + while (!Thread.interrupted()) { List events = reader.readEvents(batchSize); if (events.isEmpty()) { break; @@ -248,6 +248,7 @@ public void run() { sourceCounter.addToEventAcceptedCount(events.size()); sourceCounter.incrementAppendBatchAcceptedCount(); } + logger.info("Spooling Directory Source runner has shutdown."); } catch (Throwable t) { logger.error("FATAL: " + SpoolDirectorySource.this.toString() + ": " + "Uncaught exception in SpoolDirectorySource thread. " + From 9688cad6faabfebf8ec9690abd4ba320dd795890 Mon Sep 17 00:00:00 2001 From: Jarek Jarcec Cecho Date: Fri, 28 Feb 2014 16:06:50 -0800 Subject: [PATCH 082/341] FLUME-2329: Add an alias for the Morphline Solr Sink (Hari Shreedharan via Jarek Jarcec Cecho) --- .../org/apache/flume/conf/sink/SinkConfiguration.java | 11 ++++++++++- .../java/org/apache/flume/conf/sink/SinkType.java | 9 ++++++++- 2 files changed, 18 insertions(+), 2 deletions(-) diff --git a/flume-ng-configuration/src/main/java/org/apache/flume/conf/sink/SinkConfiguration.java b/flume-ng-configuration/src/main/java/org/apache/flume/conf/sink/SinkConfiguration.java index 965d2c5fce..ac11558005 100644 --- a/flume-ng-configuration/src/main/java/org/apache/flume/conf/sink/SinkConfiguration.java +++ b/flume-ng-configuration/src/main/java/org/apache/flume/conf/sink/SinkConfiguration.java @@ -139,7 +139,16 @@ public enum SinkConfigurationType { * * @see org.apache.flume.sink.hbase.AsyncHBaseSink */ - ASYNCHBASE("org.apache.flume.sink.hbase.HBaseSinkConfiguration"); + ASYNCHBASE("org.apache.flume.sink.hbase.HBaseSinkConfiguration"), + + + /** + * MorphlineSolr sink + * + * @see org.apache.flume.sink.solr.morphline.MorphlineSolrSink + */ + MORPHLINE_SOLR("org.apache.flume.sink.solr.morphline" + + ".MorphlineSolrSinkConfiguration"); private final String sinkConfigurationName; diff --git a/flume-ng-configuration/src/main/java/org/apache/flume/conf/sink/SinkType.java b/flume-ng-configuration/src/main/java/org/apache/flume/conf/sink/SinkType.java index 8ce372d75c..0a1cd7a9e4 100644 --- a/flume-ng-configuration/src/main/java/org/apache/flume/conf/sink/SinkType.java +++ b/flume-ng-configuration/src/main/java/org/apache/flume/conf/sink/SinkType.java @@ -92,7 +92,14 @@ public enum SinkType { * * @see org.apache.flume.sink.hbase.AsyncHBaseSink */ - ASYNCHBASE("org.apache.flume.sink.hbase.AsyncHBaseSink"); + ASYNCHBASE("org.apache.flume.sink.hbase.AsyncHBaseSink"), + + /** + * MorphlineSolr sink + * + * @see org.apache.flume.sink.solr.morphline.MorphlineSolrSink + */ + MORPHLINE_SOLR("org.apache.flume.sink.solr.morphline.MorphlineSolrSink"); private final String sinkClassName; From 96f6b6284c6e8a645b122111059ee954e9bad7b5 Mon Sep 17 00:00:00 2001 From: Jarek Jarcec Cecho Date: Fri, 28 Feb 2014 16:25:57 -0800 Subject: [PATCH 083/341] FLUME-2324: Support writing to multiple HBase clusters using HBaseSink (Hari Shreedharan via Jarek Jarcec Cecho) --- flume-ng-doc/sphinx/FlumeUserGuide.rst | 2 + .../apache/flume/sink/hbase/HBaseSink.java | 48 +++++++++++ .../flume/sink/hbase/TestHBaseSink.java | 83 ++++++++++++++++++- 3 files changed, 131 insertions(+), 2 deletions(-) diff --git a/flume-ng-doc/sphinx/FlumeUserGuide.rst b/flume-ng-doc/sphinx/FlumeUserGuide.rst index 2cd0996b27..96bf73e0cc 100644 --- a/flume-ng-doc/sphinx/FlumeUserGuide.rst +++ b/flume-ng-doc/sphinx/FlumeUserGuide.rst @@ -1836,6 +1836,8 @@ Property Name Default Desc **type** -- The component type name, needs to be ``hbase`` **table** -- The name of the table in Hbase to write to. **columnFamily** -- The column family in Hbase to write to. +zookeeperQuorum -- The quorum spec. This is the value for the property ``hbase.zookeeper.quorum`` in hbase-site.xml +znodeParent /hbase The base path for the znode for the -ROOT- region. Value of ``zookeeper.znode.parent`` in hbase-site.xml batchSize 100 Number of events to be written per txn. serializer org.apache.flume.sink.hbase.SimpleHbaseEventSerializer Default increment column = "iCol", payload column = "pCol". serializer.* -- Properties to be passed to the serializer. diff --git a/flume-ng-sinks/flume-ng-hbase-sink/src/main/java/org/apache/flume/sink/hbase/HBaseSink.java b/flume-ng-sinks/flume-ng-hbase-sink/src/main/java/org/apache/flume/sink/hbase/HBaseSink.java index f5cb2297f6..c4a666cacd 100644 --- a/flume-ng-sinks/flume-ng-hbase-sink/src/main/java/org/apache/flume/sink/hbase/HBaseSink.java +++ b/flume-ng-sinks/flume-ng-hbase-sink/src/main/java/org/apache/flume/sink/hbase/HBaseSink.java @@ -33,6 +33,7 @@ import org.apache.flume.sink.AbstractSink; import org.apache.hadoop.conf.Configuration; import org.apache.hadoop.hbase.HBaseConfiguration; +import org.apache.hadoop.hbase.HConstants; import org.apache.hadoop.hbase.client.HTable; import org.apache.hadoop.hbase.client.Increment; import org.apache.hadoop.hbase.client.Put; @@ -221,9 +222,56 @@ public void configure(Context context){ "writes to HBase will have WAL disabled, and any data in the " + "memstore of this region in the Region Server could be lost!"); } + String zkQuorum = context.getString(HBaseSinkConfigurationConstants + .ZK_QUORUM); + Integer port = null; + /** + * HBase allows multiple nodes in the quorum, but all need to use the + * same client port. So get the nodes in host:port format, + * and ignore the ports for all nodes except the first one. If no port is + * specified, use default. + */ + if (zkQuorum != null && !zkQuorum.isEmpty()) { + StringBuilder zkBuilder = new StringBuilder(); + logger.info("Using ZK Quorum: " + zkQuorum); + String[] zkHosts = zkQuorum.split(","); + int length = zkHosts.length; + for(int i = 0; i < length; i++) { + String[] zkHostAndPort = zkHosts[i].split(":"); + zkBuilder.append(zkHostAndPort[0].trim()); + if(i != length-1) { + zkBuilder.append(","); + } else { + zkQuorum = zkBuilder.toString(); + } + if (zkHostAndPort[1] == null) { + throw new FlumeException("Expected client port for the ZK node!"); + } + if (port == null) { + port = Integer.parseInt(zkHostAndPort[1].trim()); + } else if (!port.equals(Integer.parseInt(zkHostAndPort[1].trim()))) { + throw new FlumeException("All Zookeeper nodes in the quorum must " + + "use the same client port."); + } + } + if(port == null) { + port = HConstants.DEFAULT_ZOOKEPER_CLIENT_PORT; + } + this.config.set(HConstants.ZOOKEEPER_QUORUM, zkQuorum); + this.config.setInt(HConstants.ZOOKEEPER_CLIENT_PORT, port); + } + String hbaseZnode = context.getString( + HBaseSinkConfigurationConstants.ZK_ZNODE_PARENT); + if(hbaseZnode != null && !hbaseZnode.isEmpty()) { + this.config.set(HConstants.ZOOKEEPER_ZNODE_PARENT, hbaseZnode); + } sinkCounter = new SinkCounter(this.getName()); } + public Configuration getConfig() { + return config; + } + @Override public Status process() throws EventDeliveryException { Status status = Status.READY; diff --git a/flume-ng-sinks/flume-ng-hbase-sink/src/test/java/org/apache/flume/sink/hbase/TestHBaseSink.java b/flume-ng-sinks/flume-ng-hbase-sink/src/test/java/org/apache/flume/sink/hbase/TestHBaseSink.java index 068f543b1d..cb7c6ea201 100644 --- a/flume-ng-sinks/flume-ng-hbase-sink/src/test/java/org/apache/flume/sink/hbase/TestHBaseSink.java +++ b/flume-ng-sinks/flume-ng-hbase-sink/src/test/java/org/apache/flume/sink/hbase/TestHBaseSink.java @@ -42,6 +42,7 @@ import org.apache.hadoop.hbase.client.ResultScanner; import org.apache.hadoop.hbase.client.Scan; import org.apache.hadoop.hbase.util.Bytes; +import org.apache.hadoop.hbase.zookeeper.ZKConfig; import org.junit.AfterClass; import org.junit.BeforeClass; import org.junit.Ignore; @@ -77,6 +78,8 @@ public static void tearDown() throws Exception { testUtility.shutdownMiniCluster(); } + + @Test public void testOneEventWithDefaults() throws Exception { //Create a context without setting increment column and payload Column @@ -90,7 +93,7 @@ public void testOneEventWithDefaults() throws Exception { testUtility.createTable(tableName.getBytes(), columnFamily.getBytes()); HBaseSink sink = new HBaseSink(testUtility.getConfiguration()); - Configurables.configure(sink, tmpctx); + Configurables.configure(sink, ctx); Channel channel = new MemoryChannel(); Configurables.configure(channel, new Context()); sink.setChannel(channel); @@ -440,6 +443,82 @@ public void testTransactionStateOnSerializationException() throws Exception { testUtility.deleteTable(tableName.getBytes()); } + @Test + public void testWithoutConfigurationObject() throws Exception{ + ctx.put("batchSize", "2"); + ctx.put(HBaseSinkConfigurationConstants.ZK_QUORUM, + ZKConfig.getZKQuorumServersString(testUtility.getConfiguration()) ); + System.out.print(ctx.getString(HBaseSinkConfigurationConstants.ZK_QUORUM)); + ctx.put(HBaseSinkConfigurationConstants.ZK_ZNODE_PARENT, + testUtility.getConfiguration().get(HConstants.ZOOKEEPER_ZNODE_PARENT)); + testUtility.createTable(tableName.getBytes(), columnFamily.getBytes()); + HBaseSink sink = new HBaseSink(); + Configurables.configure(sink, ctx); + // Reset context to values usable by other tests. + ctx.put(HBaseSinkConfigurationConstants.ZK_QUORUM, null); + ctx.put(HBaseSinkConfigurationConstants.ZK_ZNODE_PARENT,null); + ctx.put("batchSize", "100"); + Channel channel = new MemoryChannel(); + Configurables.configure(channel, ctx); + sink.setChannel(channel); + sink.start(); + Transaction tx = channel.getTransaction(); + tx.begin(); + for(int i = 0; i < 3; i++){ + Event e = EventBuilder.withBody(Bytes.toBytes(valBase + "-" + i)); + channel.put(e); + } + tx.commit(); + tx.close(); + Status status = Status.READY; + while(status != Status.BACKOFF){ + status = sink.process(); + } + sink.stop(); + HTable table = new HTable(testUtility.getConfiguration(), tableName); + byte[][] results = getResults(table, 3); + byte[] out; + int found = 0; + for(int i = 0; i < 3; i++){ + for(int j = 0; j < 3; j++){ + if(Arrays.equals(results[j],Bytes.toBytes(valBase + "-" + i))){ + found++; + break; + } + } + } + Assert.assertEquals(3, found); + out = results[3]; + Assert.assertArrayEquals(Longs.toByteArray(3), out); + } -} + @Test + public void testZKQuorum() throws Exception{ + String zkQuorum = "zk1.flume.apache.org:3342, zk2.flume.apache.org:3342, " + + "zk3.flume.apache.org:3342"; + ctx.put("batchSize", "2"); + ctx.put(HBaseSinkConfigurationConstants.ZK_QUORUM, zkQuorum); + ctx.put(HBaseSinkConfigurationConstants.ZK_ZNODE_PARENT, + testUtility.getConfiguration().get(HConstants.ZOOKEEPER_ZNODE_PARENT)); + HBaseSink sink = new HBaseSink(); + Configurables.configure(sink, ctx); + Assert.assertEquals("zk1.flume.apache.org,zk2.flume.apache.org," + + "zk3.flume.apache.org", sink.getConfig().get(HConstants + .ZOOKEEPER_QUORUM)); + Assert.assertEquals(String.valueOf(3342), sink.getConfig().get(HConstants + .ZOOKEEPER_CLIENT_PORT)); + } + @Test (expected = FlumeException.class) + public void testZKQuorumIncorrectPorts() throws Exception{ + String zkQuorum = "zk1.flume.apache.org:3345, zk2.flume.apache.org:3342, " + + "zk3.flume.apache.org:3342"; + ctx.put("batchSize", "2"); + ctx.put(HBaseSinkConfigurationConstants.ZK_QUORUM, zkQuorum); + ctx.put(HBaseSinkConfigurationConstants.ZK_ZNODE_PARENT, + testUtility.getConfiguration().get(HConstants.ZOOKEEPER_ZNODE_PARENT)); + HBaseSink sink = new HBaseSink(); + Configurables.configure(sink, ctx); + Assert.fail(); + } +} \ No newline at end of file From ad5f286eafd0912e9db0b2c3d92d1dc408377b12 Mon Sep 17 00:00:00 2001 From: Jarek Jarcec Cecho Date: Sat, 1 Mar 2014 07:27:10 -0800 Subject: [PATCH 084/341] FLUME-2335: TestHBaseSink#testWithoutConfigurationObject() must delete the table at the end of the test (Hari Shreedharan via Jarek Jarcec Cecho) --- .../flume/sink/hbase/TestHBaseSink.java | 46 ++++++++++--------- 1 file changed, 25 insertions(+), 21 deletions(-) diff --git a/flume-ng-sinks/flume-ng-hbase-sink/src/test/java/org/apache/flume/sink/hbase/TestHBaseSink.java b/flume-ng-sinks/flume-ng-hbase-sink/src/test/java/org/apache/flume/sink/hbase/TestHBaseSink.java index cb7c6ea201..20b7fe5c93 100644 --- a/flume-ng-sinks/flume-ng-hbase-sink/src/test/java/org/apache/flume/sink/hbase/TestHBaseSink.java +++ b/flume-ng-sinks/flume-ng-hbase-sink/src/test/java/org/apache/flume/sink/hbase/TestHBaseSink.java @@ -36,6 +36,7 @@ import org.apache.flume.channel.MemoryChannel; import org.apache.flume.conf.Configurables; import org.apache.flume.event.EventBuilder; +import org.apache.hadoop.conf.Configuration; import org.apache.hadoop.hbase.*; import org.apache.hadoop.hbase.client.HTable; import org.apache.hadoop.hbase.client.Result; @@ -59,6 +60,7 @@ public class TestHBaseSink { private static String plCol = "pCol"; private static Context ctx = new Context(); private static String valBase = "testing hbase sink: jham"; + private static Configuration conf; @BeforeClass public static void setUp() throws Exception { @@ -71,6 +73,7 @@ public static void setUp() throws Exception { ctxMap.put("serializer.payloadColumn", plCol); ctxMap.put("serializer.incrementColumn", inColumn); ctx.putAll(ctxMap); + conf = new Configuration(testUtility.getConfiguration()); } @AfterClass @@ -92,7 +95,7 @@ public void testOneEventWithDefaults() throws Exception { tmpctx.putAll(ctxMap); testUtility.createTable(tableName.getBytes(), columnFamily.getBytes()); - HBaseSink sink = new HBaseSink(testUtility.getConfiguration()); + HBaseSink sink = new HBaseSink(conf); Configurables.configure(sink, ctx); Channel channel = new MemoryChannel(); Configurables.configure(channel, new Context()); @@ -108,7 +111,7 @@ public void testOneEventWithDefaults() throws Exception { sink.process(); sink.stop(); - HTable table = new HTable(testUtility.getConfiguration(), tableName); + HTable table = new HTable(conf, tableName); byte[][] results = getResults(table, 1); byte[] out = results[0]; Assert.assertArrayEquals(e.getBody(), out); @@ -120,7 +123,7 @@ public void testOneEventWithDefaults() throws Exception { @Test public void testOneEvent() throws Exception { testUtility.createTable(tableName.getBytes(), columnFamily.getBytes()); - HBaseSink sink = new HBaseSink(testUtility.getConfiguration()); + HBaseSink sink = new HBaseSink(conf); Configurables.configure(sink, ctx); Channel channel = new MemoryChannel(); Configurables.configure(channel, new Context()); @@ -136,7 +139,7 @@ public void testOneEvent() throws Exception { sink.process(); sink.stop(); - HTable table = new HTable(testUtility.getConfiguration(), tableName); + HTable table = new HTable(conf, tableName); byte[][] results = getResults(table, 1); byte[] out = results[0]; Assert.assertArrayEquals(e.getBody(), out); @@ -149,7 +152,7 @@ public void testOneEvent() throws Exception { public void testThreeEvents() throws Exception { testUtility.createTable(tableName.getBytes(), columnFamily.getBytes()); ctx.put("batchSize", "3"); - HBaseSink sink = new HBaseSink(testUtility.getConfiguration()); + HBaseSink sink = new HBaseSink(conf); Configurables.configure(sink, ctx); Channel channel = new MemoryChannel(); Configurables.configure(channel, new Context()); @@ -165,7 +168,7 @@ public void testThreeEvents() throws Exception { tx.close(); sink.process(); sink.stop(); - HTable table = new HTable(testUtility.getConfiguration(), tableName); + HTable table = new HTable(conf, tableName); byte[][] results = getResults(table, 3); byte[] out; int found = 0; @@ -187,7 +190,7 @@ public void testThreeEvents() throws Exception { public void testMultipleBatches() throws Exception { testUtility.createTable(tableName.getBytes(), columnFamily.getBytes()); ctx.put("batchSize", "2"); - HBaseSink sink = new HBaseSink(testUtility.getConfiguration()); + HBaseSink sink = new HBaseSink(conf); Configurables.configure(sink, ctx); //Reset the context to a higher batchSize ctx.put("batchSize", "100"); @@ -209,7 +212,7 @@ public void testMultipleBatches() throws Exception { } sink.stop(); Assert.assertEquals(2, count); - HTable table = new HTable(testUtility.getConfiguration(), tableName); + HTable table = new HTable(conf, tableName); byte[][] results = getResults(table, 3); byte[] out; int found = 0; @@ -230,7 +233,7 @@ public void testMultipleBatches() throws Exception { @Test(expected = FlumeException.class) public void testMissingTable() throws Exception { ctx.put("batchSize", "2"); - HBaseSink sink = new HBaseSink(testUtility.getConfiguration()); + HBaseSink sink = new HBaseSink(conf); Configurables.configure(sink, ctx); //Reset the context to a higher batchSize ctx.put("batchSize", "100"); @@ -247,7 +250,7 @@ public void testMissingTable() throws Exception { tx.commit(); tx.close(); sink.process(); - HTable table = new HTable(testUtility.getConfiguration(), tableName); + HTable table = new HTable(conf, tableName); byte[][] results = getResults(table, 2); byte[] out; int found = 0; @@ -279,7 +282,7 @@ public void testMissingTable() throws Exception { public void testHBaseFailure() throws Exception { ctx.put("batchSize", "2"); testUtility.createTable(tableName.getBytes(), columnFamily.getBytes()); - HBaseSink sink = new HBaseSink(testUtility.getConfiguration()); + HBaseSink sink = new HBaseSink(conf); Configurables.configure(sink, ctx); //Reset the context to a higher batchSize ctx.put("batchSize", "100"); @@ -296,7 +299,7 @@ public void testHBaseFailure() throws Exception { tx.commit(); tx.close(); sink.process(); - HTable table = new HTable(testUtility.getConfiguration(), tableName); + HTable table = new HTable(conf, tableName); byte[][] results = getResults(table, 2); byte[] out; int found = 0; @@ -373,7 +376,7 @@ private byte[][] getResults(HTable table, int numEvents) throws IOException{ public void testTransactionStateOnChannelException() throws Exception { ctx.put("batchSize", "1"); testUtility.createTable(tableName.getBytes(), columnFamily.getBytes()); - HBaseSink sink = new HBaseSink(testUtility.getConfiguration()); + HBaseSink sink = new HBaseSink(conf); Configurables.configure(sink, ctx); // Reset the context to a higher batchSize Channel channel = spy(new MemoryChannel()); @@ -396,7 +399,7 @@ public void testTransactionStateOnChannelException() throws Exception { doReturn(e).when(channel).take(); sink.process(); sink.stop(); - HTable table = new HTable(testUtility.getConfiguration(), tableName); + HTable table = new HTable(conf, tableName); byte[][] results = getResults(table, 1); byte[] out = results[0]; Assert.assertArrayEquals(e.getBody(), out); @@ -411,7 +414,7 @@ public void testTransactionStateOnSerializationException() throws Exception { ctx.put(HBaseSinkConfigurationConstants.CONFIG_SERIALIZER, "org.apache.flume.sink.hbase.MockSimpleHbaseEventSerializer"); testUtility.createTable(tableName.getBytes(), columnFamily.getBytes()); - HBaseSink sink = new HBaseSink(testUtility.getConfiguration()); + HBaseSink sink = new HBaseSink(conf); Configurables.configure(sink, ctx); // Reset the context to a higher batchSize Channel channel = new MemoryChannel(); @@ -434,7 +437,7 @@ public void testTransactionStateOnSerializationException() throws Exception { MockSimpleHbaseEventSerializer.throwException = false; sink.process(); sink.stop(); - HTable table = new HTable(testUtility.getConfiguration(), tableName); + HTable table = new HTable(conf, tableName); byte[][] results = getResults(table, 1); byte[] out = results[0]; Assert.assertArrayEquals(e.getBody(), out); @@ -447,10 +450,10 @@ public void testTransactionStateOnSerializationException() throws Exception { public void testWithoutConfigurationObject() throws Exception{ ctx.put("batchSize", "2"); ctx.put(HBaseSinkConfigurationConstants.ZK_QUORUM, - ZKConfig.getZKQuorumServersString(testUtility.getConfiguration()) ); + ZKConfig.getZKQuorumServersString(conf) ); System.out.print(ctx.getString(HBaseSinkConfigurationConstants.ZK_QUORUM)); ctx.put(HBaseSinkConfigurationConstants.ZK_ZNODE_PARENT, - testUtility.getConfiguration().get(HConstants.ZOOKEEPER_ZNODE_PARENT)); + conf.get(HConstants.ZOOKEEPER_ZNODE_PARENT)); testUtility.createTable(tableName.getBytes(), columnFamily.getBytes()); HBaseSink sink = new HBaseSink(); Configurables.configure(sink, ctx); @@ -475,7 +478,7 @@ public void testWithoutConfigurationObject() throws Exception{ status = sink.process(); } sink.stop(); - HTable table = new HTable(testUtility.getConfiguration(), tableName); + HTable table = new HTable(conf, tableName); byte[][] results = getResults(table, 3); byte[] out; int found = 0; @@ -490,6 +493,7 @@ public void testWithoutConfigurationObject() throws Exception{ Assert.assertEquals(3, found); out = results[3]; Assert.assertArrayEquals(Longs.toByteArray(3), out); + testUtility.deleteTable(tableName.getBytes()); } @Test @@ -499,7 +503,7 @@ public void testZKQuorum() throws Exception{ ctx.put("batchSize", "2"); ctx.put(HBaseSinkConfigurationConstants.ZK_QUORUM, zkQuorum); ctx.put(HBaseSinkConfigurationConstants.ZK_ZNODE_PARENT, - testUtility.getConfiguration().get(HConstants.ZOOKEEPER_ZNODE_PARENT)); + conf.get(HConstants.ZOOKEEPER_ZNODE_PARENT)); HBaseSink sink = new HBaseSink(); Configurables.configure(sink, ctx); Assert.assertEquals("zk1.flume.apache.org,zk2.flume.apache.org," + @@ -516,7 +520,7 @@ public void testZKQuorumIncorrectPorts() throws Exception{ ctx.put("batchSize", "2"); ctx.put(HBaseSinkConfigurationConstants.ZK_QUORUM, zkQuorum); ctx.put(HBaseSinkConfigurationConstants.ZK_ZNODE_PARENT, - testUtility.getConfiguration().get(HConstants.ZOOKEEPER_ZNODE_PARENT)); + conf.get(HConstants.ZOOKEEPER_ZNODE_PARENT)); HBaseSink sink = new HBaseSink(); Configurables.configure(sink, ctx); Assert.fail(); From a6a6c4c2a0420f42924cc31f304f4bc3ca8348ea Mon Sep 17 00:00:00 2001 From: Jarek Jarcec Cecho Date: Sat, 1 Mar 2014 16:01:49 -0800 Subject: [PATCH 085/341] FLUME-2336: HBase tests that pass in ZK configs must use a new context object (Hari Shreedharan via Jarek Jarcec Cecho) --- .../flume/sink/hbase/TestHBaseSink.java | 42 ++++++++++--------- 1 file changed, 23 insertions(+), 19 deletions(-) diff --git a/flume-ng-sinks/flume-ng-hbase-sink/src/test/java/org/apache/flume/sink/hbase/TestHBaseSink.java b/flume-ng-sinks/flume-ng-hbase-sink/src/test/java/org/apache/flume/sink/hbase/TestHBaseSink.java index 20b7fe5c93..d1b0182593 100644 --- a/flume-ng-sinks/flume-ng-hbase-sink/src/test/java/org/apache/flume/sink/hbase/TestHBaseSink.java +++ b/flume-ng-sinks/flume-ng-hbase-sink/src/test/java/org/apache/flume/sink/hbase/TestHBaseSink.java @@ -417,6 +417,7 @@ public void testTransactionStateOnSerializationException() throws Exception { HBaseSink sink = new HBaseSink(conf); Configurables.configure(sink, ctx); // Reset the context to a higher batchSize + ctx.put("batchSize", "100"); Channel channel = new MemoryChannel(); Configurables.configure(channel, new Context()); sink.setChannel(channel); @@ -448,19 +449,17 @@ public void testTransactionStateOnSerializationException() throws Exception { @Test public void testWithoutConfigurationObject() throws Exception{ - ctx.put("batchSize", "2"); - ctx.put(HBaseSinkConfigurationConstants.ZK_QUORUM, + Context tmpContext = new Context(ctx.getParameters()); + tmpContext.put("batchSize", "2"); + tmpContext.put(HBaseSinkConfigurationConstants.ZK_QUORUM, ZKConfig.getZKQuorumServersString(conf) ); System.out.print(ctx.getString(HBaseSinkConfigurationConstants.ZK_QUORUM)); - ctx.put(HBaseSinkConfigurationConstants.ZK_ZNODE_PARENT, - conf.get(HConstants.ZOOKEEPER_ZNODE_PARENT)); + tmpContext.put(HBaseSinkConfigurationConstants.ZK_ZNODE_PARENT, + conf.get(HConstants.ZOOKEEPER_ZNODE_PARENT, + HConstants.DEFAULT_ZOOKEEPER_ZNODE_PARENT)); testUtility.createTable(tableName.getBytes(), columnFamily.getBytes()); HBaseSink sink = new HBaseSink(); - Configurables.configure(sink, ctx); - // Reset context to values usable by other tests. - ctx.put(HBaseSinkConfigurationConstants.ZK_QUORUM, null); - ctx.put(HBaseSinkConfigurationConstants.ZK_ZNODE_PARENT,null); - ctx.put("batchSize", "100"); + Configurables.configure(sink, tmpContext); Channel channel = new MemoryChannel(); Configurables.configure(channel, ctx); sink.setChannel(channel); @@ -498,14 +497,16 @@ public void testWithoutConfigurationObject() throws Exception{ @Test public void testZKQuorum() throws Exception{ + Context tmpContext = new Context(ctx.getParameters()); String zkQuorum = "zk1.flume.apache.org:3342, zk2.flume.apache.org:3342, " + "zk3.flume.apache.org:3342"; - ctx.put("batchSize", "2"); - ctx.put(HBaseSinkConfigurationConstants.ZK_QUORUM, zkQuorum); - ctx.put(HBaseSinkConfigurationConstants.ZK_ZNODE_PARENT, - conf.get(HConstants.ZOOKEEPER_ZNODE_PARENT)); + tmpContext.put("batchSize", "2"); + tmpContext.put(HBaseSinkConfigurationConstants.ZK_QUORUM, zkQuorum); + tmpContext.put(HBaseSinkConfigurationConstants.ZK_ZNODE_PARENT, + conf.get(HConstants.ZOOKEEPER_ZNODE_PARENT, + HConstants.DEFAULT_ZOOKEEPER_ZNODE_PARENT)); HBaseSink sink = new HBaseSink(); - Configurables.configure(sink, ctx); + Configurables.configure(sink, tmpContext); Assert.assertEquals("zk1.flume.apache.org,zk2.flume.apache.org," + "zk3.flume.apache.org", sink.getConfig().get(HConstants .ZOOKEEPER_QUORUM)); @@ -515,14 +516,17 @@ public void testZKQuorum() throws Exception{ @Test (expected = FlumeException.class) public void testZKQuorumIncorrectPorts() throws Exception{ + Context tmpContext = new Context(ctx.getParameters()); + String zkQuorum = "zk1.flume.apache.org:3345, zk2.flume.apache.org:3342, " + "zk3.flume.apache.org:3342"; - ctx.put("batchSize", "2"); - ctx.put(HBaseSinkConfigurationConstants.ZK_QUORUM, zkQuorum); - ctx.put(HBaseSinkConfigurationConstants.ZK_ZNODE_PARENT, - conf.get(HConstants.ZOOKEEPER_ZNODE_PARENT)); + tmpContext.put("batchSize", "2"); + tmpContext.put(HBaseSinkConfigurationConstants.ZK_QUORUM, zkQuorum); + tmpContext.put(HBaseSinkConfigurationConstants.ZK_ZNODE_PARENT, + conf.get(HConstants.ZOOKEEPER_ZNODE_PARENT, + HConstants.DEFAULT_ZOOKEEPER_ZNODE_PARENT)); HBaseSink sink = new HBaseSink(); - Configurables.configure(sink, ctx); + Configurables.configure(sink, tmpContext); Assert.fail(); } } \ No newline at end of file From 674f4fcce2597e7e934ccc69eb04b426f5a9b8bb Mon Sep 17 00:00:00 2001 From: Hari Shreedharan Date: Sun, 2 Mar 2014 23:49:17 -0800 Subject: [PATCH 086/341] FLUME-2338. Support coalescing increments in HBaseSink. (Mike Percy via Hari Shreedharan) --- flume-ng-doc/sphinx/FlumeUserGuide.rst | 2 + .../apache/flume/sink/hbase/BatchAware.java | 28 ++ .../apache/flume/sink/hbase/HBaseSink.java | 195 +++++++++++- .../sink/hbase/IncrementHBaseSerializer.java | 80 +++++ .../flume/sink/hbase/TestHBaseSink.java | 298 +++++++++++++++--- 5 files changed, 550 insertions(+), 53 deletions(-) create mode 100644 flume-ng-sinks/flume-ng-hbase-sink/src/main/java/org/apache/flume/sink/hbase/BatchAware.java create mode 100644 flume-ng-sinks/flume-ng-hbase-sink/src/test/java/org/apache/flume/sink/hbase/IncrementHBaseSerializer.java diff --git a/flume-ng-doc/sphinx/FlumeUserGuide.rst b/flume-ng-doc/sphinx/FlumeUserGuide.rst index 96bf73e0cc..cedb2830b8 100644 --- a/flume-ng-doc/sphinx/FlumeUserGuide.rst +++ b/flume-ng-doc/sphinx/FlumeUserGuide.rst @@ -1839,6 +1839,8 @@ Property Name Default Desc zookeeperQuorum -- The quorum spec. This is the value for the property ``hbase.zookeeper.quorum`` in hbase-site.xml znodeParent /hbase The base path for the znode for the -ROOT- region. Value of ``zookeeper.znode.parent`` in hbase-site.xml batchSize 100 Number of events to be written per txn. +coalesceIncrements false Should the sink coalesce multiple increments to a cell per batch. This might give + better performance if there are multiple increments to a limited number of cells. serializer org.apache.flume.sink.hbase.SimpleHbaseEventSerializer Default increment column = "iCol", payload column = "pCol". serializer.* -- Properties to be passed to the serializer. kerberosPrincipal -- Kerberos user principal for accessing secure HBase diff --git a/flume-ng-sinks/flume-ng-hbase-sink/src/main/java/org/apache/flume/sink/hbase/BatchAware.java b/flume-ng-sinks/flume-ng-hbase-sink/src/main/java/org/apache/flume/sink/hbase/BatchAware.java new file mode 100644 index 0000000000..0974241db3 --- /dev/null +++ b/flume-ng-sinks/flume-ng-hbase-sink/src/main/java/org/apache/flume/sink/hbase/BatchAware.java @@ -0,0 +1,28 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, + * software distributed under the License is distributed on an + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + * KIND, either express or implied. See the License for the + * specific language governing permissions and limitations + * under the License. + */ +package org.apache.flume.sink.hbase; + +/** + * This interface allows for implementing HBase serializers that are aware of + * batching. {@link #onBatchStart()} is called at the beginning of each batch + * by the sink. + */ +public interface BatchAware { + public void onBatchStart(); +} diff --git a/flume-ng-sinks/flume-ng-hbase-sink/src/main/java/org/apache/flume/sink/hbase/HBaseSink.java b/flume-ng-sinks/flume-ng-hbase-sink/src/main/java/org/apache/flume/sink/hbase/HBaseSink.java index c4a666cacd..0390ff88ce 100644 --- a/flume-ng-sinks/flume-ng-hbase-sink/src/main/java/org/apache/flume/sink/hbase/HBaseSink.java +++ b/flume-ng-sinks/flume-ng-hbase-sink/src/main/java/org/apache/flume/sink/hbase/HBaseSink.java @@ -19,15 +19,23 @@ package org.apache.flume.sink.hbase; import java.io.IOException; +import java.lang.reflect.InvocationTargetException; +import java.lang.reflect.Method; import java.util.LinkedList; import java.util.List; +import java.util.Map; +import java.util.NavigableMap; +import com.google.common.annotations.VisibleForTesting; +import com.google.common.collect.Lists; +import com.google.common.collect.Maps; import org.apache.flume.Channel; import org.apache.flume.Context; import org.apache.flume.Event; import org.apache.flume.EventDeliveryException; import org.apache.flume.FlumeException; import org.apache.flume.Transaction; +import org.apache.flume.annotations.InterfaceAudience; import org.apache.flume.conf.Configurable; import org.apache.flume.instrumentation.SinkCounter; import org.apache.flume.sink.AbstractSink; @@ -52,7 +60,7 @@ /** * * A simple sink which reads events from a channel and writes them to HBase. - * The Hbase configution is picked up from the first hbase-site.xml + * The Hbase configuration is picked up from the first hbase-site.xml * encountered in the classpath. This sink supports batch reading of * events from the channel, and writing them to Hbase, to minimize the number * of flushes on the hbase tables. To use this sink, it has to be configured @@ -97,8 +105,13 @@ public class HBaseSink extends AbstractSink implements Configurable { private String kerberosKeytab; private User hbaseUser; private boolean enableWal = true; + private boolean batchIncrements = false; + private Method refGetFamilyMap; private SinkCounter sinkCounter; + // Internal hooks used for unit testing. + private DebugIncrementsCallback debugIncrCallback = null; + public HBaseSink(){ this(HBaseConfiguration.create()); } @@ -107,6 +120,13 @@ public HBaseSink(Configuration conf){ this.config = conf; } + @VisibleForTesting + @InterfaceAudience.Private + HBaseSink(Configuration conf, DebugIncrementsCallback cb) { + this(conf); + this.debugIncrCallback = cb; + } + @Override public void start(){ Preconditions.checkArgument(table == null, "Please call stop " + @@ -222,6 +242,17 @@ public void configure(Context context){ "writes to HBase will have WAL disabled, and any data in the " + "memstore of this region in the Region Server could be lost!"); } + + batchIncrements = context.getBoolean( + HBaseSinkConfigurationConstants.CONFIG_COALESCE_INCREMENTS, + HBaseSinkConfigurationConstants.DEFAULT_COALESCE_INCREMENTS); + + if (batchIncrements) { + logger.info("Increment coalescing is enabled. Increments will be " + + "buffered."); + reflectLookupGetFamilyMap(); + } + String zkQuorum = context.getString(HBaseSinkConfigurationConstants .ZK_QUORUM); Integer port = null; @@ -281,6 +312,11 @@ public Status process() throws EventDeliveryException { List incs = new LinkedList(); try { txn.begin(); + + if (serializer instanceof BatchAware) { + ((BatchAware)serializer).onBatchStart(); + } + long i = 0; for (; i < batchSize; i++) { Event event = channel.take(); @@ -309,7 +345,7 @@ public Status process() throws EventDeliveryException { try{ txn.rollback(); } catch (Exception e2) { - logger.error("Exception in rollback. Rollback might not have been" + + logger.error("Exception in rollback. Rollback might not have been " + "successful." , e2); } logger.error("Failed to commit transaction." + @@ -353,7 +389,20 @@ public Void run() throws Exception { runPrivileged(new PrivilegedExceptionAction() { @Override public Void run() throws Exception { - for (final Increment i : incs) { + + List processedIncrements; + if (batchIncrements) { + processedIncrements = coalesceIncrements(incs); + } else { + processedIncrements = incs; + } + + // Only used for unit testing. + if (debugIncrCallback != null) { + debugIncrCallback.onAfterCoalesce(processedIncrements); + } + + for (final Increment i : processedIncrements) { i.setWriteToWAL(enableWal); table.increment(i); } @@ -364,6 +413,7 @@ public Void run() throws Exception { txn.commit(); sinkCounter.addToEventDrainSuccessCount(actions.size()); } + private T runPrivileged(final PrivilegedExceptionAction action) throws Exception { if(hbaseUser != null) { @@ -375,4 +425,143 @@ private T runPrivileged(final PrivilegedExceptionAction action) return action.run(); } } + + /** + * The method getFamilyMap() is no longer available in Hbase 0.96. + * We must use reflection to determine which version we may use. + */ + private void reflectLookupGetFamilyMap() { + refGetFamilyMap = null; + String[] methodNames = { "getFamilyMap", "getFamilyMapOfLongs" }; + for (String methodName : methodNames) { + try { + refGetFamilyMap = Increment.class.getMethod(methodName); + if (refGetFamilyMap != null) { + logger.debug("Using Increment.{} for coalesce", methodName); + break; + } + } catch (NoSuchMethodException e) { + logger.debug("Increment.{} does not exist. Exception follows.", + methodName, e); + } catch (SecurityException e) { + logger.debug("No access to Increment.{}; Exception follows.", + methodName, e); + } + } + if (refGetFamilyMap == null) { + throw new UnsupportedOperationException( + "Cannot find Increment.getFamilyMap()"); + } + } + + @SuppressWarnings("unchecked") + private Map> getFamilyMap(Increment inc) { + Preconditions.checkNotNull(refGetFamilyMap, + "Increment.getFamilymap() not found"); + Preconditions.checkNotNull(inc, "Increment required"); + Map> familyMap = null; + try { + Object familyObj = refGetFamilyMap.invoke(inc); + familyMap = (Map>) familyObj; + } catch (IllegalAccessException e) { + logger.warn("Unexpected error calling getFamilyMap()", e); + Throwables.propagate(e); + } catch (InvocationTargetException e) { + logger.warn("Unexpected error calling getFamilyMap()", e); + Throwables.propagate(e); + } + return familyMap; + } + + /** + * Perform "compression" on the given set of increments so that Flume sends + * the minimum possible number of RPC operations to HBase per batch. + * @param incs Input: Increment objects to coalesce. + * @return List of new Increment objects after coalescing the unique counts. + */ + private List coalesceIncrements(Iterable incs) { + Preconditions.checkNotNull(incs, "List of Increments must not be null"); + // Aggregate all of the increment row/family/column counts. + // The nested map is keyed like this: {row, family, qualifier} => count. + Map>> counters = + Maps.newTreeMap(Bytes.BYTES_COMPARATOR); + for (Increment inc : incs) { + byte[] row = inc.getRow(); + Map> families = getFamilyMap(inc); + for (Map.Entry> familyEntry : families.entrySet()) { + byte[] family = familyEntry.getKey(); + NavigableMap qualifiers = familyEntry.getValue(); + for (Map.Entry qualifierEntry : qualifiers.entrySet()) { + byte[] qualifier = qualifierEntry.getKey(); + Long count = qualifierEntry.getValue(); + incrementCounter(counters, row, family, qualifier, count); + } + } + } + + // Reconstruct list of Increments per unique row/family/qualifier. + List coalesced = Lists.newLinkedList(); + for (Map.Entry>> rowEntry : counters.entrySet()) { + byte[] row = rowEntry.getKey(); + Map > families = rowEntry.getValue(); + Increment inc = new Increment(row); + for (Map.Entry> familyEntry : families.entrySet()) { + byte[] family = familyEntry.getKey(); + NavigableMap qualifiers = familyEntry.getValue(); + for (Map.Entry qualifierEntry : qualifiers.entrySet()) { + byte[] qualifier = qualifierEntry.getKey(); + long count = qualifierEntry.getValue(); + inc.addColumn(family, qualifier, count); + } + } + coalesced.add(inc); + } + + return coalesced; + } + + /** + * Helper function for {@link #coalesceIncrements} to increment a counter + * value in the passed data structure. + * @param counters Nested data structure containing the counters. + * @param row Row key to increment. + * @param family Column family to increment. + * @param qualifier Column qualifier to increment. + * @param count Amount to increment by. + */ + private void incrementCounter( + Map>> counters, + byte[] row, byte[] family, byte[] qualifier, Long count) { + + Map> families = counters.get(row); + if (families == null) { + families = Maps.newTreeMap(Bytes.BYTES_COMPARATOR); + counters.put(row, families); + } + + NavigableMap qualifiers = families.get(family); + if (qualifiers == null) { + qualifiers = Maps.newTreeMap(Bytes.BYTES_COMPARATOR); + families.put(family, qualifiers); + } + + Long existingValue = qualifiers.get(qualifier); + if (existingValue == null) { + qualifiers.put(qualifier, count); + } else { + qualifiers.put(qualifier, existingValue + count); + } + } + + @VisibleForTesting + @InterfaceAudience.Private + HbaseEventSerializer getSerializer() { + return serializer; + } + + @VisibleForTesting + @InterfaceAudience.Private + interface DebugIncrementsCallback { + public void onAfterCoalesce(Iterable increments); + } } diff --git a/flume-ng-sinks/flume-ng-hbase-sink/src/test/java/org/apache/flume/sink/hbase/IncrementHBaseSerializer.java b/flume-ng-sinks/flume-ng-hbase-sink/src/test/java/org/apache/flume/sink/hbase/IncrementHBaseSerializer.java new file mode 100644 index 0000000000..b4343eb5a2 --- /dev/null +++ b/flume-ng-sinks/flume-ng-hbase-sink/src/test/java/org/apache/flume/sink/hbase/IncrementHBaseSerializer.java @@ -0,0 +1,80 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, + * software distributed under the License is distributed on an + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + * KIND, either express or implied. See the License for the + * specific language governing permissions and limitations + * under the License. + */ +package org.apache.flume.sink.hbase; + +import com.google.common.annotations.VisibleForTesting; +import com.google.common.base.Charsets; +import com.google.common.collect.Lists; +import java.util.Collections; +import org.apache.flume.Context; +import org.apache.flume.Event; +import org.apache.flume.conf.ComponentConfiguration; +import org.apache.hadoop.hbase.client.Increment; +import org.apache.hadoop.hbase.client.Row; + +import java.util.List; + +/** + * For Increment-related unit tests. + */ +class IncrementHBaseSerializer implements HbaseEventSerializer, BatchAware { + private Event event; + private byte[] family; + private int numBatchesStarted = 0; + + @Override public void configure(Context context) { } + @Override public void configure(ComponentConfiguration conf) { } + @Override public void close() { } + + @Override + public void initialize(Event event, byte[] columnFamily) { + this.event = event; + this.family = columnFamily; + } + + // This class only creates Increments. + @Override + public List getActions() { + return Collections.emptyList(); + } + + // Treat each Event as a String, i,e, "row:qualifier". + @Override + public List getIncrements() { + List increments = Lists.newArrayList(); + String body = new String(event.getBody(), Charsets.UTF_8); + String[] pieces = body.split(":"); + String row = pieces[0]; + String qualifier = pieces[1]; + Increment inc = new Increment(row.getBytes(Charsets.UTF_8)); + inc.addColumn(family, qualifier.getBytes(Charsets.UTF_8), 1L); + increments.add(inc); + return increments; + } + + @Override + public void onBatchStart() { + numBatchesStarted++; + } + + @VisibleForTesting + public int getNumBatchesStarted() { + return numBatchesStarted; + } +} diff --git a/flume-ng-sinks/flume-ng-hbase-sink/src/test/java/org/apache/flume/sink/hbase/TestHBaseSink.java b/flume-ng-sinks/flume-ng-hbase-sink/src/test/java/org/apache/flume/sink/hbase/TestHBaseSink.java index d1b0182593..5b047dcdcf 100644 --- a/flume-ng-sinks/flume-ng-hbase-sink/src/test/java/org/apache/flume/sink/hbase/TestHBaseSink.java +++ b/flume-ng-sinks/flume-ng-hbase-sink/src/test/java/org/apache/flume/sink/hbase/TestHBaseSink.java @@ -22,9 +22,13 @@ import java.io.IOException; import java.util.Arrays; -import java.util.HashMap; +import java.util.List; import java.util.Map; - +import java.util.NavigableMap; +import com.google.common.base.Charsets; +import com.google.common.collect.Lists; +import com.google.common.collect.Maps; +import com.google.common.primitives.Longs; import org.apache.flume.Channel; import org.apache.flume.ChannelException; import org.apache.flume.Context; @@ -37,64 +41,95 @@ import org.apache.flume.conf.Configurables; import org.apache.flume.event.EventBuilder; import org.apache.hadoop.conf.Configuration; -import org.apache.hadoop.hbase.*; +import org.apache.hadoop.hbase.HBaseTestingUtility; +import org.apache.hadoop.hbase.HConstants; import org.apache.hadoop.hbase.client.HTable; +import org.apache.hadoop.hbase.client.Increment; import org.apache.hadoop.hbase.client.Result; import org.apache.hadoop.hbase.client.ResultScanner; import org.apache.hadoop.hbase.client.Scan; import org.apache.hadoop.hbase.util.Bytes; import org.apache.hadoop.hbase.zookeeper.ZKConfig; +import org.junit.After; import org.junit.AfterClass; +import org.junit.Before; import org.junit.BeforeClass; import org.junit.Ignore; import org.junit.Test; import org.junit.Assert; - -import com.google.common.primitives.Longs; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; public class TestHBaseSink { - private static HBaseTestingUtility testUtility = new HBaseTestingUtility(); - private static String tableName = "TestHbaseSink"; - private static String columnFamily = "TestColumnFamily"; - private static String inColumn = "iCol"; - private static String plCol = "pCol"; - private static Context ctx = new Context(); - private static String valBase = "testing hbase sink: jham"; - private static Configuration conf; + private static final Logger logger = + LoggerFactory.getLogger(TestHBaseSink.class); + + private static final HBaseTestingUtility testUtility = new HBaseTestingUtility(); + private static final String tableName = "TestHbaseSink"; + private static final String columnFamily = "TestColumnFamily"; + private static final String inColumn = "iCol"; + private static final String plCol = "pCol"; + private static final String valBase = "testing hbase sink: jham"; + + private Configuration conf; + private Context ctx; @BeforeClass - public static void setUp() throws Exception { + public static void setUpOnce() throws Exception { testUtility.startMiniCluster(); - Map ctxMap = new HashMap(); - ctxMap.put("table", tableName); - ctxMap.put("columnFamily", columnFamily); - ctxMap.put("serializer", - "org.apache.flume.sink.hbase.SimpleHbaseEventSerializer"); - ctxMap.put("serializer.payloadColumn", plCol); - ctxMap.put("serializer.incrementColumn", inColumn); - ctx.putAll(ctxMap); - conf = new Configuration(testUtility.getConfiguration()); } @AfterClass - public static void tearDown() throws Exception { + public static void tearDownOnce() throws Exception { testUtility.shutdownMiniCluster(); } + /** + * Most common context setup for unit tests using + * {@link SimpleHbaseEventSerializer}. + */ + @Before + public void setUp() throws IOException { + conf = new Configuration(testUtility.getConfiguration()); + ctx = new Context(); + testUtility.createTable(tableName.getBytes(), columnFamily.getBytes()); + } + @After + public void tearDown() throws IOException { + testUtility.deleteTable(tableName.getBytes()); + } + + /** + * Set up {@link Context} for use with {@link SimpleHbaseEventSerializer}. + */ + private void initContextForSimpleHbaseEventSerializer() { + ctx = new Context(); + ctx.put("table", tableName); + ctx.put("columnFamily", columnFamily); + ctx.put("serializer", SimpleHbaseEventSerializer.class.getName()); + ctx.put("serializer.payloadColumn", plCol); + ctx.put("serializer.incrementColumn", inColumn); + } + + /** + * Set up {@link Context} for use with {@link IncrementHBaseSerializer}. + */ + private void initContextForIncrementHBaseSerializer() { + ctx = new Context(); + ctx.put("table", tableName); + ctx.put("columnFamily", columnFamily); + ctx.put("serializer", IncrementHBaseSerializer.class.getName()); + } @Test public void testOneEventWithDefaults() throws Exception { //Create a context without setting increment column and payload Column - Map ctxMap = new HashMap(); - ctxMap.put("table", tableName); - ctxMap.put("columnFamily", columnFamily); - ctxMap.put("serializer", - "org.apache.flume.sink.hbase.SimpleHbaseEventSerializer"); - Context tmpctx = new Context(); - tmpctx.putAll(ctxMap); + ctx = new Context(); + ctx.put("table", tableName); + ctx.put("columnFamily", columnFamily); + ctx.put("serializer", SimpleHbaseEventSerializer.class.getName()); - testUtility.createTable(tableName.getBytes(), columnFamily.getBytes()); HBaseSink sink = new HBaseSink(conf); Configurables.configure(sink, ctx); Channel channel = new MemoryChannel(); @@ -117,12 +152,11 @@ public void testOneEventWithDefaults() throws Exception { Assert.assertArrayEquals(e.getBody(), out); out = results[1]; Assert.assertArrayEquals(Longs.toByteArray(1), out); - testUtility.deleteTable(tableName.getBytes()); } @Test public void testOneEvent() throws Exception { - testUtility.createTable(tableName.getBytes(), columnFamily.getBytes()); + initContextForSimpleHbaseEventSerializer(); HBaseSink sink = new HBaseSink(conf); Configurables.configure(sink, ctx); Channel channel = new MemoryChannel(); @@ -145,12 +179,11 @@ public void testOneEvent() throws Exception { Assert.assertArrayEquals(e.getBody(), out); out = results[1]; Assert.assertArrayEquals(Longs.toByteArray(1), out); - testUtility.deleteTable(tableName.getBytes()); } @Test public void testThreeEvents() throws Exception { - testUtility.createTable(tableName.getBytes(), columnFamily.getBytes()); + initContextForSimpleHbaseEventSerializer(); ctx.put("batchSize", "3"); HBaseSink sink = new HBaseSink(conf); Configurables.configure(sink, ctx); @@ -183,12 +216,11 @@ public void testThreeEvents() throws Exception { Assert.assertEquals(3, found); out = results[3]; Assert.assertArrayEquals(Longs.toByteArray(3), out); - testUtility.deleteTable(tableName.getBytes()); } @Test public void testMultipleBatches() throws Exception { - testUtility.createTable(tableName.getBytes(), columnFamily.getBytes()); + initContextForSimpleHbaseEventSerializer(); ctx.put("batchSize", "2"); HBaseSink sink = new HBaseSink(conf); Configurables.configure(sink, ctx); @@ -227,11 +259,17 @@ public void testMultipleBatches() throws Exception { Assert.assertEquals(3, found); out = results[3]; Assert.assertArrayEquals(Longs.toByteArray(3), out); - testUtility.deleteTable(tableName.getBytes()); } @Test(expected = FlumeException.class) public void testMissingTable() throws Exception { + logger.info("Running testMissingTable()"); + initContextForSimpleHbaseEventSerializer(); + + // setUp() will create the table, so we delete it. + logger.info("Deleting table {}", tableName); + testUtility.deleteTable(tableName.getBytes()); + ctx.put("batchSize", "2"); HBaseSink sink = new HBaseSink(conf); Configurables.configure(sink, ctx); @@ -240,7 +278,8 @@ public void testMissingTable() throws Exception { Channel channel = new MemoryChannel(); Configurables.configure(channel, new Context()); sink.setChannel(channel); - sink.start(); + + logger.info("Writing data into channel"); Transaction tx = channel.getTransaction(); tx.begin(); for(int i = 0; i < 3; i++){ @@ -249,7 +288,25 @@ public void testMissingTable() throws Exception { } tx.commit(); tx.close(); - sink.process(); + + logger.info("Starting sink and processing events"); + try { + logger.info("Calling sink.start()"); + sink.start(); // This method will throw. + + // We never get here, but we log in case the behavior changes. + logger.error("Unexpected error: Calling sink.process()"); + sink.process(); + logger.error("Unexpected error: Calling sink.stop()"); + sink.stop(); + } finally { + // Re-create the table so tearDown() doesn't throw. + testUtility.createTable(tableName.getBytes(), columnFamily.getBytes()); + } + + // FIXME: The test should never get here, the below code doesn't run. + Assert.fail(); + HTable table = new HTable(conf, tableName); byte[][] results = getResults(table, 2); byte[] out; @@ -266,9 +323,9 @@ public void testMissingTable() throws Exception { out = results[2]; Assert.assertArrayEquals(Longs.toByteArray(2), out); sink.process(); - sink.stop(); } + // TODO: Move this test to a different class and run it stand-alone. /** * This test must run last - it shuts down the minicluster :D * @throws Exception @@ -280,8 +337,8 @@ public void testMissingTable() throws Exception { "and uncomment this annotation to run this test.") @Test(expected = EventDeliveryException.class) public void testHBaseFailure() throws Exception { + initContextForSimpleHbaseEventSerializer(); ctx.put("batchSize", "2"); - testUtility.createTable(tableName.getBytes(), columnFamily.getBytes()); HBaseSink sink = new HBaseSink(conf); Configurables.configure(sink, ctx); //Reset the context to a higher batchSize @@ -374,8 +431,9 @@ private byte[][] getResults(HTable table, int numEvents) throws IOException{ @Test public void testTransactionStateOnChannelException() throws Exception { + initContextForSimpleHbaseEventSerializer(); ctx.put("batchSize", "1"); - testUtility.createTable(tableName.getBytes(), columnFamily.getBytes()); + HBaseSink sink = new HBaseSink(conf); Configurables.configure(sink, ctx); // Reset the context to a higher batchSize @@ -405,15 +463,15 @@ public void testTransactionStateOnChannelException() throws Exception { Assert.assertArrayEquals(e.getBody(), out); out = results[1]; Assert.assertArrayEquals(Longs.toByteArray(1), out); - testUtility.deleteTable(tableName.getBytes()); } @Test public void testTransactionStateOnSerializationException() throws Exception { + initContextForSimpleHbaseEventSerializer(); ctx.put("batchSize", "1"); ctx.put(HBaseSinkConfigurationConstants.CONFIG_SERIALIZER, "org.apache.flume.sink.hbase.MockSimpleHbaseEventSerializer"); - testUtility.createTable(tableName.getBytes(), columnFamily.getBytes()); + HBaseSink sink = new HBaseSink(conf); Configurables.configure(sink, ctx); // Reset the context to a higher batchSize @@ -444,11 +502,11 @@ public void testTransactionStateOnSerializationException() throws Exception { Assert.assertArrayEquals(e.getBody(), out); out = results[1]; Assert.assertArrayEquals(Longs.toByteArray(1), out); - testUtility.deleteTable(tableName.getBytes()); } @Test public void testWithoutConfigurationObject() throws Exception{ + initContextForSimpleHbaseEventSerializer(); Context tmpContext = new Context(ctx.getParameters()); tmpContext.put("batchSize", "2"); tmpContext.put(HBaseSinkConfigurationConstants.ZK_QUORUM, @@ -457,7 +515,7 @@ public void testWithoutConfigurationObject() throws Exception{ tmpContext.put(HBaseSinkConfigurationConstants.ZK_ZNODE_PARENT, conf.get(HConstants.ZOOKEEPER_ZNODE_PARENT, HConstants.DEFAULT_ZOOKEEPER_ZNODE_PARENT)); - testUtility.createTable(tableName.getBytes(), columnFamily.getBytes()); + HBaseSink sink = new HBaseSink(); Configurables.configure(sink, tmpContext); Channel channel = new MemoryChannel(); @@ -492,11 +550,11 @@ public void testWithoutConfigurationObject() throws Exception{ Assert.assertEquals(3, found); out = results[3]; Assert.assertArrayEquals(Longs.toByteArray(3), out); - testUtility.deleteTable(tableName.getBytes()); } @Test public void testZKQuorum() throws Exception{ + initContextForSimpleHbaseEventSerializer(); Context tmpContext = new Context(ctx.getParameters()); String zkQuorum = "zk1.flume.apache.org:3342, zk2.flume.apache.org:3342, " + "zk3.flume.apache.org:3342"; @@ -516,6 +574,7 @@ public void testZKQuorum() throws Exception{ @Test (expected = FlumeException.class) public void testZKQuorumIncorrectPorts() throws Exception{ + initContextForSimpleHbaseEventSerializer(); Context tmpContext = new Context(ctx.getParameters()); String zkQuorum = "zk1.flume.apache.org:3345, zk2.flume.apache.org:3342, " + @@ -529,4 +588,143 @@ public void testZKQuorumIncorrectPorts() throws Exception{ Configurables.configure(sink, tmpContext); Assert.fail(); } -} \ No newline at end of file + + @Test + public void testCoalesce() throws EventDeliveryException { + initContextForIncrementHBaseSerializer(); + ctx.put("batchSize", "100"); + ctx.put(HBaseSinkConfigurationConstants.CONFIG_COALESCE_INCREMENTS, + String.valueOf(true)); + + final Map expectedCounts = Maps.newHashMap(); + expectedCounts.put("r1:c1", 10L); + expectedCounts.put("r1:c2", 20L); + expectedCounts.put("r2:c1", 7L); + expectedCounts.put("r2:c3", 63L); + HBaseSink.DebugIncrementsCallback cb = new CoalesceValidator(expectedCounts); + + HBaseSink sink = new HBaseSink(testUtility.getConfiguration(), cb); + Configurables.configure(sink, ctx); + Channel channel = createAndConfigureMemoryChannel(sink); + + List events = Lists.newLinkedList(); + generateEvents(events, expectedCounts); + putEvents(channel, events); + + sink.start(); + sink.process(); // Calls CoalesceValidator instance. + sink.stop(); + } + + @Test(expected = AssertionError.class) + public void negativeTestCoalesce() throws EventDeliveryException { + initContextForIncrementHBaseSerializer(); + ctx.put("batchSize", "10"); + + final Map expectedCounts = Maps.newHashMap(); + expectedCounts.put("r1:c1", 10L); + HBaseSink.DebugIncrementsCallback cb = new CoalesceValidator(expectedCounts); + + HBaseSink sink = new HBaseSink(testUtility.getConfiguration(), cb); + Configurables.configure(sink, ctx); + Channel channel = createAndConfigureMemoryChannel(sink); + + List events = Lists.newLinkedList(); + generateEvents(events, expectedCounts); + putEvents(channel, events); + + sink.start(); + sink.process(); // Calls CoalesceValidator instance. + sink.stop(); + } + + @Test + public void testBatchAware() throws EventDeliveryException { + logger.info("Running testBatchAware()"); + initContextForIncrementHBaseSerializer(); + HBaseSink sink = new HBaseSink(testUtility.getConfiguration()); + Configurables.configure(sink, ctx); + Channel channel = createAndConfigureMemoryChannel(sink); + + sink.start(); + int batchCount = 3; + for (int i = 0; i < batchCount; i++) { + sink.process(); + } + sink.stop(); + Assert.assertEquals(batchCount, + ((IncrementHBaseSerializer) sink.getSerializer()).getNumBatchesStarted()); + } + + /** + * For testing that the rows coalesced, serialized by + * {@link IncrementHBaseSerializer}, are of the expected batch size. + */ + private static class CoalesceValidator + implements HBaseSink.DebugIncrementsCallback { + + private final Map expectedCounts; + + public CoalesceValidator(Map expectedCounts) { + this.expectedCounts = expectedCounts; + } + + @Override + public void onAfterCoalesce(Iterable increments) { + for (Increment inc : increments) { + byte[] row = inc.getRow(); + Map> families = inc.getFamilyMap(); + for (byte[] family : families.keySet()) { + NavigableMap qualifiers = families.get(family); + for (Map.Entry entry : qualifiers.entrySet()) { + byte[] qualifier = entry.getKey(); + Long count = entry.getValue(); + StringBuilder b = new StringBuilder(20); + b.append(new String(row, Charsets.UTF_8)); + b.append(':'); + b.append(new String(qualifier, Charsets.UTF_8)); + String key = b.toString(); + Assert.assertEquals("Expected counts don't match observed for " + key, + expectedCounts.get(key), count); + } + } + } + } + } + + /** + * Add number of Events corresponding to counts to the events list. + * @param events Destination list. + * @param counts How many events to generate for each row:qualifier pair. + */ + private void generateEvents(List events, Map counts) { + for (String key : counts.keySet()) { + long count = counts.get(key); + for (long i = 0; i < count; i++) { + events.add(EventBuilder.withBody(key, Charsets.UTF_8)); + } + } + } + + private Channel createAndConfigureMemoryChannel(HBaseSink sink) { + Channel channel = new MemoryChannel(); + Context channelCtx = new Context(); + channelCtx.put("capacity", String.valueOf(1000L)); + channelCtx.put("transactionCapacity", String.valueOf(1000L)); + Configurables.configure(channel, channelCtx); + sink.setChannel(channel); + channel.start(); + return channel; + } + + private void putEvents(Channel channel, Iterable events) { + Transaction tx = channel.getTransaction(); + tx.begin(); + for (Event event : events) { + channel.put(event); + } + tx.commit(); + tx.close(); + } + +} From 34836ce6cef7c0f0ae13b2e8d9a63ca838ce8ace Mon Sep 17 00:00:00 2001 From: Hari Shreedharan Date: Mon, 3 Mar 2014 00:25:15 -0800 Subject: [PATCH 087/341] Revert "FLUME-2338. Support coalescing increments in HBaseSink." This reverts commit 674f4fcce2597e7e934ccc69eb04b426f5a9b8bb. --- flume-ng-doc/sphinx/FlumeUserGuide.rst | 2 - .../apache/flume/sink/hbase/BatchAware.java | 28 -- .../apache/flume/sink/hbase/HBaseSink.java | 195 +----------- .../sink/hbase/IncrementHBaseSerializer.java | 80 ----- .../flume/sink/hbase/TestHBaseSink.java | 298 +++--------------- 5 files changed, 53 insertions(+), 550 deletions(-) delete mode 100644 flume-ng-sinks/flume-ng-hbase-sink/src/main/java/org/apache/flume/sink/hbase/BatchAware.java delete mode 100644 flume-ng-sinks/flume-ng-hbase-sink/src/test/java/org/apache/flume/sink/hbase/IncrementHBaseSerializer.java diff --git a/flume-ng-doc/sphinx/FlumeUserGuide.rst b/flume-ng-doc/sphinx/FlumeUserGuide.rst index cedb2830b8..96bf73e0cc 100644 --- a/flume-ng-doc/sphinx/FlumeUserGuide.rst +++ b/flume-ng-doc/sphinx/FlumeUserGuide.rst @@ -1839,8 +1839,6 @@ Property Name Default Desc zookeeperQuorum -- The quorum spec. This is the value for the property ``hbase.zookeeper.quorum`` in hbase-site.xml znodeParent /hbase The base path for the znode for the -ROOT- region. Value of ``zookeeper.znode.parent`` in hbase-site.xml batchSize 100 Number of events to be written per txn. -coalesceIncrements false Should the sink coalesce multiple increments to a cell per batch. This might give - better performance if there are multiple increments to a limited number of cells. serializer org.apache.flume.sink.hbase.SimpleHbaseEventSerializer Default increment column = "iCol", payload column = "pCol". serializer.* -- Properties to be passed to the serializer. kerberosPrincipal -- Kerberos user principal for accessing secure HBase diff --git a/flume-ng-sinks/flume-ng-hbase-sink/src/main/java/org/apache/flume/sink/hbase/BatchAware.java b/flume-ng-sinks/flume-ng-hbase-sink/src/main/java/org/apache/flume/sink/hbase/BatchAware.java deleted file mode 100644 index 0974241db3..0000000000 --- a/flume-ng-sinks/flume-ng-hbase-sink/src/main/java/org/apache/flume/sink/hbase/BatchAware.java +++ /dev/null @@ -1,28 +0,0 @@ -/* - * Licensed to the Apache Software Foundation (ASF) under one - * or more contributor license agreements. See the NOTICE file - * distributed with this work for additional information - * regarding copyright ownership. The ASF licenses this file - * to you under the Apache License, Version 2.0 (the - * "License"); you may not use this file except in compliance - * with the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, - * software distributed under the License is distributed on an - * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY - * KIND, either express or implied. See the License for the - * specific language governing permissions and limitations - * under the License. - */ -package org.apache.flume.sink.hbase; - -/** - * This interface allows for implementing HBase serializers that are aware of - * batching. {@link #onBatchStart()} is called at the beginning of each batch - * by the sink. - */ -public interface BatchAware { - public void onBatchStart(); -} diff --git a/flume-ng-sinks/flume-ng-hbase-sink/src/main/java/org/apache/flume/sink/hbase/HBaseSink.java b/flume-ng-sinks/flume-ng-hbase-sink/src/main/java/org/apache/flume/sink/hbase/HBaseSink.java index 0390ff88ce..c4a666cacd 100644 --- a/flume-ng-sinks/flume-ng-hbase-sink/src/main/java/org/apache/flume/sink/hbase/HBaseSink.java +++ b/flume-ng-sinks/flume-ng-hbase-sink/src/main/java/org/apache/flume/sink/hbase/HBaseSink.java @@ -19,23 +19,15 @@ package org.apache.flume.sink.hbase; import java.io.IOException; -import java.lang.reflect.InvocationTargetException; -import java.lang.reflect.Method; import java.util.LinkedList; import java.util.List; -import java.util.Map; -import java.util.NavigableMap; -import com.google.common.annotations.VisibleForTesting; -import com.google.common.collect.Lists; -import com.google.common.collect.Maps; import org.apache.flume.Channel; import org.apache.flume.Context; import org.apache.flume.Event; import org.apache.flume.EventDeliveryException; import org.apache.flume.FlumeException; import org.apache.flume.Transaction; -import org.apache.flume.annotations.InterfaceAudience; import org.apache.flume.conf.Configurable; import org.apache.flume.instrumentation.SinkCounter; import org.apache.flume.sink.AbstractSink; @@ -60,7 +52,7 @@ /** * * A simple sink which reads events from a channel and writes them to HBase. - * The Hbase configuration is picked up from the first hbase-site.xml + * The Hbase configution is picked up from the first hbase-site.xml * encountered in the classpath. This sink supports batch reading of * events from the channel, and writing them to Hbase, to minimize the number * of flushes on the hbase tables. To use this sink, it has to be configured @@ -105,13 +97,8 @@ public class HBaseSink extends AbstractSink implements Configurable { private String kerberosKeytab; private User hbaseUser; private boolean enableWal = true; - private boolean batchIncrements = false; - private Method refGetFamilyMap; private SinkCounter sinkCounter; - // Internal hooks used for unit testing. - private DebugIncrementsCallback debugIncrCallback = null; - public HBaseSink(){ this(HBaseConfiguration.create()); } @@ -120,13 +107,6 @@ public HBaseSink(Configuration conf){ this.config = conf; } - @VisibleForTesting - @InterfaceAudience.Private - HBaseSink(Configuration conf, DebugIncrementsCallback cb) { - this(conf); - this.debugIncrCallback = cb; - } - @Override public void start(){ Preconditions.checkArgument(table == null, "Please call stop " + @@ -242,17 +222,6 @@ public void configure(Context context){ "writes to HBase will have WAL disabled, and any data in the " + "memstore of this region in the Region Server could be lost!"); } - - batchIncrements = context.getBoolean( - HBaseSinkConfigurationConstants.CONFIG_COALESCE_INCREMENTS, - HBaseSinkConfigurationConstants.DEFAULT_COALESCE_INCREMENTS); - - if (batchIncrements) { - logger.info("Increment coalescing is enabled. Increments will be " + - "buffered."); - reflectLookupGetFamilyMap(); - } - String zkQuorum = context.getString(HBaseSinkConfigurationConstants .ZK_QUORUM); Integer port = null; @@ -312,11 +281,6 @@ public Status process() throws EventDeliveryException { List incs = new LinkedList(); try { txn.begin(); - - if (serializer instanceof BatchAware) { - ((BatchAware)serializer).onBatchStart(); - } - long i = 0; for (; i < batchSize; i++) { Event event = channel.take(); @@ -345,7 +309,7 @@ public Status process() throws EventDeliveryException { try{ txn.rollback(); } catch (Exception e2) { - logger.error("Exception in rollback. Rollback might not have been " + + logger.error("Exception in rollback. Rollback might not have been" + "successful." , e2); } logger.error("Failed to commit transaction." + @@ -389,20 +353,7 @@ public Void run() throws Exception { runPrivileged(new PrivilegedExceptionAction() { @Override public Void run() throws Exception { - - List processedIncrements; - if (batchIncrements) { - processedIncrements = coalesceIncrements(incs); - } else { - processedIncrements = incs; - } - - // Only used for unit testing. - if (debugIncrCallback != null) { - debugIncrCallback.onAfterCoalesce(processedIncrements); - } - - for (final Increment i : processedIncrements) { + for (final Increment i : incs) { i.setWriteToWAL(enableWal); table.increment(i); } @@ -413,7 +364,6 @@ public Void run() throws Exception { txn.commit(); sinkCounter.addToEventDrainSuccessCount(actions.size()); } - private T runPrivileged(final PrivilegedExceptionAction action) throws Exception { if(hbaseUser != null) { @@ -425,143 +375,4 @@ private T runPrivileged(final PrivilegedExceptionAction action) return action.run(); } } - - /** - * The method getFamilyMap() is no longer available in Hbase 0.96. - * We must use reflection to determine which version we may use. - */ - private void reflectLookupGetFamilyMap() { - refGetFamilyMap = null; - String[] methodNames = { "getFamilyMap", "getFamilyMapOfLongs" }; - for (String methodName : methodNames) { - try { - refGetFamilyMap = Increment.class.getMethod(methodName); - if (refGetFamilyMap != null) { - logger.debug("Using Increment.{} for coalesce", methodName); - break; - } - } catch (NoSuchMethodException e) { - logger.debug("Increment.{} does not exist. Exception follows.", - methodName, e); - } catch (SecurityException e) { - logger.debug("No access to Increment.{}; Exception follows.", - methodName, e); - } - } - if (refGetFamilyMap == null) { - throw new UnsupportedOperationException( - "Cannot find Increment.getFamilyMap()"); - } - } - - @SuppressWarnings("unchecked") - private Map> getFamilyMap(Increment inc) { - Preconditions.checkNotNull(refGetFamilyMap, - "Increment.getFamilymap() not found"); - Preconditions.checkNotNull(inc, "Increment required"); - Map> familyMap = null; - try { - Object familyObj = refGetFamilyMap.invoke(inc); - familyMap = (Map>) familyObj; - } catch (IllegalAccessException e) { - logger.warn("Unexpected error calling getFamilyMap()", e); - Throwables.propagate(e); - } catch (InvocationTargetException e) { - logger.warn("Unexpected error calling getFamilyMap()", e); - Throwables.propagate(e); - } - return familyMap; - } - - /** - * Perform "compression" on the given set of increments so that Flume sends - * the minimum possible number of RPC operations to HBase per batch. - * @param incs Input: Increment objects to coalesce. - * @return List of new Increment objects after coalescing the unique counts. - */ - private List coalesceIncrements(Iterable incs) { - Preconditions.checkNotNull(incs, "List of Increments must not be null"); - // Aggregate all of the increment row/family/column counts. - // The nested map is keyed like this: {row, family, qualifier} => count. - Map>> counters = - Maps.newTreeMap(Bytes.BYTES_COMPARATOR); - for (Increment inc : incs) { - byte[] row = inc.getRow(); - Map> families = getFamilyMap(inc); - for (Map.Entry> familyEntry : families.entrySet()) { - byte[] family = familyEntry.getKey(); - NavigableMap qualifiers = familyEntry.getValue(); - for (Map.Entry qualifierEntry : qualifiers.entrySet()) { - byte[] qualifier = qualifierEntry.getKey(); - Long count = qualifierEntry.getValue(); - incrementCounter(counters, row, family, qualifier, count); - } - } - } - - // Reconstruct list of Increments per unique row/family/qualifier. - List coalesced = Lists.newLinkedList(); - for (Map.Entry>> rowEntry : counters.entrySet()) { - byte[] row = rowEntry.getKey(); - Map > families = rowEntry.getValue(); - Increment inc = new Increment(row); - for (Map.Entry> familyEntry : families.entrySet()) { - byte[] family = familyEntry.getKey(); - NavigableMap qualifiers = familyEntry.getValue(); - for (Map.Entry qualifierEntry : qualifiers.entrySet()) { - byte[] qualifier = qualifierEntry.getKey(); - long count = qualifierEntry.getValue(); - inc.addColumn(family, qualifier, count); - } - } - coalesced.add(inc); - } - - return coalesced; - } - - /** - * Helper function for {@link #coalesceIncrements} to increment a counter - * value in the passed data structure. - * @param counters Nested data structure containing the counters. - * @param row Row key to increment. - * @param family Column family to increment. - * @param qualifier Column qualifier to increment. - * @param count Amount to increment by. - */ - private void incrementCounter( - Map>> counters, - byte[] row, byte[] family, byte[] qualifier, Long count) { - - Map> families = counters.get(row); - if (families == null) { - families = Maps.newTreeMap(Bytes.BYTES_COMPARATOR); - counters.put(row, families); - } - - NavigableMap qualifiers = families.get(family); - if (qualifiers == null) { - qualifiers = Maps.newTreeMap(Bytes.BYTES_COMPARATOR); - families.put(family, qualifiers); - } - - Long existingValue = qualifiers.get(qualifier); - if (existingValue == null) { - qualifiers.put(qualifier, count); - } else { - qualifiers.put(qualifier, existingValue + count); - } - } - - @VisibleForTesting - @InterfaceAudience.Private - HbaseEventSerializer getSerializer() { - return serializer; - } - - @VisibleForTesting - @InterfaceAudience.Private - interface DebugIncrementsCallback { - public void onAfterCoalesce(Iterable increments); - } } diff --git a/flume-ng-sinks/flume-ng-hbase-sink/src/test/java/org/apache/flume/sink/hbase/IncrementHBaseSerializer.java b/flume-ng-sinks/flume-ng-hbase-sink/src/test/java/org/apache/flume/sink/hbase/IncrementHBaseSerializer.java deleted file mode 100644 index b4343eb5a2..0000000000 --- a/flume-ng-sinks/flume-ng-hbase-sink/src/test/java/org/apache/flume/sink/hbase/IncrementHBaseSerializer.java +++ /dev/null @@ -1,80 +0,0 @@ -/* - * Licensed to the Apache Software Foundation (ASF) under one - * or more contributor license agreements. See the NOTICE file - * distributed with this work for additional information - * regarding copyright ownership. The ASF licenses this file - * to you under the Apache License, Version 2.0 (the - * "License"); you may not use this file except in compliance - * with the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, - * software distributed under the License is distributed on an - * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY - * KIND, either express or implied. See the License for the - * specific language governing permissions and limitations - * under the License. - */ -package org.apache.flume.sink.hbase; - -import com.google.common.annotations.VisibleForTesting; -import com.google.common.base.Charsets; -import com.google.common.collect.Lists; -import java.util.Collections; -import org.apache.flume.Context; -import org.apache.flume.Event; -import org.apache.flume.conf.ComponentConfiguration; -import org.apache.hadoop.hbase.client.Increment; -import org.apache.hadoop.hbase.client.Row; - -import java.util.List; - -/** - * For Increment-related unit tests. - */ -class IncrementHBaseSerializer implements HbaseEventSerializer, BatchAware { - private Event event; - private byte[] family; - private int numBatchesStarted = 0; - - @Override public void configure(Context context) { } - @Override public void configure(ComponentConfiguration conf) { } - @Override public void close() { } - - @Override - public void initialize(Event event, byte[] columnFamily) { - this.event = event; - this.family = columnFamily; - } - - // This class only creates Increments. - @Override - public List getActions() { - return Collections.emptyList(); - } - - // Treat each Event as a String, i,e, "row:qualifier". - @Override - public List getIncrements() { - List increments = Lists.newArrayList(); - String body = new String(event.getBody(), Charsets.UTF_8); - String[] pieces = body.split(":"); - String row = pieces[0]; - String qualifier = pieces[1]; - Increment inc = new Increment(row.getBytes(Charsets.UTF_8)); - inc.addColumn(family, qualifier.getBytes(Charsets.UTF_8), 1L); - increments.add(inc); - return increments; - } - - @Override - public void onBatchStart() { - numBatchesStarted++; - } - - @VisibleForTesting - public int getNumBatchesStarted() { - return numBatchesStarted; - } -} diff --git a/flume-ng-sinks/flume-ng-hbase-sink/src/test/java/org/apache/flume/sink/hbase/TestHBaseSink.java b/flume-ng-sinks/flume-ng-hbase-sink/src/test/java/org/apache/flume/sink/hbase/TestHBaseSink.java index 5b047dcdcf..d1b0182593 100644 --- a/flume-ng-sinks/flume-ng-hbase-sink/src/test/java/org/apache/flume/sink/hbase/TestHBaseSink.java +++ b/flume-ng-sinks/flume-ng-hbase-sink/src/test/java/org/apache/flume/sink/hbase/TestHBaseSink.java @@ -22,13 +22,9 @@ import java.io.IOException; import java.util.Arrays; -import java.util.List; +import java.util.HashMap; import java.util.Map; -import java.util.NavigableMap; -import com.google.common.base.Charsets; -import com.google.common.collect.Lists; -import com.google.common.collect.Maps; -import com.google.common.primitives.Longs; + import org.apache.flume.Channel; import org.apache.flume.ChannelException; import org.apache.flume.Context; @@ -41,95 +37,64 @@ import org.apache.flume.conf.Configurables; import org.apache.flume.event.EventBuilder; import org.apache.hadoop.conf.Configuration; -import org.apache.hadoop.hbase.HBaseTestingUtility; -import org.apache.hadoop.hbase.HConstants; +import org.apache.hadoop.hbase.*; import org.apache.hadoop.hbase.client.HTable; -import org.apache.hadoop.hbase.client.Increment; import org.apache.hadoop.hbase.client.Result; import org.apache.hadoop.hbase.client.ResultScanner; import org.apache.hadoop.hbase.client.Scan; import org.apache.hadoop.hbase.util.Bytes; import org.apache.hadoop.hbase.zookeeper.ZKConfig; -import org.junit.After; import org.junit.AfterClass; -import org.junit.Before; import org.junit.BeforeClass; import org.junit.Ignore; import org.junit.Test; import org.junit.Assert; -import org.slf4j.Logger; -import org.slf4j.LoggerFactory; - -public class TestHBaseSink { - private static final Logger logger = - LoggerFactory.getLogger(TestHBaseSink.class); - private static final HBaseTestingUtility testUtility = new HBaseTestingUtility(); - private static final String tableName = "TestHbaseSink"; - private static final String columnFamily = "TestColumnFamily"; - private static final String inColumn = "iCol"; - private static final String plCol = "pCol"; - private static final String valBase = "testing hbase sink: jham"; +import com.google.common.primitives.Longs; - private Configuration conf; - private Context ctx; +public class TestHBaseSink { + private static HBaseTestingUtility testUtility = new HBaseTestingUtility(); + private static String tableName = "TestHbaseSink"; + private static String columnFamily = "TestColumnFamily"; + private static String inColumn = "iCol"; + private static String plCol = "pCol"; + private static Context ctx = new Context(); + private static String valBase = "testing hbase sink: jham"; + private static Configuration conf; @BeforeClass - public static void setUpOnce() throws Exception { + public static void setUp() throws Exception { testUtility.startMiniCluster(); + Map ctxMap = new HashMap(); + ctxMap.put("table", tableName); + ctxMap.put("columnFamily", columnFamily); + ctxMap.put("serializer", + "org.apache.flume.sink.hbase.SimpleHbaseEventSerializer"); + ctxMap.put("serializer.payloadColumn", plCol); + ctxMap.put("serializer.incrementColumn", inColumn); + ctx.putAll(ctxMap); + conf = new Configuration(testUtility.getConfiguration()); } @AfterClass - public static void tearDownOnce() throws Exception { + public static void tearDown() throws Exception { testUtility.shutdownMiniCluster(); } - /** - * Most common context setup for unit tests using - * {@link SimpleHbaseEventSerializer}. - */ - @Before - public void setUp() throws IOException { - conf = new Configuration(testUtility.getConfiguration()); - ctx = new Context(); - testUtility.createTable(tableName.getBytes(), columnFamily.getBytes()); - } - @After - public void tearDown() throws IOException { - testUtility.deleteTable(tableName.getBytes()); - } - - /** - * Set up {@link Context} for use with {@link SimpleHbaseEventSerializer}. - */ - private void initContextForSimpleHbaseEventSerializer() { - ctx = new Context(); - ctx.put("table", tableName); - ctx.put("columnFamily", columnFamily); - ctx.put("serializer", SimpleHbaseEventSerializer.class.getName()); - ctx.put("serializer.payloadColumn", plCol); - ctx.put("serializer.incrementColumn", inColumn); - } - - /** - * Set up {@link Context} for use with {@link IncrementHBaseSerializer}. - */ - private void initContextForIncrementHBaseSerializer() { - ctx = new Context(); - ctx.put("table", tableName); - ctx.put("columnFamily", columnFamily); - ctx.put("serializer", IncrementHBaseSerializer.class.getName()); - } @Test public void testOneEventWithDefaults() throws Exception { //Create a context without setting increment column and payload Column - ctx = new Context(); - ctx.put("table", tableName); - ctx.put("columnFamily", columnFamily); - ctx.put("serializer", SimpleHbaseEventSerializer.class.getName()); + Map ctxMap = new HashMap(); + ctxMap.put("table", tableName); + ctxMap.put("columnFamily", columnFamily); + ctxMap.put("serializer", + "org.apache.flume.sink.hbase.SimpleHbaseEventSerializer"); + Context tmpctx = new Context(); + tmpctx.putAll(ctxMap); + testUtility.createTable(tableName.getBytes(), columnFamily.getBytes()); HBaseSink sink = new HBaseSink(conf); Configurables.configure(sink, ctx); Channel channel = new MemoryChannel(); @@ -152,11 +117,12 @@ public void testOneEventWithDefaults() throws Exception { Assert.assertArrayEquals(e.getBody(), out); out = results[1]; Assert.assertArrayEquals(Longs.toByteArray(1), out); + testUtility.deleteTable(tableName.getBytes()); } @Test public void testOneEvent() throws Exception { - initContextForSimpleHbaseEventSerializer(); + testUtility.createTable(tableName.getBytes(), columnFamily.getBytes()); HBaseSink sink = new HBaseSink(conf); Configurables.configure(sink, ctx); Channel channel = new MemoryChannel(); @@ -179,11 +145,12 @@ public void testOneEvent() throws Exception { Assert.assertArrayEquals(e.getBody(), out); out = results[1]; Assert.assertArrayEquals(Longs.toByteArray(1), out); + testUtility.deleteTable(tableName.getBytes()); } @Test public void testThreeEvents() throws Exception { - initContextForSimpleHbaseEventSerializer(); + testUtility.createTable(tableName.getBytes(), columnFamily.getBytes()); ctx.put("batchSize", "3"); HBaseSink sink = new HBaseSink(conf); Configurables.configure(sink, ctx); @@ -216,11 +183,12 @@ public void testThreeEvents() throws Exception { Assert.assertEquals(3, found); out = results[3]; Assert.assertArrayEquals(Longs.toByteArray(3), out); + testUtility.deleteTable(tableName.getBytes()); } @Test public void testMultipleBatches() throws Exception { - initContextForSimpleHbaseEventSerializer(); + testUtility.createTable(tableName.getBytes(), columnFamily.getBytes()); ctx.put("batchSize", "2"); HBaseSink sink = new HBaseSink(conf); Configurables.configure(sink, ctx); @@ -259,17 +227,11 @@ public void testMultipleBatches() throws Exception { Assert.assertEquals(3, found); out = results[3]; Assert.assertArrayEquals(Longs.toByteArray(3), out); + testUtility.deleteTable(tableName.getBytes()); } @Test(expected = FlumeException.class) public void testMissingTable() throws Exception { - logger.info("Running testMissingTable()"); - initContextForSimpleHbaseEventSerializer(); - - // setUp() will create the table, so we delete it. - logger.info("Deleting table {}", tableName); - testUtility.deleteTable(tableName.getBytes()); - ctx.put("batchSize", "2"); HBaseSink sink = new HBaseSink(conf); Configurables.configure(sink, ctx); @@ -278,8 +240,7 @@ public void testMissingTable() throws Exception { Channel channel = new MemoryChannel(); Configurables.configure(channel, new Context()); sink.setChannel(channel); - - logger.info("Writing data into channel"); + sink.start(); Transaction tx = channel.getTransaction(); tx.begin(); for(int i = 0; i < 3; i++){ @@ -288,25 +249,7 @@ public void testMissingTable() throws Exception { } tx.commit(); tx.close(); - - logger.info("Starting sink and processing events"); - try { - logger.info("Calling sink.start()"); - sink.start(); // This method will throw. - - // We never get here, but we log in case the behavior changes. - logger.error("Unexpected error: Calling sink.process()"); - sink.process(); - logger.error("Unexpected error: Calling sink.stop()"); - sink.stop(); - } finally { - // Re-create the table so tearDown() doesn't throw. - testUtility.createTable(tableName.getBytes(), columnFamily.getBytes()); - } - - // FIXME: The test should never get here, the below code doesn't run. - Assert.fail(); - + sink.process(); HTable table = new HTable(conf, tableName); byte[][] results = getResults(table, 2); byte[] out; @@ -323,9 +266,9 @@ public void testMissingTable() throws Exception { out = results[2]; Assert.assertArrayEquals(Longs.toByteArray(2), out); sink.process(); + sink.stop(); } - // TODO: Move this test to a different class and run it stand-alone. /** * This test must run last - it shuts down the minicluster :D * @throws Exception @@ -337,8 +280,8 @@ public void testMissingTable() throws Exception { "and uncomment this annotation to run this test.") @Test(expected = EventDeliveryException.class) public void testHBaseFailure() throws Exception { - initContextForSimpleHbaseEventSerializer(); ctx.put("batchSize", "2"); + testUtility.createTable(tableName.getBytes(), columnFamily.getBytes()); HBaseSink sink = new HBaseSink(conf); Configurables.configure(sink, ctx); //Reset the context to a higher batchSize @@ -431,9 +374,8 @@ private byte[][] getResults(HTable table, int numEvents) throws IOException{ @Test public void testTransactionStateOnChannelException() throws Exception { - initContextForSimpleHbaseEventSerializer(); ctx.put("batchSize", "1"); - + testUtility.createTable(tableName.getBytes(), columnFamily.getBytes()); HBaseSink sink = new HBaseSink(conf); Configurables.configure(sink, ctx); // Reset the context to a higher batchSize @@ -463,15 +405,15 @@ public void testTransactionStateOnChannelException() throws Exception { Assert.assertArrayEquals(e.getBody(), out); out = results[1]; Assert.assertArrayEquals(Longs.toByteArray(1), out); + testUtility.deleteTable(tableName.getBytes()); } @Test public void testTransactionStateOnSerializationException() throws Exception { - initContextForSimpleHbaseEventSerializer(); ctx.put("batchSize", "1"); ctx.put(HBaseSinkConfigurationConstants.CONFIG_SERIALIZER, "org.apache.flume.sink.hbase.MockSimpleHbaseEventSerializer"); - + testUtility.createTable(tableName.getBytes(), columnFamily.getBytes()); HBaseSink sink = new HBaseSink(conf); Configurables.configure(sink, ctx); // Reset the context to a higher batchSize @@ -502,11 +444,11 @@ public void testTransactionStateOnSerializationException() throws Exception { Assert.assertArrayEquals(e.getBody(), out); out = results[1]; Assert.assertArrayEquals(Longs.toByteArray(1), out); + testUtility.deleteTable(tableName.getBytes()); } @Test public void testWithoutConfigurationObject() throws Exception{ - initContextForSimpleHbaseEventSerializer(); Context tmpContext = new Context(ctx.getParameters()); tmpContext.put("batchSize", "2"); tmpContext.put(HBaseSinkConfigurationConstants.ZK_QUORUM, @@ -515,7 +457,7 @@ public void testWithoutConfigurationObject() throws Exception{ tmpContext.put(HBaseSinkConfigurationConstants.ZK_ZNODE_PARENT, conf.get(HConstants.ZOOKEEPER_ZNODE_PARENT, HConstants.DEFAULT_ZOOKEEPER_ZNODE_PARENT)); - + testUtility.createTable(tableName.getBytes(), columnFamily.getBytes()); HBaseSink sink = new HBaseSink(); Configurables.configure(sink, tmpContext); Channel channel = new MemoryChannel(); @@ -550,11 +492,11 @@ public void testWithoutConfigurationObject() throws Exception{ Assert.assertEquals(3, found); out = results[3]; Assert.assertArrayEquals(Longs.toByteArray(3), out); + testUtility.deleteTable(tableName.getBytes()); } @Test public void testZKQuorum() throws Exception{ - initContextForSimpleHbaseEventSerializer(); Context tmpContext = new Context(ctx.getParameters()); String zkQuorum = "zk1.flume.apache.org:3342, zk2.flume.apache.org:3342, " + "zk3.flume.apache.org:3342"; @@ -574,7 +516,6 @@ public void testZKQuorum() throws Exception{ @Test (expected = FlumeException.class) public void testZKQuorumIncorrectPorts() throws Exception{ - initContextForSimpleHbaseEventSerializer(); Context tmpContext = new Context(ctx.getParameters()); String zkQuorum = "zk1.flume.apache.org:3345, zk2.flume.apache.org:3342, " + @@ -588,143 +529,4 @@ public void testZKQuorumIncorrectPorts() throws Exception{ Configurables.configure(sink, tmpContext); Assert.fail(); } - - @Test - public void testCoalesce() throws EventDeliveryException { - initContextForIncrementHBaseSerializer(); - ctx.put("batchSize", "100"); - ctx.put(HBaseSinkConfigurationConstants.CONFIG_COALESCE_INCREMENTS, - String.valueOf(true)); - - final Map expectedCounts = Maps.newHashMap(); - expectedCounts.put("r1:c1", 10L); - expectedCounts.put("r1:c2", 20L); - expectedCounts.put("r2:c1", 7L); - expectedCounts.put("r2:c3", 63L); - HBaseSink.DebugIncrementsCallback cb = new CoalesceValidator(expectedCounts); - - HBaseSink sink = new HBaseSink(testUtility.getConfiguration(), cb); - Configurables.configure(sink, ctx); - Channel channel = createAndConfigureMemoryChannel(sink); - - List events = Lists.newLinkedList(); - generateEvents(events, expectedCounts); - putEvents(channel, events); - - sink.start(); - sink.process(); // Calls CoalesceValidator instance. - sink.stop(); - } - - @Test(expected = AssertionError.class) - public void negativeTestCoalesce() throws EventDeliveryException { - initContextForIncrementHBaseSerializer(); - ctx.put("batchSize", "10"); - - final Map expectedCounts = Maps.newHashMap(); - expectedCounts.put("r1:c1", 10L); - HBaseSink.DebugIncrementsCallback cb = new CoalesceValidator(expectedCounts); - - HBaseSink sink = new HBaseSink(testUtility.getConfiguration(), cb); - Configurables.configure(sink, ctx); - Channel channel = createAndConfigureMemoryChannel(sink); - - List events = Lists.newLinkedList(); - generateEvents(events, expectedCounts); - putEvents(channel, events); - - sink.start(); - sink.process(); // Calls CoalesceValidator instance. - sink.stop(); - } - - @Test - public void testBatchAware() throws EventDeliveryException { - logger.info("Running testBatchAware()"); - initContextForIncrementHBaseSerializer(); - HBaseSink sink = new HBaseSink(testUtility.getConfiguration()); - Configurables.configure(sink, ctx); - Channel channel = createAndConfigureMemoryChannel(sink); - - sink.start(); - int batchCount = 3; - for (int i = 0; i < batchCount; i++) { - sink.process(); - } - sink.stop(); - Assert.assertEquals(batchCount, - ((IncrementHBaseSerializer) sink.getSerializer()).getNumBatchesStarted()); - } - - /** - * For testing that the rows coalesced, serialized by - * {@link IncrementHBaseSerializer}, are of the expected batch size. - */ - private static class CoalesceValidator - implements HBaseSink.DebugIncrementsCallback { - - private final Map expectedCounts; - - public CoalesceValidator(Map expectedCounts) { - this.expectedCounts = expectedCounts; - } - - @Override - public void onAfterCoalesce(Iterable increments) { - for (Increment inc : increments) { - byte[] row = inc.getRow(); - Map> families = inc.getFamilyMap(); - for (byte[] family : families.keySet()) { - NavigableMap qualifiers = families.get(family); - for (Map.Entry entry : qualifiers.entrySet()) { - byte[] qualifier = entry.getKey(); - Long count = entry.getValue(); - StringBuilder b = new StringBuilder(20); - b.append(new String(row, Charsets.UTF_8)); - b.append(':'); - b.append(new String(qualifier, Charsets.UTF_8)); - String key = b.toString(); - Assert.assertEquals("Expected counts don't match observed for " + key, - expectedCounts.get(key), count); - } - } - } - } - } - - /** - * Add number of Events corresponding to counts to the events list. - * @param events Destination list. - * @param counts How many events to generate for each row:qualifier pair. - */ - private void generateEvents(List events, Map counts) { - for (String key : counts.keySet()) { - long count = counts.get(key); - for (long i = 0; i < count; i++) { - events.add(EventBuilder.withBody(key, Charsets.UTF_8)); - } - } - } - - private Channel createAndConfigureMemoryChannel(HBaseSink sink) { - Channel channel = new MemoryChannel(); - Context channelCtx = new Context(); - channelCtx.put("capacity", String.valueOf(1000L)); - channelCtx.put("transactionCapacity", String.valueOf(1000L)); - Configurables.configure(channel, channelCtx); - sink.setChannel(channel); - channel.start(); - return channel; - } - - private void putEvents(Channel channel, Iterable events) { - Transaction tx = channel.getTransaction(); - tx.begin(); - for (Event event : events) { - channel.put(event); - } - tx.commit(); - tx.close(); - } - -} +} \ No newline at end of file From 1dfcb4b0edd4e2fca43d37bc01896015c245e589 Mon Sep 17 00:00:00 2001 From: Hari Shreedharan Date: Sun, 2 Mar 2014 23:49:17 -0800 Subject: [PATCH 088/341] FLUME-2338. Support coalescing increments in HBaseSink. (Mike Percy via Hari Shreedharan) --- flume-ng-doc/sphinx/FlumeUserGuide.rst | 2 + .../apache/flume/sink/hbase/BatchAware.java | 28 ++ .../apache/flume/sink/hbase/HBaseSink.java | 197 ++++++++++- .../sink/hbase/IncrementHBaseSerializer.java | 80 +++++ .../flume/sink/hbase/TestHBaseSink.java | 309 +++++++++++++++--- 5 files changed, 563 insertions(+), 53 deletions(-) create mode 100644 flume-ng-sinks/flume-ng-hbase-sink/src/main/java/org/apache/flume/sink/hbase/BatchAware.java create mode 100644 flume-ng-sinks/flume-ng-hbase-sink/src/test/java/org/apache/flume/sink/hbase/IncrementHBaseSerializer.java diff --git a/flume-ng-doc/sphinx/FlumeUserGuide.rst b/flume-ng-doc/sphinx/FlumeUserGuide.rst index 96bf73e0cc..cedb2830b8 100644 --- a/flume-ng-doc/sphinx/FlumeUserGuide.rst +++ b/flume-ng-doc/sphinx/FlumeUserGuide.rst @@ -1839,6 +1839,8 @@ Property Name Default Desc zookeeperQuorum -- The quorum spec. This is the value for the property ``hbase.zookeeper.quorum`` in hbase-site.xml znodeParent /hbase The base path for the znode for the -ROOT- region. Value of ``zookeeper.znode.parent`` in hbase-site.xml batchSize 100 Number of events to be written per txn. +coalesceIncrements false Should the sink coalesce multiple increments to a cell per batch. This might give + better performance if there are multiple increments to a limited number of cells. serializer org.apache.flume.sink.hbase.SimpleHbaseEventSerializer Default increment column = "iCol", payload column = "pCol". serializer.* -- Properties to be passed to the serializer. kerberosPrincipal -- Kerberos user principal for accessing secure HBase diff --git a/flume-ng-sinks/flume-ng-hbase-sink/src/main/java/org/apache/flume/sink/hbase/BatchAware.java b/flume-ng-sinks/flume-ng-hbase-sink/src/main/java/org/apache/flume/sink/hbase/BatchAware.java new file mode 100644 index 0000000000..0974241db3 --- /dev/null +++ b/flume-ng-sinks/flume-ng-hbase-sink/src/main/java/org/apache/flume/sink/hbase/BatchAware.java @@ -0,0 +1,28 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, + * software distributed under the License is distributed on an + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + * KIND, either express or implied. See the License for the + * specific language governing permissions and limitations + * under the License. + */ +package org.apache.flume.sink.hbase; + +/** + * This interface allows for implementing HBase serializers that are aware of + * batching. {@link #onBatchStart()} is called at the beginning of each batch + * by the sink. + */ +public interface BatchAware { + public void onBatchStart(); +} diff --git a/flume-ng-sinks/flume-ng-hbase-sink/src/main/java/org/apache/flume/sink/hbase/HBaseSink.java b/flume-ng-sinks/flume-ng-hbase-sink/src/main/java/org/apache/flume/sink/hbase/HBaseSink.java index c4a666cacd..9996a4e76e 100644 --- a/flume-ng-sinks/flume-ng-hbase-sink/src/main/java/org/apache/flume/sink/hbase/HBaseSink.java +++ b/flume-ng-sinks/flume-ng-hbase-sink/src/main/java/org/apache/flume/sink/hbase/HBaseSink.java @@ -19,15 +19,23 @@ package org.apache.flume.sink.hbase; import java.io.IOException; +import java.lang.reflect.InvocationTargetException; +import java.lang.reflect.Method; import java.util.LinkedList; import java.util.List; +import java.util.Map; +import java.util.NavigableMap; +import com.google.common.annotations.VisibleForTesting; +import com.google.common.collect.Lists; +import com.google.common.collect.Maps; import org.apache.flume.Channel; import org.apache.flume.Context; import org.apache.flume.Event; import org.apache.flume.EventDeliveryException; import org.apache.flume.FlumeException; import org.apache.flume.Transaction; +import org.apache.flume.annotations.InterfaceAudience; import org.apache.flume.conf.Configurable; import org.apache.flume.instrumentation.SinkCounter; import org.apache.flume.sink.AbstractSink; @@ -52,7 +60,7 @@ /** * * A simple sink which reads events from a channel and writes them to HBase. - * The Hbase configution is picked up from the first hbase-site.xml + * The Hbase configuration is picked up from the first hbase-site.xml * encountered in the classpath. This sink supports batch reading of * events from the channel, and writing them to Hbase, to minimize the number * of flushes on the hbase tables. To use this sink, it has to be configured @@ -97,8 +105,13 @@ public class HBaseSink extends AbstractSink implements Configurable { private String kerberosKeytab; private User hbaseUser; private boolean enableWal = true; + private boolean batchIncrements = false; + private Method refGetFamilyMap = null; private SinkCounter sinkCounter; + // Internal hooks used for unit testing. + private DebugIncrementsCallback debugIncrCallback = null; + public HBaseSink(){ this(HBaseConfiguration.create()); } @@ -107,6 +120,13 @@ public HBaseSink(Configuration conf){ this.config = conf; } + @VisibleForTesting + @InterfaceAudience.Private + HBaseSink(Configuration conf, DebugIncrementsCallback cb) { + this(conf); + this.debugIncrCallback = cb; + } + @Override public void start(){ Preconditions.checkArgument(table == null, "Please call stop " + @@ -222,6 +242,17 @@ public void configure(Context context){ "writes to HBase will have WAL disabled, and any data in the " + "memstore of this region in the Region Server could be lost!"); } + + batchIncrements = context.getBoolean( + HBaseSinkConfigurationConstants.CONFIG_COALESCE_INCREMENTS, + HBaseSinkConfigurationConstants.DEFAULT_COALESCE_INCREMENTS); + + if (batchIncrements) { + logger.info("Increment coalescing is enabled. Increments will be " + + "buffered."); + refGetFamilyMap = reflectLookupGetFamilyMap(); + } + String zkQuorum = context.getString(HBaseSinkConfigurationConstants .ZK_QUORUM); Integer port = null; @@ -281,6 +312,11 @@ public Status process() throws EventDeliveryException { List incs = new LinkedList(); try { txn.begin(); + + if (serializer instanceof BatchAware) { + ((BatchAware)serializer).onBatchStart(); + } + long i = 0; for (; i < batchSize; i++) { Event event = channel.take(); @@ -309,7 +345,7 @@ public Status process() throws EventDeliveryException { try{ txn.rollback(); } catch (Exception e2) { - logger.error("Exception in rollback. Rollback might not have been" + + logger.error("Exception in rollback. Rollback might not have been " + "successful." , e2); } logger.error("Failed to commit transaction." + @@ -353,7 +389,20 @@ public Void run() throws Exception { runPrivileged(new PrivilegedExceptionAction() { @Override public Void run() throws Exception { - for (final Increment i : incs) { + + List processedIncrements; + if (batchIncrements) { + processedIncrements = coalesceIncrements(incs); + } else { + processedIncrements = incs; + } + + // Only used for unit testing. + if (debugIncrCallback != null) { + debugIncrCallback.onAfterCoalesce(processedIncrements); + } + + for (final Increment i : processedIncrements) { i.setWriteToWAL(enableWal); table.increment(i); } @@ -364,6 +413,7 @@ public Void run() throws Exception { txn.commit(); sinkCounter.addToEventDrainSuccessCount(actions.size()); } + private T runPrivileged(final PrivilegedExceptionAction action) throws Exception { if(hbaseUser != null) { @@ -375,4 +425,145 @@ private T runPrivileged(final PrivilegedExceptionAction action) return action.run(); } } + + /** + * The method getFamilyMap() is no longer available in Hbase 0.96. + * We must use reflection to determine which version we may use. + */ + @VisibleForTesting + static Method reflectLookupGetFamilyMap() { + Method m = null; + String[] methodNames = { "getFamilyMapOfLongs", "getFamilyMap" }; + for (String methodName : methodNames) { + try { + m = Increment.class.getMethod(methodName); + if (m != null && m.getReturnType().equals(Map.class)) { + logger.debug("Using Increment.{} for coalesce", methodName); + break; + } + } catch (NoSuchMethodException e) { + logger.debug("Increment.{} does not exist. Exception follows.", + methodName, e); + } catch (SecurityException e) { + logger.debug("No access to Increment.{}; Exception follows.", + methodName, e); + } + } + if (m == null) { + throw new UnsupportedOperationException( + "Cannot find Increment.getFamilyMap()"); + } + return m; + } + + @SuppressWarnings("unchecked") + private Map> getFamilyMap(Increment inc) { + Preconditions.checkNotNull(refGetFamilyMap, + "Increment.getFamilymap() not found"); + Preconditions.checkNotNull(inc, "Increment required"); + Map> familyMap = null; + try { + Object familyObj = refGetFamilyMap.invoke(inc); + familyMap = (Map>) familyObj; + } catch (IllegalAccessException e) { + logger.warn("Unexpected error calling getFamilyMap()", e); + Throwables.propagate(e); + } catch (InvocationTargetException e) { + logger.warn("Unexpected error calling getFamilyMap()", e); + Throwables.propagate(e); + } + return familyMap; + } + + /** + * Perform "compression" on the given set of increments so that Flume sends + * the minimum possible number of RPC operations to HBase per batch. + * @param incs Input: Increment objects to coalesce. + * @return List of new Increment objects after coalescing the unique counts. + */ + private List coalesceIncrements(Iterable incs) { + Preconditions.checkNotNull(incs, "List of Increments must not be null"); + // Aggregate all of the increment row/family/column counts. + // The nested map is keyed like this: {row, family, qualifier} => count. + Map>> counters = + Maps.newTreeMap(Bytes.BYTES_COMPARATOR); + for (Increment inc : incs) { + byte[] row = inc.getRow(); + Map> families = getFamilyMap(inc); + for (Map.Entry> familyEntry : families.entrySet()) { + byte[] family = familyEntry.getKey(); + NavigableMap qualifiers = familyEntry.getValue(); + for (Map.Entry qualifierEntry : qualifiers.entrySet()) { + byte[] qualifier = qualifierEntry.getKey(); + Long count = qualifierEntry.getValue(); + incrementCounter(counters, row, family, qualifier, count); + } + } + } + + // Reconstruct list of Increments per unique row/family/qualifier. + List coalesced = Lists.newLinkedList(); + for (Map.Entry>> rowEntry : counters.entrySet()) { + byte[] row = rowEntry.getKey(); + Map > families = rowEntry.getValue(); + Increment inc = new Increment(row); + for (Map.Entry> familyEntry : families.entrySet()) { + byte[] family = familyEntry.getKey(); + NavigableMap qualifiers = familyEntry.getValue(); + for (Map.Entry qualifierEntry : qualifiers.entrySet()) { + byte[] qualifier = qualifierEntry.getKey(); + long count = qualifierEntry.getValue(); + inc.addColumn(family, qualifier, count); + } + } + coalesced.add(inc); + } + + return coalesced; + } + + /** + * Helper function for {@link #coalesceIncrements} to increment a counter + * value in the passed data structure. + * @param counters Nested data structure containing the counters. + * @param row Row key to increment. + * @param family Column family to increment. + * @param qualifier Column qualifier to increment. + * @param count Amount to increment by. + */ + private void incrementCounter( + Map>> counters, + byte[] row, byte[] family, byte[] qualifier, Long count) { + + Map> families = counters.get(row); + if (families == null) { + families = Maps.newTreeMap(Bytes.BYTES_COMPARATOR); + counters.put(row, families); + } + + NavigableMap qualifiers = families.get(family); + if (qualifiers == null) { + qualifiers = Maps.newTreeMap(Bytes.BYTES_COMPARATOR); + families.put(family, qualifiers); + } + + Long existingValue = qualifiers.get(qualifier); + if (existingValue == null) { + qualifiers.put(qualifier, count); + } else { + qualifiers.put(qualifier, existingValue + count); + } + } + + @VisibleForTesting + @InterfaceAudience.Private + HbaseEventSerializer getSerializer() { + return serializer; + } + + @VisibleForTesting + @InterfaceAudience.Private + interface DebugIncrementsCallback { + public void onAfterCoalesce(Iterable increments); + } } diff --git a/flume-ng-sinks/flume-ng-hbase-sink/src/test/java/org/apache/flume/sink/hbase/IncrementHBaseSerializer.java b/flume-ng-sinks/flume-ng-hbase-sink/src/test/java/org/apache/flume/sink/hbase/IncrementHBaseSerializer.java new file mode 100644 index 0000000000..b4343eb5a2 --- /dev/null +++ b/flume-ng-sinks/flume-ng-hbase-sink/src/test/java/org/apache/flume/sink/hbase/IncrementHBaseSerializer.java @@ -0,0 +1,80 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, + * software distributed under the License is distributed on an + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + * KIND, either express or implied. See the License for the + * specific language governing permissions and limitations + * under the License. + */ +package org.apache.flume.sink.hbase; + +import com.google.common.annotations.VisibleForTesting; +import com.google.common.base.Charsets; +import com.google.common.collect.Lists; +import java.util.Collections; +import org.apache.flume.Context; +import org.apache.flume.Event; +import org.apache.flume.conf.ComponentConfiguration; +import org.apache.hadoop.hbase.client.Increment; +import org.apache.hadoop.hbase.client.Row; + +import java.util.List; + +/** + * For Increment-related unit tests. + */ +class IncrementHBaseSerializer implements HbaseEventSerializer, BatchAware { + private Event event; + private byte[] family; + private int numBatchesStarted = 0; + + @Override public void configure(Context context) { } + @Override public void configure(ComponentConfiguration conf) { } + @Override public void close() { } + + @Override + public void initialize(Event event, byte[] columnFamily) { + this.event = event; + this.family = columnFamily; + } + + // This class only creates Increments. + @Override + public List getActions() { + return Collections.emptyList(); + } + + // Treat each Event as a String, i,e, "row:qualifier". + @Override + public List getIncrements() { + List increments = Lists.newArrayList(); + String body = new String(event.getBody(), Charsets.UTF_8); + String[] pieces = body.split(":"); + String row = pieces[0]; + String qualifier = pieces[1]; + Increment inc = new Increment(row.getBytes(Charsets.UTF_8)); + inc.addColumn(family, qualifier.getBytes(Charsets.UTF_8), 1L); + increments.add(inc); + return increments; + } + + @Override + public void onBatchStart() { + numBatchesStarted++; + } + + @VisibleForTesting + public int getNumBatchesStarted() { + return numBatchesStarted; + } +} diff --git a/flume-ng-sinks/flume-ng-hbase-sink/src/test/java/org/apache/flume/sink/hbase/TestHBaseSink.java b/flume-ng-sinks/flume-ng-hbase-sink/src/test/java/org/apache/flume/sink/hbase/TestHBaseSink.java index d1b0182593..ab65a3898b 100644 --- a/flume-ng-sinks/flume-ng-hbase-sink/src/test/java/org/apache/flume/sink/hbase/TestHBaseSink.java +++ b/flume-ng-sinks/flume-ng-hbase-sink/src/test/java/org/apache/flume/sink/hbase/TestHBaseSink.java @@ -21,10 +21,16 @@ import static org.mockito.Mockito.*; import java.io.IOException; +import java.lang.reflect.Method; import java.util.Arrays; -import java.util.HashMap; +import java.util.List; import java.util.Map; - +import java.util.NavigableMap; +import com.google.common.base.Charsets; +import com.google.common.base.Throwables; +import com.google.common.collect.Lists; +import com.google.common.collect.Maps; +import com.google.common.primitives.Longs; import org.apache.flume.Channel; import org.apache.flume.ChannelException; import org.apache.flume.Context; @@ -37,64 +43,95 @@ import org.apache.flume.conf.Configurables; import org.apache.flume.event.EventBuilder; import org.apache.hadoop.conf.Configuration; -import org.apache.hadoop.hbase.*; +import org.apache.hadoop.hbase.HBaseTestingUtility; +import org.apache.hadoop.hbase.HConstants; import org.apache.hadoop.hbase.client.HTable; +import org.apache.hadoop.hbase.client.Increment; import org.apache.hadoop.hbase.client.Result; import org.apache.hadoop.hbase.client.ResultScanner; import org.apache.hadoop.hbase.client.Scan; import org.apache.hadoop.hbase.util.Bytes; import org.apache.hadoop.hbase.zookeeper.ZKConfig; +import org.junit.After; import org.junit.AfterClass; +import org.junit.Before; import org.junit.BeforeClass; import org.junit.Ignore; import org.junit.Test; import org.junit.Assert; - -import com.google.common.primitives.Longs; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; public class TestHBaseSink { - private static HBaseTestingUtility testUtility = new HBaseTestingUtility(); - private static String tableName = "TestHbaseSink"; - private static String columnFamily = "TestColumnFamily"; - private static String inColumn = "iCol"; - private static String plCol = "pCol"; - private static Context ctx = new Context(); - private static String valBase = "testing hbase sink: jham"; - private static Configuration conf; + private static final Logger logger = + LoggerFactory.getLogger(TestHBaseSink.class); + + private static final HBaseTestingUtility testUtility = new HBaseTestingUtility(); + private static final String tableName = "TestHbaseSink"; + private static final String columnFamily = "TestColumnFamily"; + private static final String inColumn = "iCol"; + private static final String plCol = "pCol"; + private static final String valBase = "testing hbase sink: jham"; + + private Configuration conf; + private Context ctx; @BeforeClass - public static void setUp() throws Exception { + public static void setUpOnce() throws Exception { testUtility.startMiniCluster(); - Map ctxMap = new HashMap(); - ctxMap.put("table", tableName); - ctxMap.put("columnFamily", columnFamily); - ctxMap.put("serializer", - "org.apache.flume.sink.hbase.SimpleHbaseEventSerializer"); - ctxMap.put("serializer.payloadColumn", plCol); - ctxMap.put("serializer.incrementColumn", inColumn); - ctx.putAll(ctxMap); - conf = new Configuration(testUtility.getConfiguration()); } @AfterClass - public static void tearDown() throws Exception { + public static void tearDownOnce() throws Exception { testUtility.shutdownMiniCluster(); } + /** + * Most common context setup for unit tests using + * {@link SimpleHbaseEventSerializer}. + */ + @Before + public void setUp() throws IOException { + conf = new Configuration(testUtility.getConfiguration()); + ctx = new Context(); + testUtility.createTable(tableName.getBytes(), columnFamily.getBytes()); + } + @After + public void tearDown() throws IOException { + testUtility.deleteTable(tableName.getBytes()); + } + + /** + * Set up {@link Context} for use with {@link SimpleHbaseEventSerializer}. + */ + private void initContextForSimpleHbaseEventSerializer() { + ctx = new Context(); + ctx.put("table", tableName); + ctx.put("columnFamily", columnFamily); + ctx.put("serializer", SimpleHbaseEventSerializer.class.getName()); + ctx.put("serializer.payloadColumn", plCol); + ctx.put("serializer.incrementColumn", inColumn); + } + + /** + * Set up {@link Context} for use with {@link IncrementHBaseSerializer}. + */ + private void initContextForIncrementHBaseSerializer() { + ctx = new Context(); + ctx.put("table", tableName); + ctx.put("columnFamily", columnFamily); + ctx.put("serializer", IncrementHBaseSerializer.class.getName()); + } @Test public void testOneEventWithDefaults() throws Exception { //Create a context without setting increment column and payload Column - Map ctxMap = new HashMap(); - ctxMap.put("table", tableName); - ctxMap.put("columnFamily", columnFamily); - ctxMap.put("serializer", - "org.apache.flume.sink.hbase.SimpleHbaseEventSerializer"); - Context tmpctx = new Context(); - tmpctx.putAll(ctxMap); + ctx = new Context(); + ctx.put("table", tableName); + ctx.put("columnFamily", columnFamily); + ctx.put("serializer", SimpleHbaseEventSerializer.class.getName()); - testUtility.createTable(tableName.getBytes(), columnFamily.getBytes()); HBaseSink sink = new HBaseSink(conf); Configurables.configure(sink, ctx); Channel channel = new MemoryChannel(); @@ -117,12 +154,11 @@ public void testOneEventWithDefaults() throws Exception { Assert.assertArrayEquals(e.getBody(), out); out = results[1]; Assert.assertArrayEquals(Longs.toByteArray(1), out); - testUtility.deleteTable(tableName.getBytes()); } @Test public void testOneEvent() throws Exception { - testUtility.createTable(tableName.getBytes(), columnFamily.getBytes()); + initContextForSimpleHbaseEventSerializer(); HBaseSink sink = new HBaseSink(conf); Configurables.configure(sink, ctx); Channel channel = new MemoryChannel(); @@ -145,12 +181,11 @@ public void testOneEvent() throws Exception { Assert.assertArrayEquals(e.getBody(), out); out = results[1]; Assert.assertArrayEquals(Longs.toByteArray(1), out); - testUtility.deleteTable(tableName.getBytes()); } @Test public void testThreeEvents() throws Exception { - testUtility.createTable(tableName.getBytes(), columnFamily.getBytes()); + initContextForSimpleHbaseEventSerializer(); ctx.put("batchSize", "3"); HBaseSink sink = new HBaseSink(conf); Configurables.configure(sink, ctx); @@ -183,12 +218,11 @@ public void testThreeEvents() throws Exception { Assert.assertEquals(3, found); out = results[3]; Assert.assertArrayEquals(Longs.toByteArray(3), out); - testUtility.deleteTable(tableName.getBytes()); } @Test public void testMultipleBatches() throws Exception { - testUtility.createTable(tableName.getBytes(), columnFamily.getBytes()); + initContextForSimpleHbaseEventSerializer(); ctx.put("batchSize", "2"); HBaseSink sink = new HBaseSink(conf); Configurables.configure(sink, ctx); @@ -227,11 +261,17 @@ public void testMultipleBatches() throws Exception { Assert.assertEquals(3, found); out = results[3]; Assert.assertArrayEquals(Longs.toByteArray(3), out); - testUtility.deleteTable(tableName.getBytes()); } @Test(expected = FlumeException.class) public void testMissingTable() throws Exception { + logger.info("Running testMissingTable()"); + initContextForSimpleHbaseEventSerializer(); + + // setUp() will create the table, so we delete it. + logger.info("Deleting table {}", tableName); + testUtility.deleteTable(tableName.getBytes()); + ctx.put("batchSize", "2"); HBaseSink sink = new HBaseSink(conf); Configurables.configure(sink, ctx); @@ -240,7 +280,8 @@ public void testMissingTable() throws Exception { Channel channel = new MemoryChannel(); Configurables.configure(channel, new Context()); sink.setChannel(channel); - sink.start(); + + logger.info("Writing data into channel"); Transaction tx = channel.getTransaction(); tx.begin(); for(int i = 0; i < 3; i++){ @@ -249,7 +290,25 @@ public void testMissingTable() throws Exception { } tx.commit(); tx.close(); - sink.process(); + + logger.info("Starting sink and processing events"); + try { + logger.info("Calling sink.start()"); + sink.start(); // This method will throw. + + // We never get here, but we log in case the behavior changes. + logger.error("Unexpected error: Calling sink.process()"); + sink.process(); + logger.error("Unexpected error: Calling sink.stop()"); + sink.stop(); + } finally { + // Re-create the table so tearDown() doesn't throw. + testUtility.createTable(tableName.getBytes(), columnFamily.getBytes()); + } + + // FIXME: The test should never get here, the below code doesn't run. + Assert.fail(); + HTable table = new HTable(conf, tableName); byte[][] results = getResults(table, 2); byte[] out; @@ -266,9 +325,9 @@ public void testMissingTable() throws Exception { out = results[2]; Assert.assertArrayEquals(Longs.toByteArray(2), out); sink.process(); - sink.stop(); } + // TODO: Move this test to a different class and run it stand-alone. /** * This test must run last - it shuts down the minicluster :D * @throws Exception @@ -280,8 +339,8 @@ public void testMissingTable() throws Exception { "and uncomment this annotation to run this test.") @Test(expected = EventDeliveryException.class) public void testHBaseFailure() throws Exception { + initContextForSimpleHbaseEventSerializer(); ctx.put("batchSize", "2"); - testUtility.createTable(tableName.getBytes(), columnFamily.getBytes()); HBaseSink sink = new HBaseSink(conf); Configurables.configure(sink, ctx); //Reset the context to a higher batchSize @@ -374,8 +433,9 @@ private byte[][] getResults(HTable table, int numEvents) throws IOException{ @Test public void testTransactionStateOnChannelException() throws Exception { + initContextForSimpleHbaseEventSerializer(); ctx.put("batchSize", "1"); - testUtility.createTable(tableName.getBytes(), columnFamily.getBytes()); + HBaseSink sink = new HBaseSink(conf); Configurables.configure(sink, ctx); // Reset the context to a higher batchSize @@ -405,15 +465,15 @@ public void testTransactionStateOnChannelException() throws Exception { Assert.assertArrayEquals(e.getBody(), out); out = results[1]; Assert.assertArrayEquals(Longs.toByteArray(1), out); - testUtility.deleteTable(tableName.getBytes()); } @Test public void testTransactionStateOnSerializationException() throws Exception { + initContextForSimpleHbaseEventSerializer(); ctx.put("batchSize", "1"); ctx.put(HBaseSinkConfigurationConstants.CONFIG_SERIALIZER, "org.apache.flume.sink.hbase.MockSimpleHbaseEventSerializer"); - testUtility.createTable(tableName.getBytes(), columnFamily.getBytes()); + HBaseSink sink = new HBaseSink(conf); Configurables.configure(sink, ctx); // Reset the context to a higher batchSize @@ -444,11 +504,11 @@ public void testTransactionStateOnSerializationException() throws Exception { Assert.assertArrayEquals(e.getBody(), out); out = results[1]; Assert.assertArrayEquals(Longs.toByteArray(1), out); - testUtility.deleteTable(tableName.getBytes()); } @Test public void testWithoutConfigurationObject() throws Exception{ + initContextForSimpleHbaseEventSerializer(); Context tmpContext = new Context(ctx.getParameters()); tmpContext.put("batchSize", "2"); tmpContext.put(HBaseSinkConfigurationConstants.ZK_QUORUM, @@ -457,7 +517,7 @@ public void testWithoutConfigurationObject() throws Exception{ tmpContext.put(HBaseSinkConfigurationConstants.ZK_ZNODE_PARENT, conf.get(HConstants.ZOOKEEPER_ZNODE_PARENT, HConstants.DEFAULT_ZOOKEEPER_ZNODE_PARENT)); - testUtility.createTable(tableName.getBytes(), columnFamily.getBytes()); + HBaseSink sink = new HBaseSink(); Configurables.configure(sink, tmpContext); Channel channel = new MemoryChannel(); @@ -492,11 +552,11 @@ public void testWithoutConfigurationObject() throws Exception{ Assert.assertEquals(3, found); out = results[3]; Assert.assertArrayEquals(Longs.toByteArray(3), out); - testUtility.deleteTable(tableName.getBytes()); } @Test public void testZKQuorum() throws Exception{ + initContextForSimpleHbaseEventSerializer(); Context tmpContext = new Context(ctx.getParameters()); String zkQuorum = "zk1.flume.apache.org:3342, zk2.flume.apache.org:3342, " + "zk3.flume.apache.org:3342"; @@ -516,6 +576,7 @@ public void testZKQuorum() throws Exception{ @Test (expected = FlumeException.class) public void testZKQuorumIncorrectPorts() throws Exception{ + initContextForSimpleHbaseEventSerializer(); Context tmpContext = new Context(ctx.getParameters()); String zkQuorum = "zk1.flume.apache.org:3345, zk2.flume.apache.org:3342, " + @@ -529,4 +590,152 @@ public void testZKQuorumIncorrectPorts() throws Exception{ Configurables.configure(sink, tmpContext); Assert.fail(); } -} \ No newline at end of file + + @Test + public void testCoalesce() throws EventDeliveryException { + initContextForIncrementHBaseSerializer(); + ctx.put("batchSize", "100"); + ctx.put(HBaseSinkConfigurationConstants.CONFIG_COALESCE_INCREMENTS, + String.valueOf(true)); + + final Map expectedCounts = Maps.newHashMap(); + expectedCounts.put("r1:c1", 10L); + expectedCounts.put("r1:c2", 20L); + expectedCounts.put("r2:c1", 7L); + expectedCounts.put("r2:c3", 63L); + HBaseSink.DebugIncrementsCallback cb = new CoalesceValidator(expectedCounts); + + HBaseSink sink = new HBaseSink(testUtility.getConfiguration(), cb); + Configurables.configure(sink, ctx); + Channel channel = createAndConfigureMemoryChannel(sink); + + List events = Lists.newLinkedList(); + generateEvents(events, expectedCounts); + putEvents(channel, events); + + sink.start(); + sink.process(); // Calls CoalesceValidator instance. + sink.stop(); + } + + @Test(expected = AssertionError.class) + public void negativeTestCoalesce() throws EventDeliveryException { + initContextForIncrementHBaseSerializer(); + ctx.put("batchSize", "10"); + + final Map expectedCounts = Maps.newHashMap(); + expectedCounts.put("r1:c1", 10L); + HBaseSink.DebugIncrementsCallback cb = new CoalesceValidator(expectedCounts); + + HBaseSink sink = new HBaseSink(testUtility.getConfiguration(), cb); + Configurables.configure(sink, ctx); + Channel channel = createAndConfigureMemoryChannel(sink); + + List events = Lists.newLinkedList(); + generateEvents(events, expectedCounts); + putEvents(channel, events); + + sink.start(); + sink.process(); // Calls CoalesceValidator instance. + sink.stop(); + } + + @Test + public void testBatchAware() throws EventDeliveryException { + logger.info("Running testBatchAware()"); + initContextForIncrementHBaseSerializer(); + HBaseSink sink = new HBaseSink(testUtility.getConfiguration()); + Configurables.configure(sink, ctx); + Channel channel = createAndConfigureMemoryChannel(sink); + + sink.start(); + int batchCount = 3; + for (int i = 0; i < batchCount; i++) { + sink.process(); + } + sink.stop(); + Assert.assertEquals(batchCount, + ((IncrementHBaseSerializer) sink.getSerializer()).getNumBatchesStarted()); + } + + /** + * For testing that the rows coalesced, serialized by + * {@link IncrementHBaseSerializer}, are of the expected batch size. + */ + private static class CoalesceValidator + implements HBaseSink.DebugIncrementsCallback { + + private final Map expectedCounts; + private final Method refGetFamilyMap; + + public CoalesceValidator(Map expectedCounts) { + this.expectedCounts = expectedCounts; + this.refGetFamilyMap = HBaseSink.reflectLookupGetFamilyMap(); + } + + @Override + @SuppressWarnings("unchecked") + public void onAfterCoalesce(Iterable increments) { + for (Increment inc : increments) { + byte[] row = inc.getRow(); + Map> families = null; + try { + families = (Map>) + refGetFamilyMap.invoke(inc); + } catch (Exception e) { + Throwables.propagate(e); + } + for (byte[] family : families.keySet()) { + NavigableMap qualifiers = families.get(family); + for (Map.Entry entry : qualifiers.entrySet()) { + byte[] qualifier = entry.getKey(); + Long count = entry.getValue(); + StringBuilder b = new StringBuilder(20); + b.append(new String(row, Charsets.UTF_8)); + b.append(':'); + b.append(new String(qualifier, Charsets.UTF_8)); + String key = b.toString(); + Assert.assertEquals("Expected counts don't match observed for " + key, + expectedCounts.get(key), count); + } + } + } + } + } + + /** + * Add number of Events corresponding to counts to the events list. + * @param events Destination list. + * @param counts How many events to generate for each row:qualifier pair. + */ + private void generateEvents(List events, Map counts) { + for (String key : counts.keySet()) { + long count = counts.get(key); + for (long i = 0; i < count; i++) { + events.add(EventBuilder.withBody(key, Charsets.UTF_8)); + } + } + } + + private Channel createAndConfigureMemoryChannel(HBaseSink sink) { + Channel channel = new MemoryChannel(); + Context channelCtx = new Context(); + channelCtx.put("capacity", String.valueOf(1000L)); + channelCtx.put("transactionCapacity", String.valueOf(1000L)); + Configurables.configure(channel, channelCtx); + sink.setChannel(channel); + channel.start(); + return channel; + } + + private void putEvents(Channel channel, Iterable events) { + Transaction tx = channel.getTransaction(); + tx.begin(); + for (Event event : events) { + channel.put(event); + } + tx.commit(); + tx.close(); + } + +} From 47507bc40202c52e7156106642ffc0ab849fb96e Mon Sep 17 00:00:00 2001 From: Hari Shreedharan Date: Tue, 11 Mar 2014 13:05:51 -0700 Subject: [PATCH 089/341] FLUME-2243. Avro Source should name threads instead of using generic names. (Ashish Paliwal via Hari Shreedharan) --- .../org/apache/flume/source/AvroSource.java | 19 ++++++++++++++++--- 1 file changed, 16 insertions(+), 3 deletions(-) diff --git a/flume-ng-core/src/main/java/org/apache/flume/source/AvroSource.java b/flume-ng-core/src/main/java/org/apache/flume/source/AvroSource.java index c1ee3a93ce..3eef6870f5 100644 --- a/flume-ng-core/src/main/java/org/apache/flume/source/AvroSource.java +++ b/flume-ng-core/src/main/java/org/apache/flume/source/AvroSource.java @@ -35,7 +35,10 @@ import javax.net.ssl.KeyManagerFactory; import javax.net.ssl.SSLContext; import javax.net.ssl.SSLEngine; + +import com.google.common.util.concurrent.ThreadFactoryBuilder; import org.apache.avro.ipc.NettyServer; +import org.apache.avro.ipc.NettyTransceiver; import org.apache.avro.ipc.Responder; import org.apache.avro.ipc.Server; import org.apache.avro.ipc.specific.SpecificResponder; @@ -246,11 +249,21 @@ private NioServerSocketChannelFactory initSocketChannelFactory() { NioServerSocketChannelFactory socketChannelFactory; if (maxThreads <= 0) { socketChannelFactory = new NioServerSocketChannelFactory - (Executors .newCachedThreadPool(), Executors.newCachedThreadPool()); + (Executors.newCachedThreadPool(new ThreadFactoryBuilder(). + setNameFormat("Avro " + NettyTransceiver.class.getSimpleName() + + " Boss-%d").build()), + Executors.newCachedThreadPool(new ThreadFactoryBuilder(). + setNameFormat("Avro " + NettyTransceiver.class.getSimpleName() + + " I/O Worker-%d").build())); } else { socketChannelFactory = new NioServerSocketChannelFactory( - Executors.newCachedThreadPool(), - Executors.newFixedThreadPool(maxThreads)); + Executors.newCachedThreadPool(new ThreadFactoryBuilder(). + setNameFormat( + "Avro " + NettyTransceiver.class.getSimpleName() + + " Boss-%d").build()), + Executors.newFixedThreadPool(maxThreads, new ThreadFactoryBuilder(). + setNameFormat("Avro " + NettyTransceiver.class.getSimpleName() + + " I/O Worker-%d").build())); } return socketChannelFactory; } From 63d26c19adf250fdd4d3b98e2b049d2b5cefc698 Mon Sep 17 00:00:00 2001 From: Hari Shreedharan Date: Tue, 11 Mar 2014 23:35:51 -0700 Subject: [PATCH 090/341] FLUME-2252. Add null check before closing table in HbaseSink (Gopinathan A via Hari Shreedharan) --- .../main/java/org/apache/flume/sink/hbase/AsyncHBaseSink.java | 4 +++- .../src/main/java/org/apache/flume/sink/hbase/HBaseSink.java | 4 +++- 2 files changed, 6 insertions(+), 2 deletions(-) diff --git a/flume-ng-sinks/flume-ng-hbase-sink/src/main/java/org/apache/flume/sink/hbase/AsyncHBaseSink.java b/flume-ng-sinks/flume-ng-hbase-sink/src/main/java/org/apache/flume/sink/hbase/AsyncHBaseSink.java index 5316cd6252..2d03271948 100644 --- a/flume-ng-sinks/flume-ng-hbase-sink/src/main/java/org/apache/flume/sink/hbase/AsyncHBaseSink.java +++ b/flume-ng-sinks/flume-ng-hbase-sink/src/main/java/org/apache/flume/sink/hbase/AsyncHBaseSink.java @@ -460,7 +460,9 @@ public Object call(Object arg) throws Exception { @Override public void stop(){ serializer.cleanUp(); - client.shutdown(); + if (client != null) { + client.shutdown(); + } sinkCounter.incrementConnectionClosedCount(); sinkCounter.stop(); sinkCallbackPool.shutdown(); diff --git a/flume-ng-sinks/flume-ng-hbase-sink/src/main/java/org/apache/flume/sink/hbase/HBaseSink.java b/flume-ng-sinks/flume-ng-hbase-sink/src/main/java/org/apache/flume/sink/hbase/HBaseSink.java index 9996a4e76e..5de0bd56ef 100644 --- a/flume-ng-sinks/flume-ng-hbase-sink/src/main/java/org/apache/flume/sink/hbase/HBaseSink.java +++ b/flume-ng-sinks/flume-ng-hbase-sink/src/main/java/org/apache/flume/sink/hbase/HBaseSink.java @@ -187,7 +187,9 @@ public Boolean run() throws IOException { @Override public void stop(){ try { - table.close(); + if (table != null) { + table.close(); + } table = null; } catch (IOException e) { throw new FlumeException("Error closing table.", e); From 96b090b5117c34bba9f5104b47d005fe1c10c775 Mon Sep 17 00:00:00 2001 From: Hari Shreedharan Date: Wed, 12 Mar 2014 11:40:04 -0700 Subject: [PATCH 091/341] FLUME-2343. Add Kerberos and user impersonation support to Dataset Sink. (Ryan Blue via Hari Shreedharan) --- flume-ng-doc/sphinx/FlumeUserGuide.rst | 24 ++- .../apache/flume/sink/kite/DatasetSink.java | 24 ++- .../flume/sink/kite/DatasetSinkConstants.java | 6 + .../apache/flume/sink/kite/KerberosUtil.java | 176 ++++++++++++++++++ 4 files changed, 218 insertions(+), 12 deletions(-) create mode 100644 flume-ng-sinks/flume-dataset-sink/src/main/java/org/apache/flume/sink/kite/KerberosUtil.java diff --git a/flume-ng-doc/sphinx/FlumeUserGuide.rst b/flume-ng-doc/sphinx/FlumeUserGuide.rst index cedb2830b8..4bcd8a2d41 100644 --- a/flume-ng-doc/sphinx/FlumeUserGuide.rst +++ b/flume-ng-doc/sphinx/FlumeUserGuide.rst @@ -2047,16 +2047,20 @@ Note 2: In some cases, file rolling may occur slightly after the roll interval has been exceeded. However, this delay will not exceed 5 seconds. In most cases, the delay is neglegible. -===================== ======= =========================================================== -Property Name Default Description -===================== ======= =========================================================== -**channel** -- -**type** -- Must be org.apache.flume.sink.kite.DatasetSink -**kite.repo.uri** -- URI of the repository to open -**kite.dataset.name** -- Name of the Dataset where records will be written -kite.batchSize 100 Number of records to process in each batch -kite.rollInterval 30 Maximum wait time (seconds) before data files are released -===================== ======= =========================================================== +======================= ======= =========================================================== +Property Name Default Description +======================= ======= =========================================================== +**channel** -- +**type** -- Must be org.apache.flume.sink.kite.DatasetSink +**kite.repo.uri** -- URI of the repository to open +**kite.dataset.name** -- Name of the Dataset where records will be written +kite.batchSize 100 Number of records to process in each batch +kite.rollInterval 30 Maximum wait time (seconds) before data files are released +auth.kerberosPrincipal -- Kerberos user principal for secure authentication to HDFS +auth.kerberosKeytab -- Kerberos keytab location (local FS) for the principal +auth.proxyUser -- The effective user for HDFS actions, if different from + the kerberos principal +======================= ======= =========================================================== Custom Sink ~~~~~~~~~~~ diff --git a/flume-ng-sinks/flume-dataset-sink/src/main/java/org/apache/flume/sink/kite/DatasetSink.java b/flume-ng-sinks/flume-dataset-sink/src/main/java/org/apache/flume/sink/kite/DatasetSink.java index 1ee0a1fe4e..ed1b8d06e3 100644 --- a/flume-ng-sinks/flume-dataset-sink/src/main/java/org/apache/flume/sink/kite/DatasetSink.java +++ b/flume-ng-sinks/flume-dataset-sink/src/main/java/org/apache/flume/sink/kite/DatasetSink.java @@ -29,6 +29,7 @@ import java.io.InputStream; import java.net.URI; import java.net.URL; +import java.security.PrivilegedExceptionAction; import java.util.List; import java.util.Map; import java.util.concurrent.ExecutionException; @@ -47,6 +48,7 @@ import org.apache.hadoop.conf.Configuration; import org.apache.hadoop.fs.FileSystem; import org.apache.hadoop.fs.Path; +import org.apache.hadoop.security.UserGroupInformation; import org.kitesdk.data.Dataset; import org.kitesdk.data.DatasetRepositories; import org.kitesdk.data.DatasetWriter; @@ -69,8 +71,10 @@ public class DatasetSink extends AbstractSink implements Configurable { private String repositoryURI = null; private String datasetName = null; private long batchSize = DatasetSinkConstants.DEFAULT_BATCH_SIZE; + private Dataset targetDataset = null; private DatasetWriter writer = null; + private UserGroupInformation login = null; private SinkCounter counter = null; // for rolling files at a given interval @@ -130,14 +134,30 @@ protected List allowedFormats() { @Override public void configure(Context context) { + // initialize login credentials + this.login = KerberosUtil.login( + context.getString(DatasetSinkConstants.AUTH_PRINCIPAL), + context.getString(DatasetSinkConstants.AUTH_KEYTAB)); + String effectiveUser = + context.getString(DatasetSinkConstants.AUTH_PROXY_USER); + if (effectiveUser != null) { + this.login = KerberosUtil.proxyAs(effectiveUser, login); + } + this.repositoryURI = context.getString( DatasetSinkConstants.CONFIG_KITE_REPO_URI); Preconditions.checkNotNull(repositoryURI, "Repository URI is missing"); this.datasetName = context.getString( DatasetSinkConstants.CONFIG_KITE_DATASET_NAME); Preconditions.checkNotNull(datasetName, "Dataset name is missing"); - this.targetDataset = DatasetRepositories.open(repositoryURI) - .load(datasetName); + + this.targetDataset = KerberosUtil.runPrivileged(login, + new PrivilegedExceptionAction>() { + @Override + public Dataset run() { + return DatasetRepositories.open(repositoryURI).load(datasetName); + } + }); String formatName = targetDataset.getDescriptor().getFormat().getName(); Preconditions.checkArgument(allowedFormats().contains(formatName), diff --git a/flume-ng-sinks/flume-dataset-sink/src/main/java/org/apache/flume/sink/kite/DatasetSinkConstants.java b/flume-ng-sinks/flume-dataset-sink/src/main/java/org/apache/flume/sink/kite/DatasetSinkConstants.java index 13c776ef9a..09dfab639d 100644 --- a/flume-ng-sinks/flume-dataset-sink/src/main/java/org/apache/flume/sink/kite/DatasetSinkConstants.java +++ b/flume-ng-sinks/flume-dataset-sink/src/main/java/org/apache/flume/sink/kite/DatasetSinkConstants.java @@ -48,4 +48,10 @@ public class DatasetSinkConstants { "flume.avro.schema.literal"; public static final String AVRO_SCHEMA_URL_HEADER = "flume.avro.schema.url"; + /** + * Hadoop authentication settings + */ + public static final String AUTH_PROXY_USER = "auth.proxyUser"; + public static final String AUTH_PRINCIPAL = "auth.kerberosPrincipal"; + public static final String AUTH_KEYTAB = "auth.kerberosKeytab"; } diff --git a/flume-ng-sinks/flume-dataset-sink/src/main/java/org/apache/flume/sink/kite/KerberosUtil.java b/flume-ng-sinks/flume-dataset-sink/src/main/java/org/apache/flume/sink/kite/KerberosUtil.java new file mode 100644 index 0000000000..92ad1413d2 --- /dev/null +++ b/flume-ng-sinks/flume-dataset-sink/src/main/java/org/apache/flume/sink/kite/KerberosUtil.java @@ -0,0 +1,176 @@ +/** + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.flume.sink.kite; + +import com.google.common.base.Preconditions; +import com.google.common.base.Throwables; +import java.io.File; +import java.io.IOException; +import java.security.PrivilegedExceptionAction; +import org.apache.hadoop.security.SecurityUtil; +import org.apache.hadoop.security.UserGroupInformation; +import org.kitesdk.data.DatasetException; +import org.kitesdk.data.DatasetIOException; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +public class KerberosUtil { + + private static final Logger LOG = LoggerFactory.getLogger(KerberosUtil.class); + + public static class SecurityException extends RuntimeException { + private SecurityException(String message) { + super(message); + } + + private SecurityException(String message, Throwable cause) { + super(message, cause); + } + + private SecurityException(Throwable cause) { + super(cause); + } + } + + public static UserGroupInformation proxyAs(String username, + UserGroupInformation login) { + Preconditions.checkArgument(username != null && !username.isEmpty(), + "Invalid username: " + String.valueOf(username)); + Preconditions.checkArgument(login != null, + "Cannot proxy without an authenticated user"); + + // hadoop impersonation works with or without kerberos security + return UserGroupInformation.createProxyUser(username, login); + } + + /** + * Static synchronized method for static Kerberos login.
    + * Static synchronized due to a thundering herd problem when multiple Sinks + * attempt to log in using the same principal at the same time with the + * intention of impersonating different users (or even the same user). + * If this is not controlled, MIT Kerberos v5 believes it is seeing a replay + * attach and it returns: + *
    Request is a replay (34) - PROCESS_TGS
    + * In addition, since the underlying Hadoop APIs we are using for + * impersonation are static, we define this method as static as well. + * + * @param principal + * Fully-qualified principal to use for authentication. + * @param keytab + * Location of keytab file containing credentials for principal. + * @return Logged-in user + * @throws SecurityException + * if login fails. + * @throws IllegalArgumentException + * if the principal or the keytab is not usable + */ + public static synchronized UserGroupInformation login(String principal, + String keytab) { + // resolve the requested principal, if it is present + String finalPrincipal = null; + if (principal != null && !principal.isEmpty()) { + try { + // resolves _HOST pattern using standard Hadoop search/replace + // via DNS lookup when 2nd argument is empty + finalPrincipal = SecurityUtil.getServerPrincipal(principal, ""); + } catch (IOException e) { + throw new SecurityException( + "Failed to resolve Kerberos principal", e); + } + } + + // check if there is a user already logged in + UserGroupInformation currentUser = null; + try { + currentUser = UserGroupInformation.getLoginUser(); + } catch (IOException e) { + // not a big deal but this shouldn't typically happen because it will + // generally fall back to the UNIX user + LOG.debug("Unable to get login user before Kerberos auth attempt", e); + } + + // if the current user is valid (matches the given principal) then use it + if (currentUser != null) { + if (finalPrincipal == null || + finalPrincipal.equals(currentUser.getUserName())) { + LOG.debug("Using existing login for {}: {}", + finalPrincipal, currentUser); + return currentUser; + } else { + // be cruel and unusual when user tries to login as multiple principals + // this isn't really valid with a reconfigure but this should be rare + // enough to warrant a restart of the agent JVM + // TODO: find a way to interrogate the entire current config state, + // since we don't have to be unnecessarily protective if they switch all + // HDFS sinks to use a different principal all at once. + throw new SecurityException( + "Cannot use multiple Kerberos principals: " + finalPrincipal + + " would replace " + currentUser.getUserName()); + } + } + + // prepare for a new login + Preconditions.checkArgument(principal != null && !principal.isEmpty(), + "Invalid Kerberos principal: " + String.valueOf(principal)); + Preconditions.checkNotNull(finalPrincipal, + "Resolved principal must not be null"); + Preconditions.checkArgument(keytab != null && !keytab.isEmpty(), + "Invalid Kerberos keytab: " + String.valueOf(keytab)); + File keytabFile = new File(keytab); + Preconditions.checkArgument(keytabFile.isFile() && keytabFile.canRead(), + "Keytab is not a readable file: " + String.valueOf(keytab)); + + try { + // attempt static kerberos login + LOG.debug("Logging in as {} with {}", finalPrincipal, keytab); + UserGroupInformation.loginUserFromKeytab(principal, keytab); + return UserGroupInformation.getLoginUser(); + } catch (IOException e) { + throw new SecurityException("Kerberos login failed", e); + } + } + + /** + * Allow methods to act with the privileges of a login. + * + * If the login is null, the current privileges will be used. + * + * @param The return type of the action + * @param login UserGroupInformation credentials to use for action + * @param action A PrivilegedExceptionAction to perform as another user + * @return the T value returned by action.run() + */ + public static T runPrivileged(UserGroupInformation login, + PrivilegedExceptionAction action) { + try { + if (login == null) { + return action.run(); + } else { + return login.doAs(action); + } + } catch (IOException ex) { + throw new DatasetIOException("Privileged action failed", ex); + } catch (InterruptedException ex) { + Thread.interrupted(); + throw new DatasetException(ex); + } catch (Exception ex) { + throw Throwables.propagate(ex); + } + } +} From 1f21df7e11f5fe8a3dcd0bd34116289c02943a64 Mon Sep 17 00:00:00 2001 From: Hari Shreedharan Date: Wed, 12 Mar 2014 12:35:38 -0700 Subject: [PATCH 092/341] FLUME-2151. Windows: Update TestExecSource to use native commands on Windows (Roshan Naik via Hari Shreedharan) --- .../apache/flume/source/TestExecSource.java | 171 +++++++++++++----- .../src/test/resources/test_command.ps1 | 3 + 2 files changed, 125 insertions(+), 49 deletions(-) create mode 100644 flume-ng-core/src/test/resources/test_command.ps1 diff --git a/flume-ng-core/src/test/java/org/apache/flume/source/TestExecSource.java b/flume-ng-core/src/test/java/org/apache/flume/source/TestExecSource.java index 54f71a1516..289c2d2ce3 100644 --- a/flume-ng-core/src/test/java/org/apache/flume/source/TestExecSource.java +++ b/flume-ng-core/src/test/java/org/apache/flume/source/TestExecSource.java @@ -34,6 +34,8 @@ import javax.management.ObjectName; import org.apache.commons.io.FileUtils; +import org.apache.commons.lang.RandomStringUtils; +import org.apache.commons.lang.SystemUtils; import org.apache.flume.Channel; import org.apache.flume.ChannelSelector; import org.apache.flume.Context; @@ -94,20 +96,38 @@ public void tearDown() { public void testProcess() throws InterruptedException, LifecycleException, EventDeliveryException, IOException { - context.put("command", "cat /etc/passwd"); + // Generates a random files for input\output + File inputFile = File.createTempFile("input", null); + File ouputFile = File.createTempFile("ouput", null); + FileUtils.forceDeleteOnExit(inputFile); + FileUtils.forceDeleteOnExit(ouputFile); + + // Generates input file with a random data set (10 lines, 200 characters each) + FileOutputStream outputStream1 = new FileOutputStream(inputFile); + for (int i=0; i<10; i++) { + outputStream1.write( + RandomStringUtils.randomAlphanumeric(200).getBytes()); + outputStream1.write('\n'); + } + outputStream1.close(); + + String command = SystemUtils.IS_OS_WINDOWS ? + String.format("cmd /c type %s", inputFile.getAbsolutePath()) : + String.format("cat %s", inputFile.getAbsolutePath()); + context.put("command", command); context.put("keep-alive", "1"); context.put("capacity", "1000"); context.put("transactionCapacity", "1000"); Configurables.configure(source, context); source.start(); + Thread.sleep(2000); Transaction transaction = channel.getTransaction(); transaction.begin(); Event event; - FileOutputStream outputStream = new FileOutputStream( - "/tmp/flume-execsource." + Thread.currentThread().getId()); + FileOutputStream outputStream = new FileOutputStream(ouputFile); while ((event = channel.take()) != null) { outputStream.write(event.getBody()); @@ -118,69 +138,101 @@ public void testProcess() throws InterruptedException, LifecycleException, transaction.commit(); transaction.close(); - File file1 = new File("/tmp/flume-execsource." - + Thread.currentThread().getId()); - File file2 = new File("/etc/passwd"); - Assert.assertEquals(FileUtils.checksumCRC32(file1), - FileUtils.checksumCRC32(file2)); - FileUtils.forceDelete(file1); + Assert.assertEquals(FileUtils.checksumCRC32(inputFile), + FileUtils.checksumCRC32(ouputFile)); } @Test public void testShellCommandSimple() throws InterruptedException, LifecycleException, - EventDeliveryException, IOException { - runTestShellCmdHelper("/bin/bash -c", "seq 5" - , new String[]{"1","2","3","4","5" } ); + EventDeliveryException, IOException { + if (SystemUtils.IS_OS_WINDOWS) { + runTestShellCmdHelper("powershell -ExecutionPolicy Unrestricted -command", + "1..5", new String[]{"1", "2", "3", "4", "5"}); + } else { + runTestShellCmdHelper("/bin/bash -c", "seq 5", + new String[]{"1", "2", "3", "4", "5"}); + } } @Test - public void testShellCommandBackTicks() throws InterruptedException, LifecycleException, - EventDeliveryException, IOException { + public void testShellCommandBackTicks() + throws InterruptedException, LifecycleException, EventDeliveryException, + IOException { // command with backticks - runTestShellCmdHelper("/bin/bash -c", "echo `seq 5`" , new String[]{"1 2 3 4 5" } ); - runTestShellCmdHelper("/bin/bash -c", "echo $(seq 5)" , new String[]{"1 2 3 4 5" } ); + if (SystemUtils.IS_OS_WINDOWS) { + runTestShellCmdHelper( + "powershell -ExecutionPolicy Unrestricted -command", "$(1..5)", + new String[]{"1", "2", "3", "4", "5"}); + } else { + runTestShellCmdHelper("/bin/bash -c", "echo `seq 5`", + new String[]{"1 2 3 4 5"}); + runTestShellCmdHelper("/bin/bash -c", "echo $(seq 5)", + new String[]{"1 2 3 4 5"}); + } } @Test - public void testShellCommandComplex() throws InterruptedException, LifecycleException, - EventDeliveryException, IOException { + public void testShellCommandComplex() + throws InterruptedException, LifecycleException, EventDeliveryException, + IOException { // command with wildcards & pipes String[] expected = {"1234", "abcd", "ijk", "xyz", "zzz"}; - // pipes - runTestShellCmdHelper("/bin/bash -c", "echo zzz 1234 xyz abcd ijk | xargs -n1 echo | sort -f" - , expected ); + if (SystemUtils.IS_OS_WINDOWS) { + runTestShellCmdHelper( + "powershell -ExecutionPolicy Unrestricted -command", + "'zzz','1234','xyz','abcd','ijk' | sort", expected); + } else { + runTestShellCmdHelper("/bin/bash -c", + "echo zzz 1234 xyz abcd ijk | xargs -n1 echo | sort -f", expected); + } } @Test - public void testShellCommandScript() throws InterruptedException, LifecycleException, - EventDeliveryException, IOException { + public void testShellCommandScript() + throws InterruptedException, LifecycleException, EventDeliveryException, + IOException { // mini script - runTestShellCmdHelper("/bin/bash -c", "for i in {1..5}; do echo $i;done" - , new String[]{"1","2","3","4","5" } ); - // shell arithmetic - runTestShellCmdHelper("/bin/bash -c", "if ((2+2>3)); then echo good; else echo not good; fi" , new String[]{"good"} ); + if (SystemUtils.IS_OS_WINDOWS) { + runTestShellCmdHelper("powershell -ExecutionPolicy Unrestricted -command", + "foreach ($i in 1..5) { $i }", new String[]{"1", "2", "3", "4", "5"}); + // shell arithmetic + runTestShellCmdHelper("powershell -ExecutionPolicy Unrestricted -command", + "if(2+2 -gt 3) { 'good' } else { 'not good' } ", new String[]{"good"}); + } else { + runTestShellCmdHelper("/bin/bash -c", "for i in {1..5}; do echo $i;done" + , new String[]{"1", "2", "3", "4", "5"}); + // shell arithmetic + runTestShellCmdHelper("/bin/bash -c", "if ((2+2>3)); " + + "then echo good; else echo not good; fi", new String[]{"good"}); + } } @Test - public void testShellCommandEmbeddingAndEscaping() throws InterruptedException, LifecycleException, - EventDeliveryException, IOException { - System.out.println( "######### PWD = " + new java.io.File( "." ).getCanonicalPath() ); + public void testShellCommandEmbeddingAndEscaping() + throws InterruptedException, LifecycleException, EventDeliveryException, + IOException { // mini script - BufferedReader reader = new BufferedReader(new FileReader("src/test/resources/test_command.txt") ); + String fileName = SystemUtils.IS_OS_WINDOWS ? + "src\\test\\resources\\test_command.ps1" : + "src/test/resources/test_command.txt"; + BufferedReader reader = new BufferedReader(new FileReader(fileName)); try { + String shell = SystemUtils.IS_OS_WINDOWS ? + "powershell -ExecutionPolicy Unrestricted -command" : + "/bin/bash -c"; String command1 = reader.readLine(); Assert.assertNotNull(command1); String[] output1 = new String[] {"'1'", "\"2\"", "\\3", "\\4"}; - runTestShellCmdHelper("/bin/bash -c", command1 , output1); + runTestShellCmdHelper( shell, command1 , output1); String command2 = reader.readLine(); Assert.assertNotNull(command2); String[] output2 = new String[]{"1","2","3","4","5" }; - runTestShellCmdHelper("/bin/bash -c", command2 , output2); + runTestShellCmdHelper(shell, command2 , output2); String command3 = reader.readLine(); Assert.assertNotNull(command3); String[] output3 = new String[]{"2","3","4","5","6" }; - runTestShellCmdHelper("/bin/bash -c", command3 , output3); + runTestShellCmdHelper(shell, command3 , output3); } finally { reader.close(); } @@ -190,8 +242,14 @@ public void testShellCommandEmbeddingAndEscaping() throws InterruptedException, public void testMonitoredCounterGroup() throws InterruptedException, LifecycleException, EventDeliveryException, IOException { // mini script - runTestShellCmdHelper("/bin/bash -c", "for i in {1..5}; do echo $i;done" - , new String[]{"1","2","3","4","5" } ); + if (SystemUtils.IS_OS_WINDOWS) { + runTestShellCmdHelper("powershell -ExecutionPolicy Unrestricted -command", + "foreach ($i in 1..5) { $i }" + , new String[]{"1", "2", "3", "4", "5"}); + } else { + runTestShellCmdHelper("/bin/bash -c", "for i in {1..5}; do echo $i;done" + , new String[]{"1", "2", "3", "4", "5"}); + } ObjectName objName = null; @@ -237,8 +295,13 @@ public void testBatchTimeout() throws InterruptedException, LifecycleException, context.put(ExecSourceConfigurationConstants.CONFIG_BATCH_SIZE, "50000"); context.put(ExecSourceConfigurationConstants.CONFIG_BATCH_TIME_OUT, "750"); - context.put("shell", "/bin/bash -c"); - context.put("command", "tail -f " + filePath); + context.put("shell", SystemUtils.IS_OS_WINDOWS ? + "powershell -ExecutionPolicy Unrestricted -command" : + "/bin/bash -c"); + context.put("command", SystemUtils.IS_OS_WINDOWS ? + "Get-Content " + filePath + + " | Select-Object -Last 10" : + ("tail -f " + filePath)); Configurables.configure(source, context); source.start(); @@ -277,6 +340,8 @@ private void runTestShellCmdHelper(String shell, String command, String[] expect source.start(); File outputFile = File.createTempFile("flumeExecSourceTest_", ""); FileOutputStream outputStream = new FileOutputStream(outputFile); + if(SystemUtils.IS_OS_WINDOWS) + Thread.sleep(2500); Transaction transaction = channel.getTransaction(); transaction.begin(); try { @@ -288,10 +353,10 @@ private void runTestShellCmdHelper(String shell, String command, String[] expect outputStream.close(); transaction.commit(); List output = Files.readLines(outputFile, Charset.defaultCharset()); - System.out.println("command : " + command); - System.out.println("output : "); - for( String line : output ) - System.out.println(); +// System.out.println("command : " + command); +// System.out.println("output : "); +// for( String line : output ) +// System.out.println(line); Assert.assertArrayEquals(expectedOutput, output.toArray(new String[]{})); } finally { FileUtils.forceDelete(outputFile); @@ -308,7 +373,8 @@ public void testRestart() throws InterruptedException, LifecycleException, context.put(ExecSourceConfigurationConstants.CONFIG_RESTART_THROTTLE, "10"); context.put(ExecSourceConfigurationConstants.CONFIG_RESTART, "true"); - context.put("command", "echo flume"); + context.put("command", + SystemUtils.IS_OS_WINDOWS ? "cmd /c echo flume" : "echo flume"); Configurables.configure(source, context); source.start(); @@ -348,12 +414,17 @@ public void testShutdown() throws Exception { // now find one that is not in use boolean searchForCommand = true; - while(searchForCommand) { + while (searchForCommand) { searchForCommand = false; - String command = "sleep " + seconds; - Pattern pattern = Pattern.compile("\b" + command + "\b"); - for(String line : exec("ps -ef")) { - if(pattern.matcher(line).find()) { + String command = SystemUtils.IS_OS_WINDOWS ? ("cmd /c sleep " + seconds) : + ("sleep " + seconds); + String searchTxt = SystemUtils.IS_OS_WINDOWS ? ("sleep.exe") : + ("\b" + command + "\b"); + Pattern pattern = Pattern.compile(searchTxt); + for (String line : exec(SystemUtils.IS_OS_WINDOWS ? + "cmd /c tasklist /FI \"SESSIONNAME eq Console\"" : + "ps -ef")) { + if (pattern.matcher(line).find()) { seconds++; searchForCommand = true; break; @@ -376,7 +447,9 @@ public void testShutdown() throws Exception { Thread.sleep(1000L); source.stop(); Thread.sleep(1000L); - for(String line : exec("ps -ef")) { + for (String line : exec(SystemUtils.IS_OS_WINDOWS ? + "cmd /c tasklist /FI \"SESSIONNAME eq Console\"" : + "ps -ef")) { if(pattern.matcher(line).find()) { Assert.fail("Found [" + line + "]"); } diff --git a/flume-ng-core/src/test/resources/test_command.ps1 b/flume-ng-core/src/test/resources/test_command.ps1 new file mode 100644 index 0000000000..4fbd820f70 --- /dev/null +++ b/flume-ng-core/src/test/resources/test_command.ps1 @@ -0,0 +1,3 @@ +" \"'1'\", \"\"\"2\"\"\",\"`\3\", \"\4\" " +foreach ($i in 1..5) { $i } +1..5 | %{$_ +1 } \ No newline at end of file From f4c6c7025e892ecf0d6fac1b72b5806fe7c9cd8d Mon Sep 17 00:00:00 2001 From: Hari Shreedharan Date: Thu, 13 Mar 2014 13:05:29 -0700 Subject: [PATCH 093/341] FLUME-2263. Bump Hadoop 2 version to 2.3.0 (Gopinathan A via Hari Shreedharan) --- flume-ng-sinks/flume-hdfs-sink/pom.xml | 4 ---- flume-tools/pom.xml | 8 -------- pom.xml | 2 +- 3 files changed, 1 insertion(+), 13 deletions(-) diff --git a/flume-ng-sinks/flume-hdfs-sink/pom.xml b/flume-ng-sinks/flume-hdfs-sink/pom.xml index c646a89e2c..43b1a0f025 100644 --- a/flume-ng-sinks/flume-hdfs-sink/pom.xml +++ b/flume-ng-sinks/flume-hdfs-sink/pom.xml @@ -111,10 +111,6 @@ limitations under the License. !hadoop.profile - - 1.0.1 - hadoop-core - diff --git a/flume-tools/pom.xml b/flume-tools/pom.xml index 1e707fa2e8..1f265a3f43 100644 --- a/flume-tools/pom.xml +++ b/flume-tools/pom.xml @@ -80,10 +80,6 @@ limitations under the License. !hadoop.profile - - 1.0.1 - hadoop-core - @@ -111,10 +107,6 @@ limitations under the License. 2 - - 2.0.0-alpha - hadoop-common - diff --git a/pom.xml b/pom.xml index 08fffc46ab..29a04cb564 100644 --- a/pom.xml +++ b/pom.xml @@ -49,7 +49,7 @@ limitations under the License. 1.7.3 0.90.1 - 2.1.0-beta + 2.3.0 From 1c8b8f5e9185d75c6b1cb06a617dfc08447935c9 Mon Sep 17 00:00:00 2001 From: Hari Shreedharan Date: Thu, 13 Mar 2014 14:51:03 -0700 Subject: [PATCH 094/341] FLUME-2345. Update to Kite 0.12.0 dependency (Ryan Blue via Hari Shreedharan) --- flume-ng-sinks/flume-ng-morphline-solr-sink/pom.xml | 1 - pom.xml | 4 +++- 2 files changed, 3 insertions(+), 2 deletions(-) diff --git a/flume-ng-sinks/flume-ng-morphline-solr-sink/pom.xml b/flume-ng-sinks/flume-ng-morphline-solr-sink/pom.xml index 7d9065199b..e4be72099d 100644 --- a/flume-ng-sinks/flume-ng-morphline-solr-sink/pom.xml +++ b/flume-ng-sinks/flume-ng-morphline-solr-sink/pom.xml @@ -33,7 +33,6 @@ limitations under the License. UTF-8 4.3.0 4.3.0 - 0.10.0 1.6.1 2.12.4 diff --git a/pom.xml b/pom.xml index 29a04cb564..2aa0ad146f 100644 --- a/pom.xml +++ b/pom.xml @@ -50,6 +50,8 @@ limitations under the License. 0.90.1 2.3.0 + + 0.12.0 @@ -1111,7 +1113,7 @@ limitations under the License. org.kitesdk kite-data-core - 0.10.1 + ${kite.version} From e12f0a7a46752eff2ff3aaddfa0b93fb606074d7 Mon Sep 17 00:00:00 2001 From: Hari Shreedharan Date: Thu, 20 Mar 2014 16:16:14 -0700 Subject: [PATCH 095/341] FLUME-2225. Elasticsearch Sink for ES HTTP API (Pawel Rog via Edward Sargisson, Hari Shreedharan) --- .../flume-ng-elasticsearch-sink/pom.xml | 24 +- ...asticSearchIndexRequestBuilderFactory.java | 40 +-- ...asticSearchIndexRequestBuilderFactory.java | 26 +- .../sink/elasticsearch/ElasticSearchSink.java | 278 +++++++++--------- .../ElasticSearchSinkConstants.java | 29 ++ .../sink/elasticsearch/IndexNameBuilder.java | 42 +++ .../elasticsearch/SimpleIndexNameBuilder.java | 45 +++ .../TimeBasedIndexNameBuilder.java | 90 ++++++ .../sink/elasticsearch/TimestampedEvent.java | 60 ++++ .../client/ElasticSearchClient.java | 58 ++++ .../client/ElasticSearchClientFactory.java | 76 +++++ .../client/ElasticSearchRestClient.java | 148 ++++++++++ .../client/ElasticSearchTransportClient.java | 231 +++++++++++++++ .../client/NoSuchClientTypeException.java | 23 ++ .../elasticsearch/client/RoundRobinList.java | 44 +++ .../TestElasticSearchDynamicSerializer.java | 12 +- ...asticSearchIndexRequestBuilderFactory.java | 42 +-- ...tElasticSearchLogStashEventSerializer.java | 14 +- .../elasticsearch/TestElasticSearchSink.java | 126 +++++--- .../TimeBasedIndexNameBuilderTest.java | 60 ++++ .../elasticsearch/TimestampedEventTest.java | 88 ++++++ .../client/RoundRobinListTest.java | 41 +++ .../TestElasticSearchClientFactory.java | 64 ++++ .../client/TestElasticSearchRestClient.java | 158 ++++++++++ .../TestElasticSearchTransportClient.java | 127 ++++++++ 25 files changed, 1679 insertions(+), 267 deletions(-) create mode 100644 flume-ng-sinks/flume-ng-elasticsearch-sink/src/main/java/org/apache/flume/sink/elasticsearch/IndexNameBuilder.java create mode 100644 flume-ng-sinks/flume-ng-elasticsearch-sink/src/main/java/org/apache/flume/sink/elasticsearch/SimpleIndexNameBuilder.java create mode 100644 flume-ng-sinks/flume-ng-elasticsearch-sink/src/main/java/org/apache/flume/sink/elasticsearch/TimeBasedIndexNameBuilder.java create mode 100644 flume-ng-sinks/flume-ng-elasticsearch-sink/src/main/java/org/apache/flume/sink/elasticsearch/TimestampedEvent.java create mode 100644 flume-ng-sinks/flume-ng-elasticsearch-sink/src/main/java/org/apache/flume/sink/elasticsearch/client/ElasticSearchClient.java create mode 100644 flume-ng-sinks/flume-ng-elasticsearch-sink/src/main/java/org/apache/flume/sink/elasticsearch/client/ElasticSearchClientFactory.java create mode 100644 flume-ng-sinks/flume-ng-elasticsearch-sink/src/main/java/org/apache/flume/sink/elasticsearch/client/ElasticSearchRestClient.java create mode 100644 flume-ng-sinks/flume-ng-elasticsearch-sink/src/main/java/org/apache/flume/sink/elasticsearch/client/ElasticSearchTransportClient.java create mode 100644 flume-ng-sinks/flume-ng-elasticsearch-sink/src/main/java/org/apache/flume/sink/elasticsearch/client/NoSuchClientTypeException.java create mode 100644 flume-ng-sinks/flume-ng-elasticsearch-sink/src/main/java/org/apache/flume/sink/elasticsearch/client/RoundRobinList.java create mode 100644 flume-ng-sinks/flume-ng-elasticsearch-sink/src/test/java/org/apache/flume/sink/elasticsearch/TimeBasedIndexNameBuilderTest.java create mode 100644 flume-ng-sinks/flume-ng-elasticsearch-sink/src/test/java/org/apache/flume/sink/elasticsearch/TimestampedEventTest.java create mode 100644 flume-ng-sinks/flume-ng-elasticsearch-sink/src/test/java/org/apache/flume/sink/elasticsearch/client/RoundRobinListTest.java create mode 100644 flume-ng-sinks/flume-ng-elasticsearch-sink/src/test/java/org/apache/flume/sink/elasticsearch/client/TestElasticSearchClientFactory.java create mode 100644 flume-ng-sinks/flume-ng-elasticsearch-sink/src/test/java/org/apache/flume/sink/elasticsearch/client/TestElasticSearchRestClient.java create mode 100644 flume-ng-sinks/flume-ng-elasticsearch-sink/src/test/java/org/apache/flume/sink/elasticsearch/client/TestElasticSearchTransportClient.java diff --git a/flume-ng-sinks/flume-ng-elasticsearch-sink/pom.xml b/flume-ng-sinks/flume-ng-elasticsearch-sink/pom.xml index bdc21d1eb8..dedb738e45 100644 --- a/flume-ng-sinks/flume-ng-elasticsearch-sink/pom.xml +++ b/flume-ng-sinks/flume-ng-elasticsearch-sink/pom.xml @@ -24,7 +24,7 @@ flume-ng-elasticsearch-sink Flume NG ElasticSearch Sink - + org.apache.rat @@ -51,9 +51,14 @@ - org.elasticsearch - elasticsearch - true + org.elasticsearch + elasticsearch + true + + + + org.apache.httpcomponents + httpclient @@ -78,5 +83,16 @@ guava + + org.apache.httpcomponents + httpclient + + + + org.mockito + mockito-all + test + + diff --git a/flume-ng-sinks/flume-ng-elasticsearch-sink/src/main/java/org/apache/flume/sink/elasticsearch/AbstractElasticSearchIndexRequestBuilderFactory.java b/flume-ng-sinks/flume-ng-elasticsearch-sink/src/main/java/org/apache/flume/sink/elasticsearch/AbstractElasticSearchIndexRequestBuilderFactory.java index 6effe343f5..de84b95433 100644 --- a/flume-ng-sinks/flume-ng-elasticsearch-sink/src/main/java/org/apache/flume/sink/elasticsearch/AbstractElasticSearchIndexRequestBuilderFactory.java +++ b/flume-ng-sinks/flume-ng-elasticsearch-sink/src/main/java/org/apache/flume/sink/elasticsearch/AbstractElasticSearchIndexRequestBuilderFactory.java @@ -19,22 +19,17 @@ package org.apache.flume.sink.elasticsearch; import java.io.IOException; -import java.util.Map; -import org.apache.commons.lang.StringUtils; import org.apache.commons.lang.time.FastDateFormat; import org.apache.flume.Context; import org.apache.flume.Event; import org.apache.flume.conf.ComponentConfiguration; import org.apache.flume.conf.Configurable; import org.apache.flume.conf.ConfigurableComponent; -import org.apache.flume.event.SimpleEvent; import org.elasticsearch.action.index.IndexRequestBuilder; import org.elasticsearch.client.Client; -import org.joda.time.DateTimeUtils; import com.google.common.annotations.VisibleForTesting; -import com.google.common.collect.Maps; /** * Abstract base class for custom implementations of @@ -122,37 +117,4 @@ protected abstract void prepareIndexRequest( IndexRequestBuilder indexRequest, String indexName, String indexType, Event event) throws IOException; -} - -/** - * {@link Event} implementation that has a timestamp. - * The timestamp is taken from (in order of precedence):
      - *
    1. The "timestamp" header of the base event, if present
    2. - *
    3. The "@timestamp" header of the base event, if present
    4. - *
    5. The current time in millis, otherwise
    6. - *
    - */ -final class TimestampedEvent extends SimpleEvent { - - private final long timestamp; - - TimestampedEvent(Event base) { - setBody(base.getBody()); - Map headers = Maps.newHashMap(base.getHeaders()); - String timestampString = headers.get("timestamp"); - if (StringUtils.isBlank(timestampString)) { - timestampString = headers.get("@timestamp"); - } - if (StringUtils.isBlank(timestampString)) { - this.timestamp = DateTimeUtils.currentTimeMillis(); - headers.put("timestamp", String.valueOf(timestamp )); - } else { - this.timestamp = Long.valueOf(timestampString); - } - setHeaders(headers); - } - - long getTimestamp() { - return timestamp; - } -} +} \ No newline at end of file diff --git a/flume-ng-sinks/flume-ng-elasticsearch-sink/src/main/java/org/apache/flume/sink/elasticsearch/ElasticSearchIndexRequestBuilderFactory.java b/flume-ng-sinks/flume-ng-elasticsearch-sink/src/main/java/org/apache/flume/sink/elasticsearch/ElasticSearchIndexRequestBuilderFactory.java index 8e77a1e61d..1ca227a897 100644 --- a/flume-ng-sinks/flume-ng-elasticsearch-sink/src/main/java/org/apache/flume/sink/elasticsearch/ElasticSearchIndexRequestBuilderFactory.java +++ b/flume-ng-sinks/flume-ng-elasticsearch-sink/src/main/java/org/apache/flume/sink/elasticsearch/ElasticSearchIndexRequestBuilderFactory.java @@ -18,9 +18,6 @@ */ package org.apache.flume.sink.elasticsearch; -import java.io.IOException; -import java.util.TimeZone; - import org.apache.commons.lang.time.FastDateFormat; import org.apache.flume.Event; import org.apache.flume.conf.Configurable; @@ -28,10 +25,13 @@ import org.elasticsearch.action.index.IndexRequestBuilder; import org.elasticsearch.client.Client; +import java.io.IOException; +import java.util.TimeZone; + /** - * Interface for creating ElasticSearch {@link IndexRequestBuilder} - * instances from serialized flume events. This is configurable, so any config - * params required should be taken through this. + * Interface for creating ElasticSearch {@link IndexRequestBuilder} instances + * from serialized flume events. This is configurable, so any config params + * required should be taken through this. */ public interface ElasticSearchIndexRequestBuilderFactory extends Configurable, ConfigurableComponent { @@ -43,16 +43,18 @@ public interface ElasticSearchIndexRequestBuilderFactory extends Configurable, * @return prepared ElasticSearch {@link IndexRequestBuilder} instance * @param client * ElasticSearch {@link Client} to prepare index from - * @param indexPrefix + * @param indexPrefix * Prefix of index name to use -- as configured on the sink - * @param indexType + * @param indexType * Index type to use -- as configured on the sink - * @param event + * @param event * Flume event to serialize and add to index request * @throws IOException * If an error occurs e.g. during serialization - */ - IndexRequestBuilder createIndexRequest(Client client, - String indexPrefix, String indexType, Event event) throws IOException; + */ + IndexRequestBuilder createIndexRequest(Client client, String indexPrefix, + String indexType, Event event) throws IOException; + + } diff --git a/flume-ng-sinks/flume-ng-elasticsearch-sink/src/main/java/org/apache/flume/sink/elasticsearch/ElasticSearchSink.java b/flume-ng-sinks/flume-ng-elasticsearch-sink/src/main/java/org/apache/flume/sink/elasticsearch/ElasticSearchSink.java index e38ab19b2f..39b6db573e 100644 --- a/flume-ng-sinks/flume-ng-elasticsearch-sink/src/main/java/org/apache/flume/sink/elasticsearch/ElasticSearchSink.java +++ b/flume-ng-sinks/flume-ng-elasticsearch-sink/src/main/java/org/apache/flume/sink/elasticsearch/ElasticSearchSink.java @@ -23,7 +23,6 @@ import static org.apache.flume.sink.elasticsearch.ElasticSearchSinkConstants.DEFAULT_CLUSTER_NAME; import static org.apache.flume.sink.elasticsearch.ElasticSearchSinkConstants.DEFAULT_INDEX_NAME; import static org.apache.flume.sink.elasticsearch.ElasticSearchSinkConstants.DEFAULT_INDEX_TYPE; -import static org.apache.flume.sink.elasticsearch.ElasticSearchSinkConstants.DEFAULT_PORT; import static org.apache.flume.sink.elasticsearch.ElasticSearchSinkConstants.DEFAULT_TTL; import static org.apache.flume.sink.elasticsearch.ElasticSearchSinkConstants.HOSTNAMES; import static org.apache.flume.sink.elasticsearch.ElasticSearchSinkConstants.INDEX_NAME; @@ -32,12 +31,6 @@ import static org.apache.flume.sink.elasticsearch.ElasticSearchSinkConstants.SERIALIZER_PREFIX; import static org.apache.flume.sink.elasticsearch.ElasticSearchSinkConstants.TTL; import static org.apache.flume.sink.elasticsearch.ElasticSearchSinkConstants.TTL_REGEX; - -import java.util.Arrays; -import java.util.concurrent.TimeUnit; -import java.util.regex.Matcher; -import java.util.regex.Pattern; - import org.apache.commons.lang.StringUtils; import org.apache.flume.Channel; import org.apache.flume.Context; @@ -48,16 +41,8 @@ import org.apache.flume.conf.Configurable; import org.apache.flume.instrumentation.SinkCounter; import org.apache.flume.sink.AbstractSink; -import org.elasticsearch.action.bulk.BulkRequestBuilder; -import org.elasticsearch.action.bulk.BulkResponse; -import org.elasticsearch.action.index.IndexRequestBuilder; -import org.elasticsearch.client.Client; -import org.elasticsearch.client.transport.TransportClient; -import org.elasticsearch.common.settings.ImmutableSettings; -import org.elasticsearch.common.settings.Settings; -import org.elasticsearch.common.transport.InetSocketTransportAddress; -import org.elasticsearch.node.Node; -import org.elasticsearch.node.NodeBuilder; +import org.apache.flume.sink.elasticsearch.client.ElasticSearchClient; +import org.apache.flume.sink.elasticsearch.client.ElasticSearchClientFactory; import org.slf4j.Logger; import org.slf4j.LoggerFactory; @@ -65,22 +50,35 @@ import com.google.common.base.Preconditions; import com.google.common.base.Throwables; +import java.util.concurrent.TimeUnit; +import java.util.regex.Matcher; +import java.util.regex.Pattern; +import static org.apache.flume.sink.elasticsearch.ElasticSearchSinkConstants.CLIENT_PREFIX; +import static org.apache.flume.sink.elasticsearch.ElasticSearchSinkConstants.CLIENT_TYPE; +import static org.apache.flume.sink.elasticsearch.ElasticSearchSinkConstants.DEFAULT_CLIENT_TYPE; +import static org.apache.flume.sink.elasticsearch.ElasticSearchSinkConstants.DEFAULT_INDEX_NAME_BUILDER_CLASS; +import static org.apache.flume.sink.elasticsearch.ElasticSearchSinkConstants.DEFAULT_SERIALIZER_CLASS; +import static org.apache.flume.sink.elasticsearch.ElasticSearchSinkConstants.INDEX_NAME_BUILDER; +import static org.apache.flume.sink.elasticsearch.ElasticSearchSinkConstants.INDEX_NAME_BUILDER_PREFIX; + +import static org.elasticsearch.common.xcontent.XContentFactory.jsonBuilder; + /** * A sink which reads events from a channel and writes them to ElasticSearch * based on the work done by https://github.com/Aconex/elasticflume.git.

    - * + * * This sink supports batch reading of events from the channel and writing them * to ElasticSearch.

    - * + * * Indexes will be rolled daily using the format 'indexname-YYYY-MM-dd' to allow * easier management of the index

    - * + * * This sink must be configured with with mandatory parameters detailed in - * {@link ElasticSearchSinkConstants}

    - * It is recommended as a secondary step the ElasticSearch indexes are optimized - * for the specified serializer. This is not handled by the sink but is - * typically done by deploying a config template alongside the ElasticSearch - * deploy

    + * {@link ElasticSearchSinkConstants}

    It is recommended as a secondary step + * the ElasticSearch indexes are optimized for the specified serializer. This is + * not handled by the sink but is typically done by deploying a config template + * alongside the ElasticSearch deploy

    + * * @see http * ://www.elasticsearch.org/guide/reference/api/admin-indices-templates. * html @@ -101,15 +99,19 @@ public class ElasticSearchSink extends AbstractSink implements Configurable { private String clusterName = DEFAULT_CLUSTER_NAME; private String indexName = DEFAULT_INDEX_NAME; private String indexType = DEFAULT_INDEX_TYPE; - private final Pattern pattern - = Pattern.compile(TTL_REGEX, Pattern.CASE_INSENSITIVE); + private String clientType = DEFAULT_CLIENT_TYPE; + private final Pattern pattern = Pattern.compile(TTL_REGEX, + Pattern.CASE_INSENSITIVE); private Matcher matcher = pattern.matcher(""); - private InetSocketTransportAddress[] serverAddresses; + private String[] serverAddresses = null; + + private ElasticSearchClient client = null; + private Context elasticSearchClientContext = null; - private Node node; - private Client client; private ElasticSearchIndexRequestBuilderFactory indexRequestFactory; + private ElasticSearchEventSerializer eventSerializer; + private IndexNameBuilder indexNameBuilder; private SinkCounter sinkCounter; /** @@ -122,12 +124,12 @@ public ElasticSearchSink() { /** * Create an {@link ElasticSearchSink}

    - * + * * @param isLocal * If true sink will be configured to only talk to an * ElasticSearch instance hosted in the same JVM, should always be * false is production - * + * */ @VisibleForTesting ElasticSearchSink(boolean isLocal) { @@ -135,7 +137,7 @@ public ElasticSearchSink() { } @VisibleForTesting - InetSocketTransportAddress[] getServerAddresses() { + String[] getServerAddresses() { return serverAddresses; } @@ -159,6 +161,16 @@ long getTTLMs() { return ttlMs; } + @VisibleForTesting + ElasticSearchEventSerializer getEventSerializer() { + return eventSerializer; + } + + @VisibleForTesting + IndexNameBuilder getIndexNameBuilder() { + return indexNameBuilder; + } + @Override public Status process() throws EventDeliveryException { logger.debug("processing..."); @@ -167,47 +179,33 @@ public Status process() throws EventDeliveryException { Transaction txn = channel.getTransaction(); try { txn.begin(); - BulkRequestBuilder bulkRequest = client.prepareBulk(); - for (int i = 0; i < batchSize; i++) { + int count; + for (count = 0; count < batchSize; ++count) { Event event = channel.take(); if (event == null) { break; } - - IndexRequestBuilder indexRequest = - indexRequestFactory.createIndexRequest( - client, indexName, indexType, event); - - if (ttlMs > 0) { - indexRequest.setTTL(ttlMs); - } - - bulkRequest.add(indexRequest); + client.addEvent(event, indexNameBuilder, indexType, ttlMs); } - int size = bulkRequest.numberOfActions(); - if (size <= 0) { + if (count <= 0) { sinkCounter.incrementBatchEmptyCount(); counterGroup.incrementAndGet("channel.underflow"); status = Status.BACKOFF; } else { - if (size < batchSize) { + if (count < batchSize) { sinkCounter.incrementBatchUnderflowCount(); status = Status.BACKOFF; } else { sinkCounter.incrementBatchCompleteCount(); } - sinkCounter.addToEventDrainAttemptCount(size); - - BulkResponse bulkResponse = bulkRequest.execute().actionGet(); - if (bulkResponse.hasFailures()) { - throw new EventDeliveryException(bulkResponse.buildFailureMessage()); - } + sinkCounter.addToEventDrainAttemptCount(count); + client.execute(); } txn.commit(); - sinkCounter.addToEventDrainSuccessCount(size); + sinkCounter.addToEventDrainSuccessCount(count); counterGroup.incrementAndGet("transaction.success"); } catch (Throwable ex) { try { @@ -238,22 +236,10 @@ public Status process() throws EventDeliveryException { @Override public void configure(Context context) { if (!isLocal) { - String[] hostNames = null; if (StringUtils.isNotBlank(context.getString(HOSTNAMES))) { - hostNames = context.getString(HOSTNAMES).split(","); - } - Preconditions.checkState(hostNames != null && hostNames.length > 0, - "Missing Param:" + HOSTNAMES); - - serverAddresses = new InetSocketTransportAddress[hostNames.length]; - for (int i = 0; i < hostNames.length; i++) { - String[] hostPort = hostNames[i].trim().split(":"); - String host = hostPort[0].trim(); - int port = hostPort.length == 2 ? Integer.parseInt(hostPort[1].trim()) - : DEFAULT_PORT; - serverAddresses[i] = new InetSocketTransportAddress(host, port); + serverAddresses = StringUtils.deleteWhitespace( + context.getString(HOSTNAMES)).split(","); } - Preconditions.checkState(serverAddresses != null && serverAddresses.length > 0, "Missing Param:" + HOSTNAMES); } @@ -280,7 +266,14 @@ public void configure(Context context) { + " must be greater than 0 or not set."); } - String serializerClazz = "org.apache.flume.sink.elasticsearch.ElasticSearchLogStashEventSerializer"; + if (StringUtils.isNotBlank(context.getString(CLIENT_TYPE))) { + clientType = context.getString(CLIENT_TYPE); + } + + elasticSearchClientContext = new Context(); + elasticSearchClientContext.putAll(context.getSubProperties(CLIENT_PREFIX)); + + String serializerClazz = DEFAULT_SERIALIZER_CLASS; if (StringUtils.isNotBlank(context.getString(SERIALIZER))) { serializerClazz = context.getString(SERIALIZER); } @@ -293,17 +286,18 @@ public void configure(Context context) { Class clazz = (Class) Class .forName(serializerClazz); Configurable serializer = clazz.newInstance(); + if (serializer instanceof ElasticSearchIndexRequestBuilderFactory) { - indexRequestFactory = (ElasticSearchIndexRequestBuilderFactory) serializer; - } else if (serializer instanceof ElasticSearchEventSerializer){ - indexRequestFactory = new EventSerializerIndexRequestBuilderFactory( - (ElasticSearchEventSerializer) serializer); + indexRequestFactory + = (ElasticSearchIndexRequestBuilderFactory) serializer; + indexRequestFactory.configure(serializerContext); + } else if (serializer instanceof ElasticSearchEventSerializer) { + eventSerializer = (ElasticSearchEventSerializer) serializer; + eventSerializer.configure(serializerContext); } else { - throw new IllegalArgumentException( - serializerClazz + " is neither an ElasticSearchEventSerializer" - + " nor an ElasticSearchIndexRequestBuilderFactory."); + throw new IllegalArgumentException(serializerClazz + + " is not an ElasticSearchEventSerializer"); } - indexRequestFactory.configure(serializerContext); } catch (Exception e) { logger.error("Could not instantiate event serializer.", e); Throwables.propagate(e); @@ -313,6 +307,32 @@ public void configure(Context context) { sinkCounter = new SinkCounter(getName()); } + String indexNameBuilderClass = DEFAULT_INDEX_NAME_BUILDER_CLASS; + if (StringUtils.isNotBlank(context.getString(INDEX_NAME_BUILDER))) { + indexNameBuilderClass = context.getString(INDEX_NAME_BUILDER); + } + + Context indexnameBuilderContext = new Context(); + serializerContext.putAll( + context.getSubProperties(INDEX_NAME_BUILDER_PREFIX)); + + try { + @SuppressWarnings("unchecked") + Class clazz + = (Class) Class + .forName(indexNameBuilderClass); + indexNameBuilder = clazz.newInstance(); + indexnameBuilderContext.put(INDEX_NAME, indexName); + indexNameBuilder.configure(indexnameBuilderContext); + } catch (Exception e) { + logger.error("Could not instantiate index name builder.", e); + Throwables.propagate(e); + } + + if (sinkCounter == null) { + sinkCounter = new SinkCounter(getName()); + } + Preconditions.checkState(StringUtils.isNotBlank(indexName), "Missing Param:" + INDEX_NAME); Preconditions.checkState(StringUtils.isNotBlank(indexType), @@ -325,13 +345,27 @@ public void configure(Context context) { @Override public void start() { + ElasticSearchClientFactory clientFactory = new ElasticSearchClientFactory(); + logger.info("ElasticSearch sink {} started"); sinkCounter.start(); try { - openConnection(); + if (isLocal) { + client = clientFactory.getLocalClient( + clientType, eventSerializer, indexRequestFactory); + } else { + client = clientFactory.getClient(clientType, serverAddresses, + clusterName, eventSerializer, indexRequestFactory); + client.configure(elasticSearchClientContext); + } + sinkCounter.incrementConnectionCreatedCount(); } catch (Exception ex) { + ex.printStackTrace(); sinkCounter.incrementConnectionFailedCount(); - closeConnection(); + if (client != null) { + client.close(); + sinkCounter.incrementConnectionClosedCount(); + } } super.start(); @@ -340,39 +374,31 @@ public void start() { @Override public void stop() { logger.info("ElasticSearch sink {} stopping"); - closeConnection(); - + if (client != null) { + client.close(); + } + sinkCounter.incrementConnectionClosedCount(); sinkCounter.stop(); super.stop(); } - private void openConnection() { - if (isLocal) { - logger.info("Using ElasticSearch AutoDiscovery mode"); - openLocalDiscoveryClient(); - } else { - logger.info("Using ElasticSearch hostnames: {} ", - Arrays.toString(serverAddresses)); - openClient(); - } - sinkCounter.incrementConnectionCreatedCount(); - } - /* - * Returns TTL value of ElasticSearch index in milliseconds - * when TTL specifier is "ms" / "s" / "m" / "h" / "d" / "w". - * In case of unknown specifier TTL is not set. When specifier - * is not provided it defaults to days in milliseconds where the number - * of days is parsed integer from TTL string provided by user. - *

    - * Elasticsearch supports ttl values being provided in the format: 1d / 1w / 1ms / 1s / 1h / 1m - * specify a time unit like d (days), m (minutes), h (hours), ms (milliseconds) or w (weeks), - * milliseconds is used as default unit. - * http://www.elasticsearch.org/guide/reference/mapping/ttl-field/. - * @param ttl TTL value provided by user in flume configuration file for the sink - * @return the ttl value in milliseconds - */ - private long parseTTL(String ttl){ + * Returns TTL value of ElasticSearch index in milliseconds when TTL specifier + * is "ms" / "s" / "m" / "h" / "d" / "w". In case of unknown specifier TTL is + * not set. When specifier is not provided it defaults to days in milliseconds + * where the number of days is parsed integer from TTL string provided by + * user.

    Elasticsearch supports ttl values being provided in the format: + * 1d / 1w / 1ms / 1s / 1h / 1m specify a time unit like d (days), m + * (minutes), h (hours), ms (milliseconds) or w (weeks), milliseconds is used + * as default unit. + * http://www.elasticsearch.org/guide/reference/mapping/ttl-field/. + * + * @param ttl TTL value provided by user in flume configuration file for the + * sink + * + * @return the ttl value in milliseconds + */ + private long parseTTL(String ttl) { matcher = matcher.reset(ttl); while (matcher.find()) { if (matcher.group(2).equals("ms")) { @@ -398,40 +424,4 @@ private long parseTTL(String ttl){ logger.info("TTL not provided. Skipping the TTL config by returning 0."); return 0; } - - /* - * FOR TESTING ONLY... - * - * Opens a local discovery node for talking to an elasticsearch server running - * in the same JVM - */ - private void openLocalDiscoveryClient() { - node = NodeBuilder.nodeBuilder().client(true).local(true).node(); - client = node.client(); - } - - private void openClient() { - Settings settings = ImmutableSettings.settingsBuilder() - .put("cluster.name", clusterName).build(); - - TransportClient transport = new TransportClient(settings); - for (InetSocketTransportAddress host : serverAddresses) { - transport.addTransportAddress(host); - } - client = transport; - } - - private void closeConnection() { - if (client != null) { - client.close(); - } - client = null; - - if (node != null) { - node.close(); - } - node = null; - - sinkCounter.incrementConnectionClosedCount(); - } } diff --git a/flume-ng-sinks/flume-ng-elasticsearch-sink/src/main/java/org/apache/flume/sink/elasticsearch/ElasticSearchSinkConstants.java b/flume-ng-sinks/flume-ng-elasticsearch-sink/src/main/java/org/apache/flume/sink/elasticsearch/ElasticSearchSinkConstants.java index dd0c59d1d9..da88def7ba 100644 --- a/flume-ng-sinks/flume-ng-elasticsearch-sink/src/main/java/org/apache/flume/sink/elasticsearch/ElasticSearchSinkConstants.java +++ b/flume-ng-sinks/flume-ng-elasticsearch-sink/src/main/java/org/apache/flume/sink/elasticsearch/ElasticSearchSinkConstants.java @@ -69,6 +69,30 @@ public class ElasticSearchSinkConstants { */ public static final String SERIALIZER_PREFIX = SERIALIZER + "."; + /** + * The fully qualified class name of the index name builder the sink + * should use to determine name of index where the event should be sent. + */ + public static final String INDEX_NAME_BUILDER = "indexNameBuilder"; + + /** + * The fully qualified class name of the index name builder the sink + * should use to determine name of index where the event should be sent. + */ + public static final String INDEX_NAME_BUILDER_PREFIX + = INDEX_NAME_BUILDER + "."; + + /** + * The client type used for sending bulks to ElasticSearch + */ + public static final String CLIENT_TYPE = "client"; + + /** + * The client prefix to extract the configuration that will be passed to + * elasticsearch client. + */ + public static final String CLIENT_PREFIX = CLIENT_TYPE + "."; + /** * DEFAULTS USED BY THE SINK */ @@ -78,5 +102,10 @@ public class ElasticSearchSinkConstants { public static final String DEFAULT_INDEX_NAME = "flume"; public static final String DEFAULT_INDEX_TYPE = "log"; public static final String DEFAULT_CLUSTER_NAME = "elasticsearch"; + public static final String DEFAULT_CLIENT_TYPE = "transport"; public static final String TTL_REGEX = "^(\\d+)(\\D*)"; + public static final String DEFAULT_SERIALIZER_CLASS = "org.apache.flume." + + "sink.elasticsearch.ElasticSearchLogStashEventSerializer"; + public static final String DEFAULT_INDEX_NAME_BUILDER_CLASS = + "org.apache.flume.sink.elasticsearch.TimeBasedIndexNameBuilder"; } diff --git a/flume-ng-sinks/flume-ng-elasticsearch-sink/src/main/java/org/apache/flume/sink/elasticsearch/IndexNameBuilder.java b/flume-ng-sinks/flume-ng-elasticsearch-sink/src/main/java/org/apache/flume/sink/elasticsearch/IndexNameBuilder.java new file mode 100644 index 0000000000..1dd4415469 --- /dev/null +++ b/flume-ng-sinks/flume-ng-elasticsearch-sink/src/main/java/org/apache/flume/sink/elasticsearch/IndexNameBuilder.java @@ -0,0 +1,42 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, + * software distributed under the License is distributed on an + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + * KIND, either express or implied. See the License for the + * specific language governing permissions and limitations + * under the License. + */ +package org.apache.flume.sink.elasticsearch; + +import org.apache.flume.Event; +import org.apache.flume.conf.Configurable; +import org.apache.flume.conf.ConfigurableComponent; + +public interface IndexNameBuilder extends Configurable, + ConfigurableComponent { + /** + * Gets the name of the index to use for an index request + * @param event + * Event which determines index name + * @return index name of the form 'indexPrefix-indexDynamicName' + */ + public String getIndexName(Event event); + + /** + * Gets the prefix of index to use for an index request. + * @param event + * Event which determines index name + * @return Index prefix name + */ + public String getIndexPrefix(Event event); +} diff --git a/flume-ng-sinks/flume-ng-elasticsearch-sink/src/main/java/org/apache/flume/sink/elasticsearch/SimpleIndexNameBuilder.java b/flume-ng-sinks/flume-ng-elasticsearch-sink/src/main/java/org/apache/flume/sink/elasticsearch/SimpleIndexNameBuilder.java new file mode 100644 index 0000000000..19079af32e --- /dev/null +++ b/flume-ng-sinks/flume-ng-elasticsearch-sink/src/main/java/org/apache/flume/sink/elasticsearch/SimpleIndexNameBuilder.java @@ -0,0 +1,45 @@ +/* + * Copyright 2014 Apache Software Foundation. + * + * Licensed under the Apache License, Version 2.0 (the "License"); + * you may not use this file except in compliance with the License. + * You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.flume.sink.elasticsearch; + +import org.apache.flume.Context; +import org.apache.flume.Event; +import org.apache.flume.conf.ComponentConfiguration; + +public class SimpleIndexNameBuilder implements IndexNameBuilder { + + private String indexName; + + @Override + public String getIndexName(Event event) { + return indexName; + } + + @Override + public String getIndexPrefix(Event event) { + return indexName; + } + + @Override + public void configure(Context context) { + indexName = context.getString(ElasticSearchSinkConstants.INDEX_NAME); + } + + @Override + public void configure(ComponentConfiguration conf) { + } +} diff --git a/flume-ng-sinks/flume-ng-elasticsearch-sink/src/main/java/org/apache/flume/sink/elasticsearch/TimeBasedIndexNameBuilder.java b/flume-ng-sinks/flume-ng-elasticsearch-sink/src/main/java/org/apache/flume/sink/elasticsearch/TimeBasedIndexNameBuilder.java new file mode 100644 index 0000000000..a8603a4ddb --- /dev/null +++ b/flume-ng-sinks/flume-ng-elasticsearch-sink/src/main/java/org/apache/flume/sink/elasticsearch/TimeBasedIndexNameBuilder.java @@ -0,0 +1,90 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, + * software distributed under the License is distributed on an + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + * KIND, either express or implied. See the License for the + * specific language governing permissions and limitations + * under the License. + */ +package org.apache.flume.sink.elasticsearch; + +import com.google.common.annotations.VisibleForTesting; +import org.apache.commons.lang.StringUtils; +import org.apache.commons.lang.time.FastDateFormat; +import org.apache.flume.Context; +import org.apache.flume.Event; +import org.apache.flume.conf.ComponentConfiguration; + +import java.util.TimeZone; + +/** + * Default index name builder. It prepares name of index using configured + * prefix and current timestamp. Default format of name is prefix-yyyy-MM-dd". + */ +public class TimeBasedIndexNameBuilder implements + IndexNameBuilder { + + public static final String DATE_FORMAT = "dateFormat"; + public static final String TIME_ZONE = "timeZone"; + + public static final String DEFAULT_DATE_FORMAT = "yyyy-MM-dd"; + public static final String DEFAULT_TIME_ZONE = "Etc/UTC"; + + private FastDateFormat fastDateFormat = FastDateFormat.getInstance("yyyy-MM-dd", + TimeZone.getTimeZone("Etc/UTC")); + + private String indexPrefix; + + @VisibleForTesting + FastDateFormat getFastDateFormat() { + return fastDateFormat; + } + + /** + * Gets the name of the index to use for an index request + * @param event + * Event for which the name of index has to be prepared + * @return index name of the form 'indexPrefix-formattedTimestamp' + */ + @Override + public String getIndexName(Event event) { + TimestampedEvent timestampedEvent = new TimestampedEvent(event); + long timestamp = timestampedEvent.getTimestamp(); + return new StringBuilder(indexPrefix).append('-') + .append(fastDateFormat.format(timestamp)).toString(); + } + + @Override + public String getIndexPrefix(Event event) { + return indexPrefix; + } + + @Override + public void configure(Context context) { + String dateFormatString = context.getString(DATE_FORMAT); + String timeZoneString = context.getString(TIME_ZONE); + if (StringUtils.isBlank(dateFormatString)) { + dateFormatString = DEFAULT_DATE_FORMAT; + } + if (StringUtils.isBlank(timeZoneString)) { + timeZoneString = DEFAULT_TIME_ZONE; + } + fastDateFormat = FastDateFormat.getInstance(dateFormatString, + TimeZone.getTimeZone(timeZoneString)); + indexPrefix = context.getString(ElasticSearchSinkConstants.INDEX_NAME); + } + + @Override + public void configure(ComponentConfiguration conf) { + } +} diff --git a/flume-ng-sinks/flume-ng-elasticsearch-sink/src/main/java/org/apache/flume/sink/elasticsearch/TimestampedEvent.java b/flume-ng-sinks/flume-ng-elasticsearch-sink/src/main/java/org/apache/flume/sink/elasticsearch/TimestampedEvent.java new file mode 100644 index 0000000000..c056839f60 --- /dev/null +++ b/flume-ng-sinks/flume-ng-elasticsearch-sink/src/main/java/org/apache/flume/sink/elasticsearch/TimestampedEvent.java @@ -0,0 +1,60 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, + * software distributed under the License is distributed on an + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + * KIND, either express or implied. See the License for the + * specific language governing permissions and limitations + * under the License. + */ +package org.apache.flume.sink.elasticsearch; + +import com.google.common.collect.Maps; +import org.apache.commons.lang.StringUtils; +import org.apache.flume.Event; +import org.apache.flume.event.SimpleEvent; +import org.joda.time.DateTimeUtils; + +import java.util.Map; + +/** + * {@link org.apache.flume.Event} implementation that has a timestamp. + * The timestamp is taken from (in order of precedence):

      + *
    1. The "timestamp" header of the base event, if present
    2. + *
    3. The "@timestamp" header of the base event, if present
    4. + *
    5. The current time in millis, otherwise
    6. + *
    + */ +final class TimestampedEvent extends SimpleEvent { + + private final long timestamp; + + TimestampedEvent(Event base) { + setBody(base.getBody()); + Map headers = Maps.newHashMap(base.getHeaders()); + String timestampString = headers.get("timestamp"); + if (StringUtils.isBlank(timestampString)) { + timestampString = headers.get("@timestamp"); + } + if (StringUtils.isBlank(timestampString)) { + this.timestamp = DateTimeUtils.currentTimeMillis(); + headers.put("timestamp", String.valueOf(timestamp )); + } else { + this.timestamp = Long.valueOf(timestampString); + } + setHeaders(headers); + } + + long getTimestamp() { + return timestamp; + } +} diff --git a/flume-ng-sinks/flume-ng-elasticsearch-sink/src/main/java/org/apache/flume/sink/elasticsearch/client/ElasticSearchClient.java b/flume-ng-sinks/flume-ng-elasticsearch-sink/src/main/java/org/apache/flume/sink/elasticsearch/client/ElasticSearchClient.java new file mode 100644 index 0000000000..655e00ae54 --- /dev/null +++ b/flume-ng-sinks/flume-ng-elasticsearch-sink/src/main/java/org/apache/flume/sink/elasticsearch/client/ElasticSearchClient.java @@ -0,0 +1,58 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, + * software distributed under the License is distributed on an + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + * KIND, either express or implied. See the License for the + * specific language governing permissions and limitations + * under the License. + */ +package org.apache.flume.sink.elasticsearch.client; + +import org.apache.flume.Event; +import org.apache.flume.conf.Configurable; +import org.apache.flume.sink.elasticsearch.IndexNameBuilder; + +/** + * Interface for an ElasticSearch client which is responsible for sending bulks + * of events to ElasticSearch. + */ +public interface ElasticSearchClient extends Configurable { + + /** + * Close connection to elastic search in client + */ + void close(); + + /** + * Add new event to the bulk + * + * @param event + * Flume Event + * @param indexNameBuilder + * Index name builder which generates name of index to feed + * @param indexType + * Name of type of document which will be sent to the elasticsearch cluster + * @param ttlMs + * Time to live expressed in milliseconds. Value <= 0 is ignored + * @throws Exception + */ + public void addEvent(Event event, IndexNameBuilder indexNameBuilder, + String indexType, long ttlMs) throws Exception; + + /** + * Sends bulk to the elasticsearch cluster + * + * @throws Exception + */ + void execute() throws Exception; +} diff --git a/flume-ng-sinks/flume-ng-elasticsearch-sink/src/main/java/org/apache/flume/sink/elasticsearch/client/ElasticSearchClientFactory.java b/flume-ng-sinks/flume-ng-elasticsearch-sink/src/main/java/org/apache/flume/sink/elasticsearch/client/ElasticSearchClientFactory.java new file mode 100644 index 0000000000..873157ae85 --- /dev/null +++ b/flume-ng-sinks/flume-ng-elasticsearch-sink/src/main/java/org/apache/flume/sink/elasticsearch/client/ElasticSearchClientFactory.java @@ -0,0 +1,76 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, + * software distributed under the License is distributed on an + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + * KIND, either express or implied. See the License for the + * specific language governing permissions and limitations + * under the License. + */ +package org.apache.flume.sink.elasticsearch.client; + +import org.apache.flume.sink.elasticsearch.ElasticSearchEventSerializer; +import org.apache.flume.sink.elasticsearch.ElasticSearchIndexRequestBuilderFactory; + +/** + * Internal ElasticSearch client factory. Responsible for creating instance + * of ElasticSearch clients. + */ +public class ElasticSearchClientFactory { + public static final String TransportClient = "transport"; + public static final String RestClient = "rest"; + + /** + * + * @param clientType + * String representation of client type + * @param hostNames + * Array of strings that represents hosntames with ports (hostname:port) + * @param clusterName + * Elasticsearch cluster name used only by Transport Client + * @param serializer + * Serializer of flume events to elasticsearch documents + * @return + */ + public ElasticSearchClient getClient(String clientType, String[] hostNames, + String clusterName, ElasticSearchEventSerializer serializer, + ElasticSearchIndexRequestBuilderFactory indexBuilder) throws NoSuchClientTypeException { + if (clientType.equalsIgnoreCase(TransportClient) && serializer != null) { + return new ElasticSearchTransportClient(hostNames, clusterName, serializer); + } else if (clientType.equalsIgnoreCase(TransportClient) && indexBuilder != null) { + return new ElasticSearchTransportClient(hostNames, clusterName, indexBuilder); + } else if (clientType.equalsIgnoreCase(RestClient) && serializer != null) { + return new ElasticSearchRestClient(hostNames, serializer); + } + throw new NoSuchClientTypeException(); + } + + /** + * Used for tests only. Creates local elasticsearch instance client. + * + * @param clientType Name of client to use + * @param serializer Serializer for the event + * @param indexBuilder Index builder factory + * + * @return Local elastic search instance client + */ + public ElasticSearchClient getLocalClient(String clientType, ElasticSearchEventSerializer serializer, + ElasticSearchIndexRequestBuilderFactory indexBuilder) throws NoSuchClientTypeException { + if (clientType.equalsIgnoreCase(TransportClient) && serializer != null) { + return new ElasticSearchTransportClient(serializer); + } else if (clientType.equalsIgnoreCase(TransportClient) && indexBuilder != null) { + return new ElasticSearchTransportClient(indexBuilder); + } else if (clientType.equalsIgnoreCase(RestClient)) { + } + throw new NoSuchClientTypeException(); + } +} diff --git a/flume-ng-sinks/flume-ng-elasticsearch-sink/src/main/java/org/apache/flume/sink/elasticsearch/client/ElasticSearchRestClient.java b/flume-ng-sinks/flume-ng-elasticsearch-sink/src/main/java/org/apache/flume/sink/elasticsearch/client/ElasticSearchRestClient.java new file mode 100644 index 0000000000..ff95e306c0 --- /dev/null +++ b/flume-ng-sinks/flume-ng-elasticsearch-sink/src/main/java/org/apache/flume/sink/elasticsearch/client/ElasticSearchRestClient.java @@ -0,0 +1,148 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, + * software distributed under the License is distributed on an + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + * KIND, either express or implied. See the License for the + * specific language governing permissions and limitations + * under the License. + */ +package org.apache.flume.sink.elasticsearch.client; + +import com.google.common.annotations.VisibleForTesting; +import com.google.gson.Gson; +import org.apache.flume.Context; +import org.apache.flume.Event; +import org.apache.flume.EventDeliveryException; +import org.apache.flume.sink.elasticsearch.ElasticSearchEventSerializer; +import org.apache.flume.sink.elasticsearch.IndexNameBuilder; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +import java.util.Arrays; +import java.util.HashMap; +import java.util.Map; +import org.apache.http.HttpResponse; +import org.apache.http.HttpStatus; +import org.apache.http.client.HttpClient; +import org.apache.http.client.methods.HttpPost; +import org.apache.http.entity.StringEntity; +import org.apache.http.impl.client.DefaultHttpClient; +import org.apache.http.util.EntityUtils; +import org.elasticsearch.common.bytes.BytesReference; + +/** + * Rest ElasticSearch client which is responsible for sending bulks of events to + * ElasticSearch using ElasticSearch HTTP API. This is configurable, so any + * config params required should be taken through this. + */ +public class ElasticSearchRestClient implements ElasticSearchClient { + + private static final String INDEX_OPERATION_NAME = "index"; + private static final String INDEX_PARAM = "_index"; + private static final String TYPE_PARAM = "_type"; + private static final String TTL_PARAM = "_ttl"; + private static final String BULK_ENDPOINT = "_bulk"; + + private static final Logger logger = LoggerFactory.getLogger(ElasticSearchRestClient.class); + + private final ElasticSearchEventSerializer serializer; + private final RoundRobinList serversList; + + private StringBuilder bulkBuilder; + private HttpClient httpClient; + + public ElasticSearchRestClient(String[] hostNames, + ElasticSearchEventSerializer serializer) { + + for (int i = 0; i < hostNames.length; ++i) { + if (!hostNames[i].contains("http://") && !hostNames[i].contains("https://")) { + hostNames[i] = "http://" + hostNames[i]; + } + } + this.serializer = serializer; + + serversList = new RoundRobinList(Arrays.asList(hostNames)); + httpClient = new DefaultHttpClient(); + bulkBuilder = new StringBuilder(); + } + + @VisibleForTesting + public ElasticSearchRestClient(String[] hostNames, + ElasticSearchEventSerializer serializer, HttpClient client) { + this(hostNames, serializer); + httpClient = client; + } + + @Override + public void configure(Context context) { + } + + @Override + public void close() { + } + + @Override + public void addEvent(Event event, IndexNameBuilder indexNameBuilder, String indexType, long ttlMs) throws Exception { + BytesReference content = serializer.getContentBuilder(event).bytes(); + Map> parameters = new HashMap>(); + Map indexParameters = new HashMap(); + indexParameters.put(INDEX_PARAM, indexNameBuilder.getIndexName(event)); + indexParameters.put(TYPE_PARAM, indexType); + if (ttlMs > 0) { + indexParameters.put(TTL_PARAM, Long.toString(ttlMs)); + } + parameters.put(INDEX_OPERATION_NAME, indexParameters); + + Gson gson = new Gson(); + synchronized(bulkBuilder) { + bulkBuilder.append(gson.toJson(parameters)); + bulkBuilder.append("\n"); + bulkBuilder.append(content.toBytesArray().toUtf8()); + bulkBuilder.append("\n"); + } + } + + @Override + public void execute() throws Exception { + int statusCode = 0, triesCount = 0; + HttpResponse response = null; + logger.info("Sending bulk request to elasticsearch cluster"); + + String entity; + synchronized (bulkBuilder) { + entity = bulkBuilder.toString(); + bulkBuilder = new StringBuilder(); + } + + while (statusCode != HttpStatus.SC_OK && triesCount < serversList.size()) { + triesCount++; + String host = serversList.get(); + String url = host + "/" + BULK_ENDPOINT; + HttpPost httpRequest = new HttpPost(url); + httpRequest.setEntity(new StringEntity(entity)); + response = httpClient.execute(httpRequest); + statusCode = response.getStatusLine().getStatusCode(); + logger.info("Status code from elasticsearch: " + statusCode); + if (response.getEntity() != null) + logger.debug("Status message from elasticsearch: " + EntityUtils.toString(response.getEntity(), "UTF-8")); + } + + if (statusCode != HttpStatus.SC_OK) { + if (response.getEntity() != null) { + throw new EventDeliveryException(EntityUtils.toString(response.getEntity(), "UTF-8")); + } else { + throw new EventDeliveryException("Elasticsearch status code was: " + statusCode); + } + } + } +} diff --git a/flume-ng-sinks/flume-ng-elasticsearch-sink/src/main/java/org/apache/flume/sink/elasticsearch/client/ElasticSearchTransportClient.java b/flume-ng-sinks/flume-ng-elasticsearch-sink/src/main/java/org/apache/flume/sink/elasticsearch/client/ElasticSearchTransportClient.java new file mode 100644 index 0000000000..e9ed0b4955 --- /dev/null +++ b/flume-ng-sinks/flume-ng-elasticsearch-sink/src/main/java/org/apache/flume/sink/elasticsearch/client/ElasticSearchTransportClient.java @@ -0,0 +1,231 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, + * software distributed under the License is distributed on an + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + * KIND, either express or implied. See the License for the + * specific language governing permissions and limitations + * under the License. + */ +package org.apache.flume.sink.elasticsearch.client; + +import com.google.common.annotations.VisibleForTesting; +import java.io.IOException; +import org.apache.flume.Context; +import org.apache.flume.Event; +import org.apache.flume.EventDeliveryException; +import org.apache.flume.sink.elasticsearch.ElasticSearchEventSerializer; +import org.apache.flume.sink.elasticsearch.IndexNameBuilder; +import org.elasticsearch.action.bulk.BulkRequestBuilder; +import org.elasticsearch.action.bulk.BulkResponse; +import org.elasticsearch.action.index.IndexRequestBuilder; +import org.elasticsearch.client.Client; +import org.elasticsearch.client.transport.TransportClient; +import org.elasticsearch.common.settings.ImmutableSettings; +import org.elasticsearch.common.settings.Settings; +import org.elasticsearch.common.transport.InetSocketTransportAddress; +import org.elasticsearch.node.Node; +import org.elasticsearch.node.NodeBuilder; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +import java.util.Arrays; +import org.apache.flume.sink.elasticsearch.ElasticSearchIndexRequestBuilderFactory; + +import static org.apache.flume.sink.elasticsearch.ElasticSearchSinkConstants.DEFAULT_PORT; + +public class ElasticSearchTransportClient implements ElasticSearchClient { + + public static final Logger logger = LoggerFactory + .getLogger(ElasticSearchTransportClient.class); + + private InetSocketTransportAddress[] serverAddresses; + private ElasticSearchEventSerializer serializer; + private ElasticSearchIndexRequestBuilderFactory indexRequestBuilderFactory; + private BulkRequestBuilder bulkRequestBuilder; + + private Client client; + + @VisibleForTesting + InetSocketTransportAddress[] getServerAddresses() { + return serverAddresses; + } + + @VisibleForTesting + void setBulkRequestBuilder(BulkRequestBuilder bulkRequestBuilder) { + this.bulkRequestBuilder = bulkRequestBuilder; + } + + /** + * Transport client for external cluster + * + * @param hostNames + * @param clusterName + * @param serializer + */ + public ElasticSearchTransportClient(String[] hostNames, String clusterName, + ElasticSearchEventSerializer serializer) { + configureHostnames(hostNames); + this.serializer = serializer; + openClient(clusterName); + } + + public ElasticSearchTransportClient(String[] hostNames, String clusterName, + ElasticSearchIndexRequestBuilderFactory indexBuilder) { + configureHostnames(hostNames); + this.indexRequestBuilderFactory = indexBuilder; + openClient(clusterName); + } + + /** + * Local transport client only for testing + * + * @param indexBuilderFactory + */ + public ElasticSearchTransportClient(ElasticSearchIndexRequestBuilderFactory indexBuilderFactory) { + this.indexRequestBuilderFactory = indexBuilderFactory; + openLocalDiscoveryClient(); + } + + /** + * Local transport client only for testing + * + * @param serializer + */ + public ElasticSearchTransportClient(ElasticSearchEventSerializer serializer) { + this.serializer = serializer; + openLocalDiscoveryClient(); + } + + /** + * Used for testing + * + * @param client + * ElasticSearch Client + * @param serializer + * Event Serializer + */ + public ElasticSearchTransportClient(Client client, + ElasticSearchEventSerializer serializer) { + this.client = client; + this.serializer = serializer; + } + + /** + * Used for testing + * + * @param client ElasticSearch Client + * @param serializer Event Serializer + */ + public ElasticSearchTransportClient(Client client, + ElasticSearchIndexRequestBuilderFactory requestBuilderFactory) throws IOException { + this.client = client; + requestBuilderFactory.createIndexRequest(client, null, null, null); + } + + private void configureHostnames(String[] hostNames) { + logger.warn(Arrays.toString(hostNames)); + serverAddresses = new InetSocketTransportAddress[hostNames.length]; + for (int i = 0; i < hostNames.length; i++) { + String[] hostPort = hostNames[i].trim().split(":"); + String host = hostPort[0].trim(); + int port = hostPort.length == 2 ? Integer.parseInt(hostPort[1].trim()) + : DEFAULT_PORT; + serverAddresses[i] = new InetSocketTransportAddress(host, port); + } + } + + @Override + public void close() { + if (client != null) { + client.close(); + } + client = null; + } + + @Override + public void addEvent(Event event, IndexNameBuilder indexNameBuilder, + String indexType, long ttlMs) throws Exception { + if (bulkRequestBuilder == null) { + bulkRequestBuilder = client.prepareBulk(); + } + + IndexRequestBuilder indexRequestBuilder = null; + if (indexRequestBuilderFactory == null) { + indexRequestBuilder = client + .prepareIndex(indexNameBuilder.getIndexName(event), indexType) + .setSource(serializer.getContentBuilder(event).bytes()); + } else { + indexRequestBuilder = indexRequestBuilderFactory.createIndexRequest( + client, indexNameBuilder.getIndexPrefix(event), indexType, event); + } + + if (ttlMs > 0) { + indexRequestBuilder.setTTL(ttlMs); + } + bulkRequestBuilder.add(indexRequestBuilder); + } + + @Override + public void execute() throws Exception { + try { + logger.info("Sending bulk to elasticsearch cluster"); + BulkResponse bulkResponse = bulkRequestBuilder.execute().actionGet(); + if (bulkResponse.hasFailures()) { + throw new EventDeliveryException(bulkResponse.buildFailureMessage()); + } + } finally { + bulkRequestBuilder = client.prepareBulk(); + } + } + + /** + * Open client to elaticsearch cluster + * + * @param clusterName + */ + private void openClient(String clusterName) { + logger.info("Using ElasticSearch hostnames: {} ", + Arrays.toString(serverAddresses)); + Settings settings = ImmutableSettings.settingsBuilder() + .put("cluster.name", clusterName).build(); + + TransportClient transportClient = new TransportClient(settings); + for (InetSocketTransportAddress host : serverAddresses) { + transportClient.addTransportAddress(host); + } + if (client != null) { + client.close(); + } + client = transportClient; + } + + /* + * FOR TESTING ONLY... + * + * Opens a local discovery node for talking to an elasticsearch server running + * in the same JVM + */ + private void openLocalDiscoveryClient() { + logger.info("Using ElasticSearch AutoDiscovery mode"); + Node node = NodeBuilder.nodeBuilder().client(true).local(true).node(); + if (client != null) { + client.close(); + } + client = node.client(); + } + + @Override + public void configure(Context context) { + //To change body of implemented methods use File | Settings | File Templates. + } +} diff --git a/flume-ng-sinks/flume-ng-elasticsearch-sink/src/main/java/org/apache/flume/sink/elasticsearch/client/NoSuchClientTypeException.java b/flume-ng-sinks/flume-ng-elasticsearch-sink/src/main/java/org/apache/flume/sink/elasticsearch/client/NoSuchClientTypeException.java new file mode 100644 index 0000000000..41fbe0d3c7 --- /dev/null +++ b/flume-ng-sinks/flume-ng-elasticsearch-sink/src/main/java/org/apache/flume/sink/elasticsearch/client/NoSuchClientTypeException.java @@ -0,0 +1,23 @@ +/* + * Copyright 2014 Apache Software Foundation. + * + * Licensed under the Apache License, Version 2.0 (the "License"); + * you may not use this file except in compliance with the License. + * You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.flume.sink.elasticsearch.client; + +/** + * Exception class + */ +class NoSuchClientTypeException extends Exception { +} diff --git a/flume-ng-sinks/flume-ng-elasticsearch-sink/src/main/java/org/apache/flume/sink/elasticsearch/client/RoundRobinList.java b/flume-ng-sinks/flume-ng-elasticsearch-sink/src/main/java/org/apache/flume/sink/elasticsearch/client/RoundRobinList.java new file mode 100644 index 0000000000..dbad8d841d --- /dev/null +++ b/flume-ng-sinks/flume-ng-elasticsearch-sink/src/main/java/org/apache/flume/sink/elasticsearch/client/RoundRobinList.java @@ -0,0 +1,44 @@ +package org.apache.flume.sink.elasticsearch.client; + +import java.util.Collection; +import java.util.Iterator; + +/* + * Copyright 2014 Apache Software Foundation. + * + * Licensed under the Apache License, Version 2.0 (the "License"); + * you may not use this file except in compliance with the License. + * You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +public class RoundRobinList { + + private Iterator iterator; + private final Collection elements; + + public RoundRobinList(Collection elements) { + this.elements = elements; + iterator = this.elements.iterator(); + } + + synchronized public T get() { + if (iterator.hasNext()) { + return iterator.next(); + } else { + iterator = elements.iterator(); + return iterator.next(); + } + } + + public int size() { + return elements.size(); + } +} diff --git a/flume-ng-sinks/flume-ng-elasticsearch-sink/src/test/java/org/apache/flume/sink/elasticsearch/TestElasticSearchDynamicSerializer.java b/flume-ng-sinks/flume-ng-elasticsearch-sink/src/test/java/org/apache/flume/sink/elasticsearch/TestElasticSearchDynamicSerializer.java index 43a4b121fb..d4e465477c 100644 --- a/flume-ng-sinks/flume-ng-elasticsearch-sink/src/test/java/org/apache/flume/sink/elasticsearch/TestElasticSearchDynamicSerializer.java +++ b/flume-ng-sinks/flume-ng-elasticsearch-sink/src/test/java/org/apache/flume/sink/elasticsearch/TestElasticSearchDynamicSerializer.java @@ -18,12 +18,6 @@ */ package org.apache.flume.sink.elasticsearch; -import static org.apache.flume.sink.elasticsearch.ElasticSearchEventSerializer.charset; -import static org.elasticsearch.common.xcontent.XContentFactory.jsonBuilder; -import static org.junit.Assert.assertEquals; - -import java.util.Map; - import org.apache.flume.Context; import org.apache.flume.Event; import org.apache.flume.event.EventBuilder; @@ -31,6 +25,12 @@ import org.elasticsearch.common.xcontent.XContentBuilder; import org.junit.Test; +import java.util.Map; + +import static org.apache.flume.sink.elasticsearch.ElasticSearchEventSerializer.charset; +import static org.elasticsearch.common.xcontent.XContentFactory.jsonBuilder; +import static org.junit.Assert.assertEquals; + public class TestElasticSearchDynamicSerializer { @Test diff --git a/flume-ng-sinks/flume-ng-elasticsearch-sink/src/test/java/org/apache/flume/sink/elasticsearch/TestElasticSearchIndexRequestBuilderFactory.java b/flume-ng-sinks/flume-ng-elasticsearch-sink/src/test/java/org/apache/flume/sink/elasticsearch/TestElasticSearchIndexRequestBuilderFactory.java index 1e4e119536..807a9c7513 100644 --- a/flume-ng-sinks/flume-ng-elasticsearch-sink/src/test/java/org/apache/flume/sink/elasticsearch/TestElasticSearchIndexRequestBuilderFactory.java +++ b/flume-ng-sinks/flume-ng-elasticsearch-sink/src/test/java/org/apache/flume/sink/elasticsearch/TestElasticSearchIndexRequestBuilderFactory.java @@ -168,27 +168,27 @@ public void shouldConfigureEventSerializer() throws Exception { assertTrue(serializer.configuredWithComponentConfiguration); } -} - -class FakeEventSerializer implements ElasticSearchEventSerializer { - - static final byte[] FAKE_BYTES = new byte[] {9,8,7,6}; - boolean configuredWithContext, configuredWithComponentConfiguration; - - @Override - public BytesStream getContentBuilder(Event event) throws IOException { - FastByteArrayOutputStream fbaos = new FastByteArrayOutputStream(4); - fbaos.write(FAKE_BYTES); - return fbaos; + static class FakeEventSerializer implements ElasticSearchEventSerializer { + + static final byte[] FAKE_BYTES = new byte[]{9, 8, 7, 6}; + boolean configuredWithContext, configuredWithComponentConfiguration; + + @Override + public BytesStream getContentBuilder(Event event) throws IOException { + FastByteArrayOutputStream fbaos = new FastByteArrayOutputStream(4); + fbaos.write(FAKE_BYTES); + return fbaos; + } + + @Override + public void configure(Context arg0) { + configuredWithContext = true; + } + + @Override + public void configure(ComponentConfiguration arg0) { + configuredWithComponentConfiguration = true; + } } - @Override - public void configure(Context arg0) { - configuredWithContext = true; - } - - @Override - public void configure(ComponentConfiguration arg0) { - configuredWithComponentConfiguration = true; - } } diff --git a/flume-ng-sinks/flume-ng-elasticsearch-sink/src/test/java/org/apache/flume/sink/elasticsearch/TestElasticSearchLogStashEventSerializer.java b/flume-ng-sinks/flume-ng-elasticsearch-sink/src/test/java/org/apache/flume/sink/elasticsearch/TestElasticSearchLogStashEventSerializer.java index 9dff4b0d08..d2c9543cee 100644 --- a/flume-ng-sinks/flume-ng-elasticsearch-sink/src/test/java/org/apache/flume/sink/elasticsearch/TestElasticSearchLogStashEventSerializer.java +++ b/flume-ng-sinks/flume-ng-elasticsearch-sink/src/test/java/org/apache/flume/sink/elasticsearch/TestElasticSearchLogStashEventSerializer.java @@ -18,13 +18,6 @@ */ package org.apache.flume.sink.elasticsearch; -import static org.apache.flume.sink.elasticsearch.ElasticSearchEventSerializer.charset; -import static org.elasticsearch.common.xcontent.XContentFactory.jsonBuilder; -import static org.junit.Assert.assertEquals; - -import java.util.Date; -import java.util.Map; - import org.apache.flume.Context; import org.apache.flume.Event; import org.apache.flume.event.EventBuilder; @@ -32,6 +25,13 @@ import org.elasticsearch.common.xcontent.XContentBuilder; import org.junit.Test; +import java.util.Date; +import java.util.Map; + +import static org.apache.flume.sink.elasticsearch.ElasticSearchEventSerializer.charset; +import static org.elasticsearch.common.xcontent.XContentFactory.jsonBuilder; +import static org.junit.Assert.assertEquals; + public class TestElasticSearchLogStashEventSerializer { @Test diff --git a/flume-ng-sinks/flume-ng-elasticsearch-sink/src/test/java/org/apache/flume/sink/elasticsearch/TestElasticSearchSink.java b/flume-ng-sinks/flume-ng-elasticsearch-sink/src/test/java/org/apache/flume/sink/elasticsearch/TestElasticSearchSink.java index 71789e80b2..15546c1fa8 100644 --- a/flume-ng-sinks/flume-ng-elasticsearch-sink/src/test/java/org/apache/flume/sink/elasticsearch/TestElasticSearchSink.java +++ b/flume-ng-sinks/flume-ng-elasticsearch-sink/src/test/java/org/apache/flume/sink/elasticsearch/TestElasticSearchSink.java @@ -20,7 +20,6 @@ import static org.apache.flume.sink.elasticsearch.ElasticSearchSinkConstants.BATCH_SIZE; import static org.apache.flume.sink.elasticsearch.ElasticSearchSinkConstants.CLUSTER_NAME; -import static org.apache.flume.sink.elasticsearch.ElasticSearchSinkConstants.DEFAULT_PORT; import static org.apache.flume.sink.elasticsearch.ElasticSearchSinkConstants.HOSTNAMES; import static org.apache.flume.sink.elasticsearch.ElasticSearchSinkConstants.INDEX_NAME; import static org.apache.flume.sink.elasticsearch.ElasticSearchSinkConstants.INDEX_TYPE; @@ -29,14 +28,15 @@ import static org.junit.Assert.assertArrayEquals; import static org.junit.Assert.assertEquals; import static org.junit.Assert.assertTrue; +import static org.junit.Assert.assertNull; import java.io.IOException; import java.util.Map; import java.util.HashMap; import java.util.TimeZone; import java.util.concurrent.TimeUnit; - import org.apache.commons.lang.time.FastDateFormat; + import org.apache.flume.Channel; import org.apache.flume.Context; import org.apache.flume.Event; @@ -49,7 +49,8 @@ import org.elasticsearch.action.index.IndexRequestBuilder; import org.elasticsearch.client.Requests; import org.elasticsearch.common.UUID; -import org.elasticsearch.common.transport.InetSocketTransportAddress; +import org.elasticsearch.common.io.BytesStream; +import org.elasticsearch.common.io.FastByteArrayOutputStream; import org.junit.After; import org.junit.Before; import org.junit.Test; @@ -169,8 +170,7 @@ public void shouldParseConfiguration() { fixture = new ElasticSearchSink(); fixture.configure(new Context(parameters)); - InetSocketTransportAddress[] expected = { new InetSocketTransportAddress( - "10.5.5.27", DEFAULT_PORT) }; + String[] expected = { "10.5.5.27" }; assertEquals("testing-cluster-name", fixture.getClusterName()); assertEquals("testing-index-name", fixture.getIndexName()); @@ -189,8 +189,7 @@ public void shouldParseConfigurationUsingDefaults() { fixture = new ElasticSearchSink(); fixture.configure(new Context(parameters)); - InetSocketTransportAddress[] expected = { new InetSocketTransportAddress( - "10.5.5.27", DEFAULT_PORT) }; + String[] expected = { "10.5.5.27" }; assertEquals(DEFAULT_INDEX_NAME, fixture.getIndexName()); assertEquals(DEFAULT_INDEX_TYPE, fixture.getIndexType()); @@ -205,10 +204,7 @@ public void shouldParseMultipleHostUsingDefaultPorts() { fixture = new ElasticSearchSink(); fixture.configure(new Context(parameters)); - InetSocketTransportAddress[] expected = { - new InetSocketTransportAddress("10.5.5.27", DEFAULT_PORT), - new InetSocketTransportAddress("10.5.5.28", DEFAULT_PORT), - new InetSocketTransportAddress("10.5.5.29", DEFAULT_PORT) }; + String[] expected = { "10.5.5.27", "10.5.5.28", "10.5.5.29" }; assertArrayEquals(expected, fixture.getServerAddresses()); } @@ -220,10 +216,7 @@ public void shouldParseMultipleHostWithWhitespacesUsingDefaultPorts() { fixture = new ElasticSearchSink(); fixture.configure(new Context(parameters)); - InetSocketTransportAddress[] expected = { - new InetSocketTransportAddress("10.5.5.27", DEFAULT_PORT), - new InetSocketTransportAddress("10.5.5.28", DEFAULT_PORT), - new InetSocketTransportAddress("10.5.5.29", DEFAULT_PORT) }; + String[] expected = { "10.5.5.27", "10.5.5.28", "10.5.5.29" }; assertArrayEquals(expected, fixture.getServerAddresses()); } @@ -235,25 +228,20 @@ public void shouldParseMultipleHostAndPorts() { fixture = new ElasticSearchSink(); fixture.configure(new Context(parameters)); - InetSocketTransportAddress[] expected = { - new InetSocketTransportAddress("10.5.5.27", 9300), - new InetSocketTransportAddress("10.5.5.28", 9301), - new InetSocketTransportAddress("10.5.5.29", 9302) }; + String[] expected = { "10.5.5.27:9300", "10.5.5.28:9301", "10.5.5.29:9302" }; assertArrayEquals(expected, fixture.getServerAddresses()); } @Test public void shouldParseMultipleHostAndPortsWithWhitespaces() { - parameters.put(HOSTNAMES, " 10.5.5.27 : 9300 , 10.5.5.28 : 9301 , 10.5.5.29 : 9302 "); + parameters.put(HOSTNAMES, + " 10.5.5.27 : 9300 , 10.5.5.28 : 9301 , 10.5.5.29 : 9302 "); fixture = new ElasticSearchSink(); fixture.configure(new Context(parameters)); - InetSocketTransportAddress[] expected = { - new InetSocketTransportAddress("10.5.5.27", 9300), - new InetSocketTransportAddress("10.5.5.28", 9301), - new InetSocketTransportAddress("10.5.5.29", 9302) }; + String[] expected = { "10.5.5.27:9300", "10.5.5.28:9301", "10.5.5.29:9302" }; assertArrayEquals(expected, fixture.getServerAddresses()); } @@ -261,11 +249,10 @@ public void shouldParseMultipleHostAndPortsWithWhitespaces() { @Test public void shouldAllowCustomElasticSearchIndexRequestBuilderFactory() throws Exception { - parameters.put(SERIALIZER, CustomElasticSearchIndexRequestBuilderFactory.class.getName()); - Configurables.configure(fixture, new Context(parameters)); + fixture.configure(new Context(parameters)); Channel channel = bindAndStartChannel(fixture); Transaction tx = channel.getTransaction(); @@ -279,7 +266,7 @@ public void shouldAllowCustomElasticSearchIndexRequestBuilderFactory() fixture.process(); fixture.stop(); - assertEquals(fixture.getIndexName()+"-05_17_36_789", + assertEquals(fixture.getIndexName() + "-05_17_36_789", CustomElasticSearchIndexRequestBuilderFactory.actualIndexName); assertEquals(fixture.getIndexType(), CustomElasticSearchIndexRequestBuilderFactory.actualIndexType); @@ -289,7 +276,7 @@ public void shouldAllowCustomElasticSearchIndexRequestBuilderFactory() } @Test - public void shouldParseFullyQualifiedTTLs(){ + public void shouldParseFullyQualifiedTTLs() { Map testTTLMap = new HashMap(); testTTLMap.put("1ms", Long.valueOf(1)); testTTLMap.put("1s", Long.valueOf(1000)); @@ -297,7 +284,7 @@ public void shouldParseFullyQualifiedTTLs(){ testTTLMap.put("1h", Long.valueOf(3600000)); testTTLMap.put("1d", Long.valueOf(86400000)); testTTLMap.put("1w", Long.valueOf(604800000)); - testTTLMap.put("1", Long.valueOf(86400000)); + testTTLMap.put("1", Long.valueOf(86400000)); parameters.put(HOSTNAMES, "10.5.5.27"); parameters.put(CLUSTER_NAME, "testing-cluster-name"); @@ -309,13 +296,10 @@ public void shouldParseFullyQualifiedTTLs(){ fixture = new ElasticSearchSink(); fixture.configure(new Context(parameters)); - InetSocketTransportAddress[] expected = {new InetSocketTransportAddress( - "10.5.5.27", DEFAULT_PORT)}; - + String[] expected = { "10.5.5.27" }; assertEquals("testing-cluster-name", fixture.getClusterName()); assertEquals("testing-index-name", fixture.getIndexName()); assertEquals("testing-index-type", fixture.getIndexType()); - System.out.println("TTL MS" + Long.toString(testTTLMap.get(ttl))); assertEquals((long) testTTLMap.get(ttl), fixture.getTTLMs()); assertArrayEquals(expected, fixture.getServerAddresses()); @@ -374,10 +358,84 @@ public void shouldFailToConfigureWithInvalidSerializerClass() } } + @Test + public void shouldUseSpecifiedSerializer() throws Exception { + Context context = new Context(); + context.put(SERIALIZER, + "org.apache.flume.sink.elasticsearch.FakeEventSerializer"); + + assertNull(fixture.getEventSerializer()); + fixture.configure(context); + assertTrue(fixture.getEventSerializer() instanceof FakeEventSerializer); + } + + @Test + public void shouldUseSpecifiedIndexNameBuilder() throws Exception { + Context context = new Context(); + context.put(ElasticSearchSinkConstants.INDEX_NAME_BUILDER, + "org.apache.flume.sink.elasticsearch.FakeIndexNameBuilder"); + + assertNull(fixture.getIndexNameBuilder()); + fixture.configure(context); + assertTrue(fixture.getIndexNameBuilder() instanceof FakeIndexNameBuilder); + } + public static class FakeConfigurable implements Configurable { @Override public void configure(Context arg0) { - // no-op + // no-op } } } + +/** + * Internal class. Fake event serializer used for tests + */ +class FakeEventSerializer implements ElasticSearchEventSerializer { + + static final byte[] FAKE_BYTES = new byte[] { 9, 8, 7, 6 }; + boolean configuredWithContext, configuredWithComponentConfiguration; + + @Override + public BytesStream getContentBuilder(Event event) throws IOException { + FastByteArrayOutputStream fbaos = new FastByteArrayOutputStream(4); + fbaos.write(FAKE_BYTES); + return fbaos; + } + + @Override + public void configure(Context arg0) { + configuredWithContext = true; + } + + @Override + public void configure(ComponentConfiguration arg0) { + configuredWithComponentConfiguration = true; + } +} + +/** + * Internal class. Fake index name builder used only for tests. + */ +class FakeIndexNameBuilder implements IndexNameBuilder { + + static final String INDEX_NAME = "index_name"; + + @Override + public String getIndexName(Event event) { + return INDEX_NAME; + } + + @Override + public String getIndexPrefix(Event event) { + return INDEX_NAME; + } + + @Override + public void configure(Context context) { + } + + @Override + public void configure(ComponentConfiguration conf) { + } +} diff --git a/flume-ng-sinks/flume-ng-elasticsearch-sink/src/test/java/org/apache/flume/sink/elasticsearch/TimeBasedIndexNameBuilderTest.java b/flume-ng-sinks/flume-ng-elasticsearch-sink/src/test/java/org/apache/flume/sink/elasticsearch/TimeBasedIndexNameBuilderTest.java new file mode 100644 index 0000000000..678342a182 --- /dev/null +++ b/flume-ng-sinks/flume-ng-elasticsearch-sink/src/test/java/org/apache/flume/sink/elasticsearch/TimeBasedIndexNameBuilderTest.java @@ -0,0 +1,60 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, + * software distributed under the License is distributed on an + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + * KIND, either express or implied. See the License for the + * specific language governing permissions and limitations + * under the License. + */ +package org.apache.flume.sink.elasticsearch; + +import org.apache.flume.Context; +import org.apache.flume.Event; +import org.apache.flume.event.SimpleEvent; +import org.junit.Before; +import org.junit.Test; + +import java.util.HashMap; +import java.util.Map; + +import static org.junit.Assert.assertEquals; + +public class TimeBasedIndexNameBuilderTest { + + private TimeBasedIndexNameBuilder indexNameBuilder; + + @Before + public void setUp() throws Exception { + Context context = new Context(); + context.put(ElasticSearchSinkConstants.INDEX_NAME, "prefix"); + indexNameBuilder = new TimeBasedIndexNameBuilder(); + indexNameBuilder.configure(context); + } + + @Test + public void shouldUseUtcAsBasisForDateFormat() { + assertEquals("Coordinated Universal Time", + indexNameBuilder.getFastDateFormat().getTimeZone().getDisplayName()); + } + + @Test + public void indexNameShouldBePrefixDashFormattedTimestamp() { + long time = 987654321L; + Event event = new SimpleEvent(); + Map headers = new HashMap(); + headers.put("timestamp", Long.toString(time)); + event.setHeaders(headers); + assertEquals("prefix-" + indexNameBuilder.getFastDateFormat().format(time), + indexNameBuilder.getIndexName(event)); + } +} diff --git a/flume-ng-sinks/flume-ng-elasticsearch-sink/src/test/java/org/apache/flume/sink/elasticsearch/TimestampedEventTest.java b/flume-ng-sinks/flume-ng-elasticsearch-sink/src/test/java/org/apache/flume/sink/elasticsearch/TimestampedEventTest.java new file mode 100644 index 0000000000..bef2ac6452 --- /dev/null +++ b/flume-ng-sinks/flume-ng-elasticsearch-sink/src/test/java/org/apache/flume/sink/elasticsearch/TimestampedEventTest.java @@ -0,0 +1,88 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, + * software distributed under the License is distributed on an + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + * KIND, either express or implied. See the License for the + * specific language governing permissions and limitations + * under the License. + */ +package org.apache.flume.sink.elasticsearch; + +import com.google.common.collect.Maps; +import org.apache.flume.event.SimpleEvent; +import org.joda.time.DateTimeUtils; +import org.junit.Before; +import org.junit.Test; + +import java.util.Map; + +import static org.junit.Assert.assertArrayEquals; +import static org.junit.Assert.assertEquals; +import static org.junit.Assert.assertNull; + +public class TimestampedEventTest { + static final long FIXED_TIME_MILLIS = 123456789L; + + @Before + public void setFixedJodaTime() { + DateTimeUtils.setCurrentMillisFixed(FIXED_TIME_MILLIS); + } + + @Test + public void shouldEnsureTimestampHeaderPresentInTimestampedEvent() { + SimpleEvent base = new SimpleEvent(); + + TimestampedEvent timestampedEvent = new TimestampedEvent(base); + assertEquals(FIXED_TIME_MILLIS, timestampedEvent.getTimestamp()); + assertEquals(String.valueOf(FIXED_TIME_MILLIS), + timestampedEvent.getHeaders().get("timestamp")); + } + + @Test + public void shouldUseExistingTimestampHeaderInTimestampedEvent() { + SimpleEvent base = new SimpleEvent(); + Map headersWithTimestamp = Maps.newHashMap(); + headersWithTimestamp.put("timestamp", "-321"); + base.setHeaders(headersWithTimestamp ); + + TimestampedEvent timestampedEvent = new TimestampedEvent(base); + assertEquals(-321L, timestampedEvent.getTimestamp()); + assertEquals("-321", timestampedEvent.getHeaders().get("timestamp")); + } + + @Test + public void shouldUseExistingAtTimestampHeaderInTimestampedEvent() { + SimpleEvent base = new SimpleEvent(); + Map headersWithTimestamp = Maps.newHashMap(); + headersWithTimestamp.put("@timestamp", "-999"); + base.setHeaders(headersWithTimestamp ); + + TimestampedEvent timestampedEvent = new TimestampedEvent(base); + assertEquals(-999L, timestampedEvent.getTimestamp()); + assertEquals("-999", timestampedEvent.getHeaders().get("@timestamp")); + assertNull(timestampedEvent.getHeaders().get("timestamp")); + } + + @Test + public void shouldPreserveBodyAndNonTimestampHeadersInTimestampedEvent() { + SimpleEvent base = new SimpleEvent(); + base.setBody(new byte[] {1,2,3,4}); + Map headersWithTimestamp = Maps.newHashMap(); + headersWithTimestamp.put("foo", "bar"); + base.setHeaders(headersWithTimestamp ); + + TimestampedEvent timestampedEvent = new TimestampedEvent(base); + assertEquals("bar", timestampedEvent.getHeaders().get("foo")); + assertArrayEquals(base.getBody(), timestampedEvent.getBody()); + } +} diff --git a/flume-ng-sinks/flume-ng-elasticsearch-sink/src/test/java/org/apache/flume/sink/elasticsearch/client/RoundRobinListTest.java b/flume-ng-sinks/flume-ng-elasticsearch-sink/src/test/java/org/apache/flume/sink/elasticsearch/client/RoundRobinListTest.java new file mode 100644 index 0000000000..38e7399a3b --- /dev/null +++ b/flume-ng-sinks/flume-ng-elasticsearch-sink/src/test/java/org/apache/flume/sink/elasticsearch/client/RoundRobinListTest.java @@ -0,0 +1,41 @@ +/* + * Copyright 2014 Apache Software Foundation. + * + * Licensed under the Apache License, Version 2.0 (the "License"); + * you may not use this file except in compliance with the License. + * You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.flume.sink.elasticsearch.client; + +import java.util.Arrays; +import static org.junit.Assert.assertEquals; +import org.junit.Before; +import org.junit.Test; + +public class RoundRobinListTest { + + private RoundRobinList fixture; + + @Before + public void setUp() { + fixture = new RoundRobinList(Arrays.asList("test1", "test2")); + } + + @Test + public void shouldReturnNextElement() { + assertEquals("test1", fixture.get()); + assertEquals("test2", fixture.get()); + assertEquals("test1", fixture.get()); + assertEquals("test2", fixture.get()); + assertEquals("test1", fixture.get()); + } +} diff --git a/flume-ng-sinks/flume-ng-elasticsearch-sink/src/test/java/org/apache/flume/sink/elasticsearch/client/TestElasticSearchClientFactory.java b/flume-ng-sinks/flume-ng-elasticsearch-sink/src/test/java/org/apache/flume/sink/elasticsearch/client/TestElasticSearchClientFactory.java new file mode 100644 index 0000000000..4b70b65740 --- /dev/null +++ b/flume-ng-sinks/flume-ng-elasticsearch-sink/src/test/java/org/apache/flume/sink/elasticsearch/client/TestElasticSearchClientFactory.java @@ -0,0 +1,64 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, + * software distributed under the License is distributed on an + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + * KIND, either express or implied. See the License for the + * specific language governing permissions and limitations + * under the License. + */ +package org.apache.flume.sink.elasticsearch.client; + +import org.apache.flume.sink.elasticsearch.ElasticSearchEventSerializer; +import org.junit.Before; +import org.junit.Test; + +import static org.hamcrest.core.IsInstanceOf.instanceOf; +import static org.junit.Assert.assertThat; +import org.mockito.Mock; +import static org.mockito.MockitoAnnotations.initMocks; + +public class TestElasticSearchClientFactory { + + ElasticSearchClientFactory factory; + + @Mock + ElasticSearchEventSerializer serializer; + + @Before + public void setUp() { + initMocks(this); + factory = new ElasticSearchClientFactory(); + } + + @Test + public void shouldReturnTransportClient() throws Exception { + String[] hostNames = { "127.0.0.1" }; + Object o = factory.getClient(ElasticSearchClientFactory.TransportClient, + hostNames, "test", serializer, null); + assertThat(o, instanceOf(ElasticSearchTransportClient.class)); + } + + @Test + public void shouldReturnRestClient() throws NoSuchClientTypeException { + String[] hostNames = { "127.0.0.1" }; + Object o = factory.getClient(ElasticSearchClientFactory.RestClient, + hostNames, "test", serializer, null); + assertThat(o, instanceOf(ElasticSearchRestClient.class)); + } + + @Test(expected=NoSuchClientTypeException.class) + public void shouldThrowNoSuchClientTypeException() throws NoSuchClientTypeException { + String[] hostNames = {"127.0.0.1"}; + factory.getClient("not_existing_client", hostNames, "test", null, null); + } +} diff --git a/flume-ng-sinks/flume-ng-elasticsearch-sink/src/test/java/org/apache/flume/sink/elasticsearch/client/TestElasticSearchRestClient.java b/flume-ng-sinks/flume-ng-elasticsearch-sink/src/test/java/org/apache/flume/sink/elasticsearch/client/TestElasticSearchRestClient.java new file mode 100644 index 0000000000..b7d8822696 --- /dev/null +++ b/flume-ng-sinks/flume-ng-elasticsearch-sink/src/test/java/org/apache/flume/sink/elasticsearch/client/TestElasticSearchRestClient.java @@ -0,0 +1,158 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, + * software distributed under the License is distributed on an + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + * KIND, either express or implied. See the License for the + * specific language governing permissions and limitations + * under the License. + */ +package org.apache.flume.sink.elasticsearch.client; + +import org.apache.flume.Event; +import org.apache.flume.EventDeliveryException; +import org.apache.flume.sink.elasticsearch.ElasticSearchEventSerializer; +import org.apache.flume.sink.elasticsearch.IndexNameBuilder; +import org.elasticsearch.common.bytes.BytesReference; +import org.elasticsearch.common.io.BytesStream; +import org.junit.Before; +import org.junit.Test; +import org.mockito.ArgumentCaptor; +import org.mockito.Mock; + +import java.io.IOException; +import java.util.List; + +import static junit.framework.Assert.assertEquals; +import org.apache.http.HttpEntity; +import org.apache.http.HttpResponse; +import org.apache.http.HttpStatus; +import org.apache.http.StatusLine; +import org.apache.http.client.HttpClient; +import org.apache.http.client.methods.HttpPost; +import org.apache.http.client.methods.HttpUriRequest; +import org.apache.http.util.EntityUtils; +import org.elasticsearch.common.bytes.BytesArray; +import static org.mockito.Mockito.*; +import static org.mockito.MockitoAnnotations.initMocks; + +public class TestElasticSearchRestClient { + + private ElasticSearchRestClient fixture; + + @Mock + private ElasticSearchEventSerializer serializer; + + @Mock + private IndexNameBuilder nameBuilder; + + @Mock + private Event event; + + @Mock + private HttpClient httpClient; + + @Mock + private HttpResponse httpResponse; + + @Mock + private StatusLine httpStatus; + + @Mock + private HttpEntity httpEntity; + + private static final String INDEX_NAME = "foo_index"; + private static final String MESSAGE_CONTENT = "{\"body\":\"test\"}"; + private static final String[] HOSTS = {"host1", "host2"}; + + @Before + public void setUp() throws IOException { + initMocks(this); + BytesReference bytesReference = mock(BytesReference.class); + BytesStream bytesStream = mock(BytesStream.class); + + when(nameBuilder.getIndexName(any(Event.class))).thenReturn(INDEX_NAME); + when(bytesReference.toBytesArray()).thenReturn(new BytesArray(MESSAGE_CONTENT)); + when(bytesStream.bytes()).thenReturn(bytesReference); + when(serializer.getContentBuilder(any(Event.class))).thenReturn(bytesStream); + fixture = new ElasticSearchRestClient(HOSTS, serializer, httpClient); + } + + @Test + public void shouldAddNewEventWithoutTTL() throws Exception { + ArgumentCaptor argument = ArgumentCaptor.forClass(HttpPost.class); + + when(httpStatus.getStatusCode()).thenReturn(HttpStatus.SC_OK); + when(httpResponse.getStatusLine()).thenReturn(httpStatus); + when(httpClient.execute(any(HttpUriRequest.class))).thenReturn(httpResponse); + + fixture.addEvent(event, nameBuilder, "bar_type", -1); + fixture.execute(); + + verify(httpClient).execute(isA(HttpUriRequest.class)); + verify(httpClient).execute(argument.capture()); + + assertEquals("http://host1/_bulk", argument.getValue().getURI().toString()); + assertEquals("{\"index\":{\"_type\":\"bar_type\",\"_index\":\"foo_index\"}}\n" + MESSAGE_CONTENT + "\n", + EntityUtils.toString(argument.getValue().getEntity())); + } + + @Test + public void shouldAddNewEventWithTTL() throws Exception { + ArgumentCaptor argument = ArgumentCaptor.forClass(HttpPost.class); + + when(httpStatus.getStatusCode()).thenReturn(HttpStatus.SC_OK); + when(httpResponse.getStatusLine()).thenReturn(httpStatus); + when(httpClient.execute(any(HttpUriRequest.class))).thenReturn(httpResponse); + + fixture.addEvent(event, nameBuilder, "bar_type", 123); + fixture.execute(); + + verify(httpClient).execute(isA(HttpUriRequest.class)); + verify(httpClient).execute(argument.capture()); + + assertEquals("http://host1/_bulk", argument.getValue().getURI().toString()); + assertEquals("{\"index\":{\"_type\":\"bar_type\",\"_index\":\"foo_index\",\"_ttl\":\"123\"}}\n" + + MESSAGE_CONTENT + "\n", EntityUtils.toString(argument.getValue().getEntity())); + } + + @Test(expected = EventDeliveryException.class) + public void shouldThrowEventDeliveryException() throws Exception { + ArgumentCaptor argument = ArgumentCaptor.forClass(HttpPost.class); + + when(httpStatus.getStatusCode()).thenReturn(HttpStatus.SC_INTERNAL_SERVER_ERROR); + when(httpResponse.getStatusLine()).thenReturn(httpStatus); + when(httpClient.execute(any(HttpUriRequest.class))).thenReturn(httpResponse); + + fixture.addEvent(event, nameBuilder, "bar_type", 123); + fixture.execute(); + } + + @Test() + public void shouldRetryBulkOperation() throws Exception { + ArgumentCaptor argument = ArgumentCaptor.forClass(HttpPost.class); + + when(httpStatus.getStatusCode()).thenReturn(HttpStatus.SC_INTERNAL_SERVER_ERROR, HttpStatus.SC_OK); + when(httpResponse.getStatusLine()).thenReturn(httpStatus); + when(httpClient.execute(any(HttpUriRequest.class))).thenReturn(httpResponse); + + fixture.addEvent(event, nameBuilder, "bar_type", 123); + fixture.execute(); + + verify(httpClient, times(2)).execute(isA(HttpUriRequest.class)); + verify(httpClient, times(2)).execute(argument.capture()); + + List allValues = argument.getAllValues(); + assertEquals("http://host1/_bulk", allValues.get(0).getURI().toString()); + assertEquals("http://host2/_bulk", allValues.get(1).getURI().toString()); + } +} diff --git a/flume-ng-sinks/flume-ng-elasticsearch-sink/src/test/java/org/apache/flume/sink/elasticsearch/client/TestElasticSearchTransportClient.java b/flume-ng-sinks/flume-ng-elasticsearch-sink/src/test/java/org/apache/flume/sink/elasticsearch/client/TestElasticSearchTransportClient.java new file mode 100644 index 0000000000..b7b8e74fcf --- /dev/null +++ b/flume-ng-sinks/flume-ng-elasticsearch-sink/src/test/java/org/apache/flume/sink/elasticsearch/client/TestElasticSearchTransportClient.java @@ -0,0 +1,127 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, + * software distributed under the License is distributed on an + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + * KIND, either express or implied. See the License for the + * specific language governing permissions and limitations + * under the License. + */ +package org.apache.flume.sink.elasticsearch.client; + +import org.apache.flume.Event; +import org.apache.flume.EventDeliveryException; +import org.apache.flume.sink.elasticsearch.ElasticSearchEventSerializer; +import org.apache.flume.sink.elasticsearch.IndexNameBuilder; +import org.elasticsearch.action.ListenableActionFuture; +import org.elasticsearch.action.bulk.BulkRequestBuilder; +import org.elasticsearch.action.bulk.BulkResponse; +import org.elasticsearch.action.index.IndexRequestBuilder; +import org.elasticsearch.client.Client; +import org.elasticsearch.common.bytes.BytesReference; +import org.elasticsearch.common.io.BytesStream; +import org.junit.Before; +import org.junit.Test; +import org.mockito.Mock; + +import java.io.IOException; + +import static org.mockito.Matchers.any; +import static org.mockito.Matchers.anyString; +import static org.mockito.Mockito.*; +import static org.mockito.MockitoAnnotations.initMocks; + +public class TestElasticSearchTransportClient { + + private ElasticSearchTransportClient fixture; + + @Mock + private ElasticSearchEventSerializer serializer; + + @Mock + private IndexNameBuilder nameBuilder; + + @Mock + private Client elasticSearchClient; + + @Mock + private BulkRequestBuilder bulkRequestBuilder; + + @Mock + private IndexRequestBuilder indexRequestBuilder; + + @Mock + private Event event; + + @Before + public void setUp() throws IOException { + initMocks(this); + BytesReference bytesReference = mock(BytesReference.class); + BytesStream bytesStream = mock(BytesStream.class); + + when(nameBuilder.getIndexName(any(Event.class))).thenReturn("foo_index"); + when(bytesReference.toBytes()).thenReturn("{\"body\":\"test\"}".getBytes()); + when(bytesStream.bytes()).thenReturn(bytesReference); + when(serializer.getContentBuilder(any(Event.class))) + .thenReturn(bytesStream); + when(elasticSearchClient.prepareIndex(anyString(), anyString())) + .thenReturn(indexRequestBuilder); + when(indexRequestBuilder.setSource(bytesReference)).thenReturn( + indexRequestBuilder); + + fixture = new ElasticSearchTransportClient(elasticSearchClient, serializer); + fixture.setBulkRequestBuilder(bulkRequestBuilder); + } + + @Test + public void shouldAddNewEventWithoutTTL() throws Exception { + fixture.addEvent(event, nameBuilder, "bar_type", -1); + verify(indexRequestBuilder).setSource( + serializer.getContentBuilder(event).bytes()); + verify(bulkRequestBuilder).add(indexRequestBuilder); + } + + @Test + public void shouldAddNewEventWithTTL() throws Exception { + fixture.addEvent(event, nameBuilder, "bar_type", 10); + verify(indexRequestBuilder).setTTL(10); + verify(indexRequestBuilder).setSource( + serializer.getContentBuilder(event).bytes()); + } + + @Test + public void shouldExecuteBulkRequestBuilder() throws Exception { + ListenableActionFuture action = + (ListenableActionFuture) mock(ListenableActionFuture.class); + BulkResponse response = mock(BulkResponse.class); + when(bulkRequestBuilder.execute()).thenReturn(action); + when(action.actionGet()).thenReturn(response); + when(response.hasFailures()).thenReturn(false); + + fixture.addEvent(event, nameBuilder, "bar_type", 10); + fixture.execute(); + verify(bulkRequestBuilder).execute(); + } + + @Test(expected = EventDeliveryException.class) + public void shouldThrowExceptionOnExecuteFailed() throws Exception { + ListenableActionFuture action = + (ListenableActionFuture) mock(ListenableActionFuture.class); + BulkResponse response = mock(BulkResponse.class); + when(bulkRequestBuilder.execute()).thenReturn(action); + when(action.actionGet()).thenReturn(response); + when(response.hasFailures()).thenReturn(true); + + fixture.addEvent(event, nameBuilder, "bar_type", 10); + fixture.execute(); + } +} From 09f0a513602c3b0bb2c9e19903ae6d5e052ab460 Mon Sep 17 00:00:00 2001 From: Jarek Jarcec Cecho Date: Mon, 24 Mar 2014 12:55:35 -0700 Subject: [PATCH 096/341] FLUME-2347: Add FLUME_JAVA_OPTS which allows users to inject java properties from cmd line (Brock Noland via Jarek Jarcec Cecho) --- bin/flume-ng | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/bin/flume-ng b/bin/flume-ng index 3a8a10da2c..e09e26b584 100755 --- a/bin/flume-ng +++ b/bin/flume-ng @@ -224,7 +224,7 @@ run_flume() { if [ ${CLEAN_FLAG} -ne 0 ]; then set -x fi - $EXEC $JAVA_HOME/bin/java $JAVA_OPTS "${arr_java_props[@]}" -cp "$FLUME_CLASSPATH" \ + $EXEC $JAVA_HOME/bin/java $JAVA_OPTS $FLUME_JAVA_OPTS "${arr_java_props[@]}" -cp "$FLUME_CLASSPATH" \ -Djava.library.path=$FLUME_JAVA_LIBRARY_PATH "$FLUME_APPLICATION_CLASS" $* } From 61b9bcbb69ae3d19f72276b3aaa78ff3679cecfc Mon Sep 17 00:00:00 2001 From: Hari Shreedharan Date: Thu, 27 Mar 2014 12:30:29 -0700 Subject: [PATCH 097/341] FLUME-2309. Spooling directory should not always consume the oldest file first. (Muhammad Ehsan ul Haque via Hari Shreedharan) --- .../avro/ReliableSpoolingFileEventReader.java | 176 ++++++++++------- .../flume/source/SpoolDirectorySource.java | 23 ++- ...DirectorySourceConfigurationConstants.java | 7 + .../TestReliableSpoolingFileEventReader.java | 187 +++++++++++++++++- .../source/TestSpoolDirectorySource.java | 26 +++ flume-ng-doc/sphinx/FlumeUserGuide.rst | 8 + 6 files changed, 335 insertions(+), 92 deletions(-) diff --git a/flume-ng-core/src/main/java/org/apache/flume/client/avro/ReliableSpoolingFileEventReader.java b/flume-ng-core/src/main/java/org/apache/flume/client/avro/ReliableSpoolingFileEventReader.java index a88ed6e453..18182504cc 100644 --- a/flume-ng-core/src/main/java/org/apache/flume/client/avro/ReliableSpoolingFileEventReader.java +++ b/flume-ng-core/src/main/java/org/apache/flume/client/avro/ReliableSpoolingFileEventReader.java @@ -19,38 +19,31 @@ package org.apache.flume.client.avro; -import java.io.File; -import java.io.FileFilter; -import java.io.FileNotFoundException; -import java.io.IOException; -import java.nio.charset.Charset; -import java.util.Arrays; -import java.util.Collections; -import java.util.Comparator; -import java.util.List; -import java.util.regex.Pattern; - +import com.google.common.base.Charsets; +import com.google.common.base.Optional; +import com.google.common.base.Preconditions; +import com.google.common.io.Files; +import org.apache.commons.io.FileUtils; +import org.apache.commons.io.filefilter.IOFileFilter; import org.apache.flume.Context; import org.apache.flume.Event; import org.apache.flume.FlumeException; import org.apache.flume.annotations.InterfaceAudience; import org.apache.flume.annotations.InterfaceStability; -import org.apache.flume.serialization.DecodeErrorPolicy; -import org.apache.flume.serialization.DurablePositionTracker; -import org.apache.flume.serialization.EventDeserializer; -import org.apache.flume.serialization.EventDeserializerFactory; -import org.apache.flume.serialization.PositionTracker; -import org.apache.flume.serialization.ResettableFileInputStream; -import org.apache.flume.serialization.ResettableInputStream; +import org.apache.flume.serialization.*; import org.apache.flume.source.SpoolDirectorySourceConfigurationConstants; +import org.apache.flume.source.SpoolDirectorySourceConfigurationConstants.ConsumeOrder; import org.apache.flume.tools.PlatformDetect; import org.slf4j.Logger; import org.slf4j.LoggerFactory; -import com.google.common.base.Charsets; -import com.google.common.base.Optional; -import com.google.common.base.Preconditions; -import com.google.common.io.Files; +import java.io.File; +import java.io.FileFilter; +import java.io.FileNotFoundException; +import java.io.IOException; +import java.nio.charset.Charset; +import java.util.*; +import java.util.regex.Pattern; /** *

    A {@link ReliableEventReader} which reads log data from files stored @@ -98,7 +91,8 @@ public class ReliableSpoolingFileEventReader implements ReliableEventReader { private final String deletePolicy; private final Charset inputCharset; private final DecodeErrorPolicy decodeErrorPolicy; - + private final ConsumeOrder consumeOrder; + private Optional currentFile = Optional.absent(); /** Always contains the last file from which lines have been read. **/ private Optional lastFileRead = Optional.absent(); @@ -113,7 +107,8 @@ private ReliableSpoolingFileEventReader(File spoolDirectory, boolean annotateBaseName, String baseNameHeader, String deserializerType, Context deserializerContext, String deletePolicy, String inputCharset, - DecodeErrorPolicy decodeErrorPolicy) throws IOException { + DecodeErrorPolicy decodeErrorPolicy, + ConsumeOrder consumeOrder) throws IOException { // Sanity checks Preconditions.checkNotNull(spoolDirectory); @@ -173,6 +168,7 @@ private ReliableSpoolingFileEventReader(File spoolDirectory, this.deletePolicy = deletePolicy; this.inputCharset = Charset.forName(inputCharset); this.decodeErrorPolicy = Preconditions.checkNotNull(decodeErrorPolicy); + this.consumeOrder = Preconditions.checkNotNull(consumeOrder); File trackerDirectory = new File(trackerDirPath); @@ -403,9 +399,16 @@ private void deleteCurrentFile(File fileToDelete) throws IOException { } /** - * Find and open the oldest file in the chosen directory. If two or more - * files are equally old, the file name with lower lexicographical value is - * returned. If the directory is empty, this will return an absent option. + * Returns the next file to be consumed from the chosen directory. + * If the directory is empty or the chosen file is not readable, + * this will return an absent option. + * If the {@link #consumeOrder} variable is {@link ConsumeOrder#OLDEST} + * then returns the oldest file. If the {@link #consumeOrder} variable + * is {@link ConsumeOrder#YOUNGEST} then returns the youngest file. + * If two or more files are equally old/young, then the file name with + * lower lexicographical value is returned. + * If the {@link #consumeOrder} variable is {@link ConsumeOrder#RANDOM} + * then returns any arbitrary file in the directory. */ private Optional getNextFile() { /* Filter to exclude finished or hidden files */ @@ -421,55 +424,72 @@ public boolean accept(File candidate) { return true; } }; - List candidateFiles = Arrays.asList(spoolDirectory.listFiles(filter)); - if (candidateFiles.isEmpty()) { + List candidateFiles = Arrays.asList( + spoolDirectory.listFiles(filter)); + if (candidateFiles.isEmpty()) { // No matching file in spooling directory. return Optional.absent(); - } else { - Collections.sort(candidateFiles, new Comparator() { - public int compare(File a, File b) { - int timeComparison = new Long(a.lastModified()).compareTo( - new Long(b.lastModified())); - if (timeComparison != 0) { - return timeComparison; - } - else { - return a.getName().compareTo(b.getName()); - } + } + + File selectedFile = candidateFiles.get(0); // Select the first random file. + if (consumeOrder == ConsumeOrder.RANDOM) { // Selected file is random. + return openFile(selectedFile); + } else if (consumeOrder == ConsumeOrder.YOUNGEST) { + for (File candidateFile: candidateFiles) { + if (candidateFile.lastModified() > + selectedFile.lastModified()) { + selectedFile = candidateFile; } - }); - File nextFile = candidateFiles.get(0); - try { - // roll the meta file, if needed - String nextPath = nextFile.getPath(); - PositionTracker tracker = - DurablePositionTracker.getInstance(metaFile, nextPath); - if (!tracker.getTarget().equals(nextPath)) { - tracker.close(); - deleteMetaFile(); - tracker = DurablePositionTracker.getInstance(metaFile, nextPath); + } + } else { // default order is OLDEST + for (File candidateFile: candidateFiles) { + if (candidateFile.lastModified() < + selectedFile.lastModified()) { + selectedFile = candidateFile; } + } + } - // sanity check - Preconditions.checkState(tracker.getTarget().equals(nextPath), - "Tracker target %s does not equal expected filename %s", - tracker.getTarget(), nextPath); - - ResettableInputStream in = - new ResettableFileInputStream(nextFile, tracker, - ResettableFileInputStream.DEFAULT_BUF_SIZE, inputCharset, - decodeErrorPolicy); - EventDeserializer deserializer = EventDeserializerFactory.getInstance - (deserializerType, deserializerContext, in); - - return Optional.of(new FileInfo(nextFile, deserializer)); - } catch (FileNotFoundException e) { - // File could have been deleted in the interim - logger.warn("Could not find file: " + nextFile, e); - return Optional.absent(); - } catch (IOException e) { - logger.error("Exception opening file: " + nextFile, e); - return Optional.absent(); + return openFile(selectedFile); + } + + /** + * Opens a file for consuming + * @param file + * @return {@link #FileInfo} for the file to consume or absent option if the + * file does not exists or readable. + */ + private Optional openFile(File file) { + try { + // roll the meta file, if needed + String nextPath = file.getPath(); + PositionTracker tracker = + DurablePositionTracker.getInstance(metaFile, nextPath); + if (!tracker.getTarget().equals(nextPath)) { + tracker.close(); + deleteMetaFile(); + tracker = DurablePositionTracker.getInstance(metaFile, nextPath); } + + // sanity check + Preconditions.checkState(tracker.getTarget().equals(nextPath), + "Tracker target %s does not equal expected filename %s", + tracker.getTarget(), nextPath); + + ResettableInputStream in = + new ResettableFileInputStream(file, tracker, + ResettableFileInputStream.DEFAULT_BUF_SIZE, inputCharset, + decodeErrorPolicy); + EventDeserializer deserializer = EventDeserializerFactory.getInstance + (deserializerType, deserializerContext, in); + + return Optional.of(new FileInfo(file, deserializer)); + } catch (FileNotFoundException e) { + // File could have been deleted in the interim + logger.warn("Could not find file: " + file, e); + return Optional.absent(); + } catch (IOException e) { + logger.error("Exception opening file: " + file, e); + return Optional.absent(); } } @@ -536,7 +556,9 @@ public static class Builder { private DecodeErrorPolicy decodeErrorPolicy = DecodeErrorPolicy.valueOf( SpoolDirectorySourceConfigurationConstants.DEFAULT_DECODE_ERROR_POLICY .toUpperCase()); - + private ConsumeOrder consumeOrder = + SpoolDirectorySourceConfigurationConstants.DEFAULT_CONSUME_ORDER; + public Builder spoolDirectory(File directory) { this.spoolDirectory = directory; return this; @@ -601,12 +623,18 @@ public Builder decodeErrorPolicy(DecodeErrorPolicy decodeErrorPolicy) { this.decodeErrorPolicy = decodeErrorPolicy; return this; } - + + public Builder consumeOrder(ConsumeOrder consumeOrder) { + this.consumeOrder = consumeOrder; + return this; + } + public ReliableSpoolingFileEventReader build() throws IOException { return new ReliableSpoolingFileEventReader(spoolDirectory, completedSuffix, ignorePattern, trackerDirPath, annotateFileName, fileNameHeader, annotateBaseName, baseNameHeader, deserializerType, - deserializerContext, deletePolicy, inputCharset, decodeErrorPolicy); + deserializerContext, deletePolicy, inputCharset, decodeErrorPolicy, + consumeOrder); } } diff --git a/flume-ng-core/src/main/java/org/apache/flume/source/SpoolDirectorySource.java b/flume-ng-core/src/main/java/org/apache/flume/source/SpoolDirectorySource.java index 49d8826442..d0c2e99673 100644 --- a/flume-ng-core/src/main/java/org/apache/flume/source/SpoolDirectorySource.java +++ b/flume-ng-core/src/main/java/org/apache/flume/source/SpoolDirectorySource.java @@ -17,14 +17,8 @@ package org.apache.flume.source; -import java.io.File; -import java.io.IOException; -import java.util.List; -import java.util.concurrent.Executors; -import java.util.concurrent.ScheduledExecutorService; -import java.util.concurrent.TimeUnit; - import com.google.common.annotations.VisibleForTesting; +import com.google.common.base.Preconditions; import com.google.common.base.Throwables; import org.apache.flume.*; import org.apache.flume.client.avro.ReliableSpoolingFileEventReader; @@ -35,10 +29,14 @@ import org.slf4j.Logger; import org.slf4j.LoggerFactory; -import com.google.common.base.Preconditions; +import java.io.File; +import java.io.IOException; +import java.util.List; +import java.util.concurrent.Executors; +import java.util.concurrent.ScheduledExecutorService; +import java.util.concurrent.TimeUnit; -import static org.apache.flume.source - .SpoolDirectorySourceConfigurationConstants.*; +import static org.apache.flume.source.SpoolDirectorySourceConfigurationConstants.*; public class SpoolDirectorySource extends AbstractSource implements Configurable, EventDrivenSource { @@ -72,6 +70,7 @@ public class SpoolDirectorySource extends AbstractSource implements private boolean backoff = true; private boolean hitChannelException = false; private int maxBackoff; + private ConsumeOrder consumeOrder; @Override public synchronized void start() { @@ -96,6 +95,7 @@ public synchronized void start() { .deletePolicy(deletePolicy) .inputCharset(inputCharset) .decodeErrorPolicy(decodeErrorPolicy) + .consumeOrder(consumeOrder) .build(); } catch (IOException ioe) { throw new FlumeException("Error instantiating spooling event parser", @@ -163,6 +163,9 @@ public synchronized void configure(Context context) { deserializerType = context.getString(DESERIALIZER, DEFAULT_DESERIALIZER); deserializerContext = new Context(context.getSubProperties(DESERIALIZER + ".")); + + consumeOrder = ConsumeOrder.valueOf(context.getString(CONSUME_ORDER, + DEFAULT_CONSUME_ORDER.toString()).toUpperCase()); // "Hack" to support backwards compatibility with previous generation of // spooling directory source, which did not support deserializers diff --git a/flume-ng-core/src/main/java/org/apache/flume/source/SpoolDirectorySourceConfigurationConstants.java b/flume-ng-core/src/main/java/org/apache/flume/source/SpoolDirectorySourceConfigurationConstants.java index 83522c0d0a..895433e411 100644 --- a/flume-ng-core/src/main/java/org/apache/flume/source/SpoolDirectorySourceConfigurationConstants.java +++ b/flume-ng-core/src/main/java/org/apache/flume/source/SpoolDirectorySourceConfigurationConstants.java @@ -86,4 +86,11 @@ public class SpoolDirectorySourceConfigurationConstants { public static final String MAX_BACKOFF = "maxBackoff"; public static final Integer DEFAULT_MAX_BACKOFF = 4000; + + /** Consume order. */ + public enum ConsumeOrder { + OLDEST, YOUNGEST, RANDOM + } + public static final String CONSUME_ORDER = "consumeOrder"; + public static final ConsumeOrder DEFAULT_CONSUME_ORDER = ConsumeOrder.OLDEST; } diff --git a/flume-ng-core/src/test/java/org/apache/flume/client/avro/TestReliableSpoolingFileEventReader.java b/flume-ng-core/src/test/java/org/apache/flume/client/avro/TestReliableSpoolingFileEventReader.java index 9d708c16ea..0b07e7a1fd 100644 --- a/flume-ng-core/src/test/java/org/apache/flume/client/avro/TestReliableSpoolingFileEventReader.java +++ b/flume-ng-core/src/test/java/org/apache/flume/client/avro/TestReliableSpoolingFileEventReader.java @@ -19,14 +19,16 @@ import com.google.common.base.Charsets; import com.google.common.collect.Lists; +import com.google.common.collect.Sets; import com.google.common.io.Files; import junit.framework.Assert; +import org.apache.commons.io.FileUtils; import org.apache.flume.Event; -import org.apache.flume.source.SpoolDirectorySourceConfigurationConstants; import org.apache.flume.client.avro.ReliableSpoolingFileEventReader.DeletePolicy; +import org.apache.flume.source.SpoolDirectorySourceConfigurationConstants; +import org.apache.flume.source.SpoolDirectorySourceConfigurationConstants.ConsumeOrder; import org.junit.After; import org.junit.Before; -import org.junit.Ignore; import org.junit.Test; import org.slf4j.Logger; import org.slf4j.LoggerFactory; @@ -34,7 +36,8 @@ import java.io.File; import java.io.FileFilter; import java.io.IOException; -import java.util.List; +import java.util.*; +import java.util.Map.Entry; public class TestReliableSpoolingFileEventReader { @@ -67,9 +70,12 @@ public void setup() throws IOException, InterruptedException { @After public void tearDown() { + deleteDir(WORK_DIR); + } + private void deleteDir(File dir) { // delete all the files & dirs we created - File[] files = WORK_DIR.listFiles(); + File[] files = dir.listFiles(); for (File f : files) { if (f.isDirectory()) { File[] subDirFiles = f.listFiles(); @@ -87,10 +93,9 @@ public void tearDown() { } } } - if (!WORK_DIR.delete()) { - logger.warn("Cannot delete work directory {}", WORK_DIR.getAbsolutePath()); + if (!dir.delete()) { + logger.warn("Cannot delete work directory {}", dir.getAbsolutePath()); } - } @Test @@ -188,6 +193,173 @@ public void testFileDeletion() throws IOException { trackerFiles.size()); } + @Test(expected = NullPointerException.class) + public void testNullConsumeOrder() throws IOException { + new ReliableSpoolingFileEventReader.Builder() + .spoolDirectory(WORK_DIR) + .consumeOrder(null) + .build(); + } + + @Test + public void testConsumeFileRandomly() throws IOException { + ReliableEventReader reader = new ReliableSpoolingFileEventReader.Builder() + .spoolDirectory(WORK_DIR) + .consumeOrder(ConsumeOrder.RANDOM) + .build(); + File fileName = new File(WORK_DIR, "new-file"); + FileUtils.write(fileName, "New file created in the end. Shoud be read randomly.\n"); + Set actual = Sets.newHashSet(); + readEventsForFilesInDir(WORK_DIR, reader, actual); + Set expected = Sets.newHashSet(); + createExpectedFromFilesInSetup(expected); + expected.add(""); + expected.add("New file created in the end. Shoud be read randomly."); + Assert.assertEquals(expected, actual); + } + + + @Test + public void testConsumeFileOldest() throws IOException, InterruptedException { + ReliableEventReader reader = new ReliableSpoolingFileEventReader.Builder() + .spoolDirectory(WORK_DIR) + .consumeOrder(ConsumeOrder.OLDEST) + .build(); + File file1 = new File(WORK_DIR, "new-file1"); + File file2 = new File(WORK_DIR, "new-file2"); + File file3 = new File(WORK_DIR, "new-file3"); + FileUtils.write(file2, "New file2 created.\n"); // file2 becoming older than file1 & file3 + Thread.sleep(1000L); + FileUtils.write(file1, "New file1 created.\n"); // file1 becoming older than file3 + FileUtils.write(file3, "New file3 created.\n"); + + List actual = Lists.newLinkedList(); + readEventsForFilesInDir(WORK_DIR, reader, actual); + List expected = Lists.newLinkedList(); + createExpectedFromFilesInSetup(expected); + expected.add(""); // Empty file was added in the last in setup. + expected.add("New file2 created."); + expected.add("New file1 created."); + expected.add("New file3 created."); + Assert.assertEquals(expected, actual); + } + + @Test + public void testConsumeFileYoungest() throws IOException, InterruptedException { + ReliableEventReader reader = new ReliableSpoolingFileEventReader.Builder() + .spoolDirectory(WORK_DIR) + .consumeOrder(ConsumeOrder.YOUNGEST) + .build(); + Thread.sleep(1000L); + File file1 = new File(WORK_DIR, "new-file1"); + File file2 = new File(WORK_DIR, "new-file2"); + File file3 = new File(WORK_DIR, "new-file3"); + FileUtils.write(file2, "New file2 created.\n"); // file2 is oldest among file1 & file3. + Thread.sleep(1000L); + FileUtils.write(file3, "New file3 created.\n"); // file3 becomes youngest then file2 but older from file1. + FileUtils.write(file1, "New file1 created.\n"); // file1 becomes youngest in file2 & file3. + List actual = Lists.newLinkedList(); + readEventsForFilesInDir(WORK_DIR, reader, actual); + List expected = Lists.newLinkedList(); + createExpectedFromFilesInSetup(expected); + Collections.sort(expected); + expected.add(0, ""); // Empty Line file was added in the last in Setup. + expected.add(0, "New file2 created."); + expected.add(0, "New file3 created."); + expected.add(0, "New file1 created."); + + Assert.assertEquals(expected, actual); + } + + @Test public void testLargeNumberOfFilesOLDEST() throws IOException { + templateTestForLargeNumberOfFiles(ConsumeOrder.OLDEST, null, 1000); + } + @Test public void testLargeNumberOfFilesYOUNGEST() throws IOException { + templateTestForLargeNumberOfFiles(ConsumeOrder.YOUNGEST, new Comparator() { + + @Override + public int compare(Long o1, Long o2) { + return o2.compareTo(o1); + } + }, 1000); + } + @Test public void testLargeNumberOfFilesRANDOM() throws IOException { + templateTestForLargeNumberOfFiles(ConsumeOrder.RANDOM, null, 1000); + } + private void templateTestForLargeNumberOfFiles(ConsumeOrder order, + Comparator comparator, + int N) throws IOException { + File dir = null; + try { + dir = new File("target/test/work/" + this.getClass().getSimpleName()+ "_large"); + Files.createParentDirs(new File(dir, "dummy")); + ReliableEventReader reader = new ReliableSpoolingFileEventReader.Builder() + .spoolDirectory(dir).consumeOrder(order).build(); + Map> expected; + if (comparator == null) { + expected = new TreeMap>(); + } else { + expected = new TreeMap>(comparator); + } + for (int i = 0; i < N; i++) { + File f = new File(dir, "file-" + i); + String data = "file-" + i; + Files.write(data, f, Charsets.UTF_8); + if (expected.containsKey(f.lastModified())) { + expected.get(f.lastModified()).add(data); + } else { + expected.put(f.lastModified(), Lists.newArrayList(data)); + } + } + Collection expectedList; + if (order == ConsumeOrder.RANDOM) { + expectedList = Sets.newHashSet(); + } else { + expectedList = Lists.newArrayList(); + } + for (Entry> entry : expected.entrySet()) { + Collections.sort(entry.getValue()); + expectedList.addAll(entry.getValue()); + } + for (int i = 0; i < N; i++) { + List events; + events = reader.readEvents(10); + for (Event e : events) { + if (order == ConsumeOrder.RANDOM) { + Assert.assertTrue(expectedList.remove(new String(e.getBody()))); + } else { + Assert.assertEquals(((ArrayList)expectedList).get(0), new String(e.getBody())); + ((ArrayList)expectedList).remove(0); + } + } + reader.commit(); + } + } finally { + deleteDir(dir); + } + } + + /* Read events, one for each file in the given directory. */ + private void readEventsForFilesInDir(File dir, ReliableEventReader reader, + Collection actual) throws IOException { + List events; + for (int i=0; i < listFiles(dir).size(); i++) { + events = reader.readEvents(10); + for (Event e: events) { + actual.add(new String(e.getBody())); + } + reader.commit(); + } + } + /* Create expected results out of the files created in the setup method. */ + private void createExpectedFromFilesInSetup(Collection expected) { + for (int i = 0; i < 4; i++) { + for (int j = 0; j < i; j++) { + expected.add("file" + i + "line" + j); + } + } + } + private static List listFiles(File dir) { List files = Lists.newArrayList(dir.listFiles(new FileFilter () { @@ -198,5 +370,4 @@ public boolean accept(File pathname) { })); return files; } - } diff --git a/flume-ng-core/src/test/java/org/apache/flume/source/TestSpoolDirectorySource.java b/flume-ng-core/src/test/java/org/apache/flume/source/TestSpoolDirectorySource.java index 503ab4d24c..89e7c8c3c5 100644 --- a/flume-ng-core/src/test/java/org/apache/flume/source/TestSpoolDirectorySource.java +++ b/flume-ng-core/src/test/java/org/apache/flume/source/TestSpoolDirectorySource.java @@ -75,6 +75,32 @@ public void tearDown() { tmpDir.delete(); } + @Test (expected = IllegalArgumentException.class) + public void testInvalidSortOrder() { + Context context = new Context(); + context.put(SpoolDirectorySourceConfigurationConstants.SPOOL_DIRECTORY, + tmpDir.getAbsolutePath()); + context.put(SpoolDirectorySourceConfigurationConstants.CONSUME_ORDER, + "undefined"); + Configurables.configure(source, context); + } + + @Test + public void testValidSortOrder() { + Context context = new Context(); + context.put(SpoolDirectorySourceConfigurationConstants.SPOOL_DIRECTORY, + tmpDir.getAbsolutePath()); + context.put(SpoolDirectorySourceConfigurationConstants.CONSUME_ORDER, + "oLdESt"); + Configurables.configure(source, context); + context.put(SpoolDirectorySourceConfigurationConstants.CONSUME_ORDER, + "yoUnGest"); + Configurables.configure(source, context); + context.put(SpoolDirectorySourceConfigurationConstants.CONSUME_ORDER, + "rAnDom"); + Configurables.configure(source, context); + } + @Test public void testPutFilenameHeader() throws IOException, InterruptedException { Context context = new Context(); diff --git a/flume-ng-doc/sphinx/FlumeUserGuide.rst b/flume-ng-doc/sphinx/FlumeUserGuide.rst index 4bcd8a2d41..7b918ed5b9 100644 --- a/flume-ng-doc/sphinx/FlumeUserGuide.rst +++ b/flume-ng-doc/sphinx/FlumeUserGuide.rst @@ -953,6 +953,14 @@ basenameHeaderKey basename Header Key to use when appending basename ignorePattern ^$ Regular expression specifying which files to ignore (skip) trackerDir .flumespool Directory to store metadata related to processing of files. If this path is not an absolute path, then it is interpreted as relative to the spoolDir. +consumeOrder oldest In which order files in the spooling directory will be consumed ``oldest``, + ``youngest`` and ``random``. In case of ``oldest`` and ``youngest``, the last modified + time of the files will be used to compare the files. In case of a tie, the file + with smallest laxicographical order will be consumed first. In case of ``random`` any + file will be picked randomly. When using ``oldest`` and ``youngest`` the whole + directory will be scanned to pick the oldest/youngest file, which might be slow if there + are a large number of files, while using ``random`` may cause old files to be consumed + very late if new files keep coming in the spooling directory. maxBackoff 4000 The maximum time (in millis) to wait between consecutive attempts to write to the channel(s) if the channel is full. The source will start at a low backoff and increase it exponentially each time the channel throws a ChannelException, upto the value specified by this parameter. batchSize 100 Granularity at which to batch transfer to the channel inputCharset UTF-8 Character set used by deserializers that treat the input file as text. From 62b383a00c3f678b0f504dc71bf36091ddd4067a Mon Sep 17 00:00:00 2001 From: Hari Shreedharan Date: Fri, 28 Mar 2014 15:55:25 -0700 Subject: [PATCH 098/341] FLUME-2350. Consume Order tests need to space out file creation. (Muhammad Ehsan ul Haque via Hari Shreedharan) --- .../avro/ReliableSpoolingFileEventReader.java | 25 +++- .../TestReliableSpoolingFileEventReader.java | 132 ++++++++++++++---- 2 files changed, 123 insertions(+), 34 deletions(-) diff --git a/flume-ng-core/src/main/java/org/apache/flume/client/avro/ReliableSpoolingFileEventReader.java b/flume-ng-core/src/main/java/org/apache/flume/client/avro/ReliableSpoolingFileEventReader.java index 18182504cc..0bc3f23f03 100644 --- a/flume-ng-core/src/main/java/org/apache/flume/client/avro/ReliableSpoolingFileEventReader.java +++ b/flume-ng-core/src/main/java/org/apache/flume/client/avro/ReliableSpoolingFileEventReader.java @@ -25,6 +25,7 @@ import com.google.common.io.Files; import org.apache.commons.io.FileUtils; import org.apache.commons.io.filefilter.IOFileFilter; +import org.apache.commons.lang.StringUtils; import org.apache.flume.Context; import org.apache.flume.Event; import org.apache.flume.FlumeException; @@ -429,21 +430,27 @@ public boolean accept(File candidate) { if (candidateFiles.isEmpty()) { // No matching file in spooling directory. return Optional.absent(); } - + File selectedFile = candidateFiles.get(0); // Select the first random file. if (consumeOrder == ConsumeOrder.RANDOM) { // Selected file is random. return openFile(selectedFile); } else if (consumeOrder == ConsumeOrder.YOUNGEST) { for (File candidateFile: candidateFiles) { - if (candidateFile.lastModified() > - selectedFile.lastModified()) { + long compare = selectedFile.lastModified() - + candidateFile.lastModified(); + if (compare == 0) { // ts is same pick smallest lexicographically. + selectedFile = smallerLexicographical(selectedFile, candidateFile); + } else if (compare < 0) { // candidate is younger (cand-ts > selec-ts) selectedFile = candidateFile; } } } else { // default order is OLDEST for (File candidateFile: candidateFiles) { - if (candidateFile.lastModified() < - selectedFile.lastModified()) { + long compare = selectedFile.lastModified() - + candidateFile.lastModified(); + if (compare == 0) { // ts is same pick smallest lexicographically. + selectedFile = smallerLexicographical(selectedFile, candidateFile); + } else if (compare > 0) { // candidate is older (cand-ts < selec-ts). selectedFile = candidateFile; } } @@ -451,7 +458,13 @@ public boolean accept(File candidate) { return openFile(selectedFile); } - + + private File smallerLexicographical(File f1, File f2) { + if (f1.getName().compareTo(f2.getName()) < 0) { + return f1; + } + return f2; + } /** * Opens a file for consuming * @param file diff --git a/flume-ng-core/src/test/java/org/apache/flume/client/avro/TestReliableSpoolingFileEventReader.java b/flume-ng-core/src/test/java/org/apache/flume/client/avro/TestReliableSpoolingFileEventReader.java index 0b07e7a1fd..6a026129f9 100644 --- a/flume-ng-core/src/test/java/org/apache/flume/client/avro/TestReliableSpoolingFileEventReader.java +++ b/flume-ng-core/src/test/java/org/apache/flume/client/avro/TestReliableSpoolingFileEventReader.java @@ -203,36 +203,42 @@ public void testNullConsumeOrder() throws IOException { @Test public void testConsumeFileRandomly() throws IOException { - ReliableEventReader reader = new ReliableSpoolingFileEventReader.Builder() + ReliableEventReader reader + = new ReliableSpoolingFileEventReader.Builder() .spoolDirectory(WORK_DIR) .consumeOrder(ConsumeOrder.RANDOM) .build(); File fileName = new File(WORK_DIR, "new-file"); - FileUtils.write(fileName, "New file created in the end. Shoud be read randomly.\n"); - Set actual = Sets.newHashSet(); + FileUtils.write(fileName, + "New file created in the end. Shoud be read randomly.\n"); + Set actual = Sets.newHashSet(); readEventsForFilesInDir(WORK_DIR, reader, actual); Set expected = Sets.newHashSet(); createExpectedFromFilesInSetup(expected); expected.add(""); - expected.add("New file created in the end. Shoud be read randomly."); + expected.add( + "New file created in the end. Shoud be read randomly."); Assert.assertEquals(expected, actual); } @Test public void testConsumeFileOldest() throws IOException, InterruptedException { - ReliableEventReader reader = new ReliableSpoolingFileEventReader.Builder() - .spoolDirectory(WORK_DIR) - .consumeOrder(ConsumeOrder.OLDEST) - .build(); + ReliableEventReader reader + = new ReliableSpoolingFileEventReader.Builder() + .spoolDirectory(WORK_DIR) + .consumeOrder(ConsumeOrder.OLDEST) + .build(); File file1 = new File(WORK_DIR, "new-file1"); File file2 = new File(WORK_DIR, "new-file2"); File file3 = new File(WORK_DIR, "new-file3"); - FileUtils.write(file2, "New file2 created.\n"); // file2 becoming older than file1 & file3 Thread.sleep(1000L); - FileUtils.write(file1, "New file1 created.\n"); // file1 becoming older than file3 + FileUtils.write(file2, "New file2 created.\n"); + Thread.sleep(1000L); + FileUtils.write(file1, "New file1 created.\n"); + Thread.sleep(1000L); FileUtils.write(file3, "New file3 created.\n"); - + // order of age oldest to youngest (file2, file1, file3) List actual = Lists.newLinkedList(); readEventsForFilesInDir(WORK_DIR, reader, actual); List expected = Lists.newLinkedList(); @@ -245,25 +251,30 @@ public void testConsumeFileOldest() throws IOException, InterruptedException { } @Test - public void testConsumeFileYoungest() throws IOException, InterruptedException { - ReliableEventReader reader = new ReliableSpoolingFileEventReader.Builder() - .spoolDirectory(WORK_DIR) - .consumeOrder(ConsumeOrder.YOUNGEST) - .build(); - Thread.sleep(1000L); - File file1 = new File(WORK_DIR, "new-file1"); - File file2 = new File(WORK_DIR, "new-file2"); + public void testConsumeFileYoungest() + throws IOException, InterruptedException { + ReliableEventReader reader + = new ReliableSpoolingFileEventReader.Builder() + .spoolDirectory(WORK_DIR) + .consumeOrder(ConsumeOrder.YOUNGEST) + .build(); + File file1 = new File(WORK_DIR, "new-file1"); + File file2 = new File(WORK_DIR, "new-file2"); File file3 = new File(WORK_DIR, "new-file3"); - FileUtils.write(file2, "New file2 created.\n"); // file2 is oldest among file1 & file3. - Thread.sleep(1000L); - FileUtils.write(file3, "New file3 created.\n"); // file3 becomes youngest then file2 but older from file1. - FileUtils.write(file1, "New file1 created.\n"); // file1 becomes youngest in file2 & file3. + Thread.sleep(1000L); + FileUtils.write(file2, "New file2 created.\n"); + Thread.sleep(1000L); + FileUtils.write(file3, "New file3 created.\n"); + Thread.sleep(1000L); + FileUtils.write(file1, "New file1 created.\n"); + // order of age youngest to oldest (file2, file3, file1) List actual = Lists.newLinkedList(); readEventsForFilesInDir(WORK_DIR, reader, actual); List expected = Lists.newLinkedList(); createExpectedFromFilesInSetup(expected); Collections.sort(expected); - expected.add(0, ""); // Empty Line file was added in the last in Setup. + // Empty Line file was added in the last in Setup. + expected.add(0, ""); expected.add(0, "New file2 created."); expected.add(0, "New file3 created."); expected.add(0, "New file1 created."); @@ -271,6 +282,66 @@ public void testConsumeFileYoungest() throws IOException, InterruptedException { Assert.assertEquals(expected, actual); } + @Test + public void testConsumeFileOldestWithLexicographicalComparision() + throws IOException, InterruptedException { + ReliableEventReader reader + = new ReliableSpoolingFileEventReader.Builder() + .spoolDirectory(WORK_DIR) + .consumeOrder(ConsumeOrder.OLDEST) + .build(); + File file1 = new File(WORK_DIR, "new-file1"); + File file2 = new File(WORK_DIR, "new-file2"); + File file3 = new File(WORK_DIR, "new-file3"); + Thread.sleep(1000L); + FileUtils.write(file3, "New file3 created.\n"); + FileUtils.write(file2, "New file2 created.\n"); + FileUtils.write(file1, "New file1 created.\n"); + file1.setLastModified(file3.lastModified()); + file1.setLastModified(file2.lastModified()); + // file ages are same now they need to be ordered + // lexicographically (file1, file2, file3). + List actual = Lists.newLinkedList(); + readEventsForFilesInDir(WORK_DIR, reader, actual); + List expected = Lists.newLinkedList(); + createExpectedFromFilesInSetup(expected); + expected.add(""); // Empty file was added in the last in setup. + expected.add("New file1 created."); + expected.add("New file2 created."); + expected.add("New file3 created."); + Assert.assertEquals(expected, actual); + } + + @Test + public void testConsumeFileYoungestWithLexicographicalComparision() + throws IOException, InterruptedException { + ReliableEventReader reader + = new ReliableSpoolingFileEventReader.Builder() + .spoolDirectory(WORK_DIR) + .consumeOrder(ConsumeOrder.YOUNGEST) + .build(); + File file1 = new File(WORK_DIR, "new-file1"); + File file2 = new File(WORK_DIR, "new-file2"); + File file3 = new File(WORK_DIR, "new-file3"); + Thread.sleep(1000L); + FileUtils.write(file1, "New file1 created.\n"); + FileUtils.write(file2, "New file2 created.\n"); + FileUtils.write(file3, "New file3 created.\n"); + file1.setLastModified(file3.lastModified()); + file1.setLastModified(file2.lastModified()); + // file ages are same now they need to be ordered + // lexicographically (file1, file2, file3). + List actual = Lists.newLinkedList(); + readEventsForFilesInDir(WORK_DIR, reader, actual); + List expected = Lists.newLinkedList(); + createExpectedFromFilesInSetup(expected); + expected.add(0, ""); // Empty file was added in the last in setup. + expected.add(0, "New file3 created."); + expected.add(0, "New file2 created."); + expected.add(0, "New file1 created."); + Assert.assertEquals(expected, actual); + } + @Test public void testLargeNumberOfFilesOLDEST() throws IOException { templateTestForLargeNumberOfFiles(ConsumeOrder.OLDEST, null, 1000); } @@ -291,9 +362,12 @@ private void templateTestForLargeNumberOfFiles(ConsumeOrder order, int N) throws IOException { File dir = null; try { - dir = new File("target/test/work/" + this.getClass().getSimpleName()+ "_large"); + dir = new File( + "target/test/work/" + this.getClass().getSimpleName() + + "_large"); Files.createParentDirs(new File(dir, "dummy")); - ReliableEventReader reader = new ReliableSpoolingFileEventReader.Builder() + ReliableEventReader reader + = new ReliableSpoolingFileEventReader.Builder() .spoolDirectory(dir).consumeOrder(order).build(); Map> expected; if (comparator == null) { @@ -328,8 +402,10 @@ private void templateTestForLargeNumberOfFiles(ConsumeOrder order, if (order == ConsumeOrder.RANDOM) { Assert.assertTrue(expectedList.remove(new String(e.getBody()))); } else { - Assert.assertEquals(((ArrayList)expectedList).get(0), new String(e.getBody())); - ((ArrayList)expectedList).remove(0); + Assert.assertEquals( + ((ArrayList) expectedList).get(0), + new String(e.getBody())); + ((ArrayList) expectedList).remove(0); } } reader.commit(); From a94594dd2c5cb980bc6f82b1fa606a922986569e Mon Sep 17 00:00:00 2001 From: Mike Percy Date: Mon, 28 Apr 2014 17:14:40 -0700 Subject: [PATCH 099/341] FLUME-2357. HDFS sink should retry closing files that previously had close errors (Hari Shreedharan via Mike Percy) --- flume-ng-doc/sphinx/FlumeUserGuide.rst | 7 + .../flume/sink/hdfs/AbstractHDFSWriter.java | 51 ----- .../apache/flume/sink/hdfs/BucketWriter.java | 182 +++++++++++++++--- .../sink/hdfs/HDFSCompressedDataStream.java | 3 +- .../flume/sink/hdfs/HDFSDataStream.java | 26 ++- .../apache/flume/sink/hdfs/HDFSEventSink.java | 51 ++++- .../flume/sink/hdfs/HDFSSequenceFile.java | 3 +- .../flume/sink/hdfs/MockDataStream.java | 39 ++++ ...eRetryWrapper.java => MockFileSystem.java} | 38 ++-- ...apper.java => MockFsDataOutputStream.java} | 32 +-- .../flume/sink/hdfs/TestBucketWriter.java | 166 +++++++++++----- .../flume/sink/hdfs/TestHDFSEventSink.java | 128 ++++++++++++ .../sink/hdfs/TestUseRawLocalFileSystem.java | 62 ------ 13 files changed, 548 insertions(+), 240 deletions(-) create mode 100644 flume-ng-sinks/flume-hdfs-sink/src/test/java/org/apache/flume/sink/hdfs/MockDataStream.java rename flume-ng-sinks/flume-hdfs-sink/src/test/java/org/apache/flume/sink/hdfs/{MockFileSystemCloseRetryWrapper.java => MockFileSystem.java} (68%) rename flume-ng-sinks/flume-hdfs-sink/src/test/java/org/apache/flume/sink/hdfs/{MockFsDataOutputStreamCloseRetryWrapper.java => MockFsDataOutputStream.java} (58%) diff --git a/flume-ng-doc/sphinx/FlumeUserGuide.rst b/flume-ng-doc/sphinx/FlumeUserGuide.rst index 7b918ed5b9..b24f8af308 100644 --- a/flume-ng-doc/sphinx/FlumeUserGuide.rst +++ b/flume-ng-doc/sphinx/FlumeUserGuide.rst @@ -1597,6 +1597,13 @@ hdfs.roundValue 1 Rounded down to the highest multiple of th hdfs.roundUnit second The unit of the round down value - ``second``, ``minute`` or ``hour``. hdfs.timeZone Local Time Name of the timezone that should be used for resolving the directory path, e.g. America/Los_Angeles. hdfs.useLocalTimeStamp false Use the local time (instead of the timestamp from the event header) while replacing the escape sequences. +hdfs.closeTries 0 Number of times the sink must try to close a file. If set to 1, this sink will not re-try a failed close + (due to, for example, NameNode or DataNode failure), and may leave the file in an open state with a .tmp extension. + If set to 0, the sink will try to close the file until the file is eventually closed + (there is no limit on the number of times it would try). +hdfs.retryInterval 180 Time in seconds between consecutive attempts to close a file. Each close call costs multiple RPC round-trips to the Namenode, + so setting this too low can cause a lot of load on the name node. If set to 0 or less, the sink will not + attempt to close the file if the first attempt fails, and may leave the file open or with a ".tmp" extension. serializer ``TEXT`` Other possible options include ``avro_event`` or the fully-qualified class name of an implementation of the ``EventSerializer.Builder`` interface. diff --git a/flume-ng-sinks/flume-hdfs-sink/src/main/java/org/apache/flume/sink/hdfs/AbstractHDFSWriter.java b/flume-ng-sinks/flume-hdfs-sink/src/main/java/org/apache/flume/sink/hdfs/AbstractHDFSWriter.java index da0466d4e6..043ca6c53c 100644 --- a/flume-ng-sinks/flume-hdfs-sink/src/main/java/org/apache/flume/sink/hdfs/AbstractHDFSWriter.java +++ b/flume-ng-sinks/flume-hdfs-sink/src/main/java/org/apache/flume/sink/hdfs/AbstractHDFSWriter.java @@ -27,7 +27,6 @@ import org.slf4j.Logger; import org.slf4j.LoggerFactory; -import java.io.IOException; import java.io.OutputStream; import java.lang.reflect.InvocationTargetException; import java.lang.reflect.Method; @@ -226,54 +225,4 @@ private Method reflectGetDefaultReplication(FileSystem fileSystem) { } return m; } - - /** - * This will - * @param outputStream - * @throws IOException - */ - protected void closeHDFSOutputStream(OutputStream outputStream) - throws IOException { - try { - outputStream.close(); - - if (numberOfCloseRetries > 0) { - try { - Method isFileClosedMethod = getIsFileClosedMethod(); - int closeAttemptsMade = 0; - if (isFileClosedMethod != null) { - while (closeAttemptsMade < numberOfCloseRetries.intValue() && - Boolean.FALSE.equals(isFileClosedMethod.invoke(fs, destPath))) { - closeAttemptsMade++; - logger.debug("Waiting: '" + timeBetweenCloseRetries + "' before retry close"); - Thread.sleep(timeBetweenCloseRetries); - try { - outputStream.close(); - } catch (IOException e) { - logger.error("Unable to close HDFS file: '" + destPath + "'"); - } - } - if (closeAttemptsMade == numberOfCloseRetries.intValue()) { - logger.warn("Failed to close '" + destPath + "' is " + - numberOfCloseRetries + " retries, over " + (timeBetweenCloseRetries * numberOfCloseRetries) + " millseconds"); - } - } - } catch (Exception e) { - logger.error("Failed to close '" + destPath + "' is " + - numberOfCloseRetries + " retries, over " + (timeBetweenCloseRetries * numberOfCloseRetries) + " millseconds", e); - } - } - } catch (IOException e) { - logger.error("Unable to close HDFS file: '" + destPath + "'"); - } - } - - private Method getIsFileClosedMethod() { - try { - return fs.getClass().getMethod("isFileClosed", Path.class); - } catch (Exception e) { - return null; - } - } - } diff --git a/flume-ng-sinks/flume-hdfs-sink/src/main/java/org/apache/flume/sink/hdfs/BucketWriter.java b/flume-ng-sinks/flume-hdfs-sink/src/main/java/org/apache/flume/sink/hdfs/BucketWriter.java index e82d13df05..fba3f66468 100644 --- a/flume-ng-sinks/flume-hdfs-sink/src/main/java/org/apache/flume/sink/hdfs/BucketWriter.java +++ b/flume-ng-sinks/flume-hdfs-sink/src/main/java/org/apache/flume/sink/hdfs/BucketWriter.java @@ -19,6 +19,7 @@ package org.apache.flume.sink.hdfs; import java.io.IOException; +import java.lang.reflect.Method; import java.security.PrivilegedExceptionAction; import java.util.concurrent.Callable; import java.util.concurrent.CancellationException; @@ -29,6 +30,7 @@ import java.util.concurrent.ScheduledFuture; import java.util.concurrent.TimeUnit; import java.util.concurrent.TimeoutException; +import java.util.concurrent.atomic.AtomicInteger; import java.util.concurrent.atomic.AtomicLong; import com.google.common.annotations.VisibleForTesting; @@ -63,8 +65,9 @@ class BucketWriter { * This lock ensures that only one thread can open a file at a time. */ private static final Integer staticLock = new Integer(1); + private Method isClosedMethod = null; - private final HDFSWriter writer; + private HDFSWriter writer; private final long rollInterval; private final long rollSize; private final long rollCount; @@ -102,11 +105,16 @@ class BucketWriter { private final ExecutorService callTimeoutPool; private final int maxConsecUnderReplRotations = 30; // make this config'able? + private boolean mockFsInjected = false; + private Clock clock = new SystemClock(); + private final long retryInterval; + private final int maxCloseTries; // flag that the bucket writer was closed due to idling and thus shouldn't be // reopened. Not ideal, but avoids internals of owners protected boolean closed = false; + AtomicInteger closeTries = new AtomicInteger(0); BucketWriter(long rollInterval, long rollSize, long rollCount, long batchSize, Context context, String filePath, String fileName, String inUsePrefix, @@ -115,7 +123,8 @@ class BucketWriter { ScheduledExecutorService timedRollerPool, UserGroupInformation user, SinkCounter sinkCounter, int idleTimeout, WriterCallback onCloseCallback, String onCloseCallbackPath, long callTimeout, - ExecutorService callTimeoutPool) { + ExecutorService callTimeoutPool, long retryInterval, + int maxCloseTries) { this.rollInterval = rollInterval; this.rollSize = rollSize; this.rollCount = rollCount; @@ -138,11 +147,24 @@ class BucketWriter { this.callTimeoutPool = callTimeoutPool; fileExtensionCounter = new AtomicLong(clock.currentTimeMillis()); + this.retryInterval = retryInterval; + this.maxCloseTries = maxCloseTries; isOpen = false; isUnderReplicated = false; this.writer.configure(context); } + @VisibleForTesting + void setFileSystem(FileSystem fs) { + this.fileSystem = fs; + mockFsInjected = true; + } + + @VisibleForTesting + void setMockStream(HDFSWriter dataWriter) { + this.writer = dataWriter; + } + /** * Allow methods to act as another user (typically used for HDFS Kerberos) * @param @@ -180,6 +202,25 @@ private void resetCounters() { batchCounter = 0; } + private Method getRefIsClosed() { + try { + return fileSystem.getClass().getMethod("isFileClosed", + Path.class); + } catch (Exception e) { + LOG.warn("isFileClosed is not available in the " + + "version of HDFS being used. Flume will not " + + "attempt to close files if the close fails on " + + "the first attempt",e); + return null; + } + } + + private Boolean isFileClosed(FileSystem fs, + Path tmpFilePath) throws Exception { + return (Boolean)(isClosedMethod.invoke(fs, + tmpFilePath)); + } + /** * open() is called by append() * @throws IOException @@ -223,12 +264,20 @@ private void open() throws IOException, InterruptedException { public Void call() throws Exception { if (codeC == null) { // Need to get reference to FS using above config before underlying - // writer does in order to avoid shutdown hook & IllegalStateExceptions - fileSystem = new Path(bucketPath).getFileSystem(config); + // writer does in order to avoid shutdown hook & + // IllegalStateExceptions + if(!mockFsInjected) { + fileSystem = new Path(bucketPath).getFileSystem( + config); + } writer.open(bucketPath); } else { - // need to get reference to FS before writer does to avoid shutdown hook - fileSystem = new Path(bucketPath).getFileSystem(config); + // need to get reference to FS before writer does to + // avoid shutdown hook + if(!mockFsInjected) { + fileSystem = new Path(bucketPath).getFileSystem( + config); + } writer.open(bucketPath, codeC, compType); } return null; @@ -243,6 +292,7 @@ public Void call() throws Exception { } } } + isClosedMethod = getRefIsClosed(); sinkCounter.incrementConnectionCreatedCount(); resetCounters(); @@ -280,6 +330,71 @@ public Void call() throws Exception { public synchronized void close() throws IOException, InterruptedException { close(false); } + + private CallRunner createCloseCallRunner() { + return new CallRunner() { + private final HDFSWriter localWriter = writer; + @Override + public Void call() throws Exception { + LOG.info("Close tries incremented"); + closeTries.incrementAndGet(); + localWriter.close(); // could block + return null; + } + }; + } + + private Callable createScheduledCloseCallable( + final CallRunner closeCallRunner) { + + return new Callable() { + private final String path = bucketPath; + private final String finalPath = targetPath; + private FileSystem fs = fileSystem; + private boolean closeSuccess = false; + private Path tmpFilePath = new Path(path); + private int closeTries = 1; // one attempt is already done + private final CallRunner closeCall = closeCallRunner; + + @Override + public Void call() throws Exception { + if (closeTries >= maxCloseTries) { + LOG.warn("Unsuccessfully attempted to close " + path + " " + + maxCloseTries + " times. File may be open, " + + "or may not have been renamed." ); + return null; + } + closeTries++; + try { + if (!closeSuccess) { + if (isClosedMethod == null) { + LOG.debug("isFileClosed method is not available in " + + "the version of HDFS client being used. " + + "Not attempting to close file again"); + return null; + } + if (!isFileClosed(fs, tmpFilePath)) { + callWithTimeout(closeCall); + } + // It is possible rename failing causes this thread + // to get rescheduled. In that case, + // don't check with NN if close succeeded as we know + // it did. This helps avoid an unnecessary RPC call. + closeSuccess = true; + } + renameBucket(path, finalPath, fs); + } catch (Exception e) { + LOG.warn("Closing file: " + path + " failed. Will " + + "retry again in " + retryInterval + " seconds.", e); + timedRollerPool.schedule(this, retryInterval, + TimeUnit.SECONDS); + return null; + } + return null; + } + }; + + } /** * Close the file handle and rename the temp file to the permanent filename. * Safe to call multiple times. Logs HDFSWriter.close() exceptions. @@ -290,21 +405,23 @@ public synchronized void close(boolean callCloseCallback) throws IOException, InterruptedException { checkAndThrowInterruptedException(); flush(); - LOG.debug("Closing {}", bucketPath); + boolean failedToClose = false; + LOG.info("Closing {}", bucketPath); + CallRunner closeCallRunner = createCloseCallRunner(); if (isOpen) { try { - callWithTimeout(new CallRunner() { - @Override - public Void call() throws Exception { - writer.close(); // could block - return null; - } - }); + callWithTimeout(closeCallRunner); sinkCounter.incrementConnectionClosedCount(); } catch (IOException e) { - LOG.warn("failed to close() HDFSWriter for file (" + bucketPath + + LOG.warn( + "failed to close() HDFSWriter for file (" + bucketPath + "). Exception follows.", e); sinkCounter.incrementConnectionFailedCount(); + failedToClose = true; + final Callable scheduledClose = + createScheduledCloseCallable(closeCallRunner); + timedRollerPool.schedule(scheduledClose, retryInterval, + TimeUnit.SECONDS); } isOpen = false; } else { @@ -322,14 +439,15 @@ public Void call() throws Exception { idleFuture = null; } - if (bucketPath != null && fileSystem != null) { - renameBucket(); // could block or throw IOException - fileSystem = null; + // Don't rename file if this failed to close + if (bucketPath != null && fileSystem != null && !failedToClose) { + // could block or throw IOException + renameBucket(bucketPath, targetPath, fileSystem); } if (callCloseCallback) { runCloseAction(); closed = true; - } + } } /** @@ -525,7 +643,18 @@ private boolean shouldRotate() { /** * Rename bucketPath file from .tmp to permanent location. */ - private void renameBucket() throws IOException, InterruptedException { + // When this bucket writer is rolled based on rollCount or + // rollSize, the same instance is reused for the new file. But if + // the previous file was not closed/renamed, + // the bucket writer fields no longer point to it and hence need + // to be passed in from the thread attempting to close it. Even + // when the bucket writer is closed due to close timeout, + // this method can get called from the scheduled thread so the + // file gets closed later - so an implicit reference to this + // bucket writer would still be alive in the Callable instance. + private void renameBucket(String bucketPath, + String targetPath, final FileSystem fs) throws IOException, + InterruptedException { if(bucketPath.equals(targetPath)) { return; } @@ -533,12 +662,12 @@ private void renameBucket() throws IOException, InterruptedException { final Path srcPath = new Path(bucketPath); final Path dstPath = new Path(targetPath); - callWithTimeout(new CallRunner() { + callWithTimeout(new CallRunner() { @Override - public Object call() throws Exception { - if(fileSystem.exists(srcPath)) { // could block + public Void call() throws Exception { + if (fs.exists(srcPath)) { // could block LOG.info("Renaming " + srcPath + " to " + dstPath); - fileSystem.rename(srcPath, dstPath); // could block + fs.rename(srcPath, dstPath); // could block } return null; } @@ -600,9 +729,8 @@ public T run() throws Exception { } catch (TimeoutException eT) { future.cancel(true); sinkCounter.incrementConnectionFailedCount(); - throw new IOException("Callable timed out after " + callTimeout + " ms" + - " on file: " + bucketPath, - eT); + throw new IOException("Callable timed out after " + + callTimeout + " ms" + " on file: " + bucketPath, eT); } catch (ExecutionException e1) { sinkCounter.incrementConnectionFailedCount(); Throwable cause = e1.getCause(); diff --git a/flume-ng-sinks/flume-hdfs-sink/src/main/java/org/apache/flume/sink/hdfs/HDFSCompressedDataStream.java b/flume-ng-sinks/flume-hdfs-sink/src/main/java/org/apache/flume/sink/hdfs/HDFSCompressedDataStream.java index 551854779f..fe857c39aa 100644 --- a/flume-ng-sinks/flume-hdfs-sink/src/main/java/org/apache/flume/sink/hdfs/HDFSCompressedDataStream.java +++ b/flume-ng-sinks/flume-hdfs-sink/src/main/java/org/apache/flume/sink/hdfs/HDFSCompressedDataStream.java @@ -147,8 +147,7 @@ public void close() throws IOException { } fsOut.flush(); fsOut.sync(); - closeHDFSOutputStream(cmpOut); - + cmpOut.close(); unregisterCurrentStream(); } diff --git a/flume-ng-sinks/flume-hdfs-sink/src/main/java/org/apache/flume/sink/hdfs/HDFSDataStream.java b/flume-ng-sinks/flume-hdfs-sink/src/main/java/org/apache/flume/sink/hdfs/HDFSDataStream.java index e20d1eeb47..6fa12eb315 100644 --- a/flume-ng-sinks/flume-hdfs-sink/src/main/java/org/apache/flume/sink/hdfs/HDFSDataStream.java +++ b/flume-ng-sinks/flume-hdfs-sink/src/main/java/org/apache/flume/sink/hdfs/HDFSDataStream.java @@ -19,6 +19,8 @@ package org.apache.flume.sink.hdfs; import java.io.IOException; + +import com.google.common.annotations.VisibleForTesting; import org.apache.flume.Context; import org.apache.flume.Event; import org.apache.flume.serialization.EventSerializer; @@ -57,11 +59,15 @@ public void configure(Context context) { + useRawLocalFileSystem); } - @Override - public void open(String filePath) throws IOException { - Configuration conf = new Configuration(); - Path dstPath = new Path(filePath); - FileSystem hdfs = dstPath.getFileSystem(conf); + @VisibleForTesting + protected FileSystem getDfs(Configuration conf, + Path dstPath) throws IOException{ + return dstPath.getFileSystem(conf); + } + + protected void doOpen(Configuration conf, + Path dstPath, FileSystem hdfs) throws + IOException { if(useRawLocalFileSystem) { if(hdfs instanceof LocalFileSystem) { hdfs = ((LocalFileSystem)hdfs).getRaw(); @@ -99,6 +105,14 @@ public void open(String filePath) throws IOException { } } + @Override + public void open(String filePath) throws IOException { + Configuration conf = new Configuration(); + Path dstPath = new Path(filePath); + FileSystem hdfs = getDfs(conf, dstPath); + doOpen(conf, dstPath, hdfs); + } + @Override public void open(String filePath, CompressionCodec codec, CompressionType cType) throws IOException { @@ -123,7 +137,7 @@ public void close() throws IOException { serializer.beforeClose(); outStream.flush(); outStream.sync(); - closeHDFSOutputStream(outStream); + outStream.close(); unregisterCurrentStream(); } diff --git a/flume-ng-sinks/flume-hdfs-sink/src/main/java/org/apache/flume/sink/hdfs/HDFSEventSink.java b/flume-ng-sinks/flume-hdfs-sink/src/main/java/org/apache/flume/sink/hdfs/HDFSEventSink.java index 4ea78c1d4b..4f3b3f064f 100644 --- a/flume-ng-sinks/flume-hdfs-sink/src/main/java/org/apache/flume/sink/hdfs/HDFSEventSink.java +++ b/flume-ng-sinks/flume-hdfs-sink/src/main/java/org/apache/flume/sink/hdfs/HDFSEventSink.java @@ -46,6 +46,7 @@ import org.apache.flume.instrumentation.SinkCounter; import org.apache.flume.sink.AbstractSink; import org.apache.hadoop.conf.Configuration; +import org.apache.hadoop.fs.FileSystem; import org.apache.hadoop.io.SequenceFile.CompressionType; import org.apache.hadoop.io.compress.CompressionCodec; import org.apache.hadoop.io.compress.CompressionCodecFactory; @@ -79,6 +80,10 @@ public interface WriterCallback { private static final long defaultBatchSize = 100; private static final String defaultFileType = HDFSWriterFactory.SequenceFileType; private static final int defaultMaxOpenFiles = 5000; + // Time between close retries, in seconds + private static final long defaultRetryInterval = 180; + // Retry forever. + private static final int defaultTryCount = Integer.MAX_VALUE; /** * Default length of time we wait for blocking BucketWriter calls @@ -140,7 +145,12 @@ public interface WriterCallback { private volatile int idleTimeout; private Clock clock; + private FileSystem mockFs; + private HDFSWriter mockWriter; private final Object sfWritersLock = new Object(); + private long retryInterval; + private int tryCount; + /* * Extended Java LinkedHashMap for open file handle LRU queue. @@ -218,6 +228,21 @@ public void configure(Context context) { kerbConfPrincipal = context.getString("hdfs.kerberosPrincipal", ""); kerbKeytab = context.getString("hdfs.kerberosKeytab", ""); proxyUserName = context.getString("hdfs.proxyUser", ""); + tryCount = context.getInteger("hdfs.closeTries", defaultTryCount); + if(tryCount <= 0) { + LOG.warn("Retry count value : " + tryCount + " is not " + + "valid. The sink will try to close the file until the file " + + "is eventually closed."); + tryCount = defaultTryCount; + } + retryInterval = context.getLong("hdfs.retryInterval", + defaultRetryInterval); + if(retryInterval <= 0) { + LOG.warn("Retry Interval value: " + retryInterval + " is not " + + "valid. If the first close of a file fails, " + + "it may remain open and will not be renamed."); + tryCount = 1; + } Preconditions.checkArgument(batchSize > 0, "batchSize must be greater than 0"); @@ -453,11 +478,18 @@ public void run(String bucketPath) { private BucketWriter initializeBucketWriter(String realPath, String realName, String lookupPath, HDFSWriter hdfsWriter, WriterCallback closeCallback) { - return new BucketWriter(rollInterval, rollSize, rollCount, + BucketWriter bucketWriter = new BucketWriter(rollInterval, + rollSize, rollCount, batchSize, context, realPath, realName, inUsePrefix, inUseSuffix, suffix, codeC, compType, hdfsWriter, timedRollerPool, proxyTicket, sinkCounter, idleTimeout, closeCallback, - lookupPath, callTimeout, callTimeoutPool); + lookupPath, callTimeout, callTimeoutPool, retryInterval, + tryCount); + if(mockFs != null) { + bucketWriter.setFileSystem(mockFs); + bucketWriter.setMockStream(mockWriter); + } + return bucketWriter; } @Override @@ -716,4 +748,19 @@ public String toString() { void setBucketClock(Clock clock) { BucketPath.setClock(clock); } + + @VisibleForTesting + void setMockFs(FileSystem mockFs) { + this.mockFs = mockFs; + } + + @VisibleForTesting + void setMockWriter(HDFSWriter writer) { + this.mockWriter = writer; + } + + @VisibleForTesting + int getTryCount() { + return tryCount; + } } diff --git a/flume-ng-sinks/flume-hdfs-sink/src/main/java/org/apache/flume/sink/hdfs/HDFSSequenceFile.java b/flume-ng-sinks/flume-hdfs-sink/src/main/java/org/apache/flume/sink/hdfs/HDFSSequenceFile.java index 5fe9f1b52a..26089872c4 100644 --- a/flume-ng-sinks/flume-hdfs-sink/src/main/java/org/apache/flume/sink/hdfs/HDFSSequenceFile.java +++ b/flume-ng-sinks/flume-hdfs-sink/src/main/java/org/apache/flume/sink/hdfs/HDFSSequenceFile.java @@ -116,8 +116,7 @@ public void sync() throws IOException { @Override public void close() throws IOException { writer.close(); - closeHDFSOutputStream(outStream); - + outStream.close(); unregisterCurrentStream(); } } diff --git a/flume-ng-sinks/flume-hdfs-sink/src/test/java/org/apache/flume/sink/hdfs/MockDataStream.java b/flume-ng-sinks/flume-hdfs-sink/src/test/java/org/apache/flume/sink/hdfs/MockDataStream.java new file mode 100644 index 0000000000..f0c6e7e005 --- /dev/null +++ b/flume-ng-sinks/flume-hdfs-sink/src/test/java/org/apache/flume/sink/hdfs/MockDataStream.java @@ -0,0 +1,39 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, + * software distributed under the License is distributed on an + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + * KIND, either express or implied. See the License for the + * specific language governing permissions and limitations + * under the License. + */ +package org.apache.flume.sink.hdfs; + +import org.apache.hadoop.conf.Configuration; +import org.apache.hadoop.fs.FileSystem; +import org.apache.hadoop.fs.Path; + +import java.io.IOException; + +class MockDataStream extends HDFSDataStream { + private final FileSystem fs; + + MockDataStream(FileSystem fs) { + this.fs = fs; + } + @Override + protected FileSystem getDfs(Configuration conf, + Path dstPath) throws IOException{ + return fs; + } + +} diff --git a/flume-ng-sinks/flume-hdfs-sink/src/test/java/org/apache/flume/sink/hdfs/MockFileSystemCloseRetryWrapper.java b/flume-ng-sinks/flume-hdfs-sink/src/test/java/org/apache/flume/sink/hdfs/MockFileSystem.java similarity index 68% rename from flume-ng-sinks/flume-hdfs-sink/src/test/java/org/apache/flume/sink/hdfs/MockFileSystemCloseRetryWrapper.java rename to flume-ng-sinks/flume-hdfs-sink/src/test/java/org/apache/flume/sink/hdfs/MockFileSystem.java index b5d89e6bb2..ca4f852504 100644 --- a/flume-ng-sinks/flume-hdfs-sink/src/test/java/org/apache/flume/sink/hdfs/MockFileSystemCloseRetryWrapper.java +++ b/flume-ng-sinks/flume-hdfs-sink/src/test/java/org/apache/flume/sink/hdfs/MockFileSystem.java @@ -31,32 +31,27 @@ import org.slf4j.Logger; import org.slf4j.LoggerFactory; -public class MockFileSystemCloseRetryWrapper extends FileSystem{ +public class MockFileSystem extends FileSystem { private static final Logger logger = - LoggerFactory.getLogger(MockFileSystemCloseRetryWrapper.class); + LoggerFactory.getLogger(MockFileSystem.class); FileSystem fs; int numberOfClosesRequired; - boolean throwExceptionsOfFailedClose; - MockFsDataOutputStreamCloseRetryWrapper latestOutputStream; + MockFsDataOutputStream latestOutputStream; - public MockFileSystemCloseRetryWrapper (FileSystem fs, - int numberOfClosesRequired, boolean throwExceptionsOfFailedClose) { + public MockFileSystem(FileSystem fs, + int numberOfClosesRequired) { this.fs = fs; - this.throwExceptionsOfFailedClose = throwExceptionsOfFailedClose; this.numberOfClosesRequired = numberOfClosesRequired; } - public MockFsDataOutputStreamCloseRetryWrapper getLastMockOutputStream() { - return latestOutputStream; - } - @Override public FSDataOutputStream append(Path arg0, int arg1, Progressable arg2) throws IOException { - latestOutputStream = new MockFsDataOutputStreamCloseRetryWrapper(fs.append(arg0, arg1, arg2), numberOfClosesRequired, throwExceptionsOfFailedClose); + latestOutputStream = new MockFsDataOutputStream( + fs.append(arg0, arg1, arg2), numberOfClosesRequired); return latestOutputStream; } @@ -64,16 +59,17 @@ public FSDataOutputStream append(Path arg0, int arg1, Progressable arg2) @Override public FSDataOutputStream create(Path arg0) throws IOException { //throw new IOException ("HI there2"); - latestOutputStream = new MockFsDataOutputStreamCloseRetryWrapper(fs.create(arg0), numberOfClosesRequired, throwExceptionsOfFailedClose); + latestOutputStream = new MockFsDataOutputStream( + fs.create(arg0), numberOfClosesRequired); return latestOutputStream; } @Override - public FSDataOutputStream create(Path arg0, FsPermission arg1, boolean arg2, - int arg3, short arg4, long arg5, Progressable arg6) throws IOException { - throw new IOException ("Not a real file system"); - //return new MockFsDataOutputStreamCloseRetryWrapper(fs.create(arg0, arg1, arg2, arg3, arg4, arg5, arg6), numberOfClosesRequired, throwExceptionsOfFailedClose); + public FSDataOutputStream create(Path arg0, FsPermission arg1, + boolean arg2, int arg3, short arg4, long arg5, Progressable arg6) + throws IOException { + throw new IOException("Not a real file system"); } @Override @@ -132,9 +128,11 @@ public void setWorkingDirectory(Path arg0) { public boolean isFileClosed(Path path) { - logger.info("isFileClosed: '" + latestOutputStream.getCurrentCloseAttempts() + "' , '" + numberOfClosesRequired + "'"); - - return latestOutputStream.getCurrentCloseAttempts() >= numberOfClosesRequired || numberOfClosesRequired == 0; + logger.info("isFileClosed: '" + + latestOutputStream.getCurrentCloseAttempts() + "' , '" + + numberOfClosesRequired + "'"); + return latestOutputStream.getCurrentCloseAttempts() >= + numberOfClosesRequired || numberOfClosesRequired == 0; } diff --git a/flume-ng-sinks/flume-hdfs-sink/src/test/java/org/apache/flume/sink/hdfs/MockFsDataOutputStreamCloseRetryWrapper.java b/flume-ng-sinks/flume-hdfs-sink/src/test/java/org/apache/flume/sink/hdfs/MockFsDataOutputStream.java similarity index 58% rename from flume-ng-sinks/flume-hdfs-sink/src/test/java/org/apache/flume/sink/hdfs/MockFsDataOutputStreamCloseRetryWrapper.java rename to flume-ng-sinks/flume-hdfs-sink/src/test/java/org/apache/flume/sink/hdfs/MockFsDataOutputStream.java index 1d8c140bb7..5bbacaeaeb 100644 --- a/flume-ng-sinks/flume-hdfs-sink/src/test/java/org/apache/flume/sink/hdfs/MockFsDataOutputStreamCloseRetryWrapper.java +++ b/flume-ng-sinks/flume-hdfs-sink/src/test/java/org/apache/flume/sink/hdfs/MockFsDataOutputStream.java @@ -18,52 +18,40 @@ package org.apache.flume.sink.hdfs; import java.io.IOException; -import java.io.OutputStream; import org.apache.hadoop.fs.FSDataOutputStream; -import org.apache.hadoop.fs.FileSystem.Statistics; import org.slf4j.Logger; import org.slf4j.LoggerFactory; -public class MockFsDataOutputStreamCloseRetryWrapper extends FSDataOutputStream{ +public class MockFsDataOutputStream extends FSDataOutputStream{ private static final Logger logger = - LoggerFactory.getLogger(MockFsDataOutputStreamCloseRetryWrapper.class); + LoggerFactory.getLogger(MockFsDataOutputStream.class); int currentCloseAttempts = 0; int numberOfClosesRequired; - boolean throwExceptionsOfFailedClose; - public MockFsDataOutputStreamCloseRetryWrapper(FSDataOutputStream wrapMe, - int numberOfClosesRequired, boolean throwExceptionsOfFailedClose) + public MockFsDataOutputStream(FSDataOutputStream wrapMe, + int numberOfClosesRequired) throws IOException { super(wrapMe.getWrappedStream(), null); this.numberOfClosesRequired = numberOfClosesRequired; - this.throwExceptionsOfFailedClose = throwExceptionsOfFailedClose; - - } - - public MockFsDataOutputStreamCloseRetryWrapper(OutputStream out, - Statistics stats) throws IOException { - super(out, stats); } @Override public void close() throws IOException { currentCloseAttempts++; - logger.info("Attempting to Close: '" + currentCloseAttempts + "' of '" + numberOfClosesRequired + "'"); - if (currentCloseAttempts > numberOfClosesRequired || numberOfClosesRequired == 0) { + logger.info( + "Attempting to Close: '" + currentCloseAttempts + "' of '" + + numberOfClosesRequired + "'"); + if (currentCloseAttempts >= numberOfClosesRequired || + numberOfClosesRequired == 0) { logger.info("closing file"); super.close(); } else { - if (throwExceptionsOfFailedClose) { - logger.info("no closed and throwing exception"); - throw new IOException("MockIOException"); - } else { - logger.info("no closed and doing nothing"); - } + throw new IOException("MockIOException"); } } diff --git a/flume-ng-sinks/flume-hdfs-sink/src/test/java/org/apache/flume/sink/hdfs/TestBucketWriter.java b/flume-ng-sinks/flume-hdfs-sink/src/test/java/org/apache/flume/sink/hdfs/TestBucketWriter.java index b7cc586cd8..bcb912ff5b 100644 --- a/flume-ng-sinks/flume-hdfs-sink/src/test/java/org/apache/flume/sink/hdfs/TestBucketWriter.java +++ b/flume-ng-sinks/flume-hdfs-sink/src/test/java/org/apache/flume/sink/hdfs/TestBucketWriter.java @@ -20,6 +20,7 @@ import java.io.File; import java.io.IOException; +import java.util.Calendar; import java.util.concurrent.Executors; import java.util.concurrent.ScheduledExecutorService; import java.util.concurrent.TimeUnit; @@ -30,6 +31,9 @@ import org.apache.flume.Event; import org.apache.flume.event.EventBuilder; import org.apache.flume.instrumentation.SinkCounter; +import org.apache.hadoop.conf.Configuration; +import org.apache.hadoop.fs.FileSystem; +import org.apache.hadoop.fs.Path; import org.apache.hadoop.io.SequenceFile; import org.apache.hadoop.io.SequenceFile.CompressionType; import org.apache.hadoop.io.compress.CompressionCodec; @@ -70,7 +74,7 @@ public void testEventCountingRoller() throws IOException, InterruptedException { "/tmp", "file", "", ".tmp", null, null, SequenceFile.CompressionType.NONE, hdfsWriter, timedRollerPool, null, new SinkCounter("test-bucket-writer-" + System.currentTimeMillis()), 0, - null, null, 30000, Executors.newSingleThreadExecutor()); + null, null, 30000, Executors.newSingleThreadExecutor(), 0, 0); Event e = EventBuilder.withBody("foo", Charsets.UTF_8); for (int i = 0; i < 1000; i++) { @@ -90,11 +94,12 @@ public void testEventCountingRoller() throws IOException, InterruptedException { public void testSizeRoller() throws IOException, InterruptedException { int maxBytes = 300; MockHDFSWriter hdfsWriter = new MockHDFSWriter(); - BucketWriter bucketWriter = new BucketWriter(0, maxBytes, 0, 0, ctx, - "/tmp", "file", "", ".tmp", null, null, SequenceFile.CompressionType.NONE, - hdfsWriter, timedRollerPool, null, - new SinkCounter("test-bucket-writer-" + System.currentTimeMillis()), - 0, null, null, 30000, Executors.newSingleThreadExecutor()); + BucketWriter bucketWriter = new BucketWriter(0, maxBytes, 0, 0, + ctx, "/tmp", "file", "", ".tmp", null, null, + SequenceFile.CompressionType.NONE, hdfsWriter,timedRollerPool, + null, new SinkCounter("test-bucket-writer-" + + System.currentTimeMillis()),0, null, null, 30000, + Executors.newSingleThreadExecutor(), 0, 0); Event e = EventBuilder.withBody("foo", Charsets.UTF_8); for (int i = 0; i < 1000; i++) { @@ -126,7 +131,7 @@ public void testIntervalRoller() throws IOException, InterruptedException { public void run(String filePath) { calledBack.set(true); } - }, null, 30000, Executors.newSingleThreadExecutor()); + }, null, 30000, Executors.newSingleThreadExecutor(), 0, 0); Event e = EventBuilder.withBody("foo", Charsets.UTF_8); long startNanos = System.nanoTime(); @@ -143,8 +148,9 @@ public void run(String filePath) { bucketWriter = new BucketWriter(ROLL_INTERVAL, 0, 0, 0, ctx, "/tmp", "file", "", ".tmp", null, null, SequenceFile.CompressionType.NONE, hdfsWriter, timedRollerPool, null, - new SinkCounter("test-bucket-writer-" + System.currentTimeMillis()), - 0, null, null, 30000, Executors.newSingleThreadExecutor()); + new SinkCounter("test-bucket-writer-" + + System.currentTimeMillis()), 0, null, null, 30000, + Executors.newSingleThreadExecutor(), 0, 0); // write one more event (to reopen a new file so we will roll again later) bucketWriter.append(e); @@ -221,11 +227,13 @@ public void append(Event e) throws IOException { String path = tmpFile.getParent(); String name = tmpFile.getName(); - BucketWriter bucketWriter = new BucketWriter(ROLL_INTERVAL, 0, 0, 0, ctx, - path, name, "", ".tmp", null, null, SequenceFile.CompressionType.NONE, hdfsWriter, - timedRollerPool, null, - new SinkCounter("test-bucket-writer-" + System.currentTimeMillis()), - 0, null, null, 30000, Executors.newSingleThreadExecutor()); + BucketWriter bucketWriter = new BucketWriter(ROLL_INTERVAL, 0, 0, + 0, ctx, path, name, "", ".tmp", null, null, + SequenceFile.CompressionType.NONE, hdfsWriter, + timedRollerPool, null, new SinkCounter("test-bucket-writer-" + + System.currentTimeMillis()), + 0, null, null, 30000, Executors.newSingleThreadExecutor(), + 0, 0); Event e = EventBuilder.withBody("foo", Charsets.UTF_8); for (int i = 0; i < NUM_EVENTS - 1; i++) { @@ -244,11 +252,12 @@ public void testFileSuffixNotGiven() throws IOException, InterruptedException { final String suffix = null; MockHDFSWriter hdfsWriter = new MockHDFSWriter(); - BucketWriter bucketWriter = new BucketWriter(ROLL_INTERVAL, 0, 0, 0, ctx, - "/tmp", "file", "", ".tmp", suffix, null, SequenceFile.CompressionType.NONE, hdfsWriter, - timedRollerPool, null, - new SinkCounter("test-bucket-writer-" + System.currentTimeMillis()), - 0, null, null, 30000, Executors.newSingleThreadExecutor()); + BucketWriter bucketWriter = new BucketWriter(ROLL_INTERVAL, 0, + 0, 0, ctx, "/tmp", "file", "", ".tmp", suffix, null, + SequenceFile.CompressionType.NONE, hdfsWriter, + timedRollerPool, null, new SinkCounter("test-bucket-writer-" + + System.currentTimeMillis()), 0, null, null, 30000, + Executors.newSingleThreadExecutor(), 0, 0); // Need to override system time use for test so we know what to expect final long testTime = System.currentTimeMillis(); @@ -270,12 +279,13 @@ public void testFileSuffixGiven() throws IOException, InterruptedException { final int ROLL_INTERVAL = 1000; // seconds. Make sure it doesn't change in course of test final String suffix = ".avro"; - MockHDFSWriter hdfsWriter = new MockHDFSWriter(); - BucketWriter bucketWriter = new BucketWriter(ROLL_INTERVAL, 0, 0, 0, ctx, - "/tmp", "file", "", ".tmp", suffix, null, SequenceFile.CompressionType.NONE, hdfsWriter, - timedRollerPool, null, - new SinkCounter("test-bucket-writer-" + System.currentTimeMillis()), - 0, null, null, 30000, Executors.newSingleThreadExecutor()); + MockHDFSWriter hdfsWriter = new MockHDFSWriter(); + BucketWriter bucketWriter = new BucketWriter(ROLL_INTERVAL, 0, + 0, 0, ctx, "/tmp", "file", "", ".tmp", suffix, null, + SequenceFile.CompressionType.NONE, hdfsWriter, + timedRollerPool, null, new SinkCounter( + "test-bucket-writer-" + System.currentTimeMillis()), 0, + null, null, 30000, Executors.newSingleThreadExecutor(), 0, 0); // Need to override system time use for test so we know what to expect @@ -291,7 +301,8 @@ public long currentTimeMillis() { Event e = EventBuilder.withBody("foo", Charsets.UTF_8); bucketWriter.append(e); - Assert.assertTrue("Incorrect suffix", hdfsWriter.getOpenedFilePath().endsWith(Long.toString(testTime+1) + suffix + ".tmp")); + Assert.assertTrue("Incorrect suffix", hdfsWriter.getOpenedFilePath().endsWith( + Long.toString(testTime + 1) + suffix + ".tmp")); } @Test @@ -301,12 +312,14 @@ public void testFileSuffixCompressed() final String suffix = ".foo"; MockHDFSWriter hdfsWriter = new MockHDFSWriter(); - BucketWriter bucketWriter = new BucketWriter(ROLL_INTERVAL, 0, 0, 0, ctx, - "/tmp", "file", "", ".tmp", suffix, HDFSEventSink.getCodec("gzip"), - SequenceFile.CompressionType.BLOCK, hdfsWriter, - timedRollerPool, null, - new SinkCounter("test-bucket-writer-" + System.currentTimeMillis()), - 0, null, null, 30000, Executors.newSingleThreadExecutor()); + BucketWriter bucketWriter = new BucketWriter(ROLL_INTERVAL, 0, 0, + 0, ctx, "/tmp", "file", "", ".tmp", suffix, + HDFSEventSink.getCodec("gzip"), + SequenceFile.CompressionType.BLOCK, hdfsWriter, + timedRollerPool, null, new SinkCounter("test-bucket-writer-" + + System.currentTimeMillis()), 0, null, null, 30000, + Executors.newSingleThreadExecutor(), 0, 0 + ); // Need to override system time use for test so we know what to expect final long testTime = System.currentTimeMillis(); @@ -332,11 +345,12 @@ public void testInUsePrefix() throws IOException, InterruptedException { MockHDFSWriter hdfsWriter = new MockHDFSWriter(); HDFSTextSerializer formatter = new HDFSTextSerializer(); - BucketWriter bucketWriter = new BucketWriter(ROLL_INTERVAL, 0, 0, 0, ctx, - "/tmp", "file", PREFIX, ".tmp", null, null, SequenceFile.CompressionType.NONE, hdfsWriter, - timedRollerPool, null, - new SinkCounter("test-bucket-writer-" + System.currentTimeMillis()), - 0, null, null, 30000, Executors.newSingleThreadExecutor()); + BucketWriter bucketWriter = new BucketWriter(ROLL_INTERVAL, 0, 0, + 0, ctx, "/tmp", "file", PREFIX, ".tmp", null, null, + SequenceFile.CompressionType.NONE, hdfsWriter, + timedRollerPool, null, new SinkCounter( + "test-bucket-writer-" + System.currentTimeMillis()), 0, + null, null, 30000, Executors.newSingleThreadExecutor(), 0, 0); Event e = EventBuilder.withBody("foo", Charsets.UTF_8); bucketWriter.append(e); @@ -351,11 +365,12 @@ public void testInUseSuffix() throws IOException, InterruptedException { MockHDFSWriter hdfsWriter = new MockHDFSWriter(); HDFSTextSerializer serializer = new HDFSTextSerializer(); - BucketWriter bucketWriter = new BucketWriter(ROLL_INTERVAL, 0, 0, 0, ctx, - "/tmp", "file", "", SUFFIX, null, null, SequenceFile.CompressionType.NONE, hdfsWriter, - timedRollerPool, null, - new SinkCounter("test-bucket-writer-" + System.currentTimeMillis()), - 0, null, null, 30000, Executors.newSingleThreadExecutor()); + BucketWriter bucketWriter = new BucketWriter(ROLL_INTERVAL, 0, 0, + 0, ctx, "/tmp", "file", "", SUFFIX, null, null, + SequenceFile.CompressionType.NONE, hdfsWriter, + timedRollerPool, null, new SinkCounter( + "test-bucket-writer-" + System.currentTimeMillis()), 0, + null, null, 30000, Executors.newSingleThreadExecutor(), 0, 0); Event e = EventBuilder.withBody("foo", Charsets.UTF_8); bucketWriter.append(e); @@ -370,16 +385,18 @@ public void testCallbackOnClose() throws IOException, InterruptedException { final AtomicBoolean callbackCalled = new AtomicBoolean(false); MockHDFSWriter hdfsWriter = new MockHDFSWriter(); - BucketWriter bucketWriter = new BucketWriter(ROLL_INTERVAL, 0, 0, 0, ctx, - "/tmp", "file", "", SUFFIX, null, null, SequenceFile.CompressionType.NONE, + BucketWriter bucketWriter = new BucketWriter(ROLL_INTERVAL, 0, 0, + 0, ctx, "/tmp", "file", "", SUFFIX, null, null, + SequenceFile.CompressionType.NONE, hdfsWriter, timedRollerPool, null, - new SinkCounter("test-bucket-writer-" + System.currentTimeMillis()), 0, + new SinkCounter( + "test-bucket-writer-" + System.currentTimeMillis()), 0, new HDFSEventSink.WriterCallback() { @Override public void run(String filePath) { callbackCalled.set(true); } - }, "blah", 30000, Executors.newSingleThreadExecutor()); + }, "blah", 30000, Executors.newSingleThreadExecutor(), 0, 0); Event e = EventBuilder.withBody("foo", Charsets.UTF_8); bucketWriter.append(e); @@ -387,4 +404,61 @@ public void run(String filePath) { Assert.assertTrue(callbackCalled.get()); } + + + + @Test + public void testSequenceFileCloseRetries() throws Exception { + SequenceFileCloseRetryCoreTest(1); + SequenceFileCloseRetryCoreTest(5); + SequenceFileCloseRetryCoreTest(2); + + } + + + public void SequenceFileCloseRetryCoreTest(int numberOfClosesRequired) throws Exception { + String hdfsPath = "file:///tmp/flume-test." + + Calendar.getInstance().getTimeInMillis() + "." + + Thread.currentThread().getId(); + + Context context = new Context(); + Configuration conf = new Configuration(); + FileSystem fs = FileSystem.get(conf); + Path dirPath = new Path(hdfsPath); + fs.delete(dirPath, true); + fs.mkdirs(dirPath); + context.put("hdfs.path", hdfsPath); + context.put("hdfs.closeTries", + String.valueOf(numberOfClosesRequired)); + context.put("hdfs.rollCount", "1"); + context.put("hdfs.retryInterval", "1"); + context.put("hdfs.callTimeout", Long.toString(1000)); + MockFileSystem mockFs = new + MockFileSystem(fs, + numberOfClosesRequired); + BucketWriter bucketWriter = new BucketWriter(0, 0, 1, 1, ctx, + hdfsPath, hdfsPath, "singleBucket", ".tmp", null, null, + null, new MockDataStream(mockFs), + timedRollerPool, null, + new SinkCounter( + "test-bucket-writer-" + System.currentTimeMillis()), + 0, null, null, 30000, Executors.newSingleThreadExecutor(), 1, + numberOfClosesRequired); + + bucketWriter.setFileSystem(mockFs); + // At this point, we checked if isFileClosed is available in + // this JVM, so lets make it check again. + Event event = EventBuilder.withBody("test", Charsets.UTF_8); + bucketWriter.append(event); + // This is what triggers the close, so a 2nd append is required :/ + bucketWriter.append(event); + + TimeUnit.SECONDS.sleep(numberOfClosesRequired + 2); + + int expectedNumberOfCloses = numberOfClosesRequired; + Assert.assertTrue("Expected " + expectedNumberOfCloses + " " + + "but got " + bucketWriter.closeTries.get(), + bucketWriter.closeTries.get() == + expectedNumberOfCloses); + } } diff --git a/flume-ng-sinks/flume-hdfs-sink/src/test/java/org/apache/flume/sink/hdfs/TestHDFSEventSink.java b/flume-ng-sinks/flume-hdfs-sink/src/test/java/org/apache/flume/sink/hdfs/TestHDFSEventSink.java index 87918d118e..f29f1f12de 100644 --- a/flume-ng-sinks/flume-hdfs-sink/src/test/java/org/apache/flume/sink/hdfs/TestHDFSEventSink.java +++ b/flume-ng-sinks/flume-hdfs-sink/src/test/java/org/apache/flume/sink/hdfs/TestHDFSEventSink.java @@ -24,9 +24,13 @@ import java.nio.ByteBuffer; import java.nio.charset.CharsetDecoder; import java.util.Calendar; +import java.util.Collection; import java.util.List; +import java.util.Map; import java.util.UUID; +import java.util.concurrent.TimeUnit; +import com.google.common.collect.Maps; import org.apache.avro.file.DataFileStream; import org.apache.avro.generic.GenericData; import org.apache.avro.generic.GenericDatumReader; @@ -43,6 +47,7 @@ import org.apache.flume.Transaction; import org.apache.flume.channel.MemoryChannel; import org.apache.flume.conf.Configurables; +import org.apache.flume.event.EventBuilder; import org.apache.flume.event.SimpleEvent; import org.apache.flume.lifecycle.LifecycleException; import org.apache.hadoop.conf.Configuration; @@ -1317,4 +1322,127 @@ public void testCloseOnIdle() throws IOException, EventDeliveryException, Interr !fList[1].getName().endsWith(".tmp")); fs.close(); } + + private Context getContextForRetryTests() { + Context context = new Context(); + + context.put("hdfs.path", testPath + "/%{retryHeader}"); + context.put("hdfs.filePrefix", "test"); + context.put("hdfs.batchSize", String.valueOf(100)); + context.put("hdfs.fileType", "DataStream"); + context.put("hdfs.serializer", "text"); + context.put("hdfs.closeTries","3"); + context.put("hdfs.rollCount", "1"); + context.put("hdfs.retryInterval", "1"); + return context; + } + + @Test + public void testBadConfigurationForRetryIntervalZero() throws + Exception { + Context context = getContextForRetryTests(); + context.put("hdfs.retryInterval", "0"); + + Configurables.configure(sink, context); + Assert.assertEquals(1, sink.getTryCount()); + } + + @Test + public void testBadConfigurationForRetryIntervalNegative() throws + Exception { + Context context = getContextForRetryTests(); + context.put("hdfs.retryInterval", "-1"); + + Configurables.configure(sink, context); + Assert.assertEquals(1, sink.getTryCount()); + } + @Test + public void testBadConfigurationForRetryCountZero() throws + Exception { + Context context = getContextForRetryTests(); + context.put("hdfs.closeTries" ,"0"); + + Configurables.configure(sink, context); + Assert.assertEquals(Integer.MAX_VALUE, sink.getTryCount()); + } + @Test + public void testBadConfigurationForRetryCountNegative() throws + Exception { + Context context = getContextForRetryTests(); + context.put("hdfs.closeTries" ,"-4"); + + Configurables.configure(sink, context); + Assert.assertEquals(Integer.MAX_VALUE, sink.getTryCount()); + } + @Test + public void testRetryClose() throws InterruptedException, + LifecycleException, + EventDeliveryException, IOException { + + LOG.debug("Starting..."); + String newPath = testPath + "/retryBucket"; + + // clear the test directory + Configuration conf = new Configuration(); + FileSystem fs = FileSystem.get(conf); + Path dirPath = new Path(newPath); + fs.delete(dirPath, true); + fs.mkdirs(dirPath); + MockFileSystem mockFs = new MockFileSystem(fs, 3); + + Context context = getContextForRetryTests(); + Configurables.configure(sink, context); + + Channel channel = new MemoryChannel(); + Configurables.configure(channel, context); + + sink.setChannel(channel); + sink.setMockFs(mockFs); + HDFSWriter hdfsWriter = new MockDataStream(mockFs); + hdfsWriter.configure(context); + sink.setMockWriter(hdfsWriter); + sink.start(); + + // push the event batches into channel + for (int i = 0; i < 2; i++) { + Transaction txn = channel.getTransaction(); + txn.begin(); + Map hdr = Maps.newHashMap(); + hdr.put("retryHeader", "v1"); + + channel.put(EventBuilder.withBody("random".getBytes(), hdr)); + txn.commit(); + txn.close(); + + // execute sink to process the events + sink.process(); + } + // push the event batches into channel + for (int i = 0; i < 2; i++) { + Transaction txn = channel.getTransaction(); + txn.begin(); + Map hdr = Maps.newHashMap(); + hdr.put("retryHeader", "v2"); + channel.put(EventBuilder.withBody("random".getBytes(), hdr)); + txn.commit(); + txn.close(); + // execute sink to process the events + sink.process(); + } + + TimeUnit.SECONDS.sleep(5); //Sleep till all retries are done. + + Collection writers = sink.getSfWriters().values(); + + int totalCloseAttempts = 0; + for(BucketWriter writer: writers) { + LOG.info("Close tries = "+ writer.closeTries.get()); + totalCloseAttempts += writer.closeTries.get(); + } + // stop clears the sfWriters map, so we need to compute the + // close tries count before stopping the sink. + sink.stop(); + Assert.assertEquals(6, totalCloseAttempts); + + } } diff --git a/flume-ng-sinks/flume-hdfs-sink/src/test/java/org/apache/flume/sink/hdfs/TestUseRawLocalFileSystem.java b/flume-ng-sinks/flume-hdfs-sink/src/test/java/org/apache/flume/sink/hdfs/TestUseRawLocalFileSystem.java index 4476530133..f3e7d10a72 100644 --- a/flume-ng-sinks/flume-hdfs-sink/src/test/java/org/apache/flume/sink/hdfs/TestUseRawLocalFileSystem.java +++ b/flume-ng-sinks/flume-hdfs-sink/src/test/java/org/apache/flume/sink/hdfs/TestUseRawLocalFileSystem.java @@ -19,29 +19,15 @@ package org.apache.flume.sink.hdfs; import java.io.File; -import java.io.IOException; -import java.util.concurrent.Executors; -import java.util.concurrent.ScheduledExecutorService; -import java.util.concurrent.TimeUnit; - import org.apache.commons.io.FileUtils; -import org.apache.flume.Clock; import org.apache.flume.Context; import org.apache.flume.Event; import org.apache.flume.event.EventBuilder; -import org.apache.flume.instrumentation.SinkCounter; -import org.apache.hadoop.conf.Configuration; -import org.apache.hadoop.fs.FileSystem; -import org.apache.hadoop.fs.Path; -import org.apache.hadoop.io.SequenceFile; import org.apache.hadoop.io.SequenceFile.CompressionType; -import org.apache.hadoop.io.compress.CompressionCodec; import org.apache.hadoop.io.compress.GzipCodec; import org.junit.After; -import org.junit.AfterClass; import org.junit.Assert; import org.junit.Before; -import org.junit.BeforeClass; import org.junit.Test; import org.slf4j.Logger; import org.slf4j.LoggerFactory; @@ -106,52 +92,4 @@ public void testSequenceFile() throws Exception { Assert.assertTrue(testFile.length() > 0); } - @Test - public void testSequenceFileCloseRetries() throws Exception { - SequenceFileCloseRetryCoreTest(3, 0, false); - SequenceFileCloseRetryCoreTest(3, 1, false); - SequenceFileCloseRetryCoreTest(3, 5, false); - - SequenceFileCloseRetryCoreTest(3, 0, true); - SequenceFileCloseRetryCoreTest(3, 1, true); - SequenceFileCloseRetryCoreTest(3, 5, true); - - SequenceFileCloseRetryCoreTest(3, 2, true); - SequenceFileCloseRetryCoreTest(3, 2, true); - - SequenceFileCloseRetryCoreTest(0, 0, true); - SequenceFileCloseRetryCoreTest(1, 0, true); - } - - - public void SequenceFileCloseRetryCoreTest(int numberOfCloseRetriesToAttempt, int numberOfClosesRequired, boolean throwExceptionsOfFailedClose) throws Exception { - String file = testFile.getCanonicalPath(); - HDFSSequenceFile stream = new HDFSSequenceFile(); - context.put("hdfs.useRawLocalFileSystem", "true"); - context.put("hdfs.closeTries", String.valueOf(numberOfCloseRetriesToAttempt)); - Configuration conf = new Configuration(); - Path dstPath = new Path(file); - MockFileSystemCloseRetryWrapper mockFs = new MockFileSystemCloseRetryWrapper(dstPath.getFileSystem(conf), numberOfClosesRequired, throwExceptionsOfFailedClose); - stream.configure(context); - stream.open(dstPath, null, CompressionType.NONE, conf, mockFs); - stream.append(event); - stream.sync(); - - stream.close(); - - if (throwExceptionsOfFailedClose) { - int expectedNumberOfCloses = 1; - Assert.assertTrue("Expected " + expectedNumberOfCloses + " but got " + mockFs.getLastMockOutputStream().getCurrentCloseAttempts() , mockFs.getLastMockOutputStream().currentCloseAttempts == expectedNumberOfCloses); - } else { - int expectedNumberOfCloses = Math.max(Math.min(numberOfClosesRequired, numberOfCloseRetriesToAttempt), 1); - Assert.assertTrue("Expected " + expectedNumberOfCloses + " but got " + mockFs.getLastMockOutputStream().getCurrentCloseAttempts() , mockFs.getLastMockOutputStream().currentCloseAttempts == expectedNumberOfCloses); - } - - - - - - } - - } \ No newline at end of file From 6115e7d6d611d2b82dc2583b95a13d4c0886a93f Mon Sep 17 00:00:00 2001 From: Brock Noland Date: Fri, 2 May 2014 07:32:33 -0700 Subject: [PATCH 100/341] FLUME-2181 - Optionally disable File Channel fsyncs (Hari via Brock) --- .../channel/file/CheckpointRebuilder.java | 11 ++- .../flume/channel/file/FileChannel.java | 26 +++++- .../file/FileChannelConfiguration.java | 7 ++ .../org/apache/flume/channel/file/Log.java | 54 ++++++++++--- .../apache/flume/channel/file/LogFile.java | 81 ++++++++++++++++--- .../flume/channel/file/LogFileFactory.java | 16 ++-- .../apache/flume/channel/file/LogFileV2.java | 5 +- .../apache/flume/channel/file/LogFileV3.java | 61 ++++++++++---- .../flume/channel/file/ReplayHandler.java | 12 ++- .../channel/file/TransactionEventRecord.java | 4 + .../encryption/AESCTRNoPaddingProvider.java | 5 +- .../DecryptionFailureException.java | 38 +++++++++ .../channel/file/TestCheckpointRebuilder.java | 2 +- .../flume/channel/file/TestFileChannel.java | 35 ++++++-- .../apache/flume/channel/file/TestLog.java | 20 ++--- .../flume/channel/file/TestLogFile.java | 18 ++--- .../apache/flume/channel/file/TestUtils.java | 4 +- .../flume/tools/FileChannelIntegrityTool.java | 2 +- .../tools/TestFileChannelIntegrityTool.java | 2 +- 19 files changed, 314 insertions(+), 89 deletions(-) create mode 100644 flume-ng-channels/flume-file-channel/src/main/java/org/apache/flume/channel/file/encryption/DecryptionFailureException.java diff --git a/flume-ng-channels/flume-file-channel/src/main/java/org/apache/flume/channel/file/CheckpointRebuilder.java b/flume-ng-channels/flume-file-channel/src/main/java/org/apache/flume/channel/file/CheckpointRebuilder.java index 43881816c6..b961ae20e2 100644 --- a/flume-ng-channels/flume-file-channel/src/main/java/org/apache/flume/channel/file/CheckpointRebuilder.java +++ b/flume-ng-channels/flume-file-channel/src/main/java/org/apache/flume/channel/file/CheckpointRebuilder.java @@ -49,14 +49,17 @@ public class CheckpointRebuilder { HashMultimap.create(); private final SetMultimap uncommittedTakes = HashMultimap.create(); + private final boolean fsyncPerTransaction; private static Logger LOG = LoggerFactory.getLogger(CheckpointRebuilder.class); public CheckpointRebuilder(List logFiles, - FlumeEventQueue queue) throws IOException { + FlumeEventQueue queue, boolean fsyncPerTransaction) throws + IOException { this.logFiles = logFiles; this.queue = queue; + this.fsyncPerTransaction = fsyncPerTransaction; } public boolean rebuild() throws IOException, Exception { @@ -64,7 +67,8 @@ public boolean rebuild() throws IOException, Exception { List logReaders = Lists.newArrayList(); for (File logFile : logFiles) { try { - logReaders.add(LogFileFactory.getSequentialReader(logFile, null)); + logReaders.add(LogFileFactory.getSequentialReader(logFile, null, + fsyncPerTransaction)); } catch(EOFException e) { LOG.warn("Ignoring " + logFile + " due to EOF", e); } @@ -252,7 +256,8 @@ public static void main(String[] args) throws Exception { new File(checkpointDir, "inflighttakes"), new File(checkpointDir, "inflightputs"), new File(checkpointDir, Log.QUEUE_SET)); - CheckpointRebuilder rebuilder = new CheckpointRebuilder(logFiles, queue); + CheckpointRebuilder rebuilder = new CheckpointRebuilder(logFiles, + queue, true); if(rebuilder.rebuild()) { rebuilder.writeCheckpoint(); } else { diff --git a/flume-ng-channels/flume-file-channel/src/main/java/org/apache/flume/channel/file/FileChannel.java b/flume-ng-channels/flume-file-channel/src/main/java/org/apache/flume/channel/file/FileChannel.java index 5203ca108c..0f242d20f5 100644 --- a/flume-ng-channels/flume-file-channel/src/main/java/org/apache/flume/channel/file/FileChannel.java +++ b/flume-ng-channels/flume-file-channel/src/main/java/org/apache/flume/channel/file/FileChannel.java @@ -95,6 +95,8 @@ public class FileChannel extends BasicChannelSemantics { private String encryptionActiveKey; private String encryptionCipherProvider; private boolean useDualCheckpoints; + private boolean fsyncPerTransaction; + private int fsyncInterval; @Override public synchronized void setName(String name) { @@ -233,6 +235,12 @@ public void configure(Context context) { "key provider name is not."); } + fsyncPerTransaction = context.getBoolean(FileChannelConfiguration + .FSYNC_PER_TXN, FileChannelConfiguration.DEFAULT_FSYNC_PRE_TXN); + + fsyncInterval = context.getInteger(FileChannelConfiguration + .FSYNC_INTERVAL, FileChannelConfiguration.DEFAULT_FSYNC_INTERVAL); + if(queueRemaining == null) { queueRemaining = new Semaphore(capacity, true); } @@ -265,6 +273,8 @@ public synchronized void start() { builder.setEncryptionCipherProvider(encryptionCipherProvider); builder.setUseDualCheckpoints(useDualCheckpoints); builder.setBackupCheckpointDir(backupCheckpointDir); + builder.setFsyncPerTransaction(fsyncPerTransaction); + builder.setFsyncInterval(fsyncInterval); log = builder.build(); log.replay(); open = true; @@ -328,8 +338,8 @@ protected BasicTransactionSemantics createTransaction() { trans.getStateAsString() + channelNameDescriptor); } trans = new FileBackedTransaction(log, TransactionIDOracle.next(), - transactionCapacity, keepAlive, queueRemaining, getName(), - channelCounter); + transactionCapacity, keepAlive, queueRemaining, getName(), + fsyncPerTransaction, channelCounter); transactions.set(trans); return trans; } @@ -401,9 +411,11 @@ static class FileBackedTransaction extends BasicTransactionSemantics { private final Semaphore queueRemaining; private final String channelNameDescriptor; private final ChannelCounter channelCounter; + private final boolean fsyncPerTransaction; public FileBackedTransaction(Log log, long transactionID, int transCapacity, int keepAlive, Semaphore queueRemaining, - String name, ChannelCounter counter) { + String name, boolean fsyncPerTransaction, ChannelCounter + counter) { this.log = log; queue = log.getFlumeEventQueue(); this.transactionID = transactionID; @@ -411,6 +423,7 @@ public FileBackedTransaction(Log log, long transactionID, this.queueRemaining = queueRemaining; putList = new LinkedBlockingDeque(transCapacity); takeList = new LinkedBlockingDeque(transCapacity); + this.fsyncPerTransaction = fsyncPerTransaction; channelNameDescriptor = "[channel=" + name + "]"; this.channelCounter = counter; } @@ -500,6 +513,13 @@ protected Event doTake() throws InterruptedException { LOG.warn("Corrupt record replaced by File Channel Integrity " + "tool found. Will retrieve next event", e); takeList.remove(ptr); + } catch (CorruptEventException ex) { + if (fsyncPerTransaction) { + throw new ChannelException(ex); + } + LOG.warn("Corrupt record found. Event will be " + + "skipped, and next event will be read.", ex); + takeList.remove(ptr); } } } diff --git a/flume-ng-channels/flume-file-channel/src/main/java/org/apache/flume/channel/file/FileChannelConfiguration.java b/flume-ng-channels/flume-file-channel/src/main/java/org/apache/flume/channel/file/FileChannelConfiguration.java index e4bc879a24..87dc653108 100644 --- a/flume-ng-channels/flume-file-channel/src/main/java/org/apache/flume/channel/file/FileChannelConfiguration.java +++ b/flume-ng-channels/flume-file-channel/src/main/java/org/apache/flume/channel/file/FileChannelConfiguration.java @@ -87,4 +87,11 @@ public class FileChannelConfiguration { public static final String USE_DUAL_CHECKPOINTS = "useDualCheckpoints"; public static final boolean DEFAULT_USE_DUAL_CHECKPOINTS = false; + public static final String FSYNC_PER_TXN = "fsyncPerTransaction"; + public static final boolean DEFAULT_FSYNC_PRE_TXN = true; + + public static final String FSYNC_INTERVAL = "fsyncInterval"; + public static final int DEFAULT_FSYNC_INTERVAL = 5; // seconds. + + } diff --git a/flume-ng-channels/flume-file-channel/src/main/java/org/apache/flume/channel/file/Log.java b/flume-ng-channels/flume-file-channel/src/main/java/org/apache/flume/channel/file/Log.java index 579ee35a8b..5bac0f4f87 100644 --- a/flume-ng-channels/flume-file-channel/src/main/java/org/apache/flume/channel/file/Log.java +++ b/flume-ng-channels/flume-file-channel/src/main/java/org/apache/flume/channel/file/Log.java @@ -125,6 +125,9 @@ public class Log { private final boolean useDualCheckpoints; private volatile boolean backupRestored = false; + private final boolean fsyncPerTransaction; + private final int fsyncInterval; + private int readCount; private int putCount; private int takeCount; @@ -150,6 +153,25 @@ static class Builder { private boolean bUseDualCheckpoints = false; private File bBackupCheckpointDir = null; + private boolean fsyncPerTransaction = true; + private int fsyncInterval; + + boolean isFsyncPerTransaction() { + return fsyncPerTransaction; + } + + void setFsyncPerTransaction(boolean fsyncPerTransaction) { + this.fsyncPerTransaction = fsyncPerTransaction; + } + + int getFsyncInterval() { + return fsyncInterval; + } + + void setFsyncInterval(int fsyncInterval) { + this.fsyncInterval = fsyncInterval; + } + Builder setUsableSpaceRefreshInterval(long usableSpaceRefreshInterval) { bUsableSpaceRefreshInterval = usableSpaceRefreshInterval; return this; @@ -231,7 +253,7 @@ Log build() throws IOException { useLogReplayV1, useFastReplay, bMinimumRequiredSpace, bEncryptionKeyProvider, bEncryptionKeyAlias, bEncryptionCipherProvider, bUsableSpaceRefreshInterval, - bLogDirs); + fsyncPerTransaction, fsyncInterval, bLogDirs); } } @@ -241,7 +263,8 @@ private Log(long checkpointInterval, long maxFileSize, int queueCapacity, long minimumRequiredSpace, @Nullable KeyProvider encryptionKeyProvider, @Nullable String encryptionKeyAlias, @Nullable String encryptionCipherProvider, - long usableSpaceRefreshInterval, File... logDirs) + long usableSpaceRefreshInterval, boolean fsyncPerTransaction, + int fsyncInterval, File... logDirs) throws IOException { Preconditions.checkArgument(checkpointInterval > 0, "checkpointInterval <= 0"); @@ -318,6 +341,8 @@ private Log(long checkpointInterval, long maxFileSize, int queueCapacity, this.checkpointDir = checkpointDir; this.backupCheckpointDir = backupCheckpointDir; this.logDirs = logDirs; + this.fsyncPerTransaction = fsyncPerTransaction; + this.fsyncInterval = fsyncInterval; logFiles = new AtomicReferenceArray(this.logDirs.length); workerExecutor = Executors.newSingleThreadScheduledExecutor(new ThreadFactoryBuilder().setNameFormat("Log-BackgroundWorker-" + name) @@ -354,7 +379,7 @@ void replay() throws IOException { dataFiles.add(file); nextFileID.set(Math.max(nextFileID.get(), id)); idLogFileMap.put(id, LogFileFactory.getRandomReader(new File(logDir, - PREFIX + id), encryptionKeyProvider)); + PREFIX + id), encryptionKeyProvider, fsyncPerTransaction)); } } LOGGER.info("Found NextFileID " + nextFileID + @@ -468,13 +493,13 @@ private void doReplay(FlumeEventQueue queue, List dataFiles, KeyProvider encryptionKeyProvider, boolean useFastReplay) throws Exception { CheckpointRebuilder rebuilder = new CheckpointRebuilder(dataFiles, - queue); + queue, fsyncPerTransaction); if (useFastReplay && rebuilder.rebuild()) { didFastReplay = true; LOGGER.info("Fast replay successful."); } else { ReplayHandler replayHandler = new ReplayHandler(queue, - encryptionKeyProvider); + encryptionKeyProvider, fsyncPerTransaction); if (useLogReplayV1) { LOGGER.info("Replaying logs with v1 replay logic"); replayHandler.replayLogv1(dataFiles); @@ -551,7 +576,7 @@ FlumeEventQueue getFlumeEventQueue() { * @throws InterruptedException */ FlumeEvent get(FlumeEventPointer pointer) throws IOException, - InterruptedException, NoopRecordException { + InterruptedException, NoopRecordException, CorruptEventException { Preconditions.checkState(open, "Log is closed"); int id = pointer.getFileID(); LogFile.RandomReader logFile = idLogFileMap.get(id); @@ -559,9 +584,12 @@ FlumeEvent get(FlumeEventPointer pointer) throws IOException, try { return logFile.get(pointer.getOffset()); } catch (CorruptEventException ex) { - open = false; - throw new IOException("Corrupt event found. Please run File Channel " + - "Integrity tool.", ex); + if (fsyncPerTransaction) { + open = false; + throw new IOException("Corrupt event found. Please run File Channel " + + "Integrity tool.", ex); + } + throw ex; } } @@ -906,9 +934,10 @@ private synchronized void roll(int index, ByteBuffer buffer) File file = new File(logDirs[index], PREFIX + fileID); LogFile.Writer writer = LogFileFactory.getWriter(file, fileID, maxFileSize, encryptionKey, encryptionKeyAlias, - encryptionCipherProvider, usableSpaceRefreshInterval); + encryptionCipherProvider, usableSpaceRefreshInterval, + fsyncPerTransaction, fsyncInterval); idLogFileMap.put(fileID, LogFileFactory.getRandomReader(file, - encryptionKeyProvider)); + encryptionKeyProvider, fsyncPerTransaction)); // writer from this point on will get new reference logFiles.set(index, writer); // close out old log @@ -991,7 +1020,8 @@ private Boolean writeCheckpoint(Boolean force) throws Exception { } finally { writer.close(); } - reader = LogFileFactory.getRandomReader(file, encryptionKeyProvider); + reader = LogFileFactory.getRandomReader(file, + encryptionKeyProvider, fsyncPerTransaction); idLogFileMap.put(id, reader); LOGGER.debug("Updated checkpoint for file: " + file + "logWriteOrderID " + logWriteOrderID); diff --git a/flume-ng-channels/flume-file-channel/src/main/java/org/apache/flume/channel/file/LogFile.java b/flume-ng-channels/flume-file-channel/src/main/java/org/apache/flume/channel/file/LogFile.java index 26a24b1814..488dcf4a75 100644 --- a/flume-ng-channels/flume-file-channel/src/main/java/org/apache/flume/channel/file/LogFile.java +++ b/flume-ng-channels/flume-file-channel/src/main/java/org/apache/flume/channel/file/LogFile.java @@ -41,6 +41,9 @@ import java.util.List; import java.util.concurrent.ArrayBlockingQueue; import java.util.concurrent.BlockingQueue; +import java.util.concurrent.Executors; +import java.util.concurrent.ScheduledExecutorService; +import java.util.concurrent.TimeUnit; import java.util.concurrent.atomic.AtomicLong; @InterfaceAudience.Private @@ -169,13 +172,18 @@ static abstract class Writer { private long lastCommitPosition; private long lastSyncPosition; + private final boolean fsyncPerTransaction; + private final int fsyncInterval; + private final ScheduledExecutorService syncExecutor; + private volatile boolean dirty = false; + // To ensure we can count the number of fsyncs. private long syncCount; Writer(File file, int logFileID, long maxFileSize, - CipherProvider.Encryptor encryptor, long usableSpaceRefreshInterval) - throws IOException { + CipherProvider.Encryptor encryptor, long usableSpaceRefreshInterval, + boolean fsyncPerTransaction, int fsyncInterval) throws IOException { this.file = file; this.logFileID = logFileID; this.maxFileSize = Math.min(maxFileSize, @@ -183,6 +191,25 @@ static abstract class Writer { this.encryptor = encryptor; writeFileHandle = new RandomAccessFile(file, "rw"); writeFileChannel = writeFileHandle.getChannel(); + this.fsyncPerTransaction = fsyncPerTransaction; + this.fsyncInterval = fsyncInterval; + if(!fsyncPerTransaction) { + LOG.info("Sync interval = " + fsyncInterval); + syncExecutor = Executors.newSingleThreadScheduledExecutor(); + syncExecutor.scheduleWithFixedDelay(new Runnable() { + @Override + public void run() { + try { + sync(); + } catch (Throwable ex) { + LOG.error("Data file, " + getFile().toString() + " could not " + + "be synced to disk due to an error.", ex); + } + } + }, fsyncInterval, fsyncInterval, TimeUnit.SECONDS); + } else { + syncExecutor = null; + } usableSpace = new CachedFSUsableSpace(file, usableSpaceRefreshInterval); LOG.info("Opened " + file); open = true; @@ -258,6 +285,7 @@ synchronized void commit(ByteBuffer buffer) throws IOException { buffer = ByteBuffer.wrap(encryptor.encrypt(buffer.array())); } write(buffer); + dirty = true; lastCommitPosition = position(); } @@ -299,6 +327,14 @@ synchronized boolean isRollRequired(ByteBuffer buffer) throws IOException { * @throws LogFileRetryableIOException - if this log file is closed. */ synchronized void sync() throws IOException { + if (!fsyncPerTransaction && !dirty) { + if(LOG.isDebugEnabled()) { + LOG.debug( + "No events written to file, " + getFile().toString() + + " in last " + fsyncInterval + " or since last commit."); + } + return; + } if (!isOpen()) { throw new LogFileRetryableIOException("File closed " + file); } @@ -306,6 +342,7 @@ synchronized void sync() throws IOException { getFileChannel().force(false); lastSyncPosition = position(); syncCount++; + dirty = false; } } @@ -322,6 +359,13 @@ protected FileChannel getFileChannel() { synchronized void close() { if(open) { open = false; + if (!fsyncPerTransaction) { + // Shutdown the executor before attempting to close. + if(syncExecutor != null) { + // No need to wait for it to shutdown. + syncExecutor.shutdown(); + } + } if(writeFileChannel.isOpen()) { LOG.info("Closing " + file); try { @@ -396,12 +440,15 @@ static abstract class RandomReader { private final BlockingQueue readFileHandles = new ArrayBlockingQueue(50, true); private final KeyProvider encryptionKeyProvider; + private final boolean fsyncPerTransaction; private volatile boolean open; - public RandomReader(File file, @Nullable KeyProvider encryptionKeyProvider) + public RandomReader(File file, @Nullable KeyProvider + encryptionKeyProvider, boolean fsyncPerTransaction) throws IOException { this.file = file; this.encryptionKeyProvider = encryptionKeyProvider; readFileHandles.add(open()); + this.fsyncPerTransaction = fsyncPerTransaction; open = true; } @@ -430,8 +477,11 @@ FlumeEvent get(int offset) throws IOException, InterruptedException, throw new NoopRecordException("No op record found. Corrupt record " + "may have been repaired by File Channel Integrity tool"); } - Preconditions.checkState(operation == OP_RECORD, - Integer.toHexString(operation)); + if (operation != OP_RECORD) { + throw new CorruptEventException( + "Operation code is invalid. File " + + "is corrupt. Please run File Channel Integrity tool."); + } TransactionEventRecord record = doGet(fileHandle); if(!(record instanceof Put)) { Preconditions.checkState(false, "Record is " + @@ -491,8 +541,8 @@ private RandomAccessFile checkOut() } int remaining = readFileHandles.remainingCapacity(); if(remaining > 0) { - LOG.info("Opening " + file + " for read, remaining capacity is " - + remaining); + LOG.info("Opening " + file + " for read, remaining number of file " + + "handles available for reads of this file is " + remaining); return open(); } return readFileHandles.take(); @@ -647,11 +697,20 @@ protected static void writeDelimitedBuffer(ByteBuffer output, ByteBuffer buffer) output.put(buffer); } protected static byte[] readDelimitedBuffer(RandomAccessFile fileHandle) - throws IOException { + throws IOException, CorruptEventException { int length = fileHandle.readInt(); - Preconditions.checkState(length >= 0, Integer.toHexString(length)); + if (length < 0) { + throw new CorruptEventException("Length of event is: " + String.valueOf + (length) + ". Event must have length >= 0. Possible corruption of " + + "data or partial fsync."); + } byte[] buffer = new byte[length]; - fileHandle.readFully(buffer); + try { + fileHandle.readFully(buffer); + } catch (EOFException ex) { + throw new CorruptEventException("Remaining data in file less than " + + "expected size of event.", ex); + } return buffer; } @@ -659,7 +718,7 @@ public static void main(String[] args) throws EOFException, IOException, Corrupt File file = new File(args[0]); LogFile.SequentialReader reader = null; try { - reader = LogFileFactory.getSequentialReader(file, null); + reader = LogFileFactory.getSequentialReader(file, null, false); LogRecord entry; FlumeEventPointer ptr; // for puts the fileId is the fileID of the file they exist in diff --git a/flume-ng-channels/flume-file-channel/src/main/java/org/apache/flume/channel/file/LogFileFactory.java b/flume-ng-channels/flume-file-channel/src/main/java/org/apache/flume/channel/file/LogFileFactory.java index 9c98d8cb6d..7d7fd8504f 100644 --- a/flume-ng-channels/flume-file-channel/src/main/java/org/apache/flume/channel/file/LogFileFactory.java +++ b/flume-ng-channels/flume-file-channel/src/main/java/org/apache/flume/channel/file/LogFileFactory.java @@ -68,17 +68,19 @@ static LogFile.Writer getWriter(File file, int logFileID, long maxFileSize, @Nullable Key encryptionKey, @Nullable String encryptionKeyAlias, @Nullable String encryptionCipherProvider, - long usableSpaceRefreshInterval) throws IOException { + long usableSpaceRefreshInterval, boolean fsyncPerTransaction, + int fsyncInterval) throws IOException { Preconditions.checkState(!file.exists(), "File already exists " + file.getAbsolutePath()); Preconditions.checkState(file.createNewFile(), "File could not be created " + file.getAbsolutePath()); return new LogFileV3.Writer(file, logFileID, maxFileSize, encryptionKey, - encryptionKeyAlias, encryptionCipherProvider, usableSpaceRefreshInterval); + encryptionKeyAlias, encryptionCipherProvider, + usableSpaceRefreshInterval, fsyncPerTransaction, fsyncInterval); } static LogFile.RandomReader getRandomReader(File file, - @Nullable KeyProvider encryptionKeyProvider) + @Nullable KeyProvider encryptionKeyProvider, boolean fsyncPerTransaction) throws IOException { RandomAccessFile logFile = new RandomAccessFile(file, "r"); try { @@ -86,7 +88,8 @@ static LogFile.RandomReader getRandomReader(File file, // either this is a rr for a just created file or // the metadata file exists and as such it's V3 if(logFile.length() == 0L || metaDataFile.exists()) { - return new LogFileV3.RandomReader(file, encryptionKeyProvider); + return new LogFileV3.RandomReader(file, encryptionKeyProvider, + fsyncPerTransaction); } int version = logFile.readInt(); if(Serialization.VERSION_2 == version) { @@ -106,7 +109,7 @@ static LogFile.RandomReader getRandomReader(File file, } static LogFile.SequentialReader getSequentialReader(File file, - @Nullable KeyProvider encryptionKeyProvider) + @Nullable KeyProvider encryptionKeyProvider, boolean fsyncPerTransaction) throws IOException { RandomAccessFile logFile = null; try { @@ -159,7 +162,8 @@ static LogFile.SequentialReader getSequentialReader(File file, throw new EOFException(String.format("MetaData file %s is empty", metaDataFile)); } - return new LogFileV3.SequentialReader(file, encryptionKeyProvider); + return new LogFileV3.SequentialReader(file, encryptionKeyProvider, + fsyncPerTransaction); } logFile = new RandomAccessFile(file, "r"); int version = logFile.readInt(); diff --git a/flume-ng-channels/flume-file-channel/src/main/java/org/apache/flume/channel/file/LogFileV2.java b/flume-ng-channels/flume-file-channel/src/main/java/org/apache/flume/channel/file/LogFileV2.java index f286c57541..bb25e95dc5 100644 --- a/flume-ng-channels/flume-file-channel/src/main/java/org/apache/flume/channel/file/LogFileV2.java +++ b/flume-ng-channels/flume-file-channel/src/main/java/org/apache/flume/channel/file/LogFileV2.java @@ -96,7 +96,8 @@ static class Writer extends LogFile.Writer { Writer(File file, int logFileID, long maxFileSize, long usableSpaceRefreshInterval) throws IOException { - super(file, logFileID, maxFileSize, null, usableSpaceRefreshInterval); + super(file, logFileID, maxFileSize, null, usableSpaceRefreshInterval, + true, 0); RandomAccessFile writeFileHandle = getFileHandle(); writeFileHandle.writeInt(getVersion()); writeFileHandle.writeInt(logFileID); @@ -116,7 +117,7 @@ int getVersion() { static class RandomReader extends LogFile.RandomReader { RandomReader(File file) throws IOException { - super(file, null); + super(file, null, true); } @Override int getVersion() { diff --git a/flume-ng-channels/flume-file-channel/src/main/java/org/apache/flume/channel/file/LogFileV3.java b/flume-ng-channels/flume-file-channel/src/main/java/org/apache/flume/channel/file/LogFileV3.java index 38f6ecb8aa..9b0ef93a9e 100644 --- a/flume-ng-channels/flume-file-channel/src/main/java/org/apache/flume/channel/file/LogFileV3.java +++ b/flume-ng-channels/flume-file-channel/src/main/java/org/apache/flume/channel/file/LogFileV3.java @@ -21,10 +21,12 @@ import com.google.common.base.Preconditions; import com.google.protobuf.ByteString; import com.google.protobuf.GeneratedMessage; +import org.apache.flume.Transaction; import org.apache.flume.annotations.InterfaceAudience; import org.apache.flume.annotations.InterfaceStability; import org.apache.flume.channel.file.encryption.CipherProvider; import org.apache.flume.channel.file.encryption.CipherProviderFactory; +import org.apache.flume.channel.file.encryption.DecryptionFailureException; import org.apache.flume.channel.file.encryption.KeyProvider; import org.apache.flume.channel.file.proto.ProtosFactory; import org.slf4j.Logger; @@ -178,11 +180,11 @@ static class Writer extends LogFile.Writer { @Nullable Key encryptionKey, @Nullable String encryptionKeyAlias, @Nullable String encryptionCipherProvider, - long usableSpaceRefreshInterval) - throws IOException { + long usableSpaceRefreshInterval, boolean fsyncPerTransaction, + int fsyncInterval) throws IOException { super(file, logFileID, maxFileSize, CipherProviderFactory. getEncrypter(encryptionCipherProvider, encryptionKey), - usableSpaceRefreshInterval); + usableSpaceRefreshInterval, fsyncPerTransaction, fsyncInterval); ProtosFactory.LogFileMetaData.Builder metaDataBuilder = ProtosFactory.LogFileMetaData.newBuilder(); if(encryptionKey != null) { @@ -219,10 +221,11 @@ static class RandomReader extends LogFile.RandomReader { private volatile String cipherProvider; private volatile byte[] parameters; private BlockingQueue decryptors = - new LinkedBlockingDeque(); - RandomReader(File file, @Nullable KeyProvider encryptionKeyProvider) - throws IOException { - super(file, encryptionKeyProvider); + new LinkedBlockingDeque(); + + RandomReader(File file, @Nullable KeyProvider encryptionKeyProvider, + boolean fsyncPerTransaction) throws IOException { + super(file, encryptionKeyProvider, fsyncPerTransaction); } private void initialize() throws IOException { File metaDataFile = Serialization.getMetaDataFile(getFile()); @@ -281,10 +284,10 @@ protected TransactionEventRecord doGet(RandomAccessFile fileHandle) initialize(); } } - byte[] buffer = readDelimitedBuffer(fileHandle); - CipherProvider.Decryptor decryptor = null; boolean success = false; + CipherProvider.Decryptor decryptor = null; try { + byte[] buffer = readDelimitedBuffer(fileHandle); if(encryptionEnabled) { decryptor = getDecryptor(); buffer = decryptor.decrypt(buffer); @@ -293,6 +296,8 @@ protected TransactionEventRecord doGet(RandomAccessFile fileHandle) fromByteArray(buffer); success = true; return event; + } catch(DecryptionFailureException ex) { + throw new CorruptEventException("Error decrypting event", ex); } finally { if(success && encryptionEnabled && decryptor != null) { decryptors.offer(decryptor); @@ -303,10 +308,12 @@ protected TransactionEventRecord doGet(RandomAccessFile fileHandle) public static class SequentialReader extends LogFile.SequentialReader { private CipherProvider.Decryptor decryptor; - + private final boolean fsyncPerTransaction; public SequentialReader(File file, @Nullable KeyProvider - encryptionKeyProvider) throws EOFException, IOException { + encryptionKeyProvider, boolean fsyncPerTransaction) throws EOFException, + IOException { super(file, encryptionKeyProvider); + this.fsyncPerTransaction = fsyncPerTransaction; File metaDataFile = Serialization.getMetaDataFile(file); FileInputStream inputStream = new FileInputStream(metaDataFile); try { @@ -351,13 +358,33 @@ public int getVersion() { } @Override - LogRecord doNext(int offset) throws IOException, CorruptEventException { - byte[] buffer = readDelimitedBuffer(getFileHandle()); - if(decryptor != null) { - buffer = decryptor.decrypt(buffer); + LogRecord doNext(int offset) throws IOException, CorruptEventException, + DecryptionFailureException { + byte[] buffer = null; + TransactionEventRecord event = null; + try { + buffer = readDelimitedBuffer(getFileHandle()); + if (decryptor != null) { + buffer = decryptor.decrypt(buffer); + } + event = TransactionEventRecord.fromByteArray(buffer); + } catch (CorruptEventException ex) { + LOGGER.warn("Corrupt file found. File id: log-" + this.getLogFileID(), + ex); + // Return null so that replay handler thinks all events in this file + // have been taken. + if (!fsyncPerTransaction) { + return null; + } + throw ex; + } catch (DecryptionFailureException ex) { + if (!fsyncPerTransaction) { + LOGGER.warn("Could not decrypt even read from channel. Skipping " + + "event.", ex); + return null; + } + throw ex; } - TransactionEventRecord event = - TransactionEventRecord.fromByteArray(buffer); return new LogRecord(getLogFileID(), offset, event); } } diff --git a/flume-ng-channels/flume-file-channel/src/main/java/org/apache/flume/channel/file/ReplayHandler.java b/flume-ng-channels/flume-file-channel/src/main/java/org/apache/flume/channel/file/ReplayHandler.java index e668c2e39e..a5595034b3 100644 --- a/flume-ng-channels/flume-file-channel/src/main/java/org/apache/flume/channel/file/ReplayHandler.java +++ b/flume-ng-channels/flume-file-channel/src/main/java/org/apache/flume/channel/file/ReplayHandler.java @@ -52,6 +52,7 @@ class ReplayHandler { private final Map readers; private final PriorityQueue logRecordBuffer; private final KeyProvider encryptionKeyProvider; + private final boolean fsyncPerTransaction; /** * This data structure stores takes for which we found a commit in the log * files before we found a commit for the put. This can happen if the channel @@ -91,19 +92,22 @@ public int getTakeCount() { public int getCommitCount() { return commitCount; } + @VisibleForTesting public int getRollbackCount() { return rollbackCount; } ReplayHandler(FlumeEventQueue queue, - @Nullable KeyProvider encryptionKeyProvider) { + @Nullable KeyProvider encryptionKeyProvider, + boolean fsyncPerTransaction) { this.queue = queue; this.lastCheckpoint = queue.getLogWriteOrderID(); pendingTakes = Lists.newArrayList(); readers = Maps.newHashMap(); logRecordBuffer = new PriorityQueue(); this.encryptionKeyProvider = encryptionKeyProvider; + this.fsyncPerTransaction = fsyncPerTransaction; } /** * Replay logic from Flume1.2 which can be activated if the v2 logic @@ -129,7 +133,8 @@ void replayLogv1(List logs) throws Exception { LOG.info("Replaying " + log); LogFile.SequentialReader reader = null; try { - reader = LogFileFactory.getSequentialReader(log, encryptionKeyProvider); + reader = LogFileFactory.getSequentialReader(log, + encryptionKeyProvider, fsyncPerTransaction); reader.skipToLastCheckpointPosition(queue.getLogWriteOrderID()); LogRecord entry; FlumeEventPointer ptr; @@ -257,7 +262,8 @@ void replayLog(List logs) throws Exception { LOG.info("Replaying " + log); try { LogFile.SequentialReader reader = - LogFileFactory.getSequentialReader(log, encryptionKeyProvider); + LogFileFactory.getSequentialReader(log, encryptionKeyProvider, + fsyncPerTransaction); reader.skipToLastCheckpointPosition(queue.getLogWriteOrderID()); Preconditions.checkState(!readers.containsKey(reader.getLogFileID()), "Readers " + readers + " already contains " diff --git a/flume-ng-channels/flume-file-channel/src/main/java/org/apache/flume/channel/file/TransactionEventRecord.java b/flume-ng-channels/flume-file-channel/src/main/java/org/apache/flume/channel/file/TransactionEventRecord.java index ea7f00c9ee..1eb3f4fa00 100644 --- a/flume-ng-channels/flume-file-channel/src/main/java/org/apache/flume/channel/file/TransactionEventRecord.java +++ b/flume-ng-channels/flume-file-channel/src/main/java/org/apache/flume/channel/file/TransactionEventRecord.java @@ -29,6 +29,7 @@ import java.lang.reflect.Constructor; import java.nio.ByteBuffer; +import com.google.protobuf.InvalidProtocolBufferException; import org.apache.flume.annotations.InterfaceAudience; import org.apache.flume.annotations.InterfaceStability; import org.apache.flume.channel.file.proto.ProtosFactory; @@ -207,6 +208,9 @@ static TransactionEventRecord fromByteArray(byte[] buffer) ProtosFactory.TransactionEventFooter. parseDelimitedFrom(in), "Footer cannot be null"); return transactionEvent; + } catch (InvalidProtocolBufferException ex) { + throw new CorruptEventException( + "Could not parse event from data file.", ex); } finally { try { in.close(); diff --git a/flume-ng-channels/flume-file-channel/src/main/java/org/apache/flume/channel/file/encryption/AESCTRNoPaddingProvider.java b/flume-ng-channels/flume-file-channel/src/main/java/org/apache/flume/channel/file/encryption/AESCTRNoPaddingProvider.java index d0a84fe834..9ee4245521 100644 --- a/flume-ng-channels/flume-file-channel/src/main/java/org/apache/flume/channel/file/encryption/AESCTRNoPaddingProvider.java +++ b/flume-ng-channels/flume-file-channel/src/main/java/org/apache/flume/channel/file/encryption/AESCTRNoPaddingProvider.java @@ -103,14 +103,15 @@ public String getCodec() { } } - private static byte[] doFinal(Cipher cipher, byte[] input) { + private static byte[] doFinal(Cipher cipher, byte[] input) + throws DecryptionFailureException{ try { return cipher.doFinal(input); } catch (Exception e) { String msg = "Unable to encrypt or decrypt data " + TYPE + " input.length " + input.length; LOG.error(msg, e); - throw Throwables.propagate(e); + throw new DecryptionFailureException(msg, e); } } diff --git a/flume-ng-channels/flume-file-channel/src/main/java/org/apache/flume/channel/file/encryption/DecryptionFailureException.java b/flume-ng-channels/flume-file-channel/src/main/java/org/apache/flume/channel/file/encryption/DecryptionFailureException.java new file mode 100644 index 0000000000..0155c39581 --- /dev/null +++ b/flume-ng-channels/flume-file-channel/src/main/java/org/apache/flume/channel/file/encryption/DecryptionFailureException.java @@ -0,0 +1,38 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, + * software distributed under the License is distributed on an + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + * KIND, either express or implied. See the License for the + * specific language governing permissions and limitations + * under the License. + */ +package org.apache.flume.channel.file.encryption; + +import org.apache.flume.FlumeException; + +/** + * Exception that is thrown when the channel is unable to decrypt an even + * read from the channel. + */ +public class DecryptionFailureException extends FlumeException { + private static final long serialVersionUID = 6646810195384793646L; + + + public DecryptionFailureException(String msg) { + super(msg); + } + + public DecryptionFailureException(String msg, Throwable th) { + super(msg, th); + } +} diff --git a/flume-ng-channels/flume-file-channel/src/test/java/org/apache/flume/channel/file/TestCheckpointRebuilder.java b/flume-ng-channels/flume-file-channel/src/test/java/org/apache/flume/channel/file/TestCheckpointRebuilder.java index 621d445ffa..c6c6ad3ec7 100644 --- a/flume-ng-channels/flume-file-channel/src/test/java/org/apache/flume/channel/file/TestCheckpointRebuilder.java +++ b/flume-ng-channels/flume-file-channel/src/test/java/org/apache/flume/channel/file/TestCheckpointRebuilder.java @@ -74,7 +74,7 @@ public void testFastReplay() throws Exception { FlumeEventQueue queue = new FlumeEventQueue(backingStore, inflightTakesFile, inflightPutsFile, queueSetDir); CheckpointRebuilder checkpointRebuilder = - new CheckpointRebuilder(getAllLogs(dataDirs), queue); + new CheckpointRebuilder(getAllLogs(dataDirs), queue, true); Assert.assertTrue(checkpointRebuilder.rebuild()); channel = createFileChannel(overrides); channel.start(); diff --git a/flume-ng-channels/flume-file-channel/src/test/java/org/apache/flume/channel/file/TestFileChannel.java b/flume-ng-channels/flume-file-channel/src/test/java/org/apache/flume/channel/file/TestFileChannel.java index 25765b534c..bb22e26652 100644 --- a/flume-ng-channels/flume-file-channel/src/test/java/org/apache/flume/channel/file/TestFileChannel.java +++ b/flume-ng-channels/flume-file-channel/src/test/java/org/apache/flume/channel/file/TestFileChannel.java @@ -27,6 +27,7 @@ import java.io.RandomAccessFile; import java.util.ArrayList; import java.util.Collections; +import java.util.HashMap; import java.util.HashSet; import java.util.Iterator; import java.util.List; @@ -573,8 +574,25 @@ public void testRollbackIncompleteTransaction() throws Exception { } @Test (expected = IllegalStateException.class) - public void testChannelDiesOnCorruptEvent() throws Exception { - final FileChannel channel = createFileChannel(); + public void testChannelDiesOnCorruptEventFsync() throws Exception { + testChannelDiesOnCorruptEvent(true); + } + + + @Test + public void testChannelDiesOnCorruptEventNoFsync() throws + Exception { + testChannelDiesOnCorruptEvent(false); + } + + + + private void testChannelDiesOnCorruptEvent(boolean fsyncPerTxn) + throws Exception { + Map overrides = new HashMap(); + overrides.put(FileChannelConfiguration.FSYNC_PER_TXN, + String.valueOf(fsyncPerTxn)); + final FileChannel channel = createFileChannel(overrides); channel.start(); putEvents(channel,"test-corrupt-event",100,100); for(File dataDir : dataDirs) { @@ -596,8 +614,9 @@ public boolean accept(File dir, String name) { } } } + Set events; try { - consumeChannel(channel, true); + events = consumeChannel(channel, true); } catch (IllegalStateException ex) { // The rollback call in takeEvents() in TestUtils will cause an // IllegalArgumentException - and this should be tested to verify the @@ -605,9 +624,13 @@ public boolean accept(File dir, String name) { Assert.assertTrue(ex.getMessage().contains("Log is closed")); throw ex; } - Assert.fail(); - - + if(fsyncPerTxn) { + Assert.fail(); + } else { + // The corrupt event must be missing, the rest should be + // returned + Assert.assertEquals(99, events.size()); + } } } diff --git a/flume-ng-channels/flume-file-channel/src/test/java/org/apache/flume/channel/file/TestLog.java b/flume-ng-channels/flume-file-channel/src/test/java/org/apache/flume/channel/file/TestLog.java index d1f51fc05e..c9a64edb91 100644 --- a/flume-ng-channels/flume-file-channel/src/test/java/org/apache/flume/channel/file/TestLog.java +++ b/flume-ng-channels/flume-file-channel/src/test/java/org/apache/flume/channel/file/TestLog.java @@ -76,7 +76,7 @@ public void cleanup() throws Exception{ */ @Test public void testPutGet() - throws IOException, InterruptedException, NoopRecordException { + throws IOException, InterruptedException, NoopRecordException, CorruptEventException { FlumeEvent eventIn = TestUtils.newPersistableEvent(); long transactionID = ++this.transactionID; FlumeEventPointer eventPointer = log.put(transactionID, eventIn); @@ -89,7 +89,7 @@ public void testPutGet() } @Test public void testRoll() - throws IOException, InterruptedException, NoopRecordException { + throws IOException, InterruptedException, NoopRecordException, CorruptEventException { log.shutdownWorker(); Thread.sleep(1000); for (int i = 0; i < 1000; i++) { @@ -119,7 +119,7 @@ public void testRoll() */ @Test public void testPutCommit() - throws IOException, InterruptedException, NoopRecordException { + throws IOException, InterruptedException, NoopRecordException, CorruptEventException { FlumeEvent eventIn = TestUtils.newPersistableEvent(); long transactionID = ++this.transactionID; FlumeEventPointer eventPointerIn = log.put(transactionID, eventIn); @@ -247,16 +247,16 @@ public void testPutTakeCommit() throws IOException, InterruptedException { */ @Test public void testPutTakeRollbackLogReplayV1() - throws IOException, InterruptedException, NoopRecordException { + throws IOException, InterruptedException, NoopRecordException, CorruptEventException { doPutTakeRollback(true); } @Test public void testPutTakeRollbackLogReplayV2() - throws IOException, InterruptedException, NoopRecordException { + throws IOException, InterruptedException, NoopRecordException, CorruptEventException { doPutTakeRollback(false); } public void doPutTakeRollback(boolean useLogReplayV1) - throws IOException, InterruptedException, NoopRecordException { + throws IOException, InterruptedException, NoopRecordException, CorruptEventException { FlumeEvent eventIn = TestUtils.newPersistableEvent(); long putTransactionID = ++transactionID; FlumeEventPointer eventPointerIn = log.put(putTransactionID, eventIn); @@ -396,7 +396,7 @@ public void doTestReplayFailsWithAllEmptyLogMetaData(boolean useFastReplay) } @Test public void testReplaySucceedsWithUnusedEmptyLogMetaDataNormalReplay() - throws IOException, InterruptedException, NoopRecordException { + throws IOException, InterruptedException, NoopRecordException, CorruptEventException { FlumeEvent eventIn = TestUtils.newPersistableEvent(); long transactionID = ++this.transactionID; FlumeEventPointer eventPointer = log.put(transactionID, eventIn); @@ -410,7 +410,7 @@ public void testReplaySucceedsWithUnusedEmptyLogMetaDataNormalReplay() } @Test public void testReplaySucceedsWithUnusedEmptyLogMetaDataFastReplay() - throws IOException, InterruptedException, NoopRecordException { + throws IOException, InterruptedException, NoopRecordException, CorruptEventException { FlumeEvent eventIn = TestUtils.newPersistableEvent(); long transactionID = ++this.transactionID; FlumeEventPointer eventPointer = log.put(transactionID, eventIn); @@ -427,7 +427,7 @@ public void testReplaySucceedsWithUnusedEmptyLogMetaDataFastReplay() } public void doTestReplaySucceedsWithUnusedEmptyLogMetaData(FlumeEvent eventIn, FlumeEventPointer eventPointer) throws IOException, - InterruptedException, NoopRecordException { + InterruptedException, NoopRecordException, CorruptEventException { for (int i = 0; i < dataDirs.length; i++) { for(File logFile : LogUtils.getLogs(dataDirs[i])) { if(logFile.length() == 0L) { @@ -467,7 +467,7 @@ public void testCachedFSUsableSpace() throws Exception { private void takeAndVerify(FlumeEventPointer eventPointerIn, FlumeEvent eventIn) - throws IOException, InterruptedException, NoopRecordException { + throws IOException, InterruptedException, NoopRecordException, CorruptEventException { FlumeEventQueue queue = log.getFlumeEventQueue(); FlumeEventPointer eventPointerOut = queue.removeHead(0); Assert.assertNotNull(eventPointerOut); diff --git a/flume-ng-channels/flume-file-channel/src/test/java/org/apache/flume/channel/file/TestLogFile.java b/flume-ng-channels/flume-file-channel/src/test/java/org/apache/flume/channel/file/TestLogFile.java index e5d830eae2..976a112999 100644 --- a/flume-ng-channels/flume-file-channel/src/test/java/org/apache/flume/channel/file/TestLogFile.java +++ b/flume-ng-channels/flume-file-channel/src/test/java/org/apache/flume/channel/file/TestLogFile.java @@ -63,7 +63,7 @@ public void setup() throws IOException { dataFile = new File(dataDir, String.valueOf(fileID)); Assert.assertTrue(dataDir.isDirectory()); logFileWriter = LogFileFactory.getWriter(dataFile, fileID, - Integer.MAX_VALUE, null, null, null, Long.MAX_VALUE); + Integer.MAX_VALUE, null, null, null, Long.MAX_VALUE, true, 0); } @After public void cleanup() throws IOException { @@ -80,7 +80,7 @@ public void testWriterRefusesToOverwriteFile() throws IOException { Assert.assertTrue(dataFile.isFile() || dataFile.createNewFile()); try { LogFileFactory.getWriter(dataFile, fileID, Integer.MAX_VALUE, null, null, - null, Long.MAX_VALUE); + null, Long.MAX_VALUE, true, 0); Assert.fail(); } catch (IllegalStateException e) { Assert.assertEquals("File already exists " + dataFile.getAbsolutePath(), @@ -94,7 +94,7 @@ public void testWriterFailsWithDirectory() throws IOException { Assert.assertTrue(dataFile.mkdirs()); try { LogFileFactory.getWriter(dataFile, fileID, Integer.MAX_VALUE, null, null, - null, Long.MAX_VALUE); + null, Long.MAX_VALUE, true, 0); Assert.fail(); } catch (IllegalStateException e) { Assert.assertEquals("File already exists " + dataFile.getAbsolutePath(), @@ -109,7 +109,7 @@ public void testPutGet() throws InterruptedException, IOException { CompletionService completionService = new ExecutorCompletionService (executorService); final LogFile.RandomReader logFileReader = - LogFileFactory.getRandomReader(dataFile, null); + LogFileFactory.getRandomReader(dataFile, null, true); for (int i = 0; i < 1000; i++) { // first try and throw failures synchronized (errors) { @@ -168,7 +168,7 @@ public void testReader() throws InterruptedException, IOException, puts.put(ptr.getOffset(), put); } LogFile.SequentialReader reader = - LogFileFactory.getSequentialReader(dataFile, null); + LogFileFactory.getSequentialReader(dataFile, null, true); LogRecord entry; while((entry = reader.next()) != null) { Integer offset = entry.getOffset(); @@ -202,7 +202,7 @@ public void testReaderOldMetaFile() throws InterruptedException, Assert.fail("Renaming to meta.old failed"); } LogFile.SequentialReader reader = - LogFileFactory.getSequentialReader(dataFile, null); + LogFileFactory.getSequentialReader(dataFile, null, true); Assert.assertTrue(metadataFile.exists()); Assert.assertFalse(oldMetadataFile.exists()); LogRecord entry; @@ -240,7 +240,7 @@ public void testReaderTempMetaFile() throws InterruptedException, Assert.fail("Renaming to meta.temp failed"); } LogFile.SequentialReader reader = - LogFileFactory.getSequentialReader(dataFile, null); + LogFileFactory.getSequentialReader(dataFile, null, true); Assert.assertTrue(metadataFile.exists()); Assert.assertFalse(tempMetadataFile.exists()); Assert.assertFalse(oldMetadataFile.exists()); @@ -281,7 +281,7 @@ public void testWriteDelimitedTo() throws IOException { @Test (expected = CorruptEventException.class) public void testPutGetCorruptEvent() throws Exception { final LogFile.RandomReader logFileReader = - LogFileFactory.getRandomReader(dataFile, null); + LogFileFactory.getRandomReader(dataFile, null, true); final FlumeEvent eventIn = TestUtils.newPersistableEvent(2500); final Put put = new Put(++transactionID, WriteOrderOracle.next(), eventIn); @@ -306,7 +306,7 @@ public void testPutGetCorruptEvent() throws Exception { @Test (expected = NoopRecordException.class) public void testPutGetNoopEvent() throws Exception { final LogFile.RandomReader logFileReader = - LogFileFactory.getRandomReader(dataFile, null); + LogFileFactory.getRandomReader(dataFile, null, true); final FlumeEvent eventIn = TestUtils.newPersistableEvent(2500); final Put put = new Put(++transactionID, WriteOrderOracle.next(), eventIn); diff --git a/flume-ng-channels/flume-file-channel/src/test/java/org/apache/flume/channel/file/TestUtils.java b/flume-ng-channels/flume-file-channel/src/test/java/org/apache/flume/channel/file/TestUtils.java index 0fb9bc2d8f..61f38d25bb 100644 --- a/flume-ng-channels/flume-file-channel/src/test/java/org/apache/flume/channel/file/TestUtils.java +++ b/flume-ng-channels/flume-file-channel/src/test/java/org/apache/flume/channel/file/TestUtils.java @@ -190,9 +190,9 @@ public static Set takeEvents(Channel channel, result.add(new String(event.getBody(), Charsets.UTF_8)); } transaction.commit(); - } catch (Exception ex) { + } catch (Throwable ex) { transaction.rollback(); - throw ex; + throw new RuntimeException(ex); } finally { transaction.close(); } diff --git a/flume-tools/src/main/java/org/apache/flume/tools/FileChannelIntegrityTool.java b/flume-tools/src/main/java/org/apache/flume/tools/FileChannelIntegrityTool.java index aa24fa5430..d0753a65a4 100644 --- a/flume-tools/src/main/java/org/apache/flume/tools/FileChannelIntegrityTool.java +++ b/flume-tools/src/main/java/org/apache/flume/tools/FileChannelIntegrityTool.java @@ -70,7 +70,7 @@ public boolean accept(File dir, String name) { for (File dataFile : dataFiles) { LOG.info("Checking for corruption in " + dataFile.toString()); LogFile.SequentialReader reader = - new LogFileV3.SequentialReader(dataFile, null); + new LogFileV3.SequentialReader(dataFile, null, true); LogFile.OperationRecordUpdater updater = new LogFile .OperationRecordUpdater(dataFile); boolean fileDone = false; diff --git a/flume-tools/src/test/java/org/apache/flume/tools/TestFileChannelIntegrityTool.java b/flume-tools/src/test/java/org/apache/flume/tools/TestFileChannelIntegrityTool.java index d32867105c..f24ae561b0 100644 --- a/flume-tools/src/test/java/org/apache/flume/tools/TestFileChannelIntegrityTool.java +++ b/flume-tools/src/test/java/org/apache/flume/tools/TestFileChannelIntegrityTool.java @@ -120,7 +120,7 @@ public boolean accept(File dir, String name) { int corrupted = 0; for (File dataFile : files) { LogFile.SequentialReader reader = - new LogFileV3.SequentialReader(dataFile, null); + new LogFileV3.SequentialReader(dataFile, null, true); RandomAccessFile handle = new RandomAccessFile(dataFile, "rw"); long eventPosition1 = reader.getPosition(); LogRecord rec = reader.next(); From 76b359b3fef1fade662388c8356e4a82c227c258 Mon Sep 17 00:00:00 2001 From: Jarek Jarcec Cecho Date: Sat, 3 May 2014 21:43:16 -0700 Subject: [PATCH 101/341] FLUME-2381: Upgrade Hadoop version in Hadoop 2 profile to 2.4.0 (Hari Shreedharan via Jarek Jarcec Cecho) --- pom.xml | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/pom.xml b/pom.xml index 2aa0ad146f..7abf833be9 100644 --- a/pom.xml +++ b/pom.xml @@ -49,7 +49,7 @@ limitations under the License. 1.7.3 0.90.1 - 2.3.0 + 2.4.0 0.12.0 From 31d45f1b377826b8755d1479bda9e2d702206bf9 Mon Sep 17 00:00:00 2001 From: Hari Shreedharan Date: Sat, 3 May 2014 22:23:56 -0700 Subject: [PATCH 102/341] FLUME-2368. Update CHANGELOG for 1.5 release --- CHANGELOG | 134 ++++++++++++++++++++++++++++++++++++++++++++++++++++++ 1 file changed, 134 insertions(+) diff --git a/CHANGELOG b/CHANGELOG index edbc4828d4..4ce391e38d 100644 --- a/CHANGELOG +++ b/CHANGELOG @@ -1,3 +1,137 @@ + +Release Notes - Flume - Version v1.5.0 + +** New Feature + * [FLUME-1227] - Introduce some sort of SpillableChannel + * [FLUME-2056] - Allow SpoolDir to pass just the filename that is the source of an event + * [FLUME-2071] - Flume Context doesn't support float or double configuration values. + * [FLUME-2185] - Upgrade morphlines to 0.7.0 + * [FLUME-2188] - flume-ng-log4jappender Support user supplied headers + * [FLUME-2225] - Elasticsearch Sink for ES HTTP API + * [FLUME-2294] - Add a sink for Kite Datasets + * [FLUME-2309] - Spooling directory should not always consume the oldest file first. + +** Improvement + * [FLUME-1542] - Flume User Guide sample configurations need fixing + * [FLUME-1899] - Make SpoolDir work with Sub-Directories + * [FLUME-2052] - Spooling directory source should be able to replace or ignore malformed characters + * [FLUME-2130] - Handle larger payloads via SyslogUDPSource + * [FLUME-2139] - upgrade morphline library dependency to a 0.5.0 release + * [FLUME-2154] - Reducing duplicate events caused by reset-connection-interval + * [FLUME-2155] - Improve replay time + * [FLUME-2181] - Optionally disable File Channel fsyncs + * [FLUME-2202] - AsyncHBaseSink should coalesce increments to reduce RPC roundtrips + * [FLUME-2206] - ElasticSearchSink ttl field modification to mimic Elasticsearch way of specifying TTL + * [FLUME-2207] - HDFS file suffix style index suffix in ElasticSearchSink + * [FLUME-2212] - upgrade to Morphlines-0.8.0 + * [FLUME-2213] - MorphlineInterceptor should share metric registry across threads for better (aggregate) reporting + * [FLUME-2217] - Preserve priority, timestamp and hostname fields in MultiportSyslogTcp and Udp sources + * [FLUME-2231] - Add details in Flume Ganglia config in User Guide + * [FLUME-2243] - AvroSource to use TransceiverThreadFactory for Thread naming while initializing NioServerSocketChannelFactory + * [FLUME-2267] - Increase default transactionCapacity for FileChannel to 10000 from 1000 + * [FLUME-2275] - Improve scalability of MorphlineInterceptor under contention + * [FLUME-2292] - Upgrade mapdb to 0.9.8 + * [FLUME-2316] - Upgrade MorphlineSolrSink to kite-0.12.0 + * [FLUME-2340] - Refactor to make room for Morphlines Elasticsearch Sink + * [FLUME-2343] - Add user impersonation to DatasetSink + * [FLUME-2351] - Ability to override any parameter from the configuration file + * [FLUME-2352] - HDFSCompressedDataStream should support appendBatch + +** Bug + * [FLUME-1666] - Syslog source strips timestamp and hostname from log message body + * [FLUME-1679] - Add dependency on Guava to flume-ng-elasticsearch-sink POM + * [FLUME-1892] - IRC Sink NPE + * [FLUME-1951] - Remove unused future from FlumeEventQueue + * [FLUME-2007] - HDFS Sink should check if file is closed and retry if it is not. + * [FLUME-2088] - Minor typo in Flume User Guide JSON Reporting section + * [FLUME-2109] - HTTPS support in HTTP Source + * [FLUME-2110] - Scribe Source must check if category is null before inserting the headers + * [FLUME-2119] - duplicate files cause flume to enter irrecoverable state + * [FLUME-2121] - Upgrade Flume to log4j 1.2.17 to be in sync with HDFS/HBase + * [FLUME-2122] - Minor cleanups of User guide + * [FLUME-2123] - Morphline Solr sink missing short type name + * [FLUME-2124] - Upgrade Morphline Solr Sink to CDK 0.4.1 + * [FLUME-2127] - JMX shutdown command for Flume + * [FLUME-2134] - AsyncHbase Sink bugfix plus tests errors on Windows + * [FLUME-2135] - Add zip to the build distribution for Windows support + * [FLUME-2136] - Windows - Fix intermitent test failure in TestMonitoredCounterGroup.java + * [FLUME-2137] - Fix StagedInstall.java to invoke the correct startup script on Windows + * [FLUME-2142] - HTTPS tests for http source + * [FLUME-2145] - TestCheckpointRebuilder.testFastReplay fails on Windows due to checkpoint file being memory mapped + * [FLUME-2151] - Windows: Update TestExecSource to use native commands on Windows + * [FLUME-2152] - Flume user guide says "Ganglia support" where it should say "JSON support" + * [FLUME-2156] - Unregister then re-register MonitoredCounterGroup JMX MBeans on reconfigure + * [FLUME-2157] - Spool directory source does not shut down correctly when Flume is reconfigured + * [FLUME-2159] - Sporadic failures in TestNettyAvroRpcClient.spinThreadsCrazily() + * [FLUME-2161] - Flume does not support spaces in -X java-opt command line args + * [FLUME-2172] - Update protocol buffer from 2.4.1 to 2.5.0 + * [FLUME-2176] - SpoolDir Source, get 'File has changed' exception but actually there is no change on the file + * [FLUME-2182] - Spooling Directory Source will not ingest data completely when a wide character appears at the edge of a buffer + * [FLUME-2184] - flume-ng-morphline-solr-sink Build failing due to incorrect hadoop-common dependency declaration + * [FLUME-2191] - HDFS Minicluster tests failing after protobuf upgrade. + * [FLUME-2192] - AbstractSinkProcessor stop incorrectly calls start + * [FLUME-2198] - Avro Source should disable itself if ipFilterRules contains invalid rules + * [FLUME-2199] - Flume builds with new version require mvn install before site can be generated + * [FLUME-2200] - HTTP Source should be able to use "port" parameter if SSL is enabled + * [FLUME-2208] - Jetty's default SocketSelector leaks File descriptors + * [FLUME-2209] - AsyncHBaseSink will never recover if the column family does not exists for the first start + * [FLUME-2210] - UnresolvedAddressException when using multiple hostNames in Elasticsearch sink configuration + * [FLUME-2220] - ElasticSearch sink - duplicate fields in indexed document + * [FLUME-2229] - Backoff period gets reset too often in OrderSelector + * [FLUME-2233] - MemoryChannel lock contention on every put due to bytesRemaining Semaphore + * [FLUME-2235] - idleFuture should be cancelled at the start of append + * [FLUME-2238] - Provide option to configure worker threads in NettyAvroRpcClient + * [FLUME-2239] - Clarify File Channel's dataDirs setting in User Guide + * [FLUME-2252] - Add null check before closing table in HbaseSink + * [FLUME-2253] - Please delete old releases from mirroring system + * [FLUME-2255] - Spooling Directory Source cannot handle channel exceptions + * [FLUME-2259] - transaction closure not happening for all the scenario in hbasesink + * [FLUME-2262] - Log4j Appender should use timeStamp field not getTimestamp + * [FLUME-2263] - Bump Hadoop 2 version to 2.3.0 + * [FLUME-2264] - Log4j Appender + Avro Reflection on string results in an invalid avro schema + * [FLUME-2265] - Closed bucket writers should be removed from sfwriters map + * [FLUME-2266] - Update Morphline Sink to kite-0.10.0 + * [FLUME-2270] - Twitter Source Documentation Does not load properly + * [FLUME-2272] - Getting start page returning 503 error + * [FLUME-2283] - Spool Dir source must check interrupt flag before writing to channel + * [FLUME-2289] - Disable maxUnderReplication test which is extremely flakey + * [FLUME-2301] - Update HBaseSink tests to reflect sink returning backoff only on empty batches + * [FLUME-2302] - TestHDFS Sink fails with Can't get Kerberos realm + * [FLUME-2303] - HBaseSink tests can fail based on order of execution + * [FLUME-2304] - DatasetSink test fails unexpectedly + * [FLUME-2305] - BucketWriter#close must cancel idleFuture + * [FLUME-2307] - Remove Log writetimeout + * [FLUME-2311] - Use standard way of finding queue/topic + * [FLUME-2312] - Add utility for adorning HTTP contexts in Jetty + * [FLUME-2314] - Upgrade to Mapdb 0.9.9 + * [FLUME-2320] - Deadlock in DatasetSink + * [FLUME-2323] - Morphline sink must increment eventDrainAttemptCount when it takes event from channel + * [FLUME-2324] - Support writing to multiple HBase clusters using HBaseSink + * [FLUME-2325] - BucketWriter might throw BucketClosedException incorrectly + * [FLUME-2328] - FileChannel Dual Checkpoint Backup Thread not released on Application stop + * [FLUME-2329] - Add an alias for the Morphline Solr Sink + * [FLUME-2330] - Remove the MorphlineHandlerImpl configuration option from MorphlineSink + * [FLUME-2334] - Upgrade Asynchbase to 1.5.0 as this include hbase 0.96.x support + * [FLUME-2335] - TestHBaseSink#testWithoutConfigurationObject() must delete the table at the end of the test + * [FLUME-2336] - HBase tests that pass in ZK configs must use a new context object + * [FLUME-2338] - Support coalescing increments in HBaseSink + * [FLUME-2345] - Update to Kite 0.12.0 dependency + * [FLUME-2347] - Add FLUME_JAVA_OPTS which allows users to inject java properties from cmd line + * [FLUME-2350] - Consume Order tests need to space out file creation + * [FLUME-2357] - HDFS sink should retry closing files that previously had close errors + * [FLUME-2381] - Upgrade Hadoop version in Hadoop 2 profile to 2.4.0 + +** Documentation + * [FLUME-1223] - Userguide improvement rolling file sink + * [FLUME-1678] - Incorrect documentation for HBase sink + * [FLUME-1851] - User Guide grammar mistake + * [FLUME-2064] - Typo/Grammar in flume main user doc under Scribe + * [FLUME-2065] - Regex Extractor Interceptor config agent name inconsistent with rest of docs + * [FLUME-2183] - Add "Other Resources" page to wiki + * [FLUME-2278] - Incorrect documentation for write-timeout of File Channel + * [FLUME-2319] - Incorrect property name of Ganglia Reporting documentation + + Release Notes - Flume - Version v1.4.0 ** New Feature From 45abf412c31f85e1b35135df529a36d94aac694c Mon Sep 17 00:00:00 2001 From: Hari Shreedharan Date: Tue, 6 May 2014 16:28:46 -0700 Subject: [PATCH 103/341] FLUME-2366. Update LICENSE --- LICENSE | 984 +------------------------------------------------------- 1 file changed, 14 insertions(+), 970 deletions(-) diff --git a/LICENSE b/LICENSE index c699bcb71d..ab42e2267f 100644 --- a/LICENSE +++ b/LICENSE @@ -210,328 +210,52 @@ their respective licenses. For apache-log4j-extras-.jar: - apache-mime4j-core-.jar: - apache-mime4j-dom-.jar: avro-.jar: avro-ipc-.jar: - boilerpipe-.jar: - cdk-morphlines-avro-.jar: - cdk-morphlines-core-.jar: - cdk-morphlines-solr-cell-.jar: - cdk-morphlines-solr-core-.jar: - cdk-morphlines-tika-core-.jar: - cdk-morphlines-tika-decompress-.jar: - cdk-morphlines-twitter-.jar: commons-cli-.jar: commons-codec-.jar: commons-collections-.jar: - commons-compress-.jar: commons-dbcp-.jar: - commons-fileupload-.jar: commons-io-.jar: + commons-jexl-.jar: commons-lang-.jar: commons-logging-.jar: commons-pool-.jar: - config-.jar: derby-.jar: - fontbox-.jar: guava-.jar: gson-.jar: httpclient-.jar: httpcore-.jar: - httpmime-.jar: irclib-.jar: - isoparser-1.0-RC-.jar: - jackson-annotations-2..jar: - jackson-core-2..jar: jackson-core-asl-1..jar: - jackson-databind-2..jar: jackson-mapper-asl-1..jar: - jempbox-.jar: jetty-.jar: jetty-util-.jar: joda-time-.jar: + kite-data-core-.jar libthrift-.jar: log4j-.jar: - lucene-analyzers-common-.jar: - lucene-analyzers-kuromoji-.jar: - lucene-analyzers-phonetic-.jar: - lucene-codecs-.jar: - lucene-core-.jar: - lucene-grouping-.jar: - lucene-highlighter-.jar: - lucene-memory-.jar: - lucene-misc-.jar: - lucene-queries-.jar: - lucene-queryparser-.jar: - lucene-spatial-.jar: - lucene-suggest-.jar: - metadata-extractor-.jar: - metrics-core-.jar: + mapdb-.jar: mina-core-.jar: netty-.jar: - noggit-.jar: - org.restlet-.jar: - org.restlet.ext.servlet-.jar: - pdfbox-.jar: - poi-.jar: - poi-ooxml-.jar: - poi-ooxml-schemas-.jar: - poi-scratchpad-.jar: - rome-.jar: + opencsv-.jar: + parquet-avro-.jar: + parquet-column-.jar: + parquet-common-.jar: + parquet-encoding-.jar: + parquet-format-.jar: + parquet-generator-.jar: + parquet-hadoop-.jar: servlet-api-.jar: snappy-java-.jar: - solr-cell-.jar: - solr-core-.jar: - solr-solrj-.jar: - spatial4j-.jar: - tagsoup-.jar: - tika-core-.jar: - tika-parsers-.jar: - tika-xmp-.jar: + twitter4j-core-.jar: + twitter4j-media-support-.jar + twitter4j-stream-.jar velocity-.jar: - vorbis-java-core--tests.jar: - vorbis-java-core-.jar: - vorbis-java-tika-.jar: - wstx-asl-.jar: - xmlbeans-.jar: zookeeper-.jar: The Apache License, Version 2.0 -For - asm-.jar: - - BSD License - - Copyright (c) 2012 France Télécom - All rights reserved. - - Redistribution and use in source and binary forms, with or without - modification, are permitted provided that the following conditions - are met: - 1. Redistributions of source code must retain the above copyright - notice, this list of conditions and the following disclaimer. - 2. Redistributions in binary form must reproduce the above copyright - notice, this list of conditions and the following disclaimer in the - documentation and/or other materials provided with the distribution. - 3. Neither the name of the copyright holders nor the names of its - contributors may be used to endorse or promote products derived from - this software without specific prior written permission. - - THIS SOFTWARE IS PROVIDED BY THE COPYRIGHT HOLDERS AND CONTRIBUTORS "AS IS" - AND ANY EXPRESS OR IMPLIED WARRANTIES, INCLUDING, BUT NOT LIMITED TO, THE - IMPLIED WARRANTIES OF MERCHANTABILITY AND FITNESS FOR A PARTICULAR PURPOSE - ARE DISCLAIMED. IN NO EVENT SHALL THE COPYRIGHT OWNER OR CONTRIBUTORS BE - LIABLE FOR ANY DIRECT, INDIRECT, INCIDENTAL, SPECIAL, EXEMPLARY, OR - CONSEQUENTIAL DAMAGES (INCLUDING, BUT NOT LIMITED TO, PROCUREMENT OF - SUBSTITUTE GOODS OR SERVICES; LOSS OF USE, DATA, OR PROFITS; OR BUSINESS - INTERRUPTION) HOWEVER CAUSED AND ON ANY THEORY OF LIABILITY, WHETHER IN - CONTRACT, STRICT LIABILITY, OR TORT (INCLUDING NEGLIGENCE OR OTHERWISE) - ARISING IN ANY WAY OUT OF THE USE OF THIS SOFTWARE, EVEN IF ADVISED OF - THE POSSIBILITY OF SUCH DAMAGE. - -For - aspectjrt-.jar: - - Eclipse Public License - v 1.0 - - Copyright (c) 2005 Contributors. All rights reserved. - - THE ACCOMPANYING PROGRAM IS PROVIDED UNDER THE TERMS OF THIS ECLIPSE PUBLIC - LICENSE ("AGREEMENT"). ANY USE, REPRODUCTION OR DISTRIBUTION OF THE PROGRAM - CONSTITUTES RECIPIENT'S ACCEPTANCE OF THIS AGREEMENT. - - 1. DEFINITIONS - - "Contribution" means: - - a) in the case of the initial Contributor, the initial code and - documentation distributed under this Agreement, and b) in the case of each - subsequent Contributor: i) changes to the Program, and ii) additions to the - Program; where such changes and/or additions to the Program originate from - and are distributed by that particular Contributor. A Contribution - 'originates' from a Contributor if it was added to the Program by such - Contributor itself or anyone acting on such Contributor's behalf. - Contributions do not include additions to the Program which: (i) are - separate modules of software distributed in conjunction with the Program - under their own license agreement, and (ii) are not derivative works of the - Program. "Contributor" means any person or entity that distributes the - Program. - - "Licensed Patents" mean patent claims licensable by a Contributor which are - necessarily infringed by the use or sale of its Contribution alone or when - combined with the Program. - - "Program" means the Contributions distributed in accordance with this - Agreement. - - "Recipient" means anyone who receives the Program under this Agreement, - including all Contributors. - - 2. GRANT OF RIGHTS - - a) Subject to the terms of this Agreement, each Contributor hereby grants - Recipient a non-exclusive, worldwide, royalty-free copyright license to - reproduce, prepare derivative works of, publicly display, publicly perform, - distribute and sublicense the Contribution of such Contributor, if any, and - such derivative works, in source code and object code form. b) Subject to - the terms of this Agreement, each Contributor hereby grants Recipient a - non-exclusive, worldwide, royalty-free patent license under Licensed Patents - to make, use, sell, offer to sell, import and otherwise transfer the - Contribution of such Contributor, if any, in source code and object code - form. This patent license shall apply to the combination of the Contribution - and the Program if, at the time the Contribution is added by the - Contributor, such addition of the Contribution causes such combination to be - covered by the Licensed Patents. The patent license shall not apply to any - other combinations which include the Contribution. No hardware per se is - licensed hereunder. c) Recipient understands that although each Contributor - grants the licenses to its Contributions set forth herein, no assurances are - provided by any Contributor that the Program does not infringe the patent or - other intellectual property rights of any other entity. Each Contributor - disclaims any liability to Recipient for claims brought by any other entity - based on infringement of intellectual property rights or otherwise. As a - condition to exercising the rights and licenses granted hereunder, each - Recipient hereby assumes sole responsibility to secure any other - intellectual property rights needed, if any. For example, if a third party - patent license is required to allow Recipient to distribute the Program, it - is Recipient's responsibility to acquire that license before distributing - the Program. d) Each Contributor represents that to its knowledge it has - sufficient copyright rights in its Contribution, if any, to grant the - copyright license set forth in this Agreement. 3. REQUIREMENTS - - A Contributor may choose to distribute the Program in object code form under - its own license agreement, provided that: - - a) it complies with the terms and conditions of this Agreement; and b) its - license agreement: i) effectively disclaims on behalf of all Contributors - all warranties and conditions, express and implied, including warranties or - conditions of title and non-infringement, and implied warranties or - conditions of merchantability and fitness for a particular purpose; ii) - effectively excludes on behalf of all Contributors all liability for - damages, including direct, indirect, special, incidental and consequential - damages, such as lost profits; iii) states that any provisions which differ - from this Agreement are offered by that Contributor alone and not by any - other party; and iv) states that source code for the Program is available - from such Contributor, and informs licensees how to obtain it in a - reasonable manner on or through a medium customarily used for software - exchange. When the Program is made available in source code form: - - a) it must be made available under this Agreement; and b) a copy of this - Agreement must be included with each copy of the Program. Contributors may - not remove or alter any copyright notices contained within the Program. - - Each Contributor must identify itself as the originator of its Contribution, - if any, in a manner that reasonably allows subsequent Recipients to identify - the originator of the Contribution. - - 4. COMMERCIAL DISTRIBUTION - - Commercial distributors of software may accept certain responsibilities with - respect to end users, business partners and the like. While this license is - intended to facilitate the commercial use of the Program, the Contributor - who includes the Program in a commercial product offering should do so in a - manner which does not create potential liability for other Contributors. - Therefore, if a Contributor includes the Program in a commercial product - offering, such Contributor ("Commercial Contributor") hereby agrees to - defend and indemnify every other Contributor ("Indemnified Contributor") - against any losses, damages and costs (collectively "Losses") arising from - claims, lawsuits and other legal actions brought by a third party against - the Indemnified Contributor to the extent caused by the acts or omissions of - such Commercial Contributor in connection with its distribution of the - Program in a commercial product offering. The obligations in this section do - not apply to any claims or Losses relating to any actual or alleged - intellectual property infringement. In order to qualify, an Indemnified - Contributor must: a) promptly notify the Commercial Contributor in writing - of such claim, and b) allow the Commercial Contributor to control, and - cooperate with the Commercial Contributor in, the defense and any related - settlement negotiations. The Indemnified Contributor may participate in any - such claim at its own expense. - - For example, a Contributor might include the Program in a commercial product - offering, Product X. That Contributor is then a Commercial Contributor. If - that Commercial Contributor then makes performance claims, or offers - warranties related to Product X, those performance claims and warranties are - such Commercial Contributor's responsibility alone. Under this section, the - Commercial Contributor would have to defend claims against the other - Contributors related to those performance claims and warranties, and if a - court requires any other Contributor to pay any damages as a result, the - Commercial Contributor must pay those damages. - - 5. NO WARRANTY - - EXCEPT AS EXPRESSLY SET FORTH IN THIS AGREEMENT, THE PROGRAM IS PROVIDED ON - AN "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, EITHER - EXPRESS OR IMPLIED INCLUDING, WITHOUT LIMITATION, ANY WARRANTIES OR - CONDITIONS OF TITLE, NON-INFRINGEMENT, MERCHANTABILITY OR FITNESS FOR A - PARTICULAR PURPOSE. Each Recipient is solely responsible for determining the - appropriateness of using and distributing the Program and assumes all risks - associated with its exercise of rights under this Agreement , including but - not limited to the risks and costs of program errors, compliance with - applicable laws, damage to or loss of data, programs or equipment, and - unavailability or interruption of operations. - - 6. DISCLAIMER OF LIABILITY - - EXCEPT AS EXPRESSLY SET FORTH IN THIS AGREEMENT, NEITHER RECIPIENT NOR ANY - CONTRIBUTORS SHALL HAVE ANY LIABILITY FOR ANY DIRECT, INDIRECT, INCIDENTAL, - SPECIAL, EXEMPLARY, OR CONSEQUENTIAL DAMAGES (INCLUDING WITHOUT LIMITATION - LOST PROFITS), HOWEVER CAUSED AND ON ANY THEORY OF LIABILITY, WHETHER IN - CONTRACT, STRICT LIABILITY, OR TORT (INCLUDING NEGLIGENCE OR OTHERWISE) - ARISING IN ANY WAY OUT OF THE USE OR DISTRIBUTION OF THE PROGRAM OR THE - EXERCISE OF ANY RIGHTS GRANTED HEREUNDER, EVEN IF ADVISED OF THE POSSIBILITY - OF SUCH DAMAGES. - - 7. GENERAL - - If any provision of this Agreement is invalid or unenforceable under - applicable law, it shall not affect the validity or enforceability of the - remainder of the terms of this Agreement, and without further action by the - parties hereto, such provision shall be reformed to the minimum extent - necessary to make such provision valid and enforceable. - - If Recipient institutes patent litigation against any entity (including a - cross-claim or counterclaim in a lawsuit) alleging that the Program itself - (excluding combinations of the Program with other software or hardware) - infringes such Recipient's patent(s), then such Recipient's rights granted - under Section 2(b) shall terminate as of the date such litigation is filed. - - All Recipient's rights under this Agreement shall terminate if it fails to - comply with any of the material terms or conditions of this Agreement and - does not cure such failure in a reasonable period of time after becoming - aware of such noncompliance. If all Recipient's rights under this Agreement - terminate, Recipient agrees to cease use and distribution of the Program as - soon as reasonably practicable. However, Recipient's obligations under this - Agreement and any licenses granted by Recipient relating to the Program - shall continue and survive. - - Everyone is permitted to copy and distribute copies of this Agreement, but - in order to avoid inconsistency the Agreement is copyrighted and may only be - modified in the following manner. The Agreement Steward reserves the right - to publish new versions (including revisions) of this Agreement from time to - time. No one other than the Agreement Steward has the right to modify this - Agreement. The Eclipse Foundation is the initial Agreement Steward. The - Eclipse Foundation may assign the responsibility to serve as the Agreement - Steward to a suitable separate entity. Each new version of the Agreement - will be given a distinguishing version number. The Program (including - Contributions) may always be distributed subject to the version of the - Agreement under which it was received. In addition, after a new version of - the Agreement is published, Contributor may elect to distribute the Program - (including its Contributions) under the new version. Except as expressly - stated in Sections 2(a) and 2(b) above, Recipient receives no rights or - licenses to the intellectual property of any Contributor under this - Agreement, whether expressly, by implication, estoppel or otherwise. All - rights in the Program not expressly granted under this Agreement are - reserved. - - This Agreement is governed by the laws of the State of New York and the - intellectual property laws of the United States of America. No party to this - Agreement will bring a legal action under this Agreement more than one year - after the cause of action arose. Each party waives its rights to a jury - trial in any resulting litigation. - - For more information, see http://eclipse.org/aspectj/ - For async-.jar: @@ -591,610 +315,6 @@ For ARISING IN ANY WAY OUT OF THE USE OF THIS SOFTWARE, EVEN IF ADVISED OF THE POSSIBILITY OF SUCH DAMAGE. -For - bcmail-jdk15-.jar: - bcprov-jdk15-.jar: - - MIT style license - - Copyright (c) 2000 - 2013 The Legion Of The Bouncy Castle - (http://www.bouncycastle.org) - - Permission is hereby granted, free of charge, to any person obtaining a copy - of this software and associated documentation files (the "Software"), to - deal in the Software without restriction, including without limitation the - rights to use, copy, modify, merge, publish, distribute, sublicense, and/or - sell copies of the Software, and to permit persons to whom the Software is - furnished to do so, subject to the following conditions: - - The above copyright notice and this permission notice shall be included in - all copies or substantial portions of the Software. - - THE SOFTWARE IS PROVIDED "AS IS", WITHOUT WARRANTY OF ANY KIND, EXPRESS OR - IMPLIED, INCLUDING BUT NOT LIMITED TO THE WARRANTIES OF MERCHANTABILITY, - FITNESS FOR A PARTICULAR PURPOSE AND NONINFRINGEMENT. IN NO EVENT SHALL THE - AUTHORS OR COPYRIGHT HOLDERS BE LIABLE FOR ANY CLAIM, DAMAGES OR OTHER - LIABILITY, WHETHER IN AN ACTION OF CONTRACT, TORT OR OTHERWISE, ARISING - FROM, OUT OF OR IN CONNECTION WITH THE SOFTWARE OR THE USE OR OTHER DEALINGS - IN THE SOFTWARE. - -For - dom4j-.jar: - - BSD style license - - Copyright 2001-2010 (C) MetaStuff, Ltd. All Rights Reserved. - - Redistribution and use of this software and associated documentation - ("Software"), with or without modification, are permitted provided - that the following conditions are met: - - 1. Redistributions of source code must retain copyright - statements and notices. Redistributions must also contain a - copy of this document. - - 2. Redistributions in binary form must reproduce the - above copyright notice, this list of conditions and the - following disclaimer in the documentation and/or other - materials provided with the distribution. - - 3. The name "DOM4J" must not be used to endorse or promote - products derived from this Software without prior written - permission of MetaStuff, Ltd. For written permission, - please contact dom4j-info@metastuff.com. - - 4. Products derived from this Software may not be called "DOM4J" - nor may "DOM4J" appear in their names without prior written - permission of MetaStuff, Ltd. DOM4J is a registered - trademark of MetaStuff, Ltd. - - 5. Due credit should be given to the DOM4J Project - - http://dom4j.sourceforge.net - - THIS SOFTWARE IS PROVIDED BY METASTUFF, LTD. AND CONTRIBUTORS - ``AS IS'' AND ANY EXPRESSED OR IMPLIED WARRANTIES, INCLUDING, BUT - NOT LIMITED TO, THE IMPLIED WARRANTIES OF MERCHANTABILITY AND - FITNESS FOR A PARTICULAR PURPOSE ARE DISCLAIMED. IN NO EVENT SHALL - METASTUFF, LTD. OR ITS CONTRIBUTORS BE LIABLE FOR ANY DIRECT, - INDIRECT, INCIDENTAL, SPECIAL, EXEMPLARY, OR CONSEQUENTIAL DAMAGES - (INCLUDING, BUT NOT LIMITED TO, PROCUREMENT OF SUBSTITUTE GOODS OR - SERVICES; LOSS OF USE, DATA, OR PROFITS; OR BUSINESS INTERRUPTION) - HOWEVER CAUSED AND ON ANY THEORY OF LIABILITY, WHETHER IN CONTRACT, - STRICT LIABILITY, OR TORT (INCLUDING NEGLIGENCE OR OTHERWISE) - ARISING IN ANY WAY OUT OF THE USE OF THIS SOFTWARE, EVEN IF ADVISED - OF THE POSSIBILITY OF SUCH DAMAGE. - - For more information, see https://code.google.com/p/juniversalchardet/ - -For - jdom-.jar: - - Apache style license - - Copyright (C) 2000-2012 Jason Hunter & Brett McLaughlin. - All rights reserved. - - Redistribution and use in source and binary forms, with or without - modification, are permitted provided that the following conditions - are met: - - 1. Redistributions of source code must retain the above copyright - notice, this list of conditions, and the following disclaimer. - - 2. Redistributions in binary form must reproduce the above copyright - notice, this list of conditions, and the disclaimer that follows - these conditions in the documentation and/or other materials - provided with the distribution. - - 3. The name "JDOM" must not be used to endorse or promote products - derived from this software without prior written permission. For - written permission, please contact . - - 4. Products derived from this software may not be called "JDOM", nor - may "JDOM" appear in their name, without prior written permission - from the JDOM Project Management . - - In addition, we request (but do not require) that you include in the - end-user documentation provided with the redistribution and/or in the - software itself an acknowledgement equivalent to the following: - "This product includes software developed by the - JDOM Project (http://www.jdom.org/)." - Alternatively, the acknowledgment may be graphical using the logos - available at http://www.jdom.org/images/logos. - - THIS SOFTWARE IS PROVIDED ``AS IS'' AND ANY EXPRESSED OR IMPLIED - WARRANTIES, INCLUDING, BUT NOT LIMITED TO, THE IMPLIED WARRANTIES - OF MERCHANTABILITY AND FITNESS FOR A PARTICULAR PURPOSE ARE - DISCLAIMED. IN NO EVENT SHALL THE JDOM AUTHORS OR THE PROJECT - CONTRIBUTORS BE LIABLE FOR ANY DIRECT, INDIRECT, INCIDENTAL, - SPECIAL, EXEMPLARY, OR CONSEQUENTIAL DAMAGES (INCLUDING, BUT NOT - LIMITED TO, PROCUREMENT OF SUBSTITUTE GOODS OR SERVICES; LOSS OF - USE, DATA, OR PROFITS; OR BUSINESS INTERRUPTION) HOWEVER CAUSED AND - ON ANY THEORY OF LIABILITY, WHETHER IN CONTRACT, STRICT LIABILITY, - OR TORT (INCLUDING NEGLIGENCE OR OTHERWISE) ARISING IN ANY WAY OUT - OF THE USE OF THIS SOFTWARE, EVEN IF ADVISED OF THE POSSIBILITY OF - SUCH DAMAGE. - - This software consists of voluntary contributions made by many - individuals on behalf of the JDOM Project and was originally - created by Jason Hunter and - Brett McLaughlin . For more information - on the JDOM Project, please see . - -For - juniversalchardet-.jar: - - MOZILLA PUBLIC LICENSE - Version 1.1 - - --------------- - - 1. Definitions. - - 1.0.1. "Commercial Use" means distribution or otherwise making the - Covered Code available to a third party. - - 1.1. "Contributor" means each entity that creates or contributes to - the creation of Modifications. - - 1.2. "Contributor Version" means the combination of the Original - Code, prior Modifications used by a Contributor, and the Modifications - made by that particular Contributor. - - 1.3. "Covered Code" means the Original Code or Modifications or the - combination of the Original Code and Modifications, in each case - including portions thereof. - - 1.4. "Electronic Distribution Mechanism" means a mechanism generally - accepted in the software development community for the electronic - transfer of data. - - 1.5. "Executable" means Covered Code in any form other than Source - Code. - - 1.6. "Initial Developer" means the individual or entity identified - as the Initial Developer in the Source Code notice required by Exhibit - A. - - 1.7. "Larger Work" means a work which combines Covered Code or - portions thereof with code not governed by the terms of this License. - - 1.8. "License" means this document. - - 1.8.1. "Licensable" means having the right to grant, to the maximum - extent possible, whether at the time of the initial grant or - subsequently acquired, any and all of the rights conveyed herein. - - 1.9. "Modifications" means any addition to or deletion from the - substance or structure of either the Original Code or any previous - Modifications. When Covered Code is released as a series of files, a - Modification is: - A. Any addition to or deletion from the contents of a file - containing Original Code or previous Modifications. - - B. Any new file that contains any part of the Original Code or - previous Modifications. - - 1.10. "Original Code" means Source Code of computer software code - which is described in the Source Code notice required by Exhibit A as - Original Code, and which, at the time of its release under this - License is not already Covered Code governed by this License. - - 1.10.1. "Patent Claims" means any patent claim(s), now owned or - hereafter acquired, including without limitation, method, process, - and apparatus claims, in any patent Licensable by grantor. - - 1.11. "Source Code" means the preferred form of the Covered Code for - making modifications to it, including all modules it contains, plus - any associated interface definition files, scripts used to control - compilation and installation of an Executable, or source code - differential comparisons against either the Original Code or another - well known, available Covered Code of the Contributor's choice. The - Source Code can be in a compressed or archival form, provided the - appropriate decompression or de-archiving software is widely available - for no charge. - - 1.12. "You" (or "Your") means an individual or a legal entity - exercising rights under, and complying with all of the terms of, this - License or a future version of this License issued under Section 6.1. - For legal entities, "You" includes any entity which controls, is - controlled by, or is under common control with You. For purposes of - this definition, "control" means (a) the power, direct or indirect, - to cause the direction or management of such entity, whether by - contract or otherwise, or (b) ownership of more than fifty percent - (50%) of the outstanding shares or beneficial ownership of such - entity. - - 2. Source Code License. - - 2.1. The Initial Developer Grant. - The Initial Developer hereby grants You a world-wide, royalty-free, - non-exclusive license, subject to third party intellectual property - claims: - (a) under intellectual property rights (other than patent or - trademark) Licensable by Initial Developer to use, reproduce, - modify, display, perform, sublicense and distribute the Original - Code (or portions thereof) with or without Modifications, and/or - as part of a Larger Work; and - - (b) under Patents Claims infringed by the making, using or - selling of Original Code, to make, have made, use, practice, - sell, and offer for sale, and/or otherwise dispose of the - Original Code (or portions thereof). - - (c) the licenses granted in this Section 2.1(a) and (b) are - effective on the date Initial Developer first distributes - Original Code under the terms of this License. - - (d) Notwithstanding Section 2.1(b) above, no patent license is - granted: 1) for code that You delete from the Original Code; 2) - separate from the Original Code; or 3) for infringements caused - by: i) the modification of the Original Code or ii) the - combination of the Original Code with other software or devices. - - 2.2. Contributor Grant. - Subject to third party intellectual property claims, each Contributor - hereby grants You a world-wide, royalty-free, non-exclusive license - - (a) under intellectual property rights (other than patent or - trademark) Licensable by Contributor, to use, reproduce, modify, - display, perform, sublicense and distribute the Modifications - created by such Contributor (or portions thereof) either on an - unmodified basis, with other Modifications, as Covered Code - and/or as part of a Larger Work; and - - (b) under Patent Claims infringed by the making, using, or - selling of Modifications made by that Contributor either alone - and/or in combination with its Contributor Version (or portions - of such combination), to make, use, sell, offer for sale, have - made, and/or otherwise dispose of: 1) Modifications made by that - Contributor (or portions thereof); and 2) the combination of - Modifications made by that Contributor with its Contributor - Version (or portions of such combination). - - (c) the licenses granted in Sections 2.2(a) and 2.2(b) are - effective on the date Contributor first makes Commercial Use of - the Covered Code. - - (d) Notwithstanding Section 2.2(b) above, no patent license is - granted: 1) for any code that Contributor has deleted from the - Contributor Version; 2) separate from the Contributor Version; - 3) for infringements caused by: i) third party modifications of - Contributor Version or ii) the combination of Modifications made - by that Contributor with other software (except as part of the - Contributor Version) or other devices; or 4) under Patent Claims - infringed by Covered Code in the absence of Modifications made by - that Contributor. - - 3. Distribution Obligations. - - 3.1. Application of License. - The Modifications which You create or to which You contribute are - governed by the terms of this License, including without limitation - Section 2.2. The Source Code version of Covered Code may be - distributed only under the terms of this License or a future version - of this License released under Section 6.1, and You must include a - copy of this License with every copy of the Source Code You - distribute. You may not offer or impose any terms on any Source Code - version that alters or restricts the applicable version of this - License or the recipients' rights hereunder. However, You may include - an additional document offering the additional rights described in - Section 3.5. - - 3.2. Availability of Source Code. - Any Modification which You create or to which You contribute must be - made available in Source Code form under the terms of this License - either on the same media as an Executable version or via an accepted - Electronic Distribution Mechanism to anyone to whom you made an - Executable version available; and if made available via Electronic - Distribution Mechanism, must remain available for at least twelve (12) - months after the date it initially became available, or at least six - (6) months after a subsequent version of that particular Modification - has been made available to such recipients. You are responsible for - ensuring that the Source Code version remains available even if the - Electronic Distribution Mechanism is maintained by a third party. - - 3.3. Description of Modifications. - You must cause all Covered Code to which You contribute to contain a - file documenting the changes You made to create that Covered Code and - the date of any change. You must include a prominent statement that - the Modification is derived, directly or indirectly, from Original - Code provided by the Initial Developer and including the name of the - Initial Developer in (a) the Source Code, and (b) in any notice in an - Executable version or related documentation in which You describe the - origin or ownership of the Covered Code. - - 3.4. Intellectual Property Matters - (a) Third Party Claims. - If Contributor has knowledge that a license under a third party's - intellectual property rights is required to exercise the rights - granted by such Contributor under Sections 2.1 or 2.2, - Contributor must include a text file with the Source Code - distribution titled "LEGAL" which describes the claim and the - party making the claim in sufficient detail that a recipient will - know whom to contact. If Contributor obtains such knowledge after - the Modification is made available as described in Section 3.2, - Contributor shall promptly modify the LEGAL file in all copies - Contributor makes available thereafter and shall take other steps - (such as notifying appropriate mailing lists or newsgroups) - reasonably calculated to inform those who received the Covered - Code that new knowledge has been obtained. - - (b) Contributor APIs. - If Contributor's Modifications include an application programming - interface and Contributor has knowledge of patent licenses which - are reasonably necessary to implement that API, Contributor must - also include this information in the LEGAL file. - - (c) Representations. - Contributor represents that, except as disclosed pursuant to - Section 3.4(a) above, Contributor believes that Contributor's - Modifications are Contributor's original creation(s) and/or - Contributor has sufficient rights to grant the rights conveyed by - this License. - - 3.5. Required Notices. - You must duplicate the notice in Exhibit A in each file of the Source - Code. If it is not possible to put such notice in a particular Source - Code file due to its structure, then You must include such notice in a - location (such as a relevant directory) where a user would be likely - to look for such a notice. If You created one or more Modification(s) - You may add your name as a Contributor to the notice described in - Exhibit A. You must also duplicate this License in any documentation - for the Source Code where You describe recipients' rights or ownership - rights relating to Covered Code. You may choose to offer, and to - charge a fee for, warranty, support, indemnity or liability - obligations to one or more recipients of Covered Code. However, You - may do so only on Your own behalf, and not on behalf of the Initial - Developer or any Contributor. You must make it absolutely clear than - any such warranty, support, indemnity or liability obligation is - offered by You alone, and You hereby agree to indemnify the Initial - Developer and every Contributor for any liability incurred by the - Initial Developer or such Contributor as a result of warranty, - support, indemnity or liability terms You offer. - - 3.6. Distribution of Executable Versions. - You may distribute Covered Code in Executable form only if the - requirements of Section 3.1-3.5 have been met for that Covered Code, - and if You include a notice stating that the Source Code version of - the Covered Code is available under the terms of this License, - including a description of how and where You have fulfilled the - obligations of Section 3.2. The notice must be conspicuously included - in any notice in an Executable version, related documentation or - collateral in which You describe recipients' rights relating to the - Covered Code. You may distribute the Executable version of Covered - Code or ownership rights under a license of Your choice, which may - contain terms different from this License, provided that You are in - compliance with the terms of this License and that the license for the - Executable version does not attempt to limit or alter the recipient's - rights in the Source Code version from the rights set forth in this - License. If You distribute the Executable version under a different - license You must make it absolutely clear that any terms which differ - from this License are offered by You alone, not by the Initial - Developer or any Contributor. You hereby agree to indemnify the - Initial Developer and every Contributor for any liability incurred by - the Initial Developer or such Contributor as a result of any such - terms You offer. - - 3.7. Larger Works. - You may create a Larger Work by combining Covered Code with other code - not governed by the terms of this License and distribute the Larger - Work as a single product. In such a case, You must make sure the - requirements of this License are fulfilled for the Covered Code. - - 4. Inability to Comply Due to Statute or Regulation. - - If it is impossible for You to comply with any of the terms of this - License with respect to some or all of the Covered Code due to - statute, judicial order, or regulation then You must: (a) comply with - the terms of this License to the maximum extent possible; and (b) - describe the limitations and the code they affect. Such description - must be included in the LEGAL file described in Section 3.4 and must - be included with all distributions of the Source Code. Except to the - extent prohibited by statute or regulation, such description must be - sufficiently detailed for a recipient of ordinary skill to be able to - understand it. - - 5. Application of this License. - - This License applies to code to which the Initial Developer has - attached the notice in Exhibit A and to related Covered Code. - - 6. Versions of the License. - - 6.1. New Versions. - Netscape Communications Corporation ("Netscape") may publish revised - and/or new versions of the License from time to time. Each version - will be given a distinguishing version number. - - 6.2. Effect of New Versions. - Once Covered Code has been published under a particular version of the - License, You may always continue to use it under the terms of that - version. You may also choose to use such Covered Code under the terms - of any subsequent version of the License published by Netscape. No one - other than Netscape has the right to modify the terms applicable to - Covered Code created under this License. - - 6.3. Derivative Works. - If You create or use a modified version of this License (which you may - only do in order to apply it to code which is not already Covered Code - governed by this License), You must (a) rename Your license so that - the phrases "Mozilla", "MOZILLAPL", "MOZPL", "Netscape", - "MPL", "NPL" or any confusingly similar phrase do not appear in your - license (except to note that your license differs from this License) - and (b) otherwise make it clear that Your version of the license - contains terms which differ from the Mozilla Public License and - Netscape Public License. (Filling in the name of the Initial - Developer, Original Code or Contributor in the notice described in - Exhibit A shall not of themselves be deemed to be modifications of - this License.) - - 7. DISCLAIMER OF WARRANTY. - - COVERED CODE IS PROVIDED UNDER THIS LICENSE ON AN "AS IS" BASIS, - WITHOUT WARRANTY OF ANY KIND, EITHER EXPRESSED OR IMPLIED, INCLUDING, - WITHOUT LIMITATION, WARRANTIES THAT THE COVERED CODE IS FREE OF - DEFECTS, MERCHANTABLE, FIT FOR A PARTICULAR PURPOSE OR NON-INFRINGING. - THE ENTIRE RISK AS TO THE QUALITY AND PERFORMANCE OF THE COVERED CODE - IS WITH YOU. SHOULD ANY COVERED CODE PROVE DEFECTIVE IN ANY RESPECT, - YOU (NOT THE INITIAL DEVELOPER OR ANY OTHER CONTRIBUTOR) ASSUME THE - COST OF ANY NECESSARY SERVICING, REPAIR OR CORRECTION. THIS DISCLAIMER - OF WARRANTY CONSTITUTES AN ESSENTIAL PART OF THIS LICENSE. NO USE OF - ANY COVERED CODE IS AUTHORIZED HEREUNDER EXCEPT UNDER THIS DISCLAIMER. - - 8. TERMINATION. - - 8.1. This License and the rights granted hereunder will terminate - automatically if You fail to comply with terms herein and fail to cure - such breach within 30 days of becoming aware of the breach. All - sublicenses to the Covered Code which are properly granted shall - survive any termination of this License. Provisions which, by their - nature, must remain in effect beyond the termination of this License - shall survive. - - 8.2. If You initiate litigation by asserting a patent infringement - claim (excluding declatory judgment actions) against Initial Developer - or a Contributor (the Initial Developer or Contributor against whom - You file such action is referred to as "Participant") alleging that: - - (a) such Participant's Contributor Version directly or indirectly - infringes any patent, then any and all rights granted by such - Participant to You under Sections 2.1 and/or 2.2 of this License - shall, upon 60 days notice from Participant terminate prospectively, - unless if within 60 days after receipt of notice You either: (i) - agree in writing to pay Participant a mutually agreeable reasonable - royalty for Your past and future use of Modifications made by such - Participant, or (ii) withdraw Your litigation claim with respect to - the Contributor Version against such Participant. If within 60 days - of notice, a reasonable royalty and payment arrangement are not - mutually agreed upon in writing by the parties or the litigation claim - is not withdrawn, the rights granted by Participant to You under - Sections 2.1 and/or 2.2 automatically terminate at the expiration of - the 60 day notice period specified above. - - (b) any software, hardware, or device, other than such Participant's - Contributor Version, directly or indirectly infringes any patent, then - any rights granted to You by such Participant under Sections 2.1(b) - and 2.2(b) are revoked effective as of the date You first made, used, - sold, distributed, or had made, Modifications made by that - Participant. - - 8.3. If You assert a patent infringement claim against Participant - alleging that such Participant's Contributor Version directly or - indirectly infringes any patent where such claim is resolved (such as - by license or settlement) prior to the initiation of patent - infringement litigation, then the reasonable value of the licenses - granted by such Participant under Sections 2.1 or 2.2 shall be taken - into account in determining the amount or value of any payment or - license. - - 8.4. In the event of termination under Sections 8.1 or 8.2 above, - all end user license agreements (excluding distributors and resellers) - which have been validly granted by You or any distributor hereunder - prior to termination shall survive termination. - - 9. LIMITATION OF LIABILITY. - - UNDER NO CIRCUMSTANCES AND UNDER NO LEGAL THEORY, WHETHER TORT - (INCLUDING NEGLIGENCE), CONTRACT, OR OTHERWISE, SHALL YOU, THE INITIAL - DEVELOPER, ANY OTHER CONTRIBUTOR, OR ANY DISTRIBUTOR OF COVERED CODE, - OR ANY SUPPLIER OF ANY OF SUCH PARTIES, BE LIABLE TO ANY PERSON FOR - ANY INDIRECT, SPECIAL, INCIDENTAL, OR CONSEQUENTIAL DAMAGES OF ANY - CHARACTER INCLUDING, WITHOUT LIMITATION, DAMAGES FOR LOSS OF GOODWILL, - WORK STOPPAGE, COMPUTER FAILURE OR MALFUNCTION, OR ANY AND ALL OTHER - COMMERCIAL DAMAGES OR LOSSES, EVEN IF SUCH PARTY SHALL HAVE BEEN - INFORMED OF THE POSSIBILITY OF SUCH DAMAGES. THIS LIMITATION OF - LIABILITY SHALL NOT APPLY TO LIABILITY FOR DEATH OR PERSONAL INJURY - RESULTING FROM SUCH PARTY'S NEGLIGENCE TO THE EXTENT APPLICABLE LAW - PROHIBITS SUCH LIMITATION. SOME JURISDICTIONS DO NOT ALLOW THE - EXCLUSION OR LIMITATION OF INCIDENTAL OR CONSEQUENTIAL DAMAGES, SO - THIS EXCLUSION AND LIMITATION MAY NOT APPLY TO YOU. - - 10. U.S. GOVERNMENT END USERS. - - The Covered Code is a "commercial item," as that term is defined in - 48 C.F.R. 2.101 (Oct. 1995), consisting of "commercial computer - software" and "commercial computer software documentation," as such - terms are used in 48 C.F.R. 12.212 (Sept. 1995). Consistent with 48 - C.F.R. 12.212 and 48 C.F.R. 227.7202-1 through 227.7202-4 (June 1995), - all U.S. Government End Users acquire Covered Code with only those - rights set forth herein. - - 11. MISCELLANEOUS. - - This License represents the complete agreement concerning subject - matter hereof. If any provision of this License is held to be - unenforceable, such provision shall be reformed only to the extent - necessary to make it enforceable. This License shall be governed by - California law provisions (except to the extent applicable law, if - any, provides otherwise), excluding its conflict-of-law provisions. - With respect to disputes in which at least one party is a citizen of, - or an entity chartered or registered to do business in the United - States of America, any litigation relating to this License shall be - subject to the jurisdiction of the Federal Courts of the Northern - District of California, with venue lying in Santa Clara County, - California, with the losing party responsible for costs, including - without limitation, court costs and reasonable attorneys' fees and - expenses. The application of the United Nations Convention on - Contracts for the International Sale of Goods is expressly excluded. - Any law or regulation which provides that the language of a contract - shall be construed against the drafter shall not apply to this - License. - - 12. RESPONSIBILITY FOR CLAIMS. - - As between Initial Developer and the Contributors, each party is - responsible for claims and damages arising, directly or indirectly, - out of its utilization of rights under this License and You agree to - work with Initial Developer and Contributors to distribute such - responsibility on an equitable basis. Nothing herein is intended or - shall be deemed to constitute any admission of liability. - - 13. MULTIPLE-LICENSED CODE. - - Initial Developer may designate portions of the Covered Code as - "Multiple-Licensed". "Multiple-Licensed" means that the Initial - Developer permits you to utilize portions of the Covered Code under - Your choice of the MPL or the alternative licenses, if any, specified - by the Initial Developer in the file described in Exhibit A. - - EXHIBIT A -Mozilla Public License. - - ``The contents of this file are subject to the Mozilla Public License - Version 1.1 (the "License"); you may not use this file except in - compliance with the License. You may obtain a copy of the License at - http://www.mozilla.org/MPL/ - - Software distributed under the License is distributed on an "AS IS" - basis, WITHOUT WARRANTY OF ANY KIND, either express or implied. See the - License for the specific language governing rights and limitations - under the License. - - The Original Code is ______________________________________. - - The Initial Developer of the Original Code is ________________________. - Portions created by ______________________ are Copyright (C) ______ - _______________________. All Rights Reserved. - - Contributor(s): ______________________________________. - - Alternatively, the contents of this file may be used under the terms - of the _____ license (the "[___] License"), in which case the - provisions of [______] License are applicable instead of those - above. If you wish to allow use of your version of this file only - under the terms of the [____] License and not to allow others to use - your version of this file under the MPL, indicate your decision by - deleting the provisions above and replace them with the notice and - other provisions required by the [___] License. If you do not delete - the provisions above, a recipient may use your version of this file - under either the MPL or the [___] License." - - [NOTE: The text of this Exhibit A may differ slightly from the text of - the notices in the Source Code files of the Original Code. You should - use the text of this Exhibit A rather than the text found in the - Original Code Source Code for Your Modifications.] - - For jsr305-.jar: @@ -1229,42 +349,6 @@ For ARISING IN ANY WAY OUT OF THE USE OF THIS SOFTWARE, EVEN IF ADVISED OF THE POSSIBILITY OF SUCH DAMAGE. -For - netcdf--min.jar: - - MIT style license - - Copyright 1993-2013 University Corporation for Atmospheric Research/Unidata - - Portions of this software were developed by the Unidata Program at the - University Corporation for Atmospheric Research. - - Access and use of this software shall impose the following obligations and - understandings on the user. The user is granted the right, without any fee - or cost, to use, copy, modify, alter, enhance and distribute this software, - and any derivative works thereof, and its supporting documentation for any - purpose whatsoever, provided that this entire notice appears in all copies - of the software, derivative works and supporting documentation. Further, - UCAR requests that the user credit UCAR/Unidata in any publications that - result from the use of this software or in any product that includes this - software, although this is not an obligation. The names UCAR and/or Unidata, - however, may not be used in any advertising or publicity to endorse or - promote any products or commercial entity unless specific written permission - is obtained from UCAR/Unidata. The user also understands that UCAR/Unidata - is not obligated to provide the user with any support, consulting, training - or assistance of any kind with regard to the use, operation and performance - of this software nor to provide the user with any updates, revisions, new - versions or "bug fixes." - - THIS SOFTWARE IS PROVIDED BY UCAR/UNIDATA "AS IS" AND ANY EXPRESS OR IMPLIED - WARRANTIES, INCLUDING, BUT NOT LIMITED TO, THE IMPLIED WARRANTIES OF - MERCHANTABILITY AND FITNESS FOR A PARTICULAR PURPOSE ARE DISCLAIMED. IN NO - EVENT SHALL UCAR/UNIDATA BE LIABLE FOR ANY SPECIAL, INDIRECT OR - CONSEQUENTIAL DAMAGES OR ANY DAMAGES WHATSOEVER RESULTING FROM LOSS OF USE, - DATA OR PROFITS, WHETHER IN AN ACTION OF CONTRACT, NEGLIGENCE OR OTHER - TORTIOUS ACTION, ARISING OUT OF OR IN CONNECTION WITH THE ACCESS, USE OR - PERFORMANCE OF THIS SOFTWARE. - For paranamer-.jar: @@ -1338,7 +422,6 @@ For support library is itself covered by the above license. For - jcl-over-slf4j-.jar: slf4j-api-.jar: slf4j-log4j12-.jar: @@ -1365,42 +448,3 @@ For LIABLE FOR ANY CLAIM, DAMAGES OR OTHER LIABILITY, WHETHER IN AN ACTION OF CONTRACT, TORT OR OTHERWISE, ARISING FROM, OUT OF OR IN CONNECTION WITH THE SOFTWARE OR THE USE OR OTHER DEALINGS IN THE SOFTWARE. - -For - xmpcore-.jar: - - The BSD License - - Copyright (c) 2009, Adobe Systems Incorporated All rights reserved. - - Redistribution and use in source and binary forms, with or without - modification, are permitted provided that the following conditions are met: - - * Redistributions of source code must retain the above copyright notice, - this list of conditions and the following disclaimer. - * Redistributions in binary form must reproduce the above copyright notice, - this list of conditions and the following disclaimer in the documentation - and/or other materials provided with the distribution. - * Neither the name of Adobe Systems Incorporated, nor the names of its - contributors may be used to endorse or promote products derived from this - software without specific prior written permission. - - THIS SOFTWARE IS PROVIDED BY THE COPYRIGHT HOLDERS AND CONTRIBUTORS "AS IS" - AND ANY EXPRESS OR IMPLIED WARRANTIES, INCLUDING, BUT NOT LIMITED TO, THE - IMPLIED WARRANTIES OF MERCHANT ABILITY AND FITNESS FOR A PARTICULAR PURPOSE - ARE DISCLAIMED. IN NO EVENT SHALL THE COPYRIGHT OWNER OR CONTRIBUTORS BE - LIABLE FOR ANY DIRECT, INDIRECT, INCIDENTAL, SPECIAL, EXEMPLARY, OR - CONSEQUENTIAL DAMAGES (INCLUDING, BUT NOT LIMITED TO, PROCUREMENT OF - SUBSTITUTE GOODS OR SERVICES; LOSS OF USE, DATA, OR PROFITS; OR BUSINESS - INTERRUPTION) HOWEVER CAUSED AND ON ANY THEORY OF LIABILITY, WHETHER IN - CONTRACT, STRICT LIABILITY, OR TORT (INCLUDING NEGLIGENCE OR OTHERWISE) - ARISING IN ANY WAY OUT OF THE USE OF THIS SOFTWARE, EVEN IF ADVISED OF THE - POSSIBILITY OF SUCH DAMAGE. - -For - xz-.jar: - - This Java implementation of XZ has been put into the public domain, thus you - can do whatever you want with it. All the files in the package have been - written by Lasse Collin, but some files are heavily based on public domain - code written by Igor Pavlov. From 63bade24bbabbe9eeb887e3294d4d3727cbac25d Mon Sep 17 00:00:00 2001 From: Hari Shreedharan Date: Tue, 6 May 2014 16:38:20 -0700 Subject: [PATCH 104/341] FLUME-2366. Preparing for Flume 1.5.0 release --- RELEASE-NOTES | 2 +- flume-ng-channels/flume-file-channel/pom.xml | 2 +- flume-ng-channels/flume-jdbc-channel/pom.xml | 2 +- .../flume-spillable-memory-channel/pom.xml | 2 +- flume-ng-channels/pom.xml | 2 +- .../flume-ng-log4jappender/pom.xml | 2 +- flume-ng-clients/pom.xml | 2 +- flume-ng-configuration/pom.xml | 2 +- flume-ng-core/pom.xml | 2 +- flume-ng-dist/pom.xml | 2 +- flume-ng-doc/sphinx/FlumeDeveloperGuide.rst | 2 +- flume-ng-doc/sphinx/FlumeUserGuide.rst | 6 +-- flume-ng-doc/sphinx/index.rst | 2 +- flume-ng-embedded-agent/pom.xml | 2 +- .../flume-avro-source/pom.xml | 2 +- .../flume-thrift-source/pom.xml | 2 +- flume-ng-legacy-sources/pom.xml | 2 +- flume-ng-node/pom.xml | 2 +- flume-ng-sdk/pom.xml | 2 +- flume-ng-sinks/flume-dataset-sink/pom.xml | 2 +- flume-ng-sinks/flume-hdfs-sink/pom.xml | 2 +- flume-ng-sinks/flume-irc-sink/pom.xml | 2 +- .../flume-ng-elasticsearch-sink/pom.xml | 2 +- flume-ng-sinks/flume-ng-hbase-sink/pom.xml | 2 +- .../flume-ng-morphline-solr-sink/pom.xml | 4 +- flume-ng-sinks/pom.xml | 2 +- flume-ng-sources/flume-jms-source/pom.xml | 2 +- flume-ng-sources/flume-scribe-source/pom.xml | 2 +- flume-ng-sources/flume-twitter-source/pom.xml | 2 +- flume-ng-sources/pom.xml | 2 +- flume-ng-tests/pom.xml | 2 +- flume-tools/pom.xml | 2 +- pom.xml | 50 +++++++++---------- 33 files changed, 60 insertions(+), 60 deletions(-) diff --git a/RELEASE-NOTES b/RELEASE-NOTES index 0558069dbf..263ba43b04 100644 --- a/RELEASE-NOTES +++ b/RELEASE-NOTES @@ -1,4 +1,4 @@ -Apache Flume 1.4.0 +Apache Flume 1.5.0 CONTENTS 1. What is Apache Flume diff --git a/flume-ng-channels/flume-file-channel/pom.xml b/flume-ng-channels/flume-file-channel/pom.xml index eacd329969..3113938a97 100644 --- a/flume-ng-channels/flume-file-channel/pom.xml +++ b/flume-ng-channels/flume-file-channel/pom.xml @@ -24,7 +24,7 @@ flume-ng-channels org.apache.flume - 1.5.0-SNAPSHOT + 1.6.0-SNAPSHOT org.apache.flume.flume-ng-channels diff --git a/flume-ng-channels/flume-jdbc-channel/pom.xml b/flume-ng-channels/flume-jdbc-channel/pom.xml index 561537704a..2483c3fbab 100644 --- a/flume-ng-channels/flume-jdbc-channel/pom.xml +++ b/flume-ng-channels/flume-jdbc-channel/pom.xml @@ -23,7 +23,7 @@ limitations under the License. flume-ng-channels org.apache.flume - 1.5.0-SNAPSHOT + 1.6.0-SNAPSHOT org.apache.flume.flume-ng-channels diff --git a/flume-ng-channels/flume-spillable-memory-channel/pom.xml b/flume-ng-channels/flume-spillable-memory-channel/pom.xml index b4646e8992..c9b5cd5e24 100644 --- a/flume-ng-channels/flume-spillable-memory-channel/pom.xml +++ b/flume-ng-channels/flume-spillable-memory-channel/pom.xml @@ -24,7 +24,7 @@ flume-ng-channels org.apache.flume - 1.5.0-SNAPSHOT + 1.6.0-SNAPSHOT org.apache.flume.flume-ng-channels diff --git a/flume-ng-channels/pom.xml b/flume-ng-channels/pom.xml index 0da3be8be7..dc8dbc69a7 100644 --- a/flume-ng-channels/pom.xml +++ b/flume-ng-channels/pom.xml @@ -23,7 +23,7 @@ limitations under the License. flume-parent org.apache.flume - 1.5.0-SNAPSHOT + 1.6.0-SNAPSHOT org.apache.flume diff --git a/flume-ng-clients/flume-ng-log4jappender/pom.xml b/flume-ng-clients/flume-ng-log4jappender/pom.xml index ef3e965ee1..7208e97108 100644 --- a/flume-ng-clients/flume-ng-log4jappender/pom.xml +++ b/flume-ng-clients/flume-ng-log4jappender/pom.xml @@ -21,7 +21,7 @@ limitations under the License. flume-ng-clients org.apache.flume - 1.5.0-SNAPSHOT + 1.6.0-SNAPSHOT .. diff --git a/flume-ng-clients/pom.xml b/flume-ng-clients/pom.xml index b0f52d1ee5..a033c87e73 100644 --- a/flume-ng-clients/pom.xml +++ b/flume-ng-clients/pom.xml @@ -20,7 +20,7 @@ limitations under the License. flume-parent org.apache.flume - 1.5.0-SNAPSHOT + 1.6.0-SNAPSHOT .. flume-ng-clients diff --git a/flume-ng-configuration/pom.xml b/flume-ng-configuration/pom.xml index e7a5350290..deacf14f86 100644 --- a/flume-ng-configuration/pom.xml +++ b/flume-ng-configuration/pom.xml @@ -20,7 +20,7 @@ limitations under the License. flume-parent org.apache.flume - 1.5.0-SNAPSHOT + 1.6.0-SNAPSHOT .. flume-ng-configuration diff --git a/flume-ng-core/pom.xml b/flume-ng-core/pom.xml index e41bcc21f7..89924142cf 100644 --- a/flume-ng-core/pom.xml +++ b/flume-ng-core/pom.xml @@ -22,7 +22,7 @@ limitations under the License. flume-parent org.apache.flume - 1.5.0-SNAPSHOT + 1.6.0-SNAPSHOT org.apache.flume diff --git a/flume-ng-dist/pom.xml b/flume-ng-dist/pom.xml index 338e108813..7821dc05c0 100644 --- a/flume-ng-dist/pom.xml +++ b/flume-ng-dist/pom.xml @@ -24,7 +24,7 @@ flume-parent org.apache.flume - 1.5.0-SNAPSHOT + 1.6.0-SNAPSHOT flume-ng-dist diff --git a/flume-ng-doc/sphinx/FlumeDeveloperGuide.rst b/flume-ng-doc/sphinx/FlumeDeveloperGuide.rst index ee7b89b43d..ec6a735bc4 100644 --- a/flume-ng-doc/sphinx/FlumeDeveloperGuide.rst +++ b/flume-ng-doc/sphinx/FlumeDeveloperGuide.rst @@ -15,7 +15,7 @@ ====================================== -Flume 1.5.0-SNAPSHOT Developer Guide +Flume 1.6.0-SNAPSHOT Developer Guide ====================================== Introduction diff --git a/flume-ng-doc/sphinx/FlumeUserGuide.rst b/flume-ng-doc/sphinx/FlumeUserGuide.rst index b24f8af308..7732c131e2 100644 --- a/flume-ng-doc/sphinx/FlumeUserGuide.rst +++ b/flume-ng-doc/sphinx/FlumeUserGuide.rst @@ -15,7 +15,7 @@ ====================================== -Flume 1.5.0-SNAPSHOT User Guide +Flume 1.6.0-SNAPSHOT User Guide ====================================== Introduction @@ -2943,7 +2943,7 @@ Log4J Appender Appends Log4j events to a flume agent's avro source. A client using this appender must have the flume-ng-sdk in the classpath (eg, -flume-ng-sdk-1.5.0-SNAPSHOT.jar). +flume-ng-sdk-1.6.0-SNAPSHOT.jar). Required properties are in **bold**. ===================== ======= ================================================================================== @@ -3007,7 +3007,7 @@ Load Balancing Log4J Appender Appends Log4j events to a list of flume agent's avro source. A client using this appender must have the flume-ng-sdk in the classpath (eg, -flume-ng-sdk-1.5.0-SNAPSHOT.jar). This appender supports a round-robin and random +flume-ng-sdk-1.6.0-SNAPSHOT.jar). This appender supports a round-robin and random scheme for performing the load balancing. It also supports a configurable backoff timeout so that down agents are removed temporarily from the set of hosts Required properties are in **bold**. diff --git a/flume-ng-doc/sphinx/index.rst b/flume-ng-doc/sphinx/index.rst index af9b64e65d..39cf13f0aa 100644 --- a/flume-ng-doc/sphinx/index.rst +++ b/flume-ng-doc/sphinx/index.rst @@ -15,7 +15,7 @@ ================================================ -Apache Flume 1.5.0-SNAPSHOT documentation +Apache Flume 1.6.0-SNAPSHOT documentation ================================================ Flume is a distributed, reliable, and available system for efficiently diff --git a/flume-ng-embedded-agent/pom.xml b/flume-ng-embedded-agent/pom.xml index a8d38d28d6..347120d861 100644 --- a/flume-ng-embedded-agent/pom.xml +++ b/flume-ng-embedded-agent/pom.xml @@ -21,7 +21,7 @@ limitations under the License. flume-parent org.apache.flume - 1.5.0-SNAPSHOT + 1.6.0-SNAPSHOT flume-ng-embedded-agent diff --git a/flume-ng-legacy-sources/flume-avro-source/pom.xml b/flume-ng-legacy-sources/flume-avro-source/pom.xml index e372ebd030..c854fd3b12 100644 --- a/flume-ng-legacy-sources/flume-avro-source/pom.xml +++ b/flume-ng-legacy-sources/flume-avro-source/pom.xml @@ -23,7 +23,7 @@ limitations under the License. flume-ng-legacy-sources org.apache.flume - 1.5.0-SNAPSHOT + 1.6.0-SNAPSHOT org.apache.flume.flume-ng-legacy-sources diff --git a/flume-ng-legacy-sources/flume-thrift-source/pom.xml b/flume-ng-legacy-sources/flume-thrift-source/pom.xml index c32247e90f..5c6ec0fd90 100644 --- a/flume-ng-legacy-sources/flume-thrift-source/pom.xml +++ b/flume-ng-legacy-sources/flume-thrift-source/pom.xml @@ -23,7 +23,7 @@ limitations under the License. flume-ng-legacy-sources org.apache.flume - 1.5.0-SNAPSHOT + 1.6.0-SNAPSHOT org.apache.flume.flume-ng-legacy-sources diff --git a/flume-ng-legacy-sources/pom.xml b/flume-ng-legacy-sources/pom.xml index 0d74022aaf..eb3c6d60ed 100644 --- a/flume-ng-legacy-sources/pom.xml +++ b/flume-ng-legacy-sources/pom.xml @@ -23,7 +23,7 @@ limitations under the License. flume-parent org.apache.flume - 1.5.0-SNAPSHOT + 1.6.0-SNAPSHOT org.apache.flume diff --git a/flume-ng-node/pom.xml b/flume-ng-node/pom.xml index 7e9c9755f5..dce2527689 100644 --- a/flume-ng-node/pom.xml +++ b/flume-ng-node/pom.xml @@ -25,7 +25,7 @@ flume-parent org.apache.flume - 1.5.0-SNAPSHOT + 1.6.0-SNAPSHOT flume-ng-node diff --git a/flume-ng-sdk/pom.xml b/flume-ng-sdk/pom.xml index d5d50cfaca..113b286ac4 100644 --- a/flume-ng-sdk/pom.xml +++ b/flume-ng-sdk/pom.xml @@ -21,7 +21,7 @@ limitations under the License. flume-parent org.apache.flume - 1.5.0-SNAPSHOT + 1.6.0-SNAPSHOT flume-ng-sdk diff --git a/flume-ng-sinks/flume-dataset-sink/pom.xml b/flume-ng-sinks/flume-dataset-sink/pom.xml index 57fd0e4a67..f6ae1f32f6 100644 --- a/flume-ng-sinks/flume-dataset-sink/pom.xml +++ b/flume-ng-sinks/flume-dataset-sink/pom.xml @@ -23,7 +23,7 @@ limitations under the License. flume-ng-sinks org.apache.flume - 1.5.0-SNAPSHOT + 1.6.0-SNAPSHOT org.apache.flume.flume-ng-sinks diff --git a/flume-ng-sinks/flume-hdfs-sink/pom.xml b/flume-ng-sinks/flume-hdfs-sink/pom.xml index 43b1a0f025..e0760ae693 100644 --- a/flume-ng-sinks/flume-hdfs-sink/pom.xml +++ b/flume-ng-sinks/flume-hdfs-sink/pom.xml @@ -23,7 +23,7 @@ limitations under the License. flume-ng-sinks org.apache.flume - 1.5.0-SNAPSHOT + 1.6.0-SNAPSHOT org.apache.flume.flume-ng-sinks diff --git a/flume-ng-sinks/flume-irc-sink/pom.xml b/flume-ng-sinks/flume-irc-sink/pom.xml index 1e157adc97..85dffba9e4 100644 --- a/flume-ng-sinks/flume-irc-sink/pom.xml +++ b/flume-ng-sinks/flume-irc-sink/pom.xml @@ -23,7 +23,7 @@ limitations under the License. flume-ng-sinks org.apache.flume - 1.5.0-SNAPSHOT + 1.6.0-SNAPSHOT org.apache.flume.flume-ng-sinks diff --git a/flume-ng-sinks/flume-ng-elasticsearch-sink/pom.xml b/flume-ng-sinks/flume-ng-elasticsearch-sink/pom.xml index dedb738e45..0284686897 100644 --- a/flume-ng-sinks/flume-ng-elasticsearch-sink/pom.xml +++ b/flume-ng-sinks/flume-ng-elasticsearch-sink/pom.xml @@ -17,7 +17,7 @@ flume-ng-sinks org.apache.flume - 1.5.0-SNAPSHOT + 1.6.0-SNAPSHOT org.apache.flume.flume-ng-sinks diff --git a/flume-ng-sinks/flume-ng-hbase-sink/pom.xml b/flume-ng-sinks/flume-ng-hbase-sink/pom.xml index 90b69912da..ddb11631ba 100644 --- a/flume-ng-sinks/flume-ng-hbase-sink/pom.xml +++ b/flume-ng-sinks/flume-ng-hbase-sink/pom.xml @@ -15,7 +15,7 @@ flume-ng-sinks org.apache.flume - 1.5.0-SNAPSHOT + 1.6.0-SNAPSHOT org.apache.flume.flume-ng-sinks flume-ng-hbase-sink diff --git a/flume-ng-sinks/flume-ng-morphline-solr-sink/pom.xml b/flume-ng-sinks/flume-ng-morphline-solr-sink/pom.xml index e4be72099d..01e9f454e9 100644 --- a/flume-ng-sinks/flume-ng-morphline-solr-sink/pom.xml +++ b/flume-ng-sinks/flume-ng-morphline-solr-sink/pom.xml @@ -21,12 +21,12 @@ limitations under the License. flume-ng-sinks org.apache.flume - 1.5.0-SNAPSHOT + 1.6.0-SNAPSHOT org.apache.flume.flume-ng-sinks flume-ng-morphline-solr-sink - 1.5.0-SNAPSHOT + 1.6.0-SNAPSHOT Flume NG Morphline Solr Sink diff --git a/flume-ng-sinks/pom.xml b/flume-ng-sinks/pom.xml index 6ac2b4d0be..c919605fc3 100644 --- a/flume-ng-sinks/pom.xml +++ b/flume-ng-sinks/pom.xml @@ -23,7 +23,7 @@ limitations under the License. flume-parent org.apache.flume - 1.5.0-SNAPSHOT + 1.6.0-SNAPSHOT org.apache.flume diff --git a/flume-ng-sources/flume-jms-source/pom.xml b/flume-ng-sources/flume-jms-source/pom.xml index 6aa267d1bb..b99d1c45b2 100644 --- a/flume-ng-sources/flume-jms-source/pom.xml +++ b/flume-ng-sources/flume-jms-source/pom.xml @@ -23,7 +23,7 @@ limitations under the License. flume-ng-sources org.apache.flume - 1.5.0-SNAPSHOT + 1.6.0-SNAPSHOT org.apache.flume.flume-ng-sources diff --git a/flume-ng-sources/flume-scribe-source/pom.xml b/flume-ng-sources/flume-scribe-source/pom.xml index 1ba2e7fa00..08da29166e 100644 --- a/flume-ng-sources/flume-scribe-source/pom.xml +++ b/flume-ng-sources/flume-scribe-source/pom.xml @@ -23,7 +23,7 @@ limitations under the License. flume-ng-sources org.apache.flume - 1.5.0-SNAPSHOT + 1.6.0-SNAPSHOT org.apache.flume.flume-ng-sources diff --git a/flume-ng-sources/flume-twitter-source/pom.xml b/flume-ng-sources/flume-twitter-source/pom.xml index a5a27cfec2..f912db3afa 100644 --- a/flume-ng-sources/flume-twitter-source/pom.xml +++ b/flume-ng-sources/flume-twitter-source/pom.xml @@ -23,7 +23,7 @@ limitations under the License. flume-ng-sources org.apache.flume - 1.5.0-SNAPSHOT + 1.6.0-SNAPSHOT org.apache.flume.flume-ng-sources diff --git a/flume-ng-sources/pom.xml b/flume-ng-sources/pom.xml index 0b57d6d44f..c03307a3d7 100644 --- a/flume-ng-sources/pom.xml +++ b/flume-ng-sources/pom.xml @@ -23,7 +23,7 @@ limitations under the License. flume-parent org.apache.flume - 1.5.0-SNAPSHOT + 1.6.0-SNAPSHOT org.apache.flume diff --git a/flume-ng-tests/pom.xml b/flume-ng-tests/pom.xml index bd69a4e2ac..3f51a537ac 100644 --- a/flume-ng-tests/pom.xml +++ b/flume-ng-tests/pom.xml @@ -24,7 +24,7 @@ flume-parent org.apache.flume - 1.5.0-SNAPSHOT + 1.6.0-SNAPSHOT flume-ng-tests diff --git a/flume-tools/pom.xml b/flume-tools/pom.xml index 1f265a3f43..b76186ec40 100644 --- a/flume-tools/pom.xml +++ b/flume-tools/pom.xml @@ -25,7 +25,7 @@ limitations under the License. flume-parent org.apache.flume - 1.5.0-SNAPSHOT + 1.6.0-SNAPSHOT org.apache.flume diff --git a/pom.xml b/pom.xml index 7abf833be9..6eb709d1e3 100644 --- a/pom.xml +++ b/pom.xml @@ -27,7 +27,7 @@ limitations under the License. 4.0.0 org.apache.flume flume-parent - 1.5.0-SNAPSHOT + 1.6.0-SNAPSHOT pom Apache Flume @@ -152,7 +152,7 @@ limitations under the License. org.apache.flume.flume-ng-sinks flume-dataset-sink - 1.5.0-SNAPSHOT + 1.6.0-SNAPSHOT @@ -910,120 +910,120 @@ limitations under the License. org.apache.flume flume-ng-configuration - 1.5.0-SNAPSHOT + 1.6.0-SNAPSHOT org.apache.flume flume-ng-core - 1.5.0-SNAPSHOT + 1.6.0-SNAPSHOT org.apache.flume flume-tools - 1.5.0-SNAPSHOT + 1.6.0-SNAPSHOT org.apache.flume flume-ng-node - 1.5.0-SNAPSHOT + 1.6.0-SNAPSHOT org.apache.flume.flume-ng-channels flume-file-channel - 1.5.0-SNAPSHOT + 1.6.0-SNAPSHOT org.apache.flume.flume-ng-channels flume-jdbc-channel - 1.5.0-SNAPSHOT + 1.6.0-SNAPSHOT org.apache.flume.flume-ng-channels flume-spillable-memory-channel - 1.5.0-SNAPSHOT + 1.6.0-SNAPSHOT org.apache.flume.flume-ng-channels flume-recoverable-memory-channel - 1.5.0-SNAPSHOT + 1.6.0-SNAPSHOT org.apache.flume.flume-ng-sinks flume-hdfs-sink - 1.5.0-SNAPSHOT + 1.6.0-SNAPSHOT org.apache.flume.flume-ng-sinks flume-irc-sink - 1.5.0-SNAPSHOT + 1.6.0-SNAPSHOT org.apache.flume.flume-ng-sinks flume-ng-hbase-sink - 1.5.0-SNAPSHOT + 1.6.0-SNAPSHOT org.apache.flume.flume-ng-sinks flume-ng-elasticsearch-sink - 1.5.0-SNAPSHOT + 1.6.0-SNAPSHOT org.apache.flume.flume-ng-sinks flume-ng-morphline-solr-sink - 1.5.0-SNAPSHOT + 1.6.0-SNAPSHOT org.apache.flume.flume-ng-sources flume-scribe-source - 1.5.0-SNAPSHOT + 1.6.0-SNAPSHOT org.apache.flume.flume-ng-sources flume-jms-source - 1.5.0-SNAPSHOT + 1.6.0-SNAPSHOT org.apache.flume.flume-ng-sources flume-twitter-source - 1.5.0-SNAPSHOT + 1.6.0-SNAPSHOT org.apache.flume.flume-ng-legacy-sources flume-thrift-source - 1.5.0-SNAPSHOT + 1.6.0-SNAPSHOT org.apache.flume.flume-ng-legacy-sources flume-avro-source - 1.5.0-SNAPSHOT + 1.6.0-SNAPSHOT org.apache.flume flume-ng-sdk - 1.5.0-SNAPSHOT + 1.6.0-SNAPSHOT org.apache.flume flume-ng-sdk - 1.5.0-SNAPSHOT + 1.6.0-SNAPSHOT tests test @@ -1031,20 +1031,20 @@ limitations under the License. org.apache.flume.flume-ng-clients flume-ng-log4jappender - 1.5.0-SNAPSHOT + 1.6.0-SNAPSHOT org.apache.flume.flume-ng-clients flume-ng-log4jappender - 1.5.0-SNAPSHOT + 1.6.0-SNAPSHOT jar-with-dependencies org.apache.flume flume-ng-embedded-agent - 1.5.0-SNAPSHOT + 1.6.0-SNAPSHOT From 73fd7bddeff11ef7af91055db1148416dca9d0b1 Mon Sep 17 00:00:00 2001 From: Hari Shreedharan Date: Tue, 6 May 2014 17:19:22 -0700 Subject: [PATCH 105/341] Adding default version for thrift --- pom.xml | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/pom.xml b/pom.xml index 6eb709d1e3..d4b76607fe 100644 --- a/pom.xml +++ b/pom.xml @@ -50,7 +50,7 @@ limitations under the License. 0.90.1 2.4.0 - + 0.7.0 0.12.0 From fd448caa8144ad3ea54a836d4ff6e944f36cfac6 Mon Sep 17 00:00:00 2001 From: Hari Shreedharan Date: Tue, 6 May 2014 17:23:12 -0700 Subject: [PATCH 106/341] FLUME-2368. Update CHANGELOG. --- CHANGELOG | 1 + 1 file changed, 1 insertion(+) diff --git a/CHANGELOG b/CHANGELOG index 4ce391e38d..47894db147 100644 --- a/CHANGELOG +++ b/CHANGELOG @@ -120,6 +120,7 @@ Release Notes - Flume - Version v1.5.0 * [FLUME-2350] - Consume Order tests need to space out file creation * [FLUME-2357] - HDFS sink should retry closing files that previously had close errors * [FLUME-2381] - Upgrade Hadoop version in Hadoop 2 profile to 2.4.0 + * [FLUME-2379] - Flume's pom files are invalid and not parseable by non-maven build tools ** Documentation * [FLUME-1223] - Userguide improvement rolling file sink From f99113042dad2a67d24068e30ed37e4b18e62a72 Mon Sep 17 00:00:00 2001 From: Hari Shreedharan Date: Tue, 6 May 2014 17:29:27 -0700 Subject: [PATCH 107/341] FLUME-2366. Update RELEASE NOTES for Flume 1.5 --- RELEASE-NOTES | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/RELEASE-NOTES b/RELEASE-NOTES index 263ba43b04..31f6f61467 100644 --- a/RELEASE-NOTES +++ b/RELEASE-NOTES @@ -17,8 +17,8 @@ simple, extensible data model that allows for online analytic application. 2. Status of this release -Apache Flume 1.4.0 is the fourth release of Flume as an Apache top-level project -(TLP). Apache Flume 1.4.0 is production-ready software. +Apache Flume 1.5.0 is the fifth release of Flume as an Apache top-level project +(TLP). Apache Flume 1.5.0 is production-ready software. 3. Major changes in this Release For a detailed list of changes, please see the CHANGELOG file included From 33cdcf0d4e85e68e6df9e1ca4be729889d480246 Mon Sep 17 00:00:00 2001 From: Hari Shreedharan Date: Wed, 14 May 2014 17:42:09 -0700 Subject: [PATCH 108/341] FLUME-2245. Pre-close flush failure can cause HDFS Sinks to not process events. (Juhani Connolly, Brock Noland via Hari Shreedharan) --- .../main/java/org/apache/flume/sink/hdfs/BucketWriter.java | 6 +++++- 1 file changed, 5 insertions(+), 1 deletion(-) diff --git a/flume-ng-sinks/flume-hdfs-sink/src/main/java/org/apache/flume/sink/hdfs/BucketWriter.java b/flume-ng-sinks/flume-hdfs-sink/src/main/java/org/apache/flume/sink/hdfs/BucketWriter.java index fba3f66468..f9e39ac15b 100644 --- a/flume-ng-sinks/flume-hdfs-sink/src/main/java/org/apache/flume/sink/hdfs/BucketWriter.java +++ b/flume-ng-sinks/flume-hdfs-sink/src/main/java/org/apache/flume/sink/hdfs/BucketWriter.java @@ -404,7 +404,11 @@ public Void call() throws Exception { public synchronized void close(boolean callCloseCallback) throws IOException, InterruptedException { checkAndThrowInterruptedException(); - flush(); + try { + flush(); + } catch (IOException e) { + LOG.warn("pre-close flush failed", e); + } boolean failedToClose = false; LOG.info("Closing {}", bucketPath); CallRunner closeCallRunner = createCloseCallRunner(); From 09472ba12278a0d3696b9d2e26d6d1b0d361c830 Mon Sep 17 00:00:00 2001 From: Juhani Connolly Date: Tue, 3 Jun 2014 11:33:54 +0900 Subject: [PATCH 109/341] FLUME-2273 - Add handling for header substitution in ElasticSearchSink Satoshi Iijima via Juhani Connolly --- flume-ng-doc/sphinx/FlumeUserGuide.rst | 6 ++++++ ...asticSearchIndexRequestBuilderFactory.java | 9 ++++++-- .../sink/elasticsearch/ElasticSearchSink.java | 6 +++--- .../elasticsearch/SimpleIndexNameBuilder.java | 5 +++-- .../TimeBasedIndexNameBuilder.java | 6 ++++-- .../client/ElasticSearchRestClient.java | 2 -- .../client/ElasticSearchTransportClient.java | 1 - ...asticSearchIndexRequestBuilderFactory.java | 21 +++++++++++++++++++ 8 files changed, 44 insertions(+), 12 deletions(-) diff --git a/flume-ng-doc/sphinx/FlumeUserGuide.rst b/flume-ng-doc/sphinx/FlumeUserGuide.rst index 7732c131e2..040fc8b7d6 100644 --- a/flume-ng-doc/sphinx/FlumeUserGuide.rst +++ b/flume-ng-doc/sphinx/FlumeUserGuide.rst @@ -2008,7 +2008,9 @@ Property Name Default **type** -- The component type name, needs to be ``org.apache.flume.sink.elasticsearch.ElasticSearchSink`` **hostNames** -- Comma separated list of hostname:port, if the port is not present the default port '9300' will be used indexName flume The name of the index which the date will be appended to. Example 'flume' -> 'flume-yyyy-MM-dd' + Arbitrary header substitution is supported, eg. %{header} replaces with value of named event header indexType logs The type to index the document to, defaults to 'log' + Arbitrary header substitution is supported, eg. %{header} replaces with value of named event header clusterName elasticsearch Name of the ElasticSearch cluster to connect to batchSize 100 Number of events to be written per txn. ttl -- TTL in days, when set will cause the expired documents to be deleted automatically, @@ -2021,6 +2023,10 @@ serializer org.apache.flume.sink.elasticsearch.ElasticSearchLogStashEvent serializer.* -- Properties to be passed to the serializer. ================ ======================================================================== ======================================================================================================= +.. note:: Header substitution is a handy to use the value of an event header to dynamically decide the indexName and indexType to use when storing the event. + Caution should be used in using this feature as the event submitter now has control of the indexName and indexType. + Furthermore, if the elasticsearch REST client is used then the event submitter has control of the URL path used. + Example for agent named a1: .. code-block:: properties diff --git a/flume-ng-sinks/flume-ng-elasticsearch-sink/src/main/java/org/apache/flume/sink/elasticsearch/AbstractElasticSearchIndexRequestBuilderFactory.java b/flume-ng-sinks/flume-ng-elasticsearch-sink/src/main/java/org/apache/flume/sink/elasticsearch/AbstractElasticSearchIndexRequestBuilderFactory.java index de84b95433..99961423ee 100644 --- a/flume-ng-sinks/flume-ng-elasticsearch-sink/src/main/java/org/apache/flume/sink/elasticsearch/AbstractElasticSearchIndexRequestBuilderFactory.java +++ b/flume-ng-sinks/flume-ng-elasticsearch-sink/src/main/java/org/apache/flume/sink/elasticsearch/AbstractElasticSearchIndexRequestBuilderFactory.java @@ -26,6 +26,7 @@ import org.apache.flume.conf.ComponentConfiguration; import org.apache.flume.conf.Configurable; import org.apache.flume.conf.ConfigurableComponent; +import org.apache.flume.formatter.output.BucketPath; import org.elasticsearch.action.index.IndexRequestBuilder; import org.elasticsearch.client.Client; @@ -75,10 +76,14 @@ protected AbstractElasticSearchIndexRequestBuilderFactory( public IndexRequestBuilder createIndexRequest(Client client, String indexPrefix, String indexType, Event event) throws IOException { IndexRequestBuilder request = prepareIndex(client); + String realIndexPrefix = BucketPath.escapeString(indexPrefix, event.getHeaders()); + String realIndexType = BucketPath.escapeString(indexType, event.getHeaders()); + TimestampedEvent timestampedEvent = new TimestampedEvent(event); long timestamp = timestampedEvent.getTimestamp(); - String indexName = getIndexName(indexPrefix, timestamp); - prepareIndexRequest(request, indexName, indexType, timestampedEvent); + + String indexName = getIndexName(realIndexPrefix, timestamp); + prepareIndexRequest(request, indexName, realIndexType, timestampedEvent); return request; } diff --git a/flume-ng-sinks/flume-ng-elasticsearch-sink/src/main/java/org/apache/flume/sink/elasticsearch/ElasticSearchSink.java b/flume-ng-sinks/flume-ng-elasticsearch-sink/src/main/java/org/apache/flume/sink/elasticsearch/ElasticSearchSink.java index 39b6db573e..1d9dfce765 100644 --- a/flume-ng-sinks/flume-ng-elasticsearch-sink/src/main/java/org/apache/flume/sink/elasticsearch/ElasticSearchSink.java +++ b/flume-ng-sinks/flume-ng-elasticsearch-sink/src/main/java/org/apache/flume/sink/elasticsearch/ElasticSearchSink.java @@ -38,6 +38,7 @@ import org.apache.flume.Event; import org.apache.flume.EventDeliveryException; import org.apache.flume.Transaction; +import org.apache.flume.formatter.output.BucketPath; import org.apache.flume.conf.Configurable; import org.apache.flume.instrumentation.SinkCounter; import org.apache.flume.sink.AbstractSink; @@ -61,8 +62,6 @@ import static org.apache.flume.sink.elasticsearch.ElasticSearchSinkConstants.INDEX_NAME_BUILDER; import static org.apache.flume.sink.elasticsearch.ElasticSearchSinkConstants.INDEX_NAME_BUILDER_PREFIX; -import static org.elasticsearch.common.xcontent.XContentFactory.jsonBuilder; - /** * A sink which reads events from a channel and writes them to ElasticSearch * based on the work done by https://github.com/Aconex/elasticflume.git.

    @@ -186,7 +185,8 @@ public Status process() throws EventDeliveryException { if (event == null) { break; } - client.addEvent(event, indexNameBuilder, indexType, ttlMs); + String realIndexType = BucketPath.escapeString(indexType, event.getHeaders()); + client.addEvent(event, indexNameBuilder, realIndexType, ttlMs); } if (count <= 0) { diff --git a/flume-ng-sinks/flume-ng-elasticsearch-sink/src/main/java/org/apache/flume/sink/elasticsearch/SimpleIndexNameBuilder.java b/flume-ng-sinks/flume-ng-elasticsearch-sink/src/main/java/org/apache/flume/sink/elasticsearch/SimpleIndexNameBuilder.java index 19079af32e..801cac9ed1 100644 --- a/flume-ng-sinks/flume-ng-elasticsearch-sink/src/main/java/org/apache/flume/sink/elasticsearch/SimpleIndexNameBuilder.java +++ b/flume-ng-sinks/flume-ng-elasticsearch-sink/src/main/java/org/apache/flume/sink/elasticsearch/SimpleIndexNameBuilder.java @@ -19,6 +19,7 @@ import org.apache.flume.Context; import org.apache.flume.Event; import org.apache.flume.conf.ComponentConfiguration; +import org.apache.flume.formatter.output.BucketPath; public class SimpleIndexNameBuilder implements IndexNameBuilder { @@ -26,12 +27,12 @@ public class SimpleIndexNameBuilder implements IndexNameBuilder { @Override public String getIndexName(Event event) { - return indexName; + return BucketPath.escapeString(indexName, event.getHeaders()); } @Override public String getIndexPrefix(Event event) { - return indexName; + return BucketPath.escapeString(indexName, event.getHeaders()); } @Override diff --git a/flume-ng-sinks/flume-ng-elasticsearch-sink/src/main/java/org/apache/flume/sink/elasticsearch/TimeBasedIndexNameBuilder.java b/flume-ng-sinks/flume-ng-elasticsearch-sink/src/main/java/org/apache/flume/sink/elasticsearch/TimeBasedIndexNameBuilder.java index a8603a4ddb..c651732b4a 100644 --- a/flume-ng-sinks/flume-ng-elasticsearch-sink/src/main/java/org/apache/flume/sink/elasticsearch/TimeBasedIndexNameBuilder.java +++ b/flume-ng-sinks/flume-ng-elasticsearch-sink/src/main/java/org/apache/flume/sink/elasticsearch/TimeBasedIndexNameBuilder.java @@ -24,6 +24,7 @@ import org.apache.flume.Context; import org.apache.flume.Event; import org.apache.flume.conf.ComponentConfiguration; +import org.apache.flume.formatter.output.BucketPath; import java.util.TimeZone; @@ -60,13 +61,14 @@ FastDateFormat getFastDateFormat() { public String getIndexName(Event event) { TimestampedEvent timestampedEvent = new TimestampedEvent(event); long timestamp = timestampedEvent.getTimestamp(); - return new StringBuilder(indexPrefix).append('-') + String realIndexPrefix = BucketPath.escapeString(indexPrefix, event.getHeaders()); + return new StringBuilder(realIndexPrefix).append('-') .append(fastDateFormat.format(timestamp)).toString(); } @Override public String getIndexPrefix(Event event) { - return indexPrefix; + return BucketPath.escapeString(indexPrefix, event.getHeaders()); } @Override diff --git a/flume-ng-sinks/flume-ng-elasticsearch-sink/src/main/java/org/apache/flume/sink/elasticsearch/client/ElasticSearchRestClient.java b/flume-ng-sinks/flume-ng-elasticsearch-sink/src/main/java/org/apache/flume/sink/elasticsearch/client/ElasticSearchRestClient.java index ff95e306c0..0d1c37f9ea 100644 --- a/flume-ng-sinks/flume-ng-elasticsearch-sink/src/main/java/org/apache/flume/sink/elasticsearch/client/ElasticSearchRestClient.java +++ b/flume-ng-sinks/flume-ng-elasticsearch-sink/src/main/java/org/apache/flume/sink/elasticsearch/client/ElasticSearchRestClient.java @@ -116,8 +116,6 @@ public void addEvent(Event event, IndexNameBuilder indexNameBuilder, String inde public void execute() throws Exception { int statusCode = 0, triesCount = 0; HttpResponse response = null; - logger.info("Sending bulk request to elasticsearch cluster"); - String entity; synchronized (bulkBuilder) { entity = bulkBuilder.toString(); diff --git a/flume-ng-sinks/flume-ng-elasticsearch-sink/src/main/java/org/apache/flume/sink/elasticsearch/client/ElasticSearchTransportClient.java b/flume-ng-sinks/flume-ng-elasticsearch-sink/src/main/java/org/apache/flume/sink/elasticsearch/client/ElasticSearchTransportClient.java index e9ed0b4955..d44c8ad9cc 100644 --- a/flume-ng-sinks/flume-ng-elasticsearch-sink/src/main/java/org/apache/flume/sink/elasticsearch/client/ElasticSearchTransportClient.java +++ b/flume-ng-sinks/flume-ng-elasticsearch-sink/src/main/java/org/apache/flume/sink/elasticsearch/client/ElasticSearchTransportClient.java @@ -178,7 +178,6 @@ public void addEvent(Event event, IndexNameBuilder indexNameBuilder, @Override public void execute() throws Exception { try { - logger.info("Sending bulk to elasticsearch cluster"); BulkResponse bulkResponse = bulkRequestBuilder.execute().actionGet(); if (bulkResponse.hasFailures()) { throw new EventDeliveryException(bulkResponse.buildFailureMessage()); diff --git a/flume-ng-sinks/flume-ng-elasticsearch-sink/src/test/java/org/apache/flume/sink/elasticsearch/TestElasticSearchIndexRequestBuilderFactory.java b/flume-ng-sinks/flume-ng-elasticsearch-sink/src/test/java/org/apache/flume/sink/elasticsearch/TestElasticSearchIndexRequestBuilderFactory.java index 807a9c7513..8022111312 100644 --- a/flume-ng-sinks/flume-ng-elasticsearch-sink/src/test/java/org/apache/flume/sink/elasticsearch/TestElasticSearchIndexRequestBuilderFactory.java +++ b/flume-ng-sinks/flume-ng-elasticsearch-sink/src/test/java/org/apache/flume/sink/elasticsearch/TestElasticSearchIndexRequestBuilderFactory.java @@ -157,6 +157,27 @@ public void shouldSetIndexNameFromTimestampHeaderWhenPresent() indexRequestBuilder.request().index()); } + @Test + public void shouldSetIndexNameTypeFromHeaderWhenPresent() + throws Exception { + String indexPrefix = "%{index-name}"; + String indexType = "%{index-type}"; + String indexValue = "testing-index-name-from-headers"; + String typeValue = "testing-index-type-from-headers"; + + Event event = new SimpleEvent(); + event.getHeaders().put("index-name", indexValue); + event.getHeaders().put("index-type", typeValue); + + IndexRequestBuilder indexRequestBuilder = factory.createIndexRequest( + null, indexPrefix, indexType, event); + + assertEquals(indexValue + '-' + + ElasticSearchIndexRequestBuilderFactory.df.format(FIXED_TIME_MILLIS), + indexRequestBuilder.request().index()); + assertEquals(typeValue, indexRequestBuilder.request().type()); + } + @Test public void shouldConfigureEventSerializer() throws Exception { assertFalse(serializer.configuredWithContext); From 0cba73698dbba6b78d0a2cd7b469f4377723470a Mon Sep 17 00:00:00 2001 From: Jarek Jarcec Cecho Date: Thu, 5 Jun 2014 09:28:02 -0700 Subject: [PATCH 110/341] FLUME-2397: HBase-98 compatibility (Hari Shreedharan via Jarek Jarcec Cecho) --- flume-ng-sinks/flume-hdfs-sink/pom.xml | 30 +++ flume-ng-sinks/flume-ng-hbase-sink/pom.xml | 114 +++++++++-- .../flume/sink/hbase/AsyncHBaseSink.java | 28 ++- .../flume/sink/hbase/TestAsyncHBaseSink.java | 1 + pom.xml | 182 ++++++++++++++++-- 5 files changed, 322 insertions(+), 33 deletions(-) diff --git a/flume-ng-sinks/flume-hdfs-sink/pom.xml b/flume-ng-sinks/flume-hdfs-sink/pom.xml index e0760ae693..83f8bec62e 100644 --- a/flume-ng-sinks/flume-hdfs-sink/pom.xml +++ b/flume-ng-sinks/flume-hdfs-sink/pom.xml @@ -161,6 +161,36 @@ limitations under the License. + + hbase-98 + + + hadoop.profile + hbase-98 + + + + + + org.apache.hadoop + hadoop-hdfs + true + + + + org.apache.hadoop + hadoop-auth + true + + + + org.apache.hadoop + hadoop-minicluster + test + + + + diff --git a/flume-ng-sinks/flume-ng-hbase-sink/pom.xml b/flume-ng-sinks/flume-ng-hbase-sink/pom.xml index ddb11631ba..cc2bbee50a 100644 --- a/flume-ng-sinks/flume-ng-hbase-sink/pom.xml +++ b/flume-ng-sinks/flume-ng-hbase-sink/pom.xml @@ -56,18 +56,6 @@ guava - - org.apache.hbase - hbase - true - - - - org.apache.hbase - hbase - tests - test - org.hbase @@ -142,6 +130,24 @@ jersey-core test + + org.apache.hbase + hbase + true + + + + org.apache.hbase + hbase + tests + test + + + + org.apache.zookeeper + zookeeper + test + @@ -158,6 +164,90 @@ hadoop-minicluster test + + org.apache.hbase + hbase + true + + + + org.apache.hbase + hbase + tests + test + + + + org.apache.zookeeper + zookeeper + test + + + + + hbase-98 + + + hadoop.profile + hbase-98 + + + + + org.apache.hadoop + hadoop-minicluster + test + + + + org.apache.hbase + hbase-client + true + + + + org.apache.hbase + hbase-client + tests + test + + + + + org.apache.hbase + hbase-server + test + + + + org.apache.hbase + hbase-server + tests + test + + + + + + org.apache.hbase + hbase-common + true + + + org.apache.hbase + hbase-testing-util + test + + + + org.apache.zookeeper + zookeeper + test + diff --git a/flume-ng-sinks/flume-ng-hbase-sink/src/main/java/org/apache/flume/sink/hbase/AsyncHBaseSink.java b/flume-ng-sinks/flume-ng-hbase-sink/src/main/java/org/apache/flume/sink/hbase/AsyncHBaseSink.java index 2d03271948..1666be4382 100644 --- a/flume-ng-sinks/flume-ng-hbase-sink/src/main/java/org/apache/flume/sink/hbase/AsyncHBaseSink.java +++ b/flume-ng-sinks/flume-ng-hbase-sink/src/main/java/org/apache/flume/sink/hbase/AsyncHBaseSink.java @@ -47,6 +47,8 @@ import org.hbase.async.AtomicIncrementRequest; import org.hbase.async.HBaseClient; import org.hbase.async.PutRequest; +import org.jboss.netty.channel.socket.nio + .NioClientSocketChannelFactory; import org.slf4j.Logger; import org.slf4j.LoggerFactory; @@ -409,13 +411,17 @@ public void start(){ + "before calling start on an old instance."); sinkCounter.start(); sinkCounter.incrementConnectionCreatedCount(); - if (!isTimeoutTest) { sinkCallbackPool = Executors.newCachedThreadPool(new ThreadFactoryBuilder() .setNameFormat(this.getName() + " HBase Call Pool").build()); + logger.info("Callback pool created"); + if(!isTimeoutTest) { + client = new HBaseClient(zkQuorum, zkBaseDir, sinkCallbackPool); } else { - sinkCallbackPool = Executors.newSingleThreadExecutor(); + client = new HBaseClient(zkQuorum, zkBaseDir, + new NioClientSocketChannelFactory(Executors + .newSingleThreadExecutor(), + Executors.newSingleThreadExecutor())); } - client = new HBaseClient(zkQuorum, zkBaseDir, sinkCallbackPool); final CountDownLatch latch = new CountDownLatch(1); final AtomicBoolean fail = new AtomicBoolean(false); client.ensureTableFamilyExists( @@ -424,6 +430,7 @@ public void start(){ @Override public Object call(Object arg) throws Exception { latch.countDown(); + logger.info("table found"); return null; } }, @@ -437,7 +444,9 @@ public Object call(Object arg) throws Exception { }); try { + logger.info("waiting on callback"); latch.await(); + logger.info("callback received"); } catch (InterruptedException e) { sinkCounter.incrementConnectionFailedCount(); throw new FlumeException( @@ -465,15 +474,20 @@ public void stop(){ } sinkCounter.incrementConnectionClosedCount(); sinkCounter.stop(); - sinkCallbackPool.shutdown(); + try { - if(!sinkCallbackPool.awaitTermination(5, TimeUnit.SECONDS)) { - sinkCallbackPool.shutdownNow(); + if (sinkCallbackPool != null) { + sinkCallbackPool.shutdown(); + if (!sinkCallbackPool.awaitTermination(5, TimeUnit.SECONDS)) { + sinkCallbackPool.shutdownNow(); + } } } catch (InterruptedException e) { logger.error("Interrupted while waiting for asynchbase sink pool to " + "die", e); - sinkCallbackPool.shutdownNow(); + if (sinkCallbackPool != null) { + sinkCallbackPool.shutdownNow(); + } } sinkCallbackPool = null; client = null; diff --git a/flume-ng-sinks/flume-ng-hbase-sink/src/test/java/org/apache/flume/sink/hbase/TestAsyncHBaseSink.java b/flume-ng-sinks/flume-ng-hbase-sink/src/test/java/org/apache/flume/sink/hbase/TestAsyncHBaseSink.java index ccbc086107..af90f99905 100644 --- a/flume-ng-sinks/flume-ng-hbase-sink/src/test/java/org/apache/flume/sink/hbase/TestAsyncHBaseSink.java +++ b/flume-ng-sinks/flume-ng-hbase-sink/src/test/java/org/apache/flume/sink/hbase/TestAsyncHBaseSink.java @@ -209,6 +209,7 @@ public void testTimeOut() throws Exception { Channel channel = new MemoryChannel(); Configurables.configure(channel, ctx); sink.setChannel(channel); + channel.start(); sink.start(); Transaction tx = channel.getTransaction(); tx.begin(); diff --git a/pom.xml b/pom.xml index d4b76607fe..5d31d4c514 100644 --- a/pom.xml +++ b/pom.xml @@ -97,6 +97,25 @@ limitations under the License. hadoop-test ${hadoop.version} + + org.apache.hbase + hbase + ${hbase.version} + + + + org.apache.hbase + hbase + ${hbase.version} + tests + test + + + org.apache.zookeeper + zookeeper + 3.4.5 + test + @@ -148,6 +167,27 @@ limitations under the License. ${hadoop.version} + + org.apache.hbase + hbase + ${hbase.version} + + + + org.apache.hbase + hbase + ${hbase.version} + tests + test + + + + org.apache.zookeeper + zookeeper + 3.4.5 + test + + org.apache.flume.flume-ng-sinks @@ -158,6 +198,134 @@ limitations under the License. + + hbase-98 + + + hadoop.profile + hbase-98 + + + + ${hadoop2.version} + 0.98.2-hadoop2 + hadoop-common + 0.8.0 + + + + + org.apache.hadoop + ${hadoop.common.artifact.id} + ${hadoop.version} + + + org.apache.hadoop + hadoop-hdfs + ${hadoop.version} + + + org.apache.hadoop + hadoop-minicluster + ${hadoop.version} + + + org.apache.hadoop + hadoop-client + ${hadoop.version} + + + org.apache.hadoop + hadoop-annotations + ${hadoop.version} + + + org.apache.hadoop + hadoop-auth + ${hadoop.version} + + + org.apache.hadoop + hadoop-minicluster + ${hadoop.version} + test + + + + + org.apache.hbase + hbase-client + ${hbase.version} + + + + org.apache.hbase + hbase-client + ${hbase.version} + tests + test + + + + org.apache.hbase + hbase-testing-util + ${hbase.version} + + + + + + org.apache.hbase + hbase-common + ${hbase.version} + + + + org.apache.hbase + hbase-common + ${hbase.version} + tests + test + + + + org.apache.hbase + hbase-server + ${hbase.version} + test + + + + org.apache.hbase + hbase-server + ${hbase.version} + tests + test + + + + org.apache.zookeeper + zookeeper + 3.4.5 + test + + + + + org.apache.flume.flume-ng-sinks + flume-dataset-sink + ${project.version} + + + + + compileThriftLegacy @@ -858,20 +1026,6 @@ limitations under the License. 2.1 - - org.apache.hbase - hbase - ${hbase.version} - - - - org.apache.hbase - hbase - ${hbase.version} - tests - test - - org.apache.hadoop hadoop From 059795fb9aa890430555f3a4c02a73ec346e7092 Mon Sep 17 00:00:00 2001 From: Hari Shreedharan Date: Tue, 10 Jun 2014 11:32:37 -0700 Subject: [PATCH 111/341] FLUME-2389. Spillable Memory Channel Example is incorrect (Roshan Naik via Hari Shreedharan) --- flume-ng-doc/sphinx/FlumeUserGuide.rst | 1 + 1 file changed, 1 insertion(+) diff --git a/flume-ng-doc/sphinx/FlumeUserGuide.rst b/flume-ng-doc/sphinx/FlumeUserGuide.rst index 040fc8b7d6..0e5e7677e3 100644 --- a/flume-ng-doc/sphinx/FlumeUserGuide.rst +++ b/flume-ng-doc/sphinx/FlumeUserGuide.rst @@ -2371,6 +2371,7 @@ To disable the use of overflow disk and function purely as a in-memory channel: a1.channels = c1 a1.channels.c1.type = SPILLABLEMEMORY a1.channels.c1.memoryCapacity = 100000 + a1.channels.c1.overflowCapacity = 0 Pseudo Transaction Channel From 38befa88c0c1004936079f9107a5b8600df5fdbd Mon Sep 17 00:00:00 2001 From: Hari Shreedharan Date: Mon, 9 Jun 2014 23:32:34 -0700 Subject: [PATCH 112/341] Updating RELEASE-NOTES for Flume 1.5.0.1 --- RELEASE-NOTES | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/RELEASE-NOTES b/RELEASE-NOTES index 31f6f61467..b3233e6c47 100644 --- a/RELEASE-NOTES +++ b/RELEASE-NOTES @@ -17,7 +17,7 @@ simple, extensible data model that allows for online analytic application. 2. Status of this release -Apache Flume 1.5.0 is the fifth release of Flume as an Apache top-level project +Apache Flume 1.5.0.1 is the sixth release of Flume as an Apache top-level project (TLP). Apache Flume 1.5.0 is production-ready software. 3. Major changes in this Release From c48c26a358966fdc4dd0ec73002269c13fbaf758 Mon Sep 17 00:00:00 2001 From: Hari Shreedharan Date: Tue, 10 Jun 2014 11:39:28 -0700 Subject: [PATCH 113/341] FLUME-2399. Update CHANGELOG for Flume 1.5.0.1 --- CHANGELOG | 6 ++++++ 1 file changed, 6 insertions(+) diff --git a/CHANGELOG b/CHANGELOG index 47894db147..ceed9e5a0b 100644 --- a/CHANGELOG +++ b/CHANGELOG @@ -1,3 +1,9 @@ +Release Notes - Flume - Version v1.5.0.1 + +** Bug + * [FLUME-2389] - Spillable Memory Channel Example is incorrect + * [FLUME-2397] - HBase-98 compatibility + Release Notes - Flume - Version v1.5.0 From 8bfc90c0e4042fd2273d0de23b377fdd69f4c049 Mon Sep 17 00:00:00 2001 From: Hari Shreedharan Date: Tue, 10 Jun 2014 12:01:16 -0700 Subject: [PATCH 114/341] FLUME-2400. Dataset Sink is not built in hbase-98 profile --- flume-ng-dist/pom.xml | 15 +++++++++++++++ flume-ng-sinks/pom.xml | 16 ++++++++++++++++ 2 files changed, 31 insertions(+) diff --git a/flume-ng-dist/pom.xml b/flume-ng-dist/pom.xml index 7821dc05c0..8c18af69ac 100644 --- a/flume-ng-dist/pom.xml +++ b/flume-ng-dist/pom.xml @@ -79,6 +79,21 @@ + + hbase-98 + + + hadoop.profile + hbase-98 + + + + + org.apache.flume.flume-ng-sinks + flume-dataset-sink + + + diff --git a/flume-ng-sinks/pom.xml b/flume-ng-sinks/pom.xml index c919605fc3..3381bde2a1 100644 --- a/flume-ng-sinks/pom.xml +++ b/flume-ng-sinks/pom.xml @@ -74,6 +74,22 @@ limitations under the License. + + hbase-98 + + + hadoop.profile + hbase-98 + + + + + flume-dataset-sink + + + + From 3708063ef740803f62e61d52bd3ba6e2a94d7c3e Mon Sep 17 00:00:00 2001 From: Hari Shreedharan Date: Tue, 10 Jun 2014 12:10:28 -0700 Subject: [PATCH 115/341] FLUME-2359. Update CHANGELOG to include FLUME-2400 --- CHANGELOG | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/CHANGELOG b/CHANGELOG index ceed9e5a0b..7b9c496040 100644 --- a/CHANGELOG +++ b/CHANGELOG @@ -3,7 +3,7 @@ Release Notes - Flume - Version v1.5.0.1 ** Bug * [FLUME-2389] - Spillable Memory Channel Example is incorrect * [FLUME-2397] - HBase-98 compatibility - + * [FLUME-2400] - Dataset Sink is not built in hbase-98 profile Release Notes - Flume - Version v1.5.0 From 1c318366a630c457afe3b6f26d98bb6df9d973c1 Mon Sep 17 00:00:00 2001 From: Hari Shreedharan Date: Tue, 10 Jun 2014 14:08:01 -0700 Subject: [PATCH 116/341] FLUME-2399. Minor update to Release notes. --- RELEASE-NOTES | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/RELEASE-NOTES b/RELEASE-NOTES index b3233e6c47..5a6ab4d461 100644 --- a/RELEASE-NOTES +++ b/RELEASE-NOTES @@ -1,4 +1,4 @@ -Apache Flume 1.5.0 +Apache Flume 1.5.0.1 CONTENTS 1. What is Apache Flume @@ -18,7 +18,7 @@ application. 2. Status of this release Apache Flume 1.5.0.1 is the sixth release of Flume as an Apache top-level project -(TLP). Apache Flume 1.5.0 is production-ready software. +(TLP). Apache Flume 1.5.0.1 is production-ready software. 3. Major changes in this Release For a detailed list of changes, please see the CHANGELOG file included From f7e240ff3e376bac2dc0fb91e7c7ae32a933f899 Mon Sep 17 00:00:00 2001 From: Hari Shreedharan Date: Fri, 13 Jun 2014 16:20:47 -0700 Subject: [PATCH 117/341] FLUME-2365. Add DOAP file for Flume. (Ashish Paliwal via Hari Shreedharan) --- doap_Flume.rdf | 57 ++++++++++++++++++++++++++++++++++++++++++++++++++ 1 file changed, 57 insertions(+) create mode 100644 doap_Flume.rdf diff --git a/doap_Flume.rdf b/doap_Flume.rdf new file mode 100644 index 0000000000..03a7973f13 --- /dev/null +++ b/doap_Flume.rdf @@ -0,0 +1,57 @@ + + + + + + 2014-06-13 + + Apache Flume + + + Apache Flume is a distributed, reliable, and available system for efficiently collecting, aggregating and moving large amounts of log data from many different sources to a centralized data store. + Apache Flume is a distributed, reliable, and available system for efficiently collecting, aggregating and moving large amounts of log data from many different sources to a centralized data store + + + + Java + + + + Apache Flume + 2014-05-20 + 1.5.0 + + + + + + + + + + + Apache Flume Developers + + + + + From b74e35aeb6aea87a29f062f55c79a499c6497abe Mon Sep 17 00:00:00 2001 From: Roshan Naik Date: Fri, 13 Jun 2014 19:26:02 -0700 Subject: [PATCH 118/341] FLUME-2024. Add ExecSource flush timeout to Flume User Guide (Ashish Paliwal via Roshan Naik) --- .../src/main/java/org/apache/flume/source/ExecSource.java | 6 ++++++ flume-ng-doc/sphinx/FlumeUserGuide.rst | 1 + 2 files changed, 7 insertions(+) diff --git a/flume-ng-core/src/main/java/org/apache/flume/source/ExecSource.java b/flume-ng-core/src/main/java/org/apache/flume/source/ExecSource.java index 1d8d26720c..18e662c79c 100644 --- a/flume-ng-core/src/main/java/org/apache/flume/source/ExecSource.java +++ b/flume-ng-core/src/main/java/org/apache/flume/source/ExecSource.java @@ -131,6 +131,12 @@ * integer * 20 * + * + * batchTimeout + * Amount of time (in milliseconds) to wait, if the buffer size was not reached, before data is pushed downstream. + * long + * 3000 + * * *

    * Metrics diff --git a/flume-ng-doc/sphinx/FlumeUserGuide.rst b/flume-ng-doc/sphinx/FlumeUserGuide.rst index 0e5e7677e3..f0dd8e8a42 100644 --- a/flume-ng-doc/sphinx/FlumeUserGuide.rst +++ b/flume-ng-doc/sphinx/FlumeUserGuide.rst @@ -792,6 +792,7 @@ restartThrottle 10000 Amount of time (in millis) to wait before attempti restart false Whether the executed cmd should be restarted if it dies logStdErr false Whether the command's stderr should be logged batchSize 20 The max number of lines to read and send to the channel at a time +batchTimeout 3000 Amount of time (in milliseconds) to wait, if the buffer size was not reached, before data is pushed downstream selector.type replicating replicating or multiplexing selector.* Depends on the selector.type value interceptors -- Space-separated list of interceptors From d0d00c3711c88c4da942dafb34ef3ab8861afdcd Mon Sep 17 00:00:00 2001 From: Hari Shreedharan Date: Wed, 18 Jun 2014 23:22:12 -0700 Subject: [PATCH 119/341] FLUME-2408. Remove FLUME-1899 from CHANGELOG. --- CHANGELOG | 1 - 1 file changed, 1 deletion(-) diff --git a/CHANGELOG b/CHANGELOG index 7b9c496040..e5def5df9f 100644 --- a/CHANGELOG +++ b/CHANGELOG @@ -19,7 +19,6 @@ Release Notes - Flume - Version v1.5.0 ** Improvement * [FLUME-1542] - Flume User Guide sample configurations need fixing - * [FLUME-1899] - Make SpoolDir work with Sub-Directories * [FLUME-2052] - Spooling directory source should be able to replace or ignore malformed characters * [FLUME-2130] - Handle larger payloads via SyslogUDPSource * [FLUME-2139] - upgrade morphline library dependency to a 0.5.0 release From 9940dcbfefbe1248f65aa83f2f84e352ce022041 Mon Sep 17 00:00:00 2001 From: Jarek Jarcec Cecho Date: Thu, 26 Jun 2014 20:57:31 -0700 Subject: [PATCH 120/341] FLUME-2416: Use CodecPool in compressed stream to prevent leak of direct buffers (Hari Shreedharan via Jarek Jarcec Cecho) --- .../flume/sink/hdfs/HDFSCompressedDataStream.java | 13 +++++++++++-- 1 file changed, 11 insertions(+), 2 deletions(-) diff --git a/flume-ng-sinks/flume-hdfs-sink/src/main/java/org/apache/flume/sink/hdfs/HDFSCompressedDataStream.java b/flume-ng-sinks/flume-hdfs-sink/src/main/java/org/apache/flume/sink/hdfs/HDFSCompressedDataStream.java index fe857c39aa..dc93e4f421 100644 --- a/flume-ng-sinks/flume-hdfs-sink/src/main/java/org/apache/flume/sink/hdfs/HDFSCompressedDataStream.java +++ b/flume-ng-sinks/flume-hdfs-sink/src/main/java/org/apache/flume/sink/hdfs/HDFSCompressedDataStream.java @@ -29,8 +29,10 @@ import org.apache.hadoop.fs.LocalFileSystem; import org.apache.hadoop.fs.Path; import org.apache.hadoop.io.SequenceFile.CompressionType; +import org.apache.hadoop.io.compress.CodecPool; import org.apache.hadoop.io.compress.CompressionCodec; import org.apache.hadoop.io.compress.CompressionOutputStream; +import org.apache.hadoop.io.compress.Compressor; import org.apache.hadoop.io.compress.DefaultCodec; import org.slf4j.Logger; import org.slf4j.LoggerFactory; @@ -48,6 +50,7 @@ public class HDFSCompressedDataStream extends AbstractHDFSWriter { private Context serializerContext; private EventSerializer serializer; private boolean useRawLocalFileSystem; + private Compressor compressor; @Override public void configure(Context context) { @@ -83,7 +86,6 @@ public void open(String filePath, CompressionCodec codec, "is not of type LocalFileSystem: " + hdfs.getClass().getName()); } } - boolean appending = false; if (conf.getBoolean("hdfs.append.support", false) == true && hdfs.isFile (dstPath)) { @@ -92,7 +94,10 @@ public void open(String filePath, CompressionCodec codec, } else { fsOut = hdfs.create(dstPath); } - cmpOut = codec.createOutputStream(fsOut); + if(compressor == null) { + compressor = CodecPool.getCompressor(codec, conf); + } + cmpOut = codec.createOutputStream(fsOut, compressor); serializer = EventSerializerFactory.getInstance(serializerType, serializerContext, cmpOut); if (appending && !serializer.supportsReopen()) { @@ -148,6 +153,10 @@ public void close() throws IOException { fsOut.flush(); fsOut.sync(); cmpOut.close(); + if (compressor != null) { + CodecPool.returnCompressor(compressor); + compressor = null; + } unregisterCurrentStream(); } From f15f20785262ac3cb3e35c2a12e669b7a836d35f Mon Sep 17 00:00:00 2001 From: Mike Percy Date: Fri, 4 Jul 2014 15:41:43 -0700 Subject: [PATCH 121/341] FLUME-2404. Make ScribeSource read buffer and max frame size configurable Scribe default Thrift service maxReadBufferBytes and frame size varies across Thrift versions. In some cases, these values are set to INT_MAX, in other cases this is set to 16MB. To avoid OOM in certain cases and incompatibilities in other cases, set the default to 16MB and also make the parameters configurable. (chenshangan and Marimuthu Ponnambalam via Mike Percy) --- flume-ng-doc/sphinx/FlumeUserGuide.rst | 15 ++++++++------- .../apache/flume/source/scribe/ScribeSource.java | 14 +++++++++++--- 2 files changed, 19 insertions(+), 10 deletions(-) diff --git a/flume-ng-doc/sphinx/FlumeUserGuide.rst b/flume-ng-doc/sphinx/FlumeUserGuide.rst index f0dd8e8a42..1e98725c26 100644 --- a/flume-ng-doc/sphinx/FlumeUserGuide.rst +++ b/flume-ng-doc/sphinx/FlumeUserGuide.rst @@ -1484,15 +1484,16 @@ Flume should use ScribeSource based on Thrift with compatible transfering protoc For deployment of Scribe please follow the guide from Facebook. Required properties are in **bold**. -============== =========== ============================================== -Property Name Default Description -============== =========== ============================================== -**type** -- The component type name, needs to be ``org.apache.flume.source.scribe.ScribeSource`` -port 1499 Port that Scribe should be connected -workerThreads 5 Handing threads number in Thrift +==================== =========== ============================================== +Property Name Default Description +==================== =========== ============================================== +**type** -- The component type name, needs to be ``org.apache.flume.source.scribe.ScribeSource`` +port 1499 Port that Scribe should be connected +maxReadBufferBytes 16384000 Thrift Default FrameBuffer Size +workerThreads 5 Handing threads number in Thrift selector.type selector.* -============== =========== ============================================== +==================== =========== ============================================== Example for agent named a1: diff --git a/flume-ng-sources/flume-scribe-source/src/main/java/org/apache/flume/source/scribe/ScribeSource.java b/flume-ng-sources/flume-scribe-source/src/main/java/org/apache/flume/source/scribe/ScribeSource.java index f9a14c1c4c..1d7da09492 100644 --- a/flume-ng-sources/flume-scribe-source/src/main/java/org/apache/flume/source/scribe/ScribeSource.java +++ b/flume-ng-sources/flume-scribe-source/src/main/java/org/apache/flume/source/scribe/ScribeSource.java @@ -59,17 +59,24 @@ public class ScribeSource extends AbstractSource implements public static final String SCRIBE_CATEGORY = "category"; + private static final int DEFAULT_PORT = 1499; private static final int DEFAULT_WORKERS = 5; + private static final int DEFAULT_MAX_READ_BUFFER_BYTES = 16384000; private TServer server; - private int port = 1499; + private int port; private int workers; + private int maxReadBufferBytes; private SourceCounter sourceCounter; @Override public void configure(Context context) { - port = context.getInteger("port", port); + port = context.getInteger("port", DEFAULT_PORT); + maxReadBufferBytes = context.getInteger("maxReadBufferBytes", DEFAULT_MAX_READ_BUFFER_BYTES); + if(maxReadBufferBytes <= 0){ + maxReadBufferBytes = DEFAULT_MAX_READ_BUFFER_BYTES; + } workers = context.getInteger("workerThreads", DEFAULT_WORKERS); if (workers <= 0) { @@ -91,8 +98,9 @@ public void run() { args.workerThreads(workers); args.processor(processor); - args.transportFactory(new TFramedTransport.Factory()); + args.transportFactory(new TFramedTransport.Factory(maxReadBufferBytes)); args.protocolFactory(new TBinaryProtocol.Factory(false, false)); + args.maxReadBufferBytes = maxReadBufferBytes; server = new THsHaServer(args); From 69fd6b3ad5e5b9ae6f1293b3d8e57ed57fd6701c Mon Sep 17 00:00:00 2001 From: Hari Shreedharan Date: Tue, 8 Jul 2014 17:20:36 -0700 Subject: [PATCH 122/341] FLUME-2401. Optionally compress backup checkpoint. (Abraham Fine via Hari Shreedharan) --- flume-ng-channels/flume-file-channel/pom.xml | 5 + .../file/EventQueueBackingStoreFactory.java | 15 +- .../file/EventQueueBackingStoreFile.java | 28 +++- .../file/EventQueueBackingStoreFileV3.java | 8 +- .../flume/channel/file/FileChannel.java | 7 + .../file/FileChannelConfiguration.java | 5 + .../org/apache/flume/channel/file/Log.java | 31 ++-- .../flume/channel/file/Serialization.java | 146 +++++++++++++++++- .../channel/file/TestFileChannelBase.java | 5 + .../channel/file/TestFileChannelRestart.java | 98 +++++++++++- pom.xml | 6 + 11 files changed, 323 insertions(+), 31 deletions(-) diff --git a/flume-ng-channels/flume-file-channel/pom.xml b/flume-ng-channels/flume-file-channel/pom.xml index 3113938a97..7b8114c2b0 100644 --- a/flume-ng-channels/flume-file-channel/pom.xml +++ b/flume-ng-channels/flume-file-channel/pom.xml @@ -107,6 +107,11 @@ mapdb + + org.xerial.snappy + snappy-java + + diff --git a/flume-ng-channels/flume-file-channel/src/main/java/org/apache/flume/channel/file/EventQueueBackingStoreFactory.java b/flume-ng-channels/flume-file-channel/src/main/java/org/apache/flume/channel/file/EventQueueBackingStoreFactory.java index 07a3781023..456df34e2d 100644 --- a/flume-ng-channels/flume-file-channel/src/main/java/org/apache/flume/channel/file/EventQueueBackingStoreFactory.java +++ b/flume-ng-channels/flume-file-channel/src/main/java/org/apache/flume/channel/file/EventQueueBackingStoreFactory.java @@ -38,11 +38,12 @@ static EventQueueBackingStore get(File checkpointFile, int capacity, static EventQueueBackingStore get(File checkpointFile, int capacity, String name, boolean upgrade) throws Exception { - return get(checkpointFile, null, capacity, name, upgrade, false); + return get(checkpointFile, null, capacity, name, upgrade, false, false); } static EventQueueBackingStore get(File checkpointFile, File backupCheckpointDir, int capacity,String name, - boolean upgrade, boolean shouldBackup) throws Exception { + boolean upgrade, boolean shouldBackup, boolean compressBackup) + throws Exception { File metaDataFile = Serialization.getMetaDataFile(checkpointFile); RandomAccessFile checkpointFileHandle = null; try { @@ -68,19 +69,19 @@ static EventQueueBackingStore get(File checkpointFile, throw new IOException("Cannot create " + checkpointFile); } return new EventQueueBackingStoreFileV3(checkpointFile, - capacity, name, backupCheckpointDir, shouldBackup); + capacity, name, backupCheckpointDir, shouldBackup, compressBackup); } // v3 due to meta file, version will be checked by backing store if(metaDataExists) { return new EventQueueBackingStoreFileV3(checkpointFile, capacity, - name, backupCheckpointDir, shouldBackup); + name, backupCheckpointDir, shouldBackup, compressBackup); } checkpointFileHandle = new RandomAccessFile(checkpointFile, "r"); int version = (int)checkpointFileHandle.readLong(); if(Serialization.VERSION_2 == version) { if(upgrade) { return upgrade(checkpointFile, capacity, name, backupCheckpointDir, - shouldBackup); + shouldBackup, compressBackup); } return new EventQueueBackingStoreFileV2(checkpointFile, capacity, name); } @@ -101,7 +102,7 @@ static EventQueueBackingStore get(File checkpointFile, private static EventQueueBackingStore upgrade(File checkpointFile, int capacity, String name, File backupCheckpointDir, - boolean shouldBackup) + boolean shouldBackup, boolean compressBackup) throws Exception { LOG.info("Attempting upgrade of " + checkpointFile + " for " + name); EventQueueBackingStoreFileV2 backingStoreV2 = @@ -114,7 +115,7 @@ private static EventQueueBackingStore upgrade(File checkpointFile, EventQueueBackingStoreFileV3.upgrade(backingStoreV2, checkpointFile, metaDataFile); return new EventQueueBackingStoreFileV3(checkpointFile, capacity, name, - backupCheckpointDir, shouldBackup); + backupCheckpointDir, shouldBackup, compressBackup); } } diff --git a/flume-ng-channels/flume-file-channel/src/main/java/org/apache/flume/channel/file/EventQueueBackingStoreFile.java b/flume-ng-channels/flume-file-channel/src/main/java/org/apache/flume/channel/file/EventQueueBackingStoreFile.java index 113dcd22c2..2b0987b40a 100644 --- a/flume-ng-channels/flume-file-channel/src/main/java/org/apache/flume/channel/file/EventQueueBackingStoreFile.java +++ b/flume-ng-channels/flume-file-channel/src/main/java/org/apache/flume/channel/file/EventQueueBackingStoreFile.java @@ -56,6 +56,8 @@ abstract class EventQueueBackingStoreFile extends EventQueueBackingStore { protected static final int CHECKPOINT_COMPLETE = 0; protected static final int CHECKPOINT_INCOMPLETE = 1; + protected static final String COMPRESSED_FILE_EXTENSION = ".snappy"; + protected LongBuffer elementsBuffer; protected final Map overwriteMap = new HashMap(); protected final Map logFileIDReferenceCounts = Maps.newHashMap(); @@ -64,22 +66,24 @@ abstract class EventQueueBackingStoreFile extends EventQueueBackingStore { protected final File checkpointFile; private final Semaphore backupCompletedSema = new Semaphore(1); protected final boolean shouldBackup; + protected final boolean compressBackup; private final File backupDir; private final ExecutorService checkpointBackUpExecutor; protected EventQueueBackingStoreFile(int capacity, String name, File checkpointFile) throws IOException, BadCheckpointException { - this(capacity, name, checkpointFile, null, false); + this(capacity, name, checkpointFile, null, false, false); } protected EventQueueBackingStoreFile(int capacity, String name, File checkpointFile, File checkpointBackupDir, - boolean backupCheckpoint) throws IOException, - BadCheckpointException { + boolean backupCheckpoint, boolean compressBackup) + throws IOException, BadCheckpointException { super(capacity, name); this.checkpointFile = checkpointFile; this.shouldBackup = backupCheckpoint; + this.compressBackup = compressBackup; this.backupDir = checkpointBackupDir; checkpointFileHandle = new RandomAccessFile(checkpointFile, "rw"); long totalBytes = (capacity + HEADER_SIZE) * Serialization.SIZE_OF_LONG; @@ -169,8 +173,13 @@ protected void backupCheckpoint(File backupDirectory) throws IOException { if(Log.EXCLUDES.contains(origFile.getName())) { continue; } - Serialization.copyFile(origFile, new File(backupDirectory, - origFile.getName())); + if (compressBackup && origFile.equals(checkpointFile)) { + Serialization.compressFile(origFile, new File(backupDirectory, + origFile.getName() + COMPRESSED_FILE_EXTENSION)); + } else { + Serialization.copyFile(origFile, new File(backupDirectory, + origFile.getName())); + } } Preconditions.checkState(!backupFile.exists(), "The backup file exists " + "while it is not supposed to. Are multiple channels configured to use " + @@ -202,7 +211,14 @@ public static boolean restoreBackup(File checkpointDir, File backupDir) String fileName = backupFile.getName(); if (!fileName.equals(BACKUP_COMPLETE_FILENAME) && !fileName.equals(Log.FILE_LOCK)) { - Serialization.copyFile(backupFile, new File(checkpointDir, fileName)); + if (fileName.endsWith(COMPRESSED_FILE_EXTENSION)){ + Serialization.decompressFile( + backupFile, new File(checkpointDir, + fileName.substring(0, fileName.lastIndexOf(".")))); + } else { + Serialization.copyFile(backupFile, new File(checkpointDir, + fileName)); + } } } return true; diff --git a/flume-ng-channels/flume-file-channel/src/main/java/org/apache/flume/channel/file/EventQueueBackingStoreFileV3.java b/flume-ng-channels/flume-file-channel/src/main/java/org/apache/flume/channel/file/EventQueueBackingStoreFileV3.java index c153558f00..9dfa0d13a1 100644 --- a/flume-ng-channels/flume-file-channel/src/main/java/org/apache/flume/channel/file/EventQueueBackingStoreFileV3.java +++ b/flume-ng-channels/flume-file-channel/src/main/java/org/apache/flume/channel/file/EventQueueBackingStoreFileV3.java @@ -40,13 +40,15 @@ final class EventQueueBackingStoreFileV3 extends EventQueueBackingStoreFile { EventQueueBackingStoreFileV3(File checkpointFile, int capacity, String name) throws IOException, BadCheckpointException { - this(checkpointFile, capacity, name, null, false); + this(checkpointFile, capacity, name, null, false, false); } EventQueueBackingStoreFileV3(File checkpointFile, int capacity, String name, File checkpointBackupDir, - boolean backupCheckpoint) throws IOException, BadCheckpointException { - super(capacity, name, checkpointFile, checkpointBackupDir, backupCheckpoint); + boolean backupCheckpoint, boolean compressBackup) + throws IOException, BadCheckpointException { + super(capacity, name, checkpointFile, checkpointBackupDir, backupCheckpoint, + compressBackup); Preconditions.checkArgument(capacity > 0, "capacity must be greater than 0 " + capacity); metaDataFile = Serialization.getMetaDataFile(checkpointFile); diff --git a/flume-ng-channels/flume-file-channel/src/main/java/org/apache/flume/channel/file/FileChannel.java b/flume-ng-channels/flume-file-channel/src/main/java/org/apache/flume/channel/file/FileChannel.java index 0f242d20f5..413bfbc268 100644 --- a/flume-ng-channels/flume-file-channel/src/main/java/org/apache/flume/channel/file/FileChannel.java +++ b/flume-ng-channels/flume-file-channel/src/main/java/org/apache/flume/channel/file/FileChannel.java @@ -95,6 +95,7 @@ public class FileChannel extends BasicChannelSemantics { private String encryptionActiveKey; private String encryptionCipherProvider; private boolean useDualCheckpoints; + private boolean compressBackupCheckpoint; private boolean fsyncPerTransaction; private int fsyncInterval; @@ -110,6 +111,11 @@ public void configure(Context context) { useDualCheckpoints = context.getBoolean( FileChannelConfiguration.USE_DUAL_CHECKPOINTS, FileChannelConfiguration.DEFAULT_USE_DUAL_CHECKPOINTS); + + compressBackupCheckpoint = context.getBoolean( + FileChannelConfiguration.COMPRESS_BACKUP_CHECKPOINT, + FileChannelConfiguration.DEFAULT_COMPRESS_BACKUP_CHECKPOINT); + String homePath = System.getProperty("user.home").replace('\\', '/'); String strCheckpointDir = @@ -272,6 +278,7 @@ public synchronized void start() { builder.setEncryptionKeyAlias(encryptionActiveKey); builder.setEncryptionCipherProvider(encryptionCipherProvider); builder.setUseDualCheckpoints(useDualCheckpoints); + builder.setCompressBackupCheckpoint(compressBackupCheckpoint); builder.setBackupCheckpointDir(backupCheckpointDir); builder.setFsyncPerTransaction(fsyncPerTransaction); builder.setFsyncInterval(fsyncInterval); diff --git a/flume-ng-channels/flume-file-channel/src/main/java/org/apache/flume/channel/file/FileChannelConfiguration.java b/flume-ng-channels/flume-file-channel/src/main/java/org/apache/flume/channel/file/FileChannelConfiguration.java index 87dc653108..f8c037884d 100644 --- a/flume-ng-channels/flume-file-channel/src/main/java/org/apache/flume/channel/file/FileChannelConfiguration.java +++ b/flume-ng-channels/flume-file-channel/src/main/java/org/apache/flume/channel/file/FileChannelConfiguration.java @@ -87,6 +87,11 @@ public class FileChannelConfiguration { public static final String USE_DUAL_CHECKPOINTS = "useDualCheckpoints"; public static final boolean DEFAULT_USE_DUAL_CHECKPOINTS = false; + public static final String COMPRESS_BACKUP_CHECKPOINT = + "compressBackupCheckpoint"; + public static final boolean DEFAULT_COMPRESS_BACKUP_CHECKPOINT + = false; + public static final String FSYNC_PER_TXN = "fsyncPerTransaction"; public static final boolean DEFAULT_FSYNC_PRE_TXN = true; diff --git a/flume-ng-channels/flume-file-channel/src/main/java/org/apache/flume/channel/file/Log.java b/flume-ng-channels/flume-file-channel/src/main/java/org/apache/flume/channel/file/Log.java index 5bac0f4f87..5b581e1c18 100644 --- a/flume-ng-channels/flume-file-channel/src/main/java/org/apache/flume/channel/file/Log.java +++ b/flume-ng-channels/flume-file-channel/src/main/java/org/apache/flume/channel/file/Log.java @@ -123,6 +123,7 @@ public class Log { private boolean didFastReplay = false; private boolean didFullReplayDueToBadCheckpointException = false; private final boolean useDualCheckpoints; + private final boolean compressBackupCheckpoint; private volatile boolean backupRestored = false; private final boolean fsyncPerTransaction; @@ -151,6 +152,7 @@ static class Builder { private String bEncryptionCipherProvider; private long bUsableSpaceRefreshInterval = 15L * 1000L; private boolean bUseDualCheckpoints = false; + private boolean bCompressBackupCheckpoint = false; private File bBackupCheckpointDir = null; private boolean fsyncPerTransaction = true; @@ -242,6 +244,11 @@ Builder setUseDualCheckpoints(boolean UseDualCheckpoints) { return this; } + Builder setCompressBackupCheckpoint(boolean compressBackupCheckpoint) { + this.bCompressBackupCheckpoint = compressBackupCheckpoint; + return this; + } + Builder setBackupCheckpointDir(File backupCheckpointDir) { this.bBackupCheckpointDir = backupCheckpointDir; return this; @@ -249,16 +256,17 @@ Builder setBackupCheckpointDir(File backupCheckpointDir) { Log build() throws IOException { return new Log(bCheckpointInterval, bMaxFileSize, bQueueCapacity, - bUseDualCheckpoints, bCheckpointDir, bBackupCheckpointDir, bName, - useLogReplayV1, useFastReplay, bMinimumRequiredSpace, - bEncryptionKeyProvider, bEncryptionKeyAlias, + bUseDualCheckpoints, bCompressBackupCheckpoint,bCheckpointDir, + bBackupCheckpointDir, bName, useLogReplayV1, useFastReplay, + bMinimumRequiredSpace, bEncryptionKeyProvider, bEncryptionKeyAlias, bEncryptionCipherProvider, bUsableSpaceRefreshInterval, fsyncPerTransaction, fsyncInterval, bLogDirs); } } private Log(long checkpointInterval, long maxFileSize, int queueCapacity, - boolean useDualCheckpoints, File checkpointDir, File backupCheckpointDir, + boolean useDualCheckpoints, boolean compressBackupCheckpoint, + File checkpointDir, File backupCheckpointDir, String name, boolean useLogReplayV1, boolean useFastReplay, long minimumRequiredSpace, @Nullable KeyProvider encryptionKeyProvider, @Nullable String encryptionKeyAlias, @@ -338,6 +346,7 @@ private Log(long checkpointInterval, long maxFileSize, int queueCapacity, this.maxFileSize = maxFileSize; this.queueCapacity = queueCapacity; this.useDualCheckpoints = useDualCheckpoints; + this.compressBackupCheckpoint = compressBackupCheckpoint; this.checkpointDir = checkpointDir; this.backupCheckpointDir = backupCheckpointDir; this.logDirs = logDirs; @@ -415,9 +424,10 @@ void replay() throws IOException { try { backingStore = - EventQueueBackingStoreFactory.get(checkpointFile, - backupCheckpointDir, queueCapacity, channelNameDescriptor, - true, this.useDualCheckpoints); + EventQueueBackingStoreFactory.get(checkpointFile, + backupCheckpointDir, queueCapacity, channelNameDescriptor, + true, this.useDualCheckpoints, + this.compressBackupCheckpoint); queue = new FlumeEventQueue(backingStore, inflightTakesFile, inflightPutsFile, queueSetDir); LOGGER.info("Last Checkpoint " + new Date(checkpointFile.lastModified()) @@ -451,9 +461,10 @@ void replay() throws IOException { "directory to recover from a corrupt or incomplete checkpoint"); } } - backingStore = EventQueueBackingStoreFactory.get(checkpointFile, - backupCheckpointDir, - queueCapacity, channelNameDescriptor, true, useDualCheckpoints); + backingStore = EventQueueBackingStoreFactory.get( + checkpointFile, backupCheckpointDir, queueCapacity, + channelNameDescriptor, true, useDualCheckpoints, + compressBackupCheckpoint); queue = new FlumeEventQueue(backingStore, inflightTakesFile, inflightPutsFile, queueSetDir); // If the checkpoint was deleted due to BadCheckpointException, then diff --git a/flume-ng-channels/flume-file-channel/src/main/java/org/apache/flume/channel/file/Serialization.java b/flume-ng-channels/flume-file-channel/src/main/java/org/apache/flume/channel/file/Serialization.java index d55660d186..a6eda75fa6 100644 --- a/flume-ng-channels/flume-file-channel/src/main/java/org/apache/flume/channel/file/Serialization.java +++ b/flume-ng-channels/flume-file-channel/src/main/java/org/apache/flume/channel/file/Serialization.java @@ -25,11 +25,14 @@ import org.apache.flume.annotations.InterfaceStability; import org.slf4j.Logger; import org.slf4j.LoggerFactory; +import org.xerial.snappy.SnappyInputStream; +import org.xerial.snappy.SnappyOutputStream; import javax.annotation.Nullable; import java.io.BufferedInputStream; import java.io.File; import java.io.FileInputStream; +import java.io.FileOutputStream; import java.io.IOException; import java.io.RandomAccessFile; import java.util.Collections; @@ -52,8 +55,8 @@ private Serialization() {} public static final String OLD_METADATA_FILENAME = METADATA_FILENAME + ".old"; - // 64 K buffer to copy files. - private static final int FILE_COPY_BUFFER_SIZE = 64 * 1024; + // 64 K buffer to copy and compress files. + private static final int FILE_BUFFER_SIZE = 64 * 1024; public static final Logger LOG = LoggerFactory.getLogger(Serialization.class); @@ -140,7 +143,7 @@ public static boolean copyFile(File from, File to) throws IOException { try { in = new BufferedInputStream(new FileInputStream(from)); out = new RandomAccessFile(to, "rw"); - byte[] buf = new byte[FILE_COPY_BUFFER_SIZE]; + byte[] buf = new byte[FILE_BUFFER_SIZE]; int total = 0; while(true) { int read = in.read(buf); @@ -184,4 +187,141 @@ public static boolean copyFile(File from, File to) throws IOException { throw new IOException("Copying file: " + from.toString() + " to: " + to .toString() + " may have failed."); } + + /** + * Compress file using Snappy + * @param uncompressed File to compress - this file should exist + * @param compressed Compressed file - this file should not exist + * @return true if compression was successful + */ + public static boolean compressFile(File uncompressed, File compressed) + throws IOException { + Preconditions.checkNotNull(uncompressed, + "Source file is null, compression failed."); + Preconditions.checkNotNull(compressed, + "Destination file is null, compression failed."); + Preconditions.checkState(uncompressed.exists(), "Source file: " + + uncompressed.toString() + " does not exist."); + Preconditions.checkState(!compressed.exists(), + "Compressed file: " + compressed.toString() + " unexpectedly " + + "exists."); + + BufferedInputStream in = null; + FileOutputStream out = null; + SnappyOutputStream snappyOut = null; + try { + in = new BufferedInputStream(new FileInputStream(uncompressed)); + out = new FileOutputStream(compressed); + snappyOut = new SnappyOutputStream(out); + + byte[] buf = new byte[FILE_BUFFER_SIZE]; + while(true) { + int read = in.read(buf); + if (read == -1) { + break; + } + snappyOut.write(buf, 0, read); + } + out.getFD().sync(); + return true; + } catch (Exception ex) { + LOG.error("Error while attempting to compress " + + uncompressed.toString() + " to " + compressed.toString() + + ".", ex); + Throwables.propagate(ex); + } finally { + Throwable th = null; + try { + if (in != null) { + in.close(); + } + } catch (Throwable ex) { + LOG.error("Error while closing input file.", ex); + th = ex; + } + try { + if (snappyOut != null) { + snappyOut.close(); + } + } catch (IOException ex) { + LOG.error("Error while closing output file.", ex); + Throwables.propagate(ex); + } + if (th != null) { + Throwables.propagate(th); + } + } + // Should never reach here. + throw new IOException("Copying file: " + uncompressed.toString() + + " to: " + compressed.toString() + " may have failed."); + } + + /** + * Decompress file using Snappy + * @param compressed File to compress - this file should exist + * @param decompressed Compressed file - this file should not exist + * @return true if decompression was successful + */ + public static boolean decompressFile(File compressed, File decompressed) + throws IOException { + Preconditions.checkNotNull(compressed, + "Source file is null, decompression failed."); + Preconditions.checkNotNull(decompressed, "Destination file is " + + "null, decompression failed."); + Preconditions.checkState(compressed.exists(), "Source file: " + + compressed.toString() + " does not exist."); + Preconditions.checkState(!decompressed.exists(), + "Decompressed file: " + decompressed.toString() + + " unexpectedly exists."); + + BufferedInputStream in = null; + SnappyInputStream snappyIn = null; + FileOutputStream out = null; + try { + in = new BufferedInputStream(new FileInputStream(compressed)); + snappyIn = new SnappyInputStream(in); + out = new FileOutputStream(decompressed); + + byte[] buf = new byte[FILE_BUFFER_SIZE]; + while(true) { + int read = snappyIn.read(buf); + if (read == -1) { + break; + } + out.write(buf, 0, read); + } + out.getFD().sync(); + return true; + } catch (Exception ex) { + LOG.error("Error while attempting to compress " + + compressed.toString() + " to " + decompressed.toString() + + ".", ex); + Throwables.propagate(ex); + } finally { + Throwable th = null; + try { + if (in != null) { + in.close(); + } + } catch (Throwable ex) { + LOG.error("Error while closing input file.", ex); + th = ex; + } + try { + if (snappyIn != null) { + snappyIn.close(); + } + } catch (IOException ex) { + LOG.error("Error while closing output file.", ex); + Throwables.propagate(ex); + } + if (th != null) { + Throwables.propagate(th); + } + } + // Should never reach here. + throw new IOException("Decompressing file: " + + compressed.toString() + " to: " + decompressed.toString() + + " may have failed."); + } } diff --git a/flume-ng-channels/flume-file-channel/src/test/java/org/apache/flume/channel/file/TestFileChannelBase.java b/flume-ng-channels/flume-file-channel/src/test/java/org/apache/flume/channel/file/TestFileChannelBase.java index 1ee532048f..9901b69908 100644 --- a/flume-ng-channels/flume-file-channel/src/test/java/org/apache/flume/channel/file/TestFileChannelBase.java +++ b/flume-ng-channels/flume-file-channel/src/test/java/org/apache/flume/channel/file/TestFileChannelBase.java @@ -38,12 +38,17 @@ public class TestFileChannelBase { protected File[] dataDirs; protected String dataDir; protected File backupDir; + protected File uncompressedBackupCheckpoint; + protected File compressedBackupCheckpoint; @Before public void setup() throws Exception { baseDir = Files.createTempDir(); checkpointDir = new File(baseDir, "chkpt"); backupDir = new File(baseDir, "backup"); + uncompressedBackupCheckpoint = new File(backupDir, "checkpoint"); + compressedBackupCheckpoint = new File(backupDir, + "checkpoint.snappy"); Assert.assertTrue(checkpointDir.mkdirs() || checkpointDir.isDirectory()); Assert.assertTrue(backupDir.mkdirs() || backupDir.isDirectory()); dataDirs = new File[3]; diff --git a/flume-ng-channels/flume-file-channel/src/test/java/org/apache/flume/channel/file/TestFileChannelRestart.java b/flume-ng-channels/flume-file-channel/src/test/java/org/apache/flume/channel/file/TestFileChannelRestart.java index d16f3d5da5..0c6afc433a 100644 --- a/flume-ng-channels/flume-file-channel/src/test/java/org/apache/flume/channel/file/TestFileChannelRestart.java +++ b/flume-ng-channels/flume-file-channel/src/test/java/org/apache/flume/channel/file/TestFileChannelRestart.java @@ -693,13 +693,27 @@ public void testWithExtraLogs() // Make sure the entire channel was not replayed, only the events from the // backup. @Test - public void testBackupUsedEnsureNoFullReplay() throws Exception { + public void testBackupUsedEnsureNoFullReplayWithoutCompression() throws + Exception { + testBackupUsedEnsureNoFullReplay(false); + } + @Test + public void testBackupUsedEnsureNoFullReplayWithCompression() throws + Exception { + testBackupUsedEnsureNoFullReplay(true); + } + + private void testBackupUsedEnsureNoFullReplay(boolean compressedBackup) + throws Exception { File dataDir = Files.createTempDir(); File tempBackup = Files.createTempDir(); Map overrides = Maps.newHashMap(); overrides.put(FileChannelConfiguration.DATA_DIRS, dataDir.getAbsolutePath()); - overrides.put(FileChannelConfiguration.USE_DUAL_CHECKPOINTS, "true"); + overrides.put(FileChannelConfiguration.USE_DUAL_CHECKPOINTS, + "true"); + overrides.put(FileChannelConfiguration.COMPRESS_BACKUP_CHECKPOINT, + String.valueOf(compressedBackup)); channel = createFileChannel(overrides); channel.start(); Assert.assertTrue(channel.isOpen()); @@ -831,6 +845,86 @@ public void testSlowBackup() throws Throwable { } } + @Test + public void testCompressBackup() throws Throwable { + Map overrides = Maps.newHashMap(); + overrides.put(FileChannelConfiguration.USE_DUAL_CHECKPOINTS, + "true"); + overrides.put(FileChannelConfiguration.MAX_FILE_SIZE, "1000"); + overrides.put(FileChannelConfiguration.COMPRESS_BACKUP_CHECKPOINT, + "true"); + channel = createFileChannel(overrides); + channel.start(); + Assert.assertTrue(channel.isOpen()); + putEvents(channel, "restart", 10, 100); + forceCheckpoint(channel); + + //Wait for the backup checkpoint + Thread.sleep(2000); + + Assert.assertTrue(compressedBackupCheckpoint.exists()); + + Serialization.decompressFile(compressedBackupCheckpoint, + uncompressedBackupCheckpoint); + + File checkpoint = new File(checkpointDir, "checkpoint"); + Assert.assertTrue(FileUtils.contentEquals(checkpoint, + uncompressedBackupCheckpoint)); + + channel.stop(); + } + + @Test + public void testToggleCheckpointCompressionFromTrueToFalse() + throws Exception { + restartToggleCompression(true); + } + + @Test + public void testToggleCheckpointCompressionFromFalseToTrue() + throws Exception { + restartToggleCompression(false); + } + + public void restartToggleCompression(boolean originalCheckpointCompressed) + throws Exception { + Map overrides = Maps.newHashMap(); + overrides.put(FileChannelConfiguration.USE_DUAL_CHECKPOINTS, + "true"); + overrides.put(FileChannelConfiguration.MAX_FILE_SIZE, "1000"); + overrides.put(FileChannelConfiguration.COMPRESS_BACKUP_CHECKPOINT, + String.valueOf(originalCheckpointCompressed)); + channel = createFileChannel(overrides); + channel.start(); + Assert.assertTrue(channel.isOpen()); + Set in = fillChannel(channel, "restart"); + forceCheckpoint(channel); + Thread.sleep(2000); + Assert.assertEquals(compressedBackupCheckpoint.exists(), + originalCheckpointCompressed); + Assert.assertEquals(uncompressedBackupCheckpoint.exists(), + !originalCheckpointCompressed); + channel.stop(); + File checkpoint = new File(checkpointDir, "checkpoint"); + Assert.assertTrue(checkpoint.delete()); + File checkpointMetaData = Serialization.getMetaDataFile( + checkpoint); + Assert.assertTrue(checkpointMetaData.delete()); + overrides.put(FileChannelConfiguration.COMPRESS_BACKUP_CHECKPOINT, + String.valueOf(!originalCheckpointCompressed)); + channel = createFileChannel(overrides); + channel.start(); + Assert.assertTrue(channel.isOpen()); + Set out = consumeChannel(channel); + compareInputAndOut(in, out); + forceCheckpoint(channel); + Thread.sleep(2000); + Assert.assertEquals(compressedBackupCheckpoint.exists(), + !originalCheckpointCompressed); + Assert.assertEquals(uncompressedBackupCheckpoint.exists(), + originalCheckpointCompressed); + } + private static void slowdownBackup(FileChannel channel) { Log log = field("log").ofType(Log.class).in(channel).get(); diff --git a/pom.xml b/pom.xml index 5d31d4c514..541548f93c 100644 --- a/pom.xml +++ b/pom.xml @@ -1270,6 +1270,12 @@ limitations under the License. ${kite.version} + + org.xerial.snappy + snappy-java + 1.1.0 + + From 8410ad307187b19ca3a4330859815223d1e6b1e2 Mon Sep 17 00:00:00 2001 From: Hari Shreedharan Date: Thu, 17 Jul 2014 17:33:26 -0700 Subject: [PATCH 123/341] FLUME-1793. Unit test TestElasticSearchLogStashEventSerializer fails with IBM JDK (Li Xiang via Hari Shreedharan. Reviewed by Edward Sargisson) --- ...tElasticSearchLogStashEventSerializer.java | 89 ++++++++++--------- 1 file changed, 49 insertions(+), 40 deletions(-) diff --git a/flume-ng-sinks/flume-ng-elasticsearch-sink/src/test/java/org/apache/flume/sink/elasticsearch/TestElasticSearchLogStashEventSerializer.java b/flume-ng-sinks/flume-ng-elasticsearch-sink/src/test/java/org/apache/flume/sink/elasticsearch/TestElasticSearchLogStashEventSerializer.java index d2c9543cee..ab9587d282 100644 --- a/flume-ng-sinks/flume-ng-elasticsearch-sink/src/test/java/org/apache/flume/sink/elasticsearch/TestElasticSearchLogStashEventSerializer.java +++ b/flume-ng-sinks/flume-ng-elasticsearch-sink/src/test/java/org/apache/flume/sink/elasticsearch/TestElasticSearchLogStashEventSerializer.java @@ -28,6 +28,9 @@ import java.util.Date; import java.util.Map; +import com.google.gson.JsonParser; +import com.google.gson.JsonElement; + import static org.apache.flume.sink.elasticsearch.ElasticSearchEventSerializer.charset; import static org.elasticsearch.common.xcontent.XContentFactory.jsonBuilder; import static org.junit.Assert.assertEquals; @@ -53,28 +56,31 @@ public void testRoundTrip() throws Exception { Event event = EventBuilder.withBody(message.getBytes(charset)); event.setHeaders(headers); - XContentBuilder expected = jsonBuilder().startObject(); - expected.field("@message", new String(message.getBytes(), charset)); - expected.field("@timestamp", new Date(timestamp)); - expected.field("@source", "flume_tail_src"); - expected.field("@type", "sometype"); - expected.field("@source_host", "test@localhost"); - expected.field("@source_path", "/tmp/test"); - expected.startObject("@fields"); - expected.field("timestamp", String.valueOf(timestamp)); - expected.field("src_path", "/tmp/test"); - expected.field("host", "test@localhost"); - expected.field("headerNameTwo", "headerValueTwo"); - expected.field("source", "flume_tail_src"); - expected.field("headerNameOne", "headerValueOne"); - expected.field("type", "sometype"); - expected.endObject(); - - expected.endObject(); + XContentBuilder expected = jsonBuilder() + .startObject(); + expected.field("@message", new String(message.getBytes(), charset)); + expected.field("@timestamp", new Date(timestamp)); + expected.field("@source", "flume_tail_src"); + expected.field("@type", "sometype"); + expected.field("@source_host", "test@localhost"); + expected.field("@source_path", "/tmp/test"); + + expected.startObject("@fields"); + expected.field("timestamp", String.valueOf(timestamp)); + expected.field("src_path", "/tmp/test"); + expected.field("host", "test@localhost"); + expected.field("headerNameTwo", "headerValueTwo"); + expected.field("source", "flume_tail_src"); + expected.field("headerNameOne", "headerValueOne"); + expected.field("type", "sometype"); + expected.endObject(); + + expected.endObject(); XContentBuilder actual = fixture.getContentBuilder(event); - assertEquals(new String(expected.bytes().array()), new String(actual - .bytes().array())); + + JsonParser parser = new JsonParser(); + assertEquals(parser.parse(expected.string()),parser.parse(actual.string())); } @Test @@ -96,27 +102,30 @@ public void shouldHandleInvalidJSONDuringComplexParsing() throws Exception { Event event = EventBuilder.withBody(message.getBytes(charset)); event.setHeaders(headers); - XContentBuilder expected = jsonBuilder().startObject(); - expected.field("@message", new String(message.getBytes(), charset)); - expected.field("@timestamp", new Date(timestamp)); - expected.field("@source", "flume_tail_src"); - expected.field("@type", "sometype"); - expected.field("@source_host", "test@localhost"); - expected.field("@source_path", "/tmp/test"); - expected.startObject("@fields"); - expected.field("timestamp", String.valueOf(timestamp)); - expected.field("src_path", "/tmp/test"); - expected.field("host", "test@localhost"); - expected.field("headerNameTwo", "headerValueTwo"); - expected.field("source", "flume_tail_src"); - expected.field("headerNameOne", "headerValueOne"); - expected.field("type", "sometype"); - expected.endObject(); - - expected.endObject(); + XContentBuilder expected = jsonBuilder(). + startObject(); + expected.field("@message", new String(message.getBytes(), charset)); + expected.field("@timestamp", new Date(timestamp)); + expected.field("@source", "flume_tail_src"); + expected.field("@type", "sometype"); + expected.field("@source_host", "test@localhost"); + expected.field("@source_path", "/tmp/test"); + + expected.startObject("@fields"); + expected.field("timestamp", String.valueOf(timestamp)); + expected.field("src_path", "/tmp/test"); + expected.field("host", "test@localhost"); + expected.field("headerNameTwo", "headerValueTwo"); + expected.field("source", "flume_tail_src"); + expected.field("headerNameOne", "headerValueOne"); + expected.field("type", "sometype"); + expected.endObject(); + + expected.endObject(); XContentBuilder actual = fixture.getContentBuilder(event); - assertEquals(new String(expected.bytes().array()), new String(actual - .bytes().array())); + + JsonParser parser = new JsonParser(); + assertEquals(parser.parse(expected.string()),parser.parse(actual.string())); } } From 61e3b3881b406b4a75897342fa12e84ecb2bcd3c Mon Sep 17 00:00:00 2001 From: Hari Shreedharan Date: Tue, 22 Jul 2014 20:00:48 -0700 Subject: [PATCH 124/341] FLUME-2431. Add simple regex search-and-replace interceptor (Mike Percy via Hari) --- .../main/java/org/apache/flume/Context.java | 9 ++ .../flume/interceptor/InterceptorType.java | 3 +- .../SearchAndReplaceInterceptor.java | 129 ++++++++++++++++++ .../TestSearchAndReplaceInterceptor.java | 87 ++++++++++++ flume-ng-doc/sphinx/FlumeUserGuide.rst | 39 ++++++ 5 files changed, 266 insertions(+), 1 deletion(-) create mode 100644 flume-ng-core/src/main/java/org/apache/flume/interceptor/SearchAndReplaceInterceptor.java create mode 100644 flume-ng-core/src/test/java/org/apache/flume/interceptor/TestSearchAndReplaceInterceptor.java diff --git a/flume-ng-configuration/src/main/java/org/apache/flume/Context.java b/flume-ng-configuration/src/main/java/org/apache/flume/Context.java index 927636c715..c0460d2199 100644 --- a/flume-ng-configuration/src/main/java/org/apache/flume/Context.java +++ b/flume-ng-configuration/src/main/java/org/apache/flume/Context.java @@ -112,6 +112,15 @@ public void putAll(Map map) { public void put(String key, String value) { parameters.put(key, value); } + + /** + * Returns true if this Context contains a mapping for key. + * Otherwise, returns false. + */ + public boolean containsKey(String key) { + return parameters.containsKey(key); + } + /** * Gets value mapped to key, returning defaultValue if unmapped. * @param key to be found diff --git a/flume-ng-core/src/main/java/org/apache/flume/interceptor/InterceptorType.java b/flume-ng-core/src/main/java/org/apache/flume/interceptor/InterceptorType.java index c84cea524d..fe341e9435 100644 --- a/flume-ng-core/src/main/java/org/apache/flume/interceptor/InterceptorType.java +++ b/flume-ng-core/src/main/java/org/apache/flume/interceptor/InterceptorType.java @@ -25,7 +25,8 @@ public enum InterceptorType { STATIC(org.apache.flume.interceptor.StaticInterceptor.Builder.class), REGEX_FILTER( org.apache.flume.interceptor.RegexFilteringInterceptor.Builder.class), - REGEX_EXTRACTOR(org.apache.flume.interceptor.RegexExtractorInterceptor.Builder.class); + REGEX_EXTRACTOR(org.apache.flume.interceptor.RegexExtractorInterceptor.Builder.class), + SEARCH_REPLACE(org.apache.flume.interceptor.SearchAndReplaceInterceptor.Builder.class); private final Class builderClass; diff --git a/flume-ng-core/src/main/java/org/apache/flume/interceptor/SearchAndReplaceInterceptor.java b/flume-ng-core/src/main/java/org/apache/flume/interceptor/SearchAndReplaceInterceptor.java new file mode 100644 index 0000000000..6f5c14687e --- /dev/null +++ b/flume-ng-core/src/main/java/org/apache/flume/interceptor/SearchAndReplaceInterceptor.java @@ -0,0 +1,129 @@ +/** + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package org.apache.flume.interceptor; + +import com.google.common.base.Charsets; +import com.google.common.base.Preconditions; +import org.apache.commons.lang.StringUtils; +import org.apache.flume.Context; +import org.apache.flume.Event; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +import java.nio.charset.Charset; +import java.util.List; +import java.util.regex.Matcher; +import java.util.regex.Pattern; + +/** + *

    + * Interceptor that allows search-and-replace of event body strings using + * regular expressions. This only works with event bodies that are valid + * strings. The charset is configurable. + *

    + * Usage: + *

    + *   agent.source-1.interceptors.search-replace.searchPattern = ^INFO:
    + *   agent.source-1.interceptors.search-replace.replaceString = Log msg:
    + * 
    + *

    + * Any regular expression search pattern and replacement pattern that can be + * used with {@link java.util.regex.Matcher#replaceAll(String)} may be used, + * including backtracking and grouping. + */ +public class SearchAndReplaceInterceptor implements Interceptor { + + private static final Logger logger = LoggerFactory + .getLogger(SearchAndReplaceInterceptor.class); + + private final Pattern searchPattern; + private final String replaceString; + private final Charset charset; + + private SearchAndReplaceInterceptor(Pattern searchPattern, + String replaceString, + Charset charset) { + this.searchPattern = searchPattern; + this.replaceString = replaceString; + this.charset = charset; + } + + @Override + public void initialize() { + } + + @Override + public void close() { + } + + @Override + public Event intercept(Event event) { + String origBody = new String(event.getBody(), charset); + Matcher matcher = searchPattern.matcher(origBody); + String newBody = matcher.replaceAll(replaceString); + event.setBody(newBody.getBytes(charset)); + return event; + } + + @Override + public List intercept(List events) { + for (Event event : events) { + intercept(event); + } + return events; + } + + public static class Builder implements Interceptor.Builder { + private static final String SEARCH_PAT_KEY = "searchPattern"; + private static final String REPLACE_STRING_KEY = "replaceString"; + private static final String CHARSET_KEY = "charset"; + + private Pattern searchRegex; + private String replaceString; + private Charset charset = Charsets.UTF_8; + + @Override + public void configure(Context context) { + String searchPattern = context.getString(SEARCH_PAT_KEY); + Preconditions.checkArgument(!StringUtils.isEmpty(searchPattern), + "Must supply a valid search pattern " + SEARCH_PAT_KEY + + " (may not be empty)"); + + replaceString = context.getString(REPLACE_STRING_KEY); + Preconditions.checkNotNull(replaceString, + "Must supply a replacement string " + REPLACE_STRING_KEY + + " (empty is ok)"); + + searchRegex = Pattern.compile(searchPattern); + + if (context.containsKey(CHARSET_KEY)) { + // May throw IllegalArgumentException for unsupported charsets. + charset = Charset.forName(context.getString(CHARSET_KEY)); + } + } + + @Override + public Interceptor build() { + Preconditions.checkNotNull(searchRegex, + "Regular expression search pattern required"); + Preconditions.checkNotNull(replaceString, + "Replacement string required"); + return new SearchAndReplaceInterceptor(searchRegex, replaceString, charset); + } + } +} diff --git a/flume-ng-core/src/test/java/org/apache/flume/interceptor/TestSearchAndReplaceInterceptor.java b/flume-ng-core/src/test/java/org/apache/flume/interceptor/TestSearchAndReplaceInterceptor.java new file mode 100644 index 0000000000..b39f91200c --- /dev/null +++ b/flume-ng-core/src/test/java/org/apache/flume/interceptor/TestSearchAndReplaceInterceptor.java @@ -0,0 +1,87 @@ +/** + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.flume.interceptor; + +import com.google.common.base.Charsets; +import org.apache.flume.Context; +import org.apache.flume.Event; +import org.apache.flume.event.EventBuilder; +import org.junit.Test; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +import static org.junit.Assert.assertEquals; + +public class TestSearchAndReplaceInterceptor { + + private static final Logger logger = + LoggerFactory.getLogger(TestSearchAndReplaceInterceptor.class); + + private void testSearchReplace(Context context, String input, String output) + throws Exception { + Interceptor.Builder builder = InterceptorBuilderFactory.newInstance( + InterceptorType.SEARCH_REPLACE.toString()); + builder.configure(context); + Interceptor interceptor = builder.build(); + + Event event = EventBuilder.withBody(input, Charsets.UTF_8); + event = interceptor.intercept(event); + String val = new String(event.getBody(), Charsets.UTF_8); + assertEquals(output, val); + logger.info(val); + } + + @Test + public void testRemovePrefix() throws Exception { + Context context = new Context(); + context.put("searchPattern", "^prefix"); + context.put("replaceString", ""); + testSearchReplace(context, "prefix non-prefix suffix", " non-prefix suffix"); + } + + @Test + public void testSyslogStripPriority() throws Exception { + final String input = "<13>Feb 5 17:32:18 10.0.0.99 Use the BFG!"; + final String output = "Feb 5 17:32:18 10.0.0.99 Use the BFG!"; + Context context = new Context(); + context.put("searchPattern", "^<[0-9]+>"); + context.put("replaceString", ""); + testSearchReplace(context, input, output); + } + + @Test + public void testCapturedGroups() throws Exception { + final String input = "The quick brown fox jumped over the lazy dog."; + final String output = "The hungry dog ate the careless fox."; + Context context = new Context(); + context.put("searchPattern", "The quick brown ([a-z]+) jumped over the lazy ([a-z]+)."); + context.put("replaceString", "The hungry $2 ate the careless $1."); + testSearchReplace(context, input, output); + } + + @Test + public void testRepeatedRemoval() throws Exception { + final String input = "Email addresses: test@test.com and foo@test.com"; + final String output = "Email addresses: REDACTED and REDACTED"; + Context context = new Context(); + context.put("searchPattern", "[A-Za-z0-9_.]+@[A-Za-z0-9_-]+\\.com"); + context.put("replaceString", "REDACTED"); + testSearchReplace(context, input, output); + } +} \ No newline at end of file diff --git a/flume-ng-doc/sphinx/FlumeUserGuide.rst b/flume-ng-doc/sphinx/FlumeUserGuide.rst index 1e98725c26..daf6e72ef9 100644 --- a/flume-ng-doc/sphinx/FlumeUserGuide.rst +++ b/flume-ng-doc/sphinx/FlumeUserGuide.rst @@ -2841,6 +2841,45 @@ Sample flume.conf file: a1.sources.avroSrc.interceptors.morphlineinterceptor.morphlineFile = /etc/flume-ng/conf/morphline.conf a1.sources.avroSrc.interceptors.morphlineinterceptor.morphlineId = morphline1 +Search and Replace Interceptor +~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~ + +This interceptor provides simple string-based search-and-replace functionality +based on Java regular expressions. Backtracking / group capture is also available. +This interceptor uses the same rules as in the Java Matcher.replaceAll() method. + +================ ======= ======================================================================== +Property Name Default Description +================ ======= ======================================================================== +**type** -- The component type name has to be ``search_replace`` +searchPattern -- The pattern to search for and replace. +replaceString -- The replacement string. +charset UTF-8 The charset of the event body. Assumed by default to be UTF-8. +================ ======= ======================================================================== + +Example configuration: + +.. code-block:: properties + + a1.sources.avroSrc.interceptors = search-replace + a1.sources.avroSrc.interceptors.search-replace.type = search_replace + + # Remove leading alphanumeric characters in an event body. + a1.sources.avroSrc.interceptors.search-replace.searchPattern = ^[A-Za-z0-9_]+ + a1.sources.avroSrc.interceptors.search-replace.replaceString = + +Another example: + +.. code-block:: properties + + a1.sources.avroSrc.interceptors = search-replace + a1.sources.avroSrc.interceptors.search-replace.type = search_replace + + # Use grouping operators to reorder and munge words on a line. + a1.sources.avroSrc.interceptors.search-replace.searchPattern = The quick brown ([a-z]+) jumped over the lazy ([a-z]+) + a1.sources.avroSrc.interceptors.search-replace.replaceString = The hungry $2 ate the careless $1 + + Regex Filtering Interceptor ~~~~~~~~~~~~~~~~~~~~~~~~~~~ From 49db6f59173bb9e9ae7ce671034d6562551cedf0 Mon Sep 17 00:00:00 2001 From: Hari Shreedharan Date: Mon, 28 Jul 2014 09:45:16 -0700 Subject: [PATCH 125/341] FLUME-2432. Update Kite dependency to 0.15.0. (Joey Echeverria via Hari) --- .../test/java/org/apache/flume/sink/kite/TestDatasetSink.java | 2 +- pom.xml | 4 ++-- 2 files changed, 3 insertions(+), 3 deletions(-) diff --git a/flume-ng-sinks/flume-dataset-sink/src/test/java/org/apache/flume/sink/kite/TestDatasetSink.java b/flume-ng-sinks/flume-dataset-sink/src/test/java/org/apache/flume/sink/kite/TestDatasetSink.java index ac275db5da..51dd4086c0 100644 --- a/flume-ng-sinks/flume-dataset-sink/src/test/java/org/apache/flume/sink/kite/TestDatasetSink.java +++ b/flume-ng-sinks/flume-dataset-sink/src/test/java/org/apache/flume/sink/kite/TestDatasetSink.java @@ -166,7 +166,7 @@ public void testFileStore() throws EventDeliveryException { public void testPartitionedData() throws EventDeliveryException { REPO.create("partitioned", new DatasetDescriptor.Builder(DESCRIPTOR) .partitionStrategy(new PartitionStrategy.Builder() - .identity("id", String.class, 10) // partition by id + .identity("id", 10) // partition by id .build()) .build()); diff --git a/pom.xml b/pom.xml index 541548f93c..4bdfcac54f 100644 --- a/pom.xml +++ b/pom.xml @@ -46,12 +46,12 @@ limitations under the License. file://people.apache.org/www/incubator.apache.org/flume ${project.basedir}/target/docs - 1.7.3 + 1.7.4 0.90.1 2.4.0 0.7.0 - 0.12.0 + 0.15.0 From 2ef2055d44986c4249bcf1e64aa39fbfc0c356df Mon Sep 17 00:00:00 2001 From: Hari Shreedharan Date: Tue, 5 Aug 2014 13:54:07 -0700 Subject: [PATCH 126/341] FLUME-2441. HTTP Source Unit tests fail on IBM JDK 7 (Li Xiang via Hari) --- .../java/org/apache/flume/source/http/TestHTTPSource.java | 8 +++++++- 1 file changed, 7 insertions(+), 1 deletion(-) diff --git a/flume-ng-core/src/test/java/org/apache/flume/source/http/TestHTTPSource.java b/flume-ng-core/src/test/java/org/apache/flume/source/http/TestHTTPSource.java index 5b07a6ea38..797eebb736 100644 --- a/flume-ng-core/src/test/java/org/apache/flume/source/http/TestHTTPSource.java +++ b/flume-ng-core/src/test/java/org/apache/flume/source/http/TestHTTPSource.java @@ -353,7 +353,13 @@ public java.security.cert.X509Certificate[] getAcceptedIssuers() { return null; } }}; - SSLContext sc = SSLContext.getInstance("SSL"); + + SSLContext sc = null; + if (System.getProperty("java.vendor").contains("IBM")) { + sc = SSLContext.getInstance("SSL_TLS"); + } else { + sc = SSLContext.getInstance("SSL"); + } HostnameVerifier hv = new HostnameVerifier() { public boolean verify(String arg0, SSLSession arg1) { From 0c5b87a5dfd53bcdcf8e2d8c570735573cd274ad Mon Sep 17 00:00:00 2001 From: Mike Percy Date: Wed, 6 Aug 2014 11:55:19 -0700 Subject: [PATCH 127/341] FLUME-1920. Remove Hadoop dependency from integration tests The file channel no longer actually depends on Hadoop, and this test was exhibiting failures when Flume is built from paths containing the string "hadoop". Since hadoop is not required for any of these tests, the straightforward thing to do was to remove the now unnecessary Hadoop JAR detection code completely. (Li Xiang via Mike Percy) --- .../flume/test/agent/TestFileChannel.java | 41 ------------------- .../apache/flume/test/util/StagedInstall.java | 9 ---- 2 files changed, 50 deletions(-) diff --git a/flume-ng-tests/src/test/java/org/apache/flume/test/agent/TestFileChannel.java b/flume-ng-tests/src/test/java/org/apache/flume/test/agent/TestFileChannel.java index 2feb506c07..4a80b8cca4 100644 --- a/flume-ng-tests/src/test/java/org/apache/flume/test/agent/TestFileChannel.java +++ b/flume-ng-tests/src/test/java/org/apache/flume/test/agent/TestFileChannel.java @@ -122,18 +122,10 @@ public void tearDown() throws Exception { public void testInOut() throws Exception { LOGGER.debug("testInOut() started."); - /* Find hadoop jar and append it to the flume agent's classpath */ - - String hadoopJarPath = findHadoopJar(); - Assert.assertNotNull("Hadoop jar not found in classpath.", - hadoopJarPath); - StagedInstall.getInstance().setAgentClasspath(hadoopJarPath); StagedInstall.getInstance().startAgent("a1", agentProps); TimeUnit.SECONDS.sleep(10); // Wait for source and sink to finish // TODO make this more deterministic - LOGGER.info("Started flume agent with hadoop in classpath"); - /* Create expected output */ StringBuffer sb = new StringBuffer(); @@ -161,37 +153,4 @@ public void testInOut() throws Exception { LOGGER.debug("testInOut() ended."); } - /** - * Search for and return the first path element found that includes hadoop. - * We search the class path of the current JVM process to grab the same - * hadoop jar that's depended on by the file channel. - * - * TODO Add all deps of hadoop jar to classpath - * - * @return path to the first hadoop jar found, null if not found - */ - private String findHadoopJar() { - //Grab classpath - String classpath = System.getProperty("java.class.path"); - String trimmedClasspath = classpath.trim(); - - //parse classpath into path elements - Iterable pathElements = Splitter.on(Pattern.compile("[;:]")) - .omitEmptyStrings() - .trimResults() - .split(trimmedClasspath); - - //find the first path element that includes the hadoop jar - for (String pathElement : pathElements) { - if (Pattern.compile("(?i)hadoop").matcher(pathElement).find()) { - return pathElement; - } - } - - LOGGER.error("Hadoop not found in classpath: |" + classpath - + "|"); - - return null; - } - } diff --git a/flume-ng-tests/src/test/java/org/apache/flume/test/util/StagedInstall.java b/flume-ng-tests/src/test/java/org/apache/flume/test/util/StagedInstall.java index 336ffc4768..a6bd5e9e64 100644 --- a/flume-ng-tests/src/test/java/org/apache/flume/test/util/StagedInstall.java +++ b/flume-ng-tests/src/test/java/org/apache/flume/test/util/StagedInstall.java @@ -70,7 +70,6 @@ public class StagedInstall { private Process process; private ProcessShutdownHook shutdownHook; private ProcessInputStreamConsumer consumer; - private String agentClasspath; private static StagedInstall INSTANCE; @@ -96,7 +95,6 @@ public synchronized void stopAgent() throws Exception { consumer.interrupt(); consumer = null; configFilePath = null; - agentClasspath = null; Runtime.getRuntime().removeShutdownHook(shutdownHook); shutdownHook = null; @@ -141,9 +139,6 @@ public synchronized void startAgent(String name, Properties properties) builder.add(launchScriptPath); builder.add("agent"); builder.add("--conf", confDirPath); - if (agentClasspath != null) { - builder.add("--classpath", agentClasspath); - } builder.add("--conf-file", configFilePath); builder.add("--name", agentName); builder.add("-D" + ENV_FLUME_LOG_DIR + "=" + logDirPath); @@ -173,10 +168,6 @@ public synchronized void startAgent(String name, Properties properties) Thread.sleep(3000); // sleep for 3s to let system initialize } - public synchronized void setAgentClasspath(String agentClasspath) { - this.agentClasspath = agentClasspath; - } - public synchronized void reconfigure(Properties properties) throws Exception { File configFile = createConfigurationFile(agentName, properties); Files.copy(configFile, new File(configFilePath)); From 75418384440fdd566649875f0c6cd61f6b841a5a Mon Sep 17 00:00:00 2001 From: Hari Shreedharan Date: Tue, 12 Aug 2014 16:13:24 -0700 Subject: [PATCH 128/341] FLUME-2439. Upgrade Dataset sink to use Kite 1.5.0 (Ryan Blue via Hari) --- flume-ng-doc/sphinx/FlumeUserGuide.rst | 22 +++-- .../apache/flume/sink/kite/DatasetSink.java | 82 +++++++++++-------- .../flume/sink/kite/DatasetSinkConstants.java | 5 ++ .../flume/sink/kite/TestDatasetSink.java | 82 +++++++++++++++++-- 4 files changed, 137 insertions(+), 54 deletions(-) diff --git a/flume-ng-doc/sphinx/FlumeUserGuide.rst b/flume-ng-doc/sphinx/FlumeUserGuide.rst index daf6e72ef9..ed90022ec2 100644 --- a/flume-ng-doc/sphinx/FlumeUserGuide.rst +++ b/flume-ng-doc/sphinx/FlumeUserGuide.rst @@ -2045,18 +2045,13 @@ Example for agent named a1: a1.sinks.k1.serializer = org.apache.flume.sink.elasticsearch.ElasticSearchDynamicSerializer a1.sinks.k1.channel = c1 -Kite Dataset Sink (experimental) -~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~ - -.. warning:: - This source is experimental and may change between minor versions of Flume. - Use at your own risk. +Kite Dataset Sink +~~~~~~~~~~~~~~~~~ -Experimental sink that writes events to a `Kite Dataset `_. +Experimental sink that writes events to a `Kite Dataset `_. This sink will deserialize the body of each incoming event and store the -resulting record in a Kite Dataset. It determines target Dataset by opening a -repository URI, ``kite.repo.uri``, and loading a Dataset by name, -``kite.dataset.name``. +resulting record in a Kite Dataset. It determines target Dataset by loading a +dataset by URI. The only supported serialization is avro, and the record schema must be passed in the event headers, using either ``flume.avro.schema.literal`` with the JSON @@ -2075,8 +2070,11 @@ Property Name Default Description ======================= ======= =========================================================== **channel** -- **type** -- Must be org.apache.flume.sink.kite.DatasetSink -**kite.repo.uri** -- URI of the repository to open -**kite.dataset.name** -- Name of the Dataset where records will be written +**kite.dataset.uri** -- URI of the dataset to open +kite.repo.uri -- URI of the repository to open + (deprecated; use kite.dataset.uri instead) +kite.dataset.name -- Name of the Dataset where records will be written + (deprecated; use kite.dataset.uri instead) kite.batchSize 100 Number of records to process in each batch kite.rollInterval 30 Maximum wait time (seconds) before data files are released auth.kerberosPrincipal -- Kerberos user principal for secure authentication to HDFS diff --git a/flume-ng-sinks/flume-dataset-sink/src/main/java/org/apache/flume/sink/kite/DatasetSink.java b/flume-ng-sinks/flume-dataset-sink/src/main/java/org/apache/flume/sink/kite/DatasetSink.java index ed1b8d06e3..8f3ae519c5 100644 --- a/flume-ng-sinks/flume-dataset-sink/src/main/java/org/apache/flume/sink/kite/DatasetSink.java +++ b/flume-ng-sinks/flume-dataset-sink/src/main/java/org/apache/flume/sink/kite/DatasetSink.java @@ -34,9 +34,11 @@ import java.util.Map; import java.util.concurrent.ExecutionException; import org.apache.avro.Schema; +import org.apache.avro.generic.GenericDatumReader; +import org.apache.avro.generic.GenericRecord; import org.apache.avro.io.BinaryDecoder; +import org.apache.avro.io.DatumReader; import org.apache.avro.io.DecoderFactory; -import org.apache.avro.reflect.ReflectDatumReader; import org.apache.flume.Channel; import org.apache.flume.Context; import org.apache.flume.Event; @@ -52,6 +54,7 @@ import org.kitesdk.data.Dataset; import org.kitesdk.data.DatasetRepositories; import org.kitesdk.data.DatasetWriter; +import org.kitesdk.data.Datasets; import org.slf4j.Logger; import org.slf4j.LoggerFactory; @@ -68,12 +71,13 @@ public class DatasetSink extends AbstractSink implements Configurable { static Configuration conf = new Configuration(); + private String datasetURI = null; private String repositoryURI = null; private String datasetName = null; private long batchSize = DatasetSinkConstants.DEFAULT_BATCH_SIZE; - private Dataset targetDataset = null; - private DatasetWriter writer = null; + private Dataset targetDataset = null; + private DatasetWriter writer = null; private UserGroupInformation login = null; private SinkCounter counter = null; @@ -82,16 +86,18 @@ public class DatasetSink extends AbstractSink implements Configurable { private long lastRolledMs = 0l; // for working with avro serialized records - private Object datum = null; + private GenericRecord datum = null; + // TODO: remove this after PARQUET-62 is released + private boolean reuseDatum = true; private BinaryDecoder decoder = null; - private LoadingCache> readers = + private LoadingCache> readers = CacheBuilder.newBuilder() - .build(new CacheLoader>() { + .build(new CacheLoader>() { @Override - public ReflectDatumReader load(Schema schema) { + public DatumReader load(Schema schema) { // must use the target dataset's schema for reading to ensure the // records are able to be stored using it - return new ReflectDatumReader( + return new GenericDatumReader( schema, targetDataset.getDescriptor().getSchema()); } }); @@ -129,7 +135,7 @@ public Schema load(String url) throws IOException { }); protected List allowedFormats() { - return Lists.newArrayList("avro"); + return Lists.newArrayList("avro", "parquet"); } @Override @@ -144,25 +150,39 @@ public void configure(Context context) { this.login = KerberosUtil.proxyAs(effectiveUser, login); } - this.repositoryURI = context.getString( - DatasetSinkConstants.CONFIG_KITE_REPO_URI); - Preconditions.checkNotNull(repositoryURI, "Repository URI is missing"); - this.datasetName = context.getString( - DatasetSinkConstants.CONFIG_KITE_DATASET_NAME); - Preconditions.checkNotNull(datasetName, "Dataset name is missing"); - - this.targetDataset = KerberosUtil.runPrivileged(login, - new PrivilegedExceptionAction>() { - @Override - public Dataset run() { - return DatasetRepositories.open(repositoryURI).load(datasetName); - } - }); + this.datasetURI = context.getString( + DatasetSinkConstants.CONFIG_KITE_DATASET_URI); + if (datasetURI != null) { + this.targetDataset = KerberosUtil.runPrivileged(login, + new PrivilegedExceptionAction>() { + @Override + public Dataset run() { + return Datasets.load(datasetURI); + } + }); + } else { + this.repositoryURI = context.getString( + DatasetSinkConstants.CONFIG_KITE_REPO_URI); + Preconditions.checkNotNull(repositoryURI, "Repository URI is missing"); + this.datasetName = context.getString( + DatasetSinkConstants.CONFIG_KITE_DATASET_NAME); + Preconditions.checkNotNull(datasetName, "Dataset name is missing"); + + this.targetDataset = KerberosUtil.runPrivileged(login, + new PrivilegedExceptionAction>() { + @Override + public Dataset run() { + return DatasetRepositories.open(repositoryURI).load(datasetName); + } + }); + } String formatName = targetDataset.getDescriptor().getFormat().getName(); Preconditions.checkArgument(allowedFormats().contains(formatName), "Unsupported format: " + formatName); + this.reuseDatum = !("parquet".equals(formatName)); + // other configuration this.batchSize = context.getLong( DatasetSinkConstants.CONFIG_KITE_BATCH_SIZE, @@ -176,7 +196,7 @@ public Dataset run() { @Override public synchronized void start() { - this.writer = openWriter(targetDataset); + this.writer = targetDataset.newWriter(); this.lastRolledMs = System.currentTimeMillis(); counter.start(); // signal that this sink is ready to process @@ -219,7 +239,7 @@ public Status process() throws EventDeliveryException { if ((System.currentTimeMillis() - lastRolledMs) / 1000 > rollIntervalS) { // close the current writer and get a new one writer.close(); - this.writer = openWriter(targetDataset); + this.writer = targetDataset.newWriter(); this.lastRolledMs = System.currentTimeMillis(); LOG.info("Rolled writer for dataset: " + datasetName); } @@ -238,7 +258,7 @@ public Status process() throws EventDeliveryException { break; } - this.datum = deserialize(event, datum); + this.datum = deserialize(event, reuseDatum ? datum : null); // writeEncoded would be an optimization in some cases, but HBase // will not support it and partitioned Datasets need to get partition @@ -302,11 +322,11 @@ public Status process() throws EventDeliveryException { * @param reuse * @return */ - private Object deserialize(Event event, Object reuse) + private GenericRecord deserialize(Event event, GenericRecord reuse) throws EventDeliveryException { decoder = DecoderFactory.get().binaryDecoder(event.getBody(), decoder); // no checked exception is thrown in the CacheLoader - ReflectDatumReader reader = readers.getUnchecked(schema(event)); + DatumReader reader = readers.getUnchecked(schema(event)); try { return reader.read(reuse, decoder); } catch (IOException ex) { @@ -330,10 +350,4 @@ private static Schema schema(Event event) throws EventDeliveryException { } } - private static DatasetWriter openWriter(Dataset target) { - DatasetWriter writer = target.newWriter(); - writer.open(); - return writer; - } - } diff --git a/flume-ng-sinks/flume-dataset-sink/src/main/java/org/apache/flume/sink/kite/DatasetSinkConstants.java b/flume-ng-sinks/flume-dataset-sink/src/main/java/org/apache/flume/sink/kite/DatasetSinkConstants.java index 09dfab639d..3c67738a49 100644 --- a/flume-ng-sinks/flume-dataset-sink/src/main/java/org/apache/flume/sink/kite/DatasetSinkConstants.java +++ b/flume-ng-sinks/flume-dataset-sink/src/main/java/org/apache/flume/sink/kite/DatasetSinkConstants.java @@ -19,6 +19,11 @@ package org.apache.flume.sink.kite; public class DatasetSinkConstants { + /** + * URI of the Kite DatasetRepository. + */ + public static final String CONFIG_KITE_DATASET_URI = "kite.dataset.uri"; + /** * URI of the Kite DatasetRepository. */ diff --git a/flume-ng-sinks/flume-dataset-sink/src/test/java/org/apache/flume/sink/kite/TestDatasetSink.java b/flume-ng-sinks/flume-dataset-sink/src/test/java/org/apache/flume/sink/kite/TestDatasetSink.java index 51dd4086c0..b448b5018d 100644 --- a/flume-ng-sinks/flume-dataset-sink/src/test/java/org/apache/flume/sink/kite/TestDatasetSink.java +++ b/flume-ng-sinks/flume-dataset-sink/src/test/java/org/apache/flume/sink/kite/TestDatasetSink.java @@ -36,6 +36,7 @@ import javax.annotation.Nullable; import org.apache.avro.Schema; import org.apache.avro.generic.GenericData; +import org.apache.avro.generic.GenericRecord; import org.apache.avro.generic.GenericRecordBuilder; import org.apache.avro.io.Encoder; import org.apache.avro.io.EncoderFactory; @@ -63,12 +64,15 @@ import org.kitesdk.data.DatasetReader; import org.kitesdk.data.DatasetRepositories; import org.kitesdk.data.DatasetRepository; +import org.kitesdk.data.Datasets; import org.kitesdk.data.PartitionStrategy; public class TestDatasetSink { public static final String FILE_REPO_URI = "repo:file:target/test-repo"; public static final String DATASET_NAME = "test"; + public static final String FILE_DATASET_URI = + "dataset:file:target/test-repo/" + DATASET_NAME; public static final DatasetRepository REPO = DatasetRepositories .open(FILE_REPO_URI); public static final File SCHEMA_FILE = new File("target/record-schema.avsc"); @@ -114,15 +118,15 @@ public static void tearDownClass() { @Before public void setup() throws EventDeliveryException { + REPO.delete(DATASET_NAME); REPO.create(DATASET_NAME, DESCRIPTOR); this.config = new Context(); - config.put(DatasetSinkConstants.CONFIG_KITE_REPO_URI, FILE_REPO_URI); - config.put(DatasetSinkConstants.CONFIG_KITE_DATASET_NAME, DATASET_NAME); - this.in = new MemoryChannel(); Configurables.configure(in, config); + config.put(DatasetSinkConstants.CONFIG_KITE_DATASET_URI, FILE_DATASET_URI); + GenericRecordBuilder builder = new GenericRecordBuilder(RECORD_SCHEMA); expected = Lists.newArrayList( builder.set("id", "1").set("msg", "msg1").build(), @@ -147,6 +151,44 @@ public void teardown() { REPO.delete(DATASET_NAME); } + @Test + public void testOldConfig() throws EventDeliveryException { + config.put(DatasetSinkConstants.CONFIG_KITE_DATASET_URI, null); + config.put(DatasetSinkConstants.CONFIG_KITE_REPO_URI, FILE_REPO_URI); + config.put(DatasetSinkConstants.CONFIG_KITE_DATASET_NAME, DATASET_NAME); + + DatasetSink sink = sink(in, config); + + // run the sink + sink.start(); + sink.process(); + sink.stop(); + + Assert.assertEquals( + Sets.newHashSet(expected), + read(REPO.load(DATASET_NAME))); + Assert.assertEquals("Should have committed", 0, remaining(in)); + } + + @Test + public void testDatasetUriOverridesOldConfig() throws EventDeliveryException { + // CONFIG_KITE_DATASET_URI is still set, otherwise this will cause an error + config.put(DatasetSinkConstants.CONFIG_KITE_REPO_URI, "bad uri"); + config.put(DatasetSinkConstants.CONFIG_KITE_DATASET_NAME, ""); + + DatasetSink sink = sink(in, config); + + // run the sink + sink.start(); + sink.process(); + sink.stop(); + + Assert.assertEquals( + Sets.newHashSet(expected), + read(REPO.load(DATASET_NAME))); + Assert.assertEquals("Should have committed", 0, remaining(in)); + } + @Test public void testFileStore() throws EventDeliveryException { DatasetSink sink = sink(in, config); @@ -162,6 +204,26 @@ public void testFileStore() throws EventDeliveryException { Assert.assertEquals("Should have committed", 0, remaining(in)); } + @Test + public void testParquetDataset() throws EventDeliveryException { + Datasets.delete(FILE_DATASET_URI); + Dataset created = Datasets.create(FILE_DATASET_URI, + new DatasetDescriptor.Builder(DESCRIPTOR) + .format("parquet") + .build(), + GenericData.Record.class); + + DatasetSink sink = sink(in, config); + + // run the sink + sink.start(); + sink.process(); + sink.stop(); + + Assert.assertEquals(Sets.newHashSet(expected), read(created)); + Assert.assertEquals("Should have committed", 0, remaining(in)); + } + @Test public void testPartitionedData() throws EventDeliveryException { REPO.create("partitioned", new DatasetDescriptor.Builder(DESCRIPTOR) @@ -171,7 +233,8 @@ public void testPartitionedData() throws EventDeliveryException { .build()); try { - config.put(DatasetSinkConstants.CONFIG_KITE_DATASET_NAME, "partitioned"); + config.put(DatasetSinkConstants.CONFIG_KITE_DATASET_URI, + "dataset:file:target/test-repo/partitioned"); DatasetSink sink = sink(in, config); // run the sink @@ -208,7 +271,8 @@ public void testMiniClusterStore() hdfsRepo.create(DATASET_NAME, DESCRIPTOR); // update the config to use the HDFS repository - config.put(DatasetSinkConstants.CONFIG_KITE_REPO_URI, repoURI); + config.put(DatasetSinkConstants.CONFIG_KITE_DATASET_URI, + "dataset:" + conf.get("fs.defaultFS") + "/tmp/repo/" + DATASET_NAME); DatasetSink sink = sink(in, config); @@ -367,12 +431,14 @@ public static DatasetSink sink(Channel in, Context config) { } public static HashSet read(Dataset dataset) { - DatasetReader reader = dataset.newReader(); + DatasetReader reader = null; try { - reader.open(); + reader = dataset.newReader(); return Sets.newHashSet(reader.iterator()); } finally { - reader.close(); + if (reader != null) { + reader.close(); + } } } From e3b9b09cca015728a16e1368b26188c10a35df09 Mon Sep 17 00:00:00 2001 From: Mike Percy Date: Thu, 14 Aug 2014 16:42:12 -0400 Subject: [PATCH 129/341] FLUME-2438. Make Syslog source message body configurable Allow for specifying particular fields to keep in the body of the event or drop, representing them only in Flume event headers. (Abraham Elmahrek via Mike Percy) --- .../source/MultiportSyslogTCPSource.java | 15 +- .../org/apache/flume/source/SyslogParser.java | 27 +- .../SyslogSourceConfigurationConstants.java | 7 +- .../apache/flume/source/SyslogTcpSource.java | 11 +- .../apache/flume/source/SyslogUDPSource.java | 15 +- .../org/apache/flume/source/SyslogUtils.java | 126 ++++++++-- .../source/TestMultiportSyslogTCPSource.java | 8 +- .../apache/flume/source/TestSyslogParser.java | 19 +- .../flume/source/TestSyslogTcpSource.java | 42 +++- .../flume/source/TestSyslogUdpSource.java | 46 +++- .../apache/flume/source/TestSyslogUtils.java | 50 +++- flume-ng-doc/sphinx/FlumeUserGuide.rst | 14 +- .../flume/test/agent/TestSyslogSource.java | 97 +++++++ .../apache/flume/test/util/StagedInstall.java | 2 + .../apache/flume/test/util/SyslogAgent.java | 238 ++++++++++++++++++ 15 files changed, 644 insertions(+), 73 deletions(-) create mode 100644 flume-ng-tests/src/test/java/org/apache/flume/test/agent/TestSyslogSource.java create mode 100644 flume-ng-tests/src/test/java/org/apache/flume/test/util/SyslogAgent.java diff --git a/flume-ng-core/src/main/java/org/apache/flume/source/MultiportSyslogTCPSource.java b/flume-ng-core/src/main/java/org/apache/flume/source/MultiportSyslogTCPSource.java index 427e0e3748..87f0db1609 100644 --- a/flume-ng-core/src/main/java/org/apache/flume/source/MultiportSyslogTCPSource.java +++ b/flume-ng-core/src/main/java/org/apache/flume/source/MultiportSyslogTCPSource.java @@ -28,6 +28,7 @@ import java.nio.charset.CharsetDecoder; import java.util.List; import java.util.Map; +import java.util.Set; import java.util.concurrent.ConcurrentHashMap; import java.util.concurrent.ConcurrentMap; import org.apache.flume.Context; @@ -67,7 +68,7 @@ public class MultiportSyslogTCPSource extends AbstractSource implements private SourceCounter sourceCounter = null; private Charset defaultCharset; private ThreadSafeDecoder defaultDecoder; - private boolean keepFields; + private Set keepFields; public MultiportSyslogTCPSource() { portCharsets = new ConcurrentHashMap(); @@ -139,9 +140,10 @@ public void configure(Context context) { SyslogSourceConfigurationConstants.CONFIG_READBUF_SIZE, SyslogSourceConfigurationConstants.DEFAULT_READBUF_SIZE); - keepFields = context.getBoolean( - SyslogSourceConfigurationConstants.CONFIG_KEEP_FIELDS, - SyslogSourceConfigurationConstants.DEFAULT_KEEP_FIELDS); + keepFields = SyslogUtils.chooseFieldsToKeep( + context.getString( + SyslogSourceConfigurationConstants.CONFIG_KEEP_FIELDS, + SyslogSourceConfigurationConstants.DEFAULT_KEEP_FIELDS)); if (sourceCounter == null) { sourceCounter = new SourceCounter(getName()); @@ -218,12 +220,13 @@ static class MultiportSyslogHandler extends IoHandlerAdapter { private final LineSplitter lineSplitter; private final ThreadSafeDecoder defaultDecoder; private final ConcurrentMap portCharsets; - private final boolean keepFields; + private Set keepFields; public MultiportSyslogHandler(int maxEventSize, int batchSize, ChannelProcessor cp, SourceCounter ctr, String portHeader, ThreadSafeDecoder defaultDecoder, - ConcurrentMap portCharsets, boolean keepFields) { + ConcurrentMap portCharsets, + Set keepFields) { channelProcessor = cp; sourceCounter = ctr; this.maxEventSize = maxEventSize; diff --git a/flume-ng-core/src/main/java/org/apache/flume/source/SyslogParser.java b/flume-ng-core/src/main/java/org/apache/flume/source/SyslogParser.java index 557d1213ef..01713096c2 100644 --- a/flume-ng-core/src/main/java/org/apache/flume/source/SyslogParser.java +++ b/flume-ng-core/src/main/java/org/apache/flume/source/SyslogParser.java @@ -30,6 +30,7 @@ import com.google.common.collect.Maps; import java.nio.charset.Charset; import java.util.Map; +import java.util.Set; import java.util.concurrent.ExecutionException; import java.util.regex.Pattern; import org.apache.flume.Event; @@ -79,7 +80,7 @@ public Long load(String key) throws Exception { * @return Parsed Flume Event * @throws IllegalArgumentException if unable to successfully parse message */ - public Event parseMessage(String msg, Charset charset, boolean keepFields) { + public Event parseMessage(String msg, Charset charset, Set keepFields) { Map headers = Maps.newHashMap(); int msgLen = msg.length(); @@ -98,6 +99,9 @@ public Event parseMessage(String msg, Charset charset, boolean keepFields) { int facility = pri / 8; int severity = pri % 8; + // Remember priority + headers.put(SyslogUtils.SYSLOG_PRIORITY, priority); + // put fac / sev into header headers.put(SyslogUtils.SYSLOG_FACILITY, String.valueOf(facility)); headers.put(SyslogUtils.SYSLOG_SEVERITY, String.valueOf(severity)); @@ -108,20 +112,25 @@ public Event parseMessage(String msg, Charset charset, boolean keepFields) { // update parsing position curPos = endBracketPos + 1; - // ignore version string + // remember version string + String version = null; if (msgLen > curPos + 2 && "1 ".equals(msg.substring(curPos, curPos + 2))) { + version = msg.substring(curPos, curPos+1); + headers.put(SyslogUtils.SYSLOG_VERSION, version); curPos += 2; } // now parse timestamp (handle different varieties) long ts; + String tsString; char dateStartChar = msg.charAt(curPos); try { // no timestamp specified; use relay current time if (dateStartChar == '-') { + tsString = Character.toString(dateStartChar); ts = System.currentTimeMillis(); if (msgLen <= curPos + 2) { throw new IllegalArgumentException( @@ -129,22 +138,23 @@ public Event parseMessage(String msg, Charset charset, boolean keepFields) { } curPos += 2; // assume we skip past a space to get to the hostname - // rfc3164 imestamp + // rfc3164 timestamp } else if (dateStartChar >= 'A' && dateStartChar <= 'Z') { if (msgLen <= curPos + RFC3164_LEN) { throw new IllegalArgumentException("bad timestamp format"); } - ts = parseRfc3164Time( - msg.substring(curPos, curPos + RFC3164_LEN)); + tsString = msg.substring(curPos, curPos + RFC3164_LEN); + ts = parseRfc3164Time(tsString); curPos += RFC3164_LEN + 1; - // rfc 5424 timestamp + // rfc 5424 timestamp } else { int nextSpace = msg.indexOf(' ', curPos); if (nextSpace == -1) { throw new IllegalArgumentException("bad timestamp format"); } - ts = parseRfc5424Date(msg.substring(curPos, nextSpace)); + tsString = msg.substring(curPos, nextSpace); + ts = parseRfc5424Date(tsString); curPos = nextSpace + 1; } @@ -167,9 +177,10 @@ public Event parseMessage(String msg, Charset charset, boolean keepFields) { // EventBuilder will do a copy of its own, so no defensive copy of the body String data = ""; - if (msgLen > nextSpace + 1 && !keepFields) { + if (msgLen > nextSpace + 1 && !SyslogUtils.keepAllFields(keepFields)) { curPos = nextSpace + 1; data = msg.substring(curPos); + data = SyslogUtils.addFieldsToBody(keepFields, data, priority, version, tsString, hostname); } else { data = msg; } diff --git a/flume-ng-core/src/main/java/org/apache/flume/source/SyslogSourceConfigurationConstants.java b/flume-ng-core/src/main/java/org/apache/flume/source/SyslogSourceConfigurationConstants.java index 985949cffc..fb8df81f3f 100644 --- a/flume-ng-core/src/main/java/org/apache/flume/source/SyslogSourceConfigurationConstants.java +++ b/flume-ng-core/src/main/java/org/apache/flume/source/SyslogSourceConfigurationConstants.java @@ -67,7 +67,12 @@ public final class SyslogSourceConfigurationConstants { public static final int DEFAULT_READBUF_SIZE = 1024; public static final String CONFIG_KEEP_FIELDS = "keepFields"; - public static final boolean DEFAULT_KEEP_FIELDS = false; + public static final String DEFAULT_KEEP_FIELDS = "none"; + + public static final String CONFIG_KEEP_FIELDS_PRIORITY = "priority"; + public static final String CONFIG_KEEP_FIELDS_VERSION = "version"; + public static final String CONFIG_KEEP_FIELDS_TIMESTAMP = "timestamp"; + public static final String CONFIG_KEEP_FIELDS_HOSTNAME = "hostname"; private SyslogSourceConfigurationConstants() { // Disable explicit creation of objects. diff --git a/flume-ng-core/src/main/java/org/apache/flume/source/SyslogTcpSource.java b/flume-ng-core/src/main/java/org/apache/flume/source/SyslogTcpSource.java index e84e4b63d5..c117813925 100644 --- a/flume-ng-core/src/main/java/org/apache/flume/source/SyslogTcpSource.java +++ b/flume-ng-core/src/main/java/org/apache/flume/source/SyslogTcpSource.java @@ -21,6 +21,7 @@ import java.net.InetSocketAddress; import java.net.SocketAddress; import java.util.Map; +import java.util.Set; import java.util.concurrent.Executors; import java.util.concurrent.TimeUnit; @@ -58,7 +59,7 @@ public class SyslogTcpSource extends AbstractSource private Integer eventSize; private Map formaterProp; private CounterGroup counterGroup = new CounterGroup(); - private Boolean keepFields; + private Set keepFields; public class syslogTcpHandler extends SimpleChannelHandler { @@ -68,7 +69,7 @@ public void setEventSize(int eventSize){ syslogUtils.setEventSize(eventSize); } - public void setKeepFields(boolean keepFields){ + public void setKeepFields(Set keepFields) { syslogUtils.setKeepFields(keepFields); } @@ -154,8 +155,10 @@ public void configure(Context context) { eventSize = context.getInteger("eventSize", SyslogUtils.DEFAULT_SIZE); formaterProp = context.getSubProperties( SyslogSourceConfigurationConstants.CONFIG_FORMAT_PREFIX); - keepFields = context.getBoolean - (SyslogSourceConfigurationConstants.CONFIG_KEEP_FIELDS, false); + keepFields = SyslogUtils.chooseFieldsToKeep( + context.getString( + SyslogSourceConfigurationConstants.CONFIG_KEEP_FIELDS, + SyslogSourceConfigurationConstants.DEFAULT_KEEP_FIELDS)); } @VisibleForTesting diff --git a/flume-ng-core/src/main/java/org/apache/flume/source/SyslogUDPSource.java b/flume-ng-core/src/main/java/org/apache/flume/source/SyslogUDPSource.java index 01b8905513..378d48418f 100644 --- a/flume-ng-core/src/main/java/org/apache/flume/source/SyslogUDPSource.java +++ b/flume-ng-core/src/main/java/org/apache/flume/source/SyslogUDPSource.java @@ -21,6 +21,7 @@ import java.net.InetSocketAddress; import java.net.SocketAddress; import java.util.Map; +import java.util.Set; import java.util.concurrent.Executors; import java.util.concurrent.TimeUnit; @@ -32,7 +33,6 @@ import org.apache.flume.EventDrivenSource; import org.apache.flume.conf.Configurable; import org.apache.flume.conf.Configurables; -import org.apache.flume.source.SyslogUtils; import org.jboss.netty.bootstrap.ConnectionlessBootstrap; import org.jboss.netty.buffer.ChannelBuffer; import org.jboss.netty.channel.*; @@ -49,7 +49,7 @@ public class SyslogUDPSource extends AbstractSource private String host = null; private Channel nettyChannel; private Map formaterProp; - private boolean keepFields; + private Set keepFields; private static final Logger logger = LoggerFactory .getLogger(SyslogUDPSource.class); @@ -61,14 +61,13 @@ public class SyslogUDPSource extends AbstractSource public static final int DEFAULT_INITIAL_SIZE = DEFAULT_MIN_SIZE; public class syslogHandler extends SimpleChannelHandler { - private SyslogUtils syslogUtils = new SyslogUtils(DEFAULT_INITIAL_SIZE, - SyslogSourceConfigurationConstants.DEFAULT_KEEP_FIELDS, true); + private SyslogUtils syslogUtils = new SyslogUtils(DEFAULT_INITIAL_SIZE, null, true); public void setFormater(Map prop) { syslogUtils.addFormats(prop); } - public void setKeepFields(boolean keepFields) { + public void setKeepFields(Set keepFields) { syslogUtils.setKeepFields(keepFields); } @@ -143,8 +142,10 @@ public void configure(Context context) { host = context.getString(SyslogSourceConfigurationConstants.CONFIG_HOST); formaterProp = context.getSubProperties( SyslogSourceConfigurationConstants.CONFIG_FORMAT_PREFIX); - keepFields = context.getBoolean(SyslogSourceConfigurationConstants.CONFIG_KEEP_FIELDS, - SyslogSourceConfigurationConstants.DEFAULT_KEEP_FIELDS); + keepFields = SyslogUtils.chooseFieldsToKeep( + context.getString( + SyslogSourceConfigurationConstants.CONFIG_KEEP_FIELDS, + SyslogSourceConfigurationConstants.DEFAULT_KEEP_FIELDS)); } @VisibleForTesting diff --git a/flume-ng-core/src/main/java/org/apache/flume/source/SyslogUtils.java b/flume-ng-core/src/main/java/org/apache/flume/source/SyslogUtils.java index a77bfc99cd..208fefe966 100644 --- a/flume-ng-core/src/main/java/org/apache/flume/source/SyslogUtils.java +++ b/flume-ng-core/src/main/java/org/apache/flume/source/SyslogUtils.java @@ -31,9 +31,13 @@ import java.text.ParseException; import java.text.SimpleDateFormat; import java.util.ArrayList; +import java.util.Arrays; import java.util.Calendar; import java.util.HashMap; +import java.util.HashSet; +import java.util.List; import java.util.Map; +import java.util.Set; import java.util.regex.MatchResult; import java.util.regex.Matcher; import java.util.regex.Pattern; @@ -48,27 +52,31 @@ public class SyslogUtils { final public static String SYSLOG_TIMESTAMP_FORMAT_RFC3164_1 = "yyyyMMM d HH:mm:ss"; final public static String SYSLOG_MSG_RFC5424_0 = - "(?:\\<\\d{1,3}\\>\\d?\\s?)" + // priority + "(?:\\<(\\d{1,3})\\>)" + // priority + "(?:(\\d?)\\s?)" + // version /* yyyy-MM-dd'T'HH:mm:ss.SZ or yyyy-MM-dd'T'HH:mm:ss.S+hh:mm or - (null stamp) */ "(?:" + "(\\d{4}[-]\\d{2}[-]\\d{2}[T]\\d{2}[:]\\d{2}[:]\\d{2}" + "(?:\\.\\d{1,6})?(?:[+-]\\d{2}[:]\\d{2}|Z)?)|-)" + // stamp "\\s" + // separator - "(?:([\\w][\\w\\d\\.@-]*)|-)" + // host name or - (null) + "(?:([\\w][\\w\\d\\.@\\-]*)|-)" + // host name or - (null) "\\s" + // separator "(.*)$"; // body final public static String SYSLOG_MSG_RFC3164_0 = - "(?:\\<\\d{1,3}\\>\\d?\\s?)" + + "(?:\\<(\\d{1,3})\\>)" + + "(?:(\\d)?\\s?)" + // version // stamp MMM d HH:mm:ss, single digit date has two spaces "([A-Z][a-z][a-z]\\s{1,2}\\d{1,2}\\s\\d{2}[:]\\d{2}[:]\\d{2})" + "\\s" + // separator "([\\w][\\w\\d\\.@-]*)" + // host "\\s(.*)$"; // body - final public static int SYSLOG_TIMESTAMP_POS = 1; - final public static int SYSLOG_HOSTNAME_POS = 2; - final public static int SYSLOG_BODY_POS = 3; + final public static int SYSLOG_PRIORITY_POS = 1; + final public static int SYSLOG_VERSION_POS = 2; + final public static int SYSLOG_TIMESTAMP_POS = 3; + final public static int SYSLOG_HOSTNAME_POS = 4; + final public static int SYSLOG_BODY_POS = 5; private Mode m = Mode.START; private StringBuilder prio = new StringBuilder(); @@ -78,6 +86,8 @@ public class SyslogUtils { final public static String SYSLOG_FACILITY = "Facility"; final public static String SYSLOG_SEVERITY = "Severity"; + final public static String SYSLOG_PRIORITY = "Priority"; + final public static String SYSLOG_VERSION = "Version"; final public static String EVENT_STATUS = "flume.syslog.status"; final public static Integer MIN_SIZE = 10; final public static Integer DEFAULT_SIZE = 2500; @@ -85,7 +95,7 @@ public class SyslogUtils { private boolean isBadEvent; private boolean isIncompleteEvent; private Integer maxSize; - private boolean keepFields; + private Set keepFields; private class SyslogFormatter { public Pattern regexPattern; @@ -96,19 +106,93 @@ private class SyslogFormatter { } private ArrayList formats = new ArrayList(); + private String priority = null; + private String version = null; private String timeStamp = null; private String hostName = null; private String msgBody = null; + private static final String[] DEFAULT_FIELDS_TO_KEEP = { + SyslogSourceConfigurationConstants.CONFIG_KEEP_FIELDS_PRIORITY, + SyslogSourceConfigurationConstants.CONFIG_KEEP_FIELDS_VERSION, + SyslogSourceConfigurationConstants.CONFIG_KEEP_FIELDS_TIMESTAMP, + SyslogSourceConfigurationConstants.CONFIG_KEEP_FIELDS_HOSTNAME + }; + public static final String KEEP_FIELDS_ALL = "--all--"; + + public static boolean keepAllFields(Set keepFields) { + if (keepFields == null) { + return false; + } + return keepFields.contains(KEEP_FIELDS_ALL); + } + + public static Set chooseFieldsToKeep(String keepFields) { + if (keepFields == null) { + return null; + } + + keepFields = keepFields.trim().toLowerCase(); + + if (keepFields.equals("false") || keepFields.equals("none")) { + return null; + } + + if (keepFields.equals("true") || keepFields.equals("all")) { + Set fieldsToKeep = new HashSet(1); + fieldsToKeep.add(KEEP_FIELDS_ALL); + return fieldsToKeep; + } + + Set fieldsToKeep = new HashSet(DEFAULT_FIELDS_TO_KEEP.length); + + for (String field : DEFAULT_FIELDS_TO_KEEP) { + if (keepFields.indexOf(field) != -1) { + fieldsToKeep.add(field); + } + } + + return fieldsToKeep; + } + + public static String addFieldsToBody(Set keepFields, + String body, + String priority, + String version, + String timestamp, + String hostname) { + // Prepend fields to be kept in message body. + if (keepFields != null) { + if (keepFields.contains(SyslogSourceConfigurationConstants.CONFIG_KEEP_FIELDS_HOSTNAME)) { + body = hostname + " " + body; + } + if (keepFields.contains(SyslogSourceConfigurationConstants.CONFIG_KEEP_FIELDS_TIMESTAMP)) { + body = timestamp + " " + body; + } + if (keepFields.contains(SyslogSourceConfigurationConstants.CONFIG_KEEP_FIELDS_VERSION)) { + if (version != null && !version.isEmpty()) { + body = version + " " + body; + } + } + if (keepFields.contains(SyslogSourceConfigurationConstants.CONFIG_KEEP_FIELDS_PRIORITY)) { + body = "<" + priority + ">" + body; + } + } + + return body; + } + public SyslogUtils() { this(false); } public SyslogUtils(boolean isUdp) { - this(DEFAULT_SIZE, SyslogSourceConfigurationConstants.DEFAULT_KEEP_FIELDS, isUdp); + this(DEFAULT_SIZE, + new HashSet(Arrays.asList(SyslogSourceConfigurationConstants.DEFAULT_KEEP_FIELDS)), + isUdp); } - public SyslogUtils(Integer eventSize, boolean keepFields, boolean isUdp) { + public SyslogUtils(Integer eventSize, Set keepFields, boolean isUdp) { this.isUdp = isUdp; isBadEvent = false; isIncompleteEvent = false; @@ -212,6 +296,12 @@ Event buildEvent() { Map headers = new HashMap(); headers.put(SYSLOG_FACILITY, String.valueOf(facility)); headers.put(SYSLOG_SEVERITY, String.valueOf(sev)); + if ((priority != null) && (priority.length() > 0)) { + headers.put("priority", priority); + } + if ((version != null) && (version.length() > 0)) { + headers.put("version", version); + } if ((timeStamp != null) && timeStamp.length() > 0) { headers.put("timestamp", timeStamp); } @@ -227,7 +317,7 @@ Event buildEvent() { headers.put(EVENT_STATUS, SyslogStatus.INCOMPLETE.getSyslogStatus()); } - if (!keepFields) { + if (!keepAllFields(keepFields)) { if ((msgBody != null) && (msgBody.length() > 0)) { body = msgBody.getBytes(); } else { @@ -245,6 +335,8 @@ Event buildEvent() { // Apply each known pattern to message private void formatHeaders() { String eventStr = baos.toString(); + String timeStampString = null; + for(int p=0; p < formats.size(); p++) { SyslogFormatter fmt = formats.get(p); Pattern pattern = fmt.regexPattern; @@ -256,6 +348,8 @@ private void formatHeaders() { for (int grp=1; grp <= res.groupCount(); grp++) { String value = res.group(grp); if (grp == SYSLOG_TIMESTAMP_POS) { + timeStampString = value; + // apply available format replacements to timestamp if (value != null) { for (int sp=0; sp < fmt.searchPattern.size(); sp++) { @@ -278,8 +372,12 @@ private void formatHeaders() { } } else if (grp == SYSLOG_HOSTNAME_POS) { hostName = value; + } else if (grp == SYSLOG_PRIORITY_POS) { + priority = value; + } else if (grp == SYSLOG_VERSION_POS) { + version = value; } else if (grp == SYSLOG_BODY_POS) { - msgBody = value; + msgBody = addFieldsToBody(keepFields, value, priority, version, timeStampString, hostName); } } break; // we successfully parsed the message using this pattern @@ -388,9 +486,9 @@ public void setEventSize(Integer eventSize) { this.maxSize = eventSize; } - public void setKeepFields(Boolean keepFields) { - this.keepFields= keepFields; - } + public void setKeepFields(Set keepFields) { + this.keepFields = keepFields; } +} diff --git a/flume-ng-core/src/test/java/org/apache/flume/source/TestMultiportSyslogTCPSource.java b/flume-ng-core/src/test/java/org/apache/flume/source/TestMultiportSyslogTCPSource.java index 9b97c8c730..c3dc241175 100644 --- a/flume-ng-core/src/test/java/org/apache/flume/source/TestMultiportSyslogTCPSource.java +++ b/flume-ng-core/src/test/java/org/apache/flume/source/TestMultiportSyslogTCPSource.java @@ -70,7 +70,6 @@ public class TestMultiportSyslogTCPSource { private final String stamp1 = time.toString(); private final String host1 = "localhost.localdomain"; private final String data1 = "proc1 - some msg"; - private final static boolean KEEP_FIELDS = false; /** * Helper function to generate a syslog message. @@ -207,7 +206,7 @@ public void testFragmented() throws CharacterCodingException { null, SyslogSourceConfigurationConstants.DEFAULT_PORT_HEADER, new ThreadSafeDecoder(Charsets.UTF_8), new ConcurrentHashMap(), - KEEP_FIELDS); + null); Event event = handler.parseEvent(parsedLine, Charsets.UTF_8.newDecoder()); String body = new String(event.getBody(), Charsets.UTF_8); @@ -234,7 +233,7 @@ public void testCharsetParsing() throws FileNotFoundException, IOException { new SourceCounter("test"), "port", new ThreadSafeDecoder(Charsets.UTF_8), new ConcurrentHashMap(), - KEEP_FIELDS); + null); ParsedBuffer parsedBuf = new ParsedBuffer(); parsedBuf.incomplete = false; @@ -334,7 +333,8 @@ public void testPortCharsetHandling() throws UnknownHostException, Exception { // defaults to UTF-8 MultiportSyslogHandler handler = new MultiportSyslogHandler( 1000, 10, chanProc, new SourceCounter("test"), "port", - new ThreadSafeDecoder(Charsets.UTF_8), portCharsets, KEEP_FIELDS); + new ThreadSafeDecoder(Charsets.UTF_8), portCharsets, + null); // initialize buffers handler.sessionCreated(session1); diff --git a/flume-ng-core/src/test/java/org/apache/flume/source/TestSyslogParser.java b/flume-ng-core/src/test/java/org/apache/flume/source/TestSyslogParser.java index 2809163e16..6e0fd665c4 100644 --- a/flume-ng-core/src/test/java/org/apache/flume/source/TestSyslogParser.java +++ b/flume-ng-core/src/test/java/org/apache/flume/source/TestSyslogParser.java @@ -21,7 +21,10 @@ import com.google.common.base.Charsets; import com.google.common.collect.Lists; import java.nio.charset.Charset; +import java.util.HashSet; import java.util.List; +import java.util.Set; + import org.apache.flume.Event; import org.joda.time.format.DateTimeFormatter; import org.joda.time.format.ISODateTimeFormat; @@ -77,7 +80,7 @@ public void testMessageParsing() { // test with default keepFields = false for (String msg : messages) { - boolean keepFields = false; + Set keepFields = new HashSet(); Event event = parser.parseMessage(msg, charset, keepFields); Assert.assertNull("Failure to parse known-good syslog message", event.getHeaders().get(SyslogUtils.EVENT_STATUS)); @@ -85,11 +88,23 @@ public void testMessageParsing() { // test that priority, timestamp and hostname are preserved in event body for (String msg : messages) { - boolean keepFields = true; + Set keepFields = new HashSet(); + keepFields.add(SyslogUtils.KEEP_FIELDS_ALL); Event event = parser.parseMessage(msg, charset, keepFields); Assert.assertArrayEquals(event.getBody(), msg.getBytes()); Assert.assertNull("Failure to parse known-good syslog message", event.getHeaders().get(SyslogUtils.EVENT_STATUS)); } + + // test that hostname is preserved in event body + for (String msg : messages) { + Set keepFields = new HashSet(); + keepFields.add(SyslogSourceConfigurationConstants.CONFIG_KEEP_FIELDS_HOSTNAME); + Event event = parser.parseMessage(msg, charset, keepFields); + Assert.assertTrue("Failure to persist hostname", + new String(event.getBody()).contains(event.getHeaders().get("host"))); + Assert.assertNull("Failure to parse known-good syslog message", + event.getHeaders().get(SyslogUtils.EVENT_STATUS)); + } } } \ No newline at end of file diff --git a/flume-ng-core/src/test/java/org/apache/flume/source/TestSyslogTcpSource.java b/flume-ng-core/src/test/java/org/apache/flume/source/TestSyslogTcpSource.java index 22fa200865..239ba51bcd 100644 --- a/flume-ng-core/src/test/java/org/apache/flume/source/TestSyslogTcpSource.java +++ b/flume-ng-core/src/test/java/org/apache/flume/source/TestSyslogTcpSource.java @@ -49,10 +49,14 @@ public class TestSyslogTcpSource { private final String stamp1 = time.toString(); private final String host1 = "localhost.localdomain"; private final String data1 = "test syslog data"; + private final String bodyWithHostname = host1 + " " + + data1; + private final String bodyWithTimestamp = stamp1 + " " + + data1; private final String bodyWithTandH = "<10>" + stamp1 + " " + host1 + " " + data1 + "\n"; - private void init(boolean keepFields){ + private void init(String keepFields){ source = new SyslogTcpSource(); channel = new MemoryChannel(); @@ -67,14 +71,14 @@ private void init(boolean keepFields){ source.setChannelProcessor(new ChannelProcessor(rcs)); Context context = new Context(); context.put("port", String.valueOf(TEST_SYSLOG_PORT)); - context.put("keepFields", String.valueOf(keepFields)); + context.put("keepFields", keepFields); source.configure(context); } /** Tests the keepFields configuration parameter (enabled or disabled) using SyslogTcpSource.*/ - private void runKeepFieldsTest(boolean keepFields) throws IOException { + private void runKeepFieldsTest(String keepFields) throws IOException { init(keepFields); source.start(); // Write some message to the syslog port @@ -110,23 +114,43 @@ private void runKeepFieldsTest(boolean keepFields) throws IOException { Assert.assertNotNull(e); String str = new String(e.getBody(), Charsets.UTF_8); logger.info(str); - if (keepFields) { + if (keepFields.equals("true") || keepFields.equals("all")) { Assert.assertArrayEquals(bodyWithTandH.trim().getBytes(), e.getBody()); - } else if (!keepFields) { + } else if (keepFields.equals("false") || keepFields.equals("none")) { Assert.assertArrayEquals(data1.getBytes(), e.getBody()); + } else if (keepFields.equals("hostname")) { + Assert.assertArrayEquals(bodyWithHostname.getBytes(), e.getBody()); + } else if (keepFields.equals("timestamp")) { + Assert.assertArrayEquals(bodyWithTimestamp.getBytes(), e.getBody()); } } } @Test - public void testKeepFields () throws IOException { - runKeepFieldsTest(true); + public void testKeepFields() throws IOException { + runKeepFieldsTest("all"); + + // Backwards compatibility + runKeepFieldsTest("true"); } @Test public void testRemoveFields() throws IOException{ - runKeepFieldsTest(false); - } + runKeepFieldsTest("none"); + + // Backwards compatibility + runKeepFieldsTest("false"); + } + + @Test + public void testKeepHostname() throws IOException{ + runKeepFieldsTest("hostname"); + } + + @Test + public void testKeepTimestamp() throws IOException{ + runKeepFieldsTest("timestamp"); } +} diff --git a/flume-ng-core/src/test/java/org/apache/flume/source/TestSyslogUdpSource.java b/flume-ng-core/src/test/java/org/apache/flume/source/TestSyslogUdpSource.java index 95ee48c4cb..8fc80be5da 100644 --- a/flume-ng-core/src/test/java/org/apache/flume/source/TestSyslogUdpSource.java +++ b/flume-ng-core/src/test/java/org/apache/flume/source/TestSyslogUdpSource.java @@ -50,11 +50,15 @@ public class TestSyslogUdpSource { private final DateTime time = new DateTime(); private final String stamp1 = time.toString(); private final String host1 = "localhost.localdomain"; - private final String data1 = "test UDP syslog data"; + private final String data1 = "test syslog data"; + private final String bodyWithHostname = host1 + " " + + data1; + private final String bodyWithTimestamp = stamp1 + " " + + data1; private final String bodyWithTandH = "<10>" + stamp1 + " " + host1 + " " + data1; - private void init(boolean keepFields) { + private void init(String keepFields) { source = new SyslogUDPSource(); channel = new MemoryChannel(); @@ -69,7 +73,7 @@ private void init(boolean keepFields) { source.setChannelProcessor(new ChannelProcessor(rcs)); Context context = new Context(); context.put("port", String.valueOf(TEST_SYSLOG_PORT)); - context.put("keepFields", String.valueOf(keepFields)); + context.put("keepFields", keepFields); source.configure(context); @@ -78,7 +82,7 @@ private void init(boolean keepFields) { /** Tests the keepFields configuration parameter (enabled or disabled) using SyslogUDPSource.*/ - private void runKeepFieldsTest(boolean keepFields) throws IOException { + private void runKeepFieldsTest(String keepFields) throws IOException { init(keepFields); source.start(); // Write some message to the syslog port @@ -115,18 +119,22 @@ private void runKeepFieldsTest(boolean keepFields) throws IOException { Assert.assertNotNull(e); String str = new String(e.getBody(), Charsets.UTF_8); logger.info(str); - if (keepFields) { - Assert.assertArrayEquals(bodyWithTandH.getBytes(), - e.getBody()); - } else if (!keepFields) { + if (keepFields.equals("true") || keepFields.equals("all")) { + Assert.assertArrayEquals(bodyWithTandH.trim().getBytes(), + e.getBody()); + } else if (keepFields.equals("false") || keepFields.equals("none")) { Assert.assertArrayEquals(data1.getBytes(), e.getBody()); + } else if (keepFields.equals("hostname")) { + Assert.assertArrayEquals(bodyWithHostname.getBytes(), e.getBody()); + } else if (keepFields.equals("timestamp")) { + Assert.assertArrayEquals(bodyWithTimestamp.getBytes(), e.getBody()); } } } @Test public void testLargePayload() throws Exception { - init(true); + init("true"); source.start(); // Write some message to the syslog port @@ -169,12 +177,28 @@ public void testLargePayload() throws Exception { @Test public void testKeepFields() throws IOException { - runKeepFieldsTest(true); + runKeepFieldsTest("all"); + + // Backwards compatibility + runKeepFieldsTest("true"); } @Test public void testRemoveFields() throws IOException { - runKeepFieldsTest(false); + runKeepFieldsTest("none"); + + // Backwards compatibility + runKeepFieldsTest("false"); + } + + @Test + public void testKeepHostname() throws IOException{ + runKeepFieldsTest("hostname"); + } + + @Test + public void testKeepTimestamp() throws IOException{ + runKeepFieldsTest("timestamp"); } private String getPayload(int length) { diff --git a/flume-ng-core/src/test/java/org/apache/flume/source/TestSyslogUtils.java b/flume-ng-core/src/test/java/org/apache/flume/source/TestSyslogUtils.java index 82b7dd0a35..76ee5b1370 100644 --- a/flume-ng-core/src/test/java/org/apache/flume/source/TestSyslogUtils.java +++ b/flume-ng-core/src/test/java/org/apache/flume/source/TestSyslogUtils.java @@ -28,6 +28,7 @@ import java.text.ParseException; import java.text.SimpleDateFormat; import java.util.Calendar; +import java.util.HashSet; import java.util.Map; public class TestSyslogUtils { @@ -54,7 +55,6 @@ public void TestHeader1() throws ParseException { @Test public void TestHeader2() throws ParseException { - String stamp1 = "2012-04-13T11:11:11"; String format1 = "yyyy-MM-dd'T'HH:mm:ssZ"; String host1 = "ubuntu-11.cloudera.com"; @@ -162,9 +162,17 @@ public void TestRfc3164HeaderApacheLogWithNulls() throws ParseException { format1, host1, data1); } - public void checkHeader(String msg1, String stamp1, String format1, - String host1, String data1) throws ParseException { - SyslogUtils util = new SyslogUtils(false); + public static void checkHeader(String keepFields, String msg1, String stamp1, String format1, + String host1, String data1) throws ParseException { + SyslogUtils util; + if (keepFields == null || keepFields.isEmpty()) { + util = new SyslogUtils(SyslogUtils.DEFAULT_SIZE, new HashSet(), false); + } else { + util = new SyslogUtils( + SyslogUtils.DEFAULT_SIZE, + SyslogUtils.chooseFieldsToKeep(keepFields), + false); + } ChannelBuffer buff = ChannelBuffers.buffer(200); buff.writeBytes(msg1.getBytes()); @@ -188,6 +196,12 @@ public void checkHeader(String msg1, String stamp1, String format1, Assert.assertEquals(data1, new String(e.getBody())); } + // Check headers for when keepFields is "none". + public static void checkHeader(String msg1, String stamp1, String format1, + String host1, String data1) throws ParseException { + checkHeader("none", msg1, stamp1, format1, host1, data1); + } + /** * Test bad event format 1: Priority is not numeric */ @@ -405,7 +419,7 @@ public void testGoodEventGoodEvent() { public void testExtractBadEventLarge() { String badData1 = "<10> bad bad data bad bad\n"; // The minimum size (which is 10) overrides the 5 specified here. - SyslogUtils util = new SyslogUtils(5, false, false); + SyslogUtils util = new SyslogUtils(5, null, false); ChannelBuffer buff = ChannelBuffers.buffer(100); buff.writeBytes(badData1.getBytes()); Event e = util.extractEvent(buff); @@ -433,4 +447,30 @@ public void testExtractBadEventLarge() { } + @Test + public void testKeepFields() throws Exception { + String stamp1 = "2012-04-13T11:11:11"; + String format1 = "yyyy-MM-dd'T'HH:mm:ssZ"; + String host1 = "ubuntu-11.cloudera.com"; + String data1 = "some msg"; + // timestamp with hh:mm format timezone + String msg1 = "<10>1 " + stamp1 + "+08:00" + " " + host1 + " " + data1 + "\n"; + checkHeader("none", msg1, stamp1 + "+0800", format1, host1, data1); + checkHeader("false", msg1, stamp1 + "+0800", format1, host1, data1); + + String data2 = "ubuntu-11.cloudera.com some msg"; + checkHeader("hostname", msg1, stamp1 + "+0800", format1, host1, data2); + + String data3 = "2012-04-13T11:11:11+08:00 ubuntu-11.cloudera.com some msg"; + checkHeader("timestamp hostname", msg1, stamp1 + "+0800", format1, host1, data3); + + String data4 = "<10>2012-04-13T11:11:11+08:00 ubuntu-11.cloudera.com some msg"; + checkHeader("priority timestamp hostname", msg1, stamp1 + "+0800", format1, host1, data4); + + String data5 = "<10>1 2012-04-13T11:11:11+08:00 ubuntu-11.cloudera.com some msg"; + checkHeader("priority version timestamp hostname", msg1, stamp1 + "+0800", format1, host1, data5); + checkHeader("all", msg1, stamp1 + "+0800", format1, host1, data5); + checkHeader("true", msg1, stamp1 + "+0800", format1, host1, data5); + } + } diff --git a/flume-ng-doc/sphinx/FlumeUserGuide.rst b/flume-ng-doc/sphinx/FlumeUserGuide.rst index ed90022ec2..a718fbf84b 100644 --- a/flume-ng-doc/sphinx/FlumeUserGuide.rst +++ b/flume-ng-doc/sphinx/FlumeUserGuide.rst @@ -1182,8 +1182,13 @@ Property Name Default Description **host** -- Host name or IP address to bind to **port** -- Port # to bind to eventSize 2500 Maximum size of a single event line, in bytes -keepFields false Setting this to true will preserve the Priority, +keepFields none Setting this to 'all' will preserve the Priority, Timestamp and Hostname in the body of the event. + A spaced separated list of fields to include + is allowed as well. Currently, the following + fields can be included: priority, version, + timestamp, hostname. The values 'true' and 'false' + have been deprecated in favor of 'all' and 'none'. selector.type replicating or multiplexing selector.* replicating Depends on the selector.type value interceptors -- Space-separated list of interceptors @@ -1220,8 +1225,13 @@ Property Name Default Description **host** -- Host name or IP address to bind to. **ports** -- Space-separated list (one or more) of ports to bind to. eventSize 2500 Maximum size of a single event line, in bytes. -keepFields false Setting this to true will preserve the +keepFields none Setting this to 'all' will preserve the Priority, Timestamp and Hostname in the body of the event. + A spaced separated list of fields to include + is allowed as well. Currently, the following + fields can be included: priority, version, + timestamp, hostname. The values 'true' and 'false' + have been deprecated in favor of 'all' and 'none'. portHeader -- If specified, the port number will be stored in the header of each event using the header name specified here. This allows for interceptors and channel selectors to customize routing logic based on the incoming port. charset.default UTF-8 Default character set used while parsing syslog events into strings. charset.port. -- Character set is configurable on a per-port basis. diff --git a/flume-ng-tests/src/test/java/org/apache/flume/test/agent/TestSyslogSource.java b/flume-ng-tests/src/test/java/org/apache/flume/test/agent/TestSyslogSource.java new file mode 100644 index 0000000000..9b1e3250db --- /dev/null +++ b/flume-ng-tests/src/test/java/org/apache/flume/test/agent/TestSyslogSource.java @@ -0,0 +1,97 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, + * software distributed under the License is distributed on an + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + * KIND, either express or implied. See the License for the + * specific language governing permissions and limitations + * under the License. + */ + +package org.apache.flume.test.agent; + +import org.apache.flume.test.util.SyslogAgent; +import org.apache.log4j.Logger; +import org.junit.After; +import org.junit.Before; +import org.junit.Test; +import org.junit.runner.RunWith; +import org.junit.runners.Parameterized; + +import java.util.ArrayList; +import java.util.Collection; +import java.util.List; + +@RunWith(Parameterized.class) +public class TestSyslogSource { + private static final Logger LOGGER = Logger.getLogger(TestSyslogSource.class); + + private SyslogAgent agent; + private SyslogAgent.SyslogSourceType sourceType; + + public TestSyslogSource(SyslogAgent.SyslogSourceType sourceType) { + this.sourceType = sourceType; + } + + @Parameterized.Parameters + public static Collection syslogSourceTypes() { + List sourceTypes = new ArrayList(); + for (SyslogAgent.SyslogSourceType sourceType : SyslogAgent.SyslogSourceType.values()) { + sourceTypes.add(new Object[]{sourceType}); + } + return sourceTypes; + } + + @Before + public void setUp() throws Exception { + agent = new SyslogAgent(); + agent.configure(sourceType); + } + + @After + public void tearDown() throws Exception { + if (agent != null) { + agent.stop(); + agent = null; + } + } + + @Test + public void testKeepFields() throws Exception { + LOGGER.debug("testKeepFields() started."); + + agent.start("all"); + agent.runKeepFieldsTest(); + + LOGGER.debug("testKeepFields() ended."); + } + + @Test + public void testRemoveFields() throws Exception { + LOGGER.debug("testRemoveFields() started."); + + agent.start("none"); + agent.runKeepFieldsTest(); + + LOGGER.debug("testRemoveFields() ended."); + } + + @Test + public void testKeepTimestampAndHostname() throws Exception { + LOGGER.debug("testKeepTimestampAndHostname() started."); + + agent.start("timestamp hostname"); + agent.runKeepFieldsTest(); + + LOGGER.debug("testKeepTimestampAndHostname() ended."); + } +} diff --git a/flume-ng-tests/src/test/java/org/apache/flume/test/util/StagedInstall.java b/flume-ng-tests/src/test/java/org/apache/flume/test/util/StagedInstall.java index a6bd5e9e64..973ff4a327 100644 --- a/flume-ng-tests/src/test/java/org/apache/flume/test/util/StagedInstall.java +++ b/flume-ng-tests/src/test/java/org/apache/flume/test/util/StagedInstall.java @@ -91,6 +91,7 @@ public synchronized void stopAgent() throws Exception { LOGGER.info("Shutting down agent process"); process.destroy(); + process.waitFor(); process = null; consumer.interrupt(); consumer = null; @@ -155,6 +156,7 @@ public synchronized void startAgent(String name, Properties properties) Map env = pb.environment(); LOGGER.debug("process environment: " + env); + pb.directory(baseDir); pb.redirectErrorStream(true); diff --git a/flume-ng-tests/src/test/java/org/apache/flume/test/util/SyslogAgent.java b/flume-ng-tests/src/test/java/org/apache/flume/test/util/SyslogAgent.java new file mode 100644 index 0000000000..7159549b46 --- /dev/null +++ b/flume-ng-tests/src/test/java/org/apache/flume/test/util/SyslogAgent.java @@ -0,0 +1,238 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, + * software distributed under the License is distributed on an + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + * KIND, either express or implied. See the License for the + * specific language governing permissions and limitations + * under the License. + */ + +package org.apache.flume.test.util; + +import com.google.common.base.Charsets; +import com.google.common.io.Files; +import org.apache.commons.io.FileUtils; +import org.apache.log4j.Logger; +import org.junit.Assert; + +import java.io.BufferedOutputStream; +import java.io.File; +import java.io.IOException; +import java.net.ServerSocket; +import java.net.Socket; +import java.util.ArrayList; +import java.util.Collection; +import java.util.Properties; +import java.util.concurrent.TimeUnit; + +/** + * Syslog Flume Agent. + * A Syslog source of some kind is configured and a client is available to write + * messages to the agent. The Flume agents port is randomly assigned (not in use). + * + */ +public class SyslogAgent { + private static final Logger LOGGER = Logger.getLogger(SyslogAgent.class); + private static final Collection tempResources = new ArrayList(); + private static final int DEFAULT_ATTEMPTS = 20; + private static final long DEFAULT_TIMEOUT = 500L; + + public enum SyslogSourceType { + TCP("syslogtcp"), + MULTIPORTTCP("multiport_syslogtcp"); + + private final String syslogSourceType; + + private SyslogSourceType(String syslogSourceType) { + this.syslogSourceType = syslogSourceType; + } + + public String toString() { + return syslogSourceType; + } + }; + + private Properties agentProps; + private File sinkOutputDir; + private String keepFields; + + private int port; + private String hostname; + + BufferedOutputStream client; + + public SyslogAgent() throws IOException { + hostname = "localhost"; + + setRandomPort(); + } + + public void setRandomPort() throws IOException { + ServerSocket s = new ServerSocket(0); + port = s.getLocalPort(); + s.close(); + } + + public void configure(SyslogSourceType sourceType) throws IOException { + /* Create 3 temp dirs, each used as value within agentProps */ + sinkOutputDir = Files.createTempDir(); + tempResources.add(sinkOutputDir); + final String sinkOutputDirPath = sinkOutputDir.getCanonicalPath(); + LOGGER.info("Created rolling file sink's output dir: " + + sinkOutputDirPath); + + /* Build props to pass to flume agent */ + agentProps = new Properties(); + + // Active sets + agentProps.put("a1.channels", "c1"); + agentProps.put("a1.sources", "r1"); + agentProps.put("a1.sinks", "k1"); + + // c1 + agentProps.put("a1.channels.c1.type", "memory"); + agentProps.put("a1.channels.c1.capacity", "1000"); + agentProps.put("a1.channels.c1.transactionCapacity", "100"); + + // r1 + agentProps.put("a1.sources.r1.channels", "c1"); + agentProps.put("a1.sources.r1.type", sourceType.toString()); + agentProps.put("a1.sources.r1.host", hostname); + if (sourceType.equals(SyslogSourceType.MULTIPORTTCP)) { + agentProps.put("a1.sources.r1.ports", Integer.toString(port)); + } else { + agentProps.put("a1.sources.r1.port", Integer.toString(port)); + } + + // k1 + agentProps.put("a1.sinks.k1.channel", "c1"); + agentProps.put("a1.sinks.k1.sink.directory", sinkOutputDirPath); + agentProps.put("a1.sinks.k1.type", "FILE_ROLL"); + agentProps.put("a1.sinks.k1.sink.rollInterval", "0"); + } + + // Blocks until flume agent boots up. + public void start(String keepFields) throws Exception { + this.keepFields = keepFields; + + // Set properties that should be different per agent start and stop. + agentProps.put("a1.sources.r1.keepFields", keepFields); + + // Recreate temporary directory. + sinkOutputDir.mkdir(); + + /* Start flume agent */ + StagedInstall.getInstance().startAgent("a1", agentProps); + + LOGGER.info("Started flume agent with syslog source on port " + port); + + // Wait for source, channel, sink to start and create client. + int numberOfAttempts = 0; + while (client == null) { + try { + client = new BufferedOutputStream(new Socket(hostname, port).getOutputStream()); + } catch(IOException e) { + if (++numberOfAttempts >= DEFAULT_ATTEMPTS) { + throw new AssertionError("Could not connect to source after " + + DEFAULT_ATTEMPTS + " attempts with " + DEFAULT_TIMEOUT + " ms timeout."); + } + + TimeUnit.MILLISECONDS.sleep(DEFAULT_TIMEOUT); + } + } + } + + public boolean isRunning() throws Exception { + return StagedInstall.getInstance().isRunning(); + } + + public void stop() throws Exception { + client.close(); + client = null; + + StagedInstall.getInstance().stopAgent(); + for (File tempResource : tempResources) { + // Should always be a directory. + FileUtils.deleteDirectory(tempResource); + } + } + + public void runKeepFieldsTest() throws Exception { + /* Create expected output and log message */ + String logMessage = "<34>1 Oct 11 22:14:15 mymachine su: Test\n"; + String expectedOutput = "su: Test\n"; + if (keepFields.equals("true") || keepFields.equals("all")) { + expectedOutput = logMessage; + } else if (!keepFields.equals("false") && !keepFields.equals("none")) { + if (keepFields.indexOf("hostname") != -1) { + expectedOutput = "mymachine " + expectedOutput; + } + if (keepFields.indexOf("timestamp") != -1) { + expectedOutput = "Oct 11 22:14:15 " + expectedOutput; + } + if (keepFields.indexOf("version") != -1) { + expectedOutput = "1 " + expectedOutput; + } + if (keepFields.indexOf("priority") != -1) { + expectedOutput = "<34>" + expectedOutput; + } + } + LOGGER.info("Created expected output: " + expectedOutput); + + /* Send test message to agent */ + sendMessage(logMessage); + + /* Wait for output file */ + int numberOfListDirAttempts = 0; + while (sinkOutputDir.listFiles().length == 0) { + if (++numberOfListDirAttempts >= DEFAULT_ATTEMPTS) { + throw new AssertionError("FILE_ROLL sink hasn't written any files after " + + DEFAULT_ATTEMPTS + " attempts with " + DEFAULT_TIMEOUT + " ms timeout."); + } + + TimeUnit.MILLISECONDS.sleep(DEFAULT_TIMEOUT); + } + + // Only 1 file should be in FILE_ROLL sink's dir (rolling is disabled) + File[] sinkOutputDirChildren = sinkOutputDir.listFiles(); + Assert.assertEquals("Expected FILE_ROLL sink's dir to have only 1 child," + + " but found " + sinkOutputDirChildren.length + " children.", + 1, sinkOutputDirChildren.length); + + /* Wait for output file stats to be as expected. */ + File outputDirChild = sinkOutputDirChildren[0]; + int numberOfStatsAttempts = 0; + while (outputDirChild.length() != expectedOutput.length()) { + if (++numberOfStatsAttempts >= DEFAULT_ATTEMPTS) { + throw new AssertionError("Expected output and FILE_ROLL sink's" + + " lengths did not match after " + DEFAULT_ATTEMPTS + + " attempts with " + DEFAULT_TIMEOUT + " ms timeout."); + } + + TimeUnit.MILLISECONDS.sleep(DEFAULT_TIMEOUT); + } + + File actualOutput = sinkOutputDirChildren[0]; + if (!Files.toString(actualOutput, Charsets.UTF_8).equals(expectedOutput)) { + LOGGER.error("Actual output doesn't match expected output.\n"); + LOGGER.debug("Output: " + Files.toString(actualOutput, Charsets.UTF_8)); + throw new AssertionError("FILE_ROLL sink's actual output doesn't " + + "match expected output."); + } + } + + private void sendMessage(String message) throws IOException { + client.write(message.getBytes()); + client.flush(); + } +} From 9e28ea671cf1baf86e26f468643f94141dc94323 Mon Sep 17 00:00:00 2001 From: Brock Noland Date: Fri, 22 Aug 2014 12:11:20 -0700 Subject: [PATCH 130/341] FLUME-2425 - FileChannel should trim data and checkpoint directories (Johny Rufus via Brock Noland) --- .../org/apache/flume/channel/file/FileChannel.java | 10 +++++++--- 1 file changed, 7 insertions(+), 3 deletions(-) diff --git a/flume-ng-channels/flume-file-channel/src/main/java/org/apache/flume/channel/file/FileChannel.java b/flume-ng-channels/flume-file-channel/src/main/java/org/apache/flume/channel/file/FileChannel.java index 413bfbc268..61c353aff9 100644 --- a/flume-ng-channels/flume-file-channel/src/main/java/org/apache/flume/channel/file/FileChannel.java +++ b/flume-ng-channels/flume-file-channel/src/main/java/org/apache/flume/channel/file/FileChannel.java @@ -21,8 +21,10 @@ import com.google.common.annotations.VisibleForTesting; import com.google.common.base.Preconditions; +import com.google.common.base.Splitter; import com.google.common.base.Strings; import com.google.common.base.Throwables; +import com.google.common.collect.Iterables; import org.apache.flume.*; import org.apache.flume.annotations.Disposable; import org.apache.flume.annotations.InterfaceAudience; @@ -120,13 +122,15 @@ public void configure(Context context) { String strCheckpointDir = context.getString(FileChannelConfiguration.CHECKPOINT_DIR, - homePath + "/.flume/file-channel/checkpoint"); + homePath + "/.flume/file-channel/checkpoint").trim(); String strBackupCheckpointDir = context.getString (FileChannelConfiguration.BACKUP_CHECKPOINT_DIR, "").trim(); - String[] strDataDirs = context.getString(FileChannelConfiguration.DATA_DIRS, - homePath + "/.flume/file-channel/data").split(","); + String[] strDataDirs = Iterables.toArray( + Splitter.on(",").trimResults().omitEmptyStrings().split( + context.getString(FileChannelConfiguration.DATA_DIRS, + homePath + "/.flume/file-channel/data")), String.class); checkpointDir = new File(strCheckpointDir); From 5c5b96a8c89d2fe58f1425a4ece8160b76f03f26 Mon Sep 17 00:00:00 2001 From: Brock Noland Date: Fri, 29 Aug 2014 12:01:03 -0700 Subject: [PATCH 131/341] FLUME-2450 - Improve replay index insertion speed. (Hari via Brock) --- .../java/org/apache/flume/channel/file/FlumeEventQueue.java | 4 +++- 1 file changed, 3 insertions(+), 1 deletion(-) diff --git a/flume-ng-channels/flume-file-channel/src/main/java/org/apache/flume/channel/file/FlumeEventQueue.java b/flume-ng-channels/flume-file-channel/src/main/java/org/apache/flume/channel/file/FlumeEventQueue.java index 83309ac43f..d305f4d2a7 100644 --- a/flume-ng-channels/flume-file-channel/src/main/java/org/apache/flume/channel/file/FlumeEventQueue.java +++ b/flume-ng-channels/flume-file-channel/src/main/java/org/apache/flume/channel/file/FlumeEventQueue.java @@ -105,8 +105,10 @@ final class FlumeEventQueue { .syncOnCommitDisable() .deleteFilesAfterClose() .cacheDisable() + .mmapFileEnableIfSupported() .make(); - queueSet = db.createTreeSet("QueueSet").make(); + queueSet = + db.createHashSet("QueueSet " + " - " + backingStore.getName()).make(); long start = System.currentTimeMillis(); for (int i = 0; i < backingStore.getSize(); i++) { queueSet.add(get(i)); From 4e08bf7d38bea365d35a6d391d1507a129cc9ba9 Mon Sep 17 00:00:00 2001 From: Hari Shreedharan Date: Fri, 12 Sep 2014 11:21:26 -0700 Subject: [PATCH 132/341] FLUME-2420. HDFS Bucketwriter must access sfWriters map only within synchronized blocks. (chenshangan via Hari) --- .../apache/flume/sink/hdfs/HDFSEventSink.java | 26 +++++++++++-------- 1 file changed, 15 insertions(+), 11 deletions(-) diff --git a/flume-ng-sinks/flume-hdfs-sink/src/main/java/org/apache/flume/sink/hdfs/HDFSEventSink.java b/flume-ng-sinks/flume-hdfs-sink/src/main/java/org/apache/flume/sink/hdfs/HDFSEventSink.java index 4f3b3f064f..33f73a9cf0 100644 --- a/flume-ng-sinks/flume-hdfs-sink/src/main/java/org/apache/flume/sink/hdfs/HDFSEventSink.java +++ b/flume-ng-sinks/flume-hdfs-sink/src/main/java/org/apache/flume/sink/hdfs/HDFSEventSink.java @@ -495,16 +495,18 @@ private BucketWriter initializeBucketWriter(String realPath, @Override public void stop() { // do not constrain close() calls with a timeout - for (Entry entry : sfWriters.entrySet()) { - LOG.info("Closing {}", entry.getKey()); + synchronized (sfWritersLock) { + for (Entry entry : sfWriters.entrySet()) { + LOG.info("Closing {}", entry.getKey()); - try { - entry.getValue().close(); - } catch (Exception ex) { - LOG.warn("Exception while closing " + entry.getKey() + ". " + - "Exception follows.", ex); - if (ex instanceof InterruptedException) { - Thread.currentThread().interrupt(); + try { + entry.getValue().close(); + } catch (Exception ex) { + LOG.warn("Exception while closing " + entry.getKey() + ". " + + "Exception follows.", ex); + if (ex instanceof InterruptedException) { + Thread.currentThread().interrupt(); + } } } } @@ -526,8 +528,10 @@ public void stop() { callTimeoutPool = null; timedRollerPool = null; - sfWriters.clear(); - sfWriters = null; + synchronized (sfWritersLock) { + sfWriters.clear(); + sfWriters = null; + } sinkCounter.stop(); super.stop(); } From 59f0b4df97231acdc0b9769dccb3211c502b36d3 Mon Sep 17 00:00:00 2001 From: Hari Shreedharan Date: Fri, 12 Sep 2014 11:50:57 -0700 Subject: [PATCH 133/341] FLUME-2426. Support interceptors in the Embedded Agent (Johny Rufus via Hari) --- flume-ng-doc/sphinx/FlumeDeveloperGuide.rst | 8 ++++++- .../flume/agent/embedded/EmbeddedAgent.java | 4 +++- .../agent/embedded/TestEmbeddedAgent.java | 24 ++++++++++++++++++- .../TestEmbeddedAgentConfiguration.java | 4 ++++ 4 files changed, 37 insertions(+), 3 deletions(-) diff --git a/flume-ng-doc/sphinx/FlumeDeveloperGuide.rst b/flume-ng-doc/sphinx/FlumeDeveloperGuide.rst index ec6a735bc4..e3b60e6701 100644 --- a/flume-ng-doc/sphinx/FlumeDeveloperGuide.rst +++ b/flume-ng-doc/sphinx/FlumeDeveloperGuide.rst @@ -450,7 +450,7 @@ sources, sinks, and channels are allowed. Specifically the source used is a special embedded source and events should be send to the source via the put, putAll methods on the EmbeddedAgent object. Only File Channel and Memory Channel are allowed as channels while Avro Sink is the only -supported sink. +supported sink. Interceptors are also supported by the embedded agent. Note: The embedded agent has a dependency on hadoop-core.jar. @@ -470,6 +470,8 @@ channel.* -- Configuration options for the channel ty sink.* -- Configuration options for the sink. See AvroSink user guide for an exhaustive list, however note AvroSink requires at least hostname and port. **processor.type** -- Either ``failover`` or ``load_balance`` which correspond to FailoverSinksProcessor and LoadBalancingSinkProcessor respectively. processor.* -- Configuration options for the sink processor selected. See FailoverSinksProcessor and LoadBalancingSinkProcessor user guide for an exhaustive list. +source.interceptors -- Space-separated list of interceptors +source.interceptors.* -- Configuration options for individual interceptors specified in the source.interceptors property ==================== ================ ============================================== Below is an example of how to use the agent: @@ -487,6 +489,10 @@ Below is an example of how to use the agent: properties.put("sink2.hostname", "collector2.apache.org"); properties.put("sink2.port", "5565"); properties.put("processor.type", "load_balance"); + properties.put("source.interceptors", "i1"); + properties.put("source.interceptors.i1.type", "static"); + properties.put("source.interceptors.i1.key", "key1"); + properties.put("source.interceptors.i1.value", "value1"); EmbeddedAgent agent = new EmbeddedAgent("myagent"); diff --git a/flume-ng-embedded-agent/src/main/java/org/apache/flume/agent/embedded/EmbeddedAgent.java b/flume-ng-embedded-agent/src/main/java/org/apache/flume/agent/embedded/EmbeddedAgent.java index d02f440d68..32c9f18cfd 100644 --- a/flume-ng-embedded-agent/src/main/java/org/apache/flume/agent/embedded/EmbeddedAgent.java +++ b/flume-ng-embedded-agent/src/main/java/org/apache/flume/agent/embedded/EmbeddedAgent.java @@ -111,7 +111,8 @@ public void start() throw new IllegalStateException("Cannot be started before being " + "configured"); } - doStart(); + // This check needs to be done before doStart(), + // as doStart() accesses sourceRunner.getSource() Source source = Preconditions.checkNotNull(sourceRunner.getSource(), "Source runner returned null source"); if(source instanceof EmbeddedSource) { @@ -120,6 +121,7 @@ public void start() throw new IllegalStateException("Unknown source type: " + source. getClass().getName()); } + doStart(); state = State.STARTED; } /** diff --git a/flume-ng-embedded-agent/src/test/java/org/apache/flume/agent/embedded/TestEmbeddedAgent.java b/flume-ng-embedded-agent/src/test/java/org/apache/flume/agent/embedded/TestEmbeddedAgent.java index 0d644c65e8..975ba8d1f3 100644 --- a/flume-ng-embedded-agent/src/test/java/org/apache/flume/agent/embedded/TestEmbeddedAgent.java +++ b/flume-ng-embedded-agent/src/test/java/org/apache/flume/agent/embedded/TestEmbeddedAgent.java @@ -61,7 +61,6 @@ public class TestEmbeddedAgent { private Map headers; private byte[] body; - @Before public void setUp() throws Exception { headers = Maps.newHashMap(); @@ -93,6 +92,7 @@ public void setUp() throws Exception { agent = new EmbeddedAgent("test-" + serialNumber.incrementAndGet()); } + @After public void tearDown() throws Exception { if(agent != null) { @@ -110,6 +110,7 @@ public void tearDown() throws Exception { } } } + @Test(timeout = 30000L) public void testPut() throws Exception { agent.configure(properties); @@ -124,6 +125,7 @@ public void testPut() throws Exception { Assert.assertArrayEquals(body, event.getBody()); Assert.assertEquals(headers, event.getHeaders()); } + @Test(timeout = 30000L) public void testPutAll() throws Exception { List events = Lists.newArrayList(); @@ -141,7 +143,27 @@ public void testPutAll() throws Exception { Assert.assertEquals(headers, event.getHeaders()); } + @Test(timeout = 30000L) + public void testPutWithInterceptors() throws Exception { + properties.put("source.interceptors", "i1"); + properties.put("source.interceptors.i1.type", "static"); + properties.put("source.interceptors.i1.key", "key2"); + properties.put("source.interceptors.i1.value", "value2"); + + agent.configure(properties); + agent.start(); + agent.put(EventBuilder.withBody(body, headers)); + Event event; + while((event = eventCollector.poll()) == null) { + Thread.sleep(500L); + } + Assert.assertNotNull(event); + Assert.assertArrayEquals(body, event.getBody()); + Map newHeaders = new HashMap(headers); + newHeaders.put("key2", "value2"); + Assert.assertEquals(newHeaders, event.getHeaders()); + } static class EventCollector implements AvroSourceProtocol { private final Queue eventQueue = diff --git a/flume-ng-embedded-agent/src/test/java/org/apache/flume/agent/embedded/TestEmbeddedAgentConfiguration.java b/flume-ng-embedded-agent/src/test/java/org/apache/flume/agent/embedded/TestEmbeddedAgentConfiguration.java index f70d0b1b6e..f4a9a58cc9 100644 --- a/flume-ng-embedded-agent/src/test/java/org/apache/flume/agent/embedded/TestEmbeddedAgentConfiguration.java +++ b/flume-ng-embedded-agent/src/test/java/org/apache/flume/agent/embedded/TestEmbeddedAgentConfiguration.java @@ -46,6 +46,8 @@ public void setUp() throws Exception { properties.put("sink2.hostname", "sink2.host"); properties.put("sink2.port", "2"); properties.put("processor.type", "load_balance"); + properties.put("source.interceptors", "i1"); + properties.put("source.interceptors.i1.type", "timestamp"); } @@ -91,6 +93,8 @@ public void doTestExcepted(Map actual) throws Exception { expected.put("test1.sources.source-test1.channels", "channel-test1"); expected.put("test1.sources.source-test1.type", EmbeddedAgentConfiguration. SOURCE_TYPE_EMBEDDED); + expected.put("test1.sources.source-test1.interceptors", "i1"); + expected.put("test1.sources.source-test1.interceptors.i1.type", "timestamp"); Assert.assertEquals(expected, actual); } From dd466c7e4623d9f5fd459b59836274d852f58d36 Mon Sep 17 00:00:00 2001 From: Hari Shreedharan Date: Fri, 12 Sep 2014 13:16:13 -0700 Subject: [PATCH 134/341] FLUME-1491. Support fetching configuration from Zookeeper. (Ashish Paliwal via Hari) --- bin/flume-ng | 33 +++-- flume-ng-node/pom.xml | 15 ++ .../node/AbstractConfigurationProvider.java | 18 ++- ...bstractZooKeeperConfigurationProvider.java | 104 ++++++++++++++ .../org/apache/flume/node/Application.java | 131 +++++++++++------ ...PollingZooKeeperConfigurationProvider.java | 135 +++++++++++++++++ .../PropertiesFileConfigurationProvider.java | 15 -- .../StaticZooKeeperConfigurationProvider.java | 55 +++++++ ...bstractZooKeeperConfigurationProvider.java | 136 ++++++++++++++++++ ...PollingZooKeeperConfigurationProvider.java | 95 ++++++++++++ ...tStaticZooKeeperConfigurationProvider.java | 44 ++++++ pom.xml | 20 +++ 12 files changed, 722 insertions(+), 79 deletions(-) create mode 100644 flume-ng-node/src/main/java/org/apache/flume/node/AbstractZooKeeperConfigurationProvider.java create mode 100644 flume-ng-node/src/main/java/org/apache/flume/node/PollingZooKeeperConfigurationProvider.java create mode 100644 flume-ng-node/src/main/java/org/apache/flume/node/StaticZooKeeperConfigurationProvider.java create mode 100644 flume-ng-node/src/test/java/org/apache/flume/node/TestAbstractZooKeeperConfigurationProvider.java create mode 100644 flume-ng-node/src/test/java/org/apache/flume/node/TestPollingZooKeeperConfigurationProvider.java create mode 100644 flume-ng-node/src/test/java/org/apache/flume/node/TestStaticZooKeeperConfigurationProvider.java diff --git a/bin/flume-ng b/bin/flume-ng index e09e26b584..4b323a6f1c 100755 --- a/bin/flume-ng +++ b/bin/flume-ng @@ -174,25 +174,28 @@ display_help() { Usage: $0 [options]... commands: - help display this help text - agent run a Flume agent - avro-client run an avro Flume client - version show Flume version info + help display this help text + agent run a Flume agent + avro-client run an avro Flume client + version show Flume version info global options: - --conf,-c use configs in directory - --classpath,-C append to the classpath - --dryrun,-d do not actually start Flume, just print the command - --plugins-path colon-separated list of plugins.d directories. See the - plugins.d section in the user guide for more details. - Default: \$FLUME_HOME/plugins.d - -Dproperty=value sets a Java system property value - -Xproperty=value sets a Java -X option + --conf,-c use configs in directory + --classpath,-C append to the classpath + --dryrun,-d do not actually start Flume, just print the command + --plugins-path colon-separated list of plugins.d directories. See the + plugins.d section in the user guide for more details. + Default: \$FLUME_HOME/plugins.d + -Dproperty=value sets a Java system property value + -Xproperty=value sets a Java -X option agent options: - --conf-file,-f specify a config file (required) - --name,-n the name of this agent (required) - --help,-h display help text + --name,-n the name of this agent (required) + --conf-file,-f specify a config file (required if -z missing) + --zkConnString,-z specify the ZooKeeper connection to use (required if -f missing) + --zkBasePath,-p specify the base path in ZooKeeper for agent configs + --no-reload-conf do not reload config file if changed + --help,-h display help text avro-client options: --rpcProps,-P RPC client properties file with server connection params diff --git a/flume-ng-node/pom.xml b/flume-ng-node/pom.xml index dce2527689..caf2c1121f 100644 --- a/flume-ng-node/pom.xml +++ b/flume-ng-node/pom.xml @@ -154,6 +154,21 @@ jackson-mapper-asl + + org.apache.curator + curator-framework + + + + org.apache.curator + curator-recipes + + + + org.apache.curator + curator-test + + diff --git a/flume-ng-node/src/main/java/org/apache/flume/node/AbstractConfigurationProvider.java b/flume-ng-node/src/main/java/org/apache/flume/node/AbstractConfigurationProvider.java index e63c601944..40abba2886 100644 --- a/flume-ng-node/src/main/java/org/apache/flume/node/AbstractConfigurationProvider.java +++ b/flume-ng-node/src/main/java/org/apache/flume/node/AbstractConfigurationProvider.java @@ -17,13 +17,8 @@ */ package org.apache.flume.node; -import java.util.ArrayList; -import java.util.HashMap; -import java.util.HashSet; -import java.util.List; -import java.util.Map; +import java.util.*; import java.util.Map.Entry; -import java.util.Set; import org.apache.flume.Channel; import org.apache.flume.ChannelFactory; @@ -506,4 +501,15 @@ private static class ChannelComponent { components = Lists.newArrayList(); } } + + protected Map toMap(Properties properties) { + Map result = Maps.newHashMap(); + Enumeration propertyNames = properties.propertyNames(); + while (propertyNames.hasMoreElements()) { + String name = (String) propertyNames.nextElement(); + String value = properties.getProperty(name); + result.put(name, value); + } + return result; + } } \ No newline at end of file diff --git a/flume-ng-node/src/main/java/org/apache/flume/node/AbstractZooKeeperConfigurationProvider.java b/flume-ng-node/src/main/java/org/apache/flume/node/AbstractZooKeeperConfigurationProvider.java new file mode 100644 index 0000000000..f193f9f90a --- /dev/null +++ b/flume-ng-node/src/main/java/org/apache/flume/node/AbstractZooKeeperConfigurationProvider.java @@ -0,0 +1,104 @@ +/** + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.flume.node; + +import java.io.IOException; +import java.io.StringReader; +import java.util.Collections; +import java.util.Map; +import java.util.Properties; + +import com.google.common.base.Charsets; +import org.apache.curator.framework.CuratorFramework; +import org.apache.curator.framework.CuratorFrameworkFactory; +import org.apache.curator.retry.ExponentialBackoffRetry; +import org.apache.flume.conf.FlumeConfiguration; + +import com.google.common.base.Preconditions; +import com.google.common.base.Strings; + +/** + * ZooKeeper based configuration implementation provider. + * + * The Agent configuration can be uploaded in ZooKeeper under a base name, which + * defaults to /flume + * + * Currently the agent configuration is stored under the agent name node in + * ZooKeeper + * + *
    + *   /flume
    + *       /a1 [agent config file]
    + *       /a2 [agent config file]
    + *       /a3 [agent config file]
    + * 
    + * + * Configuration format is same as PropertiesFileConfigurationProvider + * + * Configuration properties + * + * agentName - Name of Agent for which configuration needs to be pulled + * + * zkConnString - Connection string to ZooKeeper Ensemble + * (host:port,host1:port1) + * + * basePath - Base Path where agent configuration needs to be stored. Defaults + * to /flume + */ +public abstract class AbstractZooKeeperConfigurationProvider extends + AbstractConfigurationProvider { + + static final String DEFAULT_ZK_BASE_PATH = "/flume"; + + protected final String basePath; + + protected final String zkConnString; + + protected AbstractZooKeeperConfigurationProvider(String agentName, + String zkConnString, String basePath) { + super(agentName); + Preconditions.checkArgument(!Strings.isNullOrEmpty(zkConnString), + "Invalid Zookeeper Connection String %s", zkConnString); + this.zkConnString = zkConnString; + if (basePath == null || basePath.isEmpty()) { + this.basePath = DEFAULT_ZK_BASE_PATH; + } else { + this.basePath = basePath; + } + } + + protected CuratorFramework createClient() { + return CuratorFrameworkFactory.newClient(zkConnString, + new ExponentialBackoffRetry(1000, 1)); + } + + protected FlumeConfiguration configFromBytes(byte[] configData) + throws IOException { + Map configMap; + if (configData == null || configData.length == 0) { + configMap = Collections.emptyMap(); + } else { + String fileContent = new String(configData, Charsets.UTF_8); + Properties properties = new Properties(); + properties.load(new StringReader(fileContent)); + configMap = toMap(properties); + } + return new FlumeConfiguration(configMap); + } +} diff --git a/flume-ng-node/src/main/java/org/apache/flume/node/Application.java b/flume-ng-node/src/main/java/org/apache/flume/node/Application.java index 5250139dc9..832285a6e4 100644 --- a/flume-ng-node/src/main/java/org/apache/flume/node/Application.java +++ b/flume-ng-node/src/main/java/org/apache/flume/node/Application.java @@ -53,7 +53,7 @@ import com.google.common.eventbus.EventBus; import com.google.common.eventbus.Subscribe; -public class Application { +public class Application { private static final Logger logger = LoggerFactory .getLogger(Application.class); @@ -69,6 +69,7 @@ public class Application { public Application() { this(new ArrayList(0)); } + public Application(List components) { this.components = components; supervisor = new LifecycleSupervisor(); @@ -81,7 +82,6 @@ public synchronized void start() { } } - @Subscribe public synchronized void handleConfigurationEvent(MaterializedConfiguration conf) { stopAllComponents(); @@ -95,7 +95,6 @@ public synchronized void stop() { } } - private void stopAllComponents() { if (this.materializedConfiguration != null) { logger.info("Shutting down configuration: {}", this.materializedConfiguration); @@ -192,7 +191,6 @@ private void startAllComponents(MaterializedConfiguration materializedConfigurat this.loadMonitoring(); } - @SuppressWarnings("unchecked") private void loadMonitoring() { Properties systemProps = System.getProperties(); @@ -231,18 +229,32 @@ public static void main(String[] args) { try { + boolean isZkConfigured = false; + Options options = new Options(); Option option = new Option("n", "name", true, "the name of this agent"); option.setRequired(true); options.addOption(option); - option = new Option("f", "conf-file", true, "specify a conf file"); - option.setRequired(true); + option = new Option("f", "conf-file", true, + "specify a config file (required if -z missing)"); + option.setRequired(false); options.addOption(option); - option = new Option(null, "no-reload-conf", false, "do not reload " + - "conf file if changed"); + option = new Option(null, "no-reload-conf", false, + "do not reload config file if changed"); + options.addOption(option); + + // Options for Zookeeper + option = new Option("z", "zkConnString", true, + "specify the ZooKeeper connection to use (required if -f missing)"); + option.setRequired(false); + options.addOption(option); + + option = new Option("p", "zkBasePath", true, + "specify the base path in ZooKeeper for agent configs"); + option.setRequired(false); options.addOption(option); option = new Option("h", "help", false, "display help text"); @@ -251,47 +263,80 @@ public static void main(String[] args) { CommandLineParser parser = new GnuParser(); CommandLine commandLine = parser.parse(options, args); - File configurationFile = new File(commandLine.getOptionValue('f')); - String agentName = commandLine.getOptionValue('n'); - boolean reload = !commandLine.hasOption("no-reload-conf"); - if (commandLine.hasOption('h')) { new HelpFormatter().printHelp("flume-ng agent", options, true); return; } - /* - * The following is to ensure that by default the agent - * will fail on startup if the file does not exist. - */ - if (!configurationFile.exists()) { - // If command line invocation, then need to fail fast - if (System.getProperty(Constants.SYSPROP_CALLED_FROM_SERVICE) == null) { - String path = configurationFile.getPath(); - try { - path = configurationFile.getCanonicalPath(); - } catch (IOException ex) { - logger.error("Failed to read canonical path for file: " + path, ex); - } - throw new ParseException( - "The specified configuration file does not exist: " + path); - } + + String agentName = commandLine.getOptionValue('n'); + boolean reload = !commandLine.hasOption("no-reload-conf"); + + if (commandLine.hasOption('z') || commandLine.hasOption("zkConnString")) { + isZkConfigured = true; } - List components = Lists.newArrayList(); - Application application; - if(reload) { - EventBus eventBus = new EventBus(agentName + "-event-bus"); - PollingPropertiesFileConfigurationProvider configurationProvider = - new PollingPropertiesFileConfigurationProvider(agentName, - configurationFile, eventBus, 30); - components.add(configurationProvider); - application = new Application(components); - eventBus.register(application); + Application application = null; + if (isZkConfigured) { + // get options + String zkConnectionStr = commandLine.getOptionValue('z'); + String baseZkPath = commandLine.getOptionValue('p'); + + if (reload) { + EventBus eventBus = new EventBus(agentName + "-event-bus"); + List components = Lists.newArrayList(); + PollingZooKeeperConfigurationProvider zookeeperConfigurationProvider = + new PollingZooKeeperConfigurationProvider( + agentName, zkConnectionStr, baseZkPath, eventBus); + components.add(zookeeperConfigurationProvider); + application = new Application(components); + eventBus.register(application); + } else { + StaticZooKeeperConfigurationProvider zookeeperConfigurationProvider = + new StaticZooKeeperConfigurationProvider( + agentName, zkConnectionStr, baseZkPath); + application = new Application(); + application.handleConfigurationEvent(zookeeperConfigurationProvider + .getConfiguration()); + } } else { - PropertiesFileConfigurationProvider configurationProvider = - new PropertiesFileConfigurationProvider(agentName, - configurationFile); - application = new Application(); - application.handleConfigurationEvent(configurationProvider.getConfiguration()); + File configurationFile = new File(commandLine.getOptionValue('f')); + + /* + * The following is to ensure that by default the agent will fail on + * startup if the file does not exist. + */ + if (!configurationFile.exists()) { + // If command line invocation, then need to fail fast + if (System.getProperty(Constants.SYSPROP_CALLED_FROM_SERVICE) == + null) { + String path = configurationFile.getPath(); + try { + path = configurationFile.getCanonicalPath(); + } catch (IOException ex) { + logger.error("Failed to read canonical path for file: " + path, + ex); + } + throw new ParseException( + "The specified configuration file does not exist: " + path); + } + } + List components = Lists.newArrayList(); + + if (reload) { + EventBus eventBus = new EventBus(agentName + "-event-bus"); + PollingPropertiesFileConfigurationProvider configurationProvider = + new PollingPropertiesFileConfigurationProvider( + agentName, configurationFile, eventBus, 30); + components.add(configurationProvider); + application = new Application(components); + eventBus.register(application); + } else { + PropertiesFileConfigurationProvider configurationProvider = + new PropertiesFileConfigurationProvider( + agentName, configurationFile); + application = new Application(); + application.handleConfigurationEvent(configurationProvider + .getConfiguration()); + } } application.start(); diff --git a/flume-ng-node/src/main/java/org/apache/flume/node/PollingZooKeeperConfigurationProvider.java b/flume-ng-node/src/main/java/org/apache/flume/node/PollingZooKeeperConfigurationProvider.java new file mode 100644 index 0000000000..b950b3d3ac --- /dev/null +++ b/flume-ng-node/src/main/java/org/apache/flume/node/PollingZooKeeperConfigurationProvider.java @@ -0,0 +1,135 @@ +/** + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.flume.node; + +import java.io.IOException; + +import org.apache.curator.framework.CuratorFramework; +import org.apache.curator.framework.recipes.cache.ChildData; +import org.apache.curator.framework.recipes.cache.NodeCache; +import org.apache.curator.framework.recipes.cache.NodeCacheListener; +import org.apache.flume.FlumeException; +import org.apache.flume.conf.FlumeConfiguration; +import org.apache.flume.lifecycle.LifecycleAware; +import org.apache.flume.lifecycle.LifecycleState; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +import com.google.common.eventbus.EventBus; + +public class PollingZooKeeperConfigurationProvider extends + AbstractZooKeeperConfigurationProvider implements LifecycleAware { + + private static final Logger LOGGER = LoggerFactory + .getLogger(PollingZooKeeperConfigurationProvider.class); + + private final EventBus eventBus; + + private final CuratorFramework client; + + private NodeCache agentNodeCache; + + private FlumeConfiguration flumeConfiguration; + + private LifecycleState lifecycleState; + + public PollingZooKeeperConfigurationProvider(String agentName, + String zkConnString, String basePath, EventBus eventBus) { + super(agentName, zkConnString, basePath); + this.eventBus = eventBus; + client = createClient(); + agentNodeCache = null; + flumeConfiguration = null; + lifecycleState = LifecycleState.IDLE; + } + + @Override + protected FlumeConfiguration getFlumeConfiguration() { + return flumeConfiguration; + } + + @Override + public void start() { + LOGGER.debug("Starting..."); + try { + client.start(); + try { + agentNodeCache = new NodeCache(client, basePath + "/" + getAgentName()); + agentNodeCache.start(); + agentNodeCache.getListenable().addListener(new NodeCacheListener() { + @Override + public void nodeChanged() throws Exception { + refreshConfiguration(); + } + }); + } catch (Exception e) { + client.close(); + throw e; + } + } catch (Exception e) { + lifecycleState = LifecycleState.ERROR; + if (e instanceof RuntimeException) { + throw (RuntimeException) e; + } else { + throw new FlumeException(e); + } + } + lifecycleState = LifecycleState.START; + } + + private void refreshConfiguration() throws IOException { + LOGGER.info("Refreshing configuration from ZooKeeper"); + byte[] data = null; + ChildData childData = agentNodeCache.getCurrentData(); + if (childData != null) { + data = childData.getData(); + } + flumeConfiguration = configFromBytes(data); + eventBus.post(getConfiguration()); + } + + @Override + public void stop() { + LOGGER.debug("Stopping..."); + if (agentNodeCache != null) { + try { + agentNodeCache.close(); + } catch (IOException e) { + LOGGER.warn("Encountered exception while stopping", e); + lifecycleState = LifecycleState.ERROR; + } + } + + try { + client.close(); + } catch (Exception e) { + LOGGER.warn("Error stopping Curator client", e); + lifecycleState = LifecycleState.ERROR; + } + + if (lifecycleState != LifecycleState.ERROR) { + lifecycleState = LifecycleState.STOP; + } + } + + @Override + public LifecycleState getLifecycleState() { + return lifecycleState; + } +} diff --git a/flume-ng-node/src/main/java/org/apache/flume/node/PropertiesFileConfigurationProvider.java b/flume-ng-node/src/main/java/org/apache/flume/node/PropertiesFileConfigurationProvider.java index d7438d9662..bc5438ad3b 100644 --- a/flume-ng-node/src/main/java/org/apache/flume/node/PropertiesFileConfigurationProvider.java +++ b/flume-ng-node/src/main/java/org/apache/flume/node/PropertiesFileConfigurationProvider.java @@ -21,17 +21,13 @@ import java.io.File; import java.io.FileReader; import java.io.IOException; -import java.util.Enumeration; import java.util.HashMap; -import java.util.Map; import java.util.Properties; import org.apache.flume.conf.FlumeConfiguration; import org.slf4j.Logger; import org.slf4j.LoggerFactory; -import com.google.common.collect.Maps; - /** *

    * A configuration provider that uses properties file for specifying @@ -206,15 +202,4 @@ public FlumeConfiguration getFlumeConfiguration() { } return new FlumeConfiguration(new HashMap()); } - - private Map toMap(Properties properties) { - Map result = Maps.newHashMap(); - Enumeration propertyNames = properties.propertyNames(); - while (propertyNames.hasMoreElements()) { - String name = (String) propertyNames.nextElement(); - String value = properties.getProperty(name); - result.put(name, value); - } - return result; - } } diff --git a/flume-ng-node/src/main/java/org/apache/flume/node/StaticZooKeeperConfigurationProvider.java b/flume-ng-node/src/main/java/org/apache/flume/node/StaticZooKeeperConfigurationProvider.java new file mode 100644 index 0000000000..551e9dd53a --- /dev/null +++ b/flume-ng-node/src/main/java/org/apache/flume/node/StaticZooKeeperConfigurationProvider.java @@ -0,0 +1,55 @@ +/** + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.flume.node; + +import org.apache.curator.framework.CuratorFramework; +import org.apache.flume.FlumeException; +import org.apache.flume.conf.FlumeConfiguration; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +public class StaticZooKeeperConfigurationProvider extends + AbstractZooKeeperConfigurationProvider { + + private static final Logger LOGGER = LoggerFactory + .getLogger(StaticZooKeeperConfigurationProvider.class); + + public StaticZooKeeperConfigurationProvider(String agentName, + String zkConnString, String basePath) { + super(agentName, zkConnString, basePath); + } + + @Override + protected FlumeConfiguration getFlumeConfiguration() { + try { + CuratorFramework cf = createClient(); + cf.start(); + try { + byte[] data = cf.getData().forPath(basePath + "/" + getAgentName()); + return configFromBytes(data); + } finally { + cf.close(); + } + } catch (Exception e) { + LOGGER.error("Error getting configuration info from Zookeeper", e); + throw new FlumeException(e); + } + } + +} diff --git a/flume-ng-node/src/test/java/org/apache/flume/node/TestAbstractZooKeeperConfigurationProvider.java b/flume-ng-node/src/test/java/org/apache/flume/node/TestAbstractZooKeeperConfigurationProvider.java new file mode 100644 index 0000000000..1ab412753e --- /dev/null +++ b/flume-ng-node/src/test/java/org/apache/flume/node/TestAbstractZooKeeperConfigurationProvider.java @@ -0,0 +1,136 @@ +/** + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.flume.node; + +import java.io.InputStreamReader; +import java.io.Reader; +import java.util.Collections; +import java.util.List; +import java.util.Set; + +import com.google.common.base.Charsets; +import junit.framework.Assert; + +import org.apache.commons.io.IOUtils; +import org.apache.curator.framework.CuratorFramework; +import org.apache.curator.framework.CuratorFrameworkFactory; +import org.apache.curator.retry.ExponentialBackoffRetry; +import org.apache.curator.test.TestingServer; +import org.apache.curator.utils.EnsurePath; +import org.apache.flume.conf.FlumeConfiguration; +import org.apache.flume.conf.FlumeConfigurationError; +import org.junit.After; +import org.junit.Before; + +import com.google.common.collect.Lists; +import com.google.common.collect.Sets; + +public abstract class TestAbstractZooKeeperConfigurationProvider { + + private static final String FLUME_CONF_FILE = "flume-conf.properties"; + + protected static final String AGENT_NAME = "a1"; + + protected static final String AGENT_PATH = + AbstractZooKeeperConfigurationProvider.DEFAULT_ZK_BASE_PATH + + "/" + AGENT_NAME; + + protected TestingServer zkServer; + protected CuratorFramework client; + + @Before + public void setUp() throws Exception { + zkServer = new TestingServer(); + client = CuratorFrameworkFactory + .newClient("localhost:" + zkServer.getPort(), + new ExponentialBackoffRetry(1000, 3)); + client.start(); + + EnsurePath ensurePath = new EnsurePath(AGENT_PATH); + ensurePath.ensure(client.getZookeeperClient()); + doSetUp(); + } + + protected abstract void doSetUp() throws Exception; + + @After + public void tearDown() throws Exception { + doTearDown(); + zkServer.close(); + client.close(); + } + + protected abstract void doTearDown() throws Exception; + + protected void addData() throws Exception { + Reader in = new InputStreamReader(getClass().getClassLoader() + .getResourceAsStream(FLUME_CONF_FILE), Charsets.UTF_8); + try { + String config = IOUtils.toString(in); + client.setData().forPath(AGENT_PATH, config.getBytes()); + } finally { + in.close(); + } + } + + protected void verifyProperties(AbstractConfigurationProvider cp) { + FlumeConfiguration configuration = cp.getFlumeConfiguration(); + Assert.assertNotNull(configuration); + + /* + * Test the known errors in the file + */ + List expected = Lists.newArrayList(); + expected.add("host5 CONFIG_ERROR"); + expected.add("host5 INVALID_PROPERTY"); + expected.add("host4 CONFIG_ERROR"); + expected.add("host4 CONFIG_ERROR"); + expected.add("host4 PROPERTY_VALUE_NULL"); + expected.add("host4 PROPERTY_VALUE_NULL"); + expected.add("host4 PROPERTY_VALUE_NULL"); + expected.add("host4 AGENT_CONFIGURATION_INVALID"); + expected.add("ch2 ATTRS_MISSING"); + expected.add("host3 CONFIG_ERROR"); + expected.add("host3 PROPERTY_VALUE_NULL"); + expected.add("host3 AGENT_CONFIGURATION_INVALID"); + expected.add("host2 PROPERTY_VALUE_NULL"); + expected.add("host2 AGENT_CONFIGURATION_INVALID"); + List actual = Lists.newArrayList(); + for (FlumeConfigurationError error : configuration + .getConfigurationErrors()) { + actual.add(error.getComponentName() + " " + + error.getErrorType().toString()); + } + Collections.sort(expected); + Collections.sort(actual); + Assert.assertEquals(expected, actual); + + FlumeConfiguration.AgentConfiguration agentConfiguration = configuration + .getConfigurationFor("host1"); + Assert.assertNotNull(agentConfiguration); + + Set sources = Sets.newHashSet("source1"); + Set sinks = Sets.newHashSet("sink1"); + Set channels = Sets.newHashSet("channel1"); + + Assert.assertEquals(sources, agentConfiguration.getSourceSet()); + Assert.assertEquals(sinks, agentConfiguration.getSinkSet()); + Assert.assertEquals(channels, agentConfiguration.getChannelSet()); + } +} diff --git a/flume-ng-node/src/test/java/org/apache/flume/node/TestPollingZooKeeperConfigurationProvider.java b/flume-ng-node/src/test/java/org/apache/flume/node/TestPollingZooKeeperConfigurationProvider.java new file mode 100644 index 0000000000..e59a4380dc --- /dev/null +++ b/flume-ng-node/src/test/java/org/apache/flume/node/TestPollingZooKeeperConfigurationProvider.java @@ -0,0 +1,95 @@ +/** + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.flume.node; + +import junit.framework.Assert; + +import org.apache.flume.conf.FlumeConfiguration; +import org.apache.flume.conf.FlumeConfiguration.AgentConfiguration; +import org.apache.flume.lifecycle.LifecycleController; +import org.apache.flume.lifecycle.LifecycleState; +import org.junit.Test; + +import com.google.common.eventbus.EventBus; +import com.google.common.eventbus.Subscribe; + +public class TestPollingZooKeeperConfigurationProvider extends + TestAbstractZooKeeperConfigurationProvider { + + private EventBus eb; + + private EventSync es; + + private PollingZooKeeperConfigurationProvider cp; + + private class EventSync { + + private boolean notified; + + @Subscribe + public synchronized void notifyEvent(MaterializedConfiguration mConfig) { + notified = true; + notifyAll(); + } + + public synchronized void awaitEvent() throws InterruptedException { + while (!notified) { + wait(); + } + } + + public synchronized void reset() { + notified = false; + } + } + + @Override + protected void doSetUp() throws Exception { + eb = new EventBus("test"); + es = new EventSync(); + es.reset(); + eb.register(es); + cp = new PollingZooKeeperConfigurationProvider(AGENT_NAME, "localhost:" + + zkServer.getPort(), null, eb); + cp.start(); + LifecycleController.waitForOneOf(cp, LifecycleState.START_OR_ERROR); + } + + @Override + protected void doTearDown() throws Exception { + // do nothing + } + + @Test + public void testPolling() throws Exception { + es.awaitEvent(); + es.reset(); + + FlumeConfiguration fc = cp.getFlumeConfiguration(); + Assert.assertTrue(fc.getConfigurationErrors().isEmpty()); + AgentConfiguration ac = fc.getConfigurationFor(AGENT_NAME); + Assert.assertNull(ac); + + addData(); + es.awaitEvent(); + es.reset(); + + verifyProperties(cp); + } +} diff --git a/flume-ng-node/src/test/java/org/apache/flume/node/TestStaticZooKeeperConfigurationProvider.java b/flume-ng-node/src/test/java/org/apache/flume/node/TestStaticZooKeeperConfigurationProvider.java new file mode 100644 index 0000000000..dddcffe890 --- /dev/null +++ b/flume-ng-node/src/test/java/org/apache/flume/node/TestStaticZooKeeperConfigurationProvider.java @@ -0,0 +1,44 @@ +/** + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.flume.node; + +import org.junit.Test; + +public class TestStaticZooKeeperConfigurationProvider extends + TestAbstractZooKeeperConfigurationProvider { + + private StaticZooKeeperConfigurationProvider configurationProvider; + + @Override + protected void doSetUp() throws Exception { + addData(); + configurationProvider = new StaticZooKeeperConfigurationProvider( + AGENT_NAME, "localhost:" + zkServer.getPort(), null); + } + + @Override + protected void doTearDown() throws Exception { + // do nothing + } + + @Test + public void testPropertyRead() throws Exception { + verifyProperties(configurationProvider); + } +} \ No newline at end of file diff --git a/pom.xml b/pom.xml index 4bdfcac54f..150db2ed87 100644 --- a/pom.xml +++ b/pom.xml @@ -1276,6 +1276,26 @@ limitations under the License. 1.1.0 + + + org.apache.curator + curator-framework + 2.3.0 + + + + org.apache.curator + curator-recipes + 2.3.0 + + + + org.apache.curator + curator-test + 2.3.0 + test + + From 72be82d301895c214a10dd875260ddc308048623 Mon Sep 17 00:00:00 2001 From: Hari Shreedharan Date: Fri, 12 Sep 2014 13:23:16 -0700 Subject: [PATCH 135/341] FLUME-2337. export JAVA_HOME in flume-env.sh.template and increase heap size (Roshan Naik via Hari) --- conf/flume-env.sh.template | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/conf/flume-env.sh.template b/conf/flume-env.sh.template index 3012814268..c8b660f171 100644 --- a/conf/flume-env.sh.template +++ b/conf/flume-env.sh.template @@ -19,10 +19,10 @@ # Enviroment variables can be set here. -#JAVA_HOME=/usr/lib/jvm/java-6-sun +# export JAVA_HOME=/usr/lib/jvm/java-6-sun # Give Flume more memory and pre-allocate, enable remote monitoring via JMX -#JAVA_OPTS="-Xms100m -Xmx200m -Dcom.sun.management.jmxremote" +# export JAVA_OPTS="-Xms100m -Xmx2000m -Dcom.sun.management.jmxremote" # Note that the Flume conf directory is always included in the classpath. #FLUME_CLASSPATH="" From acc9651346fe5834cdf5cdf0eb417f624aab1d09 Mon Sep 17 00:00:00 2001 From: Hari Shreedharan Date: Sat, 13 Sep 2014 14:59:37 -0700 Subject: [PATCH 136/341] FLUME-2462. Remove use of deprecated methods in DatasetSink (Ryan Blue via Hari) --- .../apache/flume/sink/kite/DatasetSink.java | 5 +- .../flume/sink/kite/TestDatasetSink.java | 87 +++++++++---------- 2 files changed, 45 insertions(+), 47 deletions(-) diff --git a/flume-ng-sinks/flume-dataset-sink/src/main/java/org/apache/flume/sink/kite/DatasetSink.java b/flume-ng-sinks/flume-dataset-sink/src/main/java/org/apache/flume/sink/kite/DatasetSink.java index 8f3ae519c5..4cd3027d03 100644 --- a/flume-ng-sinks/flume-dataset-sink/src/main/java/org/apache/flume/sink/kite/DatasetSink.java +++ b/flume-ng-sinks/flume-dataset-sink/src/main/java/org/apache/flume/sink/kite/DatasetSink.java @@ -52,9 +52,9 @@ import org.apache.hadoop.fs.Path; import org.apache.hadoop.security.UserGroupInformation; import org.kitesdk.data.Dataset; -import org.kitesdk.data.DatasetRepositories; import org.kitesdk.data.DatasetWriter; import org.kitesdk.data.Datasets; +import org.kitesdk.data.spi.URIBuilder; import org.slf4j.Logger; import org.slf4j.LoggerFactory; @@ -172,7 +172,8 @@ public Dataset run() { new PrivilegedExceptionAction>() { @Override public Dataset run() { - return DatasetRepositories.open(repositoryURI).load(datasetName); + return Datasets.load( + new URIBuilder(repositoryURI, datasetName).build()); } }); } diff --git a/flume-ng-sinks/flume-dataset-sink/src/test/java/org/apache/flume/sink/kite/TestDatasetSink.java b/flume-ng-sinks/flume-dataset-sink/src/test/java/org/apache/flume/sink/kite/TestDatasetSink.java index b448b5018d..a277381fc8 100644 --- a/flume-ng-sinks/flume-dataset-sink/src/test/java/org/apache/flume/sink/kite/TestDatasetSink.java +++ b/flume-ng-sinks/flume-dataset-sink/src/test/java/org/apache/flume/sink/kite/TestDatasetSink.java @@ -28,6 +28,7 @@ import java.io.File; import java.io.FileWriter; import java.io.IOException; +import java.net.URI; import java.util.Arrays; import java.util.HashSet; import java.util.List; @@ -62,10 +63,9 @@ import org.kitesdk.data.Dataset; import org.kitesdk.data.DatasetDescriptor; import org.kitesdk.data.DatasetReader; -import org.kitesdk.data.DatasetRepositories; -import org.kitesdk.data.DatasetRepository; import org.kitesdk.data.Datasets; import org.kitesdk.data.PartitionStrategy; +import org.kitesdk.data.View; public class TestDatasetSink { @@ -73,8 +73,6 @@ public class TestDatasetSink { public static final String DATASET_NAME = "test"; public static final String FILE_DATASET_URI = "dataset:file:target/test-repo/" + DATASET_NAME; - public static final DatasetRepository REPO = DatasetRepositories - .open(FILE_REPO_URI); public static final File SCHEMA_FILE = new File("target/record-schema.avsc"); public static final Schema RECORD_SCHEMA = new Schema.Parser().parse( "{\"type\":\"record\",\"name\":\"rec\",\"fields\":[" + @@ -94,7 +92,7 @@ public class TestDatasetSink { Context config = null; Channel in = null; - List expected = null; + List expected = null; private static final String DFS_DIR = "target/test/dfs"; private static final String TEST_BUILD_DATA_KEY = "test.build.data"; private static String oldTestBuildDataProp = null; @@ -118,8 +116,8 @@ public static void tearDownClass() { @Before public void setup() throws EventDeliveryException { - REPO.delete(DATASET_NAME); - REPO.create(DATASET_NAME, DESCRIPTOR); + Datasets.delete(FILE_DATASET_URI); + Datasets.create(FILE_DATASET_URI, DESCRIPTOR); this.config = new Context(); this.in = new MemoryChannel(); @@ -128,17 +126,17 @@ public void setup() throws EventDeliveryException { config.put(DatasetSinkConstants.CONFIG_KITE_DATASET_URI, FILE_DATASET_URI); GenericRecordBuilder builder = new GenericRecordBuilder(RECORD_SCHEMA); - expected = Lists.newArrayList( + expected = Lists.newArrayList( builder.set("id", "1").set("msg", "msg1").build(), builder.set("id", "2").set("msg", "msg2").build(), builder.set("id", "3").set("msg", "msg3").build()); putToChannel(in, Iterables.transform(expected, - new Function() { + new Function() { private int i = 0; @Override - public Event apply(@Nullable GenericData.Record rec) { + public Event apply(@Nullable GenericRecord rec) { this.i += 1; boolean useURI = (i % 2) == 0; return event(rec, RECORD_SCHEMA, SCHEMA_FILE, useURI); @@ -148,7 +146,7 @@ public Event apply(@Nullable GenericData.Record rec) { @After public void teardown() { - REPO.delete(DATASET_NAME); + Datasets.delete(FILE_DATASET_URI); } @Test @@ -166,7 +164,7 @@ public void testOldConfig() throws EventDeliveryException { Assert.assertEquals( Sets.newHashSet(expected), - read(REPO.load(DATASET_NAME))); + read(Datasets.load(FILE_DATASET_URI))); Assert.assertEquals("Should have committed", 0, remaining(in)); } @@ -185,7 +183,7 @@ public void testDatasetUriOverridesOldConfig() throws EventDeliveryException { Assert.assertEquals( Sets.newHashSet(expected), - read(REPO.load(DATASET_NAME))); + read(Datasets.load(FILE_DATASET_URI))); Assert.assertEquals("Should have committed", 0, remaining(in)); } @@ -200,18 +198,17 @@ public void testFileStore() throws EventDeliveryException { Assert.assertEquals( Sets.newHashSet(expected), - read(REPO.load(DATASET_NAME))); + read(Datasets.load(FILE_DATASET_URI))); Assert.assertEquals("Should have committed", 0, remaining(in)); } @Test public void testParquetDataset() throws EventDeliveryException { Datasets.delete(FILE_DATASET_URI); - Dataset created = Datasets.create(FILE_DATASET_URI, + Dataset created = Datasets.create(FILE_DATASET_URI, new DatasetDescriptor.Builder(DESCRIPTOR) .format("parquet") - .build(), - GenericData.Record.class); + .build()); DatasetSink sink = sink(in, config); @@ -226,15 +223,16 @@ public void testParquetDataset() throws EventDeliveryException { @Test public void testPartitionedData() throws EventDeliveryException { - REPO.create("partitioned", new DatasetDescriptor.Builder(DESCRIPTOR) - .partitionStrategy(new PartitionStrategy.Builder() - .identity("id", 10) // partition by id - .build()) - .build()); - + URI partitionedUri = URI.create("dataset:file:target/test-repo/partitioned"); try { + Datasets.create(partitionedUri, new DatasetDescriptor.Builder(DESCRIPTOR) + .partitionStrategy(new PartitionStrategy.Builder() + .identity("id", 10) // partition by id + .build()) + .build()); + config.put(DatasetSinkConstants.CONFIG_KITE_DATASET_URI, - "dataset:file:target/test-repo/partitioned"); + partitionedUri.toString()); DatasetSink sink = sink(in, config); // run the sink @@ -244,11 +242,11 @@ public void testPartitionedData() throws EventDeliveryException { Assert.assertEquals( Sets.newHashSet(expected), - read(REPO.load("partitioned"))); + read(Datasets.load(partitionedUri))); Assert.assertEquals("Should have committed", 0, remaining(in)); } finally { - if (REPO.exists("partitioned")) { - REPO.delete("partitioned"); + if (Datasets.exists(partitionedUri)) { + Datasets.delete(partitionedUri); } } } @@ -260,19 +258,18 @@ public void testMiniClusterStore() MiniDFSCluster cluster = new MiniDFSCluster .Builder(new Configuration()) .build(); - DatasetRepository hdfsRepo = null; - try { - FileSystem dfs = cluster.getFileSystem(); - Configuration conf = dfs.getConf(); - String repoURI = "repo:" + conf.get("fs.defaultFS") + "/tmp/repo"; + FileSystem dfs = cluster.getFileSystem(); + Configuration conf = dfs.getConf(); + + URI hdfsUri = URI.create( + "dataset:" + conf.get("fs.defaultFS") + "/tmp/repo" + DATASET_NAME); + try { // create a repository and dataset in HDFS - hdfsRepo = DatasetRepositories.open(repoURI); - hdfsRepo.create(DATASET_NAME, DESCRIPTOR); + Datasets.create(hdfsUri, DESCRIPTOR); // update the config to use the HDFS repository - config.put(DatasetSinkConstants.CONFIG_KITE_DATASET_URI, - "dataset:" + conf.get("fs.defaultFS") + "/tmp/repo/" + DATASET_NAME); + config.put(DatasetSinkConstants.CONFIG_KITE_DATASET_URI, hdfsUri.toString()); DatasetSink sink = sink(in, config); @@ -283,12 +280,12 @@ public void testMiniClusterStore() Assert.assertEquals( Sets.newHashSet(expected), - read(hdfsRepo.load(DATASET_NAME))); + read(Datasets.load(hdfsUri))); Assert.assertEquals("Should have committed", 0, remaining(in)); } finally { - if (hdfsRepo != null && hdfsRepo.exists(DATASET_NAME)) { - hdfsRepo.delete(DATASET_NAME); + if (Datasets.exists(hdfsUri)) { + Datasets.delete(hdfsUri); } cluster.shutdown(); } @@ -308,13 +305,13 @@ public void testBatchSize() throws EventDeliveryException { sink.process(); // roll and process the third Assert.assertEquals( Sets.newHashSet(expected.subList(0, 2)), - read(REPO.load(DATASET_NAME))); + read(Datasets.load(FILE_DATASET_URI))); Assert.assertEquals("Should have committed", 0, remaining(in)); sink.roll(); // roll at the next process call sink.process(); // roll, the channel is empty Assert.assertEquals( Sets.newHashSet(expected), - read(REPO.load(DATASET_NAME))); + read(Datasets.load(FILE_DATASET_URI))); sink.stop(); } @@ -326,7 +323,7 @@ public void testTimedFileRolling() DatasetSink sink = sink(in, config); - Dataset records = REPO.load(DATASET_NAME); + Dataset records = Datasets.load(FILE_DATASET_URI); // run the sink sink.start(); @@ -369,7 +366,7 @@ public void testCompatibleSchemas() throws EventDeliveryException { Assert.assertEquals( Sets.newHashSet(expected), - read(REPO.load(DATASET_NAME))); + read(Datasets.load(FILE_DATASET_URI))); Assert.assertEquals("Should have committed", 0, remaining(in)); } @@ -430,10 +427,10 @@ public static DatasetSink sink(Channel in, Context config) { return sink; } - public static HashSet read(Dataset dataset) { + public static HashSet read(View view) { DatasetReader reader = null; try { - reader = dataset.newReader(); + reader = view.newReader(); return Sets.newHashSet(reader.iterator()); } finally { if (reader != null) { From 75f748cbd101d6efe8463a1c747fb87d2f668091 Mon Sep 17 00:00:00 2001 From: Hari Shreedharan Date: Mon, 15 Sep 2014 14:26:19 -0700 Subject: [PATCH 137/341] FLUME-2251. Kafka Sink. (Thilina Buddhika, Gwen Shapira via Hari) --- flume-ng-dist/pom.xml | 4 + flume-ng-sinks/flume-ng-kafka-sink/pom.xml | 68 ++++++ .../apache/flume/sink/kafka/KafkaSink.java | 219 ++++++++++++++++++ .../flume/sink/kafka/KafkaSinkConstants.java | 31 +++ .../flume/sink/kafka/TestConstants.java | 25 ++ .../flume/sink/kafka/TestKafkaSink.java | 212 +++++++++++++++++ .../flume/sink/kafka/util/KafkaConsumer.java | 98 ++++++++ .../flume/sink/kafka/util/KafkaLocal.java | 52 +++++ .../flume/sink/kafka/util/TestUtil.java | 174 ++++++++++++++ .../flume/sink/kafka/util/ZooKeeperLocal.java | 62 +++++ .../test/resources/kafka-server.properties | 117 ++++++++++ .../src/test/resources/log4j.properties | 78 +++++++ .../src/test/resources/zookeeper.properties | 20 ++ flume-ng-sinks/pom.xml | 1 + pom.xml | 6 + 15 files changed, 1167 insertions(+) create mode 100644 flume-ng-sinks/flume-ng-kafka-sink/pom.xml create mode 100644 flume-ng-sinks/flume-ng-kafka-sink/src/main/java/org/apache/flume/sink/kafka/KafkaSink.java create mode 100644 flume-ng-sinks/flume-ng-kafka-sink/src/main/java/org/apache/flume/sink/kafka/KafkaSinkConstants.java create mode 100644 flume-ng-sinks/flume-ng-kafka-sink/src/test/java/org/apache/flume/sink/kafka/TestConstants.java create mode 100644 flume-ng-sinks/flume-ng-kafka-sink/src/test/java/org/apache/flume/sink/kafka/TestKafkaSink.java create mode 100644 flume-ng-sinks/flume-ng-kafka-sink/src/test/java/org/apache/flume/sink/kafka/util/KafkaConsumer.java create mode 100644 flume-ng-sinks/flume-ng-kafka-sink/src/test/java/org/apache/flume/sink/kafka/util/KafkaLocal.java create mode 100644 flume-ng-sinks/flume-ng-kafka-sink/src/test/java/org/apache/flume/sink/kafka/util/TestUtil.java create mode 100644 flume-ng-sinks/flume-ng-kafka-sink/src/test/java/org/apache/flume/sink/kafka/util/ZooKeeperLocal.java create mode 100644 flume-ng-sinks/flume-ng-kafka-sink/src/test/resources/kafka-server.properties create mode 100644 flume-ng-sinks/flume-ng-kafka-sink/src/test/resources/log4j.properties create mode 100644 flume-ng-sinks/flume-ng-kafka-sink/src/test/resources/zookeeper.properties diff --git a/flume-ng-dist/pom.xml b/flume-ng-dist/pom.xml index 8c18af69ac..ca3cd8b742 100644 --- a/flume-ng-dist/pom.xml +++ b/flume-ng-dist/pom.xml @@ -149,6 +149,10 @@ org.apache.flume.flume-ng-sinks flume-ng-morphline-solr-sink + + org.apache.flume.flume-ng-sinks + flume-ng-kafka-sink + org.apache.flume.flume-ng-sources flume-scribe-source diff --git a/flume-ng-sinks/flume-ng-kafka-sink/pom.xml b/flume-ng-sinks/flume-ng-kafka-sink/pom.xml new file mode 100644 index 0000000000..307fa59487 --- /dev/null +++ b/flume-ng-sinks/flume-ng-kafka-sink/pom.xml @@ -0,0 +1,68 @@ + + + + 4.0.0 + + flume-ng-sinks + org.apache.flume + 1.6.0-SNAPSHOT + + org.apache.flume.flume-ng-sinks + flume-ng-kafka-sink + Flume Kafka Sink + + + + + org.apache.rat + apache-rat-plugin + + + + + + + org.apache.flume + flume-ng-sdk + + + + org.apache.flume + flume-ng-core + + + + org.apache.flume + flume-ng-configuration + + + + org.slf4j + slf4j-api + + + + junit + junit + test + + + + org.apache.kafka + kafka_2.10 + 0.8.1.1 + + + + diff --git a/flume-ng-sinks/flume-ng-kafka-sink/src/main/java/org/apache/flume/sink/kafka/KafkaSink.java b/flume-ng-sinks/flume-ng-kafka-sink/src/main/java/org/apache/flume/sink/kafka/KafkaSink.java new file mode 100644 index 0000000000..a6121ac7e8 --- /dev/null +++ b/flume-ng-sinks/flume-ng-kafka-sink/src/main/java/org/apache/flume/sink/kafka/KafkaSink.java @@ -0,0 +1,219 @@ +/** + Licensed to the Apache Software Foundation (ASF) under one or more + contributor license agreements. See the NOTICE file distributed with + this work for additional information regarding copyright ownership. + The ASF licenses this file to You under the Apache License, Version 2.0 + (the "License"); you may not use this file except in compliance with + the License. You may obtain a copy of the License at + + http://www.apache.org/licenses/LICENSE-2.0 + + Unless required by applicable law or agreed to in writing, software + distributed under the License is distributed on an "AS IS" BASIS, + WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + See the License for the specific language governing permissions and + limitations under the License. + limitations under the License. + */ + +package org.apache.flume.sink.kafka; + +import com.google.common.base.Throwables; +import kafka.javaapi.producer.Producer; +import kafka.producer.KeyedMessage; +import kafka.producer.ProducerConfig; +import org.apache.flume.*; +import org.apache.flume.conf.Configurable; +import org.apache.flume.sink.AbstractSink; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +import java.util.ArrayList; +import java.util.List; +import java.util.Map; +import java.util.Properties; + +/** + * A Flume Sink that can publish messages to Kafka. + * This is a general implementation that can be used with any Flume agent and + * a channel. + * The message can be any event and the key is a string that we read from the + * header + * For use of partitioning, use an interceptor to generate a header with the + * partition key + *

    + * Mandatory properties are: + * kafka.metadata.broker.list -- can be a partial list, + * but at least 2 are recommended for HA + * kafka.request.required.acks -- 0 (unsafe), 1 (accepted by at least one + * broker), -1 (accepted by all brokers) + * kafka.producer.type -- for safety, this should be sync + *

    + *

    + * however, any property starting with "kafka." will be passed along to the + * Kafka producer + * Read the Kafka producer documentation to see which configurations can be used + *

    + * Optional properties + * topic - there's a default, and also - this can be in the event header if + * you need to support events with + * different topics + * batchSize - how many messages to process in one batch. Larger batches + * improve throughput while adding latency. + *

    + * header properties (per event): + * topic + * key + */ +public class KafkaSink extends AbstractSink implements Configurable { + + private static final Logger logger = LoggerFactory.getLogger(KafkaSink.class); + public static final String KEY_HDR = "key"; + public static final String TOPIC_HDR = "topic"; + private Properties producerProps; + private Producer producer; + private String topic; + private int batchSize; + private List> messageList; + + @Override + public Status process() throws EventDeliveryException { + Status result = Status.READY; + Channel channel = getChannel(); + Transaction transaction = null; + Event event = null; + String eventTopic = null; + String eventKey = null; + + try { + long processedEvents = 0; + + transaction = channel.getTransaction(); + transaction.begin(); + + messageList.clear(); + for (; processedEvents < batchSize; processedEvents += 1) { + event = channel.take(); + + if (event == null) { + // no events available in channel + break; + } + + byte[] eventBody = event.getBody(); + Map headers = event.getHeaders(); + + if ((eventTopic = headers.get(TOPIC_HDR)) == null) { + eventTopic = topic; + } + + eventKey = headers.get(KEY_HDR); + + if (logger.isDebugEnabled()) { + logger.debug("{Event} " + eventTopic + " : " + eventKey + " : " + + new String(eventBody, "UTF-8")); + logger.debug("event #{}", processedEvents); + } + + // create a message and add to buffer + KeyedMessage data = new KeyedMessage + (eventTopic, eventKey, eventBody); + messageList.add(data); + + } + + // publish batch and commit. + if (processedEvents > 0) { + producer.send(messageList); + } + + transaction.commit(); + + } catch (Exception ex) { + String errorMsg = "Failed to publish events"; + logger.error("Failed to publish events", ex); + result = Status.BACKOFF; + if (transaction != null) { + try { + transaction.rollback(); + } catch (Exception e) { + logger.error("Transaction rollback failed", e); + throw Throwables.propagate(e); + } + } + throw new EventDeliveryException(errorMsg, ex); + } finally { + if (transaction != null) { + transaction.close(); + } + } + + return result; + } + + @Override + public synchronized void start() { + // instantiate the producer + ProducerConfig config = new ProducerConfig(producerProps); + producer = new Producer(config); + super.start(); + } + + @Override + public synchronized void stop() { + producer.close(); + super.stop(); + } + + + @Override + public void configure(Context context) { + + batchSize = context.getInteger(KafkaSinkConstants.BATCH_SIZE, + KafkaSinkConstants.DEFAULT_BATCH_SIZE); + logger.debug("Using batch size: {}", batchSize); + messageList = + new ArrayList>(batchSize); + Map params = context.getParameters(); + logger.debug("all params: " + params.entrySet().toString()); + setProducerProps(params); + if (!producerProps.contains("serializer.class")) { + producerProps.put("serializer.class", "kafka.serializer.DefaultEncoder"); + } + if (!producerProps.contains("key.serializer.class")) { + producerProps.put("key.serializer.class", + "kafka.serializer.StringEncoder"); + } + + topic = context.getString(KafkaSinkConstants.TOPIC, + KafkaSinkConstants.DEFAULT_TOPIC); + if (topic.equals(KafkaSinkConstants.DEFAULT_TOPIC)) { + logger.warn("The Properties 'preprocessor' or 'topic' is not set. " + + "Using the default topic name" + + KafkaSinkConstants.DEFAULT_TOPIC); + } else { + logger.info("Using the static topic: " + topic + + " this may be over-ridden by event headers"); + } + } + + + private void setProducerProps(Map params) { + producerProps = new Properties(); + for (String key : params.keySet()) { + String value = params.get(key).trim(); + key = key.trim(); + if (key.startsWith(KafkaSinkConstants.PROPERTY_PREFIX)) { + // remove the prefix + key = key.substring(KafkaSinkConstants.PROPERTY_PREFIX.length() + 1, + key.length()); + producerProps.put(key.trim(), value); + if (logger.isDebugEnabled()) { + logger.debug("Reading a Kafka Producer Property: key: " + key + + ", value: " + value); + } + } + } + } + +} diff --git a/flume-ng-sinks/flume-ng-kafka-sink/src/main/java/org/apache/flume/sink/kafka/KafkaSinkConstants.java b/flume-ng-sinks/flume-ng-kafka-sink/src/main/java/org/apache/flume/sink/kafka/KafkaSinkConstants.java new file mode 100644 index 0000000000..48d875ebfb --- /dev/null +++ b/flume-ng-sinks/flume-ng-kafka-sink/src/main/java/org/apache/flume/sink/kafka/KafkaSinkConstants.java @@ -0,0 +1,31 @@ +/** + Licensed to the Apache Software Foundation (ASF) under one or more + contributor license agreements. See the NOTICE file distributed with + this work for additional information regarding copyright ownership. + The ASF licenses this file to You under the Apache License, Version 2.0 + (the "License"); you may not use this file except in compliance with + the License. You may obtain a copy of the License at + + http://www.apache.org/licenses/LICENSE-2.0 + + Unless required by applicable law or agreed to in writing, software + distributed under the License is distributed on an "AS IS" BASIS, + WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + See the License for the specific language governing permissions and + limitations under the License. + limitations under the License. + */ + +package org.apache.flume.sink.kafka; + +public class KafkaSinkConstants { + + public static final String PROPERTY_PREFIX = "kafka"; + + /* Properties */ + public static final String DEFAULT_TOPIC = "default-flume-topic"; + public static final String TOPIC = "topic"; + public static final String BATCH_SIZE = "batchSize"; + + public static final int DEFAULT_BATCH_SIZE = 100; +} diff --git a/flume-ng-sinks/flume-ng-kafka-sink/src/test/java/org/apache/flume/sink/kafka/TestConstants.java b/flume-ng-sinks/flume-ng-kafka-sink/src/test/java/org/apache/flume/sink/kafka/TestConstants.java new file mode 100644 index 0000000000..f99be539e3 --- /dev/null +++ b/flume-ng-sinks/flume-ng-kafka-sink/src/test/java/org/apache/flume/sink/kafka/TestConstants.java @@ -0,0 +1,25 @@ +/** + Licensed to the Apache Software Foundation (ASF) under one or more + contributor license agreements. See the NOTICE file distributed with + this work for additional information regarding copyright ownership. + The ASF licenses this file to You under the Apache License, Version 2.0 + (the "License"); you may not use this file except in compliance with + the License. You may obtain a copy of the License at + + http://www.apache.org/licenses/LICENSE-2.0 + + Unless required by applicable law or agreed to in writing, software + distributed under the License is distributed on an "AS IS" BASIS, + WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + See the License for the specific language governing permissions and + limitations under the License. + limitations under the License. + */ + +package org.apache.flume.sink.kafka; + +public class TestConstants { + public static final String STATIC_TOPIC = "static-topic"; + public static final String CUSTOM_KEY = "custom-key"; + public static final String CUSTOM_TOPIC = "custom-topic"; +} diff --git a/flume-ng-sinks/flume-ng-kafka-sink/src/test/java/org/apache/flume/sink/kafka/TestKafkaSink.java b/flume-ng-sinks/flume-ng-kafka-sink/src/test/java/org/apache/flume/sink/kafka/TestKafkaSink.java new file mode 100644 index 0000000000..aed6dace83 --- /dev/null +++ b/flume-ng-sinks/flume-ng-kafka-sink/src/test/java/org/apache/flume/sink/kafka/TestKafkaSink.java @@ -0,0 +1,212 @@ +/** + Licensed to the Apache Software Foundation (ASF) under one or more + contributor license agreements. See the NOTICE file distributed with + this work for additional information regarding copyright ownership. + The ASF licenses this file to You under the Apache License, Version 2.0 + (the "License"); you may not use this file except in compliance with + the License. You may obtain a copy of the License at + + http://www.apache.org/licenses/LICENSE-2.0 + + Unless required by applicable law or agreed to in writing, software + distributed under the License is distributed on an "AS IS" BASIS, + WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + See the License for the specific language governing permissions and + limitations under the License. + limitations under the License. + */ + +package org.apache.flume.sink.kafka; + +import kafka.message.MessageAndMetadata; +import org.apache.flume.*; +import org.apache.flume.channel.MemoryChannel; +import org.apache.flume.conf.Configurables; +import org.apache.flume.event.EventBuilder; +import org.apache.flume.sink.kafka.util.TestUtil; +import org.junit.AfterClass; +import org.junit.BeforeClass; +import org.junit.Test; + +import java.io.UnsupportedEncodingException; +import java.util.ArrayList; +import java.util.HashMap; +import java.util.List; +import java.util.Map; + +import static org.junit.Assert.assertEquals; +import static org.junit.Assert.assertNull; +import static org.junit.Assert.fail; + +/** + * Unit tests for Kafka Sink + */ +public class TestKafkaSink { + + private static TestUtil testUtil = TestUtil.getInstance(); + + @BeforeClass + public static void setup() { + testUtil.prepare(); + List topics = new ArrayList(3); + topics.add(KafkaSinkConstants.DEFAULT_TOPIC); + topics.add(TestConstants.STATIC_TOPIC); + topics.add(TestConstants.CUSTOM_TOPIC); + testUtil.initTopicList(topics); + } + + @AfterClass + public static void tearDown() { + testUtil.tearDown(); + } + + @Test + public void testDefaultTopic() { + Sink kafkaSink = new KafkaSink(); + Context context = prepareDefaultContext(); + Configurables.configure(kafkaSink, context); + Channel memoryChannel = new MemoryChannel(); + Configurables.configure(memoryChannel, context); + kafkaSink.setChannel(memoryChannel); + kafkaSink.start(); + + String msg = "default-topic-test"; + Transaction tx = memoryChannel.getTransaction(); + tx.begin(); + Event event = EventBuilder.withBody(msg.getBytes()); + memoryChannel.put(event); + tx.commit(); + tx.close(); + + try { + Sink.Status status = kafkaSink.process(); + if (status == Sink.Status.BACKOFF) { + fail("Error Occurred"); + } + } catch (EventDeliveryException ex) { + // ignore + } + + String fetchedMsg = new String((byte[]) + testUtil.getNextMessageFromConsumer(KafkaSinkConstants.DEFAULT_TOPIC) + .message()); + assertEquals(msg, fetchedMsg); + } + + @Test + public void testStaticTopic() { + Context context = prepareDefaultContext(); + // add the static topic + context.put(KafkaSinkConstants.TOPIC, TestConstants.STATIC_TOPIC); + String msg = "static-topic-test"; + + try { + Sink.Status status = prepareAndSend(context, msg); + if (status == Sink.Status.BACKOFF) { + fail("Error Occurred"); + } + } catch (EventDeliveryException ex) { + // ignore + } + + String fetchedMsg = new String((byte[]) testUtil.getNextMessageFromConsumer( + TestConstants.STATIC_TOPIC).message()); + assertEquals(msg, fetchedMsg); + } + + @Test + public void testTopicAndKeyFromHeader() throws UnsupportedEncodingException { + + + Sink kafkaSink = new KafkaSink(); + Context context = prepareDefaultContext(); + Configurables.configure(kafkaSink, context); + Channel memoryChannel = new MemoryChannel(); + Configurables.configure(memoryChannel, context); + kafkaSink.setChannel(memoryChannel); + kafkaSink.start(); + + String msg = "my message"; + Map headers = new HashMap(); + headers.put("topic", TestConstants.CUSTOM_TOPIC); + headers.put("key", TestConstants.CUSTOM_KEY); + Transaction tx = memoryChannel.getTransaction(); + tx.begin(); + Event event = EventBuilder.withBody(msg.getBytes(), headers); + memoryChannel.put(event); + tx.commit(); + tx.close(); + + try { + Sink.Status status = kafkaSink.process(); + if (status == Sink.Status.BACKOFF) { + fail("Error Occurred"); + } + } catch (EventDeliveryException ex) { + // ignore + } + + MessageAndMetadata fetchedMsg = + testUtil.getNextMessageFromConsumer(TestConstants.CUSTOM_TOPIC); + + assertEquals(msg, new String((byte[]) fetchedMsg.message(), "UTF-8")); + assertEquals(TestConstants.CUSTOM_KEY, + new String((byte[]) fetchedMsg.key(), "UTF-8")); + + } + + @Test + public void testEmptyChannel() throws UnsupportedEncodingException { + + + Sink kafkaSink = new KafkaSink(); + Context context = prepareDefaultContext(); + Configurables.configure(kafkaSink, context); + Channel memoryChannel = new MemoryChannel(); + Configurables.configure(memoryChannel, context); + kafkaSink.setChannel(memoryChannel); + kafkaSink.start(); + + try { + Sink.Status status = kafkaSink.process(); + if (status == Sink.Status.BACKOFF) { + fail("Error Occurred"); + } + } catch (EventDeliveryException ex) { + // ignore + } + assertNull( + testUtil.getNextMessageFromConsumer(KafkaSinkConstants.DEFAULT_TOPIC)); + + } + + + private Context prepareDefaultContext() { + // Prepares a default context with Kafka Server Properties + Context context = new Context(); + context.put("kafka.metadata.broker.list", testUtil.getKafkaServerUrl()); + context.put("kafka.request.required.acks", "1"); + context.put("batchSize", "1"); + return context; + } + + private Sink.Status prepareAndSend(Context context, String msg) + throws EventDeliveryException { + Sink kafkaSink = new KafkaSink(); + Configurables.configure(kafkaSink, context); + Channel memoryChannel = new MemoryChannel(); + Configurables.configure(memoryChannel, context); + kafkaSink.setChannel(memoryChannel); + kafkaSink.start(); + + Transaction tx = memoryChannel.getTransaction(); + tx.begin(); + Event event = EventBuilder.withBody(msg.getBytes()); + memoryChannel.put(event); + tx.commit(); + tx.close(); + + return kafkaSink.process(); + } + +} diff --git a/flume-ng-sinks/flume-ng-kafka-sink/src/test/java/org/apache/flume/sink/kafka/util/KafkaConsumer.java b/flume-ng-sinks/flume-ng-kafka-sink/src/test/java/org/apache/flume/sink/kafka/util/KafkaConsumer.java new file mode 100644 index 0000000000..1c989223a8 --- /dev/null +++ b/flume-ng-sinks/flume-ng-kafka-sink/src/test/java/org/apache/flume/sink/kafka/util/KafkaConsumer.java @@ -0,0 +1,98 @@ +/** + Licensed to the Apache Software Foundation (ASF) under one or more + contributor license agreements. See the NOTICE file distributed with + this work for additional information regarding copyright ownership. + The ASF licenses this file to You under the Apache License, Version 2.0 + (the "License"); you may not use this file except in compliance with + the License. You may obtain a copy of the License at + + http://www.apache.org/licenses/LICENSE-2.0 + + Unless required by applicable law or agreed to in writing, software + distributed under the License is distributed on an "AS IS" BASIS, + WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + See the License for the specific language governing permissions and + limitations under the License. + limitations under the License. + */ + +package org.apache.flume.sink.kafka.util; + +import kafka.consumer.ConsumerConfig; +import kafka.consumer.ConsumerIterator; +import kafka.consumer.ConsumerTimeoutException; +import kafka.consumer.KafkaStream; +import kafka.javaapi.consumer.ConsumerConnector; +import kafka.message.MessageAndMetadata; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +import java.util.HashMap; +import java.util.List; +import java.util.Map; +import java.util.Properties; + +/** + * A Kafka Consumer implementation. This uses the current thread to fetch the + * next message from the queue and doesn't use a multi threaded implementation. + * So this implements a synchronous blocking call. + * To avoid infinite waiting, a timeout is implemented to wait only for + * 10 seconds before concluding that the message will not be available. + */ +public class KafkaConsumer { + + private static final Logger logger = LoggerFactory.getLogger( + KafkaConsumer.class); + + private final ConsumerConnector consumer; + Map>> consumerMap; + + public KafkaConsumer() { + consumer = kafka.consumer.Consumer.createJavaConsumerConnector( + createConsumerConfig(TestUtil.getInstance().getZkUrl(), "group_1")); + } + + private static ConsumerConfig createConsumerConfig(String zkUrl, + String groupId) { + Properties props = new Properties(); + props.put("zookeeper.connect", zkUrl); + props.put("group.id", groupId); + props.put("zookeeper.session.timeout.ms", "400"); + props.put("zookeeper.sync.time.ms", "200"); + props.put("auto.commit.interval.ms", "1000"); + props.put("auto.offset.reset", "smallest"); + props.put("consumer.timeout.ms","1000"); + return new ConsumerConfig(props); + } + + public void initTopicList(List topics) { + Map topicCountMap = new HashMap(); + for (String topic : topics) { + // we need only single threaded consumers + topicCountMap.put(topic, new Integer(1)); + } + consumerMap = consumer.createMessageStreams(topicCountMap); + } + + public MessageAndMetadata getNextMessage(String topic) { + List> streams = consumerMap.get(topic); + // it has only a single stream, because there is only one consumer + KafkaStream stream = streams.get(0); + final ConsumerIterator it = stream.iterator(); + int counter = 0; + try { + if (it.hasNext()) { + return it.next(); + } else { + return null; + } + } catch (ConsumerTimeoutException e) { + logger.error("0 messages available to fetch for the topic " + topic); + return null; + } + } + + public void shutdown() { + consumer.shutdown(); + } +} diff --git a/flume-ng-sinks/flume-ng-kafka-sink/src/test/java/org/apache/flume/sink/kafka/util/KafkaLocal.java b/flume-ng-sinks/flume-ng-kafka-sink/src/test/java/org/apache/flume/sink/kafka/util/KafkaLocal.java new file mode 100644 index 0000000000..3c6e064c4b --- /dev/null +++ b/flume-ng-sinks/flume-ng-kafka-sink/src/test/java/org/apache/flume/sink/kafka/util/KafkaLocal.java @@ -0,0 +1,52 @@ +/** + Licensed to the Apache Software Foundation (ASF) under one or more + contributor license agreements. See the NOTICE file distributed with + this work for additional information regarding copyright ownership. + The ASF licenses this file to You under the Apache License, Version 2.0 + (the "License"); you may not use this file except in compliance with + the License. You may obtain a copy of the License at + + http://www.apache.org/licenses/LICENSE-2.0 + + Unless required by applicable law or agreed to in writing, software + distributed under the License is distributed on an "AS IS" BASIS, + WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + See the License for the specific language governing permissions and + limitations under the License. + limitations under the License. + */ + +package org.apache.flume.sink.kafka.util; + +import kafka.server.KafkaConfig; +import kafka.server.KafkaServerStartable; + +import java.io.IOException; +import java.util.Properties; + +/** + * A local Kafka server for running unit tests. + * Reference: https://gist.github.com/fjavieralba/7930018/ + */ +public class KafkaLocal { + + public KafkaServerStartable kafka; + public ZooKeeperLocal zookeeper; + + public KafkaLocal(Properties kafkaProperties) throws IOException, + InterruptedException{ + KafkaConfig kafkaConfig = new KafkaConfig(kafkaProperties); + + //start local kafka broker + kafka = new KafkaServerStartable(kafkaConfig); + } + + public void start() throws Exception{ + kafka.startup(); + } + + public void stop(){ + kafka.shutdown(); + } + +} \ No newline at end of file diff --git a/flume-ng-sinks/flume-ng-kafka-sink/src/test/java/org/apache/flume/sink/kafka/util/TestUtil.java b/flume-ng-sinks/flume-ng-kafka-sink/src/test/java/org/apache/flume/sink/kafka/util/TestUtil.java new file mode 100644 index 0000000000..8855c535b3 --- /dev/null +++ b/flume-ng-sinks/flume-ng-kafka-sink/src/test/java/org/apache/flume/sink/kafka/util/TestUtil.java @@ -0,0 +1,174 @@ +/** + Licensed to the Apache Software Foundation (ASF) under one or more + contributor license agreements. See the NOTICE file distributed with + this work for additional information regarding copyright ownership. + The ASF licenses this file to You under the Apache License, Version 2.0 + (the "License"); you may not use this file except in compliance with + the License. You may obtain a copy of the License at + + http://www.apache.org/licenses/LICENSE-2.0 + + Unless required by applicable law or agreed to in writing, software + distributed under the License is distributed on an "AS IS" BASIS, + WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + See the License for the specific language governing permissions and + limitations under the License. + limitations under the License. + */ + +package org.apache.flume.sink.kafka.util; + +import kafka.message.MessageAndMetadata; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +import java.net.BindException; +import java.net.InetAddress; +import java.net.UnknownHostException; +import java.util.List; +import java.util.Properties; +import java.util.Random; + +/** + * A utility class for starting/stopping Kafka Server. + */ +public class TestUtil { + + private static final Logger logger = LoggerFactory.getLogger(TestUtil.class); + private static TestUtil instance = new TestUtil(); + + private Random randPortGen = new Random(System.currentTimeMillis()); + private KafkaLocal kafkaServer; + private KafkaConsumer kafkaConsumer; + private String hostname = "localhost"; + private int kafkaLocalPort; + private int zkLocalPort; + + private TestUtil() { + init(); + } + + public static TestUtil getInstance() { + return instance; + } + + private void init() { + // get the localhost. + try { + hostname = InetAddress.getLocalHost().getHostName(); + } catch (UnknownHostException e) { + logger.warn("Error getting the value of localhost. " + + "Proceeding with 'localhost'.", e); + } + } + + private boolean startKafkaServer() { + Properties kafkaProperties = new Properties(); + Properties zkProperties = new Properties(); + + try { + //load properties + zkProperties.load(Class.class.getResourceAsStream( + "/zookeeper.properties")); + + ZooKeeperLocal zookeeper; + while (true) { + //start local Zookeeper + try { + zkLocalPort = getNextPort(); + // override the Zookeeper client port with the generated one. + zkProperties.setProperty("clientPort", Integer.toString(zkLocalPort)); + zookeeper = new ZooKeeperLocal(zkProperties); + break; + } catch (BindException bindEx) { + // bind exception. port is already in use. Try a different port. + } + } + logger.info("ZooKeeper instance is successfully started on port " + + zkLocalPort); + + kafkaProperties.load(Class.class.getResourceAsStream( + "/kafka-server.properties")); + // override the Zookeeper url. + kafkaProperties.setProperty("zookeeper.connect", getZkUrl()); + while (true) { + kafkaLocalPort = getNextPort(); + // override the Kafka server port + kafkaProperties.setProperty("port", Integer.toString(kafkaLocalPort)); + kafkaServer = new KafkaLocal(kafkaProperties); + try { + kafkaServer.start(); + break; + } catch (BindException bindEx) { + // let's try another port. + } + } + logger.info("Kafka Server is successfully started on port " + + kafkaLocalPort); + return true; + + } catch (Exception e) { + logger.error("Error starting the Kafka Server.", e); + return false; + } + } + + private KafkaConsumer getKafkaConsumer() { + synchronized (this) { + if (kafkaConsumer == null) { + kafkaConsumer = new KafkaConsumer(); + } + } + return kafkaConsumer; + } + + public void initTopicList(List topics) { + getKafkaConsumer().initTopicList(topics); + } + + public MessageAndMetadata getNextMessageFromConsumer(String topic) { + return getKafkaConsumer().getNextMessage(topic); + } + + public void prepare() { + boolean startStatus = startKafkaServer(); + if (!startStatus) { + throw new RuntimeException("Error starting the server!"); + } + try { + Thread.sleep(3 * 1000); // add this sleep time to + // ensure that the server is fully started before proceeding with tests. + } catch (InterruptedException e) { + // ignore + } + getKafkaConsumer(); + logger.info("Completed the prepare phase."); + } + + public void tearDown() { + logger.info("Shutting down the Kafka Consumer."); + getKafkaConsumer().shutdown(); + try { + Thread.sleep(3 * 1000); // add this sleep time to + // ensure that the server is fully started before proceeding with tests. + } catch (InterruptedException e) { + // ignore + } + logger.info("Shutting down the kafka Server."); + kafkaServer.stop(); + logger.info("Completed the tearDown phase."); + } + + private synchronized int getNextPort() { + // generate a random port number between 49152 and 65535 + return randPortGen.nextInt(65535 - 49152) + 49152; + } + + public String getZkUrl() { + return hostname + ":" + zkLocalPort; + } + + public String getKafkaServerUrl() { + return hostname + ":" + kafkaLocalPort; + } +} diff --git a/flume-ng-sinks/flume-ng-kafka-sink/src/test/java/org/apache/flume/sink/kafka/util/ZooKeeperLocal.java b/flume-ng-sinks/flume-ng-kafka-sink/src/test/java/org/apache/flume/sink/kafka/util/ZooKeeperLocal.java new file mode 100644 index 0000000000..1a5728fcf0 --- /dev/null +++ b/flume-ng-sinks/flume-ng-kafka-sink/src/test/java/org/apache/flume/sink/kafka/util/ZooKeeperLocal.java @@ -0,0 +1,62 @@ +/** + Licensed to the Apache Software Foundation (ASF) under one or more + contributor license agreements. See the NOTICE file distributed with + this work for additional information regarding copyright ownership. + The ASF licenses this file to You under the Apache License, Version 2.0 + (the "License"); you may not use this file except in compliance with + the License. You may obtain a copy of the License at + + http://www.apache.org/licenses/LICENSE-2.0 + + Unless required by applicable law or agreed to in writing, software + distributed under the License is distributed on an "AS IS" BASIS, + WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + See the License for the specific language governing permissions and + limitations under the License. + limitations under the License. + */ + +package org.apache.flume.sink.kafka.util; + +import org.apache.zookeeper.server.ServerConfig; +import org.apache.zookeeper.server.ZooKeeperServerMain; +import org.apache.zookeeper.server.quorum.QuorumPeerConfig; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +import java.io.IOException; +import java.util.Properties; + +/** + * A local Zookeeper server for running unit tests. + * Reference: https://gist.github.com/fjavieralba/7930018/ + */ +public class ZooKeeperLocal { + + private static final Logger logger = + LoggerFactory.getLogger(ZooKeeperLocal.class); + private ZooKeeperServerMain zooKeeperServer; + + public ZooKeeperLocal(Properties zkProperties) throws IOException{ + QuorumPeerConfig quorumConfiguration = new QuorumPeerConfig(); + try { + quorumConfiguration.parseProperties(zkProperties); + } catch(Exception e) { + throw new RuntimeException(e); + } + + zooKeeperServer = new ZooKeeperServerMain(); + final ServerConfig configuration = new ServerConfig(); + configuration.readFrom(quorumConfiguration); + + new Thread() { + public void run() { + try { + zooKeeperServer.runFromConfig(configuration); + } catch (IOException e) { + logger.error("Zookeeper startup failed.", e); + } + } + }.start(); + } +} diff --git a/flume-ng-sinks/flume-ng-kafka-sink/src/test/resources/kafka-server.properties b/flume-ng-sinks/flume-ng-kafka-sink/src/test/resources/kafka-server.properties new file mode 100644 index 0000000000..c07cdea66e --- /dev/null +++ b/flume-ng-sinks/flume-ng-kafka-sink/src/test/resources/kafka-server.properties @@ -0,0 +1,117 @@ +# Licensed to the Apache Software Foundation (ASF) under one or more +# contributor license agreements. See the NOTICE file distributed with +# this work for additional information regarding copyright ownership. +# The ASF licenses this file to You under the Apache License, Version 2.0 +# (the "License"); you may not use this file except in compliance with +# the License. You may obtain a copy of the License at +# +# http://www.apache.org/licenses/LICENSE-2.0 +# +# Unless required by applicable law or agreed to in writing, software +# distributed under the License is distributed on an "AS IS" BASIS, +# WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +# See the License for the specific language governing permissions and +# limitations under the License. +# see kafka.server.KafkaConfig for additional details and defaults + +############################# Server Basics ############################# + +# The id of the broker. This must be set to a unique integer for each broker. +broker.id=0 + +############################# Socket Server Settings ############################# + +# The port the socket server listens on +port=9092 + +# Hostname the broker will bind to. If not set, the server will bind to all interfaces +#host.name=localhost + +# Hostname the broker will advertise to producers and consumers. If not set, it uses the +# value for "host.name" if configured. Otherwise, it will use the value returned from +# java.net.InetAddress.getCanonicalHostName(). +#advertised.host.name= + +# The port to publish to ZooKeeper for clients to use. If this is not set, +# it will publish the same port that the broker binds to. +#advertised.port= + +# The number of threads handling network requests +num.network.threads=2 + +# The number of threads doing disk I/O +num.io.threads=8 + +# The send buffer (SO_SNDBUF) used by the socket server +socket.send.buffer.bytes=1048576 + +# The receive buffer (SO_RCVBUF) used by the socket server +socket.receive.buffer.bytes=1048576 + +# The maximum size of a request that the socket server will accept (protection against OOM) +socket.request.max.bytes=104857600 + + +############################# Log Basics ############################# + +# A comma seperated list of directories under which to store log files +log.dirs=target/kafka-logs + +# The default number of log partitions per topic. More partitions allow greater +# parallelism for consumption, but this will also result in more files across +# the brokers. +num.partitions=2 + +############################# Log Flush Policy ############################# + +# Messages are immediately written to the filesystem but by default we only fsync() to sync +# the OS cache lazily. The following configurations control the flush of data to disk. +# There are a few important trade-offs here: +# 1. Durability: Unflushed data may be lost if you are not using replication. +# 2. Latency: Very large flush intervals may lead to latency spikes when the flush does occur as there will be a lot of data to flush. +# 3. Throughput: The flush is generally the most expensive operation, and a small flush interval may lead to exceessive seeks. +# The settings below allow one to configure the flush policy to flush data after a period of time or +# every N messages (or both). This can be done globally and overridden on a per-topic basis. + +# The number of messages to accept before forcing a flush of data to disk +#log.flush.interval.messages=10000 + +# The maximum amount of time a message can sit in a log before we force a flush +#log.flush.interval.ms=1000 + +############################# Log Retention Policy ############################# + +# The following configurations control the disposal of log segments. The policy can +# be set to delete segments after a period of time, or after a given size has accumulated. +# A segment will be deleted whenever *either* of these criteria are met. Deletion always happens +# from the end of the log. + +# The minimum age of a log file to be eligible for deletion +log.retention.hours=168 + +# A size-based retention policy for logs. Segments are pruned from the log as long as the remaining +# segments don't drop below log.retention.bytes. +#log.retention.bytes=1073741824 + +# The maximum size of a log segment file. When this size is reached a new log segment will be created. +log.segment.bytes=536870912 + +# The interval at which log segments are checked to see if they can be deleted according +# to the retention policies +log.retention.check.interval.ms=60000 + +# By default the log cleaner is disabled and the log retention policy will default to just delete segments after their retention expires. +# If log.cleaner.enable=true is set the cleaner will be enabled and individual logs can then be marked for log compaction. +log.cleaner.enable=false + +############################# Zookeeper ############################# + +# Zookeeper connection string (see zookeeper docs for details). +# This is a comma separated host:port pairs, each corresponding to a zk +# server. e.g. "127.0.0.1:3000,127.0.0.1:3001,127.0.0.1:3002". +# You can also append an optional chroot string to the urls to specify the +# root directory for all kafka znodes. +zookeeper.connect=localhost:2181 + +# Timeout in ms for connecting to zookeeper +zookeeper.connection.timeout.ms=1000000 diff --git a/flume-ng-sinks/flume-ng-kafka-sink/src/test/resources/log4j.properties b/flume-ng-sinks/flume-ng-kafka-sink/src/test/resources/log4j.properties new file mode 100644 index 0000000000..bdcb643c9e --- /dev/null +++ b/flume-ng-sinks/flume-ng-kafka-sink/src/test/resources/log4j.properties @@ -0,0 +1,78 @@ +# Licensed to the Apache Software Foundation (ASF) under one or more +# contributor license agreements. See the NOTICE file distributed with +# this work for additional information regarding copyright ownership. +# The ASF licenses this file to You under the Apache License, Version 2.0 +# (the "License"); you may not use this file except in compliance with +# the License. You may obtain a copy of the License at +# +# http://www.apache.org/licenses/LICENSE-2.0 +# +# Unless required by applicable law or agreed to in writing, software +# distributed under the License is distributed on an "AS IS" BASIS, +# WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +# See the License for the specific language governing permissions and +# limitations under the License. + +kafka.logs.dir=target/logs + +log4j.rootLogger=INFO, stdout + +log4j.appender.stdout=org.apache.log4j.ConsoleAppender +log4j.appender.stdout.layout=org.apache.log4j.PatternLayout +log4j.appender.stdout.layout.ConversionPattern=[%d] %p %m (%c)%n + +log4j.appender.kafkaAppender=org.apache.log4j.DailyRollingFileAppender +log4j.appender.kafkaAppender.DatePattern='.'yyyy-MM-dd-HH +log4j.appender.kafkaAppender.File=${kafka.logs.dir}/server.log +log4j.appender.kafkaAppender.layout=org.apache.log4j.PatternLayout +log4j.appender.kafkaAppender.layout.ConversionPattern=[%d] %p %m (%c)%n + +log4j.appender.stateChangeAppender=org.apache.log4j.DailyRollingFileAppender +log4j.appender.stateChangeAppender.DatePattern='.'yyyy-MM-dd-HH +log4j.appender.stateChangeAppender.File=${kafka.logs.dir}/state-change.log +log4j.appender.stateChangeAppender.layout=org.apache.log4j.PatternLayout +log4j.appender.stateChangeAppender.layout.ConversionPattern=[%d] %p %m (%c)%n + +log4j.appender.requestAppender=org.apache.log4j.DailyRollingFileAppender +log4j.appender.requestAppender.DatePattern='.'yyyy-MM-dd-HH +log4j.appender.requestAppender.File=${kafka.logs.dir}/kafka-request.log +log4j.appender.requestAppender.layout=org.apache.log4j.PatternLayout +log4j.appender.requestAppender.layout.ConversionPattern=[%d] %p %m (%c)%n + +log4j.appender.cleanerAppender=org.apache.log4j.DailyRollingFileAppender +log4j.appender.cleanerAppender.DatePattern='.'yyyy-MM-dd-HH +log4j.appender.cleanerAppender.File=${kafka.logs.dir}/log-cleaner.log +log4j.appender.cleanerAppender.layout=org.apache.log4j.PatternLayout +log4j.appender.cleanerAppender.layout.ConversionPattern=[%d] %p %m (%c)%n + +log4j.appender.controllerAppender=org.apache.log4j.DailyRollingFileAppender +log4j.appender.controllerAppender.DatePattern='.'yyyy-MM-dd-HH +log4j.appender.controllerAppender.File=${kafka.logs.dir}/controller.log +log4j.appender.controllerAppender.layout=org.apache.log4j.PatternLayout +log4j.appender.controllerAppender.layout.ConversionPattern=[%d] %p %m (%c)%n + +# Turn on all our debugging info +#log4j.logger.kafka.producer.async.DefaultEventHandler=DEBUG, kafkaAppender +#log4j.logger.kafka.client.ClientUtils=DEBUG, kafkaAppender +#log4j.logger.kafka.perf=DEBUG, kafkaAppender +#log4j.logger.kafka.perf.ProducerPerformance$ProducerThread=DEBUG, kafkaAppender +#log4j.logger.org.I0Itec.zkclient.ZkClient=DEBUG +log4j.logger.kafka=INFO, kafkaAppender + +log4j.logger.kafka.network.RequestChannel$=WARN, requestAppender +log4j.additivity.kafka.network.RequestChannel$=false + +#log4j.logger.kafka.network.Processor=TRACE, requestAppender +#log4j.logger.kafka.server.KafkaApis=TRACE, requestAppender +#log4j.additivity.kafka.server.KafkaApis=false +log4j.logger.kafka.request.logger=WARN, requestAppender +log4j.additivity.kafka.request.logger=false + +log4j.logger.kafka.controller=TRACE, controllerAppender +log4j.additivity.kafka.controller=false + +log4j.logger.kafka.log.LogCleaner=INFO, cleanerAppender +log4j.additivity.kafka.log.LogCleaner=false + +log4j.logger.state.change.logger=TRACE, stateChangeAppender +log4j.additivity.state.change.logger=false diff --git a/flume-ng-sinks/flume-ng-kafka-sink/src/test/resources/zookeeper.properties b/flume-ng-sinks/flume-ng-kafka-sink/src/test/resources/zookeeper.properties new file mode 100644 index 0000000000..89e1b5ec38 --- /dev/null +++ b/flume-ng-sinks/flume-ng-kafka-sink/src/test/resources/zookeeper.properties @@ -0,0 +1,20 @@ +# Licensed to the Apache Software Foundation (ASF) under one or more +# contributor license agreements. See the NOTICE file distributed with +# this work for additional information regarding copyright ownership. +# The ASF licenses this file to You under the Apache License, Version 2.0 +# (the "License"); you may not use this file except in compliance with +# the License. You may obtain a copy of the License at +# +# http://www.apache.org/licenses/LICENSE-2.0 +# +# Unless required by applicable law or agreed to in writing, software +# distributed under the License is distributed on an "AS IS" BASIS, +# WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +# See the License for the specific language governing permissions and +# limitations under the License. +# the directory where the snapshot is stored. +dataDir=target +# the port at which the clients will connect +clientPort=2181 +# disable the per-ip limit on the number of connections since this is a non-production config +maxClientCnxns=0 \ No newline at end of file diff --git a/flume-ng-sinks/pom.xml b/flume-ng-sinks/pom.xml index 3381bde2a1..4bac01916b 100644 --- a/flume-ng-sinks/pom.xml +++ b/flume-ng-sinks/pom.xml @@ -46,6 +46,7 @@ limitations under the License. flume-ng-hbase-sink flume-ng-elasticsearch-sink flume-ng-morphline-solr-sink + flume-ng-kafka-sink diff --git a/pom.xml b/pom.xml index 150db2ed87..740edc209d 100644 --- a/pom.xml +++ b/pom.xml @@ -1138,6 +1138,12 @@ limitations under the License. 1.6.0-SNAPSHOT + + org.apache.flume.flume-ng-sinks + flume-ng-kafka-sink + 1.6.0-SNAPSHOT + + org.apache.flume.flume-ng-sources flume-scribe-source From 0bbd0ad7aaad2f592c596608d4e1981aa86eb53d Mon Sep 17 00:00:00 2001 From: Hari Shreedharan Date: Tue, 16 Sep 2014 21:24:37 -0700 Subject: [PATCH 138/341] FLUME-2250. Kafka Source. (Frank Yao, Ashish Paliwal, Gwen Shapira via Hari) --- flume-ng-dist/pom.xml | 4 + flume-ng-doc/sphinx/FlumeUserGuide.rst | 126 ++++++---- flume-ng-sinks/flume-ng-kafka-sink/pom.xml | 1 - flume-ng-sources/flume-kafka-source/pom.xml | 70 ++++++ .../flume/source/kafka/KafkaSource.java | 231 ++++++++++++++++++ .../source/kafka/KafkaSourceConstants.java | 36 +++ .../flume/source/kafka/KafkaSourceUtil.java | 63 +++++ .../kafka/KafkaSourceEmbeddedKafka.java | 92 +++++++ .../kafka/KafkaSourceEmbeddedZookeeper.java | 64 +++++ .../flume/source/kafka/KafkaSourceTest.java | 195 +++++++++++++++ .../source/kafka/KafkaSourceUtilTest.java | 75 ++++++ .../src/test/resources/log4j.properties | 25 ++ flume-ng-sources/pom.xml | 1 + pom.xml | 20 ++ 14 files changed, 961 insertions(+), 42 deletions(-) create mode 100644 flume-ng-sources/flume-kafka-source/pom.xml create mode 100644 flume-ng-sources/flume-kafka-source/src/main/java/org/apache/flume/source/kafka/KafkaSource.java create mode 100644 flume-ng-sources/flume-kafka-source/src/main/java/org/apache/flume/source/kafka/KafkaSourceConstants.java create mode 100644 flume-ng-sources/flume-kafka-source/src/main/java/org/apache/flume/source/kafka/KafkaSourceUtil.java create mode 100644 flume-ng-sources/flume-kafka-source/src/test/java/org/apache/flume/source/kafka/KafkaSourceEmbeddedKafka.java create mode 100644 flume-ng-sources/flume-kafka-source/src/test/java/org/apache/flume/source/kafka/KafkaSourceEmbeddedZookeeper.java create mode 100644 flume-ng-sources/flume-kafka-source/src/test/java/org/apache/flume/source/kafka/KafkaSourceTest.java create mode 100644 flume-ng-sources/flume-kafka-source/src/test/java/org/apache/flume/source/kafka/KafkaSourceUtilTest.java create mode 100644 flume-ng-sources/flume-kafka-source/src/test/resources/log4j.properties diff --git a/flume-ng-dist/pom.xml b/flume-ng-dist/pom.xml index ca3cd8b742..a5db0c7bc8 100644 --- a/flume-ng-dist/pom.xml +++ b/flume-ng-dist/pom.xml @@ -165,6 +165,10 @@ org.apache.flume.flume-ng-sources flume-twitter-source + + org.apache.flume.flume-ng-sources + flume-kafka-source + org.apache.flume.flume-ng-legacy-sources flume-avro-source diff --git a/flume-ng-doc/sphinx/FlumeUserGuide.rst b/flume-ng-doc/sphinx/FlumeUserGuide.rst index a718fbf84b..3a4756091c 100644 --- a/flume-ng-doc/sphinx/FlumeUserGuide.rst +++ b/flume-ng-doc/sphinx/FlumeUserGuide.rst @@ -994,47 +994,6 @@ Example for an agent named agent-1: agent-1.sources.src-1.spoolDir = /var/log/apache/flumeSpool agent-1.sources.src-1.fileHeader = true -Twitter 1% firehose Source (experimental) -~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~ - -.. warning:: - This source is hightly experimental and may change between minor versions of Flume. - Use at your own risk. - -Experimental source that connects via Streaming API to the 1% sample twitter -firehose, continously downloads tweets, converts them to Avro format and -sends Avro events to a downstream Flume sink. Requires the consumer and -access tokens and secrets of a Twitter developer account. -Required properties are in **bold**. - -====================== =========== =================================================== -Property Name Default Description -====================== =========== =================================================== -**channels** -- -**type** -- The component type name, needs to be ``org.apache.flume.source.twitter.TwitterSource`` -**consumerKey** -- OAuth consumer key -**consumerSecret** -- OAuth consumer secret -**accessToken** -- OAuth access token -**accessTokenSecret** -- OAuth toekn secret -maxBatchSize 1000 Maximum number of twitter messages to put in a single batch -maxBatchDurationMillis 1000 Maximum number of milliseconds to wait before closing a batch -====================== =========== =================================================== - -Example for agent named a1: - -.. code-block:: properties - - a1.sources = r1 - a1.channels = c1 - a1.sources.r1.type = org.apache.flume.source.twitter.TwitterSource - a1.sources.r1.channels = c1 - a1.sources.r1.consumerKey = YOUR_TWITTER_CONSUMER_KEY - a1.sources.r1.consumerSecret = YOUR_TWITTER_CONSUMER_SECRET - a1.sources.r1.accessToken = YOUR_TWITTER_ACCESS_TOKEN - a1.sources.r1.accessTokenSecret = YOUR_TWITTER_ACCESS_TOKEN_SECRET - a1.sources.r1.maxBatchSize = 10 - a1.sources.r1.maxBatchDurationMillis = 200 - Event Deserializers ''''''''''''''''''' @@ -1094,6 +1053,91 @@ Property Name Default Description deserializer.maxBlobLength 100000000 The maximum number of bytes to read and buffer for a given request ========================== ================== ======================================================================= +Twitter 1% firehose Source (experimental) +~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~ + +.. warning:: + This source is hightly experimental and may change between minor versions of Flume. + Use at your own risk. + +Experimental source that connects via Streaming API to the 1% sample twitter +firehose, continously downloads tweets, converts them to Avro format and +sends Avro events to a downstream Flume sink. Requires the consumer and +access tokens and secrets of a Twitter developer account. +Required properties are in **bold**. + +====================== =========== =================================================== +Property Name Default Description +====================== =========== =================================================== +**channels** -- +**type** -- The component type name, needs to be ``org.apache.flume.source.twitter.TwitterSource`` +**consumerKey** -- OAuth consumer key +**consumerSecret** -- OAuth consumer secret +**accessToken** -- OAuth access token +**accessTokenSecret** -- OAuth toekn secret +maxBatchSize 1000 Maximum number of twitter messages to put in a single batch +maxBatchDurationMillis 1000 Maximum number of milliseconds to wait before closing a batch +====================== =========== =================================================== + +Example for agent named a1: + +.. code-block:: properties + + a1.sources = r1 + a1.channels = c1 + a1.sources.r1.type = org.apache.flume.source.twitter.TwitterSource + a1.sources.r1.channels = c1 + a1.sources.r1.consumerKey = YOUR_TWITTER_CONSUMER_KEY + a1.sources.r1.consumerSecret = YOUR_TWITTER_CONSUMER_SECRET + a1.sources.r1.accessToken = YOUR_TWITTER_ACCESS_TOKEN + a1.sources.r1.accessTokenSecret = YOUR_TWITTER_ACCESS_TOKEN_SECRET + a1.sources.r1.maxBatchSize = 10 + a1.sources.r1.maxBatchDurationMillis = 200 + +Kafka Source +~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~ + +Kafka Source is an Apache Kafka consumer that reads messages from a Kafka topic. +If you have multiple Kafka sources running, you can configure them with the same Consumer Group +so each will read a unique set of partitions for the topic. + +The properties below are required properties, but you can specify any Kafka parameter you want +and it will be passed to the consumer. Check `Kafka documentation `_ +for details + +=========================== =========== =================================================== +Property Name Default Description +=========================== =========== =================================================== +**channels** -- +**type** -- The component type name, needs to be ``org.apache.flume.source.kafka,KafkaSource`` +**kafka.zookeeper.connect** -- URI of ZooKeeper used by Kafka cluster +**kadka.group.id** -- Unique identified of consumer group. Setting the same id in multiple sources or agents + indicates that they are part of the same consumer group +**topic** -- Kafka topic we'll read messages from. At the time, this is a single topic only. +batchSize 1000 Maximum number of messages written to Channel in one batch +batchDurationMillis 1000 Maximum time (in ms) before a batch will be written to Channel + The batch will be written whenever the first of size and time will be reached. +kafka.auto.commit.enable false If true, Kafka will commit events automatically - faster but less durable option. + when false, the Kafka Source will commit events before writing batch to channel +consumer.timeout.ms 10 Polling interval for new data for batch. + Low value means more CPU usage. + High value means the maxBatchDurationMillis may be missed while waiting for + additional data. +=========================== =========== =================================================== + +Example for agent named tier1: + +.. code-block:: properties + + tier1.sources.source1.type = org.apache.flume.source.kafka.KafkaSource + tier1.sources.source1.channels = channel1 + tier1.sources.source1.kafka.zookeeper.connect = localhost:2181 + tier1.sources.source1.topic = test1 + tier1.sources.source1.kafka.group.id = flume + tier1.sources.source1.kafka.consumer.timeout.ms = 100 + + + NetCat Source ~~~~~~~~~~~~~ diff --git a/flume-ng-sinks/flume-ng-kafka-sink/pom.xml b/flume-ng-sinks/flume-ng-kafka-sink/pom.xml index 307fa59487..746a39539a 100644 --- a/flume-ng-sinks/flume-ng-kafka-sink/pom.xml +++ b/flume-ng-sinks/flume-ng-kafka-sink/pom.xml @@ -61,7 +61,6 @@ org.apache.kafka kafka_2.10 - 0.8.1.1 diff --git a/flume-ng-sources/flume-kafka-source/pom.xml b/flume-ng-sources/flume-kafka-source/pom.xml new file mode 100644 index 0000000000..8ad29d7228 --- /dev/null +++ b/flume-ng-sources/flume-kafka-source/pom.xml @@ -0,0 +1,70 @@ + + + + + + flume-ng-sources + org.apache.flume + 1.6.0-SNAPSHOT + + 4.0.0 + + org.apache.flume.flume-ng-sources + flume-kafka-source + Flume Kafka Source + + + + + org.apache.rat + apache-rat-plugin + + + + + + + junit + junit + + + org.apache.flume + flume-ng-core + + + org.apache.kafka + kafka_2.10 + + + org.apache.zookeeper + zookeeper + + + org.mockito + mockito-all + + + org.apache.kafka + kafka_2.10 + test + test + + + \ No newline at end of file diff --git a/flume-ng-sources/flume-kafka-source/src/main/java/org/apache/flume/source/kafka/KafkaSource.java b/flume-ng-sources/flume-kafka-source/src/main/java/org/apache/flume/source/kafka/KafkaSource.java new file mode 100644 index 0000000000..da78f80add --- /dev/null +++ b/flume-ng-sources/flume-kafka-source/src/main/java/org/apache/flume/source/kafka/KafkaSource.java @@ -0,0 +1,231 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package org.apache.flume.source.kafka; + +import java.util.ArrayList; +import java.util.HashMap; +import java.util.List; +import java.util.Map; + +import kafka.consumer.ConsumerIterator; +import kafka.consumer.ConsumerTimeoutException; +import kafka.consumer.KafkaStream; +import kafka.javaapi.consumer.ConsumerConnector; + +import org.apache.flume.*; +import org.apache.flume.conf.Configurable; +import org.apache.flume.conf.ConfigurationException; +import org.apache.flume.event.EventBuilder; +import org.apache.flume.source.AbstractSource; + +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + + +/** + * A Source for Kafka which reads messages from kafka. + * I use this in company production environment and its performance is good. + * Over 100k messages per second can be read from kafka in one source.

    + * kafka.zookeeper.connect: the zookeeper ip kafka use.

    + * kafka.group.id: the groupid of consumer group.

    + * topic: the topic to read from kafka.

    + * maxBatchSize - maximum number of messages written to Channel in one batch + * maxBatchDurationMillis - maximum time before a batch (of any size) + * will be written to Channel + * kafka.auto.commit.enable - if true, commit automatically every time period. + * if false, commit on each batch. + * kafka.consumer.timeout.ms - polling interval for new data for batch. + * Low value means more CPU usage. + * High value means the time.upper.limit may be missed. + * + * Any property starting with "kafka" will be passed to the kafka consumer + * So you can use any configuration supported by Kafka 0.8.1.1 + */ +public class KafkaSource extends AbstractSource + implements Configurable, PollableSource { + private static final Logger log = LoggerFactory.getLogger(KafkaSource.class); + private ConsumerConnector consumer; + private ConsumerIterator it; + private String topic; + private int batchUpperLimit; + private int timeUpperLimit; + private int consumerTimeout; + private boolean kafkaAutoCommitEnabled; + private Context context; + private final List eventList = new ArrayList(); + + public Status process() throws EventDeliveryException { + eventList.clear(); + byte[] bytes; + Event event; + Map headers; + try { + int eventCounter = 0; + int timeWaited = 0; + IterStatus iterStatus = new IterStatus(false, -1); + while (eventCounter < batchUpperLimit && timeWaited < timeUpperLimit) { + iterStatus = timedHasNext(); + if (iterStatus.hasData()) { + // get next message + bytes = it.next().message(); + + headers = new HashMap(); + headers.put(KafkaSourceConstants.TIMESTAMP, + String.valueOf(System.currentTimeMillis())); + headers.put(KafkaSourceConstants.TOPIC,topic); + if (log.isDebugEnabled()) { + log.debug("Message: {}", new String(bytes)); + } + event = EventBuilder.withBody(bytes, headers); + eventList.add(event); + eventCounter++; + } + timeWaited += iterStatus.getWaitTime(); + if (log.isDebugEnabled()) { + log.debug("Waited: {} ", timeWaited); + log.debug("Event #: {}", eventCounter); + } + } + // If we have events, send events to channel + // and commit if Kafka doesn't auto-commit + if (eventCounter > 0) { + getChannelProcessor().processEventBatch(eventList); + if (!kafkaAutoCommitEnabled) { + // commit the read transactions to Kafka to avoid duplicates + consumer.commitOffsets(); + } + } + if (!iterStatus.hasData()) { + if (log.isDebugEnabled()) { + log.debug("Returning with backoff. No more data to read"); + } + return Status.BACKOFF; + } + return Status.READY; + } catch (Exception e) { + log.error("KafkaSource EXCEPTION, {}", e); + return Status.BACKOFF; + } + } + + public void configure(Context context) { + this.context = context; + batchUpperLimit = context.getInteger(KafkaSourceConstants.BATCH_SIZE, + KafkaSourceConstants.DEFAULT_BATCH_SIZE); + timeUpperLimit = context.getInteger(KafkaSourceConstants.BATCH_DURATION_MS, + KafkaSourceConstants.DEFAULT_BATCH_DURATION); + topic = context.getString(KafkaSourceConstants.TOPIC); + + //if consumer timeout and autocommit were not set by user, + // set them to 10ms and false + consumerTimeout = context.getInteger(KafkaSourceConstants.CONSUMER_TIMEOUT, + KafkaSourceConstants.DEFAULT_CONSUMER_TIMEOUT); + context.put(KafkaSourceConstants.CONSUMER_TIMEOUT, + Integer.toString(consumerTimeout)); + String autoCommit = context.getString( + KafkaSourceConstants.AUTO_COMMIT_ENABLED, + String.valueOf(KafkaSourceConstants.DEFAULT_AUTO_COMMIT)); + kafkaAutoCommitEnabled = Boolean.valueOf(autoCommit); + context.put(KafkaSourceConstants.AUTO_COMMIT_ENABLED,autoCommit); + + if(topic == null) { + throw new ConfigurationException("Kafka topic must be specified."); + } + } + + @Override + public synchronized void start() { + log.info("Starting {}...", this); + + try { + //initialize a consumer. This creates the connection to ZooKeeper + consumer = KafkaSourceUtil.getConsumer(context); + } catch (Exception e) { + throw new FlumeException("Unable to create consumer. " + + "Check whether the ZooKeeper server is up and that the " + + "Flume agent can connect to it.", e); + } + + Map topicCountMap = new HashMap(); + // We always have just one topic being read by one thread + topicCountMap.put(topic, 1); + + // Get the message iterator for our topic + // Note that this succeeds even if the topic doesn't exist + // in that case we simply get no messages for the topic + // Also note that currently we only support a single topic + try { + Map>> consumerMap = + consumer.createMessageStreams(topicCountMap); + List> topicList = consumerMap.get(topic); + KafkaStream stream = topicList.get(0); + it = stream.iterator(); + } catch (Exception e) { + throw new FlumeException("Unable to get message iterator from Kafka", e); + } + log.info("Kafka source {} started.", getName()); + super.start(); + } + + @Override + public synchronized void stop() { + if (consumer != null) { + // exit cleanly. This syncs offsets of messages read to ZooKeeper + // to avoid reading the same messages again + consumer.shutdown(); + } + super.stop(); + } + + + /** + * Check if there are messages waiting in Kafka, + * waiting until timeout (10ms by default) for messages to arrive. + * And timing our wait. + * @return IterStatus object. + * Indicating whether a message was found and how long we waited for it + */ + IterStatus timedHasNext() { + try { + long startTime = System.currentTimeMillis(); + it.hasNext(); + long endTime = System.currentTimeMillis(); + return new IterStatus(true, endTime - startTime); + } catch (ConsumerTimeoutException e) { + return new IterStatus(false, consumerTimeout); + } + } + + private class IterStatus { + private long waitTime; + private boolean hasData; + + + private IterStatus(boolean hasData,long waitTime) { + this.waitTime = waitTime; + this.hasData = hasData; + } + + public long getWaitTime() { + return waitTime; + } + + public boolean hasData() { + return hasData; + } + } +} \ No newline at end of file diff --git a/flume-ng-sources/flume-kafka-source/src/main/java/org/apache/flume/source/kafka/KafkaSourceConstants.java b/flume-ng-sources/flume-kafka-source/src/main/java/org/apache/flume/source/kafka/KafkaSourceConstants.java new file mode 100644 index 0000000000..ac86f6592e --- /dev/null +++ b/flume-ng-sources/flume-kafka-source/src/main/java/org/apache/flume/source/kafka/KafkaSourceConstants.java @@ -0,0 +1,36 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package org.apache.flume.source.kafka; + +public class KafkaSourceConstants { + public static final String TOPIC = "topic"; + public static final String TIMESTAMP = "timestamp"; + public static final String BATCH_SIZE = "batchSize"; + public static final String BATCH_DURATION_MS = "batchDurationMillis"; + public static final String CONSUMER_TIMEOUT = "kafka.consumer.timeout.ms"; + public static final String AUTO_COMMIT_ENABLED = "kafka.auto.commit.enabled"; + public static final String ZOOKEEPER_CONNECT = "kafka.zookeeper.connect"; + public static final String GROUP_ID = "kafka.group.id"; + public static final String PROPERTY_PREFIX = "kafka"; + + + public static final int DEFAULT_BATCH_SIZE = 1000; + public static final int DEFAULT_BATCH_DURATION = 1000; + public static final int DEFAULT_CONSUMER_TIMEOUT = 10; + public static final boolean DEFAULT_AUTO_COMMIT = false; + +} diff --git a/flume-ng-sources/flume-kafka-source/src/main/java/org/apache/flume/source/kafka/KafkaSourceUtil.java b/flume-ng-sources/flume-kafka-source/src/main/java/org/apache/flume/source/kafka/KafkaSourceUtil.java new file mode 100644 index 0000000000..8397272bfe --- /dev/null +++ b/flume-ng-sources/flume-kafka-source/src/main/java/org/apache/flume/source/kafka/KafkaSourceUtil.java @@ -0,0 +1,63 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package org.apache.flume.source.kafka; + +import java.util.Map; +import java.util.Properties; + +import kafka.common.KafkaException; +import kafka.consumer.Consumer; +import kafka.consumer.ConsumerConfig; +import kafka.javaapi.consumer.ConsumerConnector; + +import org.apache.flume.Context; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +public class KafkaSourceUtil { + private static final Logger log = + LoggerFactory.getLogger(KafkaSourceUtil.class); + + public static Properties getKafkaConfigProperties(Context context) { + log.info("context={}",context.toString()); + Properties props = new Properties(); + Map contextMap = context.getParameters(); + for(String key : contextMap.keySet()) { + String value = contextMap.get(key).trim(); + key = key.trim(); + if (key.startsWith(KafkaSourceConstants.PROPERTY_PREFIX)) { + // remove the prefix + key = key.substring(KafkaSourceConstants.PROPERTY_PREFIX.length() + 1, + key.length()); + props.put(key, value); + if (log.isDebugEnabled()) { + log.debug("Reading a Kafka Producer Property: key: " + key + + ", value: " + value); + } + } + } + return props; + } + + public static ConsumerConnector getConsumer(Context context) { + ConsumerConfig consumerConfig = + new ConsumerConfig(getKafkaConfigProperties(context)); + ConsumerConnector consumer = + Consumer.createJavaConsumerConnector(consumerConfig); + return consumer; + } +} \ No newline at end of file diff --git a/flume-ng-sources/flume-kafka-source/src/test/java/org/apache/flume/source/kafka/KafkaSourceEmbeddedKafka.java b/flume-ng-sources/flume-kafka-source/src/test/java/org/apache/flume/source/kafka/KafkaSourceEmbeddedKafka.java new file mode 100644 index 0000000000..26c5c9d0aa --- /dev/null +++ b/flume-ng-sources/flume-kafka-source/src/test/java/org/apache/flume/source/kafka/KafkaSourceEmbeddedKafka.java @@ -0,0 +1,92 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package org.apache.flume.source.kafka; + +import kafka.server.KafkaConfig; +import kafka.server.KafkaServerStartable; +import kafka.javaapi.producer.Producer; +import kafka.producer.KeyedMessage; +import kafka.producer.ProducerConfig; +import kafka.admin.AdminUtils; +import org.I0Itec.zkclient.ZkClient; +import kafka.utils.ZKStringSerializer$; + +import java.io.IOException; +import java.util.Properties; + +public class KafkaSourceEmbeddedKafka { + KafkaServerStartable kafkaServer; + KafkaSourceEmbeddedZookeeper zookeeper; + int zkPort = 21818; // none-standard + Producer producer; + + public KafkaSourceEmbeddedKafka() { + zookeeper = new KafkaSourceEmbeddedZookeeper(zkPort); + Properties props = new Properties(); + props.put("zookeeper.connect",zookeeper.getConnectString()); + props.put("broker.id","1"); + KafkaConfig config = new KafkaConfig(props); + kafkaServer = new KafkaServerStartable(config); + kafkaServer.startup(); + initProducer(); + } + + public void stop() throws IOException { + producer.close(); + kafkaServer.shutdown(); + zookeeper.stopZookeeper(); + } + + public String getZkConnectString() { + return zookeeper.getConnectString(); + } + + private void initProducer() + { + Properties props = new Properties(); + props.put("metadata.broker.list","127.0.0.1:" + + kafkaServer.serverConfig().port()); + props.put("serializer.class","kafka.serializer.StringEncoder"); + props.put("request.required.acks", "1"); + + ProducerConfig config = new ProducerConfig(props); + + producer = new Producer(config); + + } + + public void produce(String topic, String k, String v) { + KeyedMessage message = new KeyedMessage(topic,k,v); + producer.send(message); + } + + public void createTopic(String topicName) { + // Create a ZooKeeper client + int sessionTimeoutMs = 10000; + int connectionTimeoutMs = 10000; + ZkClient zkClient = new ZkClient(zookeeper.getConnectString(), + sessionTimeoutMs, connectionTimeoutMs, + ZKStringSerializer$.MODULE$); + + int numPartitions = 1; + int replicationFactor = 1; + Properties topicConfig = new Properties(); + AdminUtils.createTopic(zkClient, topicName, numPartitions, + replicationFactor, topicConfig); + } + +} diff --git a/flume-ng-sources/flume-kafka-source/src/test/java/org/apache/flume/source/kafka/KafkaSourceEmbeddedZookeeper.java b/flume-ng-sources/flume-kafka-source/src/test/java/org/apache/flume/source/kafka/KafkaSourceEmbeddedZookeeper.java new file mode 100644 index 0000000000..1b8a27106e --- /dev/null +++ b/flume-ng-sources/flume-kafka-source/src/test/java/org/apache/flume/source/kafka/KafkaSourceEmbeddedZookeeper.java @@ -0,0 +1,64 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package org.apache.flume.source.kafka; + +import org.apache.zookeeper.server.NIOServerCnxnFactory; +import org.apache.zookeeper.server.ZooKeeperServer; +import org.apache.commons.io.FileUtils; + +import java.io.File; +import java.io.IOException; +import java.net.InetSocketAddress; + +public class KafkaSourceEmbeddedZookeeper { + private int zkPort; + private ZooKeeperServer zookeeper; + private NIOServerCnxnFactory factory; + File dir; + + + public KafkaSourceEmbeddedZookeeper(int zkPort){ + int numConnections = 5000; + int tickTime = 2000; + + this.zkPort = zkPort; + + String dataDirectory = System.getProperty("java.io.tmpdir"); + dir = new File(dataDirectory, "zookeeper").getAbsoluteFile(); + + try { + this.zookeeper = new ZooKeeperServer(dir,dir,tickTime); + this.factory = new NIOServerCnxnFactory(); + factory.configure(new InetSocketAddress("127.0.0.1",zkPort),0); + factory.startup(zookeeper); + } catch (IOException e) { + e.printStackTrace(); + } catch (InterruptedException e) { + e.printStackTrace(); + } + } + + public void stopZookeeper() throws IOException { + zookeeper.shutdown(); + factory.shutdown(); + FileUtils.deleteDirectory(dir); + } + + public String getConnectString() { + return "127.0.0.1:"+zkPort; + } +} diff --git a/flume-ng-sources/flume-kafka-source/src/test/java/org/apache/flume/source/kafka/KafkaSourceTest.java b/flume-ng-sources/flume-kafka-source/src/test/java/org/apache/flume/source/kafka/KafkaSourceTest.java new file mode 100644 index 0000000000..1009f1c091 --- /dev/null +++ b/flume-ng-sources/flume-kafka-source/src/test/java/org/apache/flume/source/kafka/KafkaSourceTest.java @@ -0,0 +1,195 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.flume.source.kafka; + +import static org.junit.Assert.assertEquals; +import static org.mockito.Matchers.any; +import static org.mockito.Matchers.anyList; +import static org.mockito.Mockito.*; + +import java.lang.reflect.Field; +import java.nio.ByteBuffer; +import java.util.List; + +import com.google.common.base.Charsets; +import com.google.common.collect.Lists; +import junit.framework.Assert; +import kafka.common.TopicExistsException; +import kafka.consumer.ConsumerIterator; +import kafka.message.Message; + +import kafka.message.MessageAndMetadata; + +import org.apache.flume.*; +import org.apache.flume.PollableSource.Status; +import org.apache.flume.channel.ChannelProcessor; +import org.apache.flume.source.AbstractSource; +import org.junit.After; +import org.junit.Before; +import org.junit.Test; +import org.junit.rules.ExpectedException; +import org.mockito.invocation.InvocationOnMock; +import org.mockito.stubbing.Answer; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +public class KafkaSourceTest { + private static final Logger log = + LoggerFactory.getLogger(KafkaSourceTest.class); + + private KafkaSource kafkaSource; + private KafkaSourceEmbeddedKafka kafkaServer; + private ConsumerIterator mockIt; + private Message message; + private Context context; + private List events; + private String topicName = "test1"; + + + @SuppressWarnings("unchecked") + @Before + public void setup() throws Exception { + + kafkaSource = new KafkaSource(); + kafkaServer = new KafkaSourceEmbeddedKafka(); + try { + kafkaServer.createTopic(topicName); + } catch (TopicExistsException e) { + //do nothing + } + + context = new Context(); + context.put(KafkaSourceConstants.ZOOKEEPER_CONNECT, + kafkaServer.getZkConnectString()); + context.put(KafkaSourceConstants.GROUP_ID,"flume"); + context.put(KafkaSourceConstants.TOPIC,topicName); + context.put(KafkaSourceConstants.CONSUMER_TIMEOUT,"100"); + + ChannelProcessor channelProcessor = mock(ChannelProcessor.class); + + events = Lists.newArrayList(); + + doAnswer(new Answer() { + @Override + public Void answer(InvocationOnMock invocation) throws Throwable { + events.addAll((List)invocation.getArguments()[0]); + return null; + } + }).when(channelProcessor).processEventBatch(any(List.class)); + kafkaSource.setChannelProcessor(channelProcessor); + } + + @After + public void tearDown() throws Exception { + kafkaSource.stop(); + kafkaServer.stop(); + } + + @SuppressWarnings("unchecked") + @Test + public void testProcessItNotEmpty() throws EventDeliveryException, + SecurityException, NoSuchFieldException, IllegalArgumentException, + IllegalAccessException, InterruptedException { + context.put(KafkaSourceConstants.BATCH_SIZE,"1"); + kafkaSource.configure(context); + kafkaSource.start(); + + Thread.sleep(500L); + + kafkaServer.produce(topicName, "", "hello, world"); + + Thread.sleep(500L); + + Assert.assertEquals(Status.READY, kafkaSource.process()); + Assert.assertEquals(Status.BACKOFF, kafkaSource.process()); + Assert.assertEquals(1, events.size()); + + Assert.assertEquals("hello, world", new String(events.get(0).getBody(), + Charsets.UTF_8)); + + + } + + @SuppressWarnings("unchecked") + @Test + public void testProcessItNotEmptyBatch() throws EventDeliveryException, + SecurityException, NoSuchFieldException, IllegalArgumentException, + IllegalAccessException, InterruptedException { + context.put(KafkaSourceConstants.BATCH_SIZE,"2"); + kafkaSource.configure(context); + kafkaSource.start(); + + Thread.sleep(500L); + + kafkaServer.produce(topicName, "", "hello, world"); + kafkaServer.produce(topicName, "", "foo, bar"); + + Thread.sleep(500L); + + Status status = kafkaSource.process(); + assertEquals(Status.READY, status); + Assert.assertEquals("hello, world", new String(events.get(0).getBody(), + Charsets.UTF_8)); + Assert.assertEquals("foo, bar", new String(events.get(1).getBody(), + Charsets.UTF_8)); + + } + + + @SuppressWarnings("unchecked") + @Test + public void testProcessItEmpty() throws EventDeliveryException, + SecurityException, NoSuchFieldException, IllegalArgumentException, + IllegalAccessException, InterruptedException { + kafkaSource.configure(context); + kafkaSource.start(); + Thread.sleep(500L); + + Status status = kafkaSource.process(); + assertEquals(Status.BACKOFF, status); + } + + @SuppressWarnings("unchecked") + @Test + public void testNonExistingTopic() throws EventDeliveryException, + SecurityException, NoSuchFieldException, IllegalArgumentException, + IllegalAccessException, InterruptedException { + context.put(KafkaSourceConstants.TOPIC,"faketopic"); + kafkaSource.configure(context); + kafkaSource.start(); + Thread.sleep(500L); + + Status status = kafkaSource.process(); + assertEquals(Status.BACKOFF, status); + } + + @SuppressWarnings("unchecked") + @Test(expected= FlumeException.class) + public void testNonExistingZk() throws EventDeliveryException, + SecurityException, NoSuchFieldException, IllegalArgumentException, + IllegalAccessException, InterruptedException { + context.put(KafkaSourceConstants.ZOOKEEPER_CONNECT,"blabla:666"); + kafkaSource.configure(context); + kafkaSource.start(); + Thread.sleep(500L); + + Status status = kafkaSource.process(); + assertEquals(Status.BACKOFF, status); + } + +} \ No newline at end of file diff --git a/flume-ng-sources/flume-kafka-source/src/test/java/org/apache/flume/source/kafka/KafkaSourceUtilTest.java b/flume-ng-sources/flume-kafka-source/src/test/java/org/apache/flume/source/kafka/KafkaSourceUtilTest.java new file mode 100644 index 0000000000..b9a1b25e27 --- /dev/null +++ b/flume-ng-sources/flume-kafka-source/src/test/java/org/apache/flume/source/kafka/KafkaSourceUtilTest.java @@ -0,0 +1,75 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.flume.source.kafka; + +import static org.junit.Assert.assertEquals; +import static org.junit.Assert.assertNotNull; +import static org.junit.Assert.assertNull; + +import java.io.IOException; +import java.util.Properties; + +import kafka.javaapi.consumer.ConsumerConnector; +import org.apache.flume.Context; +import org.apache.zookeeper.server.*; +import org.junit.After; +import org.junit.Before; +import org.junit.Test; + +public class KafkaSourceUtilTest { + private Properties props = new Properties(); + private Context context = new Context(); + private int zkPort = 21818; // none-standard + private KafkaSourceEmbeddedZookeeper zookeeper; + + @Before + public void setUp() throws Exception { + context.put("consumer.timeout", "10"); + context.put("type", "KafkaSource"); + context.put("topic", "test"); + props = KafkaSourceUtil.getKafkaConfigProperties(context); + zookeeper = new KafkaSourceEmbeddedZookeeper(zkPort); + + + } + + @After + public void tearDown() throws Exception { + zookeeper.stopZookeeper(); + } + + @Test + public void testGetKafkaConfigParameter() { + assertEquals("10",props.getProperty("consumer.timeout")); + assertEquals("test",props.getProperty("topic")); + assertNull(props.getProperty("type")); + } + + + @Test + public void testGetConsumer() { + context.put("zookeeper.connect", "127.0.0.1:"+zkPort); + context.put("group.id","test"); + + ConsumerConnector cc = KafkaSourceUtil.getConsumer(context); + assertNotNull(cc); + + } + + +} diff --git a/flume-ng-sources/flume-kafka-source/src/test/resources/log4j.properties b/flume-ng-sources/flume-kafka-source/src/test/resources/log4j.properties new file mode 100644 index 0000000000..78b106775c --- /dev/null +++ b/flume-ng-sources/flume-kafka-source/src/test/resources/log4j.properties @@ -0,0 +1,25 @@ +# +# Licensed to the Apache Software Foundation (ASF) under one or more +# contributor license agreements. See the NOTICE file distributed with +# this work for additional information regarding copyright ownership. +# The ASF licenses this file to You under the Apache License, Version 2.0 +# (the "License"); you may not use this file except in compliance with +# the License. You may obtain a copy of the License at +# +# http://www.apache.org/licenses/LICENSE-2.0 +# +# Unless required by applicable law or agreed to in writing, software +# distributed under the License is distributed on an "AS IS" BASIS, +# WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +# See the License for the specific language governing permissions and +# limitations under the License. +# + + +log4j.rootLogger = INFO, out + +log4j.appender.out = org.apache.log4j.ConsoleAppender +log4j.appender.out.layout = org.apache.log4j.PatternLayout +log4j.appender.out.layout.ConversionPattern = %d (%t) [%p - %l] %m%n + +log4j.logger.org.apache.flume = INFO \ No newline at end of file diff --git a/flume-ng-sources/pom.xml b/flume-ng-sources/pom.xml index c03307a3d7..ab8eca4e35 100644 --- a/flume-ng-sources/pom.xml +++ b/flume-ng-sources/pom.xml @@ -44,6 +44,7 @@ limitations under the License. flume-scribe-source flume-jms-source flume-twitter-source + flume-kafka-source diff --git a/pom.xml b/pom.xml index 740edc209d..8ee82f38f5 100644 --- a/pom.xml +++ b/pom.xml @@ -1168,6 +1168,12 @@ limitations under the License. 1.6.0-SNAPSHOT + + org.apache.flume.flume-ng-sources + flume-kafka-source + 1.6.0-SNAPSHOT + + org.apache.flume.flume-ng-legacy-sources flume-avro-source @@ -1270,6 +1276,20 @@ limitations under the License. 3.0.3 + + + org.apache.kafka + kafka_2.10 + 0.8.1.1 + + + org.apache.kafka + kafka_2.10 + 0.8.1.1 + test + test + + org.kitesdk kite-data-core From 186a3b808b371cbf2c4b5a47325caa324c7c7a6e Mon Sep 17 00:00:00 2001 From: Hari Shreedharan Date: Tue, 23 Sep 2014 22:33:55 -0700 Subject: [PATCH 139/341] FLUME-2455. Kafka Sink Documentation. (Thilina Buddika, Gwen Shapira via Hari) --- flume-ng-doc/sphinx/FlumeUserGuide.rst | 54 ++++++++++++++++++++++++++ 1 file changed, 54 insertions(+) diff --git a/flume-ng-doc/sphinx/FlumeUserGuide.rst b/flume-ng-doc/sphinx/FlumeUserGuide.rst index 3a4756091c..11c1ad78f1 100644 --- a/flume-ng-doc/sphinx/FlumeUserGuide.rst +++ b/flume-ng-doc/sphinx/FlumeUserGuide.rst @@ -2137,6 +2137,60 @@ auth.proxyUser -- The effective user for HDFS actions, if differ the kerberos principal ======================= ======= =========================================================== + +Kafka Sink +~~~~~~~~~~ +This is a Flume Sink implementation that can publish data to a +`Kafka `_ topic. One of the objective is to integrate Flume +with Kafka so that pull based processing systems can process the data coming +through various Flume sources. This currently supports Kafka 0.8.x series of releases. + +Required properties are marked in bold font. + + +=============================== =================== ============================================================================================= +Property Name Default Description +=============================== =================== ============================================================================================= +**type** -- Must be set to ``org.apache.flume.sink.kafka.KafkaSink`` +**kafka.metadata.broker.list** -- List of brokers Kafka-Sink will connect to, to get the list of topic partitions + This can be a partial list of brokers, but we recommend at least two for HA. + The format is comma separated list of hostname:port +topic default-flume-topic The topic in Kafka to which the messages will be published. If this parameter is configured, + messages will be published to this topic. + If the event header contains a "topic" field, the event will be published to that topic + overriding the topic configured here. +batchSize 100 How many messages to process in one batch. Larger batches improve throughput while adding latency. +kafka.request.required.acks 0 How many replicas must acknowledge a message before its considered successfully written. + Accepted values are 0 (Never wait for acknowledgement), 1 (wait for leader only), -1 (wait for all replicas) + The default is the fastest option, but we *highly recommend* setting this to -1 to avoid data loss +kafka.producer.type sync Whether messages should be sent to broker synchronously or using an asynchronous background thread. + Accepted values are sync (safest) and async (faster but potentially unsafe) +Other Kafka Producer Properties -- These properties are used to configure the Kafka Producer. Any producer property supported + by Kafka can be used. The only requirement is to prepend the property name with the prefix ``kafka.``. +=============================== =================== ============================================================================================= + +.. note:: Kafka Sink uses the ``topic`` and ``key`` properties from the FlumeEvent headers to send events to Kafka. + If ``topic`` exists in the headers, the event will be sent to that specific topic, overriding the topic configured for the Sink. + If ``key`` exists in the headers, the key will used by Kafka to partition the data between the topic partitions. Events with same key + will be sent to the same partition. If the key is null, events will be sent to random partitions. + +An example configuration of a Kafka sink is given below. Properties starting +with the prefix ``kafka`` (the last 3 properties) are used when instantiating +the Kafka producer. The properties that are passed when creating the Kafka +producer are not limited to the properties given in this example. +Also it's possible include your custom properties here and access them inside +the preprocessor through the Flume Context object passed in as a method +argument. + +.. code-block:: properties + + a1.sinks.k1.type = org.apache.flume.sink.kafka.KafkaSink + a1.sinks.k1.topic = mytopic + a1.sinks.k1.kafka.metadata.broker.list = localhost:9092 + a1.sinks.k1.kafka.request.required.acks = 1 + a1.sinks.k1.batchSize = 20 + a1.sinks.k1.channel = c1 + Custom Sink ~~~~~~~~~~~ From bde2c28211a2d05a9930f1599cb15864ad3cdba0 Mon Sep 17 00:00:00 2001 From: Hari Shreedharan Date: Tue, 23 Sep 2014 23:10:25 -0700 Subject: [PATCH 140/341] FLUME-2470. Kafka Sink and Source must use camel case for all configs. (Gwen Shapira via Hari) --- flume-ng-doc/sphinx/FlumeUserGuide.rst | 67 +++++++------- .../apache/flume/sink/kafka/KafkaSink.java | 65 ++++++-------- .../flume/sink/kafka/KafkaSinkConstants.java | 20 ++++- .../flume/sink/kafka/TestKafkaSink.java | 23 ++--- .../test/resources/kafka-server.properties | 1 + .../src/test/resources/log4j.properties | 2 +- .../flume/source/kafka/KafkaSource.java | 40 ++++++--- .../source/kafka/KafkaSourceConstants.java | 15 ++-- .../flume/source/kafka/KafkaSourceUtil.java | 87 +++++++++++++++---- .../flume/source/kafka/KafkaSourceTest.java | 12 ++- .../source/kafka/KafkaSourceUtilTest.java | 43 ++++++--- 11 files changed, 232 insertions(+), 143 deletions(-) diff --git a/flume-ng-doc/sphinx/FlumeUserGuide.rst b/flume-ng-doc/sphinx/FlumeUserGuide.rst index 11c1ad78f1..ce52946636 100644 --- a/flume-ng-doc/sphinx/FlumeUserGuide.rst +++ b/flume-ng-doc/sphinx/FlumeUserGuide.rst @@ -1101,29 +1101,33 @@ Kafka Source is an Apache Kafka consumer that reads messages from a Kafka topic. If you have multiple Kafka sources running, you can configure them with the same Consumer Group so each will read a unique set of partitions for the topic. -The properties below are required properties, but you can specify any Kafka parameter you want -and it will be passed to the consumer. Check `Kafka documentation `_ -for details - -=========================== =========== =================================================== -Property Name Default Description -=========================== =========== =================================================== -**channels** -- -**type** -- The component type name, needs to be ``org.apache.flume.source.kafka,KafkaSource`` -**kafka.zookeeper.connect** -- URI of ZooKeeper used by Kafka cluster -**kadka.group.id** -- Unique identified of consumer group. Setting the same id in multiple sources or agents - indicates that they are part of the same consumer group -**topic** -- Kafka topic we'll read messages from. At the time, this is a single topic only. -batchSize 1000 Maximum number of messages written to Channel in one batch -batchDurationMillis 1000 Maximum time (in ms) before a batch will be written to Channel - The batch will be written whenever the first of size and time will be reached. -kafka.auto.commit.enable false If true, Kafka will commit events automatically - faster but less durable option. - when false, the Kafka Source will commit events before writing batch to channel -consumer.timeout.ms 10 Polling interval for new data for batch. - Low value means more CPU usage. - High value means the maxBatchDurationMillis may be missed while waiting for - additional data. -=========================== =========== =================================================== + + +=============================== =========== =================================================== +Property Name Default Description +=============================== =========== =================================================== +**channels** -- +**type** -- The component type name, needs to be ``org.apache.flume.source.kafka,KafkaSource`` +**zookeeperConnect** -- URI of ZooKeeper used by Kafka cluster +**groupId** flume Unique identified of consumer group. Setting the same id in multiple sources or agents + indicates that they are part of the same consumer group +**topic** -- Kafka topic we'll read messages from. At the time, this is a single topic only. +batchSize 1000 Maximum number of messages written to Channel in one batch +batchDurationMillis 1000 Maximum time (in ms) before a batch will be written to Channel + The batch will be written whenever the first of size and time will be reached. +Other Kafka Consumer Properties -- These properties are used to configure the Kafka Consumer. Any producer property supported + by Kafka can be used. The only requirement is to prepend the property name with the prefix ``kafka.``. + For example: kafka.consumer.timeout.ms + Check `Kafka documentation ` for details +=============================== =========== =================================================== + +.. note:: The Kafka Source overrides two Kafka consumer parameters: + auto.commit.enable is set to "false" by the source and we commit every batch. For improved performance + this can be set to "true", however, this can lead to loss of data + consumer.timeout.ms is set to 10ms, so when we check Kafka for new data we wait at most 10ms for the data to arrive + setting this to a higher value can reduce CPU utilization (we'll poll Kafka in less of a tight loop), but also means + higher latency in writing batches to channel (since we'll wait longer for data to arrive). + Example for agent named tier1: @@ -1131,9 +1135,9 @@ Example for agent named tier1: tier1.sources.source1.type = org.apache.flume.source.kafka.KafkaSource tier1.sources.source1.channels = channel1 - tier1.sources.source1.kafka.zookeeper.connect = localhost:2181 + tier1.sources.source1.zookeeperConnect = localhost:2181 tier1.sources.source1.topic = test1 - tier1.sources.source1.kafka.group.id = flume + tier1.sources.source1.groupId = flume tier1.sources.source1.kafka.consumer.timeout.ms = 100 @@ -2152,7 +2156,7 @@ Required properties are marked in bold font. Property Name Default Description =============================== =================== ============================================================================================= **type** -- Must be set to ``org.apache.flume.sink.kafka.KafkaSink`` -**kafka.metadata.broker.list** -- List of brokers Kafka-Sink will connect to, to get the list of topic partitions +**brokerList** -- List of brokers Kafka-Sink will connect to, to get the list of topic partitions This can be a partial list of brokers, but we recommend at least two for HA. The format is comma separated list of hostname:port topic default-flume-topic The topic in Kafka to which the messages will be published. If this parameter is configured, @@ -2160,13 +2164,12 @@ topic default-flume-topic The topic in Kafka to whic If the event header contains a "topic" field, the event will be published to that topic overriding the topic configured here. batchSize 100 How many messages to process in one batch. Larger batches improve throughput while adding latency. -kafka.request.required.acks 0 How many replicas must acknowledge a message before its considered successfully written. +requiredAcks 1 How many replicas must acknowledge a message before its considered successfully written. Accepted values are 0 (Never wait for acknowledgement), 1 (wait for leader only), -1 (wait for all replicas) - The default is the fastest option, but we *highly recommend* setting this to -1 to avoid data loss -kafka.producer.type sync Whether messages should be sent to broker synchronously or using an asynchronous background thread. - Accepted values are sync (safest) and async (faster but potentially unsafe) + Set this to -1 to avoid data loss in some cases of leader failure. Other Kafka Producer Properties -- These properties are used to configure the Kafka Producer. Any producer property supported by Kafka can be used. The only requirement is to prepend the property name with the prefix ``kafka.``. + For example: kafka.producer.type =============================== =================== ============================================================================================= .. note:: Kafka Sink uses the ``topic`` and ``key`` properties from the FlumeEvent headers to send events to Kafka. @@ -2186,8 +2189,8 @@ argument. a1.sinks.k1.type = org.apache.flume.sink.kafka.KafkaSink a1.sinks.k1.topic = mytopic - a1.sinks.k1.kafka.metadata.broker.list = localhost:9092 - a1.sinks.k1.kafka.request.required.acks = 1 + a1.sinks.k1.brokerList = localhost:9092 + a1.sinks.k1.requiredAcks = 1 a1.sinks.k1.batchSize = 20 a1.sinks.k1.channel = c1 diff --git a/flume-ng-sinks/flume-ng-kafka-sink/src/main/java/org/apache/flume/sink/kafka/KafkaSink.java b/flume-ng-sinks/flume-ng-kafka-sink/src/main/java/org/apache/flume/sink/kafka/KafkaSink.java index a6121ac7e8..a90b950684 100644 --- a/flume-ng-sinks/flume-ng-kafka-sink/src/main/java/org/apache/flume/sink/kafka/KafkaSink.java +++ b/flume-ng-sinks/flume-ng-kafka-sink/src/main/java/org/apache/flume/sink/kafka/KafkaSink.java @@ -28,10 +28,10 @@ Licensed to the Apache Software Foundation (ASF) under one or more import org.slf4j.Logger; import org.slf4j.LoggerFactory; -import java.util.ArrayList; +import java.util.Properties; import java.util.List; import java.util.Map; -import java.util.Properties; +import java.util.ArrayList; /** * A Flume Sink that can publish messages to Kafka. @@ -43,11 +43,7 @@ Licensed to the Apache Software Foundation (ASF) under one or more * partition key *

    * Mandatory properties are: - * kafka.metadata.broker.list -- can be a partial list, - * but at least 2 are recommended for HA - * kafka.request.required.acks -- 0 (unsafe), 1 (accepted by at least one - * broker), -1 (accepted by all brokers) - * kafka.producer.type -- for safety, this should be sync + * brokerList -- can be a partial list, but at least 2 are recommended for HA *

    *

    * however, any property starting with "kafka." will be passed along to the @@ -60,6 +56,8 @@ Licensed to the Apache Software Foundation (ASF) under one or more * different topics * batchSize - how many messages to process in one batch. Larger batches * improve throughput while adding latency. + * requiredAcks -- 0 (unsafe), 1 (accepted by at least one broker, default), + * -1 (accepted by all brokers) *

    * header properties (per event): * topic @@ -70,7 +68,7 @@ public class KafkaSink extends AbstractSink implements Configurable { private static final Logger logger = LoggerFactory.getLogger(KafkaSink.class); public static final String KEY_HDR = "key"; public static final String TOPIC_HDR = "topic"; - private Properties producerProps; + private Properties kafkaProps; private Producer producer; private String topic; private int batchSize; @@ -154,7 +152,7 @@ public Status process() throws EventDeliveryException { @Override public synchronized void start() { // instantiate the producer - ProducerConfig config = new ProducerConfig(producerProps); + ProducerConfig config = new ProducerConfig(kafkaProps); producer = new Producer(config); super.start(); } @@ -166,54 +164,43 @@ public synchronized void stop() { } + /** + * We configure the sink and generate properties for the Kafka Producer + * + * Kafka producer properties is generated as follows: + * 1. We generate a properties object with some static defaults that + * can be overridden by Sink configuration + * 2. We add the configuration users added for Kafka (parameters starting + * with .kafka. and must be valid Kafka Producer properties + * 3. We add the sink's documented parameters which can override other + * properties + * + * @param context + */ @Override public void configure(Context context) { batchSize = context.getInteger(KafkaSinkConstants.BATCH_SIZE, KafkaSinkConstants.DEFAULT_BATCH_SIZE); - logger.debug("Using batch size: {}", batchSize); messageList = new ArrayList>(batchSize); - Map params = context.getParameters(); - logger.debug("all params: " + params.entrySet().toString()); - setProducerProps(params); - if (!producerProps.contains("serializer.class")) { - producerProps.put("serializer.class", "kafka.serializer.DefaultEncoder"); - } - if (!producerProps.contains("key.serializer.class")) { - producerProps.put("key.serializer.class", - "kafka.serializer.StringEncoder"); - } + logger.debug("Using batch size: {}", batchSize); topic = context.getString(KafkaSinkConstants.TOPIC, KafkaSinkConstants.DEFAULT_TOPIC); if (topic.equals(KafkaSinkConstants.DEFAULT_TOPIC)) { - logger.warn("The Properties 'preprocessor' or 'topic' is not set. " + - "Using the default topic name" + + logger.warn("The Property 'topic' is not set. " + + "Using the default topic name: " + KafkaSinkConstants.DEFAULT_TOPIC); } else { logger.info("Using the static topic: " + topic + " this may be over-ridden by event headers"); } - } + kafkaProps = KafkaSinkUtil.getKafkaProperties(context); - private void setProducerProps(Map params) { - producerProps = new Properties(); - for (String key : params.keySet()) { - String value = params.get(key).trim(); - key = key.trim(); - if (key.startsWith(KafkaSinkConstants.PROPERTY_PREFIX)) { - // remove the prefix - key = key.substring(KafkaSinkConstants.PROPERTY_PREFIX.length() + 1, - key.length()); - producerProps.put(key.trim(), value); - if (logger.isDebugEnabled()) { - logger.debug("Reading a Kafka Producer Property: key: " + key + - ", value: " + value); - } - } + if (logger.isDebugEnabled()) { + logger.debug("Kafka producer properties: " + kafkaProps); } } - } diff --git a/flume-ng-sinks/flume-ng-kafka-sink/src/main/java/org/apache/flume/sink/kafka/KafkaSinkConstants.java b/flume-ng-sinks/flume-ng-kafka-sink/src/main/java/org/apache/flume/sink/kafka/KafkaSinkConstants.java index 48d875ebfb..3ee12de636 100644 --- a/flume-ng-sinks/flume-ng-kafka-sink/src/main/java/org/apache/flume/sink/kafka/KafkaSinkConstants.java +++ b/flume-ng-sinks/flume-ng-kafka-sink/src/main/java/org/apache/flume/sink/kafka/KafkaSinkConstants.java @@ -18,14 +18,30 @@ Licensed to the Apache Software Foundation (ASF) under one or more package org.apache.flume.sink.kafka; +import kafka.serializer.StringDecoder; + public class KafkaSinkConstants { - public static final String PROPERTY_PREFIX = "kafka"; + public static final String PROPERTY_PREFIX = "kafka."; /* Properties */ - public static final String DEFAULT_TOPIC = "default-flume-topic"; + public static final String TOPIC = "topic"; public static final String BATCH_SIZE = "batchSize"; + public static final String MESSAGE_SERIALIZER_KEY = "serializer.class"; + public static final String KEY_SERIALIZER_KEY = "key.serializer.class"; + public static final String BROKER_LIST_KEY = "metadata.broker.list"; + public static final String REQUIRED_ACKS_KEY = "request.required.acks"; + public static final String BROKER_LIST_FLUME_KEY = "brokerList"; + public static final String REQUIRED_ACKS_FLUME_KEY = "requiredAcks"; + + public static final int DEFAULT_BATCH_SIZE = 100; + public static final String DEFAULT_TOPIC = "default-flume-topic"; + public static final String DEFAULT_MESSAGE_SERIALIZER = + "kafka.serializer.DefaultEncoder"; + public static final String DEFAULT_KEY_SERIALIZER = + "kafka.serializer.StringEncoder"; + public static final String DEFAULT_REQUIRED_ACKS = "1"; } diff --git a/flume-ng-sinks/flume-ng-kafka-sink/src/test/java/org/apache/flume/sink/kafka/TestKafkaSink.java b/flume-ng-sinks/flume-ng-kafka-sink/src/test/java/org/apache/flume/sink/kafka/TestKafkaSink.java index aed6dace83..80f764f729 100644 --- a/flume-ng-sinks/flume-ng-kafka-sink/src/test/java/org/apache/flume/sink/kafka/TestKafkaSink.java +++ b/flume-ng-sinks/flume-ng-kafka-sink/src/test/java/org/apache/flume/sink/kafka/TestKafkaSink.java @@ -34,6 +34,7 @@ Licensed to the Apache Software Foundation (ASF) under one or more import java.util.List; import java.util.Map; + import static org.junit.Assert.assertEquals; import static org.junit.Assert.assertNull; import static org.junit.Assert.fail; @@ -126,7 +127,7 @@ public void testTopicAndKeyFromHeader() throws UnsupportedEncodingException { kafkaSink.setChannel(memoryChannel); kafkaSink.start(); - String msg = "my message"; + String msg = "test-topic-and-key-from-header"; Map headers = new HashMap(); headers.put("topic", TestConstants.CUSTOM_TOPIC); headers.put("key", TestConstants.CUSTOM_KEY); @@ -156,9 +157,8 @@ public void testTopicAndKeyFromHeader() throws UnsupportedEncodingException { } @Test - public void testEmptyChannel() throws UnsupportedEncodingException { - - + public void testEmptyChannel() throws UnsupportedEncodingException, + EventDeliveryException { Sink kafkaSink = new KafkaSink(); Context context = prepareDefaultContext(); Configurables.configure(kafkaSink, context); @@ -167,25 +167,20 @@ public void testEmptyChannel() throws UnsupportedEncodingException { kafkaSink.setChannel(memoryChannel); kafkaSink.start(); - try { - Sink.Status status = kafkaSink.process(); - if (status == Sink.Status.BACKOFF) { - fail("Error Occurred"); - } - } catch (EventDeliveryException ex) { - // ignore + Sink.Status status = kafkaSink.process(); + if (status == Sink.Status.BACKOFF) { + fail("Error Occurred"); } assertNull( testUtil.getNextMessageFromConsumer(KafkaSinkConstants.DEFAULT_TOPIC)); - } - private Context prepareDefaultContext() { // Prepares a default context with Kafka Server Properties Context context = new Context(); - context.put("kafka.metadata.broker.list", testUtil.getKafkaServerUrl()); + context.put("brokerList", testUtil.getKafkaServerUrl()); context.put("kafka.request.required.acks", "1"); + context.put("kafka.producer.type","sync"); context.put("batchSize", "1"); return context; } diff --git a/flume-ng-sinks/flume-ng-kafka-sink/src/test/resources/kafka-server.properties b/flume-ng-sinks/flume-ng-kafka-sink/src/test/resources/kafka-server.properties index c07cdea66e..02a81e2901 100644 --- a/flume-ng-sinks/flume-ng-kafka-sink/src/test/resources/kafka-server.properties +++ b/flume-ng-sinks/flume-ng-kafka-sink/src/test/resources/kafka-server.properties @@ -1,4 +1,5 @@ # Licensed to the Apache Software Foundation (ASF) under one or more +# Licensed to the Apache Software Foundation (ASF) under one or more # contributor license agreements. See the NOTICE file distributed with # this work for additional information regarding copyright ownership. # The ASF licenses this file to You under the Apache License, Version 2.0 diff --git a/flume-ng-sinks/flume-ng-kafka-sink/src/test/resources/log4j.properties b/flume-ng-sinks/flume-ng-kafka-sink/src/test/resources/log4j.properties index bdcb643c9e..b86600b863 100644 --- a/flume-ng-sinks/flume-ng-kafka-sink/src/test/resources/log4j.properties +++ b/flume-ng-sinks/flume-ng-kafka-sink/src/test/resources/log4j.properties @@ -15,7 +15,7 @@ kafka.logs.dir=target/logs -log4j.rootLogger=INFO, stdout +log4j.rootLogger=INFO, stdout log4j.appender.stdout=org.apache.log4j.ConsoleAppender log4j.appender.stdout.layout=org.apache.log4j.PatternLayout diff --git a/flume-ng-sources/flume-kafka-source/src/main/java/org/apache/flume/source/kafka/KafkaSource.java b/flume-ng-sources/flume-kafka-source/src/main/java/org/apache/flume/source/kafka/KafkaSource.java index da78f80add..231ae4237b 100644 --- a/flume-ng-sources/flume-kafka-source/src/main/java/org/apache/flume/source/kafka/KafkaSource.java +++ b/flume-ng-sources/flume-kafka-source/src/main/java/org/apache/flume/source/kafka/KafkaSource.java @@ -20,6 +20,7 @@ import java.util.HashMap; import java.util.List; import java.util.Map; +import java.util.Properties; import kafka.consumer.ConsumerIterator; import kafka.consumer.ConsumerTimeoutException; @@ -66,6 +67,7 @@ public class KafkaSource extends AbstractSource private int consumerTimeout; private boolean kafkaAutoCommitEnabled; private Context context; + private Properties kafkaProps; private final List eventList = new ArrayList(); public Status process() throws EventDeliveryException { @@ -122,6 +124,19 @@ public Status process() throws EventDeliveryException { } } + /** + * We configure the source and generate properties for the Kafka Consumer + * + * Kafka Consumer properties are generated as follows: + * 1. Generate a properties object with some static defaults that + * can be overridden by Source configuration + * 2. We add the configuration users added for Kafka (parameters starting + * with kafka. and must be valid Kafka Consumer properties + * 3. We add the source documented parameters which can override other + * properties + * + * @param context + */ public void configure(Context context) { this.context = context; batchUpperLimit = context.getInteger(KafkaSourceConstants.BATCH_SIZE, @@ -130,21 +145,16 @@ public void configure(Context context) { KafkaSourceConstants.DEFAULT_BATCH_DURATION); topic = context.getString(KafkaSourceConstants.TOPIC); - //if consumer timeout and autocommit were not set by user, - // set them to 10ms and false - consumerTimeout = context.getInteger(KafkaSourceConstants.CONSUMER_TIMEOUT, - KafkaSourceConstants.DEFAULT_CONSUMER_TIMEOUT); - context.put(KafkaSourceConstants.CONSUMER_TIMEOUT, - Integer.toString(consumerTimeout)); - String autoCommit = context.getString( - KafkaSourceConstants.AUTO_COMMIT_ENABLED, - String.valueOf(KafkaSourceConstants.DEFAULT_AUTO_COMMIT)); - kafkaAutoCommitEnabled = Boolean.valueOf(autoCommit); - context.put(KafkaSourceConstants.AUTO_COMMIT_ENABLED,autoCommit); - if(topic == null) { throw new ConfigurationException("Kafka topic must be specified."); } + + kafkaProps = KafkaSourceUtil.getKafkaProperties(context); + consumerTimeout = Integer.parseInt(kafkaProps.getProperty( + KafkaSourceConstants.CONSUMER_TIMEOUT)); + kafkaAutoCommitEnabled = Boolean.parseBoolean(kafkaProps.getProperty( + KafkaSourceConstants.AUTO_COMMIT_ENABLED)); + } @Override @@ -153,7 +163,7 @@ public synchronized void start() { try { //initialize a consumer. This creates the connection to ZooKeeper - consumer = KafkaSourceUtil.getConsumer(context); + consumer = KafkaSourceUtil.getConsumer(kafkaProps); } catch (Exception e) { throw new FlumeException("Unable to create consumer. " + "Check whether the ZooKeeper server is up and that the " + @@ -192,6 +202,10 @@ public synchronized void stop() { } + + + + /** * Check if there are messages waiting in Kafka, * waiting until timeout (10ms by default) for messages to arrive. diff --git a/flume-ng-sources/flume-kafka-source/src/main/java/org/apache/flume/source/kafka/KafkaSourceConstants.java b/flume-ng-sources/flume-kafka-source/src/main/java/org/apache/flume/source/kafka/KafkaSourceConstants.java index ac86f6592e..169cc1001e 100644 --- a/flume-ng-sources/flume-kafka-source/src/main/java/org/apache/flume/source/kafka/KafkaSourceConstants.java +++ b/flume-ng-sources/flume-kafka-source/src/main/java/org/apache/flume/source/kafka/KafkaSourceConstants.java @@ -21,16 +21,19 @@ public class KafkaSourceConstants { public static final String TIMESTAMP = "timestamp"; public static final String BATCH_SIZE = "batchSize"; public static final String BATCH_DURATION_MS = "batchDurationMillis"; - public static final String CONSUMER_TIMEOUT = "kafka.consumer.timeout.ms"; - public static final String AUTO_COMMIT_ENABLED = "kafka.auto.commit.enabled"; - public static final String ZOOKEEPER_CONNECT = "kafka.zookeeper.connect"; - public static final String GROUP_ID = "kafka.group.id"; - public static final String PROPERTY_PREFIX = "kafka"; + public static final String CONSUMER_TIMEOUT = "consumer.timeout.ms"; + public static final String AUTO_COMMIT_ENABLED = "auto.commit.enabled"; + public static final String ZOOKEEPER_CONNECT = "zookeeper.connect"; + public static final String ZOOKEEPER_CONNECT_FLUME = "zookeeperConnect"; + public static final String GROUP_ID = "group.id"; + public static final String GROUP_ID_FLUME = "groupId"; + public static final String PROPERTY_PREFIX = "kafka."; public static final int DEFAULT_BATCH_SIZE = 1000; public static final int DEFAULT_BATCH_DURATION = 1000; - public static final int DEFAULT_CONSUMER_TIMEOUT = 10; + public static final String DEFAULT_CONSUMER_TIMEOUT = "10"; public static final boolean DEFAULT_AUTO_COMMIT = false; + public static final String DEFAULT_GROUP_ID = "flume"; } diff --git a/flume-ng-sources/flume-kafka-source/src/main/java/org/apache/flume/source/kafka/KafkaSourceUtil.java b/flume-ng-sources/flume-kafka-source/src/main/java/org/apache/flume/source/kafka/KafkaSourceUtil.java index 8397272bfe..4a4034bd82 100644 --- a/flume-ng-sources/flume-kafka-source/src/main/java/org/apache/flume/source/kafka/KafkaSourceUtil.java +++ b/flume-ng-sources/flume-kafka-source/src/main/java/org/apache/flume/source/kafka/KafkaSourceUtil.java @@ -25,6 +25,7 @@ import kafka.javaapi.consumer.ConsumerConnector; import org.apache.flume.Context; +import org.apache.flume.conf.ConfigurationException; import org.slf4j.Logger; import org.slf4j.LoggerFactory; @@ -32,32 +33,80 @@ public class KafkaSourceUtil { private static final Logger log = LoggerFactory.getLogger(KafkaSourceUtil.class); - public static Properties getKafkaConfigProperties(Context context) { + public static Properties getKafkaProperties(Context context) { log.info("context={}",context.toString()); - Properties props = new Properties(); - Map contextMap = context.getParameters(); - for(String key : contextMap.keySet()) { - String value = contextMap.get(key).trim(); - key = key.trim(); - if (key.startsWith(KafkaSourceConstants.PROPERTY_PREFIX)) { - // remove the prefix - key = key.substring(KafkaSourceConstants.PROPERTY_PREFIX.length() + 1, - key.length()); - props.put(key, value); - if (log.isDebugEnabled()) { - log.debug("Reading a Kafka Producer Property: key: " + key + - ", value: " + value); - } - } - } + Properties props = generateDefaultKafkaProps(); + setKafkaProps(context,props); + addDocumentedKafkaProps(context,props); return props; } - public static ConsumerConnector getConsumer(Context context) { + public static ConsumerConnector getConsumer(Properties kafkaProps) { ConsumerConfig consumerConfig = - new ConsumerConfig(getKafkaConfigProperties(context)); + new ConsumerConfig(kafkaProps); ConsumerConnector consumer = Consumer.createJavaConsumerConnector(consumerConfig); return consumer; } + + /** + * Generate consumer properties object with some defaults + * @return + */ + private static Properties generateDefaultKafkaProps() { + Properties props = new Properties(); + props.put(KafkaSourceConstants.AUTO_COMMIT_ENABLED, + KafkaSourceConstants.DEFAULT_AUTO_COMMIT); + props.put(KafkaSourceConstants.CONSUMER_TIMEOUT, + KafkaSourceConstants.DEFAULT_CONSUMER_TIMEOUT); + props.put(KafkaSourceConstants.GROUP_ID, + KafkaSourceConstants.DEFAULT_GROUP_ID); + return props; + } + + /** + * Add all configuration parameters starting with "kafka" + * to consumer properties + */ + private static void setKafkaProps(Context context,Properties kafkaProps) { + + Map kafkaProperties = + context.getSubProperties(KafkaSourceConstants.PROPERTY_PREFIX); + + for (Map.Entry prop : kafkaProperties.entrySet()) { + + kafkaProps.put(prop.getKey(), prop.getValue()); + if (log.isDebugEnabled()) { + log.debug("Reading a Kafka Producer Property: key: " + + prop.getKey() + ", value: " + prop.getValue()); + } + } + } + + /** + * Some of the producer properties are especially important + * We documented them and gave them a camel-case name to match Flume config + * If user set these, we will override any existing parameters with these + * settings. + * Knowledge of which properties are documented is maintained here for now. + * If this will become a maintenance issue we'll set a proper data structure. + */ + private static void addDocumentedKafkaProps(Context context, + Properties kafkaProps) + throws ConfigurationException { + String zookeeperConnect = context.getString( + KafkaSourceConstants.ZOOKEEPER_CONNECT_FLUME); + if (zookeeperConnect == null) { + throw new ConfigurationException("ZookeeperConnect must contain " + + "at least one ZooKeeper server"); + } + kafkaProps.put(KafkaSourceConstants.ZOOKEEPER_CONNECT, zookeeperConnect); + + String groupID = context.getString(KafkaSourceConstants.GROUP_ID_FLUME); + + if (groupID != null ) { + kafkaProps.put(KafkaSourceConstants.GROUP_ID, groupID); + } + } + } \ No newline at end of file diff --git a/flume-ng-sources/flume-kafka-source/src/test/java/org/apache/flume/source/kafka/KafkaSourceTest.java b/flume-ng-sources/flume-kafka-source/src/test/java/org/apache/flume/source/kafka/KafkaSourceTest.java index 1009f1c091..d067e24d81 100644 --- a/flume-ng-sources/flume-kafka-source/src/test/java/org/apache/flume/source/kafka/KafkaSourceTest.java +++ b/flume-ng-sources/flume-kafka-source/src/test/java/org/apache/flume/source/kafka/KafkaSourceTest.java @@ -25,6 +25,7 @@ import java.lang.reflect.Field; import java.nio.ByteBuffer; import java.util.List; +import java.util.Properties; import com.google.common.base.Charsets; import com.google.common.collect.Lists; @@ -38,6 +39,7 @@ import org.apache.flume.*; import org.apache.flume.PollableSource.Status; import org.apache.flume.channel.ChannelProcessor; +import org.apache.flume.conf.Configurables; import org.apache.flume.source.AbstractSource; import org.junit.After; import org.junit.Before; @@ -74,11 +76,11 @@ public void setup() throws Exception { } context = new Context(); - context.put(KafkaSourceConstants.ZOOKEEPER_CONNECT, + context.put(KafkaSourceConstants.ZOOKEEPER_CONNECT_FLUME, kafkaServer.getZkConnectString()); - context.put(KafkaSourceConstants.GROUP_ID,"flume"); + context.put(KafkaSourceConstants.GROUP_ID_FLUME,"flume"); context.put(KafkaSourceConstants.TOPIC,topicName); - context.put(KafkaSourceConstants.CONSUMER_TIMEOUT,"100"); + context.put("kafka.consumer.timeout.ms","100"); ChannelProcessor channelProcessor = mock(ChannelProcessor.class); @@ -183,7 +185,7 @@ public void testNonExistingTopic() throws EventDeliveryException, public void testNonExistingZk() throws EventDeliveryException, SecurityException, NoSuchFieldException, IllegalArgumentException, IllegalAccessException, InterruptedException { - context.put(KafkaSourceConstants.ZOOKEEPER_CONNECT,"blabla:666"); + context.put(KafkaSourceConstants.ZOOKEEPER_CONNECT_FLUME,"blabla:666"); kafkaSource.configure(context); kafkaSource.start(); Thread.sleep(500L); @@ -192,4 +194,6 @@ public void testNonExistingZk() throws EventDeliveryException, assertEquals(Status.BACKOFF, status); } + + } \ No newline at end of file diff --git a/flume-ng-sources/flume-kafka-source/src/test/java/org/apache/flume/source/kafka/KafkaSourceUtilTest.java b/flume-ng-sources/flume-kafka-source/src/test/java/org/apache/flume/source/kafka/KafkaSourceUtilTest.java index b9a1b25e27..f87e5aea9e 100644 --- a/flume-ng-sources/flume-kafka-source/src/test/java/org/apache/flume/source/kafka/KafkaSourceUtilTest.java +++ b/flume-ng-sources/flume-kafka-source/src/test/java/org/apache/flume/source/kafka/KafkaSourceUtilTest.java @@ -39,10 +39,12 @@ public class KafkaSourceUtilTest { @Before public void setUp() throws Exception { - context.put("consumer.timeout", "10"); + context.put("kafka.consumer.timeout", "10"); context.put("type", "KafkaSource"); context.put("topic", "test"); - props = KafkaSourceUtil.getKafkaConfigProperties(context); + context.put("zookeeperConnect", "127.0.0.1:"+zkPort); + context.put("groupId","test"); + props = KafkaSourceUtil.getKafkaProperties(context); zookeeper = new KafkaSourceEmbeddedZookeeper(zkPort); @@ -53,23 +55,38 @@ public void tearDown() throws Exception { zookeeper.stopZookeeper(); } - @Test - public void testGetKafkaConfigParameter() { - assertEquals("10",props.getProperty("consumer.timeout")); - assertEquals("test",props.getProperty("topic")); - assertNull(props.getProperty("type")); - } - @Test public void testGetConsumer() { - context.put("zookeeper.connect", "127.0.0.1:"+zkPort); - context.put("group.id","test"); - - ConsumerConnector cc = KafkaSourceUtil.getConsumer(context); + ConsumerConnector cc = KafkaSourceUtil.getConsumer(props); assertNotNull(cc); } + @Test + public void testKafkaConsumerProperties() { + Context context = new Context(); + context.put("kafka.auto.commit.enabled", "override.default.autocommit"); + context.put("kafka.fake.property", "kafka.property.value"); + context.put("kafka.zookeeper.connect","bad-zookeeper-list"); + context.put("zookeeperConnect","real-zookeeper-list"); + Properties kafkaProps = KafkaSourceUtil.getKafkaProperties(context); + + //check that we have defaults set + assertEquals( + kafkaProps.getProperty(KafkaSourceConstants.GROUP_ID), + KafkaSourceConstants.DEFAULT_GROUP_ID); + //check that kafka properties override the default and get correct name + assertEquals( + kafkaProps.getProperty(KafkaSourceConstants.AUTO_COMMIT_ENABLED), + "override.default.autocommit"); + //check that any kafka property gets in + assertEquals(kafkaProps.getProperty("fake.property"), + "kafka.property.value"); + //check that documented property overrides defaults + assertEquals(kafkaProps.getProperty("zookeeper.connect") + ,"real-zookeeper-list"); + } + } From 26444fd7a8c804b6f2507ee3bf54e1c811ee5168 Mon Sep 17 00:00:00 2001 From: Hari Shreedharan Date: Tue, 23 Sep 2014 23:17:40 -0700 Subject: [PATCH 141/341] FLUME-2470. Kafka Sink and Source config updates. Missed a couple files in the last commit, adding them. (Gwen Shapira via Hari) --- .../flume/sink/kafka/KafkaSinkUtil.java | 103 ++++++++++++++++++ .../flume/sink/kafka/KafkaSinkUtilTest.java | 55 ++++++++++ 2 files changed, 158 insertions(+) create mode 100644 flume-ng-sinks/flume-ng-kafka-sink/src/main/java/org/apache/flume/sink/kafka/KafkaSinkUtil.java create mode 100644 flume-ng-sinks/flume-ng-kafka-sink/src/test/java/org/apache/flume/sink/kafka/KafkaSinkUtilTest.java diff --git a/flume-ng-sinks/flume-ng-kafka-sink/src/main/java/org/apache/flume/sink/kafka/KafkaSinkUtil.java b/flume-ng-sinks/flume-ng-kafka-sink/src/main/java/org/apache/flume/sink/kafka/KafkaSinkUtil.java new file mode 100644 index 0000000000..66bde85bb0 --- /dev/null +++ b/flume-ng-sinks/flume-ng-kafka-sink/src/main/java/org/apache/flume/sink/kafka/KafkaSinkUtil.java @@ -0,0 +1,103 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package org.apache.flume.sink.kafka; + +import org.apache.flume.Context; +import org.apache.flume.conf.ConfigurationException; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; +import scala.util.PropertiesTrait; + +import java.util.Map; +import java.util.Properties; + +public class KafkaSinkUtil { + + private static final Logger log = + LoggerFactory.getLogger(KafkaSinkUtil.class); + + public static Properties getKafkaProperties(Context context) { + log.info("context={}",context.toString()); + Properties props = generateDefaultKafkaProps(); + setKafkaProps(context, props); + addDocumentedKafkaProps(context, props); + return props; + } + + /** + * Some of the producer properties are especially important + * We documented them and gave them a camel-case name to match Flume config + * If user set these, we will override any existing parameters with these + * settings. + * Knowledge of which properties are documented is maintained here for now. + * If this will become a maintenance issue we'll set a proper data structure. + */ + private static void addDocumentedKafkaProps(Context context, + Properties kafkaProps) + throws ConfigurationException { + String brokerList = context.getString(KafkaSinkConstants + .BROKER_LIST_FLUME_KEY); + if (brokerList == null) { + throw new ConfigurationException("brokerList must contain at least " + + "one Kafka broker"); + } + kafkaProps.put(KafkaSinkConstants.BROKER_LIST_KEY, brokerList); + + String requiredKey = context.getString( + KafkaSinkConstants.REQUIRED_ACKS_FLUME_KEY); + + if (requiredKey != null ) { + kafkaProps.put(KafkaSinkConstants.REQUIRED_ACKS_KEY, requiredKey); + } + } + + + /** + * Generate producer properties object with some defaults + * @return + */ + private static Properties generateDefaultKafkaProps() { + Properties props = new Properties(); + props.put(KafkaSinkConstants.MESSAGE_SERIALIZER_KEY, + KafkaSinkConstants.DEFAULT_MESSAGE_SERIALIZER); + props.put(KafkaSinkConstants.KEY_SERIALIZER_KEY, + KafkaSinkConstants.DEFAULT_KEY_SERIALIZER); + props.put(KafkaSinkConstants.REQUIRED_ACKS_KEY, + KafkaSinkConstants.DEFAULT_REQUIRED_ACKS); + return props; + } + + + /** + * Add all configuration parameters starting with "kafka" + * to producer properties + */ + private static void setKafkaProps(Context context, Properties kafkaProps) { + + Map kafkaProperties = + context.getSubProperties(KafkaSinkConstants.PROPERTY_PREFIX); + + for (Map.Entry prop : kafkaProperties.entrySet()) { + + kafkaProps.put(prop.getKey(), prop.getValue()); + if (log.isDebugEnabled()) { + log.debug("Reading a Kafka Producer Property: key: " + + prop.getKey() + ", value: " + prop.getValue()); + } + } + } +} diff --git a/flume-ng-sinks/flume-ng-kafka-sink/src/test/java/org/apache/flume/sink/kafka/KafkaSinkUtilTest.java b/flume-ng-sinks/flume-ng-kafka-sink/src/test/java/org/apache/flume/sink/kafka/KafkaSinkUtilTest.java new file mode 100644 index 0000000000..84d213ccf0 --- /dev/null +++ b/flume-ng-sinks/flume-ng-kafka-sink/src/test/java/org/apache/flume/sink/kafka/KafkaSinkUtilTest.java @@ -0,0 +1,55 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.flume.sink.kafka; + +import junit.framework.TestCase; +import org.apache.flume.Context; +import org.apache.flume.conf.Configurables; +import org.junit.Test; + +import java.util.Properties; + +import static org.junit.Assert.assertEquals; + +public class KafkaSinkUtilTest extends TestCase { + + @Test + public void testGetKafkaProperties() { + Context context = new Context(); + context.put("kafka.serializer.class", "override.default.serializer"); + context.put("kafka.fake.property", "kafka.property.value"); + context.put("kafka.metadata.broker.list","bad-broker-list"); + context.put("brokerList","real-broker-list"); + Properties kafkaProps = KafkaSinkUtil.getKafkaProperties(context); + + //check that we have defaults set + assertEquals( + kafkaProps.getProperty(KafkaSinkConstants.KEY_SERIALIZER_KEY), + KafkaSinkConstants.DEFAULT_KEY_SERIALIZER); + //check that kafka properties override the default and get correct name + assertEquals( + kafkaProps.getProperty(KafkaSinkConstants.MESSAGE_SERIALIZER_KEY), + "override.default.serializer"); + //check that any kafka property gets in + assertEquals(kafkaProps.getProperty("fake.property"), + "kafka.property.value"); + //check that documented property overrides defaults + assertEquals(kafkaProps.getProperty("metadata.broker.list") + ,"real-broker-list"); + } +} \ No newline at end of file From ef2b089ab70e04e67b70c23416ef7051e992122b Mon Sep 17 00:00:00 2001 From: Hari Shreedharan Date: Wed, 24 Sep 2014 23:58:50 -0700 Subject: [PATCH 142/341] FLUME-2469. DatasetSink should load dataset when needed, not at startup. (Ryan Blue via Hari) --- .../apache/flume/sink/kite/DatasetSink.java | 93 ++++++++++++------- .../flume/sink/kite/TestDatasetSink.java | 89 ++++++++++++++++++ 2 files changed, 147 insertions(+), 35 deletions(-) diff --git a/flume-ng-sinks/flume-dataset-sink/src/main/java/org/apache/flume/sink/kite/DatasetSink.java b/flume-ng-sinks/flume-dataset-sink/src/main/java/org/apache/flume/sink/kite/DatasetSink.java index 4cd3027d03..a05d776472 100644 --- a/flume-ng-sinks/flume-dataset-sink/src/main/java/org/apache/flume/sink/kite/DatasetSink.java +++ b/flume-ng-sinks/flume-dataset-sink/src/main/java/org/apache/flume/sink/kite/DatasetSink.java @@ -52,8 +52,12 @@ import org.apache.hadoop.fs.Path; import org.apache.hadoop.security.UserGroupInformation; import org.kitesdk.data.Dataset; +import org.kitesdk.data.DatasetDescriptor; +import org.kitesdk.data.DatasetException; import org.kitesdk.data.DatasetWriter; import org.kitesdk.data.Datasets; +import org.kitesdk.data.View; +import org.kitesdk.data.spi.Registration; import org.kitesdk.data.spi.URIBuilder; import org.slf4j.Logger; import org.slf4j.LoggerFactory; @@ -71,12 +75,11 @@ public class DatasetSink extends AbstractSink implements Configurable { static Configuration conf = new Configuration(); - private String datasetURI = null; - private String repositoryURI = null; private String datasetName = null; private long batchSize = DatasetSinkConstants.DEFAULT_BATCH_SIZE; - private Dataset targetDataset = null; + private URI target = null; + private Schema targetSchema = null; private DatasetWriter writer = null; private UserGroupInformation login = null; private SinkCounter counter = null; @@ -98,7 +101,7 @@ public DatumReader load(Schema schema) { // must use the target dataset's schema for reading to ensure the // records are able to be stored using it return new GenericDatumReader( - schema, targetDataset.getDescriptor().getSchema()); + schema, targetSchema); } }); private static LoadingCache schemasFromLiteral = CacheBuilder @@ -150,39 +153,23 @@ public void configure(Context context) { this.login = KerberosUtil.proxyAs(effectiveUser, login); } - this.datasetURI = context.getString( - DatasetSinkConstants.CONFIG_KITE_DATASET_URI); + String datasetURI = context.getString( + DatasetSinkConstants.CONFIG_KITE_DATASET_URI); if (datasetURI != null) { - this.targetDataset = KerberosUtil.runPrivileged(login, - new PrivilegedExceptionAction>() { - @Override - public Dataset run() { - return Datasets.load(datasetURI); - } - }); + this.target = URI.create(datasetURI); + this.datasetName = uriToName(target); } else { - this.repositoryURI = context.getString( - DatasetSinkConstants.CONFIG_KITE_REPO_URI); + String repositoryURI = context.getString( + DatasetSinkConstants.CONFIG_KITE_REPO_URI); Preconditions.checkNotNull(repositoryURI, "Repository URI is missing"); this.datasetName = context.getString( - DatasetSinkConstants.CONFIG_KITE_DATASET_NAME); + DatasetSinkConstants.CONFIG_KITE_DATASET_NAME); Preconditions.checkNotNull(datasetName, "Dataset name is missing"); - this.targetDataset = KerberosUtil.runPrivileged(login, - new PrivilegedExceptionAction>() { - @Override - public Dataset run() { - return Datasets.load( - new URIBuilder(repositoryURI, datasetName).build()); - } - }); + this.target = new URIBuilder(repositoryURI, datasetName).build(); } - String formatName = targetDataset.getDescriptor().getFormat().getName(); - Preconditions.checkArgument(allowedFormats().contains(formatName), - "Unsupported format: " + formatName); - - this.reuseDatum = !("parquet".equals(formatName)); + this.setName(target.toString()); // other configuration this.batchSize = context.getLong( @@ -192,12 +179,11 @@ public Dataset run() { DatasetSinkConstants.CONFIG_KITE_ROLL_INTERVAL, DatasetSinkConstants.DEFAULT_ROLL_INTERVAL); - this.counter = new SinkCounter(getName()); + this.counter = new SinkCounter(datasetName); } @Override public synchronized void start() { - this.writer = targetDataset.newWriter(); this.lastRolledMs = System.currentTimeMillis(); counter.start(); // signal that this sink is ready to process @@ -232,17 +218,22 @@ public synchronized void stop() { @Override public Status process() throws EventDeliveryException { if (writer == null) { - throw new EventDeliveryException( - "Cannot recover after previous failure"); + try { + this.writer = newWriter(login, target); + } catch (DatasetException e) { + // DatasetException includes DatasetNotFoundException + throw new EventDeliveryException( + "Cannot write to " + getName(), e); + } } // handle file rolling if ((System.currentTimeMillis() - lastRolledMs) / 1000 > rollIntervalS) { // close the current writer and get a new one writer.close(); - this.writer = targetDataset.newWriter(); + this.writer = newWriter(login, target); this.lastRolledMs = System.currentTimeMillis(); - LOG.info("Rolled writer for dataset: " + datasetName); + LOG.info("Rolled writer for " + getName()); } Channel channel = getChannel(); @@ -316,6 +307,34 @@ public Status process() throws EventDeliveryException { } } + private DatasetWriter newWriter( + final UserGroupInformation login, final URI uri) { + View view = KerberosUtil.runPrivileged(login, + new PrivilegedExceptionAction>() { + @Override + public Dataset run() { + return Datasets.load(uri); + } + }); + + DatasetDescriptor descriptor = view.getDataset().getDescriptor(); + String formatName = descriptor.getFormat().getName(); + Preconditions.checkArgument(allowedFormats().contains(formatName), + "Unsupported format: " + formatName); + + Schema newSchema = descriptor.getSchema(); + if (targetSchema == null || !newSchema.equals(targetSchema)) { + this.targetSchema = descriptor.getSchema(); + // target dataset schema has changed, invalidate all readers based on it + readers.invalidateAll(); + } + + this.reuseDatum = !("parquet".equals(formatName)); + this.datasetName = view.getDataset().getName(); + + return view.newWriter(); + } + /** * Not thread-safe. * @@ -351,4 +370,8 @@ private static Schema schema(Event event) throws EventDeliveryException { } } + private static String uriToName(URI uri) { + return Registration.lookupDatasetUri(URI.create( + uri.getRawSchemeSpecificPart())).second().get("dataset"); + } } diff --git a/flume-ng-sinks/flume-dataset-sink/src/test/java/org/apache/flume/sink/kite/TestDatasetSink.java b/flume-ng-sinks/flume-dataset-sink/src/test/java/org/apache/flume/sink/kite/TestDatasetSink.java index a277381fc8..c46d66cdf0 100644 --- a/flume-ng-sinks/flume-dataset-sink/src/test/java/org/apache/flume/sink/kite/TestDatasetSink.java +++ b/flume-ng-sinks/flume-dataset-sink/src/test/java/org/apache/flume/sink/kite/TestDatasetSink.java @@ -33,6 +33,7 @@ import java.util.HashSet; import java.util.List; import java.util.Map; +import java.util.Set; import java.util.concurrent.Callable; import javax.annotation.Nullable; import org.apache.avro.Schema; @@ -85,6 +86,12 @@ public class TestDatasetSink { public static final Schema INCOMPATIBLE_SCHEMA = new Schema.Parser().parse( "{\"type\":\"record\",\"name\":\"user\",\"fields\":[" + "{\"name\":\"username\",\"type\":\"string\"}]}"); + public static final Schema UPDATED_SCHEMA = new Schema.Parser().parse( + "{\"type\":\"record\",\"name\":\"rec\",\"fields\":[" + + "{\"name\":\"id\",\"type\":\"string\"}," + + "{\"name\":\"priority\",\"type\":\"int\", \"default\": 0}," + + "{\"name\":\"msg\",\"type\":[\"string\",\"null\"]," + + "\"default\":\"default\"}]}"); public static final DatasetDescriptor DESCRIPTOR = new DatasetDescriptor .Builder() .schema(RECORD_SCHEMA) @@ -251,6 +258,88 @@ public void testPartitionedData() throws EventDeliveryException { } } + @Test + public void testStartBeforeDatasetCreated() throws EventDeliveryException { + // delete the dataset created by setup + Datasets.delete(FILE_DATASET_URI); + + DatasetSink sink = sink(in, config); + + // start the sink + sink.start(); + + // run the sink without a target dataset + try { + sink.process(); + Assert.fail("Should have thrown an exception: no such dataset"); + } catch (EventDeliveryException e) { + // expected + } + + // create the target dataset + Datasets.create(FILE_DATASET_URI, DESCRIPTOR); + + // run the sink + sink.process(); + sink.stop(); + + Assert.assertEquals( + Sets.newHashSet(expected), + read(Datasets.load(FILE_DATASET_URI))); + Assert.assertEquals("Should have committed", 0, remaining(in)); + } + + @Test + public void testDatasetUpdate() throws EventDeliveryException { + // add an updated record that is missing the msg field + GenericRecordBuilder updatedBuilder = new GenericRecordBuilder( + UPDATED_SCHEMA); + GenericData.Record updatedRecord = updatedBuilder + .set("id", "0") + .set("priority", 1) + .set("msg", "Priority 1 message!") + .build(); + + // make a set of the expected records with the new schema + Set expectedAsUpdated = Sets.newHashSet(); + for (GenericRecord record : expected) { + expectedAsUpdated.add(updatedBuilder + .clear("priority") + .set("id", record.get("id")) + .set("msg", record.get("msg")) + .build()); + } + expectedAsUpdated.add(updatedRecord); + + DatasetSink sink = sink(in, config); + + // run the sink + sink.start(); + sink.process(); + + // update the dataset's schema + DatasetDescriptor updated = new DatasetDescriptor + .Builder(Datasets.load(FILE_DATASET_URI).getDataset().getDescriptor()) + .schema(UPDATED_SCHEMA) + .build(); + Datasets.update(FILE_DATASET_URI, updated); + + // trigger a roll on the next process call to refresh the writer + sink.roll(); + + // add the record to the incoming channel and the expected list + putToChannel(in, event(updatedRecord, UPDATED_SCHEMA, null, false)); + + // process events with the updated schema + sink.process(); + sink.stop(); + + Assert.assertEquals( + expectedAsUpdated, + read(Datasets.load(FILE_DATASET_URI))); + Assert.assertEquals("Should have committed", 0, remaining(in)); + } + @Test public void testMiniClusterStore() throws EventDeliveryException, IOException { From 6d0243112fa1ff1cb796ebe158630ae681a2afc8 Mon Sep 17 00:00:00 2001 From: Hari Shreedharan Date: Fri, 26 Sep 2014 11:45:54 -0700 Subject: [PATCH 143/341] FLUME-2463. Add Hive and HBase dataset support in DatasetSink. (Ryan Blue via Hari) --- flume-ng-sinks/flume-dataset-sink/pom.xml | 20 ++++++++++++++++++++ pom.xml | 23 +++++++++++++++++++++++ 2 files changed, 43 insertions(+) diff --git a/flume-ng-sinks/flume-dataset-sink/pom.xml b/flume-ng-sinks/flume-dataset-sink/pom.xml index f6ae1f32f6..0b447a5330 100644 --- a/flume-ng-sinks/flume-dataset-sink/pom.xml +++ b/flume-ng-sinks/flume-dataset-sink/pom.xml @@ -84,11 +84,31 @@ limitations under the License. kite-data-core + + org.kitesdk + kite-data-hcatalog + + + + org.kitesdk + kite-data-hbase + + org.apache.avro avro + + org.apache.hive + hive-exec + + + + org.apache.hive + hive-metastore + + + + org.apache.hive + hive-exec + ${hive.version} + + + org.apache.hive + hive-metastore + ${hive.version} + org.xerial.snappy From d2cb4ba7524b3dacbce9f19a524a56c786eb3e48 Mon Sep 17 00:00:00 2001 From: Hari Shreedharan Date: Tue, 30 Sep 2014 12:21:55 -0700 Subject: [PATCH 144/341] FLUME-1501. Flume Scribe Source needs unit tests. (Ashish Paliwal via Hari) --- .../flume/source/scribe/TestScribeSource.java | 140 ++++++++++++++++++ 1 file changed, 140 insertions(+) create mode 100644 flume-ng-sources/flume-scribe-source/src/test/java/org/apache/flume/source/scribe/TestScribeSource.java diff --git a/flume-ng-sources/flume-scribe-source/src/test/java/org/apache/flume/source/scribe/TestScribeSource.java b/flume-ng-sources/flume-scribe-source/src/test/java/org/apache/flume/source/scribe/TestScribeSource.java new file mode 100644 index 0000000000..9059ebae77 --- /dev/null +++ b/flume-ng-sources/flume-scribe-source/src/test/java/org/apache/flume/source/scribe/TestScribeSource.java @@ -0,0 +1,140 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.flume.source.scribe; + +import junit.framework.Assert; +import org.apache.flume.*; +import org.apache.flume.channel.ChannelProcessor; +import org.apache.flume.channel.MemoryChannel; +import org.apache.flume.channel.ReplicatingChannelSelector; +import org.apache.flume.conf.Configurables; +import org.apache.thrift.protocol.TBinaryProtocol; +import org.apache.thrift.protocol.TProtocol; +import org.apache.thrift.transport.TFramedTransport; +import org.apache.thrift.transport.TSocket; +import org.apache.thrift.transport.TTransport; +import org.junit.AfterClass; +import org.junit.BeforeClass; +import org.junit.Test; + +import java.io.IOException; +import java.net.ServerSocket; +import java.util.ArrayList; +import java.util.List; + +/** + * + */ +public class TestScribeSource { + + private static int port; + private static Channel memoryChannel; + private static ScribeSource scribeSource; + + private static int findFreePort() throws IOException { + ServerSocket socket = new ServerSocket(0); + int port = socket.getLocalPort(); + socket.close(); + return port; + } + + @BeforeClass + public static void setUpClass() throws Exception { + port = findFreePort(); + Context context = new Context(); + context.put("port", String.valueOf(port)); + + scribeSource = new ScribeSource(); + scribeSource.setName("Scribe Source"); + + Configurables.configure(scribeSource, context); + + memoryChannel = new MemoryChannel(); + Configurables.configure(memoryChannel, context); + + List channels = new ArrayList(1); + channels.add(memoryChannel); + + ChannelSelector rcs = new ReplicatingChannelSelector(); + rcs.setChannels(channels); + + memoryChannel.start(); + + scribeSource.setChannelProcessor(new ChannelProcessor(rcs)); + scribeSource.start(); + } + + @Test + public void testScribeMessage() throws Exception { + TTransport transport = new TFramedTransport(new TSocket("localhost", port)); + + TProtocol protocol = new TBinaryProtocol(transport); + Scribe.Client client = new Scribe.Client(protocol); + transport.open(); + LogEntry logEntry = new LogEntry("INFO", "Sending info msg to scribe source"); + List logEntries = new ArrayList(1); + logEntries.add(logEntry); + client.Log(logEntries); + + // try to get it from Channels + Transaction tx = memoryChannel.getTransaction(); + tx.begin(); + Event e = memoryChannel.take(); + Assert.assertNotNull(e); + Assert.assertEquals("Sending info msg to scribe source", new String(e.getBody())); + tx.commit(); + tx.close(); + } + + @Test + public void testScribeMultipleMessages() throws Exception { + TTransport transport = new TFramedTransport(new TSocket("localhost", port)); + + TProtocol protocol = new TBinaryProtocol(transport); + Scribe.Client client = new Scribe.Client(protocol); + transport.open(); + + List logEntries = new ArrayList(10); + for (int i = 0; i < 10; i++) { + LogEntry logEntry = new LogEntry("INFO", String.format("Sending info msg# %d to scribe source", i)); + logEntries.add(logEntry); + } + + client.Log(logEntries); + + // try to get it from Channels + Transaction tx = memoryChannel.getTransaction(); + tx.begin(); + + for (int i = 0; i < 10; i++) { + Event e = memoryChannel.take(); + Assert.assertNotNull(e); + Assert.assertEquals(String.format("Sending info msg# %d to scribe source", i), new String(e.getBody())); + } + tx.commit(); + tx.close(); + } + + @AfterClass + public static void cleanup() { + memoryChannel.stop(); + scribeSource.stop(); + } + +} \ No newline at end of file From e1e5acea2fbb04699eba76ea930def38e34f328d Mon Sep 17 00:00:00 2001 From: Hari Shreedharan Date: Wed, 1 Oct 2014 22:23:15 -0700 Subject: [PATCH 145/341] FLUME-2481. TestFileChannelEncryption fails with System locale other than en_US (Santiago M. Mola via Hari) --- .../file/encryption/TestFileChannelEncryption.java | 8 ++++---- 1 file changed, 4 insertions(+), 4 deletions(-) diff --git a/flume-ng-channels/flume-file-channel/src/test/java/org/apache/flume/channel/file/encryption/TestFileChannelEncryption.java b/flume-ng-channels/flume-file-channel/src/test/java/org/apache/flume/channel/file/encryption/TestFileChannelEncryption.java index 6ea12168b0..d4537a8ac9 100644 --- a/flume-ng-channels/flume-file-channel/src/test/java/org/apache/flume/channel/file/encryption/TestFileChannelEncryption.java +++ b/flume-ng-channels/flume-file-channel/src/test/java/org/apache/flume/channel/file/encryption/TestFileChannelEncryption.java @@ -307,8 +307,8 @@ public void testMissingKeyStoreFile() throws Exception { channel = createFileChannel(overrides); Assert.fail(); } catch(RuntimeException ex) { - Assert.assertEquals("java.io.FileNotFoundException: /path/does/not/exist " + - "(No such file or directory)", ex.getMessage()); + Assert.assertTrue("Exception message is incorrect: " + ex.getMessage(), + ex.getMessage().startsWith("java.io.FileNotFoundException: /path/does/not/exist ")); } } @Test @@ -322,8 +322,8 @@ public void testMissingKeyStorePasswordFile() throws Exception { channel = createFileChannel(overrides); Assert.fail(); } catch(RuntimeException ex) { - Assert.assertEquals("java.io.FileNotFoundException: /path/does/not/exist " + - "(No such file or directory)", ex.getMessage()); + Assert.assertTrue("Exception message is incorrect: " + ex.getMessage(), + ex.getMessage().startsWith("java.io.FileNotFoundException: /path/does/not/exist ")); } } @Test From 9f75c40a69fa4404ea2c344fa29285a00eb082fa Mon Sep 17 00:00:00 2001 From: Hari Shreedharan Date: Fri, 3 Oct 2014 11:24:24 -0700 Subject: [PATCH 146/341] FLUME-2492. Flume's Kafka Source doesn't account time correctly (Gwen Shapira via Hari) --- .../flume/source/kafka/KafkaSource.java | 46 ++++++------------- .../flume/source/kafka/KafkaSourceTest.java | 24 ++++++++++ 2 files changed, 37 insertions(+), 33 deletions(-) diff --git a/flume-ng-sources/flume-kafka-source/src/main/java/org/apache/flume/source/kafka/KafkaSource.java b/flume-ng-sources/flume-kafka-source/src/main/java/org/apache/flume/source/kafka/KafkaSource.java index 231ae4237b..8cdc9670fd 100644 --- a/flume-ng-sources/flume-kafka-source/src/main/java/org/apache/flume/source/kafka/KafkaSource.java +++ b/flume-ng-sources/flume-kafka-source/src/main/java/org/apache/flume/source/kafka/KafkaSource.java @@ -75,13 +75,16 @@ public Status process() throws EventDeliveryException { byte[] bytes; Event event; Map headers; + long batchStartTime = System.currentTimeMillis(); + long batchEndTime = System.currentTimeMillis() + timeUpperLimit; try { int eventCounter = 0; int timeWaited = 0; - IterStatus iterStatus = new IterStatus(false, -1); - while (eventCounter < batchUpperLimit && timeWaited < timeUpperLimit) { - iterStatus = timedHasNext(); - if (iterStatus.hasData()) { + boolean iterStatus = false; + while (eventCounter < batchUpperLimit && + System.currentTimeMillis() < batchEndTime) { + iterStatus = hasNext(); + if (iterStatus) { // get next message bytes = it.next().message(); @@ -96,9 +99,8 @@ public Status process() throws EventDeliveryException { eventList.add(event); eventCounter++; } - timeWaited += iterStatus.getWaitTime(); if (log.isDebugEnabled()) { - log.debug("Waited: {} ", timeWaited); + log.debug("Waited: {} ", System.currentTimeMillis() - batchStartTime); log.debug("Event #: {}", eventCounter); } } @@ -111,7 +113,7 @@ public Status process() throws EventDeliveryException { consumer.commitOffsets(); } } - if (!iterStatus.hasData()) { + if (!iterStatus) { if (log.isDebugEnabled()) { log.debug("Returning with backoff. No more data to read"); } @@ -209,37 +211,15 @@ public synchronized void stop() { /** * Check if there are messages waiting in Kafka, * waiting until timeout (10ms by default) for messages to arrive. - * And timing our wait. - * @return IterStatus object. - * Indicating whether a message was found and how long we waited for it + * and catching the timeout exception to return a boolean */ - IterStatus timedHasNext() { + boolean hasNext() { try { - long startTime = System.currentTimeMillis(); it.hasNext(); - long endTime = System.currentTimeMillis(); - return new IterStatus(true, endTime - startTime); + return true; } catch (ConsumerTimeoutException e) { - return new IterStatus(false, consumerTimeout); + return false; } } - private class IterStatus { - private long waitTime; - private boolean hasData; - - - private IterStatus(boolean hasData,long waitTime) { - this.waitTime = waitTime; - this.hasData = hasData; - } - - public long getWaitTime() { - return waitTime; - } - - public boolean hasData() { - return hasData; - } - } } \ No newline at end of file diff --git a/flume-ng-sources/flume-kafka-source/src/test/java/org/apache/flume/source/kafka/KafkaSourceTest.java b/flume-ng-sources/flume-kafka-source/src/test/java/org/apache/flume/source/kafka/KafkaSourceTest.java index d067e24d81..7684616ecb 100644 --- a/flume-ng-sources/flume-kafka-source/src/test/java/org/apache/flume/source/kafka/KafkaSourceTest.java +++ b/flume-ng-sources/flume-kafka-source/src/test/java/org/apache/flume/source/kafka/KafkaSourceTest.java @@ -18,6 +18,7 @@ package org.apache.flume.source.kafka; import static org.junit.Assert.assertEquals; +import static org.junit.Assert.assertTrue; import static org.mockito.Matchers.any; import static org.mockito.Matchers.anyList; import static org.mockito.Mockito.*; @@ -194,6 +195,29 @@ public void testNonExistingZk() throws EventDeliveryException, assertEquals(Status.BACKOFF, status); } + @Test + public void testBatchTime() throws InterruptedException, + EventDeliveryException { + context.put(KafkaSourceConstants.BATCH_DURATION_MS,"250"); + kafkaSource.configure(context); + kafkaSource.start(); + + Thread.sleep(500L); + + for (int i=1; i<5000; i++) { + kafkaServer.produce(topicName, "", "hello, world " + i); + } + Thread.sleep(500L); + + long startTime = System.currentTimeMillis(); + Status status = kafkaSource.process(); + long endTime = System.currentTimeMillis(); + assertEquals(Status.READY, status); + assertTrue(endTime - startTime < + ( context.getLong(KafkaSourceConstants.BATCH_DURATION_MS) + + context.getLong("kafka.consumer.timeout.ms")) ); + } + } \ No newline at end of file From e1cb6913f358e4de54d69d1f6f21edecf06a9da2 Mon Sep 17 00:00:00 2001 From: Hari Shreedharan Date: Fri, 3 Oct 2014 11:52:13 -0700 Subject: [PATCH 147/341] FLUME-2479. Kafka property auto.commit.enable is incorrect for KafkaSource. (Gwen Shapira via Hari) --- .../source/kafka/KafkaSourceConstants.java | 4 +- .../flume/source/kafka/KafkaSourceTest.java | 223 ------------------ .../source/kafka/KafkaSourceUtilTest.java | 92 -------- 3 files changed, 2 insertions(+), 317 deletions(-) delete mode 100644 flume-ng-sources/flume-kafka-source/src/test/java/org/apache/flume/source/kafka/KafkaSourceTest.java delete mode 100644 flume-ng-sources/flume-kafka-source/src/test/java/org/apache/flume/source/kafka/KafkaSourceUtilTest.java diff --git a/flume-ng-sources/flume-kafka-source/src/main/java/org/apache/flume/source/kafka/KafkaSourceConstants.java b/flume-ng-sources/flume-kafka-source/src/main/java/org/apache/flume/source/kafka/KafkaSourceConstants.java index 169cc1001e..7390618189 100644 --- a/flume-ng-sources/flume-kafka-source/src/main/java/org/apache/flume/source/kafka/KafkaSourceConstants.java +++ b/flume-ng-sources/flume-kafka-source/src/main/java/org/apache/flume/source/kafka/KafkaSourceConstants.java @@ -22,7 +22,7 @@ public class KafkaSourceConstants { public static final String BATCH_SIZE = "batchSize"; public static final String BATCH_DURATION_MS = "batchDurationMillis"; public static final String CONSUMER_TIMEOUT = "consumer.timeout.ms"; - public static final String AUTO_COMMIT_ENABLED = "auto.commit.enabled"; + public static final String AUTO_COMMIT_ENABLED = "auto.commit.enable"; public static final String ZOOKEEPER_CONNECT = "zookeeper.connect"; public static final String ZOOKEEPER_CONNECT_FLUME = "zookeeperConnect"; public static final String GROUP_ID = "group.id"; @@ -33,7 +33,7 @@ public class KafkaSourceConstants { public static final int DEFAULT_BATCH_SIZE = 1000; public static final int DEFAULT_BATCH_DURATION = 1000; public static final String DEFAULT_CONSUMER_TIMEOUT = "10"; - public static final boolean DEFAULT_AUTO_COMMIT = false; + public static final String DEFAULT_AUTO_COMMIT = "false"; public static final String DEFAULT_GROUP_ID = "flume"; } diff --git a/flume-ng-sources/flume-kafka-source/src/test/java/org/apache/flume/source/kafka/KafkaSourceTest.java b/flume-ng-sources/flume-kafka-source/src/test/java/org/apache/flume/source/kafka/KafkaSourceTest.java deleted file mode 100644 index 7684616ecb..0000000000 --- a/flume-ng-sources/flume-kafka-source/src/test/java/org/apache/flume/source/kafka/KafkaSourceTest.java +++ /dev/null @@ -1,223 +0,0 @@ -/* - * Licensed to the Apache Software Foundation (ASF) under one or more - * contributor license agreements. See the NOTICE file distributed with - * this work for additional information regarding copyright ownership. - * The ASF licenses this file to You under the Apache License, Version 2.0 - * (the "License"); you may not use this file except in compliance with - * the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. - */ - -package org.apache.flume.source.kafka; - -import static org.junit.Assert.assertEquals; -import static org.junit.Assert.assertTrue; -import static org.mockito.Matchers.any; -import static org.mockito.Matchers.anyList; -import static org.mockito.Mockito.*; - -import java.lang.reflect.Field; -import java.nio.ByteBuffer; -import java.util.List; -import java.util.Properties; - -import com.google.common.base.Charsets; -import com.google.common.collect.Lists; -import junit.framework.Assert; -import kafka.common.TopicExistsException; -import kafka.consumer.ConsumerIterator; -import kafka.message.Message; - -import kafka.message.MessageAndMetadata; - -import org.apache.flume.*; -import org.apache.flume.PollableSource.Status; -import org.apache.flume.channel.ChannelProcessor; -import org.apache.flume.conf.Configurables; -import org.apache.flume.source.AbstractSource; -import org.junit.After; -import org.junit.Before; -import org.junit.Test; -import org.junit.rules.ExpectedException; -import org.mockito.invocation.InvocationOnMock; -import org.mockito.stubbing.Answer; -import org.slf4j.Logger; -import org.slf4j.LoggerFactory; - -public class KafkaSourceTest { - private static final Logger log = - LoggerFactory.getLogger(KafkaSourceTest.class); - - private KafkaSource kafkaSource; - private KafkaSourceEmbeddedKafka kafkaServer; - private ConsumerIterator mockIt; - private Message message; - private Context context; - private List events; - private String topicName = "test1"; - - - @SuppressWarnings("unchecked") - @Before - public void setup() throws Exception { - - kafkaSource = new KafkaSource(); - kafkaServer = new KafkaSourceEmbeddedKafka(); - try { - kafkaServer.createTopic(topicName); - } catch (TopicExistsException e) { - //do nothing - } - - context = new Context(); - context.put(KafkaSourceConstants.ZOOKEEPER_CONNECT_FLUME, - kafkaServer.getZkConnectString()); - context.put(KafkaSourceConstants.GROUP_ID_FLUME,"flume"); - context.put(KafkaSourceConstants.TOPIC,topicName); - context.put("kafka.consumer.timeout.ms","100"); - - ChannelProcessor channelProcessor = mock(ChannelProcessor.class); - - events = Lists.newArrayList(); - - doAnswer(new Answer() { - @Override - public Void answer(InvocationOnMock invocation) throws Throwable { - events.addAll((List)invocation.getArguments()[0]); - return null; - } - }).when(channelProcessor).processEventBatch(any(List.class)); - kafkaSource.setChannelProcessor(channelProcessor); - } - - @After - public void tearDown() throws Exception { - kafkaSource.stop(); - kafkaServer.stop(); - } - - @SuppressWarnings("unchecked") - @Test - public void testProcessItNotEmpty() throws EventDeliveryException, - SecurityException, NoSuchFieldException, IllegalArgumentException, - IllegalAccessException, InterruptedException { - context.put(KafkaSourceConstants.BATCH_SIZE,"1"); - kafkaSource.configure(context); - kafkaSource.start(); - - Thread.sleep(500L); - - kafkaServer.produce(topicName, "", "hello, world"); - - Thread.sleep(500L); - - Assert.assertEquals(Status.READY, kafkaSource.process()); - Assert.assertEquals(Status.BACKOFF, kafkaSource.process()); - Assert.assertEquals(1, events.size()); - - Assert.assertEquals("hello, world", new String(events.get(0).getBody(), - Charsets.UTF_8)); - - - } - - @SuppressWarnings("unchecked") - @Test - public void testProcessItNotEmptyBatch() throws EventDeliveryException, - SecurityException, NoSuchFieldException, IllegalArgumentException, - IllegalAccessException, InterruptedException { - context.put(KafkaSourceConstants.BATCH_SIZE,"2"); - kafkaSource.configure(context); - kafkaSource.start(); - - Thread.sleep(500L); - - kafkaServer.produce(topicName, "", "hello, world"); - kafkaServer.produce(topicName, "", "foo, bar"); - - Thread.sleep(500L); - - Status status = kafkaSource.process(); - assertEquals(Status.READY, status); - Assert.assertEquals("hello, world", new String(events.get(0).getBody(), - Charsets.UTF_8)); - Assert.assertEquals("foo, bar", new String(events.get(1).getBody(), - Charsets.UTF_8)); - - } - - - @SuppressWarnings("unchecked") - @Test - public void testProcessItEmpty() throws EventDeliveryException, - SecurityException, NoSuchFieldException, IllegalArgumentException, - IllegalAccessException, InterruptedException { - kafkaSource.configure(context); - kafkaSource.start(); - Thread.sleep(500L); - - Status status = kafkaSource.process(); - assertEquals(Status.BACKOFF, status); - } - - @SuppressWarnings("unchecked") - @Test - public void testNonExistingTopic() throws EventDeliveryException, - SecurityException, NoSuchFieldException, IllegalArgumentException, - IllegalAccessException, InterruptedException { - context.put(KafkaSourceConstants.TOPIC,"faketopic"); - kafkaSource.configure(context); - kafkaSource.start(); - Thread.sleep(500L); - - Status status = kafkaSource.process(); - assertEquals(Status.BACKOFF, status); - } - - @SuppressWarnings("unchecked") - @Test(expected= FlumeException.class) - public void testNonExistingZk() throws EventDeliveryException, - SecurityException, NoSuchFieldException, IllegalArgumentException, - IllegalAccessException, InterruptedException { - context.put(KafkaSourceConstants.ZOOKEEPER_CONNECT_FLUME,"blabla:666"); - kafkaSource.configure(context); - kafkaSource.start(); - Thread.sleep(500L); - - Status status = kafkaSource.process(); - assertEquals(Status.BACKOFF, status); - } - - @Test - public void testBatchTime() throws InterruptedException, - EventDeliveryException { - context.put(KafkaSourceConstants.BATCH_DURATION_MS,"250"); - kafkaSource.configure(context); - kafkaSource.start(); - - Thread.sleep(500L); - - for (int i=1; i<5000; i++) { - kafkaServer.produce(topicName, "", "hello, world " + i); - } - Thread.sleep(500L); - - long startTime = System.currentTimeMillis(); - Status status = kafkaSource.process(); - long endTime = System.currentTimeMillis(); - assertEquals(Status.READY, status); - assertTrue(endTime - startTime < - ( context.getLong(KafkaSourceConstants.BATCH_DURATION_MS) + - context.getLong("kafka.consumer.timeout.ms")) ); - } - - - -} \ No newline at end of file diff --git a/flume-ng-sources/flume-kafka-source/src/test/java/org/apache/flume/source/kafka/KafkaSourceUtilTest.java b/flume-ng-sources/flume-kafka-source/src/test/java/org/apache/flume/source/kafka/KafkaSourceUtilTest.java deleted file mode 100644 index f87e5aea9e..0000000000 --- a/flume-ng-sources/flume-kafka-source/src/test/java/org/apache/flume/source/kafka/KafkaSourceUtilTest.java +++ /dev/null @@ -1,92 +0,0 @@ -/* - * Licensed to the Apache Software Foundation (ASF) under one or more - * contributor license agreements. See the NOTICE file distributed with - * this work for additional information regarding copyright ownership. - * The ASF licenses this file to You under the Apache License, Version 2.0 - * (the "License"); you may not use this file except in compliance with - * the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. - */ - -package org.apache.flume.source.kafka; - -import static org.junit.Assert.assertEquals; -import static org.junit.Assert.assertNotNull; -import static org.junit.Assert.assertNull; - -import java.io.IOException; -import java.util.Properties; - -import kafka.javaapi.consumer.ConsumerConnector; -import org.apache.flume.Context; -import org.apache.zookeeper.server.*; -import org.junit.After; -import org.junit.Before; -import org.junit.Test; - -public class KafkaSourceUtilTest { - private Properties props = new Properties(); - private Context context = new Context(); - private int zkPort = 21818; // none-standard - private KafkaSourceEmbeddedZookeeper zookeeper; - - @Before - public void setUp() throws Exception { - context.put("kafka.consumer.timeout", "10"); - context.put("type", "KafkaSource"); - context.put("topic", "test"); - context.put("zookeeperConnect", "127.0.0.1:"+zkPort); - context.put("groupId","test"); - props = KafkaSourceUtil.getKafkaProperties(context); - zookeeper = new KafkaSourceEmbeddedZookeeper(zkPort); - - - } - - @After - public void tearDown() throws Exception { - zookeeper.stopZookeeper(); - } - - - @Test - public void testGetConsumer() { - ConsumerConnector cc = KafkaSourceUtil.getConsumer(props); - assertNotNull(cc); - - } - - @Test - public void testKafkaConsumerProperties() { - Context context = new Context(); - context.put("kafka.auto.commit.enabled", "override.default.autocommit"); - context.put("kafka.fake.property", "kafka.property.value"); - context.put("kafka.zookeeper.connect","bad-zookeeper-list"); - context.put("zookeeperConnect","real-zookeeper-list"); - Properties kafkaProps = KafkaSourceUtil.getKafkaProperties(context); - - //check that we have defaults set - assertEquals( - kafkaProps.getProperty(KafkaSourceConstants.GROUP_ID), - KafkaSourceConstants.DEFAULT_GROUP_ID); - //check that kafka properties override the default and get correct name - assertEquals( - kafkaProps.getProperty(KafkaSourceConstants.AUTO_COMMIT_ENABLED), - "override.default.autocommit"); - //check that any kafka property gets in - assertEquals(kafkaProps.getProperty("fake.property"), - "kafka.property.value"); - //check that documented property overrides defaults - assertEquals(kafkaProps.getProperty("zookeeper.connect") - ,"real-zookeeper-list"); - } - - -} From 9b2c28d805b89b8b3eac8f082e580e313f44c384 Mon Sep 17 00:00:00 2001 From: Hari Shreedharan Date: Fri, 3 Oct 2014 11:56:15 -0700 Subject: [PATCH 148/341] FLUME-2479. Adding new test files. (Gwen Shapira via Hari) --- .../flume/source/kafka/TestKafkaSource.java | 213 ++++++++++++++++++ .../source/kafka/TestKafkaSourceUtil.java | 92 ++++++++ 2 files changed, 305 insertions(+) create mode 100644 flume-ng-sources/flume-kafka-source/src/test/java/org/apache/flume/source/kafka/TestKafkaSource.java create mode 100644 flume-ng-sources/flume-kafka-source/src/test/java/org/apache/flume/source/kafka/TestKafkaSourceUtil.java diff --git a/flume-ng-sources/flume-kafka-source/src/test/java/org/apache/flume/source/kafka/TestKafkaSource.java b/flume-ng-sources/flume-kafka-source/src/test/java/org/apache/flume/source/kafka/TestKafkaSource.java new file mode 100644 index 0000000000..3695860e06 --- /dev/null +++ b/flume-ng-sources/flume-kafka-source/src/test/java/org/apache/flume/source/kafka/TestKafkaSource.java @@ -0,0 +1,213 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.flume.source.kafka; + +import static org.junit.Assert.assertEquals; +import static org.junit.Assert.assertTrue; +import static org.mockito.Matchers.any; +import static org.mockito.Mockito.*; + +import java.util.List; + +import com.google.common.base.Charsets; +import com.google.common.collect.Lists; +import junit.framework.Assert; +import kafka.common.TopicExistsException; +import kafka.consumer.ConsumerIterator; +import kafka.message.Message; + +import org.apache.flume.*; +import org.apache.flume.PollableSource.Status; +import org.apache.flume.channel.ChannelProcessor; +import org.junit.After; +import org.junit.Before; +import org.junit.Test; +import org.mockito.invocation.InvocationOnMock; +import org.mockito.stubbing.Answer; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +public class TestKafkaSource { + private static final Logger log = + LoggerFactory.getLogger(TestKafkaSource.class); + + private KafkaSource kafkaSource; + private KafkaSourceEmbeddedKafka kafkaServer; + private ConsumerIterator mockIt; + private Message message; + private Context context; + private List events; + private String topicName = "test1"; + + + @SuppressWarnings("unchecked") + @Before + public void setup() throws Exception { + + kafkaSource = new KafkaSource(); + kafkaServer = new KafkaSourceEmbeddedKafka(); + try { + kafkaServer.createTopic(topicName); + } catch (TopicExistsException e) { + //do nothing + } + + + context = new Context(); + context.put(KafkaSourceConstants.ZOOKEEPER_CONNECT_FLUME, + kafkaServer.getZkConnectString()); + context.put(KafkaSourceConstants.GROUP_ID_FLUME,"flume"); + context.put(KafkaSourceConstants.TOPIC,topicName); + context.put("kafka.consumer.timeout.ms","100"); + + + ChannelProcessor channelProcessor = mock(ChannelProcessor.class); + + events = Lists.newArrayList(); + + doAnswer(new Answer() { + @Override + public Void answer(InvocationOnMock invocation) throws Throwable { + events.addAll((List)invocation.getArguments()[0]); + return null; + } + }).when(channelProcessor).processEventBatch(any(List.class)); + kafkaSource.setChannelProcessor(channelProcessor); + } + + @After + public void tearDown() throws Exception { + kafkaSource.stop(); + kafkaServer.stop(); + } + + @SuppressWarnings("unchecked") + @Test + public void testProcessItNotEmpty() throws EventDeliveryException, + SecurityException, NoSuchFieldException, IllegalArgumentException, + IllegalAccessException, InterruptedException { + context.put(KafkaSourceConstants.BATCH_SIZE,"1"); + kafkaSource.configure(context); + kafkaSource.start(); + + Thread.sleep(500L); + + kafkaServer.produce(topicName, "", "hello, world"); + + Thread.sleep(500L); + + Assert.assertEquals(Status.READY, kafkaSource.process()); + Assert.assertEquals(Status.BACKOFF, kafkaSource.process()); + Assert.assertEquals(1, events.size()); + + Assert.assertEquals("hello, world", new String(events.get(0).getBody(), + Charsets.UTF_8)); + + + } + + @SuppressWarnings("unchecked") + @Test + public void testProcessItNotEmptyBatch() throws EventDeliveryException, + SecurityException, NoSuchFieldException, IllegalArgumentException, + IllegalAccessException, InterruptedException { + context.put(KafkaSourceConstants.BATCH_SIZE,"2"); + kafkaSource.configure(context); + kafkaSource.start(); + + Thread.sleep(500L); + + kafkaServer.produce(topicName, "", "hello, world"); + kafkaServer.produce(topicName, "", "foo, bar"); + + Thread.sleep(500L); + + Status status = kafkaSource.process(); + assertEquals(Status.READY, status); + Assert.assertEquals("hello, world", new String(events.get(0).getBody(), + Charsets.UTF_8)); + Assert.assertEquals("foo, bar", new String(events.get(1).getBody(), + Charsets.UTF_8)); + + } + + + @SuppressWarnings("unchecked") + @Test + public void testProcessItEmpty() throws EventDeliveryException, + SecurityException, NoSuchFieldException, IllegalArgumentException, + IllegalAccessException, InterruptedException { + kafkaSource.configure(context); + kafkaSource.start(); + Thread.sleep(500L); + + Status status = kafkaSource.process(); + assertEquals(Status.BACKOFF, status); + } + + @SuppressWarnings("unchecked") + @Test + public void testNonExistingTopic() throws EventDeliveryException, + SecurityException, NoSuchFieldException, IllegalArgumentException, + IllegalAccessException, InterruptedException { + context.put(KafkaSourceConstants.TOPIC,"faketopic"); + kafkaSource.configure(context); + kafkaSource.start(); + Thread.sleep(500L); + + Status status = kafkaSource.process(); + assertEquals(Status.BACKOFF, status); + } + + @SuppressWarnings("unchecked") + @Test(expected= FlumeException.class) + public void testNonExistingZk() throws EventDeliveryException, + SecurityException, NoSuchFieldException, IllegalArgumentException, + IllegalAccessException, InterruptedException { + context.put(KafkaSourceConstants.ZOOKEEPER_CONNECT_FLUME,"blabla:666"); + kafkaSource.configure(context); + kafkaSource.start(); + Thread.sleep(500L); + + Status status = kafkaSource.process(); + assertEquals(Status.BACKOFF, status); + } + + @Test + public void testBatchTime() throws InterruptedException, + EventDeliveryException { + context.put(KafkaSourceConstants.BATCH_DURATION_MS,"250"); + kafkaSource.configure(context); + kafkaSource.start(); + + Thread.sleep(500L); + + for (int i=1; i<5000; i++) { + kafkaServer.produce(topicName, "", "hello, world " + i); + } + Thread.sleep(500L); + + long startTime = System.currentTimeMillis(); + Status status = kafkaSource.process(); + long endTime = System.currentTimeMillis(); + assertEquals(Status.READY, status); + assertTrue(endTime - startTime < + ( context.getLong(KafkaSourceConstants.BATCH_DURATION_MS) + + context.getLong("kafka.consumer.timeout.ms")) ); + } +} \ No newline at end of file diff --git a/flume-ng-sources/flume-kafka-source/src/test/java/org/apache/flume/source/kafka/TestKafkaSourceUtil.java b/flume-ng-sources/flume-kafka-source/src/test/java/org/apache/flume/source/kafka/TestKafkaSourceUtil.java new file mode 100644 index 0000000000..0cbb4b69ec --- /dev/null +++ b/flume-ng-sources/flume-kafka-source/src/test/java/org/apache/flume/source/kafka/TestKafkaSourceUtil.java @@ -0,0 +1,92 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.flume.source.kafka; + +import static org.junit.Assert.assertEquals; +import static org.junit.Assert.assertNotNull; +import static org.junit.Assert.assertNull; + +import java.io.IOException; +import java.util.Properties; + +import kafka.javaapi.consumer.ConsumerConnector; +import org.apache.flume.Context; +import org.apache.zookeeper.server.*; +import org.junit.After; +import org.junit.Before; +import org.junit.Test; + +public class TestKafkaSourceUtil { + private Properties props = new Properties(); + private Context context = new Context(); + private int zkPort = 21818; // none-standard + private KafkaSourceEmbeddedZookeeper zookeeper; + + @Before + public void setUp() throws Exception { + context.put("kafka.consumer.timeout", "10"); + context.put("type", "KafkaSource"); + context.put("topic", "test"); + context.put("zookeeperConnect", "127.0.0.1:"+zkPort); + context.put("groupId","test"); + props = KafkaSourceUtil.getKafkaProperties(context); + zookeeper = new KafkaSourceEmbeddedZookeeper(zkPort); + + + } + + @After + public void tearDown() throws Exception { + zookeeper.stopZookeeper(); + } + + + @Test + public void testGetConsumer() { + ConsumerConnector cc = KafkaSourceUtil.getConsumer(props); + assertNotNull(cc); + + } + + @Test + public void testKafkaConsumerProperties() { + Context context = new Context(); + context.put("kafka.auto.commit.enable", "override.default.autocommit"); + context.put("kafka.fake.property", "kafka.property.value"); + context.put("kafka.zookeeper.connect","bad-zookeeper-list"); + context.put("zookeeperConnect","real-zookeeper-list"); + Properties kafkaProps = KafkaSourceUtil.getKafkaProperties(context); + + //check that we have defaults set + assertEquals( + kafkaProps.getProperty(KafkaSourceConstants.GROUP_ID), + KafkaSourceConstants.DEFAULT_GROUP_ID); + //check that kafka properties override the default and get correct name + assertEquals( + kafkaProps.getProperty(KafkaSourceConstants.AUTO_COMMIT_ENABLED), + "override.default.autocommit"); + //check that any kafka property gets in + assertEquals(kafkaProps.getProperty("fake.property"), + "kafka.property.value"); + //check that documented property overrides defaults + assertEquals(kafkaProps.getProperty("zookeeper.connect") + ,"real-zookeeper-list"); + } + + +} From 9f0abea027c408669977aa34cd32f5e525bb508b Mon Sep 17 00:00:00 2001 From: Hari Shreedharan Date: Tue, 7 Oct 2014 14:12:02 -0700 Subject: [PATCH 149/341] FLUME-2495. Kafka Source may miss events when channel is not available (Gwen Shapira via Hari) --- .../flume/source/kafka/KafkaSource.java | 21 ++- .../flume/source/kafka/TestKafkaSource.java | 147 ++++++++++++++++-- 2 files changed, 143 insertions(+), 25 deletions(-) diff --git a/flume-ng-sources/flume-kafka-source/src/main/java/org/apache/flume/source/kafka/KafkaSource.java b/flume-ng-sources/flume-kafka-source/src/main/java/org/apache/flume/source/kafka/KafkaSource.java index 8cdc9670fd..9d77b470e4 100644 --- a/flume-ng-sources/flume-kafka-source/src/main/java/org/apache/flume/source/kafka/KafkaSource.java +++ b/flume-ng-sources/flume-kafka-source/src/main/java/org/apache/flume/source/kafka/KafkaSource.java @@ -71,17 +71,15 @@ public class KafkaSource extends AbstractSource private final List eventList = new ArrayList(); public Status process() throws EventDeliveryException { - eventList.clear(); + byte[] bytes; Event event; Map headers; long batchStartTime = System.currentTimeMillis(); long batchEndTime = System.currentTimeMillis() + timeUpperLimit; try { - int eventCounter = 0; - int timeWaited = 0; boolean iterStatus = false; - while (eventCounter < batchUpperLimit && + while (eventList.size() < batchUpperLimit && System.currentTimeMillis() < batchEndTime) { iterStatus = hasNext(); if (iterStatus) { @@ -97,17 +95,21 @@ public Status process() throws EventDeliveryException { } event = EventBuilder.withBody(bytes, headers); eventList.add(event); - eventCounter++; } if (log.isDebugEnabled()) { log.debug("Waited: {} ", System.currentTimeMillis() - batchStartTime); - log.debug("Event #: {}", eventCounter); + log.debug("Event #: {}", eventList.size()); } } // If we have events, send events to channel + // clear the event list // and commit if Kafka doesn't auto-commit - if (eventCounter > 0) { + if (eventList.size() > 0) { getChannelProcessor().processEventBatch(eventList); + eventList.clear(); + if (log.isDebugEnabled()) { + log.debug("Wrote {} events to channel", eventList.size()); + } if (!kafkaAutoCommitEnabled) { // commit the read transactions to Kafka to avoid duplicates consumer.commitOffsets(); @@ -203,11 +205,6 @@ public synchronized void stop() { super.stop(); } - - - - - /** * Check if there are messages waiting in Kafka, * waiting until timeout (10ms by default) for messages to arrive. diff --git a/flume-ng-sources/flume-kafka-source/src/test/java/org/apache/flume/source/kafka/TestKafkaSource.java b/flume-ng-sources/flume-kafka-source/src/test/java/org/apache/flume/source/kafka/TestKafkaSource.java index 3695860e06..72eec77d69 100644 --- a/flume-ng-sources/flume-kafka-source/src/test/java/org/apache/flume/source/kafka/TestKafkaSource.java +++ b/flume-ng-sources/flume-kafka-source/src/test/java/org/apache/flume/source/kafka/TestKafkaSource.java @@ -75,19 +75,7 @@ public void setup() throws Exception { context.put(KafkaSourceConstants.TOPIC,topicName); context.put("kafka.consumer.timeout.ms","100"); - - ChannelProcessor channelProcessor = mock(ChannelProcessor.class); - - events = Lists.newArrayList(); - - doAnswer(new Answer() { - @Override - public Void answer(InvocationOnMock invocation) throws Throwable { - events.addAll((List)invocation.getArguments()[0]); - return null; - } - }).when(channelProcessor).processEventBatch(any(List.class)); - kafkaSource.setChannelProcessor(channelProcessor); + kafkaSource.setChannelProcessor(createGoodChannel()); } @After @@ -210,4 +198,137 @@ public void testBatchTime() throws InterruptedException, ( context.getLong(KafkaSourceConstants.BATCH_DURATION_MS) + context.getLong("kafka.consumer.timeout.ms")) ); } + + // Consume event, stop source, start again and make sure we are not + // consuming same event again + @Test + public void testCommit() throws InterruptedException, EventDeliveryException { + context.put(KafkaSourceConstants.BATCH_SIZE,"1"); + kafkaSource.configure(context); + kafkaSource.start(); + + Thread.sleep(500L); + + kafkaServer.produce(topicName, "", "hello, world"); + + Thread.sleep(500L); + + Assert.assertEquals(Status.READY, kafkaSource.process()); + kafkaSource.stop(); + Thread.sleep(500L); + kafkaSource.start(); + Thread.sleep(500L); + Assert.assertEquals(Status.BACKOFF, kafkaSource.process()); + + } + + // Remove channel processor and test if we can consume events again + @Test + public void testNonCommit() throws EventDeliveryException, + InterruptedException { + + context.put(KafkaSourceConstants.BATCH_SIZE,"1"); + context.put(KafkaSourceConstants.BATCH_DURATION_MS,"30000"); + kafkaSource.configure(context); + kafkaSource.start(); + Thread.sleep(500L); + + kafkaServer.produce(topicName, "", "hello, world"); + Thread.sleep(500L); + + kafkaSource.setChannelProcessor(createBadChannel()); + log.debug("processing from kafka to bad channel"); + Assert.assertEquals(Status.BACKOFF, kafkaSource.process()); + + log.debug("repairing channel"); + kafkaSource.setChannelProcessor(createGoodChannel()); + + log.debug("re-process to good channel - this should work"); + kafkaSource.process(); + Assert.assertEquals("hello, world", new String(events.get(0).getBody(), + Charsets.UTF_8)); + + + } + + @Test + public void testTwoBatches() throws InterruptedException, + EventDeliveryException { + context.put(KafkaSourceConstants.BATCH_SIZE,"1"); + context.put(KafkaSourceConstants.BATCH_DURATION_MS,"30000"); + kafkaSource.configure(context); + kafkaSource.start(); + Thread.sleep(500L); + + kafkaServer.produce(topicName, "", "event 1"); + Thread.sleep(500L); + + kafkaSource.process(); + Assert.assertEquals("event 1", new String(events.get(0).getBody(), + Charsets.UTF_8)); + events.clear(); + + kafkaServer.produce(topicName, "", "event 2"); + Thread.sleep(500L); + kafkaSource.process(); + Assert.assertEquals("event 2", new String(events.get(0).getBody(), + Charsets.UTF_8)); + } + + @Test + public void testTwoBatchesWithAutocommit() throws InterruptedException, + EventDeliveryException { + context.put(KafkaSourceConstants.BATCH_SIZE,"1"); + context.put(KafkaSourceConstants.BATCH_DURATION_MS,"30000"); + context.put("kafka.auto.commit.enable","true"); + kafkaSource.configure(context); + kafkaSource.start(); + Thread.sleep(500L); + + kafkaServer.produce(topicName, "", "event 1"); + Thread.sleep(500L); + + kafkaSource.process(); + Assert.assertEquals("event 1", new String(events.get(0).getBody(), + Charsets.UTF_8)); + events.clear(); + + kafkaServer.produce(topicName, "", "event 2"); + Thread.sleep(500L); + kafkaSource.process(); + Assert.assertEquals("event 2", new String(events.get(0).getBody(), + Charsets.UTF_8)); + + } + + ChannelProcessor createGoodChannel() { + + ChannelProcessor channelProcessor = mock(ChannelProcessor.class); + + events = Lists.newArrayList(); + + doAnswer(new Answer() { + @Override + public Void answer(InvocationOnMock invocation) throws Throwable { + events.addAll((List)invocation.getArguments()[0]); + return null; + } + }).when(channelProcessor).processEventBatch(any(List.class)); + + return channelProcessor; + + } + + ChannelProcessor createBadChannel() { + ChannelProcessor channelProcessor = mock(ChannelProcessor.class); + + doAnswer(new Answer() { + @Override + public Void answer(InvocationOnMock invocation) throws Throwable { + throw new ChannelException("channel intentional broken"); + } + }).when(channelProcessor).processEventBatch(any(List.class)); + + return channelProcessor; + } } \ No newline at end of file From 9968f0fb00f8d25245067242a48040229952e46f Mon Sep 17 00:00:00 2001 From: Hari Shreedharan Date: Wed, 8 Oct 2014 20:21:25 -0700 Subject: [PATCH 150/341] FLUME-2466. Embedded agent name cannot contain spaces (Johny Rufus via Hari) --- .../embedded/EmbeddedAgentConfiguration.java | 7 ++--- .../agent/embedded/TestEmbeddedAgent.java | 27 +++++++++++++++++++ 2 files changed, 31 insertions(+), 3 deletions(-) diff --git a/flume-ng-embedded-agent/src/main/java/org/apache/flume/agent/embedded/EmbeddedAgentConfiguration.java b/flume-ng-embedded-agent/src/main/java/org/apache/flume/agent/embedded/EmbeddedAgentConfiguration.java index 06ea7c042e..02182ddb8d 100644 --- a/flume-ng-embedded-agent/src/main/java/org/apache/flume/agent/embedded/EmbeddedAgentConfiguration.java +++ b/flume-ng-embedded-agent/src/main/java/org/apache/flume/agent/embedded/EmbeddedAgentConfiguration.java @@ -206,9 +206,10 @@ static Map configure(String name, } String sinkNames = properties.remove(SINKS); - String sourceName = "source-" + name; - String channelName = "channel-" + name; - String sinkGroupName = "sink-group-" + name; + String strippedName = name.replaceAll("\\s+",""); + String sourceName = "source-" + strippedName; + String channelName = "channel-" + strippedName; + String sinkGroupName = "sink-group-" + strippedName; /* * Now we are going to process the user supplied configuration diff --git a/flume-ng-embedded-agent/src/test/java/org/apache/flume/agent/embedded/TestEmbeddedAgent.java b/flume-ng-embedded-agent/src/test/java/org/apache/flume/agent/embedded/TestEmbeddedAgent.java index 975ba8d1f3..241e2b5264 100644 --- a/flume-ng-embedded-agent/src/test/java/org/apache/flume/agent/embedded/TestEmbeddedAgent.java +++ b/flume-ng-embedded-agent/src/test/java/org/apache/flume/agent/embedded/TestEmbeddedAgent.java @@ -165,6 +165,33 @@ public void testPutWithInterceptors() throws Exception { Assert.assertEquals(newHeaders, event.getHeaders()); } + + @Test(timeout = 30000L) + public void testEmbeddedAgentName() throws Exception { + EmbeddedAgent embedAgent = new EmbeddedAgent("test 1 2" + serialNumber.incrementAndGet()); + List events = Lists.newArrayList(); + events.add(EventBuilder.withBody(body, headers)); + embedAgent.configure(properties); + embedAgent.start(); + embedAgent.putAll(events); + + Event event; + while((event = eventCollector.poll()) == null) { + Thread.sleep(500L); + } + Assert.assertNotNull(event); + Assert.assertArrayEquals(body, event.getBody()); + Assert.assertEquals(headers, event.getHeaders()); + if(embedAgent != null) { + try { + embedAgent.stop(); + } catch (Exception e) { + LOGGER.debug("Error shutting down agent", e); + } + } + } + + static class EventCollector implements AvroSourceProtocol { private final Queue eventQueue = new LinkedBlockingQueue(); From 35cf900017a797e5ff3aedb01f05c88bb3f86ca0 Mon Sep 17 00:00:00 2001 From: Hari Shreedharan Date: Thu, 9 Oct 2014 16:10:10 -0700 Subject: [PATCH 151/341] FLUME-2346. idLogFileMap in Log can lose track of file ids. (Johny Rufus via Hari) --- .../src/main/java/org/apache/flume/channel/file/Log.java | 7 ++++--- 1 file changed, 4 insertions(+), 3 deletions(-) diff --git a/flume-ng-channels/flume-file-channel/src/main/java/org/apache/flume/channel/file/Log.java b/flume-ng-channels/flume-file-channel/src/main/java/org/apache/flume/channel/file/Log.java index 5b581e1c18..e797797d3c 100644 --- a/flume-ng-channels/flume-file-channel/src/main/java/org/apache/flume/channel/file/Log.java +++ b/flume-ng-channels/flume-file-channel/src/main/java/org/apache/flume/channel/file/Log.java @@ -1029,11 +1029,12 @@ private Boolean writeCheckpoint(Boolean force) throws Exception { try { writer.markCheckpoint(logWriteOrderID); } finally { + reader = LogFileFactory.getRandomReader(file, + encryptionKeyProvider, fsyncPerTransaction); + idLogFileMap.put(id, reader); writer.close(); } - reader = LogFileFactory.getRandomReader(file, - encryptionKeyProvider, fsyncPerTransaction); - idLogFileMap.put(id, reader); + LOGGER.debug("Updated checkpoint for file: " + file + "logWriteOrderID " + logWriteOrderID); idIterator.remove(); From 72b54c9909290fa5b98123b4b0c1332a425c5c64 Mon Sep 17 00:00:00 2001 From: Hari Shreedharan Date: Thu, 9 Oct 2014 23:17:30 -0700 Subject: [PATCH 152/341] FLUME-2472. SyslogUtils fails when system locale is not English (Santiago M. Mola via Hari) --- .../java/org/apache/flume/source/SyslogUtils.java | 11 ++++++----- .../java/org/apache/flume/source/TestSyslogUtils.java | 3 ++- 2 files changed, 8 insertions(+), 6 deletions(-) diff --git a/flume-ng-core/src/main/java/org/apache/flume/source/SyslogUtils.java b/flume-ng-core/src/main/java/org/apache/flume/source/SyslogUtils.java index 208fefe966..43f8db08e1 100644 --- a/flume-ng-core/src/main/java/org/apache/flume/source/SyslogUtils.java +++ b/flume-ng-core/src/main/java/org/apache/flume/source/SyslogUtils.java @@ -36,6 +36,7 @@ import java.util.HashMap; import java.util.HashSet; import java.util.List; +import java.util.Locale; import java.util.Map; import java.util.Set; import java.util.regex.MatchResult; @@ -240,10 +241,10 @@ private void initHeaderFormats() { // timezone in RFC5424 is [+-]tt:tt, so remove the ':' for java date formatting fmt1.searchPattern.add("([+-])(\\d{2})[:](\\d{2})"); fmt1.replacePattern.add("$1$2$3"); - fmt1.dateFormat.add(new SimpleDateFormat(SYSLOG_TIMESTAMP_FORMAT_RFC5424_1)); - fmt1.dateFormat.add(new SimpleDateFormat(SYSLOG_TIMESTAMP_FORMAT_RFC5424_2)); - fmt1.dateFormat.add(new SimpleDateFormat(SYSLOG_TIMESTAMP_FORMAT_RFC5424_3)); - fmt1.dateFormat.add(new SimpleDateFormat(SYSLOG_TIMESTAMP_FORMAT_RFC5424_4)); + fmt1.dateFormat.add(new SimpleDateFormat(SYSLOG_TIMESTAMP_FORMAT_RFC5424_1, Locale.ENGLISH)); + fmt1.dateFormat.add(new SimpleDateFormat(SYSLOG_TIMESTAMP_FORMAT_RFC5424_2, Locale.ENGLISH)); + fmt1.dateFormat.add(new SimpleDateFormat(SYSLOG_TIMESTAMP_FORMAT_RFC5424_3, Locale.ENGLISH)); + fmt1.dateFormat.add(new SimpleDateFormat(SYSLOG_TIMESTAMP_FORMAT_RFC5424_4, Locale.ENGLISH)); fmt1.addYear = false; // setup RFC3164 formater @@ -252,7 +253,7 @@ private void initHeaderFormats() { // the single digit date has two spaces, so trim it fmt2.searchPattern.add(" "); fmt2.replacePattern.add(" "); - fmt2.dateFormat.add(new SimpleDateFormat(SYSLOG_TIMESTAMP_FORMAT_RFC3164_1)); + fmt2.dateFormat.add(new SimpleDateFormat(SYSLOG_TIMESTAMP_FORMAT_RFC3164_1, Locale.ENGLISH)); fmt2.addYear = true; formats.add(fmt1); diff --git a/flume-ng-core/src/test/java/org/apache/flume/source/TestSyslogUtils.java b/flume-ng-core/src/test/java/org/apache/flume/source/TestSyslogUtils.java index 76ee5b1370..86ef40f09f 100644 --- a/flume-ng-core/src/test/java/org/apache/flume/source/TestSyslogUtils.java +++ b/flume-ng-core/src/test/java/org/apache/flume/source/TestSyslogUtils.java @@ -29,6 +29,7 @@ import java.text.SimpleDateFormat; import java.util.Calendar; import java.util.HashSet; +import java.util.Locale; import java.util.Map; public class TestSyslogUtils { @@ -184,7 +185,7 @@ public static void checkHeader(String keepFields, String msg1, String stamp1, St if (stamp1 == null) { Assert.assertFalse(headers2.containsKey("timestamp")); } else { - SimpleDateFormat formater = new SimpleDateFormat(format1); + SimpleDateFormat formater = new SimpleDateFormat(format1, Locale.ENGLISH); Assert.assertEquals(String.valueOf(formater.parse(stamp1).getTime()), headers2.get("timestamp")); } if (host1 == null) { From 7ffbd86bc01013ef0a0cbf855243aec5d772a386 Mon Sep 17 00:00:00 2001 From: Hari Shreedharan Date: Fri, 10 Oct 2014 00:11:21 -0700 Subject: [PATCH 153/341] FLUME-2489. Upgrade Apache Curator to 2.6.0 (Santiago M. Mola via Hari) --- pom.xml | 6 +++--- 1 file changed, 3 insertions(+), 3 deletions(-) diff --git a/pom.xml b/pom.xml index 806d2a1733..4f550d388b 100644 --- a/pom.xml +++ b/pom.xml @@ -1329,19 +1329,19 @@ limitations under the License. org.apache.curator curator-framework - 2.3.0 + 2.6.0 org.apache.curator curator-recipes - 2.3.0 + 2.6.0 org.apache.curator curator-test - 2.3.0 + 2.6.0 test From f17c7d5022d3e9d112a3843909ad523535fe7e4f Mon Sep 17 00:00:00 2001 From: Hari Shreedharan Date: Fri, 10 Oct 2014 10:43:58 -0700 Subject: [PATCH 154/341] FLUME-2475. toLowerCase/toUpperCase must specify Locale. (Santiago M. Mola via Hari) --- .../file/encryption/CipherProviderFactory.java | 3 ++- .../file/encryption/KeyProviderFactory.java | 4 +++- .../apache/flume/channel/jdbc/DatabaseType.java | 4 +++- .../flume/channel/jdbc/TransactionIsolation.java | 3 ++- .../flume/channel/jdbc/TestDatabaseTypeEnum.java | 3 ++- .../jdbc/TestTransactionIsolationLevelEnum.java | 3 ++- .../clients/log4jappender/Log4jAvroHeaders.java | 4 +++- .../flume/conf/ComponentConfigurationFactory.java | 14 ++++++++------ .../org/apache/flume/conf/FlumeConfiguration.java | 7 ++++--- .../flume/conf/source/SourceConfiguration.java | 3 ++- .../flume/channel/ChannelSelectorFactory.java | 3 ++- .../flume/channel/DefaultChannelFactory.java | 4 +++- .../avro/ReliableSpoolingFileEventReader.java | 2 +- .../instrumentation/MonitoredCounterGroup.java | 5 +++-- .../interceptor/InterceptorBuilderFactory.java | 4 +++- .../flume/serialization/AvroEventDeserializer.java | 3 ++- .../serialization/EventDeserializerFactory.java | 4 +++- .../serialization/EventSerializerFactory.java | 4 +++- .../flume/serialization/LineDeserializer.java | 1 - .../org/apache/flume/sink/DefaultSinkFactory.java | 4 +++- .../apache/flume/sink/SinkProcessorFactory.java | 5 +++-- .../apache/flume/source/DefaultSourceFactory.java | 4 +++- .../apache/flume/source/SpoolDirectorySource.java | 5 +++-- .../java/org/apache/flume/source/SyslogUtils.java | 2 +- .../org/apache/flume/tools/DirectMemoryUtils.java | 5 +++-- .../org/apache/flume/tools/PlatformDetect.java | 4 +++- .../agent/embedded/EmbeddedAgentConfiguration.java | 3 ++- .../java/org/apache/flume/node/Application.java | 3 ++- .../org/apache/flume/api/RpcClientFactory.java | 3 ++- .../org/apache/flume/sink/kite/DatasetSink.java | 3 ++- .../flume/sink/hdfs/AvroEventSerializer.java | 3 ++- .../org/apache/flume/source/jms/JMSSource.java | 5 +++-- .../java/org/apache/flume/tools/FlumeToolType.java | 4 +++- 33 files changed, 86 insertions(+), 45 deletions(-) diff --git a/flume-ng-channels/flume-file-channel/src/main/java/org/apache/flume/channel/file/encryption/CipherProviderFactory.java b/flume-ng-channels/flume-file-channel/src/main/java/org/apache/flume/channel/file/encryption/CipherProviderFactory.java index d5e9353ed9..ca11f6bbac 100644 --- a/flume-ng-channels/flume-file-channel/src/main/java/org/apache/flume/channel/file/encryption/CipherProviderFactory.java +++ b/flume-ng-channels/flume-file-channel/src/main/java/org/apache/flume/channel/file/encryption/CipherProviderFactory.java @@ -19,6 +19,7 @@ package org.apache.flume.channel.file.encryption; import java.security.Key; +import java.util.Locale; import org.apache.flume.FlumeException; import org.slf4j.Logger; @@ -54,7 +55,7 @@ private static CipherProvider getProvider(String cipherProviderType) { // try to find builder class in enum of known providers CipherProviderType type; try { - type = CipherProviderType.valueOf(cipherProviderType.toUpperCase()); + type = CipherProviderType.valueOf(cipherProviderType.toUpperCase(Locale.ENGLISH)); } catch (IllegalArgumentException e) { logger.debug("Not in enum, loading provider class: {}", cipherProviderType); type = CipherProviderType.OTHER; diff --git a/flume-ng-channels/flume-file-channel/src/main/java/org/apache/flume/channel/file/encryption/KeyProviderFactory.java b/flume-ng-channels/flume-file-channel/src/main/java/org/apache/flume/channel/file/encryption/KeyProviderFactory.java index 60d51ed5d3..c45f35a3a7 100644 --- a/flume-ng-channels/flume-file-channel/src/main/java/org/apache/flume/channel/file/encryption/KeyProviderFactory.java +++ b/flume-ng-channels/flume-file-channel/src/main/java/org/apache/flume/channel/file/encryption/KeyProviderFactory.java @@ -18,6 +18,8 @@ */ package org.apache.flume.channel.file.encryption; +import java.util.Locale; + import org.apache.flume.Context; import org.apache.flume.FlumeException; import org.slf4j.Logger; @@ -37,7 +39,7 @@ public static KeyProvider getInstance(String keyProviderType, Context context) { // try to find builder class in enum of known providers KeyProviderType type; try { - type = KeyProviderType.valueOf(keyProviderType.toUpperCase()); + type = KeyProviderType.valueOf(keyProviderType.toUpperCase(Locale.ENGLISH)); } catch (IllegalArgumentException e) { logger.debug("Not in enum, loading provider class: {}", keyProviderType); type = KeyProviderType.OTHER; diff --git a/flume-ng-channels/flume-jdbc-channel/src/main/java/org/apache/flume/channel/jdbc/DatabaseType.java b/flume-ng-channels/flume-jdbc-channel/src/main/java/org/apache/flume/channel/jdbc/DatabaseType.java index e26cb8429b..183a7318c2 100644 --- a/flume-ng-channels/flume-jdbc-channel/src/main/java/org/apache/flume/channel/jdbc/DatabaseType.java +++ b/flume-ng-channels/flume-jdbc-channel/src/main/java/org/apache/flume/channel/jdbc/DatabaseType.java @@ -17,6 +17,8 @@ */ package org.apache.flume.channel.jdbc; +import java.util.Locale; + public enum DatabaseType { /** All other databases */ OTHER("OTHER", null), @@ -56,7 +58,7 @@ public String getValidationQuery() { public static DatabaseType getByName(String dbName) { DatabaseType type = null; try { - type = DatabaseType.valueOf(dbName.trim().toUpperCase()); + type = DatabaseType.valueOf(dbName.trim().toUpperCase(Locale.ENGLISH)); } catch (IllegalArgumentException ex) { type = DatabaseType.OTHER; } diff --git a/flume-ng-channels/flume-jdbc-channel/src/main/java/org/apache/flume/channel/jdbc/TransactionIsolation.java b/flume-ng-channels/flume-jdbc-channel/src/main/java/org/apache/flume/channel/jdbc/TransactionIsolation.java index ed157cfc0d..5cc2489149 100644 --- a/flume-ng-channels/flume-jdbc-channel/src/main/java/org/apache/flume/channel/jdbc/TransactionIsolation.java +++ b/flume-ng-channels/flume-jdbc-channel/src/main/java/org/apache/flume/channel/jdbc/TransactionIsolation.java @@ -18,6 +18,7 @@ package org.apache.flume.channel.jdbc; import java.sql.Connection; +import java.util.Locale; public enum TransactionIsolation { @@ -47,6 +48,6 @@ public String toString() { } public static TransactionIsolation getByName(String name) { - return valueOf(name.trim().toUpperCase()); + return valueOf(name.trim().toUpperCase(Locale.ENGLISH)); } } diff --git a/flume-ng-channels/flume-jdbc-channel/src/test/java/org/apache/flume/channel/jdbc/TestDatabaseTypeEnum.java b/flume-ng-channels/flume-jdbc-channel/src/test/java/org/apache/flume/channel/jdbc/TestDatabaseTypeEnum.java index 55ad1eb530..39f488c211 100644 --- a/flume-ng-channels/flume-jdbc-channel/src/test/java/org/apache/flume/channel/jdbc/TestDatabaseTypeEnum.java +++ b/flume-ng-channels/flume-jdbc-channel/src/test/java/org/apache/flume/channel/jdbc/TestDatabaseTypeEnum.java @@ -18,6 +18,7 @@ package org.apache.flume.channel.jdbc; import java.util.HashMap; +import java.util.Locale; import java.util.Map; import org.junit.Assert; @@ -61,7 +62,7 @@ public void testDatabaseTypeLookup() { Assert.assertSame(type, lookupType); Assert.assertEquals(key, lookupTypeName); - DatabaseType lookupType2 = DatabaseType.getByName(key.toLowerCase()); + DatabaseType lookupType2 = DatabaseType.getByName(key.toLowerCase(Locale.ENGLISH)); Assert.assertSame(type, lookupType2); } } diff --git a/flume-ng-channels/flume-jdbc-channel/src/test/java/org/apache/flume/channel/jdbc/TestTransactionIsolationLevelEnum.java b/flume-ng-channels/flume-jdbc-channel/src/test/java/org/apache/flume/channel/jdbc/TestTransactionIsolationLevelEnum.java index ea0bac37c2..b9f35c6d7a 100644 --- a/flume-ng-channels/flume-jdbc-channel/src/test/java/org/apache/flume/channel/jdbc/TestTransactionIsolationLevelEnum.java +++ b/flume-ng-channels/flume-jdbc-channel/src/test/java/org/apache/flume/channel/jdbc/TestTransactionIsolationLevelEnum.java @@ -18,6 +18,7 @@ package org.apache.flume.channel.jdbc; import java.util.HashMap; +import java.util.Locale; import java.util.Map; import org.junit.Assert; @@ -62,7 +63,7 @@ public void testReverseLookup() { Assert.assertEquals(key, lookupTxIsolationName); TransactionIsolation lookupTxIsolation2 = - TransactionIsolation.getByName(key.toLowerCase()); + TransactionIsolation.getByName(key.toLowerCase(Locale.ENGLISH)); Assert.assertSame(txIsolation, lookupTxIsolation2); } } diff --git a/flume-ng-clients/flume-ng-log4jappender/src/main/java/org/apache/flume/clients/log4jappender/Log4jAvroHeaders.java b/flume-ng-clients/flume-ng-log4jappender/src/main/java/org/apache/flume/clients/log4jappender/Log4jAvroHeaders.java index 08a7203a52..b68e7496d2 100644 --- a/flume-ng-clients/flume-ng-log4jappender/src/main/java/org/apache/flume/clients/log4jappender/Log4jAvroHeaders.java +++ b/flume-ng-clients/flume-ng-log4jappender/src/main/java/org/apache/flume/clients/log4jappender/Log4jAvroHeaders.java @@ -18,6 +18,8 @@ */ package org.apache.flume.clients.log4jappender; +import java.util.Locale; + public enum Log4jAvroHeaders { OTHER("flume.client.log4j.logger.other"), LOGGER_NAME("flume.client.log4j.logger.name"), @@ -43,7 +45,7 @@ public String toString(){ public static Log4jAvroHeaders getByName(String headerName){ Log4jAvroHeaders hdrs = null; try{ - hdrs = Log4jAvroHeaders.valueOf(headerName.toLowerCase().trim()); + hdrs = Log4jAvroHeaders.valueOf(headerName.toLowerCase(Locale.ENGLISH).trim()); } catch(IllegalArgumentException e){ hdrs = Log4jAvroHeaders.OTHER; diff --git a/flume-ng-configuration/src/main/java/org/apache/flume/conf/ComponentConfigurationFactory.java b/flume-ng-configuration/src/main/java/org/apache/flume/conf/ComponentConfigurationFactory.java index c867a03dbc..0433c9ca58 100644 --- a/flume-ng-configuration/src/main/java/org/apache/flume/conf/ComponentConfigurationFactory.java +++ b/flume-ng-configuration/src/main/java/org/apache/flume/conf/ComponentConfigurationFactory.java @@ -16,6 +16,8 @@ */ package org.apache.flume.conf; +import java.util.Locale; + import org.apache.flume.conf.ComponentConfiguration.ComponentType; import org.apache.flume.conf.channel.ChannelConfiguration.ChannelConfigurationType; import org.apache.flume.conf.channel.ChannelSelectorConfiguration.ChannelSelectorConfigurationType; @@ -40,22 +42,22 @@ public class ComponentConfigurationFactory { return confType.getConstructor(String.class).newInstance(type); } catch (Exception ignored) { try { - type = type.toUpperCase(); + type = type.toUpperCase(Locale.ENGLISH); switch(component){ case SOURCE: - return SourceConfigurationType.valueOf(type.toUpperCase()) + return SourceConfigurationType.valueOf(type.toUpperCase(Locale.ENGLISH)) .getConfiguration(name); case SINK: - return SinkConfigurationType.valueOf(type.toUpperCase()) + return SinkConfigurationType.valueOf(type.toUpperCase(Locale.ENGLISH)) .getConfiguration(name); case CHANNEL: - return ChannelConfigurationType.valueOf(type.toUpperCase()) + return ChannelConfigurationType.valueOf(type.toUpperCase(Locale.ENGLISH)) .getConfiguration(name); case SINK_PROCESSOR: - return SinkProcessorConfigurationType.valueOf(type.toUpperCase()) + return SinkProcessorConfigurationType.valueOf(type.toUpperCase(Locale.ENGLISH)) .getConfiguration(name); case CHANNELSELECTOR: - return ChannelSelectorConfigurationType.valueOf(type.toUpperCase()) + return ChannelSelectorConfigurationType.valueOf(type.toUpperCase(Locale.ENGLISH)) .getConfiguration(name); case SINKGROUP: return new SinkGroupConfiguration(name); diff --git a/flume-ng-configuration/src/main/java/org/apache/flume/conf/FlumeConfiguration.java b/flume-ng-configuration/src/main/java/org/apache/flume/conf/FlumeConfiguration.java index f214149fe8..5997406c68 100644 --- a/flume-ng-configuration/src/main/java/org/apache/flume/conf/FlumeConfiguration.java +++ b/flume-ng-configuration/src/main/java/org/apache/flume/conf/FlumeConfiguration.java @@ -24,6 +24,7 @@ import java.util.Iterator; import java.util.LinkedList; import java.util.List; +import java.util.Locale; import java.util.Map; import java.util.Properties; import java.util.Set; @@ -457,7 +458,7 @@ private Set validateChannels(Set channelSet) { configSpecified = true; } } else { - config = chType.toString().toUpperCase(); + config = chType.toString().toUpperCase(Locale.ENGLISH); configSpecified = true; } @@ -553,7 +554,7 @@ private Set validateSources(Set channelSet) { configSpecified = true; } } else { - config = srcType.toString().toUpperCase(); + config = srcType.toString().toUpperCase(Locale.ENGLISH); configSpecified = true; } try { @@ -667,7 +668,7 @@ private Set validateSinks(Set channelSet) { configSpecified = true; } } else { - config = sinkType.toString().toUpperCase(); + config = sinkType.toString().toUpperCase(Locale.ENGLISH); configSpecified = true; } try { diff --git a/flume-ng-configuration/src/main/java/org/apache/flume/conf/source/SourceConfiguration.java b/flume-ng-configuration/src/main/java/org/apache/flume/conf/source/SourceConfiguration.java index 13c3360598..899d805058 100644 --- a/flume-ng-configuration/src/main/java/org/apache/flume/conf/source/SourceConfiguration.java +++ b/flume-ng-configuration/src/main/java/org/apache/flume/conf/source/SourceConfiguration.java @@ -18,6 +18,7 @@ import java.util.Arrays; import java.util.HashSet; +import java.util.Locale; import java.util.Map; import java.util.Set; @@ -93,7 +94,7 @@ public void configure(Context context) throws ConfigurationException { config = "OTHER"; } } else { - config = selectorType.toString().toUpperCase(); + config = selectorType.toString().toUpperCase(Locale.ENGLISH); } this.selectorConf = diff --git a/flume-ng-core/src/main/java/org/apache/flume/channel/ChannelSelectorFactory.java b/flume-ng-core/src/main/java/org/apache/flume/channel/ChannelSelectorFactory.java index 998c8c09ee..80b1453c82 100644 --- a/flume-ng-core/src/main/java/org/apache/flume/channel/ChannelSelectorFactory.java +++ b/flume-ng-core/src/main/java/org/apache/flume/channel/ChannelSelectorFactory.java @@ -19,6 +19,7 @@ package org.apache.flume.channel; import java.util.List; +import java.util.Locale; import java.util.Map; import org.apache.flume.Channel; @@ -73,7 +74,7 @@ private static ChannelSelector getSelectorForType(String type) { ChannelSelectorType selectorType = ChannelSelectorType.OTHER; try { - selectorType = ChannelSelectorType.valueOf(type.toUpperCase()); + selectorType = ChannelSelectorType.valueOf(type.toUpperCase(Locale.ENGLISH)); } catch (IllegalArgumentException ex) { LOGGER.debug("Selector type {} is a custom type", type); } diff --git a/flume-ng-core/src/main/java/org/apache/flume/channel/DefaultChannelFactory.java b/flume-ng-core/src/main/java/org/apache/flume/channel/DefaultChannelFactory.java index 06e1763726..1dd124efb7 100644 --- a/flume-ng-core/src/main/java/org/apache/flume/channel/DefaultChannelFactory.java +++ b/flume-ng-core/src/main/java/org/apache/flume/channel/DefaultChannelFactory.java @@ -19,6 +19,8 @@ package org.apache.flume.channel; +import java.util.Locale; + import org.apache.flume.Channel; import org.apache.flume.ChannelFactory; import org.apache.flume.FlumeException; @@ -54,7 +56,7 @@ public Class getClass(String type) String channelClassName = type; ChannelType channelType = ChannelType.OTHER; try { - channelType = ChannelType.valueOf(type.toUpperCase()); + channelType = ChannelType.valueOf(type.toUpperCase(Locale.ENGLISH)); } catch (IllegalArgumentException ex) { logger.debug("Channel type {} is a custom type", type); } diff --git a/flume-ng-core/src/main/java/org/apache/flume/client/avro/ReliableSpoolingFileEventReader.java b/flume-ng-core/src/main/java/org/apache/flume/client/avro/ReliableSpoolingFileEventReader.java index 0bc3f23f03..f858b56e53 100644 --- a/flume-ng-core/src/main/java/org/apache/flume/client/avro/ReliableSpoolingFileEventReader.java +++ b/flume-ng-core/src/main/java/org/apache/flume/client/avro/ReliableSpoolingFileEventReader.java @@ -568,7 +568,7 @@ public static class Builder { SpoolDirectorySourceConfigurationConstants.DEFAULT_INPUT_CHARSET; private DecodeErrorPolicy decodeErrorPolicy = DecodeErrorPolicy.valueOf( SpoolDirectorySourceConfigurationConstants.DEFAULT_DECODE_ERROR_POLICY - .toUpperCase()); + .toUpperCase(Locale.ENGLISH)); private ConsumeOrder consumeOrder = SpoolDirectorySourceConfigurationConstants.DEFAULT_CONSUME_ORDER; diff --git a/flume-ng-core/src/main/java/org/apache/flume/instrumentation/MonitoredCounterGroup.java b/flume-ng-core/src/main/java/org/apache/flume/instrumentation/MonitoredCounterGroup.java index 1d3f0f1a23..44e26e4aeb 100644 --- a/flume-ng-core/src/main/java/org/apache/flume/instrumentation/MonitoredCounterGroup.java +++ b/flume-ng-core/src/main/java/org/apache/flume/instrumentation/MonitoredCounterGroup.java @@ -24,6 +24,7 @@ import java.util.HashMap; import java.util.Iterator; import java.util.List; +import java.util.Locale; import java.util.Map; import java.util.concurrent.atomic.AtomicLong; @@ -105,7 +106,7 @@ void register() { if (!registered) { try { ObjectName objName = new ObjectName("org.apache.flume." - + type.name().toLowerCase() + ":type=" + this.name); + + type.name().toLowerCase(Locale.ENGLISH) + ":type=" + this.name); if (ManagementFactory.getPlatformMBeanServer().isRegistered(objName)) { logger.debug("Monitored counter group for type: " + type + ", name: " @@ -149,7 +150,7 @@ public void stop() { logger.info("Component type: " + type + ", name: " + name + " stopped"); // Retrieve the type for this counter group - final String typePrefix = type.name().toLowerCase(); + final String typePrefix = type.name().toLowerCase(Locale.ENGLISH); // Print out the startTime for this component logger.info("Shutdown Metric for type: " + type + ", " diff --git a/flume-ng-core/src/main/java/org/apache/flume/interceptor/InterceptorBuilderFactory.java b/flume-ng-core/src/main/java/org/apache/flume/interceptor/InterceptorBuilderFactory.java index 50e6fbc02f..7c64613d71 100644 --- a/flume-ng-core/src/main/java/org/apache/flume/interceptor/InterceptorBuilderFactory.java +++ b/flume-ng-core/src/main/java/org/apache/flume/interceptor/InterceptorBuilderFactory.java @@ -18,6 +18,8 @@ package org.apache.flume.interceptor; +import java.util.Locale; + import org.apache.flume.interceptor.Interceptor.Builder; /** @@ -28,7 +30,7 @@ public class InterceptorBuilderFactory { private static Class lookup(String name) { try { - return InterceptorType.valueOf(name.toUpperCase()).getBuilderClass(); + return InterceptorType.valueOf(name.toUpperCase(Locale.ENGLISH)).getBuilderClass(); } catch (IllegalArgumentException e) { return null; } diff --git a/flume-ng-core/src/main/java/org/apache/flume/serialization/AvroEventDeserializer.java b/flume-ng-core/src/main/java/org/apache/flume/serialization/AvroEventDeserializer.java index e44978f423..5faf449f3a 100644 --- a/flume-ng-core/src/main/java/org/apache/flume/serialization/AvroEventDeserializer.java +++ b/flume-ng-core/src/main/java/org/apache/flume/serialization/AvroEventDeserializer.java @@ -43,6 +43,7 @@ import java.io.IOException; import java.security.NoSuchAlgorithmException; import java.util.List; +import java.util.Locale; /** * A deserializer that parses Avro container files, generating one Flume event @@ -83,7 +84,7 @@ private AvroEventDeserializer(Context context, ResettableInputStream ris) { schemaType = AvroSchemaType.valueOf( context.getString(CONFIG_SCHEMA_TYPE_KEY, - AvroSchemaType.HASH.toString()).toUpperCase()); + AvroSchemaType.HASH.toString()).toUpperCase(Locale.ENGLISH)); if (schemaType == AvroSchemaType.LITERAL) { logger.warn(CONFIG_SCHEMA_TYPE_KEY + " set to " + AvroSchemaType.LITERAL.toString() + ", so storing full Avro " + diff --git a/flume-ng-core/src/main/java/org/apache/flume/serialization/EventDeserializerFactory.java b/flume-ng-core/src/main/java/org/apache/flume/serialization/EventDeserializerFactory.java index 80b9cc3b52..73a7761d77 100644 --- a/flume-ng-core/src/main/java/org/apache/flume/serialization/EventDeserializerFactory.java +++ b/flume-ng-core/src/main/java/org/apache/flume/serialization/EventDeserializerFactory.java @@ -26,6 +26,8 @@ import org.slf4j.Logger; import org.slf4j.LoggerFactory; +import java.util.Locale; + @InterfaceAudience.Private @InterfaceStability.Stable public class EventDeserializerFactory { @@ -42,7 +44,7 @@ public static EventDeserializer getInstance( // try to find builder class in enum of known output serializers EventDeserializerType type; try { - type = EventDeserializerType.valueOf(deserializerType.toUpperCase()); + type = EventDeserializerType.valueOf(deserializerType.toUpperCase(Locale.ENGLISH)); } catch (IllegalArgumentException e) { logger.debug("Not in enum, loading builder class: {}", deserializerType); type = EventDeserializerType.OTHER; diff --git a/flume-ng-core/src/main/java/org/apache/flume/serialization/EventSerializerFactory.java b/flume-ng-core/src/main/java/org/apache/flume/serialization/EventSerializerFactory.java index dbb3935452..9c3682f98b 100644 --- a/flume-ng-core/src/main/java/org/apache/flume/serialization/EventSerializerFactory.java +++ b/flume-ng-core/src/main/java/org/apache/flume/serialization/EventSerializerFactory.java @@ -20,6 +20,8 @@ import com.google.common.base.Preconditions; import java.io.OutputStream; +import java.util.Locale; + import org.apache.flume.Context; import org.apache.flume.FlumeException; import org.apache.flume.annotations.InterfaceAudience; @@ -43,7 +45,7 @@ public static EventSerializer getInstance( // try to find builder class in enum of known output serializers EventSerializerType type; try { - type = EventSerializerType.valueOf(serializerType.toUpperCase()); + type = EventSerializerType.valueOf(serializerType.toUpperCase(Locale.ENGLISH)); } catch (IllegalArgumentException e) { logger.debug("Not in enum, loading builder class: {}", serializerType); type = EventSerializerType.OTHER; diff --git a/flume-ng-core/src/main/java/org/apache/flume/serialization/LineDeserializer.java b/flume-ng-core/src/main/java/org/apache/flume/serialization/LineDeserializer.java index d8e5ae22b1..7c872353a2 100644 --- a/flume-ng-core/src/main/java/org/apache/flume/serialization/LineDeserializer.java +++ b/flume-ng-core/src/main/java/org/apache/flume/serialization/LineDeserializer.java @@ -30,7 +30,6 @@ import java.io.IOException; import java.nio.charset.Charset; import java.util.List; -import java.util.concurrent.atomic.AtomicBoolean; /** * A deserializer that parses text lines from a file. diff --git a/flume-ng-core/src/main/java/org/apache/flume/sink/DefaultSinkFactory.java b/flume-ng-core/src/main/java/org/apache/flume/sink/DefaultSinkFactory.java index 0e49e0b623..6a5be921fd 100644 --- a/flume-ng-core/src/main/java/org/apache/flume/sink/DefaultSinkFactory.java +++ b/flume-ng-core/src/main/java/org/apache/flume/sink/DefaultSinkFactory.java @@ -19,6 +19,8 @@ package org.apache.flume.sink; +import java.util.Locale; + import org.apache.flume.FlumeException; import org.apache.flume.Sink; import org.apache.flume.SinkFactory; @@ -56,7 +58,7 @@ public Class getClass(String type) String sinkClassName = type; SinkType sinkType = SinkType.OTHER; try { - sinkType = SinkType.valueOf(type.toUpperCase()); + sinkType = SinkType.valueOf(type.toUpperCase(Locale.ENGLISH)); } catch (IllegalArgumentException ex) { logger.debug("Sink type {} is a custom type", type); } diff --git a/flume-ng-core/src/main/java/org/apache/flume/sink/SinkProcessorFactory.java b/flume-ng-core/src/main/java/org/apache/flume/sink/SinkProcessorFactory.java index 149987253b..dcdcad2160 100644 --- a/flume-ng-core/src/main/java/org/apache/flume/sink/SinkProcessorFactory.java +++ b/flume-ng-core/src/main/java/org/apache/flume/sink/SinkProcessorFactory.java @@ -18,6 +18,7 @@ package org.apache.flume.sink; import java.util.List; +import java.util.Locale; import java.util.Map; import org.apache.flume.Context; @@ -65,7 +66,7 @@ public static SinkProcessor getProcessor(Context context, SinkProcessorType type = SinkProcessorType.OTHER; String processorClassName = typeStr; try { - type = SinkProcessorType.valueOf(typeStr.toUpperCase()); + type = SinkProcessorType.valueOf(typeStr.toUpperCase(Locale.ENGLISH)); } catch (Exception ex) { logger.warn("Sink Processor type {} is a custom type", typeStr); } @@ -103,7 +104,7 @@ public static SinkProcessor getProcessor(ComponentConfiguration conf, SinkProcessor processor; SinkProcessorType type = SinkProcessorType.DEFAULT; try { - type = SinkProcessorType.valueOf(typeStr.toUpperCase()); + type = SinkProcessorType.valueOf(typeStr.toUpperCase(Locale.ENGLISH)); } catch (Exception ex) { logger.warn("Sink type {} does not exist, using default", typeStr); } diff --git a/flume-ng-core/src/main/java/org/apache/flume/source/DefaultSourceFactory.java b/flume-ng-core/src/main/java/org/apache/flume/source/DefaultSourceFactory.java index 95cf13deae..bb9d3f1693 100644 --- a/flume-ng-core/src/main/java/org/apache/flume/source/DefaultSourceFactory.java +++ b/flume-ng-core/src/main/java/org/apache/flume/source/DefaultSourceFactory.java @@ -18,6 +18,8 @@ */ package org.apache.flume.source; +import java.util.Locale; + import org.apache.flume.FlumeException; import org.apache.flume.Source; import org.apache.flume.SourceFactory; @@ -54,7 +56,7 @@ public Class getClass(String type) String sourceClassName = type; SourceType srcType = SourceType.OTHER; try { - srcType = SourceType.valueOf(type.toUpperCase()); + srcType = SourceType.valueOf(type.toUpperCase(Locale.ENGLISH)); } catch (IllegalArgumentException ex) { logger.debug("Source type {} is a custom type", type); } diff --git a/flume-ng-core/src/main/java/org/apache/flume/source/SpoolDirectorySource.java b/flume-ng-core/src/main/java/org/apache/flume/source/SpoolDirectorySource.java index d0c2e99673..f5be9ae5e9 100644 --- a/flume-ng-core/src/main/java/org/apache/flume/source/SpoolDirectorySource.java +++ b/flume-ng-core/src/main/java/org/apache/flume/source/SpoolDirectorySource.java @@ -32,6 +32,7 @@ import java.io.File; import java.io.IOException; import java.util.List; +import java.util.Locale; import java.util.concurrent.Executors; import java.util.concurrent.ScheduledExecutorService; import java.util.concurrent.TimeUnit; @@ -155,7 +156,7 @@ public synchronized void configure(Context context) { inputCharset = context.getString(INPUT_CHARSET, DEFAULT_INPUT_CHARSET); decodeErrorPolicy = DecodeErrorPolicy.valueOf( context.getString(DECODE_ERROR_POLICY, DEFAULT_DECODE_ERROR_POLICY) - .toUpperCase()); + .toUpperCase(Locale.ENGLISH)); ignorePattern = context.getString(IGNORE_PAT, DEFAULT_IGNORE_PAT); trackerDirPath = context.getString(TRACKER_DIR, DEFAULT_TRACKER_DIR); @@ -165,7 +166,7 @@ public synchronized void configure(Context context) { ".")); consumeOrder = ConsumeOrder.valueOf(context.getString(CONSUME_ORDER, - DEFAULT_CONSUME_ORDER.toString()).toUpperCase()); + DEFAULT_CONSUME_ORDER.toString()).toUpperCase(Locale.ENGLISH)); // "Hack" to support backwards compatibility with previous generation of // spooling directory source, which did not support deserializers diff --git a/flume-ng-core/src/main/java/org/apache/flume/source/SyslogUtils.java b/flume-ng-core/src/main/java/org/apache/flume/source/SyslogUtils.java index 43f8db08e1..50593a7dd5 100644 --- a/flume-ng-core/src/main/java/org/apache/flume/source/SyslogUtils.java +++ b/flume-ng-core/src/main/java/org/apache/flume/source/SyslogUtils.java @@ -133,7 +133,7 @@ public static Set chooseFieldsToKeep(String keepFields) { return null; } - keepFields = keepFields.trim().toLowerCase(); + keepFields = keepFields.trim().toLowerCase(Locale.ENGLISH); if (keepFields.equals("false") || keepFields.equals("none")) { return null; diff --git a/flume-ng-core/src/main/java/org/apache/flume/tools/DirectMemoryUtils.java b/flume-ng-core/src/main/java/org/apache/flume/tools/DirectMemoryUtils.java index b12571b37a..8c2db2c0b7 100644 --- a/flume-ng-core/src/main/java/org/apache/flume/tools/DirectMemoryUtils.java +++ b/flume-ng-core/src/main/java/org/apache/flume/tools/DirectMemoryUtils.java @@ -23,6 +23,7 @@ import java.lang.reflect.Method; import java.nio.ByteBuffer; import java.util.List; +import java.util.Locale; import java.util.concurrent.atomic.AtomicInteger; import org.slf4j.Logger; @@ -82,8 +83,8 @@ public static long getDirectMemorySize() { long multiplier = 1; //for the byte case. for (String s : arguments) { if (s.contains(MAX_DIRECT_MEMORY_PARAM)) { - String memSize = s.toLowerCase() - .replace(MAX_DIRECT_MEMORY_PARAM.toLowerCase(), "").trim(); + String memSize = s.toLowerCase(Locale.ENGLISH) + .replace(MAX_DIRECT_MEMORY_PARAM.toLowerCase(Locale.ENGLISH), "").trim(); if (memSize.contains("k")) { multiplier = 1024; diff --git a/flume-ng-core/src/main/java/org/apache/flume/tools/PlatformDetect.java b/flume-ng-core/src/main/java/org/apache/flume/tools/PlatformDetect.java index 3beec06f59..94bab663ed 100644 --- a/flume-ng-core/src/main/java/org/apache/flume/tools/PlatformDetect.java +++ b/flume-ng-core/src/main/java/org/apache/flume/tools/PlatformDetect.java @@ -17,6 +17,8 @@ */ package org.apache.flume.tools; +import java.util.Locale; + /** * Utilities for platform & operating system detection */ @@ -28,7 +30,7 @@ public class PlatformDetect { */ public static boolean isWindows() { String os = System.getProperty("os.name"); - boolean isWin = (os.toLowerCase().indexOf("win") >= 0); + boolean isWin = (os.toLowerCase(Locale.ENGLISH).indexOf("win") >= 0); return isWin; } diff --git a/flume-ng-embedded-agent/src/main/java/org/apache/flume/agent/embedded/EmbeddedAgentConfiguration.java b/flume-ng-embedded-agent/src/main/java/org/apache/flume/agent/embedded/EmbeddedAgentConfiguration.java index 02182ddb8d..4a49fa0921 100644 --- a/flume-ng-embedded-agent/src/main/java/org/apache/flume/agent/embedded/EmbeddedAgentConfiguration.java +++ b/flume-ng-embedded-agent/src/main/java/org/apache/flume/agent/embedded/EmbeddedAgentConfiguration.java @@ -21,6 +21,7 @@ import java.util.Arrays; import java.util.HashMap; import java.util.HashSet; +import java.util.Locale; import java.util.Map; import java.util.Set; @@ -172,7 +173,7 @@ private static void validate(String name, checkRequired(properties, SINKS); String sinkNames = properties.get(SINKS); for(String sink : sinkNames.split("\\s+")) { - if(DISALLOWED_SINK_NAMES.contains(sink.toLowerCase())) { + if(DISALLOWED_SINK_NAMES.contains(sink.toLowerCase(Locale.ENGLISH))) { throw new FlumeException("Sink name " + sink + " is one of the" + " disallowed sink names: " + DISALLOWED_SINK_NAMES); } diff --git a/flume-ng-node/src/main/java/org/apache/flume/node/Application.java b/flume-ng-node/src/main/java/org/apache/flume/node/Application.java index 832285a6e4..959fa779d8 100644 --- a/flume-ng-node/src/main/java/org/apache/flume/node/Application.java +++ b/flume-ng-node/src/main/java/org/apache/flume/node/Application.java @@ -23,6 +23,7 @@ import java.io.IOException; import java.util.ArrayList; import java.util.List; +import java.util.Locale; import java.util.Map.Entry; import java.util.Properties; import java.util.Set; @@ -202,7 +203,7 @@ private void loadMonitoring() { try { //Is it a known type? klass = MonitoringType.valueOf( - monitorType.toUpperCase()).getMonitorClass(); + monitorType.toUpperCase(Locale.ENGLISH)).getMonitorClass(); } catch (Exception e) { //Not a known type, use FQCN klass = (Class) Class.forName(monitorType); diff --git a/flume-ng-sdk/src/main/java/org/apache/flume/api/RpcClientFactory.java b/flume-ng-sdk/src/main/java/org/apache/flume/api/RpcClientFactory.java index 00d9cf0b30..11bc94c495 100644 --- a/flume-ng-sdk/src/main/java/org/apache/flume/api/RpcClientFactory.java +++ b/flume-ng-sdk/src/main/java/org/apache/flume/api/RpcClientFactory.java @@ -23,6 +23,7 @@ import java.io.FileReader; import java.io.IOException; import java.io.Reader; +import java.util.Locale; import java.util.Properties; import org.apache.flume.FlumeException; @@ -63,7 +64,7 @@ public static RpcClient getInstance(Properties properties) String clientClassType = type; ClientType clientType = null; try{ - clientType = ClientType.valueOf(type.toUpperCase()); + clientType = ClientType.valueOf(type.toUpperCase(Locale.ENGLISH)); } catch (IllegalArgumentException e){ clientType = ClientType.OTHER; } diff --git a/flume-ng-sinks/flume-dataset-sink/src/main/java/org/apache/flume/sink/kite/DatasetSink.java b/flume-ng-sinks/flume-dataset-sink/src/main/java/org/apache/flume/sink/kite/DatasetSink.java index a05d776472..ebcc6174fe 100644 --- a/flume-ng-sinks/flume-dataset-sink/src/main/java/org/apache/flume/sink/kite/DatasetSink.java +++ b/flume-ng-sinks/flume-dataset-sink/src/main/java/org/apache/flume/sink/kite/DatasetSink.java @@ -31,6 +31,7 @@ import java.net.URL; import java.security.PrivilegedExceptionAction; import java.util.List; +import java.util.Locale; import java.util.Map; import java.util.concurrent.ExecutionException; import org.apache.avro.Schema; @@ -123,7 +124,7 @@ public Schema load(String url) throws IOException { InputStream is = null; try { FileSystem fs = FileSystem.get(URI.create(url), conf); - if (url.toLowerCase().startsWith("hdfs:/")) { + if (url.toLowerCase(Locale.ENGLISH).startsWith("hdfs:/")) { is = fs.open(new Path(url)); } else { is = new URL(url).openStream(); diff --git a/flume-ng-sinks/flume-hdfs-sink/src/main/java/org/apache/flume/sink/hdfs/AvroEventSerializer.java b/flume-ng-sinks/flume-hdfs-sink/src/main/java/org/apache/flume/sink/hdfs/AvroEventSerializer.java index 4b8fc78a54..fea62188a9 100644 --- a/flume-ng-sinks/flume-hdfs-sink/src/main/java/org/apache/flume/sink/hdfs/AvroEventSerializer.java +++ b/flume-ng-sinks/flume-hdfs-sink/src/main/java/org/apache/flume/sink/hdfs/AvroEventSerializer.java @@ -24,6 +24,7 @@ import java.net.URL; import java.nio.ByteBuffer; import java.util.HashMap; +import java.util.Locale; import java.util.Map; import org.apache.avro.AvroRuntimeException; import org.apache.avro.Schema; @@ -144,7 +145,7 @@ private void initialize(Event event) throws IOException { private Schema loadFromUrl(String schemaUrl) throws IOException { Configuration conf = new Configuration(); Schema.Parser parser = new Schema.Parser(); - if (schemaUrl.toLowerCase().startsWith("hdfs://")) { + if (schemaUrl.toLowerCase(Locale.ENGLISH).startsWith("hdfs://")) { FileSystem fs = FileSystem.get(conf); FSDataInputStream input = null; try { diff --git a/flume-ng-sources/flume-jms-source/src/main/java/org/apache/flume/source/jms/JMSSource.java b/flume-ng-sources/flume-jms-source/src/main/java/org/apache/flume/source/jms/JMSSource.java index addd97a670..1d7f60f7f9 100644 --- a/flume-ng-sources/flume-jms-source/src/main/java/org/apache/flume/source/jms/JMSSource.java +++ b/flume-ng-sources/flume-jms-source/src/main/java/org/apache/flume/source/jms/JMSSource.java @@ -20,6 +20,7 @@ import java.io.File; import java.io.IOException; import java.util.List; +import java.util.Locale; import java.util.Properties; import javax.jms.ConnectionFactory; @@ -101,11 +102,11 @@ protected void doConfigure(Context context) throws FlumeException { DESTINATION_NAME, "").trim(); String destinationTypeName = context.getString(JMSSourceConfiguration. - DESTINATION_TYPE, "").trim().toUpperCase(); + DESTINATION_TYPE, "").trim().toUpperCase(Locale.ENGLISH); String destinationLocatorName = context.getString(JMSSourceConfiguration. DESTINATION_LOCATOR, JMSSourceConfiguration.DESTINATION_LOCATOR_DEFAULT) - .trim().toUpperCase(); + .trim().toUpperCase(Locale.ENGLISH); messageSelector = context.getString(JMSSourceConfiguration. MESSAGE_SELECTOR, "").trim(); diff --git a/flume-tools/src/main/java/org/apache/flume/tools/FlumeToolType.java b/flume-tools/src/main/java/org/apache/flume/tools/FlumeToolType.java index f886c89784..1bb122dcee 100644 --- a/flume-tools/src/main/java/org/apache/flume/tools/FlumeToolType.java +++ b/flume-tools/src/main/java/org/apache/flume/tools/FlumeToolType.java @@ -18,6 +18,8 @@ */ package org.apache.flume.tools; +import java.util.Locale; + public enum FlumeToolType { FCINTEGRITYTOOL(FileChannelIntegrityTool.class); @@ -33,7 +35,7 @@ public Class getClassInstance() { public static String getNames() { StringBuilder builder = new StringBuilder(); for(FlumeToolType type: values()) { - builder.append(type.name().toLowerCase() + "\n"); + builder.append(type.name().toLowerCase(Locale.ENGLISH) + "\n"); } return builder.toString(); } From 84c526fedbf30f14bf80ac7bc40aa043f400ec98 Mon Sep 17 00:00:00 2001 From: Jarek Jarcec Cecho Date: Sat, 11 Oct 2014 13:05:13 -0700 Subject: [PATCH 155/341] FLUME-2237. Example config in User Guide to use same notation for Agent name (Ashish Paliwal via Jarek Jarcec Cecho) --- flume-ng-doc/sphinx/FlumeUserGuide.rst | 18 +++++++++--------- 1 file changed, 9 insertions(+), 9 deletions(-) diff --git a/flume-ng-doc/sphinx/FlumeUserGuide.rst b/flume-ng-doc/sphinx/FlumeUserGuide.rst index ce52946636..804bc85c35 100644 --- a/flume-ng-doc/sphinx/FlumeUserGuide.rst +++ b/flume-ng-doc/sphinx/FlumeUserGuide.rst @@ -841,9 +841,9 @@ invoked directly. Common values for 'shell' : '/bin/sh -c', '/bin/ksh -c', .. code-block:: properties - agent_foo.sources.tailsource-1.type = exec - agent_foo.sources.tailsource-1.shell = /bin/bash -c - agent_foo.sources.tailsource-1.command = for i in /path/*.txt; do cat $i; done + a1.sources.tailsource-1.type = exec + a1.sources.tailsource-1.shell = /bin/bash -c + a1.sources.tailsource-1.command = for i in /path/*.txt; do cat $i; done JMS Source ~~~~~~~~~~~ @@ -986,13 +986,13 @@ Example for an agent named agent-1: .. code-block:: properties - agent-1.channels = ch-1 - agent-1.sources = src-1 + a1.channels = ch-1 + a1.sources = src-1 - agent-1.sources.src-1.type = spooldir - agent-1.sources.src-1.channels = ch-1 - agent-1.sources.src-1.spoolDir = /var/log/apache/flumeSpool - agent-1.sources.src-1.fileHeader = true + a1.sources.src-1.type = spooldir + a1.sources.src-1.channels = ch-1 + a1.sources.src-1.spoolDir = /var/log/apache/flumeSpool + a1.sources.src-1.fileHeader = true Event Deserializers ''''''''''''''''''' From 8328bccd41077d457cab064541127fc993e97619 Mon Sep 17 00:00:00 2001 From: Hari Shreedharan Date: Tue, 14 Oct 2014 17:28:25 -0700 Subject: [PATCH 156/341] FLUME-2126. Problem in elasticsearch sink when the event body is a complex field (Ashish Paliwal via Hari) --- .../elasticsearch/ContentBuilderUtil.java | 2 +- .../elasticsearch/TestElasticSearchSink.java | 21 +++++++++++++++++++ 2 files changed, 22 insertions(+), 1 deletion(-) diff --git a/flume-ng-sinks/flume-ng-elasticsearch-sink/src/main/java/org/apache/flume/sink/elasticsearch/ContentBuilderUtil.java b/flume-ng-sinks/flume-ng-elasticsearch-sink/src/main/java/org/apache/flume/sink/elasticsearch/ContentBuilderUtil.java index bf7c57c7d7..70d0b8689a 100644 --- a/flume-ng-sinks/flume-ng-elasticsearch-sink/src/main/java/org/apache/flume/sink/elasticsearch/ContentBuilderUtil.java +++ b/flume-ng-sinks/flume-ng-elasticsearch-sink/src/main/java/org/apache/flume/sink/elasticsearch/ContentBuilderUtil.java @@ -61,7 +61,7 @@ public static void addComplexField(XContentBuilder builder, String fieldName, parser = XContentFactory.xContent(contentType).createParser(data); parser.nextToken(); tmp.copyCurrentStructure(parser); - builder.field(fieldName, tmp); + builder.field(fieldName, tmp.string()); } catch (JsonParseException ex) { // If we get an exception here the most likely cause is nested JSON that // can't be figured out in the body. At this point just push it through diff --git a/flume-ng-sinks/flume-ng-elasticsearch-sink/src/test/java/org/apache/flume/sink/elasticsearch/TestElasticSearchSink.java b/flume-ng-sinks/flume-ng-elasticsearch-sink/src/test/java/org/apache/flume/sink/elasticsearch/TestElasticSearchSink.java index 15546c1fa8..3e11726893 100644 --- a/flume-ng-sinks/flume-ng-elasticsearch-sink/src/test/java/org/apache/flume/sink/elasticsearch/TestElasticSearchSink.java +++ b/flume-ng-sinks/flume-ng-elasticsearch-sink/src/test/java/org/apache/flume/sink/elasticsearch/TestElasticSearchSink.java @@ -93,6 +93,27 @@ public void shouldIndexOneEvent() throws Exception { assertBodyQuery(1, event); } + @Test + public void shouldIndexComplexJsonEvent() throws Exception { + Configurables.configure(fixture, new Context(parameters)); + Channel channel = bindAndStartChannel(fixture); + + Transaction tx = channel.getTransaction(); + tx.begin(); + Event event = EventBuilder.withBody("{\"event\":\"json content\"}".getBytes()); + channel.put(event); + tx.commit(); + tx.close(); + + fixture.process(); + fixture.stop(); + client.admin().indices() + .refresh(Requests.refreshRequest(timestampedIndexName)).actionGet(); + + assertMatchAllQuery(1, event); + assertBodyQuery(1, event); + } + @Test public void shouldIndexFiveEvents() throws Exception { // Make it so we only need to call process once From aa6fb7fbd9273c905a242c045f99a5b114fb3dc0 Mon Sep 17 00:00:00 2001 From: Hari Shreedharan Date: Tue, 14 Oct 2014 18:08:52 -0700 Subject: [PATCH 157/341] FLUME-1710. JSONEvent.getBody should not return null (Ashish Paliwal via Hari) --- .../main/java/org/apache/flume/event/JSONEvent.java | 4 ++-- .../java/org/apache/flume/event/TestEventBuilder.java | 10 +++++++++- 2 files changed, 11 insertions(+), 3 deletions(-) diff --git a/flume-ng-sdk/src/main/java/org/apache/flume/event/JSONEvent.java b/flume-ng-sdk/src/main/java/org/apache/flume/event/JSONEvent.java index 5c6c0d4070..0ec1678a3a 100644 --- a/flume-ng-sdk/src/main/java/org/apache/flume/event/JSONEvent.java +++ b/flume-ng-sdk/src/main/java/org/apache/flume/event/JSONEvent.java @@ -21,6 +21,7 @@ import java.io.UnsupportedEncodingException; import java.util.Map; import org.apache.flume.Event; +import org.apache.flume.FlumeException; /** * @@ -46,8 +47,7 @@ public byte[] getBody() { try { return body.getBytes(charset); } catch (UnsupportedEncodingException ex) { - //Should never happen - return null; + throw new FlumeException(String.format("%s encoding not supported", charset), ex); } } else { return new byte[0]; diff --git a/flume-ng-sdk/src/test/java/org/apache/flume/event/TestEventBuilder.java b/flume-ng-sdk/src/test/java/org/apache/flume/event/TestEventBuilder.java index 2102770448..ad88ce5c52 100644 --- a/flume-ng-sdk/src/test/java/org/apache/flume/event/TestEventBuilder.java +++ b/flume-ng-sdk/src/test/java/org/apache/flume/event/TestEventBuilder.java @@ -23,6 +23,7 @@ import java.util.Map; import org.apache.flume.Event; +import org.apache.flume.FlumeException; import org.junit.Assert; import org.junit.Test; @@ -56,4 +57,11 @@ public void testHeaders() { Assert.assertEquals("e1 has a one key", "1", e1.getHeaders().get("one")); } -} + @Test (expected = FlumeException.class) + public void testJsonEventUnsupportedEncoding() { + JSONEvent jsonEvent = new JSONEvent(); + jsonEvent.setCharset("dummy"); + jsonEvent.setBody("This is json event".getBytes()); + jsonEvent.getBody(); + } +} \ No newline at end of file From a582c100f5f0b368a6dcc77c2b29138ef4b28840 Mon Sep 17 00:00:00 2001 From: Hari Shreedharan Date: Tue, 14 Oct 2014 18:17:34 -0700 Subject: [PATCH 158/341] FLUME-2482. File Channel tests must disable scheduled checkpoint to avoid a race condition with forced checkpoint. (Johny Rufus via Hari) --- .../flume/channel/file/TestFileChannelRestart.java | 9 +++++++++ 1 file changed, 9 insertions(+) diff --git a/flume-ng-channels/flume-file-channel/src/test/java/org/apache/flume/channel/file/TestFileChannelRestart.java b/flume-ng-channels/flume-file-channel/src/test/java/org/apache/flume/channel/file/TestFileChannelRestart.java index 0c6afc433a..d5fe6fba5c 100644 --- a/flume-ng-channels/flume-file-channel/src/test/java/org/apache/flume/channel/file/TestFileChannelRestart.java +++ b/flume-ng-channels/flume-file-channel/src/test/java/org/apache/flume/channel/file/TestFileChannelRestart.java @@ -67,6 +67,15 @@ public void setup() throws Exception { public void teardown() { super.teardown(); } + + @Override + protected FileChannel createFileChannel(Map overrides) { + // FLUME-2482, making sure scheduled checkpoint never gets called + overrides.put(FileChannelConfiguration.CHECKPOINT_INTERVAL, "6000000"); + return TestUtils.createFileChannel(checkpointDir.getAbsolutePath(), + dataDir, backupDir.getAbsolutePath(), overrides); + } + @Test public void testRestartLogReplayV1() throws Exception { doTestRestart(true, false, false, false); From f979b2683fc48d85806ae7593ee0e393bd812260 Mon Sep 17 00:00:00 2001 From: Hari Shreedharan Date: Wed, 15 Oct 2014 21:29:46 -0700 Subject: [PATCH 159/341] FLUME-2502. Improve Spool Directory Source's performance by not listing files each time. (Prateek Rungta via Hari) --- .../avro/ReliableSpoolingFileEventReader.java | 48 ++++++++++---- .../TestReliableSpoolingFileEventReader.java | 66 ++++++++++++++++++- 2 files changed, 97 insertions(+), 17 deletions(-) diff --git a/flume-ng-core/src/main/java/org/apache/flume/client/avro/ReliableSpoolingFileEventReader.java b/flume-ng-core/src/main/java/org/apache/flume/client/avro/ReliableSpoolingFileEventReader.java index f858b56e53..1833076d16 100644 --- a/flume-ng-core/src/main/java/org/apache/flume/client/avro/ReliableSpoolingFileEventReader.java +++ b/flume-ng-core/src/main/java/org/apache/flume/client/avro/ReliableSpoolingFileEventReader.java @@ -19,6 +19,7 @@ package org.apache.flume.client.avro; +import com.google.common.annotations.VisibleForTesting; import com.google.common.base.Charsets; import com.google.common.base.Optional; import com.google.common.base.Preconditions; @@ -45,6 +46,7 @@ import java.nio.charset.Charset; import java.util.*; import java.util.regex.Pattern; +import java.util.ArrayList; /** *

    A {@link ReliableEventReader} which reads log data from files stored @@ -99,6 +101,10 @@ public class ReliableSpoolingFileEventReader implements ReliableEventReader { private Optional lastFileRead = Optional.absent(); private boolean committed = true; + /** Instance var to Cache directory listing **/ + private Iterator candidateFileIter = null; + private int listFilesCount = 0; + /** * Create a ReliableSpoolingFileEventReader to watch the given directory. */ @@ -195,6 +201,11 @@ private ReliableSpoolingFileEventReader(File spoolDirectory, this.metaFile = new File(trackerDirectory, metaFileName); } + @VisibleForTesting + int getListFilesCount() { + return listFilesCount; + } + /** Return the filename which generated the data from the last successful * {@link #readEvents(int)} call. Returns null if called before any file * contents are read. */ @@ -409,29 +420,38 @@ private void deleteCurrentFile(File fileToDelete) throws IOException { * If two or more files are equally old/young, then the file name with * lower lexicographical value is returned. * If the {@link #consumeOrder} variable is {@link ConsumeOrder#RANDOM} - * then returns any arbitrary file in the directory. + * then cache the directory listing to amortize retreival cost, and return + * any arbitary file from the directory. */ private Optional getNextFile() { - /* Filter to exclude finished or hidden files */ - FileFilter filter = new FileFilter() { - public boolean accept(File candidate) { - String fileName = candidate.getName(); - if ((candidate.isDirectory()) || + List candidateFiles = Collections.emptyList(); + + if (consumeOrder != ConsumeOrder.RANDOM || + candidateFileIter == null || + !candidateFileIter.hasNext()) { + /* Filter to exclude finished or hidden files */ + FileFilter filter = new FileFilter() { + public boolean accept(File candidate) { + String fileName = candidate.getName(); + if ((candidate.isDirectory()) || (fileName.endsWith(completedSuffix)) || (fileName.startsWith(".")) || ignorePattern.matcher(fileName).matches()) { - return false; + return false; + } + return true; } - return true; - } - }; - List candidateFiles = Arrays.asList( - spoolDirectory.listFiles(filter)); - if (candidateFiles.isEmpty()) { // No matching file in spooling directory. + }; + candidateFiles = Arrays.asList(spoolDirectory.listFiles(filter)); + listFilesCount++; + candidateFileIter = candidateFiles.iterator(); + } + + if (!candidateFileIter.hasNext()) { // No matching file in spooling directory. return Optional.absent(); } - File selectedFile = candidateFiles.get(0); // Select the first random file. + File selectedFile = candidateFileIter.next(); if (consumeOrder == ConsumeOrder.RANDOM) { // Selected file is random. return openFile(selectedFile); } else if (consumeOrder == ConsumeOrder.YOUNGEST) { diff --git a/flume-ng-core/src/test/java/org/apache/flume/client/avro/TestReliableSpoolingFileEventReader.java b/flume-ng-core/src/test/java/org/apache/flume/client/avro/TestReliableSpoolingFileEventReader.java index 6a026129f9..a6b247348b 100644 --- a/flume-ng-core/src/test/java/org/apache/flume/client/avro/TestReliableSpoolingFileEventReader.java +++ b/flume-ng-core/src/test/java/org/apache/flume/client/avro/TestReliableSpoolingFileEventReader.java @@ -23,6 +23,7 @@ import com.google.common.io.Files; import junit.framework.Assert; import org.apache.commons.io.FileUtils; +import org.apache.commons.lang.SystemUtils; import org.apache.flume.Event; import org.apache.flume.client.avro.ReliableSpoolingFileEventReader.DeletePolicy; import org.apache.flume.source.SpoolDirectorySourceConfigurationConstants; @@ -38,6 +39,7 @@ import java.io.IOException; import java.util.*; import java.util.Map.Entry; +import java.util.concurrent.*; public class TestReliableSpoolingFileEventReader { @@ -212,7 +214,7 @@ public void testConsumeFileRandomly() throws IOException { FileUtils.write(fileName, "New file created in the end. Shoud be read randomly.\n"); Set actual = Sets.newHashSet(); - readEventsForFilesInDir(WORK_DIR, reader, actual); + readEventsForFilesInDir(WORK_DIR, reader, actual); Set expected = Sets.newHashSet(); createExpectedFromFilesInSetup(expected); expected.add(""); @@ -221,6 +223,52 @@ public void testConsumeFileRandomly() throws IOException { Assert.assertEquals(expected, actual); } + @Test + public void testConsumeFileRandomlyNewFile() throws Exception { + // Atomic moves are not supported in Windows. + if (SystemUtils.IS_OS_WINDOWS) { + return; + } + final ReliableEventReader reader + = new ReliableSpoolingFileEventReader.Builder() + .spoolDirectory(WORK_DIR) + .consumeOrder(ConsumeOrder.RANDOM) + .build(); + File fileName = new File(WORK_DIR, "new-file"); + FileUtils.write(fileName, + "New file created in the end. Shoud be read randomly.\n"); + Set expected = Sets.newHashSet(); + File tempDir = Files.createTempDir(); + File tempFile = new File(tempDir, "t"); + File finalFile = new File(WORK_DIR, "t-file"); + FileUtils.write(tempFile, "Last file"); + final Set actual = Sets.newHashSet(); + ExecutorService executor = Executors.newSingleThreadExecutor(); + final Semaphore semaphore = new Semaphore(0); + Future wait = executor.submit( + new Callable() { + @Override + public Void call() throws Exception { + readEventsForFilesInDir(WORK_DIR, reader, actual, semaphore); + return null; + } + } + ); + semaphore.acquire(); + tempFile.renameTo(finalFile); + wait.get(); + finalFile.delete(); + FileUtils.deleteQuietly(tempDir); + createExpectedFromFilesInSetup(expected); + expected.add(""); + expected.add( + "New file created in the end. Shoud be read randomly."); + expected.add("Last file"); + Assert.assertEquals(2, ((ReliableSpoolingFileEventReader)reader) + .getListFilesCount()); + Assert.assertEquals(expected, actual); + } + @Test public void testConsumeFileOldest() throws IOException, InterruptedException { @@ -414,17 +462,29 @@ private void templateTestForLargeNumberOfFiles(ConsumeOrder order, deleteDir(dir); } } + + private void readEventsForFilesInDir(File dir, ReliableEventReader reader, + Collection actual) throws IOException { + readEventsForFilesInDir(dir, reader, actual, null); + } /* Read events, one for each file in the given directory. */ private void readEventsForFilesInDir(File dir, ReliableEventReader reader, - Collection actual) throws IOException { + Collection actual, Semaphore semaphore) throws IOException { List events; for (int i=0; i < listFiles(dir).size(); i++) { events = reader.readEvents(10); - for (Event e: events) { + for (Event e : events) { actual.add(new String(e.getBody())); } reader.commit(); + try { + if (semaphore != null) { + semaphore.release(); + } + } catch (Exception ex) { + throw new IOException(ex); + } } } /* Create expected results out of the files created in the setup method. */ From 622faa6f8812cd751dfad22deaa21bf0aa613111 Mon Sep 17 00:00:00 2001 From: Hari Shreedharan Date: Wed, 15 Oct 2014 22:20:34 -0700 Subject: [PATCH 160/341] FLUME-2499. Include Kafka Message Key in Event Headers. (Ricky Saltzer via Hari) --- .../flume/source/kafka/KafkaSource.java | 67 +++++++++++-------- .../source/kafka/KafkaSourceConstants.java | 1 + 2 files changed, 41 insertions(+), 27 deletions(-) diff --git a/flume-ng-sources/flume-kafka-source/src/main/java/org/apache/flume/source/kafka/KafkaSource.java b/flume-ng-sources/flume-kafka-source/src/main/java/org/apache/flume/source/kafka/KafkaSource.java index 9d77b470e4..7bc03da6c0 100644 --- a/flume-ng-sources/flume-kafka-source/src/main/java/org/apache/flume/source/kafka/KafkaSource.java +++ b/flume-ng-sources/flume-kafka-source/src/main/java/org/apache/flume/source/kafka/KafkaSource.java @@ -27,6 +27,7 @@ import kafka.consumer.KafkaStream; import kafka.javaapi.consumer.ConsumerConnector; +import kafka.message.MessageAndMetadata; import org.apache.flume.*; import org.apache.flume.conf.Configurable; import org.apache.flume.conf.ConfigurationException; @@ -38,23 +39,30 @@ /** - * A Source for Kafka which reads messages from kafka. - * I use this in company production environment and its performance is good. - * Over 100k messages per second can be read from kafka in one source.

    - * kafka.zookeeper.connect: the zookeeper ip kafka use.

    - * kafka.group.id: the groupid of consumer group.

    - * topic: the topic to read from kafka.

    - * maxBatchSize - maximum number of messages written to Channel in one batch - * maxBatchDurationMillis - maximum time before a batch (of any size) - * will be written to Channel - * kafka.auto.commit.enable - if true, commit automatically every time period. - * if false, commit on each batch. - * kafka.consumer.timeout.ms - polling interval for new data for batch. - * Low value means more CPU usage. - * High value means the time.upper.limit may be missed. + * A Source for Kafka which reads messages from a kafka topic. * - * Any property starting with "kafka" will be passed to the kafka consumer - * So you can use any configuration supported by Kafka 0.8.1.1 + * zookeeperConnect: Kafka's zookeeper connection string. + * Required + *

    + * groupId: the group ID of consumer group. Required + *

    + * topic: the topic to consume messages from. Required + *

    + * maxBatchSize: Maximum number of messages written to Channel in one + * batch. Default: 1000 + *

    + * maxBatchDurationMillis: Maximum number of milliseconds before a + * batch (of any size) will be written to a channel. Default: 1000 + *

    + * kafka.auto.commit.enable: If true, commit automatically every time + * period. if false, commit on each batch. Default: false + *

    + * kafka.consumer.timeout.ms: Polling interval for new data for batch. + * Low value means more CPU usage. High value means the time.upper.limit may be + * missed. Default: 10 + * + * Any property starting with "kafka" will be passed to the kafka consumer So + * you can use any configuration supported by Kafka 0.8.1.1 */ public class KafkaSource extends AbstractSource implements Configurable, PollableSource { @@ -72,7 +80,8 @@ public class KafkaSource extends AbstractSource public Status process() throws EventDeliveryException { - byte[] bytes; + byte[] kafkaMessage; + byte[] kafkaKey; Event event; Map headers; long batchStartTime = System.currentTimeMillis(); @@ -84,16 +93,20 @@ public Status process() throws EventDeliveryException { iterStatus = hasNext(); if (iterStatus) { // get next message - bytes = it.next().message(); + MessageAndMetadata messageAndMetadata = it.next(); + kafkaMessage = messageAndMetadata.message(); + kafkaKey = messageAndMetadata.key(); + // Add headers to event (topic, timestamp, and key) headers = new HashMap(); headers.put(KafkaSourceConstants.TIMESTAMP, String.valueOf(System.currentTimeMillis())); - headers.put(KafkaSourceConstants.TOPIC,topic); + headers.put(KafkaSourceConstants.TOPIC, topic); + headers.put(KafkaSourceConstants.KEY, new String(kafkaKey)); if (log.isDebugEnabled()) { - log.debug("Message: {}", new String(bytes)); + log.debug("Message: {}", new String(kafkaMessage)); } - event = EventBuilder.withBody(bytes, headers); + event = EventBuilder.withBody(kafkaMessage, headers); eventList.add(event); } if (log.isDebugEnabled()) { @@ -132,12 +145,12 @@ public Status process() throws EventDeliveryException { * We configure the source and generate properties for the Kafka Consumer * * Kafka Consumer properties are generated as follows: - * 1. Generate a properties object with some static defaults that - * can be overridden by Source configuration - * 2. We add the configuration users added for Kafka (parameters starting - * with kafka. and must be valid Kafka Consumer properties - * 3. We add the source documented parameters which can override other - * properties + * + * 1. Generate a properties object with some static defaults that can be + * overridden by Source configuration 2. We add the configuration users added + * for Kafka (parameters starting with kafka. and must be valid Kafka Consumer + * properties 3. We add the source documented parameters which can override + * other properties * * @param context */ diff --git a/flume-ng-sources/flume-kafka-source/src/main/java/org/apache/flume/source/kafka/KafkaSourceConstants.java b/flume-ng-sources/flume-kafka-source/src/main/java/org/apache/flume/source/kafka/KafkaSourceConstants.java index 7390618189..911012cefc 100644 --- a/flume-ng-sources/flume-kafka-source/src/main/java/org/apache/flume/source/kafka/KafkaSourceConstants.java +++ b/flume-ng-sources/flume-kafka-source/src/main/java/org/apache/flume/source/kafka/KafkaSourceConstants.java @@ -18,6 +18,7 @@ public class KafkaSourceConstants { public static final String TOPIC = "topic"; + public static final String KEY = "key"; public static final String TIMESTAMP = "timestamp"; public static final String BATCH_SIZE = "batchSize"; public static final String BATCH_DURATION_MS = "batchDurationMillis"; From f99adaabcf9d6c6b8f4c8fe5895fe478c8307694 Mon Sep 17 00:00:00 2001 From: Hari Shreedharan Date: Wed, 15 Oct 2014 23:16:15 -0700 Subject: [PATCH 161/341] FLUME-2486. TestExecSource fails on some environments (Santiago M. Mola via Hari) --- .../org/apache/flume/source/TestExecSource.java | 14 +++++--------- 1 file changed, 5 insertions(+), 9 deletions(-) diff --git a/flume-ng-core/src/test/java/org/apache/flume/source/TestExecSource.java b/flume-ng-core/src/test/java/org/apache/flume/source/TestExecSource.java index 289c2d2ce3..afa93bfcf0 100644 --- a/flume-ng-core/src/test/java/org/apache/flume/source/TestExecSource.java +++ b/flume-ng-core/src/test/java/org/apache/flume/source/TestExecSource.java @@ -338,28 +338,24 @@ private void runTestShellCmdHelper(String shell, String command, String[] expect context.put("command", command); Configurables.configure(source, context); source.start(); - File outputFile = File.createTempFile("flumeExecSourceTest_", ""); - FileOutputStream outputStream = new FileOutputStream(outputFile); - if(SystemUtils.IS_OS_WINDOWS) - Thread.sleep(2500); + // Some commands might take longer to complete, specially on Windows + // or on slow environments (e.g. Travis CI). + Thread.sleep(2500); Transaction transaction = channel.getTransaction(); transaction.begin(); try { + List output = Lists.newArrayList(); Event event; while ((event = channel.take()) != null) { - outputStream.write(event.getBody()); - outputStream.write('\n'); + output.add(new String(event.getBody(), Charset.defaultCharset())); } - outputStream.close(); transaction.commit(); - List output = Files.readLines(outputFile, Charset.defaultCharset()); // System.out.println("command : " + command); // System.out.println("output : "); // for( String line : output ) // System.out.println(line); Assert.assertArrayEquals(expectedOutput, output.toArray(new String[]{})); } finally { - FileUtils.forceDelete(outputFile); transaction.close(); source.stop(); } From 375143493c79d8c1a05723ea16ccd25374fe7294 Mon Sep 17 00:00:00 2001 From: Hari Shreedharan Date: Wed, 22 Oct 2014 18:27:36 -0700 Subject: [PATCH 162/341] FLUME-2511. Allow configuration of enabled protocols in Avro source and RpcClient. (Tom White via Hari) --- .../org/apache/flume/source/AvroSource.java | 14 ++++++++ flume-ng-doc/sphinx/FlumeUserGuide.rst | 36 ++++++++++--------- .../apache/flume/api/NettyAvroRpcClient.java | 27 +++++++++++--- .../api/RpcClientConfigurationConstants.java | 1 + 4 files changed, 57 insertions(+), 21 deletions(-) diff --git a/flume-ng-core/src/main/java/org/apache/flume/source/AvroSource.java b/flume-ng-core/src/main/java/org/apache/flume/source/AvroSource.java index 3eef6870f5..59ee43a8e1 100644 --- a/flume-ng-core/src/main/java/org/apache/flume/source/AvroSource.java +++ b/flume-ng-core/src/main/java/org/apache/flume/source/AvroSource.java @@ -26,6 +26,7 @@ import java.security.KeyStore; import java.security.Security; import java.util.ArrayList; +import java.util.Arrays; import java.util.HashMap; import java.util.List; import java.util.Map; @@ -141,12 +142,14 @@ public class AvroSource extends AbstractSource implements EventDrivenSource, private static final String KEYSTORE_KEY = "keystore"; private static final String KEYSTORE_PASSWORD_KEY = "keystore-password"; private static final String KEYSTORE_TYPE_KEY = "keystore-type"; + private static final String EXCLUDE_PROTOCOLS = "exclude-protocols"; private int port; private String bindAddress; private String compressionType; private String keystore; private String keystorePassword; private String keystoreType; + private List excludeProtocols; private boolean enableSsl = false; private boolean enableIpFilter; private String patternRuleConfigDefinition; @@ -178,6 +181,8 @@ public void configure(Context context) { keystore = context.getString(KEYSTORE_KEY); keystorePassword = context.getString(KEYSTORE_PASSWORD_KEY); keystoreType = context.getString(KEYSTORE_TYPE_KEY, "JKS"); + excludeProtocols = Arrays.asList( + context.getString(EXCLUDE_PROTOCOLS, "SSLv2Hello SSLv3").split(" ")); if (enableSsl) { Preconditions.checkNotNull(keystore, @@ -501,6 +506,15 @@ public ChannelPipeline getPipeline() throws Exception { if (enableSsl) { SSLEngine sslEngine = createServerSSLContext().createSSLEngine(); sslEngine.setUseClientMode(false); + List enabledProtocols = new ArrayList(); + for (String protocol : sslEngine.getEnabledProtocols()) { + if (!excludeProtocols.contains(protocol)) { + enabledProtocols.add(protocol); + } + } + sslEngine.setEnabledProtocols(enabledProtocols.toArray(new String[0])); + logger.info("SSLEngine protocols enabled: " + + Arrays.asList(sslEngine.getEnabledProtocols())); // addFirst() will make SSL handling the first stage of decoding // and the last stage of encoding this must be added after // adding compression handling above diff --git a/flume-ng-doc/sphinx/FlumeUserGuide.rst b/flume-ng-doc/sphinx/FlumeUserGuide.rst index 804bc85c35..e3aedeb193 100644 --- a/flume-ng-doc/sphinx/FlumeUserGuide.rst +++ b/flume-ng-doc/sphinx/FlumeUserGuide.rst @@ -688,26 +688,27 @@ When paired with the built-in Avro Sink on another (previous hop) Flume agent, it can create tiered collection topologies. Required properties are in **bold**. -================== =========== =================================================== -Property Name Default Description -================== =========== =================================================== +================== ================ =================================================== +Property Name Default Description +================== ================ =================================================== **channels** -- -**type** -- The component type name, needs to be ``avro`` -**bind** -- hostname or IP address to listen on -**port** -- Port # to bind to -threads -- Maximum number of worker threads to spawn +**type** -- The component type name, needs to be ``avro`` +**bind** -- hostname or IP address to listen on +**port** -- Port # to bind to +threads -- Maximum number of worker threads to spawn selector.type selector.* -interceptors -- Space-separated list of interceptors +interceptors -- Space-separated list of interceptors interceptors.* -compression-type none This can be "none" or "deflate". The compression-type must match the compression-type of matching AvroSource -ssl false Set this to true to enable SSL encryption. You must also specify a "keystore" and a "keystore-password". -keystore -- This is the path to a Java keystore file. Required for SSL. -keystore-password -- The password for the Java keystore. Required for SSL. -keystore-type JKS The type of the Java keystore. This can be "JKS" or "PKCS12". -ipFilter false Set this to true to enable ipFiltering for netty -ipFilter.rules -- Define N netty ipFilter pattern rules with this config. -================== =========== =================================================== +compression-type none This can be "none" or "deflate". The compression-type must match the compression-type of matching AvroSource +ssl false Set this to true to enable SSL encryption. You must also specify a "keystore" and a "keystore-password". +keystore -- This is the path to a Java keystore file. Required for SSL. +keystore-password -- The password for the Java keystore. Required for SSL. +keystore-type JKS The type of the Java keystore. This can be "JKS" or "PKCS12". +exclude-protocols SSLv2Hello SSLv3 Space-separated list of SSL/TLS protocols to exclude +ipFilter false Set this to true to enable ipFiltering for netty +ipFilter.rules -- Define N netty ipFilter pattern rules with this config. +================== ================ =================================================== Example for agent named a1: @@ -1720,7 +1721,7 @@ batches of the configured batch size. Required properties are in **bold**. ========================== ===================================================== =========================================================================================== -Property Name Default Description +Property Name Default Description ========================== ===================================================== =========================================================================================== **channel** -- **type** -- The component type name, needs to be ``avro``. @@ -1737,6 +1738,7 @@ trust-all-certs false truststore -- The path to a custom Java truststore file. Flume uses the certificate authority information in this file to determine whether the remote Avro Source's SSL authentication credentials should be trusted. If not specified, the default Java JSSE certificate authority files (typically "jssecacerts" or "cacerts" in the Oracle JRE) will be used. truststore-password -- The password for the specified truststore. truststore-type JKS The type of the Java truststore. This can be "JKS" or other supported Java truststore type. +exclude-protocols SSLv2Hello SSLv3 Space-separated list of SSL/TLS protocols to exclude maxIoWorkers 2 * the number of available processors in the machine The maximum number of I/O worker threads. This is configured on the NettyAvroRpcClient NioClientSocketChannelFactory. ========================== ===================================================== =========================================================================================== diff --git a/flume-ng-sdk/src/main/java/org/apache/flume/api/NettyAvroRpcClient.java b/flume-ng-sdk/src/main/java/org/apache/flume/api/NettyAvroRpcClient.java index a2eb264fc1..ad9b58020d 100644 --- a/flume-ng-sdk/src/main/java/org/apache/flume/api/NettyAvroRpcClient.java +++ b/flume-ng-sdk/src/main/java/org/apache/flume/api/NettyAvroRpcClient.java @@ -25,6 +25,8 @@ import java.nio.ByteBuffer; import java.security.KeyStore; import java.security.cert.X509Certificate; +import java.util.ArrayList; +import java.util.Arrays; import java.util.HashMap; import java.util.Iterator; import java.util.LinkedList; @@ -94,6 +96,7 @@ public class NettyAvroRpcClient extends AbstractRpcClient private String truststore; private String truststorePassword; private String truststoreType; + private List excludeProtocols; private Transceiver transceiver; private AvroSourceProtocol.Callback avroClient; @@ -144,12 +147,13 @@ private void connect(long timeout, TimeUnit tu) throws FlumeException { bossExecutor, workerExecutor, enableDeflateCompression, enableSsl, trustAllCerts, compressionLevel, truststore, truststorePassword, truststoreType, - maxIoWorkers); + excludeProtocols, maxIoWorkers); } else { socketChannelFactory = new SSLCompressionChannelFactory( bossExecutor, workerExecutor, enableDeflateCompression, enableSsl, trustAllCerts, - compressionLevel, truststore, truststorePassword, truststoreType); + compressionLevel, truststore, truststorePassword, truststoreType, + excludeProtocols); } } else { if (maxIoWorkers >= 1) { @@ -603,6 +607,9 @@ public synchronized void configure(Properties properties) RpcClientConfigurationConstants.CONFIG_TRUSTSTORE_PASSWORD); truststoreType = properties.getProperty( RpcClientConfigurationConstants.CONFIG_TRUSTSTORE_TYPE, "JKS"); + excludeProtocols = Arrays.asList(properties.getProperty( + RpcClientConfigurationConstants.CONFIG_EXCLUDE_PROTOCOLS, "SSLv2Hello SSLv3") + .split(" ")); String maxIoWorkersStr = properties.getProperty( RpcClientConfigurationConstants.MAX_IO_WORKERS); @@ -669,11 +676,12 @@ private static class SSLCompressionChannelFactory extends NioClientSocketChannel private final String truststore; private final String truststorePassword; private final String truststoreType; + private final List excludeProtocols; public SSLCompressionChannelFactory(Executor bossExecutor, Executor workerExecutor, boolean enableCompression, boolean enableSsl, boolean trustAllCerts, int compressionLevel, String truststore, String truststorePassword, - String truststoreType) { + String truststoreType, List excludeProtocols) { super(bossExecutor, workerExecutor); this.enableCompression = enableCompression; this.enableSsl = enableSsl; @@ -682,12 +690,13 @@ public SSLCompressionChannelFactory(Executor bossExecutor, Executor workerExecut this.truststore = truststore; this.truststorePassword = truststorePassword; this.truststoreType = truststoreType; + this.excludeProtocols = excludeProtocols; } public SSLCompressionChannelFactory(Executor bossExecutor, Executor workerExecutor, boolean enableCompression, boolean enableSsl, boolean trustAllCerts, int compressionLevel, String truststore, String truststorePassword, - String truststoreType, int maxIOWorkers) { + String truststoreType, List excludeProtocols, int maxIOWorkers) { super(bossExecutor, workerExecutor, maxIOWorkers); this.enableCompression = enableCompression; this.enableSsl = enableSsl; @@ -696,6 +705,7 @@ public SSLCompressionChannelFactory(Executor bossExecutor, Executor workerExecut this.truststore = truststore; this.truststorePassword = truststorePassword; this.truststoreType = truststoreType; + this.excludeProtocols = excludeProtocols; } @Override @@ -735,6 +745,15 @@ public SocketChannel newChannel(ChannelPipeline pipeline) { sslContext.init(null, managers, null); SSLEngine sslEngine = sslContext.createSSLEngine(); sslEngine.setUseClientMode(true); + List enabledProtocols = new ArrayList(); + for (String protocol : sslEngine.getEnabledProtocols()) { + if (!excludeProtocols.contains(protocol)) { + enabledProtocols.add(protocol); + } + } + sslEngine.setEnabledProtocols(enabledProtocols.toArray(new String[0])); + logger.info("SSLEngine protocols enabled: " + + Arrays.asList(sslEngine.getEnabledProtocols())); // addFirst() will make SSL handling the first stage of decoding // and the last stage of encoding this must be added after // adding compression handling above diff --git a/flume-ng-sdk/src/main/java/org/apache/flume/api/RpcClientConfigurationConstants.java b/flume-ng-sdk/src/main/java/org/apache/flume/api/RpcClientConfigurationConstants.java index 136c50496a..33a2330b01 100644 --- a/flume-ng-sdk/src/main/java/org/apache/flume/api/RpcClientConfigurationConstants.java +++ b/flume-ng-sdk/src/main/java/org/apache/flume/api/RpcClientConfigurationConstants.java @@ -143,6 +143,7 @@ public final class RpcClientConfigurationConstants { public static final String CONFIG_TRUSTSTORE = "truststore"; public static final String CONFIG_TRUSTSTORE_PASSWORD = "truststore-password"; public static final String CONFIG_TRUSTSTORE_TYPE = "truststore-type"; + public static final String CONFIG_EXCLUDE_PROTOCOLS = "exclude-protocols"; /** * Configuration constants for the NettyAvroRpcClient From 77d56e95ead7a04499aa83d1a78fcfbd957b20c7 Mon Sep 17 00:00:00 2001 From: Hari Shreedharan Date: Mon, 27 Oct 2014 13:41:25 -0700 Subject: [PATCH 163/341] FLUME-2517. Cache SimpleDataFormat objects in bucketwriter for better performance. (Pal Konyves via Hari) --- .../flume/formatter/output/BucketPath.java | 26 ++++++++++++++++++- 1 file changed, 25 insertions(+), 1 deletion(-) diff --git a/flume-ng-core/src/main/java/org/apache/flume/formatter/output/BucketPath.java b/flume-ng-core/src/main/java/org/apache/flume/formatter/output/BucketPath.java index bef4b1f436..fe38d2c130 100644 --- a/flume-ng-core/src/main/java/org/apache/flume/formatter/output/BucketPath.java +++ b/flume-ng-core/src/main/java/org/apache/flume/formatter/output/BucketPath.java @@ -24,10 +24,12 @@ import java.util.HashMap; import java.util.Map; import java.util.TimeZone; +import java.util.concurrent.ConcurrentHashMap; import java.util.regex.Matcher; import java.util.regex.Pattern; import com.google.common.annotations.VisibleForTesting; + import org.apache.flume.Clock; import org.apache.flume.SystemClock; import org.apache.flume.tools.TimestampRoundDownUtil; @@ -184,6 +186,28 @@ public static String replaceShorthand(char c, Map headers, return replaceShorthand(c, headers, timeZone, needRounding, unit, roundDown, false, ts); } + + protected static final ThreadLocal> simpleDateFormatCache = new ThreadLocal>() { + + @Override + protected HashMap initialValue() { + return new HashMap(); + } + }; + + protected static SimpleDateFormat getSimpleDateFormat(String string) { + HashMap localCache = simpleDateFormatCache.get(); + + SimpleDateFormat simpleDateFormat = localCache.get(string); + if (simpleDateFormat == null) { + simpleDateFormat = new SimpleDateFormat(string); + localCache.put(string, simpleDateFormat); + simpleDateFormatCache.set(localCache); + } + + return simpleDateFormat; + } + /** * Not intended as a public API @@ -287,7 +311,7 @@ protected static String replaceShorthand(char c, Map headers, return ""; } - SimpleDateFormat format = new SimpleDateFormat(formatString); + SimpleDateFormat format = getSimpleDateFormat(formatString); if (timeZone != null) { format.setTimeZone(timeZone); } From ea2fcdaa5132bd4fe01bc00a7da2dddb29756b02 Mon Sep 17 00:00:00 2001 From: Jarek Jarcec Cecho Date: Mon, 27 Oct 2014 16:27:00 -0700 Subject: [PATCH 164/341] FLUME-2520: HTTP Source should be able to block a prefixed set of protocols. (Hari Shreedharan via Jarek Jarcec Cecho) --- .../apache/flume/source/http/HTTPSource.java | 28 +++++- .../flume/source/http/TestHTTPSource.java | 98 +++++++++++++++++-- 2 files changed, 116 insertions(+), 10 deletions(-) diff --git a/flume-ng-core/src/main/java/org/apache/flume/source/http/HTTPSource.java b/flume-ng-core/src/main/java/org/apache/flume/source/http/HTTPSource.java index 115b34f584..4b2717c4b6 100644 --- a/flume-ng-core/src/main/java/org/apache/flume/source/http/HTTPSource.java +++ b/flume-ng-core/src/main/java/org/apache/flume/source/http/HTTPSource.java @@ -35,13 +35,13 @@ import org.slf4j.Logger; import org.slf4j.LoggerFactory; +import javax.net.ssl.SSLServerSocket; import javax.servlet.http.HttpServlet; import javax.servlet.http.HttpServletRequest; import javax.servlet.http.HttpServletResponse; import java.io.IOException; -import java.util.Collections; -import java.util.List; -import java.util.Map; +import java.net.ServerSocket; +import java.util.*; /** * A source which accepts Flume Events by HTTP POST and GET. GET should be used @@ -172,7 +172,7 @@ public void start() { if (sslEnabled) { - SslSocketConnector sslSocketConnector = new SslSocketConnector(); + SslSocketConnector sslSocketConnector = new HTTPSourceSocketConnector(); sslSocketConnector.setKeystore(keyStorePath); sslSocketConnector.setKeyPassword(keyStorePassword); sslSocketConnector.setReuseAddress(true); @@ -271,4 +271,24 @@ public void doGet(HttpServletRequest request, HttpServletResponse response) doPost(request, response); } } + + private static class HTTPSourceSocketConnector extends SslSocketConnector { + + @Override + public ServerSocket newServerSocket(String host, int port, + int backlog) throws IOException { + SSLServerSocket socket = (SSLServerSocket)super.newServerSocket(host, + port, backlog); + String[] protocols = socket.getEnabledProtocols(); + List newProtocols = new ArrayList(protocols.length); + for(String protocol: protocols) { + if (!(protocol.equals("SSLv3") || protocol.equals("SSLv2Hello"))) { + newProtocols.add(protocol); + } + } + socket.setEnabledProtocols( + newProtocols.toArray(new String[newProtocols.size()])); + return socket; + } + } } diff --git a/flume-ng-core/src/test/java/org/apache/flume/source/http/TestHTTPSource.java b/flume-ng-core/src/test/java/org/apache/flume/source/http/TestHTTPSource.java index 797eebb736..9ba0a8c491 100644 --- a/flume-ng-core/src/test/java/org/apache/flume/source/http/TestHTTPSource.java +++ b/flume-ng-core/src/test/java/org/apache/flume/source/http/TestHTTPSource.java @@ -45,9 +45,7 @@ import javax.servlet.http.HttpServletResponse; import java.io.IOException; import java.lang.reflect.Type; -import java.net.HttpURLConnection; -import java.net.ServerSocket; -import java.net.URL; +import java.net.*; import java.security.SecureRandom; import java.security.cert.CertificateException; import java.util.ArrayList; @@ -315,6 +313,20 @@ private ResultWrapper putWithEncoding(String encoding, int n) @Test public void testHttps() throws Exception { + doTestHttps(null); + } + + @Test (expected = javax.net.ssl.SSLHandshakeException.class) + public void testHttpsSSLv3() throws Exception { + doTestHttps("SSLv3"); + } + + @Test (expected = javax.net.ssl.SSLHandshakeException.class) + public void testHttpsSSLv2Hello() throws Exception { + doTestHttps("SSLv2Hello"); + } + + public void doTestHttps(String protocol) throws Exception { Type listType = new TypeToken>() { }.getType(); List events = Lists.newArrayList(); @@ -367,7 +379,13 @@ public boolean verify(String arg0, SSLSession arg1) { } }; sc.init(null, trustAllCerts, new SecureRandom()); - HttpsURLConnection.setDefaultSSLSocketFactory(sc.getSocketFactory()); + javax.net.ssl.SSLSocketFactory factory = null; + if(protocol != null) { + factory = new TestSSLSocketFactory(sc.getSocketFactory(), protocol); + } else { + factory = sc.getSocketFactory(); + } + HttpsURLConnection.setDefaultSSLSocketFactory(factory); HttpsURLConnection.setDefaultHostnameVerifier( SSLSocketFactory.ALLOW_ALL_HOSTNAME_VERIFIER); URL sslUrl = new URL("https://0.0.0.0:" + sslPort); @@ -390,8 +408,6 @@ public boolean verify(String arg0, SSLSession arg1) { transaction.commit(); transaction.close(); - } catch (Exception exception) { - Assert.fail("Exception not expected"); } finally { httpsURLConnection.disconnect(); } @@ -473,4 +489,74 @@ public ResultWrapper(HttpResponse resp, List events){ this.events = events; } } + + private class TestSSLSocketFactory extends javax.net.ssl.SSLSocketFactory { + + private final javax.net.ssl.SSLSocketFactory socketFactory; + private final String[] protocols; + + TestSSLSocketFactory(javax.net.ssl.SSLSocketFactory factory, String protocol) { + this.socketFactory = factory; + if(protocol.equals("SSLv2Hello")) { + protocols = new String[2]; + protocols[0] = "TLSv1.2"; + protocols[1] = protocol; + } else { + protocols = new String[1]; + protocols[0] = protocol; + } + } + + @Override + public String[] getDefaultCipherSuites() { + return socketFactory.getDefaultCipherSuites(); + } + + @Override + public String[] getSupportedCipherSuites() { + return socketFactory.getSupportedCipherSuites(); + } + + @Override + public Socket createSocket(Socket socket, String s, int i, boolean b) + throws IOException { + SSLSocket sc = (SSLSocket) socketFactory.createSocket(socket, s, i, b); + sc.setEnabledProtocols(protocols); + return sc; + } + + @Override + public Socket createSocket(String s, int i) + throws IOException, UnknownHostException { + SSLSocket sc = (SSLSocket)socketFactory.createSocket(s, i); + sc.setEnabledProtocols(protocols); + return sc; + } + + @Override + public Socket createSocket(String s, int i, InetAddress inetAddress, int i2) + throws IOException, UnknownHostException { + SSLSocket sc = (SSLSocket)socketFactory.createSocket(s, i, inetAddress, + i2); + sc.setEnabledProtocols(protocols); + return sc; + } + + @Override + public Socket createSocket(InetAddress inetAddress, int i) + throws IOException { + SSLSocket sc = (SSLSocket)socketFactory.createSocket(inetAddress, i); + sc.setEnabledProtocols(protocols); + return sc; + } + + @Override + public Socket createSocket(InetAddress inetAddress, int i, + InetAddress inetAddress2, int i2) throws IOException { + SSLSocket sc = (SSLSocket)socketFactory.createSocket(inetAddress, i, + inetAddress2, i2); + sc.setEnabledProtocols(protocols); + return sc; + } + } } From aef02df10a26a6b6911b771a506994f2069857cf Mon Sep 17 00:00:00 2001 From: Jarek Jarcec Cecho Date: Tue, 28 Oct 2014 14:59:23 -0700 Subject: [PATCH 165/341] FLUME-2500: Add a channel that uses Kafka (Hari Shreedharan via Jarek Jarcec Cecho) --- flume-ng-channels/flume-kafka-channel/pom.xml | 54 +++ .../flume/channel/kafka/KafkaChannel.java | 411 +++++++++++++++++ .../kafka/KafkaChannelConfiguration.java | 44 ++ .../flume/channel/kafka/TestKafkaChannel.java | 418 ++++++++++++++++++ .../test/resources/kafka-server.properties | 118 +++++ .../src/test/resources/log4j.properties | 78 ++++ .../src/test/resources/zookeeper.properties | 20 + flume-ng-channels/pom.xml | 1 + flume-ng-sinks/flume-ng-kafka-sink/pom.xml | 11 + .../flume/sink/kafka/util/KafkaConsumer.java | 2 +- .../flume/sink/kafka/util/TestUtil.java | 1 + 11 files changed, 1157 insertions(+), 1 deletion(-) create mode 100644 flume-ng-channels/flume-kafka-channel/pom.xml create mode 100644 flume-ng-channels/flume-kafka-channel/src/main/java/org/apache/flume/channel/kafka/KafkaChannel.java create mode 100644 flume-ng-channels/flume-kafka-channel/src/main/java/org/apache/flume/channel/kafka/KafkaChannelConfiguration.java create mode 100644 flume-ng-channels/flume-kafka-channel/src/test/java/org/apache/flume/channel/kafka/TestKafkaChannel.java create mode 100644 flume-ng-channels/flume-kafka-channel/src/test/resources/kafka-server.properties create mode 100644 flume-ng-channels/flume-kafka-channel/src/test/resources/log4j.properties create mode 100644 flume-ng-channels/flume-kafka-channel/src/test/resources/zookeeper.properties diff --git a/flume-ng-channels/flume-kafka-channel/pom.xml b/flume-ng-channels/flume-kafka-channel/pom.xml new file mode 100644 index 0000000000..2da98b9982 --- /dev/null +++ b/flume-ng-channels/flume-kafka-channel/pom.xml @@ -0,0 +1,54 @@ + + + + + flume-ng-channels + org.apache.flume + 1.6.0-SNAPSHOT + + 4.0.0 + + org.apache.flume.flume-ng-channels + flume-kafka-channel + + + + org.apache.flume + flume-ng-core + + + org.apache.flume + flume-ng-sdk + + + org.apache.kafka + kafka_2.10 + + + org.apache.flume.flume-ng-sinks + flume-ng-kafka-sink + ${project.version} + tests + test + + + + + \ No newline at end of file diff --git a/flume-ng-channels/flume-kafka-channel/src/main/java/org/apache/flume/channel/kafka/KafkaChannel.java b/flume-ng-channels/flume-kafka-channel/src/main/java/org/apache/flume/channel/kafka/KafkaChannel.java new file mode 100644 index 0000000000..d767aac987 --- /dev/null +++ b/flume-ng-channels/flume-kafka-channel/src/main/java/org/apache/flume/channel/kafka/KafkaChannel.java @@ -0,0 +1,411 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, + * software distributed under the License is distributed on an + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + * KIND, either express or implied. See the License for the + * specific language governing permissions and limitations + * under the License. + */ +package org.apache.flume.channel.kafka; + +import com.google.common.annotations.VisibleForTesting; +import com.google.common.base.Optional; +import kafka.consumer.*; +import kafka.javaapi.consumer.ConsumerConnector; +import kafka.javaapi.producer.Producer; +import kafka.producer.KeyedMessage; +import kafka.producer.ProducerConfig; +import org.apache.avro.io.*; +import org.apache.avro.specific.SpecificDatumReader; +import org.apache.avro.specific.SpecificDatumWriter; +import org.apache.flume.*; +import org.apache.flume.channel.BasicChannelSemantics; +import org.apache.flume.channel.BasicTransactionSemantics; +import org.apache.flume.conf.ConfigurationException; + +import static org.apache.flume.channel.kafka.KafkaChannelConfiguration.*; + +import org.apache.flume.event.EventBuilder; +import org.apache.flume.source.avro.AvroFlumeEvent; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +import java.io.*; +import java.nio.ByteBuffer; +import java.util.*; +import java.util.concurrent.atomic.AtomicReference; + +public class KafkaChannel extends BasicChannelSemantics { + + private final static Logger LOGGER = + LoggerFactory.getLogger(KafkaChannel.class); + + + private final Properties kafkaConf = new Properties(); + private Producer producer; + private final String channelUUID = UUID.randomUUID().toString(); + + private AtomicReference topic = new AtomicReference(); + private boolean parseAsFlumeEvent = DEFAULT_PARSE_AS_FLUME_EVENT; + private final Map topicCountMap = + Collections.synchronizedMap(new HashMap()); + + // Track all consumers to close them eventually. + private final List consumers = + Collections.synchronizedList(new LinkedList()); + + /* Each ConsumerConnector commit will commit all partitions owned by it. To + * ensure that each partition is only committed when all events are + * actually done, we will need to keep a ConsumerConnector per thread. + * See Neha's answer here: + * http://grokbase.com/t/kafka/users/13b4gmk2jk/commit-offset-per-topic + * Since only one consumer connector will a partition at any point in time, + * when we commit the partition we would have committed all events to the + * final destination from that partition. + * + * If a new partition gets assigned to this connector, + * my understanding is that all message from the last partition commit will + * get replayed which may cause duplicates -- which is fine as this + * happens only on partition rebalancing which is on failure or new nodes + * coming up, which is rare. + */ + private final ThreadLocal consumerAndIter = new + ThreadLocal() { + @Override + public ConsumerAndIterator initialValue() { + return createConsumerAndIter(); + } + }; + + @Override + public void start() { + try { + LOGGER.info("Starting Kafka Channel: " + getName()); + producer = new Producer(new ProducerConfig(kafkaConf)); + // We always have just one topic being read by one thread + LOGGER.info("Topic = " + topic.get()); + topicCountMap.put(topic.get(), 1); + super.start(); + } catch (Exception e) { + LOGGER.error("Could not start producer"); + throw new FlumeException("Unable to create Kafka Connections. " + + "Check whether Kafka Brokers are up and that the " + + "Flume agent can connect to it.", e); + } + } + + @Override + public void stop() { + for (ConsumerAndIterator c : consumers) { + try { + decommissionConsumerAndIterator(c); + } catch (Exception ex) { + LOGGER.warn("Error while shutting down consumer.", ex); + } + } + producer.close(); + super.stop(); + } + + @Override + protected BasicTransactionSemantics createTransaction() { + return new KafkaTransaction(); + } + + private synchronized ConsumerAndIterator createConsumerAndIter() { + try { + ConsumerConfig consumerConfig = new ConsumerConfig(kafkaConf); + ConsumerConnector consumer = + Consumer.createJavaConsumerConnector(consumerConfig); + Map>> consumerMap = + consumer.createMessageStreams(topicCountMap); + final List> streamList = consumerMap + .get(topic.get()); + KafkaStream stream = streamList.remove(0); + ConsumerAndIterator ret = + new ConsumerAndIterator(consumer, stream.iterator(), channelUUID); + consumers.add(ret); + LOGGER.info("Created new consumer to connect to Kafka"); + return ret; + } catch (Exception e) { + throw new FlumeException("Unable to connect to Kafka", e); + } + } + + Properties getKafkaConf() { + return kafkaConf; + } + + @Override + public void configure(Context ctx) { + String topicStr = ctx.getString(TOPIC); + if (topicStr == null || topicStr.isEmpty()) { + topicStr = DEFAULT_TOPIC; + LOGGER + .info("Topic was not specified. Using " + topicStr + " as the topic."); + } + topic.set(topicStr); + String groupId = ctx.getString(GROUP_ID_FLUME); + if (groupId == null || groupId.isEmpty()) { + groupId = DEFAULT_GROUP_ID; + LOGGER.info( + "Group ID was not specified. Using " + groupId + " as the group id."); + } + String brokerList = ctx.getString(BROKER_LIST_FLUME_KEY); + if (brokerList == null || brokerList.isEmpty()) { + throw new ConfigurationException("Broker List must be specified"); + } + String zkConnect = ctx.getString(ZOOKEEPER_CONNECT_FLUME_KEY); + if (zkConnect == null || zkConnect.isEmpty()) { + throw new ConfigurationException( + "Zookeeper Connection must be specified"); + } + Long timeout = ctx.getLong(TIMEOUT, Long.valueOf(DEFAULT_TIMEOUT)); + kafkaConf.putAll(ctx.getSubProperties(KAFKA_PREFIX)); + kafkaConf.put(GROUP_ID, groupId); + kafkaConf.put(BROKER_LIST_KEY, brokerList); + kafkaConf.put(ZOOKEEPER_CONNECT, zkConnect); + kafkaConf.put(AUTO_COMMIT_ENABLED, String.valueOf(false)); + kafkaConf.put(CONSUMER_TIMEOUT, String.valueOf(timeout)); + kafkaConf.put(REQUIRED_ACKS_KEY, "-1"); + LOGGER.info(kafkaConf.toString()); + parseAsFlumeEvent = + ctx.getBoolean(PARSE_AS_FLUME_EVENT, DEFAULT_PARSE_AS_FLUME_EVENT); + + boolean readSmallest = ctx.getBoolean(READ_SMALLEST_OFFSET, + DEFAULT_READ_SMALLEST_OFFSET); + // If the data is to be parsed as Flume events, we always read the smallest. + // Else, we read the configuration, which by default reads the largest. + if (parseAsFlumeEvent || readSmallest) { + // readSmallest is eval-ed only if parseAsFlumeEvent is false. + // The default is largest, so we don't need to set it explicitly. + kafkaConf.put("auto.offset.reset", "smallest"); + } + + } + + private void decommissionConsumerAndIterator(ConsumerAndIterator c) { + if (c.failedEvents.isEmpty()) { + c.consumer.commitOffsets(); + } + c.failedEvents.clear(); + c.consumer.shutdown(); + } + + // Force a consumer to be initialized. There are many duplicates in + // tests due to rebalancing - making testing tricky. In production, + // this is less of an issue as + // rebalancing would happen only on startup. + @VisibleForTesting + void registerThread() { + consumerAndIter.get(); + } + + private enum TransactionType { + PUT, + TAKE, + NONE + } + + + private class KafkaTransaction extends BasicTransactionSemantics { + + private TransactionType type = TransactionType.NONE; + // For Puts + private Optional tempOutStream = Optional + .absent(); + + // For put transactions, serialize the events and batch them and send it. + private Optional> serializedEvents = Optional.absent(); + // For take transactions, deserialize and hold them till commit goes through + private Optional> events = Optional.absent(); + private Optional> writer = + Optional.absent(); + private Optional> reader = + Optional.absent(); + + // Fine to use null for initial value, Avro will create new ones if this + // is null + private BinaryEncoder encoder = null; + private BinaryDecoder decoder = null; + private final String batchUUID = UUID.randomUUID().toString(); + private boolean eventTaken = false; + + @Override + protected void doPut(Event event) throws InterruptedException { + type = TransactionType.PUT; + if (!serializedEvents.isPresent()) { + serializedEvents = Optional.of(new LinkedList()); + } + + try { + if (!tempOutStream.isPresent()) { + tempOutStream = Optional.of(new ByteArrayOutputStream()); + } + if (!writer.isPresent()) { + writer = Optional.of(new + SpecificDatumWriter(AvroFlumeEvent.class)); + } + tempOutStream.get().reset(); + AvroFlumeEvent e = new AvroFlumeEvent( + toCharSeqMap(event.getHeaders()), ByteBuffer.wrap(event.getBody())); + encoder = EncoderFactory.get() + .directBinaryEncoder(tempOutStream.get(), encoder); + writer.get().write(e, encoder); + // Not really possible to avoid this copy :( + serializedEvents.get().add(tempOutStream.get().toByteArray()); + } catch (Exception e) { + throw new ChannelException("Error while serializing event", e); + } + } + + @SuppressWarnings("unchecked") + @Override + protected Event doTake() throws InterruptedException { + type = TransactionType.TAKE; + try { + if (!(consumerAndIter.get().uuid.equals(channelUUID))) { + LOGGER.info("UUID mismatch, creating new consumer"); + decommissionConsumerAndIterator(consumerAndIter.get()); + consumerAndIter.remove(); + } + } catch (Exception ex) { + LOGGER.warn("Error while shutting down consumer", ex); + } + if (!events.isPresent()) { + events = Optional.of(new LinkedList()); + } + Event e; + if (!consumerAndIter.get().failedEvents.isEmpty()) { + e = consumerAndIter.get().failedEvents.removeFirst(); + } else { + try { + ConsumerIterator it = consumerAndIter.get().iterator; + it.hasNext(); + if (parseAsFlumeEvent) { + ByteArrayInputStream in = + new ByteArrayInputStream(it.next().message()); + decoder = DecoderFactory.get().directBinaryDecoder(in, decoder); + if (!reader.isPresent()) { + reader = Optional.of( + new SpecificDatumReader(AvroFlumeEvent.class)); + } + AvroFlumeEvent event = reader.get().read(null, decoder); + e = EventBuilder.withBody(event.getBody().array(), + toStringMap(event.getHeaders())); + } else { + e = EventBuilder.withBody(it.next().message(), + Collections.EMPTY_MAP); + } + + } catch (ConsumerTimeoutException ex) { + if (LOGGER.isDebugEnabled()) { + LOGGER.debug("Timed out while waiting for data to come from Kafka", + ex); + } + return null; + } catch (Exception ex) { + LOGGER.warn("Error while getting events from Kafka", ex); + throw new ChannelException("Error while getting events from Kafka", + ex); + } + } + eventTaken = true; + events.get().add(e); + return e; + } + + @Override + protected void doCommit() throws InterruptedException { + if (type.equals(TransactionType.NONE)) { + return; + } + if (type.equals(TransactionType.PUT)) { + try { + List> messages = new + ArrayList>(serializedEvents.get() + .size()); + for (byte[] event : serializedEvents.get()) { + messages.add(new KeyedMessage(topic.get(), null, + batchUUID, event)); + } + producer.send(messages); + serializedEvents.get().clear(); + } catch (Exception ex) { + LOGGER.warn("Sending events to Kafka failed", ex); + throw new ChannelException("Commit failed as send to Kafka failed", + ex); + } + } else { + if (consumerAndIter.get().failedEvents.isEmpty() && eventTaken) { + consumerAndIter.get().consumer.commitOffsets(); + } + events.get().clear(); + } + } + + @Override + protected void doRollback() throws InterruptedException { + if (type.equals(TransactionType.NONE)) { + return; + } + if (type.equals(TransactionType.PUT)) { + serializedEvents.get().clear(); + } else { + consumerAndIter.get().failedEvents.addAll(events.get()); + events.get().clear(); + } + } + } + + + private class ConsumerAndIterator { + final ConsumerConnector consumer; + final ConsumerIterator iterator; + final String uuid; + final LinkedList failedEvents = new LinkedList(); + + ConsumerAndIterator(ConsumerConnector consumerConnector, + ConsumerIterator iterator, String uuid) { + this.consumer = consumerConnector; + this.iterator = iterator; + this.uuid = uuid; + } + } + + /** + * Helper function to convert a map of String to a map of CharSequence. + */ + private static Map toCharSeqMap( + Map stringMap) { + Map charSeqMap = + new HashMap(); + for (Map.Entry entry : stringMap.entrySet()) { + charSeqMap.put(entry.getKey(), entry.getValue()); + } + return charSeqMap; + } + + /** + * Helper function to convert a map of CharSequence to a map of String. + */ + private static Map toStringMap( + Map charSeqMap) { + Map stringMap = + new HashMap(); + for (Map.Entry entry : charSeqMap.entrySet()) { + stringMap.put(entry.getKey().toString(), entry.getValue().toString()); + } + return stringMap; + } +} diff --git a/flume-ng-channels/flume-kafka-channel/src/main/java/org/apache/flume/channel/kafka/KafkaChannelConfiguration.java b/flume-ng-channels/flume-kafka-channel/src/main/java/org/apache/flume/channel/kafka/KafkaChannelConfiguration.java new file mode 100644 index 0000000000..9a342efb06 --- /dev/null +++ b/flume-ng-channels/flume-kafka-channel/src/main/java/org/apache/flume/channel/kafka/KafkaChannelConfiguration.java @@ -0,0 +1,44 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, + * software distributed under the License is distributed on an + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + * KIND, either express or implied. See the License for the + * specific language governing permissions and limitations + * under the License. + */ +package org.apache.flume.channel.kafka; + +public class KafkaChannelConfiguration { + + public static final String KAFKA_PREFIX = "kafka."; + public static final String BROKER_LIST_KEY = "metadata.broker.list"; + public static final String REQUIRED_ACKS_KEY = "request.required.acks"; + public static final String BROKER_LIST_FLUME_KEY = "brokerList"; + public static final String TOPIC = "topic"; + public static final String GROUP_ID = "group.id"; + public static final String GROUP_ID_FLUME = "groupId"; + public static final String AUTO_COMMIT_ENABLED = "auto.commit.enable"; + public static final String ZOOKEEPER_CONNECT = "zookeeper.connect"; + public static final String ZOOKEEPER_CONNECT_FLUME_KEY = "zookeeperConnect"; + public static final String DEFAULT_GROUP_ID = "flume"; + public static final String DEFAULT_TOPIC = "flume-channel"; + public static final String TIMEOUT = "timeout"; + public static final String DEFAULT_TIMEOUT = "100"; + public static final String CONSUMER_TIMEOUT = "consumer.timeout.ms"; + + public static final String PARSE_AS_FLUME_EVENT = "parseAsFlumeEvent"; + public static final boolean DEFAULT_PARSE_AS_FLUME_EVENT = true; + + public static final String READ_SMALLEST_OFFSET = "readSmallestOffset"; + public static final boolean DEFAULT_READ_SMALLEST_OFFSET = false; +} diff --git a/flume-ng-channels/flume-kafka-channel/src/test/java/org/apache/flume/channel/kafka/TestKafkaChannel.java b/flume-ng-channels/flume-kafka-channel/src/test/java/org/apache/flume/channel/kafka/TestKafkaChannel.java new file mode 100644 index 0000000000..e665431a00 --- /dev/null +++ b/flume-ng-channels/flume-kafka-channel/src/test/java/org/apache/flume/channel/kafka/TestKafkaChannel.java @@ -0,0 +1,418 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, + * software distributed under the License is distributed on an + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + * KIND, either express or implied. See the License for the + * specific language governing permissions and limitations + * under the License. + */ +package org.apache.flume.channel.kafka; + +import com.google.common.collect.Lists; +import com.google.common.collect.Sets; +import kafka.admin.AdminUtils; +import kafka.javaapi.producer.Producer; +import kafka.producer.KeyedMessage; +import kafka.producer.ProducerConfig; +import kafka.utils.ZKStringSerializer$; +import org.I0Itec.zkclient.ZkClient; +import org.apache.commons.lang.RandomStringUtils; +import org.apache.flume.Context; +import org.apache.flume.Event; +import org.apache.flume.Transaction; +import org.apache.flume.conf.Configurables; +import org.apache.flume.event.EventBuilder; +import org.apache.flume.sink.kafka.util.TestUtil; +import org.junit.*; + +import java.util.*; +import java.util.concurrent.*; +import java.util.concurrent.atomic.AtomicBoolean; +import java.util.concurrent.atomic.AtomicInteger; + +public class TestKafkaChannel { + + private static TestUtil testUtil = TestUtil.getInstance(); + private String topic = null; + private final Set usedTopics = new HashSet(); + private CountDownLatch latch = null; + + @BeforeClass + public static void setupClass() throws Exception { + testUtil.prepare(); + Thread.sleep(2500); + } + + @Before + public void setup() throws Exception { + boolean topicFound = false; + while (!topicFound) { + topic = RandomStringUtils.randomAlphabetic(8); + if (!usedTopics.contains(topic)) { + usedTopics.add(topic); + topicFound = true; + } + } + try { + createTopic(topic); + } catch (Exception e) { + } + Thread.sleep(2500); + latch = new CountDownLatch(5); + } + + @AfterClass + public static void tearDown() { + testUtil.tearDown(); + } + + @Test + public void testSuccess() throws Exception { + doTestSuccessRollback(false, false); + } + + @Test + public void testSuccessInterleave() throws Exception { + doTestSuccessRollback(false, true); + } + + @Test + public void testRollbacks() throws Exception { + doTestSuccessRollback(true, false); + } + + @Test + public void testRollbacksInterleave() throws Exception { + doTestSuccessRollback(true, true); + } + + private void doTestSuccessRollback(final boolean rollback, + final boolean interleave) throws Exception { + final KafkaChannel channel = startChannel(true); + writeAndVerify(rollback, channel, interleave); + channel.stop(); + } + + + @Test + public void testStopAndStart() throws Exception { + doTestStopAndStart(false, false); + } + + @Test + public void testStopAndStartWithRollback() throws Exception { + doTestStopAndStart(true, true); + } + + @Test + public void testStopAndStartWithRollbackAndNoRetry() throws Exception { + doTestStopAndStart(true, false); + } + + @Test + public void testNoParsingAsFlumeAgent() throws Exception { + final KafkaChannel channel = startChannel(false); + Producer producer = new Producer( + new ProducerConfig(channel.getKafkaConf())); + List> original = Lists.newArrayList(); + for (int i = 0; i < 50; i++) { + KeyedMessage data = new KeyedMessage(topic, null, RandomStringUtils.randomAlphabetic(6), + String.valueOf(i).getBytes()); + original.add(data); + } + producer.send(original); + ExecutorCompletionService submitterSvc = new + ExecutorCompletionService(Executors.newCachedThreadPool()); + List events = pullEvents(channel, submitterSvc, + 50, false, false); + wait(submitterSvc, 5); + Set finals = Sets.newHashSet(); + for (int i = 0; i < 50; i++) { + finals.add(Integer.parseInt(new String(events.get(i).getBody()))); + } + for (int i = 0; i < 50; i++) { + Assert.assertTrue(finals.contains(i)); + finals.remove(i); + } + Assert.assertTrue(finals.isEmpty()); + channel.stop(); + } + + /** + * This method starts a channel, puts events into it. The channel is then + * stopped and restarted. Then we check to make sure if all events we put + * come out. Optionally, 10 events are rolled back, + * and optionally we restart the agent immediately after and we try to pull it + * out. + * + * @param rollback + * @param retryAfterRollback + * @throws Exception + */ + private void doTestStopAndStart(boolean rollback, + boolean retryAfterRollback) throws Exception { + final KafkaChannel channel = startChannel(true); + ExecutorService underlying = Executors + .newCachedThreadPool(); + ExecutorCompletionService submitterSvc = + new ExecutorCompletionService(underlying); + final List> events = createBaseList(); + putEvents(channel, events, submitterSvc); + int completed = 0; + wait(submitterSvc, 5); + channel.stop(); + final KafkaChannel channel2 = startChannel(true); + int total = 50; + if (rollback && !retryAfterRollback) { + total = 40; + } + final List eventsPulled = + pullEvents(channel2, submitterSvc, total, rollback, retryAfterRollback); + wait(submitterSvc, 5); + channel2.stop(); + if (!retryAfterRollback && rollback) { + final KafkaChannel channel3 = startChannel(true); + int expectedRemaining = 50 - eventsPulled.size(); + final List eventsPulled2 = + pullEvents(channel3, submitterSvc, expectedRemaining, false, false); + wait(submitterSvc, 5); + Assert.assertEquals(expectedRemaining, eventsPulled2.size()); + eventsPulled.addAll(eventsPulled2); + channel3.stop(); + } + underlying.shutdownNow(); + verify(eventsPulled); + } + + private KafkaChannel startChannel(boolean parseAsFlume) throws Exception { + Context context = prepareDefaultContext(parseAsFlume); + final KafkaChannel channel = new KafkaChannel(); + Configurables.configure(channel, context); + channel.start(); + return channel; + } + + private void writeAndVerify(final boolean testRollbacks, + final KafkaChannel channel) throws Exception { + writeAndVerify(testRollbacks, channel, false); + } + + private void writeAndVerify(final boolean testRollbacks, + final KafkaChannel channel, final boolean interleave) throws Exception { + + final List> events = createBaseList(); + + ExecutorCompletionService submitterSvc = + new ExecutorCompletionService(Executors + .newCachedThreadPool()); + + putEvents(channel, events, submitterSvc); + + if (interleave) { + wait(submitterSvc, 5); + } + + ExecutorCompletionService submitterSvc2 = + new ExecutorCompletionService(Executors + .newCachedThreadPool()); + + final List eventsPulled = + pullEvents(channel, submitterSvc2, 50, testRollbacks, true); + + if (!interleave) { + wait(submitterSvc, 5); + } + wait(submitterSvc2, 5); + + verify(eventsPulled); + } + + private List> createBaseList() { + final List> events = new ArrayList>(); + for (int i = 0; i < 5; i++) { + List eventList = new ArrayList(10); + events.add(eventList); + for (int j = 0; j < 10; j++) { + Map hdrs = new HashMap(); + String v = (String.valueOf(i) + " - " + String + .valueOf(j)); + hdrs.put("header", v); + eventList.add(EventBuilder.withBody(v.getBytes(), hdrs)); + } + } + return events; + } + + private void putEvents(final KafkaChannel channel, final List> + events, ExecutorCompletionService submitterSvc) { + for (int i = 0; i < 5; i++) { + final int index = i; + submitterSvc.submit(new Callable() { + @Override + public Void call() { + Transaction tx = channel.getTransaction(); + tx.begin(); + List eventsToPut = events.get(index); + for (int j = 0; j < 10; j++) { + channel.put(eventsToPut.get(j)); + } + try { + tx.commit(); + } finally { + tx.close(); + } + return null; + } + }); + } + } + + private List pullEvents(final KafkaChannel channel, + ExecutorCompletionService submitterSvc, final int total, + final boolean testRollbacks, final boolean retryAfterRollback) { + final List eventsPulled = Collections.synchronizedList(new + ArrayList(50)); + final CyclicBarrier barrier = new CyclicBarrier(5); + final AtomicInteger counter = new AtomicInteger(0); + final AtomicInteger rolledBackCount = new AtomicInteger(0); + final AtomicBoolean startedGettingEvents = new AtomicBoolean(false); + final AtomicBoolean rolledBack = new AtomicBoolean(false); + for (int k = 0; k < 5; k++) { + final int index = k; + submitterSvc.submit(new Callable() { + @Override + public Void call() throws Exception { + Transaction tx = null; + final List eventsLocal = Lists.newLinkedList(); + int takenByThisThread = 0; + channel.registerThread(); + Thread.sleep(1000); + barrier.await(); + while (counter.get() < (total - rolledBackCount.get())) { + if (tx == null) { + tx = channel.getTransaction(); + tx.begin(); + } + try { + Event e = channel.take(); + if (e != null) { + startedGettingEvents.set(true); + eventsLocal.add(e); + } else { + if (testRollbacks && + index == 4 && + (!rolledBack.get()) && + startedGettingEvents.get()) { + tx.rollback(); + tx.close(); + tx = null; + rolledBack.set(true); + final int eventsLocalSize = eventsLocal.size(); + eventsLocal.clear(); + if (!retryAfterRollback) { + rolledBackCount.set(eventsLocalSize); + return null; + } + } else { + tx.commit(); + tx.close(); + tx = null; + eventsPulled.addAll(eventsLocal); + counter.getAndAdd(eventsLocal.size()); + eventsLocal.clear(); + } + } + } catch (Exception ex) { + eventsLocal.clear(); + if (tx != null) { + tx.rollback(); + tx.close(); + } + tx = null; + ex.printStackTrace(); + } + } + // Close txn. + return null; + } + }); + } + return eventsPulled; + } + + private void wait(ExecutorCompletionService submitterSvc, int max) + throws Exception { + int completed = 0; + while (completed < max) { + submitterSvc.take(); + completed++; + } + } + + private void verify(List eventsPulled) { + Assert.assertFalse(eventsPulled.isEmpty()); + Assert.assertEquals(50, eventsPulled.size()); + Set eventStrings = new HashSet(); + for (Event e : eventsPulled) { + Assert + .assertEquals(e.getHeaders().get("header"), new String(e.getBody())); + eventStrings.add(e.getHeaders().get("header")); + } + for (int i = 0; i < 5; i++) { + for (int j = 0; j < 10; j++) { + String v = String.valueOf(i) + " - " + String.valueOf(j); + Assert.assertTrue(eventStrings.contains(v)); + eventStrings.remove(v); + } + } + Assert.assertTrue(eventStrings.isEmpty()); + } + + private Context prepareDefaultContext(boolean parseAsFlume) { + // Prepares a default context with Kafka Server Properties + Context context = new Context(); + context.put(KafkaChannelConfiguration.BROKER_LIST_FLUME_KEY, + testUtil.getKafkaServerUrl()); + context.put(KafkaChannelConfiguration.ZOOKEEPER_CONNECT_FLUME_KEY, + testUtil.getZkUrl()); + context.put(KafkaChannelConfiguration.PARSE_AS_FLUME_EVENT, + String.valueOf(parseAsFlume)); + context.put(KafkaChannelConfiguration.READ_SMALLEST_OFFSET, "true"); + context.put(KafkaChannelConfiguration.TOPIC, topic); + return context; + } + + public static void createTopic(String topicName) { + int numPartitions = 5; + int sessionTimeoutMs = 10000; + int connectionTimeoutMs = 10000; + ZkClient zkClient = new ZkClient(testUtil.getZkUrl(), + sessionTimeoutMs, connectionTimeoutMs, + ZKStringSerializer$.MODULE$); + + int replicationFactor = 1; + Properties topicConfig = new Properties(); + AdminUtils.createTopic(zkClient, topicName, numPartitions, + replicationFactor, topicConfig); + } + + public static void deleteTopic(String topicName) { + int sessionTimeoutMs = 10000; + int connectionTimeoutMs = 10000; + ZkClient zkClient = new ZkClient(testUtil.getZkUrl(), + sessionTimeoutMs, connectionTimeoutMs, + ZKStringSerializer$.MODULE$); + AdminUtils.deleteTopic(zkClient, topicName); + } +} diff --git a/flume-ng-channels/flume-kafka-channel/src/test/resources/kafka-server.properties b/flume-ng-channels/flume-kafka-channel/src/test/resources/kafka-server.properties new file mode 100644 index 0000000000..c10c89de41 --- /dev/null +++ b/flume-ng-channels/flume-kafka-channel/src/test/resources/kafka-server.properties @@ -0,0 +1,118 @@ +# Licensed to the Apache Software Foundation (ASF) under one or more +# Licensed to the Apache Software Foundation (ASF) under one or more +# contributor license agreements. See the NOTICE file distributed with +# this work for additional information regarding copyright ownership. +# The ASF licenses this file to You under the Apache License, Version 2.0 +# (the "License"); you may not use this file except in compliance with +# the License. You may obtain a copy of the License at +# +# http://www.apache.org/licenses/LICENSE-2.0 +# +# Unless required by applicable law or agreed to in writing, software +# distributed under the License is distributed on an "AS IS" BASIS, +# WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +# See the License for the specific language governing permissions and +# limitations under the License. +# see kafka.server.KafkaConfig for additional details and defaults + +############################# Server Basics ############################# + +# The id of the broker. This must be set to a unique integer for each broker. +broker.id=0 + +############################# Socket Server Settings ############################# + +# The port the socket server listens on +port=9092 + +# Hostname the broker will bind to. If not set, the server will bind to all interfaces +#host.name=localhost + +# Hostname the broker will advertise to producers and consumers. If not set, it uses the +# value for "host.name" if configured. Otherwise, it will use the value returned from +# java.net.InetAddress.getCanonicalHostName(). +#advertised.host.name= + +# The port to publish to ZooKeeper for clients to use. If this is not set, +# it will publish the same port that the broker binds to. +#advertised.port= + +# The number of threads handling network requests +num.network.threads=2 + +# The number of threads doing disk I/O +num.io.threads=8 + +# The send buffer (SO_SNDBUF) used by the socket server +socket.send.buffer.bytes=1048576 + +# The receive buffer (SO_RCVBUF) used by the socket server +socket.receive.buffer.bytes=1048576 + +# The maximum size of a request that the socket server will accept (protection against OOM) +socket.request.max.bytes=104857600 + + +############################# Log Basics ############################# + +# A comma seperated list of directories under which to store log files +log.dirs=target/kafka-logs + +# The default number of log partitions per topic. More partitions allow greater +# parallelism for consumption, but this will also result in more files across +# the brokers. +num.partitions=5 + +############################# Log Flush Policy ############################# + +# Messages are immediately written to the filesystem but by default we only fsync() to sync +# the OS cache lazily. The following configurations control the flush of data to disk. +# There are a few important trade-offs here: +# 1. Durability: Unflushed data may be lost if you are not using replication. +# 2. Latency: Very large flush intervals may lead to latency spikes when the flush does occur as there will be a lot of data to flush. +# 3. Throughput: The flush is generally the most expensive operation, and a small flush interval may lead to exceessive seeks. +# The settings below allow one to configure the flush policy to flush data after a period of time or +# every N messages (or both). This can be done globally and overridden on a per-topic basis. + +# The number of messages to accept before forcing a flush of data to disk +#log.flush.interval.messages=10000 + +# The maximum amount of time a message can sit in a log before we force a flush +#log.flush.interval.ms=1000 + +############################# Log Retention Policy ############################# + +# The following configurations control the disposal of log segments. The policy can +# be set to delete segments after a period of time, or after a given size has accumulated. +# A segment will be deleted whenever *either* of these criteria are met. Deletion always happens +# from the end of the log. + +# The minimum age of a log file to be eligible for deletion +log.retention.hours=168 + +# A size-based retention policy for logs. Segments are pruned from the log as long as the remaining +# segments don't drop below log.retention.bytes. +#log.retention.bytes=1073741824 + +# The maximum size of a log segment file. When this size is reached a new log segment will be created. +log.segment.bytes=536870912 + +# The interval at which log segments are checked to see if they can be deleted according +# to the retention policies +log.retention.check.interval.ms=60000 + +# By default the log cleaner is disabled and the log retention policy will default to just delete segments after their retention expires. +# If log.cleaner.enable=true is set the cleaner will be enabled and individual logs can then be marked for log compaction. +log.cleaner.enable=false + +############################# Zookeeper ############################# + +# Zookeeper connection string (see zookeeper docs for details). +# This is a comma separated host:port pairs, each corresponding to a zk +# server. e.g. "127.0.0.1:3000,127.0.0.1:3001,127.0.0.1:3002". +# You can also append an optional chroot string to the urls to specify the +# root directory for all kafka znodes. +zookeeper.connect=localhost:2181 + +# Timeout in ms for connecting to zookeeper +zookeeper.connection.timeout.ms=1000000 diff --git a/flume-ng-channels/flume-kafka-channel/src/test/resources/log4j.properties b/flume-ng-channels/flume-kafka-channel/src/test/resources/log4j.properties new file mode 100644 index 0000000000..b86600b863 --- /dev/null +++ b/flume-ng-channels/flume-kafka-channel/src/test/resources/log4j.properties @@ -0,0 +1,78 @@ +# Licensed to the Apache Software Foundation (ASF) under one or more +# contributor license agreements. See the NOTICE file distributed with +# this work for additional information regarding copyright ownership. +# The ASF licenses this file to You under the Apache License, Version 2.0 +# (the "License"); you may not use this file except in compliance with +# the License. You may obtain a copy of the License at +# +# http://www.apache.org/licenses/LICENSE-2.0 +# +# Unless required by applicable law or agreed to in writing, software +# distributed under the License is distributed on an "AS IS" BASIS, +# WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +# See the License for the specific language governing permissions and +# limitations under the License. + +kafka.logs.dir=target/logs + +log4j.rootLogger=INFO, stdout + +log4j.appender.stdout=org.apache.log4j.ConsoleAppender +log4j.appender.stdout.layout=org.apache.log4j.PatternLayout +log4j.appender.stdout.layout.ConversionPattern=[%d] %p %m (%c)%n + +log4j.appender.kafkaAppender=org.apache.log4j.DailyRollingFileAppender +log4j.appender.kafkaAppender.DatePattern='.'yyyy-MM-dd-HH +log4j.appender.kafkaAppender.File=${kafka.logs.dir}/server.log +log4j.appender.kafkaAppender.layout=org.apache.log4j.PatternLayout +log4j.appender.kafkaAppender.layout.ConversionPattern=[%d] %p %m (%c)%n + +log4j.appender.stateChangeAppender=org.apache.log4j.DailyRollingFileAppender +log4j.appender.stateChangeAppender.DatePattern='.'yyyy-MM-dd-HH +log4j.appender.stateChangeAppender.File=${kafka.logs.dir}/state-change.log +log4j.appender.stateChangeAppender.layout=org.apache.log4j.PatternLayout +log4j.appender.stateChangeAppender.layout.ConversionPattern=[%d] %p %m (%c)%n + +log4j.appender.requestAppender=org.apache.log4j.DailyRollingFileAppender +log4j.appender.requestAppender.DatePattern='.'yyyy-MM-dd-HH +log4j.appender.requestAppender.File=${kafka.logs.dir}/kafka-request.log +log4j.appender.requestAppender.layout=org.apache.log4j.PatternLayout +log4j.appender.requestAppender.layout.ConversionPattern=[%d] %p %m (%c)%n + +log4j.appender.cleanerAppender=org.apache.log4j.DailyRollingFileAppender +log4j.appender.cleanerAppender.DatePattern='.'yyyy-MM-dd-HH +log4j.appender.cleanerAppender.File=${kafka.logs.dir}/log-cleaner.log +log4j.appender.cleanerAppender.layout=org.apache.log4j.PatternLayout +log4j.appender.cleanerAppender.layout.ConversionPattern=[%d] %p %m (%c)%n + +log4j.appender.controllerAppender=org.apache.log4j.DailyRollingFileAppender +log4j.appender.controllerAppender.DatePattern='.'yyyy-MM-dd-HH +log4j.appender.controllerAppender.File=${kafka.logs.dir}/controller.log +log4j.appender.controllerAppender.layout=org.apache.log4j.PatternLayout +log4j.appender.controllerAppender.layout.ConversionPattern=[%d] %p %m (%c)%n + +# Turn on all our debugging info +#log4j.logger.kafka.producer.async.DefaultEventHandler=DEBUG, kafkaAppender +#log4j.logger.kafka.client.ClientUtils=DEBUG, kafkaAppender +#log4j.logger.kafka.perf=DEBUG, kafkaAppender +#log4j.logger.kafka.perf.ProducerPerformance$ProducerThread=DEBUG, kafkaAppender +#log4j.logger.org.I0Itec.zkclient.ZkClient=DEBUG +log4j.logger.kafka=INFO, kafkaAppender + +log4j.logger.kafka.network.RequestChannel$=WARN, requestAppender +log4j.additivity.kafka.network.RequestChannel$=false + +#log4j.logger.kafka.network.Processor=TRACE, requestAppender +#log4j.logger.kafka.server.KafkaApis=TRACE, requestAppender +#log4j.additivity.kafka.server.KafkaApis=false +log4j.logger.kafka.request.logger=WARN, requestAppender +log4j.additivity.kafka.request.logger=false + +log4j.logger.kafka.controller=TRACE, controllerAppender +log4j.additivity.kafka.controller=false + +log4j.logger.kafka.log.LogCleaner=INFO, cleanerAppender +log4j.additivity.kafka.log.LogCleaner=false + +log4j.logger.state.change.logger=TRACE, stateChangeAppender +log4j.additivity.state.change.logger=false diff --git a/flume-ng-channels/flume-kafka-channel/src/test/resources/zookeeper.properties b/flume-ng-channels/flume-kafka-channel/src/test/resources/zookeeper.properties new file mode 100644 index 0000000000..89e1b5ec38 --- /dev/null +++ b/flume-ng-channels/flume-kafka-channel/src/test/resources/zookeeper.properties @@ -0,0 +1,20 @@ +# Licensed to the Apache Software Foundation (ASF) under one or more +# contributor license agreements. See the NOTICE file distributed with +# this work for additional information regarding copyright ownership. +# The ASF licenses this file to You under the Apache License, Version 2.0 +# (the "License"); you may not use this file except in compliance with +# the License. You may obtain a copy of the License at +# +# http://www.apache.org/licenses/LICENSE-2.0 +# +# Unless required by applicable law or agreed to in writing, software +# distributed under the License is distributed on an "AS IS" BASIS, +# WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +# See the License for the specific language governing permissions and +# limitations under the License. +# the directory where the snapshot is stored. +dataDir=target +# the port at which the clients will connect +clientPort=2181 +# disable the per-ip limit on the number of connections since this is a non-production config +maxClientCnxns=0 \ No newline at end of file diff --git a/flume-ng-channels/pom.xml b/flume-ng-channels/pom.xml index dc8dbc69a7..f44171da9d 100644 --- a/flume-ng-channels/pom.xml +++ b/flume-ng-channels/pom.xml @@ -44,5 +44,6 @@ limitations under the License. flume-jdbc-channel flume-file-channel flume-spillable-memory-channel + flume-kafka-channel diff --git a/flume-ng-sinks/flume-ng-kafka-sink/pom.xml b/flume-ng-sinks/flume-ng-kafka-sink/pom.xml index 746a39539a..e32365872f 100644 --- a/flume-ng-sinks/flume-ng-kafka-sink/pom.xml +++ b/flume-ng-sinks/flume-ng-kafka-sink/pom.xml @@ -28,6 +28,17 @@ org.apache.rat apache-rat-plugin + + org.apache.maven.plugins + maven-jar-plugin + + + + test-jar + + + + diff --git a/flume-ng-sinks/flume-ng-kafka-sink/src/test/java/org/apache/flume/sink/kafka/util/KafkaConsumer.java b/flume-ng-sinks/flume-ng-kafka-sink/src/test/java/org/apache/flume/sink/kafka/util/KafkaConsumer.java index 1c989223a8..d5dfbd688f 100644 --- a/flume-ng-sinks/flume-ng-kafka-sink/src/test/java/org/apache/flume/sink/kafka/util/KafkaConsumer.java +++ b/flume-ng-sinks/flume-ng-kafka-sink/src/test/java/org/apache/flume/sink/kafka/util/KafkaConsumer.java @@ -57,7 +57,7 @@ private static ConsumerConfig createConsumerConfig(String zkUrl, Properties props = new Properties(); props.put("zookeeper.connect", zkUrl); props.put("group.id", groupId); - props.put("zookeeper.session.timeout.ms", "400"); + props.put("zookeeper.session.timeout.ms", "1000"); props.put("zookeeper.sync.time.ms", "200"); props.put("auto.commit.interval.ms", "1000"); props.put("auto.offset.reset", "smallest"); diff --git a/flume-ng-sinks/flume-ng-kafka-sink/src/test/java/org/apache/flume/sink/kafka/util/TestUtil.java b/flume-ng-sinks/flume-ng-kafka-sink/src/test/java/org/apache/flume/sink/kafka/util/TestUtil.java index 8855c535b3..6405d6ca78 100644 --- a/flume-ng-sinks/flume-ng-kafka-sink/src/test/java/org/apache/flume/sink/kafka/util/TestUtil.java +++ b/flume-ng-sinks/flume-ng-kafka-sink/src/test/java/org/apache/flume/sink/kafka/util/TestUtil.java @@ -66,6 +66,7 @@ private boolean startKafkaServer() { Properties kafkaProperties = new Properties(); Properties zkProperties = new Properties(); + logger.info("Starting kafka server."); try { //load properties zkProperties.load(Class.class.getResourceAsStream( From fdd61e840b004ca1b5c23b28e8d9eeda300f56f7 Mon Sep 17 00:00:00 2001 From: Hari Shreedharan Date: Fri, 31 Oct 2014 14:27:21 -0700 Subject: [PATCH 166/341] FLUME-2523. Document Kafka channel (Gwen Shapira via Hari) --- flume-ng-doc/sphinx/FlumeUserGuide.rst | 54 ++++++++++++++++++++++++++ 1 file changed, 54 insertions(+) diff --git a/flume-ng-doc/sphinx/FlumeUserGuide.rst b/flume-ng-doc/sphinx/FlumeUserGuide.rst index e3aedeb193..0ab23fd33e 100644 --- a/flume-ng-doc/sphinx/FlumeUserGuide.rst +++ b/flume-ng-doc/sphinx/FlumeUserGuide.rst @@ -2305,6 +2305,60 @@ Example for agent named a1: a1.channels = c1 a1.channels.c1.type = jdbc +Kafka Channel +~~~~~~~~~~~~~ + +The events are stored in a Kafka cluster (must be installed separately). Kafka provides high availability and +replication, so in case an agent or a kafka broker crashes, the events are immediately available to other sinks + +The Kafka channel can be used for multiple scenarios: +* With Flume source and sink - it provides a reliable and highly available channel for events +* With Flume source and interceptor but no sink - it allows writing Flume events into a Kafka topic, for use by other apps +* With Flume sink, but no source - it is a low-latency, fault tolerant way to send events from Kafka to Flume sources such as HDFS, HBase or Solr + +Required properties are in **bold**. + +====================== ========================== =============================================================================================================== +Property Name Default Description +====================== ========================== =============================================================================================================== +**type** -- The component type name, needs to be ``org.apache.flume.channel.kafka.KafkaChannel`` +**brokerList** -- List of brokers in the Kafka cluster used by the channel + This can be a partial list of brokers, but we recommend at least two for HA. + The format is comma separated list of hostname:port +**zookeeperConnect** -- URI of ZooKeeper used by Kafka cluster + The format is comma separated list of hostname:port. If chroot is used, it is added once at the end. + For example: zookeeper-1:2181,zookeeper-2:2182,zookeeper-3:2181/kafka +topic flume-channel Kafka topic which the channel will use +groupId flume Consumer group ID the channel uses to register with Kafka. + Multiple channels must use the same topic and group to ensure that when one agent fails another can get the data + Note that having non-channel consumers with the same ID can lead to data loss. +parseAsFlumeEvent true Expecting Avro datums with FlumeEvent schema in the channel. + This should be true if Flume source is writing to the channel + And false if other producers are writing into the topic that the channel is using + Flume source messages to Kafka can be parsed outside of Flume by using + org.apache.flume.source.avro.AvroFlumeEvent provided by the flume-ng-sdk artifact +readSmallestOffset false When set to true, the channel will read all data in the topic, starting from the oldest event + when false, it will read only events written after the channel started + When "parseAsFlumeEvent" is true, this will be false. Flume source will start prior to the sinks and this + guarantees that events sent by source before sinks start will not be lost. +Other Kafka Properties -- These properties are used to configure the Kafka Producer and Consumer used by the channel. + Any property supported by Kafka can be used. + The only requirement is to prepend the property name with the prefix ``kafka.``. + For example: kafka.producer.type +====================== ========================== =============================================================================================================== + +.. note:: Due to the way the channel is load balanced, there may be duplicate events when the agent first starts up + +Example for agent named a1: + +.. code-block:: properties + + a1.channels.channel1.type = org.apache.flume.channel.kafka.KafkaChannel + a1.channels.channel1.capacity = 10000 + a1.channels.channel1.transactionCapacity = 1000 + a1.channels.channel1.brokerList=kafka-2:9092,kafka-3:9092 + a1.channels.channel1.topic=channel1 + a1.channels.channel1.zookeeperConnect=kafka-1:2181 File Channel ~~~~~~~~~~~~ From 7e21ad36c05e07c4c6b92aa489f08c3cb562cb28 Mon Sep 17 00:00:00 2001 From: Jarek Jarcec Cecho Date: Wed, 5 Nov 2014 16:53:13 -0800 Subject: [PATCH 167/341] FLUME-2533: HTTPS tests fail on Java 6 (Hari Shreedharan via Jarek Jarcec Cecho) --- .../flume/source/http/TestHTTPSource.java | 77 +++++++++++++++++-- 1 file changed, 71 insertions(+), 6 deletions(-) diff --git a/flume-ng-core/src/test/java/org/apache/flume/source/http/TestHTTPSource.java b/flume-ng-core/src/test/java/org/apache/flume/source/http/TestHTTPSource.java index 9ba0a8c491..64111beeb7 100644 --- a/flume-ng-core/src/test/java/org/apache/flume/source/http/TestHTTPSource.java +++ b/flume-ng-core/src/test/java/org/apache/flume/source/http/TestHTTPSource.java @@ -367,6 +367,7 @@ public java.security.cert.X509Certificate[] getAcceptedIssuers() { }}; SSLContext sc = null; + javax.net.ssl.SSLSocketFactory factory = null; if (System.getProperty("java.vendor").contains("IBM")) { sc = SSLContext.getInstance("SSL_TLS"); } else { @@ -379,11 +380,11 @@ public boolean verify(String arg0, SSLSession arg1) { } }; sc.init(null, trustAllCerts, new SecureRandom()); - javax.net.ssl.SSLSocketFactory factory = null; + if(protocol != null) { - factory = new TestSSLSocketFactory(sc.getSocketFactory(), protocol); + factory = new DisabledProtocolsSocketFactory(sc.getSocketFactory(), protocol); } else { - factory = sc.getSocketFactory(); + factory = new EnabledProtocolsSocketFactory(sc.getSocketFactory()); } HttpsURLConnection.setDefaultSSLSocketFactory(factory); HttpsURLConnection.setDefaultHostnameVerifier( @@ -490,16 +491,16 @@ public ResultWrapper(HttpResponse resp, List events){ } } - private class TestSSLSocketFactory extends javax.net.ssl.SSLSocketFactory { + private class DisabledProtocolsSocketFactory extends javax.net.ssl.SSLSocketFactory { private final javax.net.ssl.SSLSocketFactory socketFactory; private final String[] protocols; - TestSSLSocketFactory(javax.net.ssl.SSLSocketFactory factory, String protocol) { + DisabledProtocolsSocketFactory(javax.net.ssl.SSLSocketFactory factory, String protocol) { this.socketFactory = factory; if(protocol.equals("SSLv2Hello")) { protocols = new String[2]; - protocols[0] = "TLSv1.2"; + protocols[0] = "TLSv1"; protocols[1] = protocol; } else { protocols = new String[1]; @@ -559,4 +560,68 @@ public Socket createSocket(InetAddress inetAddress, int i, return sc; } } + + private class EnabledProtocolsSocketFactory extends javax.net.ssl.SSLSocketFactory { + + private final javax.net.ssl.SSLSocketFactory socketFactory; + private final String[] protocols; + + EnabledProtocolsSocketFactory(javax.net.ssl.SSLSocketFactory factory) { + this.socketFactory = factory; + protocols = new String[1]; + protocols[0] = "TLSv1"; + } + + @Override + public String[] getDefaultCipherSuites() { + return socketFactory.getDefaultCipherSuites(); + } + + @Override + public String[] getSupportedCipherSuites() { + return socketFactory.getSupportedCipherSuites(); + } + + @Override + public Socket createSocket(Socket socket, String s, int i, boolean b) + throws IOException { + SSLSocket sc = (SSLSocket) socketFactory.createSocket(socket, s, i, b); + sc.setEnabledProtocols(protocols); + return sc; + } + + @Override + public Socket createSocket(String s, int i) + throws IOException, UnknownHostException { + SSLSocket sc = (SSLSocket)socketFactory.createSocket(s, i); + sc.setEnabledProtocols(protocols); + return sc; + } + + @Override + public Socket createSocket(String s, int i, InetAddress inetAddress, int i2) + throws IOException, UnknownHostException { + SSLSocket sc = (SSLSocket)socketFactory.createSocket(s, i, inetAddress, + i2); + sc.setEnabledProtocols(protocols); + return sc; + } + + @Override + public Socket createSocket(InetAddress inetAddress, int i) + throws IOException { + SSLSocket sc = (SSLSocket)socketFactory.createSocket(inetAddress, i); + sc.setEnabledProtocols(protocols); + return sc; + } + + @Override + public Socket createSocket(InetAddress inetAddress, int i, + InetAddress inetAddress2, int i2) throws IOException { + SSLSocket sc = (SSLSocket)socketFactory.createSocket(inetAddress, i, + inetAddress2, i2); + sc.setEnabledProtocols(protocols); + return sc; + } + } } From efbf87fb6ddc0bbc736446a5a91cf6a83d34d2d4 Mon Sep 17 00:00:00 2001 From: Hari Shreedharan Date: Thu, 6 Nov 2014 11:31:35 -0800 Subject: [PATCH 168/341] FLUME-2525. Handle a zero byte .flumespool-main.meta file for the spooldir source. (Johny Rufus via Hari) --- .../avro/ReliableSpoolingFileEventReader.java | 3 ++ .../TestReliableSpoolingFileEventReader.java | 29 +++++++++++++++++++ 2 files changed, 32 insertions(+) diff --git a/flume-ng-core/src/main/java/org/apache/flume/client/avro/ReliableSpoolingFileEventReader.java b/flume-ng-core/src/main/java/org/apache/flume/client/avro/ReliableSpoolingFileEventReader.java index 1833076d16..27e9c1ebdd 100644 --- a/flume-ng-core/src/main/java/org/apache/flume/client/avro/ReliableSpoolingFileEventReader.java +++ b/flume-ng-core/src/main/java/org/apache/flume/client/avro/ReliableSpoolingFileEventReader.java @@ -199,6 +199,9 @@ private ReliableSpoolingFileEventReader(File spoolDirectory, } this.metaFile = new File(trackerDirectory, metaFileName); + if(metaFile.exists() && metaFile.length() == 0) { + deleteMetaFile(); + } } @VisibleForTesting diff --git a/flume-ng-core/src/test/java/org/apache/flume/client/avro/TestReliableSpoolingFileEventReader.java b/flume-ng-core/src/test/java/org/apache/flume/client/avro/TestReliableSpoolingFileEventReader.java index a6b247348b..c6ff63e73f 100644 --- a/flume-ng-core/src/test/java/org/apache/flume/client/avro/TestReliableSpoolingFileEventReader.java +++ b/flume-ng-core/src/test/java/org/apache/flume/client/avro/TestReliableSpoolingFileEventReader.java @@ -405,6 +405,35 @@ public int compare(Long o1, Long o2) { @Test public void testLargeNumberOfFilesRANDOM() throws IOException { templateTestForLargeNumberOfFiles(ConsumeOrder.RANDOM, null, 1000); } + + @Test + public void testZeroByteTrackerFile() throws IOException { + String trackerDirPath = + SpoolDirectorySourceConfigurationConstants.DEFAULT_TRACKER_DIR; + File trackerDir = new File(WORK_DIR, trackerDirPath); + if(!trackerDir.exists()) { + trackerDir.mkdir(); + } + File trackerFile = new File(trackerDir, ReliableSpoolingFileEventReader.metaFileName); + if(trackerFile.exists()) { + trackerFile.delete(); + } + trackerFile.createNewFile(); + + ReliableEventReader reader = new ReliableSpoolingFileEventReader.Builder() + .spoolDirectory(WORK_DIR).trackerDirPath(trackerDirPath).build(); + final int expectedLines = 1; + int seenLines = 0; + List events = reader.readEvents(10); + int numEvents = events.size(); + if (numEvents > 0) { + seenLines += numEvents; + reader.commit(); + } + // This line will fail, if the zero-byte tracker file has not been handled + Assert.assertEquals(expectedLines, seenLines); + } + private void templateTestForLargeNumberOfFiles(ConsumeOrder order, Comparator comparator, int N) throws IOException { From 543c4c43d5bb04f99ea89cbe9c7c9ab4174563cb Mon Sep 17 00:00:00 2001 From: Hari Shreedharan Date: Thu, 6 Nov 2014 13:26:02 -0800 Subject: [PATCH 169/341] Updating CHANGELOG for 1.5.1 release --- CHANGELOG | 10 ++++++++++ 1 file changed, 10 insertions(+) diff --git a/CHANGELOG b/CHANGELOG index e5def5df9f..7086807e85 100644 --- a/CHANGELOG +++ b/CHANGELOG @@ -1,3 +1,13 @@ +Release Notes - Flume - Version v1.5.1 + +** Bug + * [FLUME-2441] - Unit test TestHTTPSource.java failed with IBM JDK 1.7 + * [FLUME-2520] - HTTP Source should be able to block a prefixed set of protocols. + * [FLUME-2533] - HTTPS tests fail on Java 6 + +** Improvement + * [FLUME-2511] - Allow configuration of enabled protocols in Avro source and Rpc client + Release Notes - Flume - Version v1.5.0.1 ** Bug From 29e31bfb5fe2be5e3e73b0e02858d49603d6eff8 Mon Sep 17 00:00:00 2001 From: Hari Shreedharan Date: Thu, 6 Nov 2014 11:39:28 -0800 Subject: [PATCH 170/341] Preparing for Flume-1.5.1 Release --- RELEASE-NOTES | 6 +++--- 1 file changed, 3 insertions(+), 3 deletions(-) diff --git a/RELEASE-NOTES b/RELEASE-NOTES index 5a6ab4d461..6f22b21169 100644 --- a/RELEASE-NOTES +++ b/RELEASE-NOTES @@ -1,4 +1,4 @@ -Apache Flume 1.5.0.1 +Apache Flume 1.5.1 CONTENTS 1. What is Apache Flume @@ -17,8 +17,8 @@ simple, extensible data model that allows for online analytic application. 2. Status of this release -Apache Flume 1.5.0.1 is the sixth release of Flume as an Apache top-level project -(TLP). Apache Flume 1.5.0.1 is production-ready software. +Apache Flume 1.5.1 is the seventh release of Flume as an Apache top-level project +(TLP). Apache Flume 1.5.1 is production-ready software. 3. Major changes in this Release For a detailed list of changes, please see the CHANGELOG file included From 76ddc82fb829995f2c88976d4153de132edf0a20 Mon Sep 17 00:00:00 2001 From: Jarek Jarcec Cecho Date: Sat, 8 Nov 2014 12:09:49 -0800 Subject: [PATCH 171/341] FLUME-2505: Test added in FLUME-2502 is flaky (Hari Shreedharan via Jarek Jarcec Cecho) --- .../client/avro/TestReliableSpoolingFileEventReader.java | 6 ++++-- 1 file changed, 4 insertions(+), 2 deletions(-) diff --git a/flume-ng-core/src/test/java/org/apache/flume/client/avro/TestReliableSpoolingFileEventReader.java b/flume-ng-core/src/test/java/org/apache/flume/client/avro/TestReliableSpoolingFileEventReader.java index c6ff63e73f..476bbff6c7 100644 --- a/flume-ng-core/src/test/java/org/apache/flume/client/avro/TestReliableSpoolingFileEventReader.java +++ b/flume-ng-core/src/test/java/org/apache/flume/client/avro/TestReliableSpoolingFileEventReader.java @@ -241,6 +241,7 @@ public void testConsumeFileRandomlyNewFile() throws Exception { File tempDir = Files.createTempDir(); File tempFile = new File(tempDir, "t"); File finalFile = new File(WORK_DIR, "t-file"); + int totalFiles = WORK_DIR.listFiles().length; FileUtils.write(tempFile, "Last file"); final Set actual = Sets.newHashSet(); ExecutorService executor = Executors.newSingleThreadExecutor(); @@ -257,6 +258,8 @@ public Void call() throws Exception { semaphore.acquire(); tempFile.renameTo(finalFile); wait.get(); + int listFilesCount = ((ReliableSpoolingFileEventReader)reader) + .getListFilesCount(); finalFile.delete(); FileUtils.deleteQuietly(tempDir); createExpectedFromFilesInSetup(expected); @@ -264,8 +267,7 @@ public Void call() throws Exception { expected.add( "New file created in the end. Shoud be read randomly."); expected.add("Last file"); - Assert.assertEquals(2, ((ReliableSpoolingFileEventReader)reader) - .getListFilesCount()); + Assert.assertTrue(listFilesCount < (totalFiles + 2)); Assert.assertEquals(expected, actual); } From 534fe268d1d0ad197f6e4a867ab0ae0155d1a927 Mon Sep 17 00:00:00 2001 From: Mike Percy Date: Mon, 10 Nov 2014 18:57:20 -0800 Subject: [PATCH 172/341] FLUME-2497. Support fractional seconds in Syslog timestamps This fixes a bug in the SyslogTcpSource and SyslogUdpSource where fractional timestamps fail to parse. (Johny Rufus via Mike Percy) --- .../java/org/apache/flume/source/SyslogUtils.java | 3 +++ .../org/apache/flume/source/TestSyslogUtils.java | 15 +++++++++++++++ 2 files changed, 18 insertions(+) diff --git a/flume-ng-core/src/main/java/org/apache/flume/source/SyslogUtils.java b/flume-ng-core/src/main/java/org/apache/flume/source/SyslogUtils.java index 50593a7dd5..42e3f7136c 100644 --- a/flume-ng-core/src/main/java/org/apache/flume/source/SyslogUtils.java +++ b/flume-ng-core/src/main/java/org/apache/flume/source/SyslogUtils.java @@ -241,6 +241,9 @@ private void initHeaderFormats() { // timezone in RFC5424 is [+-]tt:tt, so remove the ':' for java date formatting fmt1.searchPattern.add("([+-])(\\d{2})[:](\\d{2})"); fmt1.replacePattern.add("$1$2$3"); + // FLUME-2497: SimpleDateFormat does not handle microseconds, Truncate after 3 digits. + fmt1.searchPattern.add("(T\\d{2}:\\d{2}:\\d{2}\\.\\d{3})(\\d*)"); + fmt1.replacePattern.add("$1"); fmt1.dateFormat.add(new SimpleDateFormat(SYSLOG_TIMESTAMP_FORMAT_RFC5424_1, Locale.ENGLISH)); fmt1.dateFormat.add(new SimpleDateFormat(SYSLOG_TIMESTAMP_FORMAT_RFC5424_2, Locale.ENGLISH)); fmt1.dateFormat.add(new SimpleDateFormat(SYSLOG_TIMESTAMP_FORMAT_RFC5424_3, Locale.ENGLISH)); diff --git a/flume-ng-core/src/test/java/org/apache/flume/source/TestSyslogUtils.java b/flume-ng-core/src/test/java/org/apache/flume/source/TestSyslogUtils.java index 86ef40f09f..6da173391b 100644 --- a/flume-ng-core/src/test/java/org/apache/flume/source/TestSyslogUtils.java +++ b/flume-ng-core/src/test/java/org/apache/flume/source/TestSyslogUtils.java @@ -152,6 +152,21 @@ public void TestHeader10() throws ParseException { format1, host1, data1); } + @Test + public void TestHeader11() throws ParseException { + // SyslogUtils should truncate microsecond precision to only 3 digits. + // This is to maintain consistency between the two syslog implementations. + String inputStamp = "2014-10-03T17:20:01.123456-07:00"; + String outputStamp = "2014-10-03T17:20:01.123-07:00"; + + String format1 = "yyyy-MM-dd'T'HH:mm:ss.S"; + String host1 = "ubuntu-11.cloudera.com"; + String data1 = "some msg"; + + String msg1 = "<10>" + inputStamp + " " + host1 + " " + data1 + "\n"; + checkHeader(msg1, outputStamp, format1, host1, data1); + } + @Test public void TestRfc3164HeaderApacheLogWithNulls() throws ParseException { String stamp1 = "Apr 1 13:14:04"; From 2c18533253be786b9c60bf687cdf38d2384d2625 Mon Sep 17 00:00:00 2001 From: Hari Shreedharan Date: Mon, 10 Nov 2014 22:38:46 -0800 Subject: [PATCH 173/341] FLUME-2385. Remove incorrect log message at INFO level in Spool Directory Source. (Phil Scala via Hari Shreedharan) --- .../main/java/org/apache/flume/source/SpoolDirectorySource.java | 1 - 1 file changed, 1 deletion(-) diff --git a/flume-ng-core/src/main/java/org/apache/flume/source/SpoolDirectorySource.java b/flume-ng-core/src/main/java/org/apache/flume/source/SpoolDirectorySource.java index f5be9ae5e9..0b11fc9b04 100644 --- a/flume-ng-core/src/main/java/org/apache/flume/source/SpoolDirectorySource.java +++ b/flume-ng-core/src/main/java/org/apache/flume/source/SpoolDirectorySource.java @@ -252,7 +252,6 @@ public void run() { sourceCounter.addToEventAcceptedCount(events.size()); sourceCounter.incrementAppendBatchAcceptedCount(); } - logger.info("Spooling Directory Source runner has shutdown."); } catch (Throwable t) { logger.error("FATAL: " + SpoolDirectorySource.this.toString() + ": " + "Uncaught exception in SpoolDirectorySource thread. " + From 8c7f6936045222026c93f34bda861acec79066cf Mon Sep 17 00:00:00 2001 From: Jarek Jarcec Cecho Date: Wed, 12 Nov 2014 06:16:47 -0800 Subject: [PATCH 174/341] FLUME-2548: Enable SSLv2Hello for Avro Source and NettyAvroRpcClient (Hari Shreedharan via Jarek Jarcec Cecho) --- .../org/apache/flume/source/AvroSource.java | 19 +++++++++++-------- flume-ng-doc/sphinx/FlumeUserGuide.rst | 2 +- .../apache/flume/api/NettyAvroRpcClient.java | 15 +++++++++++---- 3 files changed, 23 insertions(+), 13 deletions(-) diff --git a/flume-ng-core/src/main/java/org/apache/flume/source/AvroSource.java b/flume-ng-core/src/main/java/org/apache/flume/source/AvroSource.java index 59ee43a8e1..6eb6a0a180 100644 --- a/flume-ng-core/src/main/java/org/apache/flume/source/AvroSource.java +++ b/flume-ng-core/src/main/java/org/apache/flume/source/AvroSource.java @@ -25,11 +25,7 @@ import java.net.InetSocketAddress; import java.security.KeyStore; import java.security.Security; -import java.util.ArrayList; -import java.util.Arrays; -import java.util.HashMap; -import java.util.List; -import java.util.Map; +import java.util.*; import java.util.concurrent.Executors; import java.util.concurrent.ScheduledExecutorService; import java.util.concurrent.TimeUnit; @@ -149,7 +145,7 @@ public class AvroSource extends AbstractSource implements EventDrivenSource, private String keystore; private String keystorePassword; private String keystoreType; - private List excludeProtocols; + private final List excludeProtocols = new LinkedList(); private boolean enableSsl = false; private boolean enableIpFilter; private String patternRuleConfigDefinition; @@ -181,8 +177,15 @@ public void configure(Context context) { keystore = context.getString(KEYSTORE_KEY); keystorePassword = context.getString(KEYSTORE_PASSWORD_KEY); keystoreType = context.getString(KEYSTORE_TYPE_KEY, "JKS"); - excludeProtocols = Arrays.asList( - context.getString(EXCLUDE_PROTOCOLS, "SSLv2Hello SSLv3").split(" ")); + String excludeProtocolsStr = context.getString(EXCLUDE_PROTOCOLS); + if (excludeProtocolsStr == null) { + excludeProtocols.add("SSLv3"); + } else { + excludeProtocols.addAll(Arrays.asList(excludeProtocolsStr.split(" "))); + if (!excludeProtocols.contains("SSLv3")) { + excludeProtocols.add("SSLv3"); + } + } if (enableSsl) { Preconditions.checkNotNull(keystore, diff --git a/flume-ng-doc/sphinx/FlumeUserGuide.rst b/flume-ng-doc/sphinx/FlumeUserGuide.rst index 0ab23fd33e..bab01db0e4 100644 --- a/flume-ng-doc/sphinx/FlumeUserGuide.rst +++ b/flume-ng-doc/sphinx/FlumeUserGuide.rst @@ -705,7 +705,7 @@ ssl false Set this to true to enable SSL encryption keystore -- This is the path to a Java keystore file. Required for SSL. keystore-password -- The password for the Java keystore. Required for SSL. keystore-type JKS The type of the Java keystore. This can be "JKS" or "PKCS12". -exclude-protocols SSLv2Hello SSLv3 Space-separated list of SSL/TLS protocols to exclude +exclude-protocols SSLv3 Space-separated list of SSL/TLS protocols to exclude. SSLv3 will always be excluded in addition to the protocols specified. ipFilter false Set this to true to enable ipFiltering for netty ipFilter.rules -- Define N netty ipFilter pattern rules with this config. ================== ================ =================================================== diff --git a/flume-ng-sdk/src/main/java/org/apache/flume/api/NettyAvroRpcClient.java b/flume-ng-sdk/src/main/java/org/apache/flume/api/NettyAvroRpcClient.java index ad9b58020d..36616729ae 100644 --- a/flume-ng-sdk/src/main/java/org/apache/flume/api/NettyAvroRpcClient.java +++ b/flume-ng-sdk/src/main/java/org/apache/flume/api/NettyAvroRpcClient.java @@ -96,7 +96,7 @@ public class NettyAvroRpcClient extends AbstractRpcClient private String truststore; private String truststorePassword; private String truststoreType; - private List excludeProtocols; + private final List excludeProtocols = new LinkedList(); private Transceiver transceiver; private AvroSourceProtocol.Callback avroClient; @@ -607,9 +607,16 @@ public synchronized void configure(Properties properties) RpcClientConfigurationConstants.CONFIG_TRUSTSTORE_PASSWORD); truststoreType = properties.getProperty( RpcClientConfigurationConstants.CONFIG_TRUSTSTORE_TYPE, "JKS"); - excludeProtocols = Arrays.asList(properties.getProperty( - RpcClientConfigurationConstants.CONFIG_EXCLUDE_PROTOCOLS, "SSLv2Hello SSLv3") - .split(" ")); + String excludeProtocolsStr = properties.getProperty( + RpcClientConfigurationConstants.CONFIG_EXCLUDE_PROTOCOLS); + if (excludeProtocolsStr == null) { + excludeProtocols.add("SSLv3"); + } else { + excludeProtocols.addAll(Arrays.asList(excludeProtocolsStr.split(" "))); + if (!excludeProtocols.contains("SSLv3")) { + excludeProtocols.add("SSLv3"); + } + } String maxIoWorkersStr = properties.getProperty( RpcClientConfigurationConstants.MAX_IO_WORKERS); From 94b25aebc6d90480bb00898de4b98257cc7d8cbb Mon Sep 17 00:00:00 2001 From: Jarek Jarcec Cecho Date: Wed, 12 Nov 2014 07:35:06 -0800 Subject: [PATCH 175/341] FLUME-2549: Enable SSLv2Hello for HttpSource (Hari Shreedharan via Jarek Jarcec Cecho) --- .../apache/flume/source/http/HTTPSource.java | 23 +++++- .../HTTPSourceConfigurationConstants.java | 1 + .../flume/source/http/TestHTTPSource.java | 79 +------------------ flume-ng-doc/sphinx/FlumeUserGuide.rst | 31 ++++---- 4 files changed, 39 insertions(+), 95 deletions(-) diff --git a/flume-ng-core/src/main/java/org/apache/flume/source/http/HTTPSource.java b/flume-ng-core/src/main/java/org/apache/flume/source/http/HTTPSource.java index 4b2717c4b6..b520b03282 100644 --- a/flume-ng-core/src/main/java/org/apache/flume/source/http/HTTPSource.java +++ b/flume-ng-core/src/main/java/org/apache/flume/source/http/HTTPSource.java @@ -93,6 +93,7 @@ public class HTTPSource extends AbstractSource implements private volatile String keyStorePath; private volatile String keyStorePassword; private volatile Boolean sslEnabled; + private final List excludedProtocols = new LinkedList(); @Override @@ -120,7 +121,18 @@ public void configure(Context context) { Preconditions.checkArgument(keyStorePath != null && !keyStorePath.isEmpty(), "Keystore is required for SSL Conifguration" ); keyStorePassword = context.getString(HTTPSourceConfigurationConstants.SSL_KEYSTORE_PASSWORD); - Preconditions.checkArgument(keyStorePassword != null, "Keystore password is required for SSL Configuration"); + Preconditions.checkArgument(keyStorePassword != null, + "Keystore password is required for SSL Configuration"); + String excludeProtocolsStr = context.getString(HTTPSourceConfigurationConstants + .EXCLUDE_PROTOCOLS); + if (excludeProtocolsStr == null) { + excludedProtocols.add("SSLv3"); + } else { + excludedProtocols.addAll(Arrays.asList(excludeProtocolsStr.split(" "))); + if (!excludedProtocols.contains("SSLv3")) { + excludedProtocols.add("SSLv3"); + } + } } @@ -172,7 +184,7 @@ public void start() { if (sslEnabled) { - SslSocketConnector sslSocketConnector = new HTTPSourceSocketConnector(); + SslSocketConnector sslSocketConnector = new HTTPSourceSocketConnector(excludedProtocols); sslSocketConnector.setKeystore(keyStorePath); sslSocketConnector.setKeyPassword(keyStorePassword); sslSocketConnector.setReuseAddress(true); @@ -274,6 +286,11 @@ public void doGet(HttpServletRequest request, HttpServletResponse response) private static class HTTPSourceSocketConnector extends SslSocketConnector { + private final List excludedProtocols; + HTTPSourceSocketConnector(List excludedProtocols) { + this.excludedProtocols = excludedProtocols; + } + @Override public ServerSocket newServerSocket(String host, int port, int backlog) throws IOException { @@ -282,7 +299,7 @@ public ServerSocket newServerSocket(String host, int port, String[] protocols = socket.getEnabledProtocols(); List newProtocols = new ArrayList(protocols.length); for(String protocol: protocols) { - if (!(protocol.equals("SSLv3") || protocol.equals("SSLv2Hello"))) { + if (!excludedProtocols.contains(protocol)) { newProtocols.add(protocol); } } diff --git a/flume-ng-core/src/main/java/org/apache/flume/source/http/HTTPSourceConfigurationConstants.java b/flume-ng-core/src/main/java/org/apache/flume/source/http/HTTPSourceConfigurationConstants.java index ed52827ed8..86caf7d72d 100644 --- a/flume-ng-core/src/main/java/org/apache/flume/source/http/HTTPSourceConfigurationConstants.java +++ b/flume-ng-core/src/main/java/org/apache/flume/source/http/HTTPSourceConfigurationConstants.java @@ -37,5 +37,6 @@ public class HTTPSourceConfigurationConstants { public static final String SSL_KEYSTORE = "keystore"; public static final String SSL_KEYSTORE_PASSWORD = "keystorePassword"; public static final String SSL_ENABLED = "enableSSL"; + public static final String EXCLUDE_PROTOCOLS = "excludeProtocols"; } diff --git a/flume-ng-core/src/test/java/org/apache/flume/source/http/TestHTTPSource.java b/flume-ng-core/src/test/java/org/apache/flume/source/http/TestHTTPSource.java index 64111beeb7..c59fdd4b3b 100644 --- a/flume-ng-core/src/test/java/org/apache/flume/source/http/TestHTTPSource.java +++ b/flume-ng-core/src/test/java/org/apache/flume/source/http/TestHTTPSource.java @@ -321,11 +321,6 @@ public void testHttpsSSLv3() throws Exception { doTestHttps("SSLv3"); } - @Test (expected = javax.net.ssl.SSLHandshakeException.class) - public void testHttpsSSLv2Hello() throws Exception { - doTestHttps("SSLv2Hello"); - } - public void doTestHttps(String protocol) throws Exception { Type listType = new TypeToken>() { }.getType(); @@ -384,7 +379,7 @@ public boolean verify(String arg0, SSLSession arg1) { if(protocol != null) { factory = new DisabledProtocolsSocketFactory(sc.getSocketFactory(), protocol); } else { - factory = new EnabledProtocolsSocketFactory(sc.getSocketFactory()); + factory = sc.getSocketFactory(); } HttpsURLConnection.setDefaultSSLSocketFactory(factory); HttpsURLConnection.setDefaultHostnameVerifier( @@ -497,79 +492,9 @@ private class DisabledProtocolsSocketFactory extends javax.net.ssl.SSLSocketFact private final String[] protocols; DisabledProtocolsSocketFactory(javax.net.ssl.SSLSocketFactory factory, String protocol) { - this.socketFactory = factory; - if(protocol.equals("SSLv2Hello")) { - protocols = new String[2]; - protocols[0] = "TLSv1"; - protocols[1] = protocol; - } else { - protocols = new String[1]; - protocols[0] = protocol; - } - } - - @Override - public String[] getDefaultCipherSuites() { - return socketFactory.getDefaultCipherSuites(); - } - - @Override - public String[] getSupportedCipherSuites() { - return socketFactory.getSupportedCipherSuites(); - } - - @Override - public Socket createSocket(Socket socket, String s, int i, boolean b) - throws IOException { - SSLSocket sc = (SSLSocket) socketFactory.createSocket(socket, s, i, b); - sc.setEnabledProtocols(protocols); - return sc; - } - - @Override - public Socket createSocket(String s, int i) - throws IOException, UnknownHostException { - SSLSocket sc = (SSLSocket)socketFactory.createSocket(s, i); - sc.setEnabledProtocols(protocols); - return sc; - } - - @Override - public Socket createSocket(String s, int i, InetAddress inetAddress, int i2) - throws IOException, UnknownHostException { - SSLSocket sc = (SSLSocket)socketFactory.createSocket(s, i, inetAddress, - i2); - sc.setEnabledProtocols(protocols); - return sc; - } - - @Override - public Socket createSocket(InetAddress inetAddress, int i) - throws IOException { - SSLSocket sc = (SSLSocket)socketFactory.createSocket(inetAddress, i); - sc.setEnabledProtocols(protocols); - return sc; - } - - @Override - public Socket createSocket(InetAddress inetAddress, int i, - InetAddress inetAddress2, int i2) throws IOException { - SSLSocket sc = (SSLSocket)socketFactory.createSocket(inetAddress, i, - inetAddress2, i2); - sc.setEnabledProtocols(protocols); - return sc; - } - } - - private class EnabledProtocolsSocketFactory extends javax.net.ssl.SSLSocketFactory { - - private final javax.net.ssl.SSLSocketFactory socketFactory; - private final String[] protocols; - - EnabledProtocolsSocketFactory(javax.net.ssl.SSLSocketFactory factory) { this.socketFactory = factory; protocols = new String[1]; - protocols[0] = "TLSv1"; + protocols[0] = protocol; } @Override diff --git a/flume-ng-doc/sphinx/FlumeUserGuide.rst b/flume-ng-doc/sphinx/FlumeUserGuide.rst index bab01db0e4..122f2e8ea6 100644 --- a/flume-ng-doc/sphinx/FlumeUserGuide.rst +++ b/flume-ng-doc/sphinx/FlumeUserGuide.rst @@ -1351,22 +1351,23 @@ unavailable status. All events sent in one post request are considered to be one batch and inserted into the channel in one transaction. -============== ============================================ ==================================================================== -Property Name Default Description -============== ============================================ ==================================================================== -**type** The component type name, needs to be ``http`` -**port** -- The port the source should bind to. -bind 0.0.0.0 The hostname or IP address to listen on -handler ``org.apache.flume.source.http.JSONHandler`` The FQCN of the handler class. -handler.* -- Config parameters for the handler -selector.type replicating replicating or multiplexing -selector.* Depends on the selector.type value -interceptors -- Space-separated list of interceptors +================= ============================================ ===================================================================================== +Property Name Default Description +================= ============================================ ===================================================================================== +**type** The component type name, needs to be ``http`` +**port** -- The port the source should bind to. +bind 0.0.0.0 The hostname or IP address to listen on +handler ``org.apache.flume.source.http.JSONHandler`` The FQCN of the handler class. +handler.* -- Config parameters for the handler +selector.type replicating replicating or multiplexing +selector.* Depends on the selector.type value +interceptors -- Space-separated list of interceptors interceptors.* -enableSSL false Set the property true, to enable SSL -keystore Location of the keystore includng keystore file name -keystorePassword Keystore password -================================================================================================================================== +enableSSL false Set the property true, to enable SSL. *HTTP Source does not support SSLv3.* +excludeProtocols SSLv3 Space-separated list of SSL/TLS protocols to exclude. SSLv3 is always excluded. +keystore Location of the keystore includng keystore file name +keystorePassword Keystore password +====================================================================================================================================================== For example, a http source for agent named a1: From 293a8c0db2b3b2e696eab0e64bd83d140bfe56d8 Mon Sep 17 00:00:00 2001 From: Hari Shreedharan Date: Wed, 12 Nov 2014 11:48:34 -0800 Subject: [PATCH 176/341] FLUME-2551. Add dev-support directory to source tarball. --- flume-ng-dist/src/main/assembly/src.xml | 1 + 1 file changed, 1 insertion(+) diff --git a/flume-ng-dist/src/main/assembly/src.xml b/flume-ng-dist/src/main/assembly/src.xml index dc8b92ba17..b1e79a2279 100644 --- a/flume-ng-dist/src/main/assembly/src.xml +++ b/flume-ng-dist/src/main/assembly/src.xml @@ -112,6 +112,7 @@ pom.xml flume-ng-doc/** flume-ng-tests/** + dev-support/** From 2b2ac63b39734b084b1ed1ff884a9f0c3f7e3033 Mon Sep 17 00:00:00 2001 From: Hari Shreedharan Date: Wed, 12 Nov 2014 11:56:22 -0800 Subject: [PATCH 177/341] Update RELEASE-NOTES for 1.5.2 --- RELEASE-NOTES | 6 +++--- 1 file changed, 3 insertions(+), 3 deletions(-) diff --git a/RELEASE-NOTES b/RELEASE-NOTES index 6f22b21169..8a47eafbe7 100644 --- a/RELEASE-NOTES +++ b/RELEASE-NOTES @@ -1,4 +1,4 @@ -Apache Flume 1.5.1 +Apache Flume 1.5.2 CONTENTS 1. What is Apache Flume @@ -17,8 +17,8 @@ simple, extensible data model that allows for online analytic application. 2. Status of this release -Apache Flume 1.5.1 is the seventh release of Flume as an Apache top-level project -(TLP). Apache Flume 1.5.1 is production-ready software. +Apache Flume 1.5.2 is the eighth release of Flume as an Apache top-level project +(TLP). Apache Flume 1.5.2 is production-ready software. 3. Major changes in this Release For a detailed list of changes, please see the CHANGELOG file included From 619e78fe68658db242808a18f41ee5137b127748 Mon Sep 17 00:00:00 2001 From: Hari Shreedharan Date: Wed, 12 Nov 2014 12:00:53 -0800 Subject: [PATCH 178/341] Update CHANGELOG for 1.5.2 --- CHANGELOG | 12 ++++++++++++ 1 file changed, 12 insertions(+) diff --git a/CHANGELOG b/CHANGELOG index 7086807e85..cc7321361d 100644 --- a/CHANGELOG +++ b/CHANGELOG @@ -1,3 +1,15 @@ +Release Notes - Flume - Version v1.5.2 + +** Bug + * [FLUME-2547] - Removing SSLv2Hello causes Java 6 clients to break + +** Sub-task + * [FLUME-2548] - Enable SSLv2Hello for Avro Source and NettyAvroRpcClient + * [FLUME-2549] - Enable SSLv2Hello for HttpSource + +** Improvement + * [FLUME-2551] - Add dev-support to src tarball + Release Notes - Flume - Version v1.5.1 ** Bug From 698f0ba2de9c697ea9fe52134e36a694abc28d88 Mon Sep 17 00:00:00 2001 From: Hari Shreedharan Date: Thu, 13 Nov 2014 15:23:37 -0800 Subject: [PATCH 179/341] FLUME-2538. TestResettableFileInputStream fails on JDK 8. (Johny Rufus via Hari) --- .../serialization/TestResettableFileInputStream.java | 12 ++++++++++-- 1 file changed, 10 insertions(+), 2 deletions(-) diff --git a/flume-ng-core/src/test/java/org/apache/flume/serialization/TestResettableFileInputStream.java b/flume-ng-core/src/test/java/org/apache/flume/serialization/TestResettableFileInputStream.java index 73df9c9155..d1240fbead 100644 --- a/flume-ng-core/src/test/java/org/apache/flume/serialization/TestResettableFileInputStream.java +++ b/flume-ng-core/src/test/java/org/apache/flume/serialization/TestResettableFileInputStream.java @@ -139,8 +139,16 @@ public void testUtf8DecodeErrorHandlingReplace() throws IOException { while ((c = in.readChar()) != -1) { sb.append((char)c); } - assertEquals("Latin1: (X)\nLong: (XXX)\nNonUnicode: (X)\n" - .replaceAll("X", "\ufffd"), sb.toString()); + String preJdk8ExpectedStr = "Latin1: (X)\nLong: (XXX)\nNonUnicode: (X)\n"; + String expectedStr = "Latin1: (X)\nLong: (XXX)\nNonUnicode: (XXXXX)\n"; + String javaVersionStr = System.getProperty("java.version"); + double javaVersion = Double.parseDouble(javaVersionStr.substring(0, 3)); + + if(javaVersion < 1.8) { + assertTrue(preJdk8ExpectedStr.replaceAll("X", "\ufffd").equals(sb.toString())); + } else { + assertTrue(expectedStr.replaceAll("X", "\ufffd").equals(sb.toString())); + } } @Test(expected = MalformedInputException.class) From f20e0a9410481e874e5d4916ff3b32d7f893e780 Mon Sep 17 00:00:00 2001 From: Roshan Naik Date: Wed, 19 Nov 2014 18:25:00 -0800 Subject: [PATCH 180/341] FLUME-2509. Add description for properties in Failover Sink Processor (Ashish Paliwal via Roshan Naik) --- flume-ng-doc/sphinx/FlumeUserGuide.rst | 12 +++++++++--- 1 file changed, 9 insertions(+), 3 deletions(-) diff --git a/flume-ng-doc/sphinx/FlumeUserGuide.rst b/flume-ng-doc/sphinx/FlumeUserGuide.rst index 122f2e8ea6..43a4d5c62f 100644 --- a/flume-ng-doc/sphinx/FlumeUserGuide.rst +++ b/flume-ng-doc/sphinx/FlumeUserGuide.rst @@ -2704,7 +2704,12 @@ that so long as one is available events will be processed (delivered). The failover mechanism works by relegating failed sinks to a pool where they are assigned a cool down period, increasing with sequential failures before they are retried. Once a sink successfully sends an event, it is -restored to the live pool. +restored to the live pool. The Sinks have a priority associated with them, +larger the number, higher the priority. If a Sink fails while sending a Event +the next Sink with highest priority shall be tried next for sending Events. +For example, a sink with priority 100 is activated before the Sink with priority +80. If no priority is specified, thr priority is determined based on the order in which +the Sinks are specified in configuration. To configure, set a sink groups processor to ``failover`` and set priorities for all individual sinks. All specified priorities must @@ -2718,8 +2723,9 @@ Property Name Default Description ================================= =========== =================================================================================== **sinks** -- Space-separated list of sinks that are participating in the group **processor.type** ``default`` The component type name, needs to be ``failover`` -**processor.priority.** -- must be one of the sink instances associated with the current sink group -processor.maxpenalty 30000 (in millis) +**processor.priority.** -- Priority value. must be one of the sink instances associated with the current sink group + A higher priority value Sink gets activated earlier. A larger absolute value indicates higher priority +processor.maxpenalty 30000 The maximum backoff period for the failed Sink (in millis) ================================= =========== =================================================================================== Example for agent named a1: From 35de8ff18d096bd41caf3dabbcadc70b7d90ac40 Mon Sep 17 00:00:00 2001 From: Roshan Naik Date: Wed, 19 Nov 2014 18:39:43 -0800 Subject: [PATCH 181/341] FLUME-1594. Document JMX options (Ashish Paliwal via Roshan Naik) --- flume-ng-doc/sphinx/FlumeUserGuide.rst | 10 ++++++++++ 1 file changed, 10 insertions(+) diff --git a/flume-ng-doc/sphinx/FlumeUserGuide.rst b/flume-ng-doc/sphinx/FlumeUserGuide.rst index 43a4d5c62f..b9a8329075 100644 --- a/flume-ng-doc/sphinx/FlumeUserGuide.rst +++ b/flume-ng-doc/sphinx/FlumeUserGuide.rst @@ -3303,6 +3303,16 @@ Monitoring in Flume is still a work in progress. Changes can happen very often. Several Flume components report metrics to the JMX platform MBean server. These metrics can be queried using Jconsole. +JMX Reporting +------------- + +JMX Reporting can be enabled by specifying JMX parameters in the JAVA_OPTS environment variable using +flume-env.sh, like + + export JAVA_OPTS="-Dcom.sun.management.jmxremote -Dcom.sun.management.jmxremote.port=5445 -Dcom.sun.management.jmxremote.authenticate=false -Dcom.sun.management.jmxremote.ssl=false" + +NOTE: The sample above disables the security. To enable Security, please refer http://docs.oracle.com/javase/6/docs/technotes/guides/management/agent.html + Ganglia Reporting ----------------- Flume can also report these metrics to From a73b6785332a55eeb82f4a915e09cee305714959 Mon Sep 17 00:00:00 2001 From: Roshan Naik Date: Wed, 19 Nov 2014 19:24:48 -0800 Subject: [PATCH 182/341] FLUME-1521. Document the StressSource (Ashish Paliwal via Roshan Naik) --- flume-ng-doc/sphinx/FlumeUserGuide.rst | 31 ++++++++++++++++++++++++++ 1 file changed, 31 insertions(+) diff --git a/flume-ng-doc/sphinx/FlumeUserGuide.rst b/flume-ng-doc/sphinx/FlumeUserGuide.rst index b9a8329075..0199d62136 100644 --- a/flume-ng-doc/sphinx/FlumeUserGuide.rst +++ b/flume-ng-doc/sphinx/FlumeUserGuide.rst @@ -1433,6 +1433,37 @@ Property Name Default Description handler.maxBlobLength 100000000 The maximum number of bytes to read and buffer for a given request ===================== ================== ============================================================================ +Stress Source +~~~~~~~~~~~~~ + +StressSource is an internal load-generating source implementation which is very useful for +stress tests. It allows User to configure the size of Event payload, with empty headers. +User can configure total number of events to be sent as well maximum number of Successful +Event to be delivered. + +Required properties are in **bold**. + +=================== =========== =================================================== +Property Name Default Description +=================== =========== =================================================== +**type** -- The component type name, needs to be ``org.apache.flume.source.StressSource`` +size 500 Payload size of each Event. Unit:**byte** +maxTotalEvents -1 Maximum number of Events to be sent +maxSuccessfulEvents -1 Maximum number of Events successfully sent +batchSize 1 Number of Events to be sent in one batch +=================== =========== =================================================== + +Example for agent named **a1**: + +.. code-block:: properties + + a1.sources = stresssource-1 + a1.channels = memoryChannel-1 + a1.sources.stresssource-1.type = org.apache.flume.source.StressSource + a1.sources.stresssource-1.size = 10240 + a1.sources.stresssource-1.maxTotalEvents = 1000000 + a1.sources.stresssource-1.channels = memoryChannel-1 + Legacy Sources ~~~~~~~~~~~~~~ From 55a4666434d892f838208b315cd9ba06ea07659b Mon Sep 17 00:00:00 2001 From: Roshan Naik Date: Wed, 19 Nov 2014 19:58:31 -0800 Subject: [PATCH 183/341] FLUME-2487. SyslogParser has rounding errors in timestamp parsing (Santiago M. Mola via Roshan Naik) --- .../java/org/apache/flume/source/SyslogParser.java | 11 ++++++++--- .../org/apache/flume/source/TestSyslogParser.java | 3 ++- 2 files changed, 10 insertions(+), 4 deletions(-) diff --git a/flume-ng-core/src/main/java/org/apache/flume/source/SyslogParser.java b/flume-ng-core/src/main/java/org/apache/flume/source/SyslogParser.java index 01713096c2..b61f7459ec 100644 --- a/flume-ng-core/src/main/java/org/apache/flume/source/SyslogParser.java +++ b/flume-ng-core/src/main/java/org/apache/flume/source/SyslogParser.java @@ -237,9 +237,14 @@ protected long parseRfc5424Date(String msg) { } // if they had a valid fractional second, append it rounded to millis - if (endMillisPos - (curPos + 1) > 0) { - float frac = Float.parseFloat(msg.substring(curPos, endMillisPos)); - long milliseconds = (long) (frac * 1000f); + final int fractionalPositions = endMillisPos - (curPos + 1); + if (fractionalPositions > 0) { + long milliseconds = Long.parseLong(msg.substring(curPos + 1, endMillisPos)); + if (fractionalPositions > 3) { + milliseconds /= Math.pow(10, (fractionalPositions - 3)); + } else if (fractionalPositions < 3) { + milliseconds *= Math.pow(10, (3 - fractionalPositions)); + } ts += milliseconds; } else { throw new IllegalArgumentException( diff --git a/flume-ng-core/src/test/java/org/apache/flume/source/TestSyslogParser.java b/flume-ng-core/src/test/java/org/apache/flume/source/TestSyslogParser.java index 6e0fd665c4..265157eb5f 100644 --- a/flume-ng-core/src/test/java/org/apache/flume/source/TestSyslogParser.java +++ b/flume-ng-core/src/test/java/org/apache/flume/source/TestSyslogParser.java @@ -37,7 +37,8 @@ public void testRfc5424DateParsing() { final String[] examples = { "1985-04-12T23:20:50.52Z", "1985-04-12T19:20:50.52-04:00", "2003-10-11T22:14:15.003Z", "2003-08-24T05:14:15.000003-07:00", - "2012-04-13T11:11:11-08:00", "2012-04-13T08:08:08.0001+00:00" + "2012-04-13T11:11:11-08:00", "2012-04-13T08:08:08.0001+00:00", + "2012-04-13T08:08:08.251+00:00" }; SyslogParser parser = new SyslogParser(); From 22cd3909bbc0d9dbdb42b845f36be133326eb67c Mon Sep 17 00:00:00 2001 From: Roshan Naik Date: Wed, 19 Nov 2014 20:14:11 -0800 Subject: [PATCH 184/341] FLUME-2480. Typo in file channel exception: "Usable space exhaused" (Santiago M. Mola via Roshan Naik) --- .../main/java/org/apache/flume/channel/file/Log.java | 10 +++++----- .../java/org/apache/flume/channel/file/TestLog.java | 4 ++-- 2 files changed, 7 insertions(+), 7 deletions(-) diff --git a/flume-ng-channels/flume-file-channel/src/main/java/org/apache/flume/channel/file/Log.java b/flume-ng-channels/flume-file-channel/src/main/java/org/apache/flume/channel/file/Log.java index e797797d3c..0e9171e0ea 100644 --- a/flume-ng-channels/flume-file-channel/src/main/java/org/apache/flume/channel/file/Log.java +++ b/flume-ng-channels/flume-file-channel/src/main/java/org/apache/flume/channel/file/Log.java @@ -624,7 +624,7 @@ FlumeEventPointer put(long transactionID, Event event) long usableSpace = logFiles.get(logFileIndex).getUsableSpace(); long requiredSpace = minimumRequiredSpace + buffer.limit(); if(usableSpace <= requiredSpace) { - throw new IOException("Usable space exhaused, only " + usableSpace + + throw new IOException("Usable space exhausted, only " + usableSpace + " bytes remaining, required " + requiredSpace + " bytes"); } boolean error = true; @@ -667,7 +667,7 @@ void take(long transactionID, FlumeEventPointer pointer) long usableSpace = logFiles.get(logFileIndex).getUsableSpace(); long requiredSpace = minimumRequiredSpace + buffer.limit(); if(usableSpace <= requiredSpace) { - throw new IOException("Usable space exhaused, only " + usableSpace + + throw new IOException("Usable space exhausted, only " + usableSpace + " bytes remaining, required " + requiredSpace + " bytes"); } boolean error = true; @@ -709,7 +709,7 @@ void rollback(long transactionID) throws IOException { long usableSpace = logFiles.get(logFileIndex).getUsableSpace(); long requiredSpace = minimumRequiredSpace + buffer.limit(); if(usableSpace <= requiredSpace) { - throw new IOException("Usable space exhaused, only " + usableSpace + + throw new IOException("Usable space exhausted, only " + usableSpace + " bytes remaining, required " + requiredSpace + " bytes"); } boolean error = true; @@ -866,7 +866,7 @@ private void commit(long transactionID, short type) throws IOException { long usableSpace = logFiles.get(logFileIndex).getUsableSpace(); long requiredSpace = minimumRequiredSpace + buffer.limit(); if(usableSpace <= requiredSpace) { - throw new IOException("Usable space exhaused, only " + usableSpace + + throw new IOException("Usable space exhausted, only " + usableSpace + " bytes remaining, required " + requiredSpace + " bytes"); } boolean error = true; @@ -982,7 +982,7 @@ private Boolean writeCheckpoint(Boolean force) throws Exception { boolean checkpointCompleted = false; long usableSpace = checkpointDir.getUsableSpace(); if(usableSpace <= minimumRequiredSpace) { - throw new IOException("Usable space exhaused, only " + usableSpace + + throw new IOException("Usable space exhausted, only " + usableSpace + " bytes remaining, required " + minimumRequiredSpace + " bytes"); } lockExclusive(); diff --git a/flume-ng-channels/flume-file-channel/src/test/java/org/apache/flume/channel/file/TestLog.java b/flume-ng-channels/flume-file-channel/src/test/java/org/apache/flume/channel/file/TestLog.java index c9a64edb91..f7f1afa3b2 100644 --- a/flume-ng-channels/flume-file-channel/src/test/java/org/apache/flume/channel/file/TestLog.java +++ b/flume-ng-channels/flume-file-channel/src/test/java/org/apache/flume/channel/file/TestLog.java @@ -168,7 +168,7 @@ public void testMinimumRequiredSpaceTooSmallOnStartup() throws IOException, Assert.fail(); } catch (IOException e) { Assert.assertTrue(e.getMessage(), e.getMessage() - .startsWith("Usable space exhaused")); + .startsWith("Usable space exhausted")); } } /** @@ -214,7 +214,7 @@ public void doTestMinimumRequiredSpaceTooSmallForPut() throws IOException, Assert.fail(); } catch (IOException e) { Assert.assertTrue(e.getMessage(), e.getMessage() - .startsWith("Usable space exhaused")); + .startsWith("Usable space exhausted")); } } /** From ac999bebeaa59659960fe9e998a0c20b1d476564 Mon Sep 17 00:00:00 2001 From: Roshan Naik Date: Wed, 19 Nov 2014 20:28:23 -0800 Subject: [PATCH 185/341] FLUME-2246. Make event data size configurable for logger sinker (Ashish Paliwal via Roshan Naik) --- .../org/apache/flume/sink/LoggerSink.java | 32 ++++++++++++++++--- .../org/apache/flume/sink/TestLoggerSink.java | 23 +++++++++++++ flume-ng-doc/sphinx/FlumeUserGuide.rst | 1 + 3 files changed, 52 insertions(+), 4 deletions(-) diff --git a/flume-ng-core/src/main/java/org/apache/flume/sink/LoggerSink.java b/flume-ng-core/src/main/java/org/apache/flume/sink/LoggerSink.java index 128fa8427a..9cf9bc2b1d 100644 --- a/flume-ng-core/src/main/java/org/apache/flume/sink/LoggerSink.java +++ b/flume-ng-core/src/main/java/org/apache/flume/sink/LoggerSink.java @@ -18,18 +18,20 @@ package org.apache.flume.sink; +import com.google.common.base.Strings; import org.apache.flume.Channel; +import org.apache.flume.Context; import org.apache.flume.Event; import org.apache.flume.EventDeliveryException; -import org.apache.flume.Sink; import org.apache.flume.Transaction; +import org.apache.flume.conf.Configurable; import org.apache.flume.event.EventHelper; import org.slf4j.Logger; import org.slf4j.LoggerFactory; /** *

    - * A {@link Sink} implementation that logs all events received at the INFO level + * A {@link org.apache.flume.Sink} implementation that logs all events received at the INFO level * to the org.apache.flume.sink.LoggerSink logger. *

    *

    @@ -49,11 +51,33 @@ * TODO *

    */ -public class LoggerSink extends AbstractSink { +public class LoggerSink extends AbstractSink implements Configurable { private static final Logger logger = LoggerFactory .getLogger(LoggerSink.class); + // Default Max bytes to dump + public static final int DEFAULT_MAX_BYTE_DUMP = 16; + + // Max number of bytes to be dumped + private int maxBytesToLog = DEFAULT_MAX_BYTE_DUMP; + + public static final String MAX_BYTES_DUMP_KEY = "maxBytesToLog"; + + @Override + public void configure(Context context) { + String strMaxBytes = context.getString(MAX_BYTES_DUMP_KEY); + if (!Strings.isNullOrEmpty(strMaxBytes)) { + try { + maxBytesToLog = Integer.parseInt(strMaxBytes); + } catch (NumberFormatException e) { + logger.warn(String.format("Unable to convert %s to integer, using default value(%d) for maxByteToDump", + strMaxBytes, DEFAULT_MAX_BYTE_DUMP)); + maxBytesToLog = DEFAULT_MAX_BYTE_DUMP; + } + } + } + @Override public Status process() throws EventDeliveryException { Status result = Status.READY; @@ -67,7 +91,7 @@ public Status process() throws EventDeliveryException { if (event != null) { if (logger.isInfoEnabled()) { - logger.info("Event: " + EventHelper.dumpEvent(event)); + logger.info("Event: " + EventHelper.dumpEvent(event, maxBytesToLog)); } } else { // No event found, request back-off semantics from the sink runner diff --git a/flume-ng-core/src/test/java/org/apache/flume/sink/TestLoggerSink.java b/flume-ng-core/src/test/java/org/apache/flume/sink/TestLoggerSink.java index 92ff6fee9f..3257cedb9f 100644 --- a/flume-ng-core/src/test/java/org/apache/flume/sink/TestLoggerSink.java +++ b/flume-ng-core/src/test/java/org/apache/flume/sink/TestLoggerSink.java @@ -19,6 +19,7 @@ package org.apache.flume.sink; +import com.google.common.base.Strings; import org.apache.flume.Channel; import org.apache.flume.Context; import org.apache.flume.Event; @@ -56,6 +57,28 @@ public void testAppend() throws InterruptedException, LifecycleException, for (int i = 0; i < 10; i++) { Event event = EventBuilder.withBody(("Test " + i).getBytes()); + channel.put(event); + sink.process(); + } + + sink.stop(); + } + + @Test + public void testAppendWithCustomSize() throws InterruptedException, LifecycleException, + EventDeliveryException { + + Channel channel = new PseudoTxnMemoryChannel(); + Context context = new Context(); + context.put(LoggerSink.MAX_BYTES_DUMP_KEY, String.valueOf(30)); + Configurables.configure(channel, context); + Configurables.configure(sink, context); + + sink.setChannel(channel); + sink.start(); + + for (int i = 0; i < 10; i++) { + Event event = EventBuilder.withBody((Strings.padStart("Test " + i, 30, 'P')).getBytes()); channel.put(event); sink.process(); diff --git a/flume-ng-doc/sphinx/FlumeUserGuide.rst b/flume-ng-doc/sphinx/FlumeUserGuide.rst index 0199d62136..bcadc2d14d 100644 --- a/flume-ng-doc/sphinx/FlumeUserGuide.rst +++ b/flume-ng-doc/sphinx/FlumeUserGuide.rst @@ -1732,6 +1732,7 @@ Property Name Default Description ============== ======= =========================================== **channel** -- **type** -- The component type name, needs to be ``logger`` +maxBytesToLog 16 Maximum number of bytes of the Event body to log ============== ======= =========================================== Example for agent named a1: From 9dd7d95357f925134b0ef2aadd8636722f57533d Mon Sep 17 00:00:00 2001 From: Hari Shreedharan Date: Thu, 20 Nov 2014 17:04:14 -0800 Subject: [PATCH 186/341] FLUME-2560. Add Kafka Channel to dist tarballs. (Johny Rufus via Hari) --- flume-ng-dist/pom.xml | 4 ++++ pom.xml | 6 ++++++ 2 files changed, 10 insertions(+) diff --git a/flume-ng-dist/pom.xml b/flume-ng-dist/pom.xml index a5db0c7bc8..4d2ebfec86 100644 --- a/flume-ng-dist/pom.xml +++ b/flume-ng-dist/pom.xml @@ -129,6 +129,10 @@ org.apache.flume.flume-ng-channels flume-jdbc-channel + + org.apache.flume.flume-ng-channels + flume-kafka-channel + org.apache.flume.flume-ng-sinks flume-hdfs-sink diff --git a/pom.xml b/pom.xml index 4f550d388b..7d4f01a5a6 100644 --- a/pom.xml +++ b/pom.xml @@ -1097,6 +1097,12 @@ limitations under the License. 1.6.0-SNAPSHOT + + org.apache.flume.flume-ng-channels + flume-kafka-channel + 1.6.0-SNAPSHOT + + org.apache.flume.flume-ng-channels flume-spillable-memory-channel From cec64a54abb8410a5e98cd2f815a5a839b49ce74 Mon Sep 17 00:00:00 2001 From: Mike Percy Date: Thu, 20 Nov 2014 20:39:23 -0800 Subject: [PATCH 187/341] FLUME-2557. Make DatasetSink work with Kerberos (Joey Echeverria via Mike Percy) --- flume-ng-sinks/flume-dataset-sink/pom.xml | 20 ++- .../apache/flume/sink/kite/KerberosUtil.java | 15 ++- .../flume/sink/kite/TestKerberosUtil.java | 121 ++++++++++++++++++ .../src/test/resources/enable-kerberos.xml | 30 +++++ 4 files changed, 181 insertions(+), 5 deletions(-) create mode 100644 flume-ng-sinks/flume-dataset-sink/src/test/java/org/apache/flume/sink/kite/TestKerberosUtil.java create mode 100644 flume-ng-sinks/flume-dataset-sink/src/test/resources/enable-kerberos.xml diff --git a/flume-ng-sinks/flume-dataset-sink/pom.xml b/flume-ng-sinks/flume-dataset-sink/pom.xml index 0b447a5330..e929d60544 100644 --- a/flume-ng-sinks/flume-dataset-sink/pom.xml +++ b/flume-ng-sinks/flume-dataset-sink/pom.xml @@ -16,7 +16,7 @@ See the License for the specific language governing permissions and limitations under the License. --> + xsi:schemaLocation="http://maven.apache.org/POM/4.0.0 http://maven.apache.org/xsd/maven-4.0.0.xsd"> 4.0.0 @@ -36,6 +36,13 @@ limitations under the License. org.apache.rat apache-rat-plugin + + org.apache.felix + maven-bundle-plugin + 2.3.7 + true + true + @@ -111,8 +118,8 @@ limitations under the License. + because kite uses hflush. + --> org.apache.hadoop hadoop-common ${hadoop2.version} @@ -142,6 +149,13 @@ limitations under the License. test + + org.apache.hadoop + hadoop-minikdc + ${hadoop2.version} + test + + org.slf4j slf4j-log4j12 diff --git a/flume-ng-sinks/flume-dataset-sink/src/main/java/org/apache/flume/sink/kite/KerberosUtil.java b/flume-ng-sinks/flume-dataset-sink/src/main/java/org/apache/flume/sink/kite/KerberosUtil.java index 92ad1413d2..c0dbffbd64 100644 --- a/flume-ng-sinks/flume-dataset-sink/src/main/java/org/apache/flume/sink/kite/KerberosUtil.java +++ b/flume-ng-sinks/flume-dataset-sink/src/main/java/org/apache/flume/sink/kite/KerberosUtil.java @@ -82,6 +82,16 @@ public static UserGroupInformation proxyAs(String username, */ public static synchronized UserGroupInformation login(String principal, String keytab) { + // If the principal or keytab isn't set, get the current (Linux) user + if (principal == null || keytab == null) { + try { + return UserGroupInformation.getCurrentUser(); + } catch (IOException ex) { + LOG.error("Can't get current user: {}", ex.getMessage()); + throw new RuntimeException(ex); + } + } + // resolve the requested principal, if it is present String finalPrincipal = null; if (principal != null && !principal.isEmpty()) { @@ -105,8 +115,9 @@ public static synchronized UserGroupInformation login(String principal, LOG.debug("Unable to get login user before Kerberos auth attempt", e); } - // if the current user is valid (matches the given principal) then use it - if (currentUser != null) { + // if the current user is valid (matches the given principal and has a TGT) + // then use it + if (currentUser != null && currentUser.hasKerberosCredentials()) { if (finalPrincipal == null || finalPrincipal.equals(currentUser.getUserName())) { LOG.debug("Using existing login for {}: {}", diff --git a/flume-ng-sinks/flume-dataset-sink/src/test/java/org/apache/flume/sink/kite/TestKerberosUtil.java b/flume-ng-sinks/flume-dataset-sink/src/test/java/org/apache/flume/sink/kite/TestKerberosUtil.java new file mode 100644 index 0000000000..f53ef73597 --- /dev/null +++ b/flume-ng-sinks/flume-dataset-sink/src/test/java/org/apache/flume/sink/kite/TestKerberosUtil.java @@ -0,0 +1,121 @@ +/** + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package org.apache.flume.sink.kite; + +import java.io.File; +import java.io.IOException; +import java.net.URL; +import java.util.Properties; +import org.apache.hadoop.conf.Configuration; +import org.apache.hadoop.minikdc.MiniKdc; +import org.apache.hadoop.security.UserGroupInformation; +import org.junit.AfterClass; +import org.junit.BeforeClass; +import org.junit.Test; +import static org.junit.Assert.*; + +public class TestKerberosUtil { + + private static MiniKdc kdc; + private static File workDir; + private static File flumeKeytab; + private static String flumePrincipal = "flume/localhost"; + private static File aliceKeytab; + private static String alicePrincipal = "alice"; + private static Properties conf; + + @BeforeClass + public static void startMiniKdc() throws Exception { + URL resource = Thread.currentThread() + .getContextClassLoader().getResource("enable-kerberos.xml"); + Configuration.addDefaultResource("enable-kerberos.xml"); + + workDir = new File(System.getProperty("test.dir", "target"), + TestKerberosUtil.class.getSimpleName()); + flumeKeytab = new File(workDir, "flume.keytab"); + aliceKeytab = new File(workDir, "alice.keytab"); + conf = MiniKdc.createConf(); + + kdc = new MiniKdc(conf, workDir); + kdc.start(); + + kdc.createPrincipal(flumeKeytab, flumePrincipal); + flumePrincipal = flumePrincipal + "@" + kdc.getRealm(); + + kdc.createPrincipal(aliceKeytab, alicePrincipal); + alicePrincipal = alicePrincipal + "@" + kdc.getRealm(); + } + + @AfterClass + public static void stopMiniKdc() { + if (kdc != null) { + kdc.stop(); + } + } + + @Test + public void testNullLogin() throws IOException { + String principal = null; + String keytab = null; + UserGroupInformation expResult = UserGroupInformation.getCurrentUser(); + UserGroupInformation result = KerberosUtil.login(principal, keytab); + assertEquals(expResult, result); + } + + @Test + public void testFlumeLogin() throws IOException { + String principal = flumePrincipal; + String keytab = flumeKeytab.getAbsolutePath(); + String expResult = principal; + + String result = KerberosUtil.login(principal, keytab).getUserName(); + assertEquals("Initial login failed", expResult, result); + + result = KerberosUtil.login(principal, keytab).getUserName(); + assertEquals("Re-login failed", expResult, result); + + principal = alicePrincipal; + keytab = aliceKeytab.getAbsolutePath(); + try { + result = KerberosUtil.login(principal, keytab).getUserName(); + fail("Login should have failed with a new principal: " + result); + } catch (KerberosUtil.SecurityException ex) { + assertTrue("Login with a new principal failed, but for an unexpected " + + "reason: " + ex.getMessage(), + ex.getMessage().contains("Cannot use multiple Kerberos principals: ")); + } + } + + @Test + public void testProxyAs() throws IOException { + String username = "alice"; + + UserGroupInformation login = UserGroupInformation.getCurrentUser(); + String expResult = username; + String result = KerberosUtil.proxyAs(username, login).getUserName(); + assertEquals("Proxy as didn't generate the expected username", expResult, result); + + login = KerberosUtil.login(flumePrincipal, flumeKeytab.getAbsolutePath()); + assertEquals("Login succeeded, but the principal doesn't match", + flumePrincipal, login.getUserName()); + + result = KerberosUtil.proxyAs(username, login).getUserName(); + assertEquals("Proxy as didn't generate the expected username", expResult, result); + } + +} diff --git a/flume-ng-sinks/flume-dataset-sink/src/test/resources/enable-kerberos.xml b/flume-ng-sinks/flume-dataset-sink/src/test/resources/enable-kerberos.xml new file mode 100644 index 0000000000..85b04472a2 --- /dev/null +++ b/flume-ng-sinks/flume-dataset-sink/src/test/resources/enable-kerberos.xml @@ -0,0 +1,30 @@ + + + + + + + hadoop.security.authentication + kerberos + + + + hadoop.security.authorization + true + + + From d1227feb601f30d0c7ec2910b91e648ee410d179 Mon Sep 17 00:00:00 2001 From: Jarek Jarcec Cecho Date: Fri, 21 Nov 2014 17:28:37 -0800 Subject: [PATCH 188/341] FLUME-2488: TestElasticSearchRestClient fails on Oracle JDK 8 (Johny Rufus via Jarek Jarcec Cecho) --- .../client/TestElasticSearchRestClient.java | 23 +++++++++++++++---- 1 file changed, 19 insertions(+), 4 deletions(-) diff --git a/flume-ng-sinks/flume-ng-elasticsearch-sink/src/test/java/org/apache/flume/sink/elasticsearch/client/TestElasticSearchRestClient.java b/flume-ng-sinks/flume-ng-elasticsearch-sink/src/test/java/org/apache/flume/sink/elasticsearch/client/TestElasticSearchRestClient.java index b7d8822696..1fe983a8da 100644 --- a/flume-ng-sinks/flume-ng-elasticsearch-sink/src/test/java/org/apache/flume/sink/elasticsearch/client/TestElasticSearchRestClient.java +++ b/flume-ng-sinks/flume-ng-elasticsearch-sink/src/test/java/org/apache/flume/sink/elasticsearch/client/TestElasticSearchRestClient.java @@ -18,6 +18,9 @@ */ package org.apache.flume.sink.elasticsearch.client; +import com.google.common.base.Splitter; +import com.google.gson.JsonObject; +import com.google.gson.JsonParser; import org.apache.flume.Event; import org.apache.flume.EventDeliveryException; import org.apache.flume.sink.elasticsearch.ElasticSearchEventSerializer; @@ -30,9 +33,11 @@ import org.mockito.Mock; import java.io.IOException; +import java.util.Iterator; import java.util.List; import static junit.framework.Assert.assertEquals; +import static junit.framework.Assert.assertTrue; import org.apache.http.HttpEntity; import org.apache.http.HttpResponse; import org.apache.http.HttpStatus; @@ -102,8 +107,8 @@ public void shouldAddNewEventWithoutTTL() throws Exception { verify(httpClient).execute(argument.capture()); assertEquals("http://host1/_bulk", argument.getValue().getURI().toString()); - assertEquals("{\"index\":{\"_type\":\"bar_type\",\"_index\":\"foo_index\"}}\n" + MESSAGE_CONTENT + "\n", - EntityUtils.toString(argument.getValue().getEntity())); + assertTrue(verifyJsonEvents("{\"index\":{\"_type\":\"bar_type\", \"_index\":\"foo_index\"}}\n", + MESSAGE_CONTENT, EntityUtils.toString(argument.getValue().getEntity()))); } @Test @@ -121,8 +126,18 @@ public void shouldAddNewEventWithTTL() throws Exception { verify(httpClient).execute(argument.capture()); assertEquals("http://host1/_bulk", argument.getValue().getURI().toString()); - assertEquals("{\"index\":{\"_type\":\"bar_type\",\"_index\":\"foo_index\",\"_ttl\":\"123\"}}\n" + - MESSAGE_CONTENT + "\n", EntityUtils.toString(argument.getValue().getEntity())); + assertTrue(verifyJsonEvents("{\"index\":{\"_type\":\"bar_type\",\"_index\":\"foo_index\",\"_ttl\":\"123\"}}\n", + MESSAGE_CONTENT, EntityUtils.toString(argument.getValue().getEntity()))); + } + + private boolean verifyJsonEvents(String expectedIndex, String expectedBody, String actual) { + Iterator it = Splitter.on("\n").split(actual).iterator(); + JsonParser parser = new JsonParser(); + JsonObject[] arr = new JsonObject[2]; + for(int i = 0; i < 2; i++) { + arr[i] = (JsonObject) parser.parse(it.next()); + } + return arr[0].equals(parser.parse(expectedIndex)) && arr[1].equals(parser.parse(expectedBody)); } @Test(expected = EventDeliveryException.class) From 1003d1f410bd5f9e94ebafe6df641d8ab3e90d3e Mon Sep 17 00:00:00 2001 From: Roshan Naik Date: Tue, 25 Nov 2014 21:00:11 -0800 Subject: [PATCH 189/341] FLUME-2558. Update javadoc for StressSource (Ashish Paliwal via Roshan Naik) --- .../org/apache/flume/source/StressSource.java | 25 ++++++++++++++----- 1 file changed, 19 insertions(+), 6 deletions(-) diff --git a/flume-ng-core/src/main/java/org/apache/flume/source/StressSource.java b/flume-ng-core/src/main/java/org/apache/flume/source/StressSource.java index 562b9836e7..0e7020b792 100644 --- a/flume-ng-core/src/main/java/org/apache/flume/source/StressSource.java +++ b/flume-ng-core/src/main/java/org/apache/flume/source/StressSource.java @@ -35,7 +35,21 @@ import org.slf4j.LoggerFactory; /** - * Internal load-generating source implementation. Useful for tests. + * StressSource is an internal load-generating source implementation + * which is very useful for stress tests. It allows User to configure + * the size of Event payload, with empty headers. User can configure + * total number of events to be sent as well maximum number of Successful + * Events to be delivered. Useful for tests + * + * Example configuration for Agent a1 + *
    + *   a1.sources = stresssource-1
    + *   a1.channels = memoryChannel-1
    + *   a1.sources.stresssource-1.type = org.apache.flume.source.StressSource
    + *   a1.sources.stresssource-1.size = 10240
    + *   a1.sources.stresssource-1.maxTotalEvents = 1000000
    + *   a1.sources.stresssource-1.channels = memoryChannel-1
    + * 
    * * See {@link StressSource#configure(Context)} for configuration options. */ @@ -57,15 +71,14 @@ public class StressSource extends AbstractSource implements public StressSource() { counterGroup = new CounterGroup(); - } /** * Read parameters from context - *
  • -maxTotalEvents = type long that defines the total number of events to be sent - *
  • -maxSuccessfulEvents = type long that defines the total number of events to be sent - *
  • -size = type int that defines the number of bytes in each event - *
  • -batchSize = type int that defines the number of events being sent in one batch + *
  • -maxTotalEvents = type long that defines the total number of Events to be sent + *
  • -maxSuccessfulEvents = type long that defines the number of successful Events + *
  • -size = type int that defines the number of bytes in each Event + *
  • -batchSize = type int that defines the number of Events being sent in one batch */ @Override public void configure(Context context) { From 34e9bda312506a118fad87fcbdecc48bf3918c95 Mon Sep 17 00:00:00 2001 From: Hari Shreedharan Date: Thu, 18 Dec 2014 12:13:19 -0800 Subject: [PATCH 190/341] FLUME-2373. Support TBinaryProtocol in Thrift RPC. (Stefan Krawczyk via Hari) --- .../org/apache/flume/sink/ThriftSink.java | 4 +-- .../org/apache/flume/source/ThriftSource.java | 29 +++++++++++++++-- .../org/apache/flume/sink/TestThriftSink.java | 11 ++++--- .../org/apache/flume/api/ThriftRpcClient.java | 32 +++++++++++++++++-- .../apache/flume/api/TestThriftRpcClient.java | 13 ++++---- .../apache/flume/api/ThriftTestingSource.java | 15 +++++++-- 6 files changed, 85 insertions(+), 19 deletions(-) diff --git a/flume-ng-core/src/main/java/org/apache/flume/sink/ThriftSink.java b/flume-ng-core/src/main/java/org/apache/flume/sink/ThriftSink.java index 48a97751f0..baa60d0cf1 100644 --- a/flume-ng-core/src/main/java/org/apache/flume/sink/ThriftSink.java +++ b/flume-ng-core/src/main/java/org/apache/flume/sink/ThriftSink.java @@ -18,11 +18,11 @@ */ package org.apache.flume.sink; +import java.util.Properties; + import org.apache.flume.api.RpcClient; import org.apache.flume.api.RpcClientConfigurationConstants; import org.apache.flume.api.RpcClientFactory; - -import java.util.Properties; /** *

    * A {@link org.apache.flume.Sink} implementation that can send events to an RPC server (such as diff --git a/flume-ng-core/src/main/java/org/apache/flume/source/ThriftSource.java b/flume-ng-core/src/main/java/org/apache/flume/source/ThriftSource.java index c3881b46e1..551fe133c7 100644 --- a/flume-ng-core/src/main/java/org/apache/flume/source/ThriftSource.java +++ b/flume-ng-core/src/main/java/org/apache/flume/source/ThriftSource.java @@ -34,6 +34,7 @@ import org.apache.flume.thrift.ThriftFlumeEvent; import org.apache.thrift.TException; import org.apache.thrift.protocol.TCompactProtocol; +import org.apache.thrift.protocol.TBinaryProtocol; import org.apache.thrift.server.TNonblockingServer; import org.apache.thrift.server.TServer; import org.apache.thrift.transport.TFastFramedTransport; @@ -70,6 +71,13 @@ public class ThriftSource extends AbstractSource implements Configurable, * Config param for the port to listen on. */ public static final String CONFIG_PORT = "port"; + /** + * Config param for the thrift protocol to use. + */ + public static final String CONFIG_PROTOCOL = "protocol"; + public static final String BINARY_PROTOCOL = "binary"; + public static final String COMPACT_PROTOCOL = "compact"; + private Integer port; private String bindAddress; private int maxThreads = 0; @@ -77,6 +85,7 @@ public class ThriftSource extends AbstractSource implements Configurable, private TServer server; private TServerTransport serverTransport; private ExecutorService servingExecutor; + private String protocol; @Override public void configure(Context context) { @@ -98,6 +107,17 @@ public void configure(Context context) { if (sourceCounter == null) { sourceCounter = new SourceCounter(getName()); } + + protocol = context.getString(CONFIG_PROTOCOL); + if (protocol == null) { + // default is to use the compact protocol. + protocol = COMPACT_PROTOCOL; + } + Preconditions.checkArgument( + (protocol.equalsIgnoreCase(BINARY_PROTOCOL) || + protocol.equalsIgnoreCase(COMPACT_PROTOCOL)), + "binary or compact are the only valid Thrift protocol types to " + + "choose from."); } @Override @@ -167,8 +187,13 @@ public void start() { } try { - - args.protocolFactory(new TCompactProtocol.Factory()); + if (protocol.equals(BINARY_PROTOCOL)) { + logger.info("Using TBinaryProtocol"); + args.protocolFactory(new TBinaryProtocol.Factory()); + } else { + logger.info("Using TCompactProtocol"); + args.protocolFactory(new TCompactProtocol.Factory()); + } args.inputTransportFactory(new TFastFramedTransport.Factory()); args.outputTransportFactory(new TFastFramedTransport.Factory()); args.processor(new ThriftSourceProtocol diff --git a/flume-ng-core/src/test/java/org/apache/flume/sink/TestThriftSink.java b/flume-ng-core/src/test/java/org/apache/flume/sink/TestThriftSink.java index 5f70d1b877..fccaede5cf 100644 --- a/flume-ng-core/src/test/java/org/apache/flume/sink/TestThriftSink.java +++ b/flume-ng-core/src/test/java/org/apache/flume/sink/TestThriftSink.java @@ -19,15 +19,18 @@ package org.apache.flume.sink; import com.google.common.base.Charsets; + import org.apache.flume.Context; import org.apache.flume.Event; import org.apache.flume.EventDeliveryException; import org.apache.flume.Sink; import org.apache.flume.Transaction; +import org.apache.flume.api.ThriftRpcClient; import org.apache.flume.api.ThriftTestingSource; import org.apache.flume.channel.MemoryChannel; import org.apache.flume.conf.Configurables; import org.apache.flume.event.EventBuilder; +import org.apache.flume.source.ThriftSource; import org.junit.After; import org.junit.Assert; import org.junit.Before; @@ -58,7 +61,7 @@ public void setUp() throws Exception { context.put("port", String.valueOf(port)); context.put("batch-size", String.valueOf(2)); context.put("request-timeout", String.valueOf(2000L)); - + context.put(ThriftRpcClient.CONFIG_PROTOCOL, ThriftRpcClient.COMPACT_PROTOCOL); sink.setChannel(channel); Configurables.configure(sink, context); @@ -77,7 +80,7 @@ public void testProcess() throws Exception { Event event = EventBuilder.withBody("test event 1", Charsets.UTF_8); src = new ThriftTestingSource(ThriftTestingSource.HandlerType.OK.name(), - port); + port, ThriftRpcClient.COMPACT_PROTOCOL); channel.start(); sink.start(); @@ -108,7 +111,7 @@ public void testProcess() throws Exception { public void testTimeout() throws Exception { AtomicLong delay = new AtomicLong(); src = new ThriftTestingSource(ThriftTestingSource.HandlerType.ALTERNATE - .name(), port); + .name(), port, ThriftRpcClient.COMPACT_PROTOCOL); src.setDelay(delay); delay.set(2500); @@ -182,7 +185,7 @@ public void testFailedConnect() throws Exception { } src = new ThriftTestingSource(ThriftTestingSource.HandlerType.OK.name(), - port); + port, ThriftRpcClient.COMPACT_PROTOCOL); for (int i = 0; i < 5; i++) { Sink.Status status = sink.process(); diff --git a/flume-ng-sdk/src/main/java/org/apache/flume/api/ThriftRpcClient.java b/flume-ng-sdk/src/main/java/org/apache/flume/api/ThriftRpcClient.java index cf45ab9e57..6382a0eb24 100644 --- a/flume-ng-sdk/src/main/java/org/apache/flume/api/ThriftRpcClient.java +++ b/flume-ng-sdk/src/main/java/org/apache/flume/api/ThriftRpcClient.java @@ -24,6 +24,7 @@ import org.apache.flume.thrift.Status; import org.apache.flume.thrift.ThriftFlumeEvent; import org.apache.flume.thrift.ThriftSourceProtocol; +import org.apache.thrift.protocol.TBinaryProtocol; import org.apache.thrift.protocol.TCompactProtocol; import org.apache.thrift.transport.TFastFramedTransport; import org.apache.thrift.transport.TSocket; @@ -57,6 +58,13 @@ public class ThriftRpcClient extends AbstractRpcClient { private static final Logger LOGGER = LoggerFactory.getLogger(ThriftRpcClient.class); + /** + * Config param for the thrift protocol to use. + */ + public static final String CONFIG_PROTOCOL = "protocol"; + public static final String BINARY_PROTOCOL = "binary"; + public static final String COMPACT_PROTOCOL = "compact"; + private int batchSize; private long requestTimeout; private final Lock stateLock; @@ -68,6 +76,7 @@ public class ThriftRpcClient extends AbstractRpcClient { private final AtomicLong threadCounter; private int connectionPoolSize; private final Random random = new Random(); + private String protocol; public ThriftRpcClient() { stateLock = new ReentrantLock(true); @@ -267,6 +276,18 @@ protected void configure(Properties properties) throws FlumeException { HostInfo host = HostInfo.getHostInfoList(properties).get(0); hostname = host.getHostName(); port = host.getPortNumber(); + protocol = properties.getProperty(CONFIG_PROTOCOL); + if (protocol == null) { + // default is to use the compact protocol. + protocol = COMPACT_PROTOCOL; + } + // check in case that garbage was put in. + if (!(protocol.equalsIgnoreCase(BINARY_PROTOCOL) || + protocol.equalsIgnoreCase(COMPACT_PROTOCOL))) { + LOGGER.warn("'binary' or 'compact' are the only valid Thrift protocol types to " + + "choose from. Defaulting to 'compact'."); + protocol = COMPACT_PROTOCOL; + } batchSize = Integer.parseInt(properties.getProperty( RpcClientConfigurationConstants.CONFIG_BATCH_SIZE, RpcClientConfigurationConstants.DEFAULT_BATCH_SIZE.toString())); @@ -322,8 +343,15 @@ private class ClientWrapper { public ClientWrapper() throws Exception{ transport = new TFastFramedTransport(new TSocket(hostname, port)); transport.open(); - client = new ThriftSourceProtocol.Client(new TCompactProtocol - (transport)); + if (protocol.equals(BINARY_PROTOCOL)) { + LOGGER.info("Using TBinaryProtocol"); + client = new ThriftSourceProtocol.Client(new TBinaryProtocol + (transport)); + } else { + LOGGER.info("Using TCompactProtocol"); + client = new ThriftSourceProtocol.Client(new TCompactProtocol + (transport)); + } // Not a great hash code, but since this class is immutable and there // is at most one instance of the components of this class, // this works fine [If the objects are equal, hash code is the same] diff --git a/flume-ng-sdk/src/test/java/org/apache/flume/api/TestThriftRpcClient.java b/flume-ng-sdk/src/test/java/org/apache/flume/api/TestThriftRpcClient.java index 88eb5e7389..a8baaa838d 100644 --- a/flume-ng-sdk/src/test/java/org/apache/flume/api/TestThriftRpcClient.java +++ b/flume-ng-sdk/src/test/java/org/apache/flume/api/TestThriftRpcClient.java @@ -56,6 +56,7 @@ public void setUp() throws Exception { props.setProperty(RpcClientConfigurationConstants.CONFIG_BATCH_SIZE, "10"); props.setProperty(RpcClientConfigurationConstants.CONFIG_REQUEST_TIMEOUT, "2000"); + props.setProperty(ThriftRpcClient.CONFIG_PROTOCOL, ThriftRpcClient.COMPACT_PROTOCOL); } @After @@ -103,7 +104,7 @@ public static void insertAsBatch(RpcClient client, int start, @Test public void testOK() throws Exception { src = new ThriftTestingSource(ThriftTestingSource.HandlerType.OK.name(), - port); + port, ThriftRpcClient.COMPACT_PROTOCOL); client = (ThriftRpcClient) RpcClientFactory.getInstance(props); insertEvents(client, 10); //10 events insertAsBatch(client, 10, 25); //16 events @@ -121,7 +122,7 @@ public void testOK() throws Exception { @Test public void testSlow() throws Exception { src = new ThriftTestingSource(ThriftTestingSource.HandlerType.SLOW.name(), - port); + port, ThriftRpcClient.COMPACT_PROTOCOL); client = (ThriftRpcClient) RpcClientFactory.getInstance(props); insertEvents(client, 2); //2 events insertAsBatch(client, 2, 25); //24 events (3 batches) @@ -139,7 +140,7 @@ public void testSlow() throws Exception { @Test(expected = EventDeliveryException.class) public void testFail() throws Exception { src = new ThriftTestingSource(ThriftTestingSource.HandlerType.FAIL.name(), - port); + port, ThriftRpcClient.COMPACT_PROTOCOL); client = (ThriftRpcClient) RpcClientFactory.getInstance(props); insertEvents(client, 2); //2 events Assert.fail("Expected EventDeliveryException to be thrown."); @@ -149,7 +150,7 @@ public void testFail() throws Exception { public void testError() throws Throwable { try { src = new ThriftTestingSource(ThriftTestingSource.HandlerType.ERROR - .name(), port); + .name(), port, ThriftRpcClient.COMPACT_PROTOCOL); client = (ThriftRpcClient) RpcClientFactory.getThriftInstance("0.0.0" + ".0", port); insertEvents(client, 2); //2 events @@ -163,7 +164,7 @@ public void testError() throws Throwable { public void testTimeout() throws Throwable { try { src = new ThriftTestingSource(ThriftTestingSource.HandlerType.TIMEOUT - .name(), port); + .name(), port, ThriftRpcClient.COMPACT_PROTOCOL); client = (ThriftRpcClient) RpcClientFactory.getThriftInstance(props); insertEvents(client, 2); //2 events } catch (EventDeliveryException ex) { @@ -174,7 +175,7 @@ public void testTimeout() throws Throwable { @Test public void testMultipleThreads() throws Throwable { src = new ThriftTestingSource(ThriftTestingSource.HandlerType.OK.name(), - port); + port, ThriftRpcClient.COMPACT_PROTOCOL); client = (ThriftRpcClient) RpcClientFactory.getThriftInstance("0.0.0" + ".0", port, 10); int threadCount = 100; diff --git a/flume-ng-sdk/src/test/java/org/apache/flume/api/ThriftTestingSource.java b/flume-ng-sdk/src/test/java/org/apache/flume/api/ThriftTestingSource.java index cde726975e..63d2fc39e0 100644 --- a/flume-ng-sdk/src/test/java/org/apache/flume/api/ThriftTestingSource.java +++ b/flume-ng-sdk/src/test/java/org/apache/flume/api/ThriftTestingSource.java @@ -25,7 +25,11 @@ import org.apache.flume.thrift.ThriftFlumeEvent; import org.apache.flume.thrift.ThriftSourceProtocol; import org.apache.thrift.TException; +import org.apache.thrift.protocol.TBinaryProtocol; +import org.apache.thrift.protocol.TBinaryProtocol.Factory; import org.apache.thrift.protocol.TCompactProtocol; +import org.apache.thrift.protocol.TProtocol; +import org.apache.thrift.protocol.TProtocolFactory; import org.apache.thrift.server.THsHaServer; import org.apache.thrift.server.TServer; import org.apache.thrift.transport.TNonblockingServerSocket; @@ -180,7 +184,7 @@ public Status appendBatch(List events) throws } } - public ThriftTestingSource(String handlerName, int port) throws Exception { + public ThriftTestingSource(String handlerName, int port, String protocol) throws Exception { TNonblockingServerTransport serverTransport = new TNonblockingServerSocket(new InetSocketAddress("0.0.0.0", port)); ThriftSourceProtocol.Iface handler = null; @@ -197,11 +201,16 @@ public ThriftTestingSource(String handlerName, int port) throws Exception { } else if (handlerName.equals(HandlerType.ALTERNATE.name())) { handler = new ThriftAlternateHandler(); } - + TProtocolFactory transportProtocolFactory = null; + if (protocol != null && protocol == ThriftRpcClient.BINARY_PROTOCOL) { + transportProtocolFactory = new TBinaryProtocol.Factory(); + } else { + transportProtocolFactory = new TCompactProtocol.Factory(); + } server = new THsHaServer(new THsHaServer.Args (serverTransport).processor( new ThriftSourceProtocol.Processor(handler)).protocolFactory( - new TCompactProtocol.Factory())); + transportProtocolFactory)); Executors.newSingleThreadExecutor().submit(new Runnable() { @Override public void run() { From 84465664c21ed0a1261f015a010b94a98153c8d7 Mon Sep 17 00:00:00 2001 From: Hari Shreedharan Date: Thu, 18 Dec 2014 13:28:20 -0800 Subject: [PATCH 191/341] FLUME-2556. BucketWriter DateFormat test fails intermittently. (Johny Rufus via Hari) --- .../flume/formatter/output/BucketPath.java | 2 ++ .../flume/formatter/output/TestBucketPath.java | 17 +++++++++++++++++ 2 files changed, 19 insertions(+) diff --git a/flume-ng-core/src/main/java/org/apache/flume/formatter/output/BucketPath.java b/flume-ng-core/src/main/java/org/apache/flume/formatter/output/BucketPath.java index fe38d2c130..afe095c78f 100644 --- a/flume-ng-core/src/main/java/org/apache/flume/formatter/output/BucketPath.java +++ b/flume-ng-core/src/main/java/org/apache/flume/formatter/output/BucketPath.java @@ -314,6 +314,8 @@ protected static String replaceShorthand(char c, Map headers, SimpleDateFormat format = getSimpleDateFormat(formatString); if (timeZone != null) { format.setTimeZone(timeZone); + } else { + format.setTimeZone(TimeZone.getDefault()); } Date date = new Date(ts); diff --git a/flume-ng-core/src/test/java/org/apache/flume/formatter/output/TestBucketPath.java b/flume-ng-core/src/test/java/org/apache/flume/formatter/output/TestBucketPath.java index c441c4aeda..8d1da12dc9 100644 --- a/flume-ng-core/src/test/java/org/apache/flume/formatter/output/TestBucketPath.java +++ b/flume-ng-core/src/test/java/org/apache/flume/formatter/output/TestBucketPath.java @@ -47,6 +47,23 @@ public void setUp(){ headers = new HashMap(); headers.put("timestamp", String.valueOf(cal.getTimeInMillis())); } + + @Test + public void testDateFormatCache(){ + TimeZone utcTimeZone = TimeZone.getTimeZone("UTC"); + String test = "%c"; + BucketPath.escapeString( + test, headers, utcTimeZone, false, Calendar.HOUR_OF_DAY, 12, false); + String escapedString = BucketPath.escapeString( + test, headers, false, Calendar.HOUR_OF_DAY, 12); + System.out.println("Escaped String: " + escapedString); + SimpleDateFormat format = new SimpleDateFormat("EEE MMM d HH:mm:ss yyyy"); + Date d = new Date(cal.getTimeInMillis()); + String expectedString = format.format(d); + System.out.println("Expected String: "+ expectedString); + Assert.assertEquals(expectedString, escapedString); + } + @Test public void testDateFormatHours() { String test = "%c"; From 199684b62ec983b8f922b1d6d706479032a18e64 Mon Sep 17 00:00:00 2001 From: Hari Shreedharan Date: Tue, 30 Dec 2014 11:42:47 -0800 Subject: [PATCH 192/341] FLUME-2578. Kafka source throws NPE if Kafka record has null key (Gwen Shapira via Hari) --- .../flume/source/kafka/KafkaSource.java | 4 ++- .../flume/source/kafka/TestKafkaSource.java | 25 +++++++++++++++++-- 2 files changed, 26 insertions(+), 3 deletions(-) diff --git a/flume-ng-sources/flume-kafka-source/src/main/java/org/apache/flume/source/kafka/KafkaSource.java b/flume-ng-sources/flume-kafka-source/src/main/java/org/apache/flume/source/kafka/KafkaSource.java index 7bc03da6c0..00a81c6669 100644 --- a/flume-ng-sources/flume-kafka-source/src/main/java/org/apache/flume/source/kafka/KafkaSource.java +++ b/flume-ng-sources/flume-kafka-source/src/main/java/org/apache/flume/source/kafka/KafkaSource.java @@ -102,7 +102,9 @@ public Status process() throws EventDeliveryException { headers.put(KafkaSourceConstants.TIMESTAMP, String.valueOf(System.currentTimeMillis())); headers.put(KafkaSourceConstants.TOPIC, topic); - headers.put(KafkaSourceConstants.KEY, new String(kafkaKey)); + if (kafkaKey != null) { + headers.put(KafkaSourceConstants.KEY, new String(kafkaKey)); + } if (log.isDebugEnabled()) { log.debug("Message: {}", new String(kafkaMessage)); } diff --git a/flume-ng-sources/flume-kafka-source/src/test/java/org/apache/flume/source/kafka/TestKafkaSource.java b/flume-ng-sources/flume-kafka-source/src/test/java/org/apache/flume/source/kafka/TestKafkaSource.java index 72eec77d69..8ec14cccf5 100644 --- a/flume-ng-sources/flume-kafka-source/src/test/java/org/apache/flume/source/kafka/TestKafkaSource.java +++ b/flume-ng-sources/flume-kafka-source/src/test/java/org/apache/flume/source/kafka/TestKafkaSource.java @@ -105,8 +105,6 @@ public void testProcessItNotEmpty() throws EventDeliveryException, Assert.assertEquals("hello, world", new String(events.get(0).getBody(), Charsets.UTF_8)); - - } @SuppressWarnings("unchecked") @@ -301,6 +299,29 @@ public void testTwoBatchesWithAutocommit() throws InterruptedException, } + @SuppressWarnings("unchecked") + @Test + public void testNullKey() throws EventDeliveryException, + SecurityException, NoSuchFieldException, IllegalArgumentException, + IllegalAccessException, InterruptedException { + context.put(KafkaSourceConstants.BATCH_SIZE,"1"); + kafkaSource.configure(context); + kafkaSource.start(); + + Thread.sleep(500L); + + kafkaServer.produce(topicName, null , "hello, world"); + + Thread.sleep(500L); + + Assert.assertEquals(Status.READY, kafkaSource.process()); + Assert.assertEquals(Status.BACKOFF, kafkaSource.process()); + Assert.assertEquals(1, events.size()); + + Assert.assertEquals("hello, world", new String(events.get(0).getBody(), + Charsets.UTF_8)); + } + ChannelProcessor createGoodChannel() { ChannelProcessor channelProcessor = mock(ChannelProcessor.class); From 1d9bab6760df38e538705a74dd599de03129777b Mon Sep 17 00:00:00 2001 From: Hari Shreedharan Date: Fri, 16 Jan 2015 16:24:24 -0800 Subject: [PATCH 193/341] FLUME-2562. Add metrics for Kafka Source, Kafka Sink and Kafka Channel. (Gwen Shapira via Hari) --- .../flume/channel/kafka/KafkaChannel.java | 25 +++++- .../flume/instrumentation/ChannelCounter.java | 7 ++ .../flume/instrumentation/SinkCounter.java | 8 ++ .../flume/instrumentation/SourceCounter.java | 7 ++ .../kafka/KafkaChannelCounter.java | 82 +++++++++++++++++++ .../kafka/KafkaChannelCounterMBean.java | 50 +++++++++++ .../kafka/KafkaSinkCounter.java | 53 ++++++++++++ .../kafka/KafkaSinkCounterMBean.java | 48 +++++++++++ .../kafka/KafkaSourceCounter.java | 64 +++++++++++++++ .../kafka/KafkaSourceCounterMBean.java | 47 +++++++++++ .../apache/flume/sink/kafka/KafkaSink.java | 15 ++++ .../flume/source/kafka/KafkaSource.java | 18 ++++ 12 files changed, 423 insertions(+), 1 deletion(-) create mode 100644 flume-ng-core/src/main/java/org/apache/flume/instrumentation/kafka/KafkaChannelCounter.java create mode 100644 flume-ng-core/src/main/java/org/apache/flume/instrumentation/kafka/KafkaChannelCounterMBean.java create mode 100644 flume-ng-core/src/main/java/org/apache/flume/instrumentation/kafka/KafkaSinkCounter.java create mode 100644 flume-ng-core/src/main/java/org/apache/flume/instrumentation/kafka/KafkaSinkCounterMBean.java create mode 100644 flume-ng-core/src/main/java/org/apache/flume/instrumentation/kafka/KafkaSourceCounter.java create mode 100644 flume-ng-core/src/main/java/org/apache/flume/instrumentation/kafka/KafkaSourceCounterMBean.java diff --git a/flume-ng-channels/flume-kafka-channel/src/main/java/org/apache/flume/channel/kafka/KafkaChannel.java b/flume-ng-channels/flume-kafka-channel/src/main/java/org/apache/flume/channel/kafka/KafkaChannel.java index d767aac987..80a122db9e 100644 --- a/flume-ng-channels/flume-kafka-channel/src/main/java/org/apache/flume/channel/kafka/KafkaChannel.java +++ b/flume-ng-channels/flume-kafka-channel/src/main/java/org/apache/flume/channel/kafka/KafkaChannel.java @@ -36,6 +36,7 @@ import static org.apache.flume.channel.kafka.KafkaChannelConfiguration.*; import org.apache.flume.event.EventBuilder; +import org.apache.flume.instrumentation.kafka.KafkaChannelCounter; import org.apache.flume.source.avro.AvroFlumeEvent; import org.slf4j.Logger; import org.slf4j.LoggerFactory; @@ -64,6 +65,8 @@ public class KafkaChannel extends BasicChannelSemantics { private final List consumers = Collections.synchronizedList(new LinkedList()); + private KafkaChannelCounter counter; + /* Each ConsumerConnector commit will commit all partitions owned by it. To * ensure that each partition is only committed when all events are * actually done, we will need to keep a ConsumerConnector per thread. @@ -95,6 +98,7 @@ public void start() { // We always have just one topic being read by one thread LOGGER.info("Topic = " + topic.get()); topicCountMap.put(topic.get(), 1); + counter.start(); super.start(); } catch (Exception e) { LOGGER.error("Could not start producer"); @@ -114,7 +118,10 @@ public void stop() { } } producer.close(); + counter.stop(); super.stop(); + LOGGER.info("Kafka channel {} stopped. Metrics: {}", getName(), + counter); } @Override @@ -192,6 +199,10 @@ public void configure(Context ctx) { kafkaConf.put("auto.offset.reset", "smallest"); } + if (counter == null) { + counter = new KafkaChannelCounter(getName()); + } + } private void decommissionConsumerAndIterator(ConsumerAndIterator c) { @@ -291,7 +302,10 @@ protected Event doTake() throws InterruptedException { } else { try { ConsumerIterator it = consumerAndIter.get().iterator; + long startTime = System.nanoTime(); it.hasNext(); + long endTime = System.nanoTime(); + counter.addToKafkaEventGetTimer((endTime-startTime)/(1000*1000)); if (parseAsFlumeEvent) { ByteArrayInputStream in = new ByteArrayInputStream(it.next().message()); @@ -339,7 +353,11 @@ protected void doCommit() throws InterruptedException { messages.add(new KeyedMessage(topic.get(), null, batchUUID, event)); } + long startTime = System.nanoTime(); producer.send(messages); + long endTime = System.nanoTime(); + counter.addToKafkaEventSendTimer((endTime-startTime)/(1000*1000)); + counter.addToEventPutSuccessCount(Long.valueOf(messages.size())); serializedEvents.get().clear(); } catch (Exception ex) { LOGGER.warn("Sending events to Kafka failed", ex); @@ -348,8 +366,12 @@ protected void doCommit() throws InterruptedException { } } else { if (consumerAndIter.get().failedEvents.isEmpty() && eventTaken) { + long startTime = System.nanoTime(); consumerAndIter.get().consumer.commitOffsets(); - } + long endTime = System.nanoTime(); + counter.addToKafkaCommitTimer((endTime-startTime)/(1000*1000)); + } + counter.addToEventTakeSuccessCount(Long.valueOf(events.get().size())); events.get().clear(); } } @@ -362,6 +384,7 @@ protected void doRollback() throws InterruptedException { if (type.equals(TransactionType.PUT)) { serializedEvents.get().clear(); } else { + counter.addToRollbackCounter(Long.valueOf(events.get().size())); consumerAndIter.get().failedEvents.addAll(events.get()); events.get().clear(); } diff --git a/flume-ng-core/src/main/java/org/apache/flume/instrumentation/ChannelCounter.java b/flume-ng-core/src/main/java/org/apache/flume/instrumentation/ChannelCounter.java index 9938c0ad77..977ad6c6a5 100644 --- a/flume-ng-core/src/main/java/org/apache/flume/instrumentation/ChannelCounter.java +++ b/flume-ng-core/src/main/java/org/apache/flume/instrumentation/ChannelCounter.java @@ -18,6 +18,8 @@ */ package org.apache.flume.instrumentation; +import org.apache.commons.lang.ArrayUtils; + public class ChannelCounter extends MonitoredCounterGroup implements ChannelCounterMBean { @@ -48,6 +50,11 @@ public ChannelCounter(String name) { super(MonitoredCounterGroup.Type.CHANNEL, name, ATTRIBUTES); } + public ChannelCounter(String name, String[] attributes) { + super(MonitoredCounterGroup.Type.CHANNEL, name, + (String[])ArrayUtils.addAll(attributes,ATTRIBUTES)); + } + @Override public long getChannelSize() { return get(COUNTER_CHANNEL_SIZE); diff --git a/flume-ng-core/src/main/java/org/apache/flume/instrumentation/SinkCounter.java b/flume-ng-core/src/main/java/org/apache/flume/instrumentation/SinkCounter.java index 41b28cf08e..54f4a4c30f 100644 --- a/flume-ng-core/src/main/java/org/apache/flume/instrumentation/SinkCounter.java +++ b/flume-ng-core/src/main/java/org/apache/flume/instrumentation/SinkCounter.java @@ -17,6 +17,8 @@ */ package org.apache.flume.instrumentation; +import org.apache.commons.lang.ArrayUtils; + public class SinkCounter extends MonitoredCounterGroup implements SinkCounterMBean { @@ -56,6 +58,12 @@ public SinkCounter(String name) { super(MonitoredCounterGroup.Type.SINK, name, ATTRIBUTES); } + public SinkCounter(String name, String[] attributes) { + super(MonitoredCounterGroup.Type.SINK, name, + (String[]) ArrayUtils.addAll(attributes,ATTRIBUTES)); + } + + @Override public long getConnectionCreatedCount() { return get(COUNTER_CONNECTION_CREATED); diff --git a/flume-ng-core/src/main/java/org/apache/flume/instrumentation/SourceCounter.java b/flume-ng-core/src/main/java/org/apache/flume/instrumentation/SourceCounter.java index 972d2c64c9..02ef6ed679 100644 --- a/flume-ng-core/src/main/java/org/apache/flume/instrumentation/SourceCounter.java +++ b/flume-ng-core/src/main/java/org/apache/flume/instrumentation/SourceCounter.java @@ -18,6 +18,8 @@ */ package org.apache.flume.instrumentation; +import org.apache.commons.lang.ArrayUtils; + public class SourceCounter extends MonitoredCounterGroup implements SourceCounterMBean { @@ -53,6 +55,11 @@ public SourceCounter(String name) { super(MonitoredCounterGroup.Type.SOURCE, name, ATTRIBUTES); } + public SourceCounter(String name, String[] attributes) { + super(Type.SOURCE, name, + (String[]) ArrayUtils.addAll(attributes,ATTRIBUTES)); + } + @Override public long getEventReceivedCount() { return get(COUNTER_EVENTS_RECEIVED); diff --git a/flume-ng-core/src/main/java/org/apache/flume/instrumentation/kafka/KafkaChannelCounter.java b/flume-ng-core/src/main/java/org/apache/flume/instrumentation/kafka/KafkaChannelCounter.java new file mode 100644 index 0000000000..6e142cfd12 --- /dev/null +++ b/flume-ng-core/src/main/java/org/apache/flume/instrumentation/kafka/KafkaChannelCounter.java @@ -0,0 +1,82 @@ +/** + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package org.apache.flume.instrumentation.kafka; + +import org.apache.commons.lang.ArrayUtils; +import org.apache.flume.instrumentation.ChannelCounter; + +public class KafkaChannelCounter extends ChannelCounter + implements KafkaChannelCounterMBean { + + + private static final String TIMER_KAFKA_EVENT_GET = + "channel.kafka.event.get.time"; + + private static final String TIMER_KAFKA_EVENT_SEND = + "channel.kafka.event.send.time"; + + private static final String TIMER_KAFKA_COMMIT = + "channel.kafka.commit.time"; + + private static final String COUNT_ROLLBACK = + "channel.rollback.count"; + + + private static String[] ATTRIBUTES = { + TIMER_KAFKA_COMMIT,TIMER_KAFKA_EVENT_SEND,TIMER_KAFKA_EVENT_GET, + COUNT_ROLLBACK + }; + + + public KafkaChannelCounter(String name) { + super(name,ATTRIBUTES); + } + + public long addToKafkaEventGetTimer(long delta) { + return addAndGet(TIMER_KAFKA_EVENT_GET, delta); + } + + public long addToKafkaEventSendTimer(long delta) { + return addAndGet(TIMER_KAFKA_EVENT_SEND, delta); + } + + public long addToKafkaCommitTimer(long delta) { + return addAndGet(TIMER_KAFKA_COMMIT, delta); + } + + public long addToRollbackCounter(long delta) { + return addAndGet(COUNT_ROLLBACK,delta); + } + + public long getKafkaEventGetTimer() { + return get(TIMER_KAFKA_EVENT_GET); + } + + public long getKafkaEventSendTimer() { + return get(TIMER_KAFKA_EVENT_SEND); + } + + public long getKafkaCommitTimer() { + return get(TIMER_KAFKA_COMMIT); + } + + public long getRollbackCount() { + return get(COUNT_ROLLBACK); + } + +} diff --git a/flume-ng-core/src/main/java/org/apache/flume/instrumentation/kafka/KafkaChannelCounterMBean.java b/flume-ng-core/src/main/java/org/apache/flume/instrumentation/kafka/KafkaChannelCounterMBean.java new file mode 100644 index 0000000000..da64f0ca99 --- /dev/null +++ b/flume-ng-core/src/main/java/org/apache/flume/instrumentation/kafka/KafkaChannelCounterMBean.java @@ -0,0 +1,50 @@ +/** + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package org.apache.flume.instrumentation.kafka; + +public interface KafkaChannelCounterMBean { + + long getKafkaEventGetTimer(); + + long getKafkaEventSendTimer(); + + long getKafkaCommitTimer(); + + long getRollbackCount(); + + long getChannelSize(); + + long getEventPutAttemptCount(); + + long getEventTakeAttemptCount(); + + long getEventPutSuccessCount(); + + long getEventTakeSuccessCount(); + + long getStartTime(); + + long getStopTime(); + + long getChannelCapacity(); + + String getType(); + + double getChannelFillPercentage(); + +} diff --git a/flume-ng-core/src/main/java/org/apache/flume/instrumentation/kafka/KafkaSinkCounter.java b/flume-ng-core/src/main/java/org/apache/flume/instrumentation/kafka/KafkaSinkCounter.java new file mode 100644 index 0000000000..1308ff3a9d --- /dev/null +++ b/flume-ng-core/src/main/java/org/apache/flume/instrumentation/kafka/KafkaSinkCounter.java @@ -0,0 +1,53 @@ +/** + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package org.apache.flume.instrumentation.kafka; + +import org.apache.flume.instrumentation.SinkCounter; + +public class KafkaSinkCounter extends SinkCounter implements KafkaSinkCounterMBean { + + private static final String TIMER_KAFKA_EVENT_SEND = + "channel.kafka.event.send.time"; + + private static final String COUNT_ROLLBACK = + "channel.rollback.count"; + + private static final String[] ATTRIBUTES = + {COUNT_ROLLBACK,TIMER_KAFKA_EVENT_SEND}; + + public KafkaSinkCounter(String name) { + super(name,ATTRIBUTES); + } + + public long addToKafkaEventSendTimer(long delta) { + return addAndGet(TIMER_KAFKA_EVENT_SEND,delta); + } + + public long incrementRollbackCount() { + return increment(COUNT_ROLLBACK); + } + + public long getKafkaEventSendTimer() { + return get(TIMER_KAFKA_EVENT_SEND); + } + + public long getRollbackCount() { + return get(COUNT_ROLLBACK); + } + +} diff --git a/flume-ng-core/src/main/java/org/apache/flume/instrumentation/kafka/KafkaSinkCounterMBean.java b/flume-ng-core/src/main/java/org/apache/flume/instrumentation/kafka/KafkaSinkCounterMBean.java new file mode 100644 index 0000000000..f49ca26a83 --- /dev/null +++ b/flume-ng-core/src/main/java/org/apache/flume/instrumentation/kafka/KafkaSinkCounterMBean.java @@ -0,0 +1,48 @@ +/** + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package org.apache.flume.instrumentation.kafka; + +public interface KafkaSinkCounterMBean { + + long getKafkaEventSendTimer(); + + long getRollbackCount(); + + long getConnectionCreatedCount(); + + long getConnectionClosedCount(); + + long getConnectionFailedCount(); + + long getBatchEmptyCount(); + + long getBatchUnderflowCount(); + + long getBatchCompleteCount(); + + long getEventDrainAttemptCount(); + + long getEventDrainSuccessCount(); + + long getStartTime(); + + long getStopTime(); + + String getType(); +} + diff --git a/flume-ng-core/src/main/java/org/apache/flume/instrumentation/kafka/KafkaSourceCounter.java b/flume-ng-core/src/main/java/org/apache/flume/instrumentation/kafka/KafkaSourceCounter.java new file mode 100644 index 0000000000..1cb911d765 --- /dev/null +++ b/flume-ng-core/src/main/java/org/apache/flume/instrumentation/kafka/KafkaSourceCounter.java @@ -0,0 +1,64 @@ +/** + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package org.apache.flume.instrumentation.kafka; + +import org.apache.flume.instrumentation.SourceCounter; + +public class KafkaSourceCounter extends SourceCounter implements KafkaSourceCounterMBean { + + private static final String TIMER_KAFKA_EVENT_GET = + "source.kafka.event.get.time"; + + private static final String TIMER_KAFKA_COMMIT = + "source.kafka.commit.time"; + + private static final String COUNTER_KAFKA_EMPTY = + "source.kafka.empty.count"; + + private static final String[] ATTRIBUTES = + {TIMER_KAFKA_COMMIT, TIMER_KAFKA_EVENT_GET}; + + public KafkaSourceCounter(String name) { + super(name, ATTRIBUTES); + } + + public long addToKafkaEventGetTimer(long delta) { + return addAndGet(TIMER_KAFKA_EVENT_GET,delta); + } + + public long addToKafkaCommitTimer(long delta) { + return addAndGet(TIMER_KAFKA_COMMIT,delta); + } + + public long incrementKafkaEmptyCount() { + return increment(COUNTER_KAFKA_EMPTY); + } + + public long getKafkaCommitTimer() { + return get(TIMER_KAFKA_COMMIT); + } + + public long getKafkaEventGetTimer() { + return get(TIMER_KAFKA_EVENT_GET); + } + + public long getKafkaEmptyCount() { + return get(COUNTER_KAFKA_EMPTY); + } + +} diff --git a/flume-ng-core/src/main/java/org/apache/flume/instrumentation/kafka/KafkaSourceCounterMBean.java b/flume-ng-core/src/main/java/org/apache/flume/instrumentation/kafka/KafkaSourceCounterMBean.java new file mode 100644 index 0000000000..219a5b603d --- /dev/null +++ b/flume-ng-core/src/main/java/org/apache/flume/instrumentation/kafka/KafkaSourceCounterMBean.java @@ -0,0 +1,47 @@ +/** + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package org.apache.flume.instrumentation.kafka; + +public interface KafkaSourceCounterMBean { + + long getKafkaEventGetTimer(); + + long getKafkaCommitTimer(); + + long getKafkaEmptyCount(); + + long getEventReceivedCount(); + + long getEventAcceptedCount(); + + long getAppendReceivedCount(); + + long getAppendAcceptedCount(); + + long getAppendBatchReceivedCount(); + + long getAppendBatchAcceptedCount(); + + long getStartTime(); + + long getStopTime(); + + String getType(); + + long getOpenConnectionCount(); +} diff --git a/flume-ng-sinks/flume-ng-kafka-sink/src/main/java/org/apache/flume/sink/kafka/KafkaSink.java b/flume-ng-sinks/flume-ng-kafka-sink/src/main/java/org/apache/flume/sink/kafka/KafkaSink.java index a90b950684..eada17ca13 100644 --- a/flume-ng-sinks/flume-ng-kafka-sink/src/main/java/org/apache/flume/sink/kafka/KafkaSink.java +++ b/flume-ng-sinks/flume-ng-kafka-sink/src/main/java/org/apache/flume/sink/kafka/KafkaSink.java @@ -24,6 +24,7 @@ Licensed to the Apache Software Foundation (ASF) under one or more import kafka.producer.ProducerConfig; import org.apache.flume.*; import org.apache.flume.conf.Configurable; +import org.apache.flume.instrumentation.kafka.KafkaSinkCounter; import org.apache.flume.sink.AbstractSink; import org.slf4j.Logger; import org.slf4j.LoggerFactory; @@ -73,6 +74,8 @@ public class KafkaSink extends AbstractSink implements Configurable { private String topic; private int batchSize; private List> messageList; + private KafkaSinkCounter counter; + @Override public Status process() throws EventDeliveryException { @@ -122,7 +125,11 @@ public Status process() throws EventDeliveryException { // publish batch and commit. if (processedEvents > 0) { + long startTime = System.nanoTime(); producer.send(messageList); + long endTime = System.nanoTime(); + counter.addToKafkaEventSendTimer((endTime-startTime)/(1000*1000)); + counter.addToEventDrainSuccessCount(Long.valueOf(messageList.size())); } transaction.commit(); @@ -134,6 +141,7 @@ public Status process() throws EventDeliveryException { if (transaction != null) { try { transaction.rollback(); + counter.incrementRollbackCount(); } catch (Exception e) { logger.error("Transaction rollback failed", e); throw Throwables.propagate(e); @@ -154,12 +162,15 @@ public synchronized void start() { // instantiate the producer ProducerConfig config = new ProducerConfig(kafkaProps); producer = new Producer(config); + counter.start(); super.start(); } @Override public synchronized void stop() { producer.close(); + counter.stop(); + logger.info("Kafka Sink {} stopped. Metrics: {}", getName(), counter); super.stop(); } @@ -202,5 +213,9 @@ public void configure(Context context) { if (logger.isDebugEnabled()) { logger.debug("Kafka producer properties: " + kafkaProps); } + + if (counter == null) { + counter = new KafkaSinkCounter(getName()); + } } } diff --git a/flume-ng-sources/flume-kafka-source/src/main/java/org/apache/flume/source/kafka/KafkaSource.java b/flume-ng-sources/flume-kafka-source/src/main/java/org/apache/flume/source/kafka/KafkaSource.java index 00a81c6669..3777639cd4 100644 --- a/flume-ng-sources/flume-kafka-source/src/main/java/org/apache/flume/source/kafka/KafkaSource.java +++ b/flume-ng-sources/flume-kafka-source/src/main/java/org/apache/flume/source/kafka/KafkaSource.java @@ -32,6 +32,8 @@ import org.apache.flume.conf.Configurable; import org.apache.flume.conf.ConfigurationException; import org.apache.flume.event.EventBuilder; +import org.apache.flume.instrumentation.SourceCounter; +import org.apache.flume.instrumentation.kafka.KafkaSourceCounter; import org.apache.flume.source.AbstractSource; import org.slf4j.Logger; @@ -77,6 +79,7 @@ public class KafkaSource extends AbstractSource private Context context; private Properties kafkaProps; private final List eventList = new ArrayList(); + private KafkaSourceCounter counter; public Status process() throws EventDeliveryException { @@ -88,6 +91,7 @@ public Status process() throws EventDeliveryException { long batchEndTime = System.currentTimeMillis() + timeUpperLimit; try { boolean iterStatus = false; + long startTime = System.nanoTime(); while (eventList.size() < batchUpperLimit && System.currentTimeMillis() < batchEndTime) { iterStatus = hasNext(); @@ -116,22 +120,30 @@ public Status process() throws EventDeliveryException { log.debug("Event #: {}", eventList.size()); } } + long endTime = System.nanoTime(); + counter.addToKafkaEventGetTimer((endTime-startTime)/(1000*1000)); + counter.addToEventReceivedCount(Long.valueOf(eventList.size())); // If we have events, send events to channel // clear the event list // and commit if Kafka doesn't auto-commit if (eventList.size() > 0) { getChannelProcessor().processEventBatch(eventList); + counter.addToEventAcceptedCount(eventList.size()); eventList.clear(); if (log.isDebugEnabled()) { log.debug("Wrote {} events to channel", eventList.size()); } if (!kafkaAutoCommitEnabled) { // commit the read transactions to Kafka to avoid duplicates + long commitStartTime = System.nanoTime(); consumer.commitOffsets(); + long commitEndTime = System.nanoTime(); + counter.addToKafkaCommitTimer((commitEndTime-commitStartTime)/(1000*1000)); } } if (!iterStatus) { if (log.isDebugEnabled()) { + counter.incrementKafkaEmptyCount(); log.debug("Returning with backoff. No more data to read"); } return Status.BACKOFF; @@ -174,6 +186,9 @@ public void configure(Context context) { kafkaAutoCommitEnabled = Boolean.parseBoolean(kafkaProps.getProperty( KafkaSourceConstants.AUTO_COMMIT_ENABLED)); + if (counter == null) { + counter = new KafkaSourceCounter(getName()); + } } @Override @@ -207,6 +222,7 @@ public synchronized void start() { throw new FlumeException("Unable to get message iterator from Kafka", e); } log.info("Kafka source {} started.", getName()); + counter.start(); super.start(); } @@ -217,6 +233,8 @@ public synchronized void stop() { // to avoid reading the same messages again consumer.shutdown(); } + counter.stop(); + log.info("Kafka Source {} stopped. Metrics: {}", getName(), counter); super.stop(); } From 91c58804da51a551fcbbe290261810c7750ee749 Mon Sep 17 00:00:00 2001 From: Hari Shreedharan Date: Fri, 16 Jan 2015 16:56:48 -0800 Subject: [PATCH 194/341] FLUME-2568. Additional fix for TestReliableSpoolingFileEventReader (Johny Rufus via Hari) --- .../TestReliableSpoolingFileEventReader.java | 31 +++++++++++-------- 1 file changed, 18 insertions(+), 13 deletions(-) diff --git a/flume-ng-core/src/test/java/org/apache/flume/client/avro/TestReliableSpoolingFileEventReader.java b/flume-ng-core/src/test/java/org/apache/flume/client/avro/TestReliableSpoolingFileEventReader.java index 476bbff6c7..4e90054889 100644 --- a/flume-ng-core/src/test/java/org/apache/flume/client/avro/TestReliableSpoolingFileEventReader.java +++ b/flume-ng-core/src/test/java/org/apache/flume/client/avro/TestReliableSpoolingFileEventReader.java @@ -238,30 +238,28 @@ public void testConsumeFileRandomlyNewFile() throws Exception { FileUtils.write(fileName, "New file created in the end. Shoud be read randomly.\n"); Set expected = Sets.newHashSet(); - File tempDir = Files.createTempDir(); - File tempFile = new File(tempDir, "t"); - File finalFile = new File(WORK_DIR, "t-file"); int totalFiles = WORK_DIR.listFiles().length; - FileUtils.write(tempFile, "Last file"); final Set actual = Sets.newHashSet(); ExecutorService executor = Executors.newSingleThreadExecutor(); - final Semaphore semaphore = new Semaphore(0); + final Semaphore semaphore1 = new Semaphore(0); + final Semaphore semaphore2 = new Semaphore(0); Future wait = executor.submit( new Callable() { @Override public Void call() throws Exception { - readEventsForFilesInDir(WORK_DIR, reader, actual, semaphore); + readEventsForFilesInDir(WORK_DIR, reader, actual, semaphore1, semaphore2); return null; } } ); - semaphore.acquire(); - tempFile.renameTo(finalFile); + semaphore1.acquire(); + File finalFile = new File(WORK_DIR, "t-file"); + FileUtils.write(finalFile, "Last file"); + semaphore2.release(); wait.get(); int listFilesCount = ((ReliableSpoolingFileEventReader)reader) .getListFilesCount(); finalFile.delete(); - FileUtils.deleteQuietly(tempDir); createExpectedFromFilesInSetup(expected); expected.add(""); expected.add( @@ -496,13 +494,14 @@ private void templateTestForLargeNumberOfFiles(ConsumeOrder order, private void readEventsForFilesInDir(File dir, ReliableEventReader reader, Collection actual) throws IOException { - readEventsForFilesInDir(dir, reader, actual, null); + readEventsForFilesInDir(dir, reader, actual, null, null); } /* Read events, one for each file in the given directory. */ private void readEventsForFilesInDir(File dir, ReliableEventReader reader, - Collection actual, Semaphore semaphore) throws IOException { + Collection actual, Semaphore semaphore1, Semaphore semaphore2) throws IOException { List events; + boolean executed = false; for (int i=0; i < listFiles(dir).size(); i++) { events = reader.readEvents(10); for (Event e : events) { @@ -510,8 +509,14 @@ private void readEventsForFilesInDir(File dir, ReliableEventReader reader, } reader.commit(); try { - if (semaphore != null) { - semaphore.release(); + if(!executed) { + executed = true; + if (semaphore1 != null) { + semaphore1.release(); + } + if (semaphore2 != null) { + semaphore2.acquire(); + } } } catch (Exception ex) { throw new IOException(ex); From 82631f811a7c6a4d8e6ec886f80d1a21876947e5 Mon Sep 17 00:00:00 2001 From: Jarek Jarcec Cecho Date: Wed, 21 Jan 2015 13:49:41 -0800 Subject: [PATCH 195/341] FLUME-2594: Close Async HBase Client if there are large number of consecutive timeouts (Hari Shreedharan via Jarek Jarcec Cecho) --- .../flume/sink/hbase/AsyncHBaseSink.java | 74 +++++++++++++++++-- 1 file changed, 69 insertions(+), 5 deletions(-) diff --git a/flume-ng-sinks/flume-ng-hbase-sink/src/main/java/org/apache/flume/sink/hbase/AsyncHBaseSink.java b/flume-ng-sinks/flume-ng-hbase-sink/src/main/java/org/apache/flume/sink/hbase/AsyncHBaseSink.java index 1666be4382..1d05189456 100644 --- a/flume-ng-sinks/flume-ng-hbase-sink/src/main/java/org/apache/flume/sink/hbase/AsyncHBaseSink.java +++ b/flume-ng-sinks/flume-ng-hbase-sink/src/main/java/org/apache/flume/sink/hbase/AsyncHBaseSink.java @@ -127,12 +127,21 @@ public class AsyncHBaseSink extends AbstractSink implements Configurable { private boolean batchIncrements = false; private volatile int totalCallbacksReceived = 0; private Map incrementBuffer; + // The HBaseClient buffers the requests until a callback is received. In the event of a + // timeout, there is no way to clear these buffers. If there is a major cluster issue, this + // buffer can become too big and cause crashes. So if we hit a fixed number of HBase write + // failures/timeouts, then close the HBase Client (gracefully or not) and force a GC to get rid + // of the buffered data. + private int consecutiveHBaseFailures = 0; + private boolean lastTxnFailed = false; // Does not need to be thread-safe. Always called only from the sink's // process method. private final Comparator COMPARATOR = UnsignedBytes .lexicographicalComparator(); + private static final int MAX_CONSECUTIVE_FAILS = 10; + public AsyncHBaseSink(){ this(null); } @@ -162,6 +171,12 @@ public Status process() throws EventDeliveryException { throw new EventDeliveryException("Sink was never opened. " + "Please fix the configuration."); } + if (client == null) { + client = initHBaseClient(); + if (client == null) { + throw new EventDeliveryException("Could not establish connection to HBase!"); + } + } AtomicBoolean txnFail = new AtomicBoolean(false); AtomicInteger callbacksReceived = new AtomicInteger(0); AtomicInteger callbacksExpected = new AtomicInteger(0); @@ -292,11 +307,19 @@ public Status process() throws EventDeliveryException { * */ if (txnFail.get()) { + // We enter this if condition only if the failure was due to HBase failure, so we make sure + // we track the consecutive failures. + if (lastTxnFailed) { + consecutiveHBaseFailures++; + } + lastTxnFailed = true; this.handleTransactionFailure(txn); throw new EventDeliveryException("Could not write events to Hbase. " + "Transaction failed, and rolled back."); } else { try { + lastTxnFailed = false; + consecutiveHBaseFailures = 0; txn.commit(); txn.close(); sinkCounter.addToEventDrainSuccessCount(i); @@ -414,7 +437,12 @@ public void start(){ sinkCallbackPool = Executors.newCachedThreadPool(new ThreadFactoryBuilder() .setNameFormat(this.getName() + " HBase Call Pool").build()); logger.info("Callback pool created"); - if(!isTimeoutTest) { + client = initHBaseClient(); + super.start(); + } + + private HBaseClient initHBaseClient() { + if (!isTimeoutTest) { client = new HBaseClient(zkQuorum, zkBaseDir, sinkCallbackPool); } else { client = new HBaseClient(zkQuorum, zkBaseDir, @@ -454,8 +482,9 @@ public Object call(Object arg) throws Exception { } if(fail.get()){ sinkCounter.incrementConnectionFailedCount(); - client.shutdown(); - client = null; + if (client != null) { + shutdownHBaseClient(); + } throw new FlumeException( "Could not start sink. " + "Table or column family does not exist in Hbase."); @@ -463,14 +492,14 @@ public Object call(Object arg) throws Exception { open = true; } client.setFlushInterval((short) 0); - super.start(); + return client; } @Override public void stop(){ serializer.cleanUp(); if (client != null) { - client.shutdown(); + shutdownHBaseClient(); } sinkCounter.incrementConnectionClosedCount(); sinkCounter.stop(); @@ -496,8 +525,43 @@ public void stop(){ super.stop(); } + private void shutdownHBaseClient() { + final CountDownLatch waiter = new CountDownLatch(1); + try { + client.shutdown().addCallback(new Callback() { + @Override + public Object call(Object arg) throws Exception { + waiter.countDown(); + return null; + } + }).addErrback(new Callback() { + @Override + public Object call(Object arg) throws Exception { + logger.error("Failed to shutdown HBase client cleanly! HBase cluster might be down"); + waiter.countDown(); + return null; + } + }); + if (!waiter.await(timeout, TimeUnit.NANOSECONDS)) { + logger.error("HBase connection could not be closed within timeout! HBase cluster might " + + "be down!"); + } + } catch (Exception ex) { + logger.warn("Error while attempting to close connections to HBase"); + } finally { + // Dereference the client to force GC to clear up any buffered requests. + client = null; + } + } + private void handleTransactionFailure(Transaction txn) throws EventDeliveryException { + if (consecutiveHBaseFailures >= MAX_CONSECUTIVE_FAILS) { + if (client != null) { + shutdownHBaseClient(); + } + consecutiveHBaseFailures = 0; + } try { txn.rollback(); } catch (Throwable e) { From 0d6eccad2781884fe0f000f74b8f964cbdd7971f Mon Sep 17 00:00:00 2001 From: Hari Shreedharan Date: Wed, 28 Jan 2015 13:02:50 -0800 Subject: [PATCH 196/341] FLUME-2586. HDFS Sink must try to rename files even if close fails. (Johny Rufus via Hari) --- .../apache/flume/sink/hdfs/BucketWriter.java | 64 +++++++------------ .../flume/sink/hdfs/MockFileSystem.java | 44 +++++++------ .../sink/hdfs/MockFsDataOutputStream.java | 20 ++---- .../flume/sink/hdfs/TestBucketWriter.java | 32 +++++----- .../flume/sink/hdfs/TestHDFSEventSink.java | 18 ++++-- 5 files changed, 83 insertions(+), 95 deletions(-) diff --git a/flume-ng-sinks/flume-hdfs-sink/src/main/java/org/apache/flume/sink/hdfs/BucketWriter.java b/flume-ng-sinks/flume-hdfs-sink/src/main/java/org/apache/flume/sink/hdfs/BucketWriter.java index f9e39ac15b..62f4eee875 100644 --- a/flume-ng-sinks/flume-hdfs-sink/src/main/java/org/apache/flume/sink/hdfs/BucketWriter.java +++ b/flume-ng-sinks/flume-hdfs-sink/src/main/java/org/apache/flume/sink/hdfs/BucketWriter.java @@ -109,12 +109,12 @@ class BucketWriter { private Clock clock = new SystemClock(); private final long retryInterval; - private final int maxCloseTries; + private final int maxRenameTries; // flag that the bucket writer was closed due to idling and thus shouldn't be // reopened. Not ideal, but avoids internals of owners protected boolean closed = false; - AtomicInteger closeTries = new AtomicInteger(0); + AtomicInteger renameTries = new AtomicInteger(0); BucketWriter(long rollInterval, long rollSize, long rollCount, long batchSize, Context context, String filePath, String fileName, String inUsePrefix, @@ -148,7 +148,7 @@ class BucketWriter { fileExtensionCounter = new AtomicLong(clock.currentTimeMillis()); this.retryInterval = retryInterval; - this.maxCloseTries = maxCloseTries; + this.maxRenameTries = maxCloseTries; isOpen = false; isUnderReplicated = false; this.writer.configure(context); @@ -336,55 +336,32 @@ private CallRunner createCloseCallRunner() { private final HDFSWriter localWriter = writer; @Override public Void call() throws Exception { - LOG.info("Close tries incremented"); - closeTries.incrementAndGet(); localWriter.close(); // could block return null; } }; } - private Callable createScheduledCloseCallable( - final CallRunner closeCallRunner) { + private Callable createScheduledRenameCallable() { return new Callable() { private final String path = bucketPath; private final String finalPath = targetPath; private FileSystem fs = fileSystem; - private boolean closeSuccess = false; - private Path tmpFilePath = new Path(path); - private int closeTries = 1; // one attempt is already done - private final CallRunner closeCall = closeCallRunner; + private int renameTries = 1; // one attempt is already done @Override public Void call() throws Exception { - if (closeTries >= maxCloseTries) { - LOG.warn("Unsuccessfully attempted to close " + path + " " + - maxCloseTries + " times. File may be open, " + - "or may not have been renamed." ); + if (renameTries >= maxRenameTries) { + LOG.warn("Unsuccessfully attempted to rename " + path + " " + + maxRenameTries + " times. File may still be open."); return null; } - closeTries++; + renameTries++; try { - if (!closeSuccess) { - if (isClosedMethod == null) { - LOG.debug("isFileClosed method is not available in " + - "the version of HDFS client being used. " + - "Not attempting to close file again"); - return null; - } - if (!isFileClosed(fs, tmpFilePath)) { - callWithTimeout(closeCall); - } - // It is possible rename failing causes this thread - // to get rescheduled. In that case, - // don't check with NN if close succeeded as we know - // it did. This helps avoid an unnecessary RPC call. - closeSuccess = true; - } renameBucket(path, finalPath, fs); } catch (Exception e) { - LOG.warn("Closing file: " + path + " failed. Will " + + LOG.warn("Renaming file: " + path + " failed. Will " + "retry again in " + retryInterval + " seconds.", e); timedRollerPool.schedule(this, retryInterval, TimeUnit.SECONDS); @@ -422,10 +399,6 @@ public synchronized void close(boolean callCloseCallback) "). Exception follows.", e); sinkCounter.incrementConnectionFailedCount(); failedToClose = true; - final Callable scheduledClose = - createScheduledCloseCallable(closeCallRunner); - timedRollerPool.schedule(scheduledClose, retryInterval, - TimeUnit.SECONDS); } isOpen = false; } else { @@ -443,10 +416,20 @@ public synchronized void close(boolean callCloseCallback) idleFuture = null; } - // Don't rename file if this failed to close - if (bucketPath != null && fileSystem != null && !failedToClose) { + if (bucketPath != null && fileSystem != null) { // could block or throw IOException - renameBucket(bucketPath, targetPath, fileSystem); + try { + renameBucket(bucketPath, targetPath, fileSystem); + } catch(Exception e) { + LOG.warn( + "failed to rename() file (" + bucketPath + + "). Exception follows.", e); + sinkCounter.incrementConnectionFailedCount(); + final Callable scheduledRename = + createScheduledRenameCallable(); + timedRollerPool.schedule(scheduledRename, retryInterval, + TimeUnit.SECONDS); + } } if (callCloseCallback) { runCloseAction(); @@ -671,6 +654,7 @@ private void renameBucket(String bucketPath, public Void call() throws Exception { if (fs.exists(srcPath)) { // could block LOG.info("Renaming " + srcPath + " to " + dstPath); + renameTries.incrementAndGet(); fs.rename(srcPath, dstPath); // could block } return null; diff --git a/flume-ng-sinks/flume-hdfs-sink/src/test/java/org/apache/flume/sink/hdfs/MockFileSystem.java b/flume-ng-sinks/flume-hdfs-sink/src/test/java/org/apache/flume/sink/hdfs/MockFileSystem.java index ca4f852504..444333534b 100644 --- a/flume-ng-sinks/flume-hdfs-sink/src/test/java/org/apache/flume/sink/hdfs/MockFileSystem.java +++ b/flume-ng-sinks/flume-hdfs-sink/src/test/java/org/apache/flume/sink/hdfs/MockFileSystem.java @@ -37,13 +37,22 @@ public class MockFileSystem extends FileSystem { LoggerFactory.getLogger(MockFileSystem.class); FileSystem fs; - int numberOfClosesRequired; + int numberOfRetriesRequired; MockFsDataOutputStream latestOutputStream; + int currentRenameAttempts; + boolean closeSucceed = true; public MockFileSystem(FileSystem fs, - int numberOfClosesRequired) { + int numberOfRetriesRequired) { this.fs = fs; - this.numberOfClosesRequired = numberOfClosesRequired; + this.numberOfRetriesRequired = numberOfRetriesRequired; + } + + public MockFileSystem(FileSystem fs, + int numberOfRetriesRequired, boolean closeSucceed) { + this.fs = fs; + this.numberOfRetriesRequired = numberOfRetriesRequired; + this.closeSucceed = closeSucceed; } @Override @@ -51,7 +60,7 @@ public FSDataOutputStream append(Path arg0, int arg1, Progressable arg2) throws IOException { latestOutputStream = new MockFsDataOutputStream( - fs.append(arg0, arg1, arg2), numberOfClosesRequired); + fs.append(arg0, arg1, arg2), closeSucceed); return latestOutputStream; } @@ -60,7 +69,7 @@ public FSDataOutputStream append(Path arg0, int arg1, Progressable arg2) public FSDataOutputStream create(Path arg0) throws IOException { //throw new IOException ("HI there2"); latestOutputStream = new MockFsDataOutputStream( - fs.create(arg0), numberOfClosesRequired); + fs.create(arg0), closeSucceed); return latestOutputStream; } @@ -116,8 +125,17 @@ public FSDataInputStream open(Path arg0, int arg1) throws IOException { @Override public boolean rename(Path arg0, Path arg1) throws IOException { - - return fs.rename(arg0, arg1); + currentRenameAttempts++; + logger.info( + "Attempting to Rename: '" + currentRenameAttempts + "' of '" + + numberOfRetriesRequired + "'"); + if (currentRenameAttempts >= numberOfRetriesRequired || + numberOfRetriesRequired == 0) { + logger.info("Renaming file"); + return fs.rename(arg0, arg1); + } else { + throw new IOException("MockIOException"); + } } @Override @@ -125,16 +143,4 @@ public void setWorkingDirectory(Path arg0) { fs.setWorkingDirectory(arg0); } - - public boolean isFileClosed(Path path) { - - logger.info("isFileClosed: '" + - latestOutputStream.getCurrentCloseAttempts() + "' , '" + - numberOfClosesRequired + "'"); - return latestOutputStream.getCurrentCloseAttempts() >= - numberOfClosesRequired || numberOfClosesRequired == 0; - } - - - } diff --git a/flume-ng-sinks/flume-hdfs-sink/src/test/java/org/apache/flume/sink/hdfs/MockFsDataOutputStream.java b/flume-ng-sinks/flume-hdfs-sink/src/test/java/org/apache/flume/sink/hdfs/MockFsDataOutputStream.java index 5bbacaeaeb..35b034ef7b 100644 --- a/flume-ng-sinks/flume-hdfs-sink/src/test/java/org/apache/flume/sink/hdfs/MockFsDataOutputStream.java +++ b/flume-ng-sinks/flume-hdfs-sink/src/test/java/org/apache/flume/sink/hdfs/MockFsDataOutputStream.java @@ -28,34 +28,24 @@ public class MockFsDataOutputStream extends FSDataOutputStream{ private static final Logger logger = LoggerFactory.getLogger(MockFsDataOutputStream.class); - int currentCloseAttempts = 0; - int numberOfClosesRequired; + boolean closeSucceed; public MockFsDataOutputStream(FSDataOutputStream wrapMe, - int numberOfClosesRequired) + boolean closeSucceed) throws IOException { super(wrapMe.getWrappedStream(), null); - - this.numberOfClosesRequired = numberOfClosesRequired; - + this.closeSucceed = closeSucceed; } @Override public void close() throws IOException { - currentCloseAttempts++; logger.info( - "Attempting to Close: '" + currentCloseAttempts + "' of '" + - numberOfClosesRequired + "'"); - if (currentCloseAttempts >= numberOfClosesRequired || - numberOfClosesRequired == 0) { + "Close Succeeded - " + closeSucceed); + if (closeSucceed) { logger.info("closing file"); super.close(); } else { throw new IOException("MockIOException"); } } - - public int getCurrentCloseAttempts() { - return currentCloseAttempts; - } } diff --git a/flume-ng-sinks/flume-hdfs-sink/src/test/java/org/apache/flume/sink/hdfs/TestBucketWriter.java b/flume-ng-sinks/flume-hdfs-sink/src/test/java/org/apache/flume/sink/hdfs/TestBucketWriter.java index bcb912ff5b..7c74b162e7 100644 --- a/flume-ng-sinks/flume-hdfs-sink/src/test/java/org/apache/flume/sink/hdfs/TestBucketWriter.java +++ b/flume-ng-sinks/flume-hdfs-sink/src/test/java/org/apache/flume/sink/hdfs/TestBucketWriter.java @@ -408,15 +408,18 @@ public void run(String filePath) { @Test - public void testSequenceFileCloseRetries() throws Exception { - SequenceFileCloseRetryCoreTest(1); - SequenceFileCloseRetryCoreTest(5); - SequenceFileCloseRetryCoreTest(2); + public void testSequenceFileRenameRetries() throws Exception { + SequenceFileRenameRetryCoreTest(1, true); + SequenceFileRenameRetryCoreTest(5, true); + SequenceFileRenameRetryCoreTest(2, true); - } + SequenceFileRenameRetryCoreTest(1, false); + SequenceFileRenameRetryCoreTest(5, false); + SequenceFileRenameRetryCoreTest(2, false); + } - public void SequenceFileCloseRetryCoreTest(int numberOfClosesRequired) throws Exception { + public void SequenceFileRenameRetryCoreTest(int numberOfRetriesRequired, boolean closeSucceed) throws Exception { String hdfsPath = "file:///tmp/flume-test." + Calendar.getInstance().getTimeInMillis() + "." + Thread.currentThread().getId(); @@ -429,13 +432,13 @@ public void SequenceFileCloseRetryCoreTest(int numberOfClosesRequired) throws Ex fs.mkdirs(dirPath); context.put("hdfs.path", hdfsPath); context.put("hdfs.closeTries", - String.valueOf(numberOfClosesRequired)); + String.valueOf(numberOfRetriesRequired)); context.put("hdfs.rollCount", "1"); context.put("hdfs.retryInterval", "1"); context.put("hdfs.callTimeout", Long.toString(1000)); MockFileSystem mockFs = new MockFileSystem(fs, - numberOfClosesRequired); + numberOfRetriesRequired, closeSucceed); BucketWriter bucketWriter = new BucketWriter(0, 0, 1, 1, ctx, hdfsPath, hdfsPath, "singleBucket", ".tmp", null, null, null, new MockDataStream(mockFs), @@ -443,7 +446,7 @@ null, new MockDataStream(mockFs), new SinkCounter( "test-bucket-writer-" + System.currentTimeMillis()), 0, null, null, 30000, Executors.newSingleThreadExecutor(), 1, - numberOfClosesRequired); + numberOfRetriesRequired); bucketWriter.setFileSystem(mockFs); // At this point, we checked if isFileClosed is available in @@ -453,12 +456,11 @@ null, new MockDataStream(mockFs), // This is what triggers the close, so a 2nd append is required :/ bucketWriter.append(event); - TimeUnit.SECONDS.sleep(numberOfClosesRequired + 2); + TimeUnit.SECONDS.sleep(numberOfRetriesRequired + 2); - int expectedNumberOfCloses = numberOfClosesRequired; - Assert.assertTrue("Expected " + expectedNumberOfCloses + " " + - "but got " + bucketWriter.closeTries.get(), - bucketWriter.closeTries.get() == - expectedNumberOfCloses); + Assert.assertTrue("Expected " + numberOfRetriesRequired + " " + + "but got " + bucketWriter.renameTries.get(), + bucketWriter.renameTries.get() == + numberOfRetriesRequired); } } diff --git a/flume-ng-sinks/flume-hdfs-sink/src/test/java/org/apache/flume/sink/hdfs/TestHDFSEventSink.java b/flume-ng-sinks/flume-hdfs-sink/src/test/java/org/apache/flume/sink/hdfs/TestHDFSEventSink.java index f29f1f12de..1b7a364716 100644 --- a/flume-ng-sinks/flume-hdfs-sink/src/test/java/org/apache/flume/sink/hdfs/TestHDFSEventSink.java +++ b/flume-ng-sinks/flume-hdfs-sink/src/test/java/org/apache/flume/sink/hdfs/TestHDFSEventSink.java @@ -1375,10 +1375,16 @@ public void testBadConfigurationForRetryCountNegative() throws Assert.assertEquals(Integer.MAX_VALUE, sink.getTryCount()); } @Test - public void testRetryClose() throws InterruptedException, + public void testRetryRename() throws InterruptedException, LifecycleException, EventDeliveryException, IOException { + testRetryRename(true); + testRetryRename(false); + } + private void testRetryRename(boolean closeSucceed) throws InterruptedException, + LifecycleException, + EventDeliveryException, IOException { LOG.debug("Starting..."); String newPath = testPath + "/retryBucket"; @@ -1388,7 +1394,7 @@ public void testRetryClose() throws InterruptedException, Path dirPath = new Path(newPath); fs.delete(dirPath, true); fs.mkdirs(dirPath); - MockFileSystem mockFs = new MockFileSystem(fs, 3); + MockFileSystem mockFs = new MockFileSystem(fs, 6, closeSucceed); Context context = getContextForRetryTests(); Configurables.configure(sink, context); @@ -1434,15 +1440,15 @@ public void testRetryClose() throws InterruptedException, Collection writers = sink.getSfWriters().values(); - int totalCloseAttempts = 0; + int totalRenameAttempts = 0; for(BucketWriter writer: writers) { - LOG.info("Close tries = "+ writer.closeTries.get()); - totalCloseAttempts += writer.closeTries.get(); + LOG.info("Rename tries = "+ writer.renameTries.get()); + totalRenameAttempts += writer.renameTries.get(); } // stop clears the sfWriters map, so we need to compute the // close tries count before stopping the sink. sink.stop(); - Assert.assertEquals(6, totalCloseAttempts); + Assert.assertEquals(6, totalRenameAttempts); } } From 1d49ef704a8bb08280b4e653e6db94dc3d2c2475 Mon Sep 17 00:00:00 2001 From: Hari Shreedharan Date: Wed, 28 Jan 2015 17:41:51 -0800 Subject: [PATCH 197/341] FLUME-2591. DatasetSink 2.0 (Joey Echeverria via Hari) --- flume-ng-doc/sphinx/FlumeUserGuide.rst | 62 +- .../apache/flume/sink/kite/DatasetSink.java | 650 ++++++++++++------ .../flume/sink/kite/DatasetSinkConstants.java | 74 +- .../kite/NonRecoverableEventException.java | 54 ++ .../flume/sink/kite/parser/AvroParser.java | 208 ++++++ .../flume/sink/kite/parser/EntityParser.java | 56 ++ .../sink/kite/parser/EntityParserFactory.java | 82 +++ .../flume/sink/kite/policy/FailurePolicy.java | 105 +++ .../kite/policy/FailurePolicyFactory.java | 82 +++ .../flume/sink/kite/policy/RetryPolicy.java | 63 ++ .../flume/sink/kite/policy/SavePolicy.java | 125 ++++ .../flume/sink/kite/TestDatasetSink.java | 411 ++++++++++- pom.xml | 2 +- 13 files changed, 1730 insertions(+), 244 deletions(-) create mode 100644 flume-ng-sinks/flume-dataset-sink/src/main/java/org/apache/flume/sink/kite/NonRecoverableEventException.java create mode 100644 flume-ng-sinks/flume-dataset-sink/src/main/java/org/apache/flume/sink/kite/parser/AvroParser.java create mode 100644 flume-ng-sinks/flume-dataset-sink/src/main/java/org/apache/flume/sink/kite/parser/EntityParser.java create mode 100644 flume-ng-sinks/flume-dataset-sink/src/main/java/org/apache/flume/sink/kite/parser/EntityParserFactory.java create mode 100644 flume-ng-sinks/flume-dataset-sink/src/main/java/org/apache/flume/sink/kite/policy/FailurePolicy.java create mode 100644 flume-ng-sinks/flume-dataset-sink/src/main/java/org/apache/flume/sink/kite/policy/FailurePolicyFactory.java create mode 100644 flume-ng-sinks/flume-dataset-sink/src/main/java/org/apache/flume/sink/kite/policy/RetryPolicy.java create mode 100644 flume-ng-sinks/flume-dataset-sink/src/main/java/org/apache/flume/sink/kite/policy/SavePolicy.java diff --git a/flume-ng-doc/sphinx/FlumeUserGuide.rst b/flume-ng-doc/sphinx/FlumeUserGuide.rst index bcadc2d14d..7a1dfce394 100644 --- a/flume-ng-doc/sphinx/FlumeUserGuide.rst +++ b/flume-ng-doc/sphinx/FlumeUserGuide.rst @@ -2158,23 +2158,51 @@ Note 2: In some cases, file rolling may occur slightly after the roll interval has been exceeded. However, this delay will not exceed 5 seconds. In most cases, the delay is neglegible. -======================= ======= =========================================================== -Property Name Default Description -======================= ======= =========================================================== -**channel** -- -**type** -- Must be org.apache.flume.sink.kite.DatasetSink -**kite.dataset.uri** -- URI of the dataset to open -kite.repo.uri -- URI of the repository to open - (deprecated; use kite.dataset.uri instead) -kite.dataset.name -- Name of the Dataset where records will be written - (deprecated; use kite.dataset.uri instead) -kite.batchSize 100 Number of records to process in each batch -kite.rollInterval 30 Maximum wait time (seconds) before data files are released -auth.kerberosPrincipal -- Kerberos user principal for secure authentication to HDFS -auth.kerberosKeytab -- Kerberos keytab location (local FS) for the principal -auth.proxyUser -- The effective user for HDFS actions, if different from - the kerberos principal -======================= ======= =========================================================== +============================ ======= =========================================================== +Property Name Default Description +============================ ======= =========================================================== +**channel** -- +**type** -- Must be org.apache.flume.sink.kite.DatasetSink +**kite.dataset.uri** -- URI of the dataset to open +kite.repo.uri -- URI of the repository to open + (deprecated; use kite.dataset.uri instead) +kite.dataset.namespace -- Namespace of the Dataset where records will be written + (deprecated; use kite.dataset.uri instead) +kite.dataset.name -- Name of the Dataset where records will be written + (deprecated; use kite.dataset.uri instead) +kite.batchSize 100 Number of records to process in each batch +kite.rollInterval 30 Maximum wait time (seconds) before data files are released +kite.flushable.commitOnBatch true If ``true``, the Flume transaction will be commited and the + writer will be flushed on each batch of ``kite.batchSize`` + records. This setting only applies to flushable datasets. When + ``true``, it's possible for temp files with commited data to be + left in the dataset directory. These files need to be recovered + by hand for the data to be visible to DatasetReaders. +kite.syncable.syncOnBatch true Controls whether the sink will also sync data when committing + the transaction. This setting only applies to syncable datasets. + Syncing gaurentees that data will be written on stable storage + on the remote system while flushing only gaurentees that data + has left Flume's client buffers. When the + ``kite.flushable.commitOnBatch`` property is set to ``false``, + this property must also be set to ``false``. +kite.entityParser avro Parser that turns Flume ``Events`` into Kite entities. + Valid values are ``avro`` and the fully-qualified class name + of an implementation of the ``EntityParser.Builder`` interface. +kite.failurePolicy retry Policy that handles non-recoverable errors such as a missing + ``Schema`` in the ``Event`` header. The default value, ``retry``, + will fail the current batch and try again which matches the old + behavior. Other valid values are ``save``, which will write the + raw ``Event`` to the ``kite.error.dataset.uri`` dataset, and the + fully-qualified class name of an implementation of the + ``FailurePolicy.Builder`` interface. +kite.error.dataset.uri -- URI of the dataset where failed events are saved when + ``kite.failurePolicy`` is set to ``save``. **Required** when + the ``kite.failurePolicy`` is set to ``save``. +auth.kerberosPrincipal -- Kerberos user principal for secure authentication to HDFS +auth.kerberosKeytab -- Kerberos keytab location (local FS) for the principal +auth.proxyUser -- The effective user for HDFS actions, if different from + the kerberos principal +============================ ======= =========================================================== Kafka Sink diff --git a/flume-ng-sinks/flume-dataset-sink/src/main/java/org/apache/flume/sink/kite/DatasetSink.java b/flume-ng-sinks/flume-dataset-sink/src/main/java/org/apache/flume/sink/kite/DatasetSink.java index ebcc6174fe..3e665324b7 100644 --- a/flume-ng-sinks/flume-dataset-sink/src/main/java/org/apache/flume/sink/kite/DatasetSink.java +++ b/flume-ng-sinks/flume-dataset-sink/src/main/java/org/apache/flume/sink/kite/DatasetSink.java @@ -15,31 +15,23 @@ * See the License for the specific language governing permissions and * limitations under the License. */ - package org.apache.flume.sink.kite; +import org.apache.flume.sink.kite.parser.EntityParserFactory; +import org.apache.flume.sink.kite.parser.EntityParser; +import org.apache.flume.sink.kite.policy.FailurePolicy; +import org.apache.flume.sink.kite.policy.FailurePolicyFactory; import com.google.common.annotations.VisibleForTesting; import com.google.common.base.Preconditions; import com.google.common.base.Throwables; -import com.google.common.cache.CacheBuilder; -import com.google.common.cache.CacheLoader; -import com.google.common.cache.LoadingCache; import com.google.common.collect.Lists; -import java.io.IOException; -import java.io.InputStream; import java.net.URI; -import java.net.URL; import java.security.PrivilegedExceptionAction; import java.util.List; -import java.util.Locale; -import java.util.Map; -import java.util.concurrent.ExecutionException; +import java.util.concurrent.TimeUnit; import org.apache.avro.Schema; -import org.apache.avro.generic.GenericDatumReader; +import org.apache.avro.file.DataFileWriter; import org.apache.avro.generic.GenericRecord; -import org.apache.avro.io.BinaryDecoder; -import org.apache.avro.io.DatumReader; -import org.apache.avro.io.DecoderFactory; import org.apache.flume.Channel; import org.apache.flume.Context; import org.apache.flume.Event; @@ -48,144 +40,180 @@ import org.apache.flume.conf.Configurable; import org.apache.flume.instrumentation.SinkCounter; import org.apache.flume.sink.AbstractSink; -import org.apache.hadoop.conf.Configuration; -import org.apache.hadoop.fs.FileSystem; -import org.apache.hadoop.fs.Path; import org.apache.hadoop.security.UserGroupInformation; import org.kitesdk.data.Dataset; import org.kitesdk.data.DatasetDescriptor; -import org.kitesdk.data.DatasetException; +import org.kitesdk.data.DatasetIOException; +import org.kitesdk.data.DatasetNotFoundException; import org.kitesdk.data.DatasetWriter; +import org.kitesdk.data.DatasetWriterException; import org.kitesdk.data.Datasets; import org.kitesdk.data.View; import org.kitesdk.data.spi.Registration; -import org.kitesdk.data.spi.URIBuilder; +import org.kitesdk.data.URIBuilder; import org.slf4j.Logger; import org.slf4j.LoggerFactory; +import static org.apache.flume.sink.kite.DatasetSinkConstants.*; +import org.kitesdk.data.Format; +import org.kitesdk.data.Formats; + /** - * Experimental sink that writes events to a Kite Dataset. This sink will - * deserialize the body of each incoming event and store the resulting record - * in a Kite Dataset. It determines target Dataset by opening a repository URI, - * {@code kite.repo.uri}, and loading a Dataset by name, - * {@code kite.dataset.name}. + * Sink that writes events to a Kite Dataset. This sink will parse the body of + * each incoming event and store the resulting entity in a Kite Dataset. It + * determines the destination Dataset by opening a dataset URI + * {@code kite.dataset.uri} or opening a repository URI, {@code kite.repo.uri}, + * and loading a Dataset by name, {@code kite.dataset.name}, and namespace, + * {@code kite.dataset.namespace}. */ public class DatasetSink extends AbstractSink implements Configurable { private static final Logger LOG = LoggerFactory.getLogger(DatasetSink.class); - static Configuration conf = new Configuration(); + private Context context = null; + private UserGroupInformation login = null; private String datasetName = null; - private long batchSize = DatasetSinkConstants.DEFAULT_BATCH_SIZE; - - private URI target = null; - private Schema targetSchema = null; + private URI datasetUri = null; + private Schema datasetSchema = null; private DatasetWriter writer = null; - private UserGroupInformation login = null; - private SinkCounter counter = null; - // for rolling files at a given interval - private int rollIntervalS = DatasetSinkConstants.DEFAULT_ROLL_INTERVAL; - private long lastRolledMs = 0l; + /** + * The number of events to process as a single batch. + */ + private long batchSize = DEFAULT_BATCH_SIZE; + + /** + * The number of seconds to wait before rolling a writer. + */ + private int rollIntervalSeconds = DEFAULT_ROLL_INTERVAL; + + /** + * Flag that says if Flume should commit on every batch. + */ + private boolean commitOnBatch = DEFAULT_FLUSHABLE_COMMIT_ON_BATCH; + + /** + * Flag that says if Flume should sync on every batch. + */ + private boolean syncOnBatch = DEFAULT_SYNCABLE_SYNC_ON_BATCH; + + /** + * The last time the writer rolled. + */ + private long lastRolledMillis = 0l; - // for working with avro serialized records - private GenericRecord datum = null; + /** + * The raw number of bytes parsed. + */ + private long bytesParsed = 0l; + + /** + * A class for parsing Kite entities from Flume Events. + */ + private EntityParser parser = null; + + /** + * A class implementing a failure newPolicy for events that had a + non-recoverable error during processing. + */ + private FailurePolicy failurePolicy = null; + + private SinkCounter counter = null; + + /** + * The Kite entity + */ + private GenericRecord entity = null; // TODO: remove this after PARQUET-62 is released - private boolean reuseDatum = true; - private BinaryDecoder decoder = null; - private LoadingCache> readers = - CacheBuilder.newBuilder() - .build(new CacheLoader>() { - @Override - public DatumReader load(Schema schema) { - // must use the target dataset's schema for reading to ensure the - // records are able to be stored using it - return new GenericDatumReader( - schema, targetSchema); - } - }); - private static LoadingCache schemasFromLiteral = CacheBuilder - .newBuilder() - .build(new CacheLoader() { - @Override - public Schema load(String literal) { - Preconditions.checkNotNull(literal, - "Schema literal cannot be null without a Schema URL"); - return new Schema.Parser().parse(literal); - } - }); - private static LoadingCache schemasFromURL = CacheBuilder - .newBuilder() - .build(new CacheLoader() { - @Override - public Schema load(String url) throws IOException { - Schema.Parser parser = new Schema.Parser(); - InputStream is = null; - try { - FileSystem fs = FileSystem.get(URI.create(url), conf); - if (url.toLowerCase(Locale.ENGLISH).startsWith("hdfs:/")) { - is = fs.open(new Path(url)); - } else { - is = new URL(url).openStream(); - } - return parser.parse(is); - } finally { - if (is != null) { - is.close(); - } - } - } - }); + private boolean reuseEntity = true; + + /** + * The Flume transaction. Used to keep transactions open across calls to + * process. + */ + private Transaction transaction = null; + + /** + * Internal flag on if there has been a batch of records committed. This is + * used during rollback to know if the current writer needs to be closed. + */ + private boolean committedBatch = false; + // Factories + private static final EntityParserFactory ENTITY_PARSER_FACTORY = + new EntityParserFactory(); + private static final FailurePolicyFactory FAILURE_POLICY_FACTORY = + new FailurePolicyFactory(); + + /** + * Return the list of allowed formats. + * @return The list of allowed formats. + */ protected List allowedFormats() { return Lists.newArrayList("avro", "parquet"); } @Override public void configure(Context context) { + this.context = context; + // initialize login credentials this.login = KerberosUtil.login( - context.getString(DatasetSinkConstants.AUTH_PRINCIPAL), - context.getString(DatasetSinkConstants.AUTH_KEYTAB)); - String effectiveUser = - context.getString(DatasetSinkConstants.AUTH_PROXY_USER); + context.getString(AUTH_PRINCIPAL), + context.getString(AUTH_KEYTAB)); + String effectiveUser + = context.getString(AUTH_PROXY_USER); if (effectiveUser != null) { this.login = KerberosUtil.proxyAs(effectiveUser, login); } - String datasetURI = context.getString( - DatasetSinkConstants.CONFIG_KITE_DATASET_URI); + // Get the dataset URI and name from the context + String datasetURI = context.getString(CONFIG_KITE_DATASET_URI); if (datasetURI != null) { - this.target = URI.create(datasetURI); - this.datasetName = uriToName(target); + this.datasetUri = URI.create(datasetURI); + this.datasetName = uriToName(datasetUri); } else { - String repositoryURI = context.getString( - DatasetSinkConstants.CONFIG_KITE_REPO_URI); - Preconditions.checkNotNull(repositoryURI, "Repository URI is missing"); - this.datasetName = context.getString( - DatasetSinkConstants.CONFIG_KITE_DATASET_NAME); - Preconditions.checkNotNull(datasetName, "Dataset name is missing"); - - this.target = new URIBuilder(repositoryURI, datasetName).build(); + String repositoryURI = context.getString(CONFIG_KITE_REPO_URI); + Preconditions.checkNotNull(repositoryURI, "No dataset configured. Setting " + + CONFIG_KITE_DATASET_URI + " is required."); + + this.datasetName = context.getString(CONFIG_KITE_DATASET_NAME); + Preconditions.checkNotNull(datasetName, "No dataset configured. Setting " + + CONFIG_KITE_DATASET_URI + " is required."); + + String namespace = context.getString(CONFIG_KITE_DATASET_NAMESPACE, + DEFAULT_NAMESPACE); + + this.datasetUri = new URIBuilder(repositoryURI, namespace, datasetName) + .build(); + } + this.setName(datasetUri.toString()); + + if (context.getBoolean(CONFIG_SYNCABLE_SYNC_ON_BATCH, + DEFAULT_SYNCABLE_SYNC_ON_BATCH)) { + Preconditions.checkArgument( + context.getBoolean(CONFIG_FLUSHABLE_COMMIT_ON_BATCH, + DEFAULT_FLUSHABLE_COMMIT_ON_BATCH), "Configuration error: " + + CONFIG_FLUSHABLE_COMMIT_ON_BATCH + " must be set to true when " + + CONFIG_SYNCABLE_SYNC_ON_BATCH + " is set to true."); } - this.setName(target.toString()); + // Create the configured failure failurePolicy + this.failurePolicy = FAILURE_POLICY_FACTORY.newPolicy(context); // other configuration - this.batchSize = context.getLong( - DatasetSinkConstants.CONFIG_KITE_BATCH_SIZE, - DatasetSinkConstants.DEFAULT_BATCH_SIZE); - this.rollIntervalS = context.getInteger( - DatasetSinkConstants.CONFIG_KITE_ROLL_INTERVAL, - DatasetSinkConstants.DEFAULT_ROLL_INTERVAL); + this.batchSize = context.getLong(CONFIG_KITE_BATCH_SIZE, + DEFAULT_BATCH_SIZE); + this.rollIntervalSeconds = context.getInteger(CONFIG_KITE_ROLL_INTERVAL, + DEFAULT_ROLL_INTERVAL); this.counter = new SinkCounter(datasetName); } @Override public synchronized void start() { - this.lastRolledMs = System.currentTimeMillis(); + this.lastRolledMillis = System.currentTimeMillis(); counter.start(); // signal that this sink is ready to process LOG.info("Started DatasetSink " + getName()); @@ -196,183 +224,359 @@ public synchronized void start() { * Causes the sink to roll at the next {@link #process()} call. */ @VisibleForTesting - public void roll() { - this.lastRolledMs = 0l; + void roll() { + this.lastRolledMillis = 0l; + } + + @VisibleForTesting + DatasetWriter getWriter() { + return writer; + } + + @VisibleForTesting + void setWriter(DatasetWriter writer) { + this.writer = writer; + } + + @VisibleForTesting + void setParser(EntityParser parser) { + this.parser = parser; + } + + @VisibleForTesting + void setFailurePolicy(FailurePolicy failurePolicy) { + this.failurePolicy = failurePolicy; } @Override public synchronized void stop() { counter.stop(); - if (writer != null) { - // any write problems invalidate the writer, which is immediately closed - writer.close(); - this.writer = null; - this.lastRolledMs = System.currentTimeMillis(); + try { + // Close the writer and commit the transaction, but don't create a new + // writer since we're stopping + closeWriter(); + commitTransaction(); + } catch (EventDeliveryException ex) { + rollbackTransaction(); + + LOG.warn("Closing the writer failed: " + ex.getLocalizedMessage()); + LOG.debug("Exception follows.", ex); + // We don't propogate the exception as the transaction would have been + // rolled back and we can still finish stopping } - // signal that this sink has stopped + // signal that this sink has stopped LOG.info("Stopped dataset sink: " + getName()); super.stop(); } @Override public Status process() throws EventDeliveryException { - if (writer == null) { - try { - this.writer = newWriter(login, target); - } catch (DatasetException e) { - // DatasetException includes DatasetNotFoundException - throw new EventDeliveryException( - "Cannot write to " + getName(), e); + long processedEvents = 0; + + try { + if (shouldRoll()) { + closeWriter(); + commitTransaction(); + createWriter(); } - } - // handle file rolling - if ((System.currentTimeMillis() - lastRolledMs) / 1000 > rollIntervalS) { - // close the current writer and get a new one - writer.close(); - this.writer = newWriter(login, target); - this.lastRolledMs = System.currentTimeMillis(); - LOG.info("Rolled writer for " + getName()); - } + // The writer shouldn't be null at this point + Preconditions.checkNotNull(writer, + "Can't process events with a null writer. This is likely a bug."); + Channel channel = getChannel(); - Channel channel = getChannel(); - Transaction transaction = null; - try { - long processedEvents = 0; + // Enter the transaction boundary if we haven't already + enterTransaction(channel); - transaction = channel.getTransaction(); - transaction.begin(); for (; processedEvents < batchSize; processedEvents += 1) { Event event = channel.take(); + if (event == null) { // no events available in the channel break; } - this.datum = deserialize(event, reuseDatum ? datum : null); - - // writeEncoded would be an optimization in some cases, but HBase - // will not support it and partitioned Datasets need to get partition - // info from the entity Object. We may be able to avoid the - // serialization round-trip otherwise. - writer.write(datum); + write(event); } - // TODO: Add option to sync, depends on CDK-203 - writer.flush(); - - // commit after data has been written and flushed - transaction.commit(); - - if (processedEvents == 0) { - counter.incrementBatchEmptyCount(); - return Status.BACKOFF; - } else if (processedEvents < batchSize) { - counter.incrementBatchUnderflowCount(); - } else { - counter.incrementBatchCompleteCount(); + // commit transaction + if (commitOnBatch) { + // Flush/sync before commiting. A failure here will result in rolling back + // the transaction + if (syncOnBatch) { + writer.sync(); + } else { + writer.flush(); + } + boolean committed = commitTransaction(); + Preconditions.checkState(committed, + "Tried to commit a batch when there was no transaction"); + committedBatch |= committed; } - - counter.addToEventDrainSuccessCount(processedEvents); - - return Status.READY; - } catch (Throwable th) { // catch-all for any unhandled Throwable so that the transaction is // correctly rolled back. - if (transaction != null) { + rollbackTransaction(); + + if (commitOnBatch && committedBatch) { try { - transaction.rollback(); - } catch (Exception ex) { - LOG.error("Transaction rollback failed", ex); - throw Throwables.propagate(ex); + closeWriter(); + } catch (EventDeliveryException ex) { + LOG.warn("Error closing writer there may be temp files that need to" + + " be manually recovered: " + ex.getLocalizedMessage()); + LOG.debug("Exception follows.", ex); } + } else { + this.writer = null; } - // close the writer and remove the its reference - writer.close(); - this.writer = null; - this.lastRolledMs = System.currentTimeMillis(); - // handle the exception Throwables.propagateIfInstanceOf(th, Error.class); Throwables.propagateIfInstanceOf(th, EventDeliveryException.class); throw new EventDeliveryException(th); - - } finally { - if (transaction != null) { - transaction.close(); - } } - } - private DatasetWriter newWriter( - final UserGroupInformation login, final URI uri) { - View view = KerberosUtil.runPrivileged(login, - new PrivilegedExceptionAction>() { - @Override - public Dataset run() { - return Datasets.load(uri); - } - }); - - DatasetDescriptor descriptor = view.getDataset().getDescriptor(); - String formatName = descriptor.getFormat().getName(); - Preconditions.checkArgument(allowedFormats().contains(formatName), - "Unsupported format: " + formatName); - - Schema newSchema = descriptor.getSchema(); - if (targetSchema == null || !newSchema.equals(targetSchema)) { - this.targetSchema = descriptor.getSchema(); - // target dataset schema has changed, invalidate all readers based on it - readers.invalidateAll(); + if (processedEvents == 0) { + counter.incrementBatchEmptyCount(); + return Status.BACKOFF; + } else if (processedEvents < batchSize) { + counter.incrementBatchUnderflowCount(); + } else { + counter.incrementBatchCompleteCount(); } - this.reuseDatum = !("parquet".equals(formatName)); - this.datasetName = view.getDataset().getName(); + counter.addToEventDrainSuccessCount(processedEvents); - return view.newWriter(); + return Status.READY; } /** - * Not thread-safe. + * Parse the event using the entity parser and write the entity to the dataset. * - * @param event - * @param reuse - * @return + * @param event The event to write + * @throws EventDeliveryException An error occurred trying to write to the + dataset that couldn't or shouldn't be + handled by the failure policy. */ - private GenericRecord deserialize(Event event, GenericRecord reuse) - throws EventDeliveryException { - decoder = DecoderFactory.get().binaryDecoder(event.getBody(), decoder); - // no checked exception is thrown in the CacheLoader - DatumReader reader = readers.getUnchecked(schema(event)); + @VisibleForTesting + void write(Event event) throws EventDeliveryException { try { - return reader.read(reuse, decoder); - } catch (IOException ex) { - throw new EventDeliveryException("Cannot deserialize event", ex); + this.entity = parser.parse(event, reuseEntity ? entity : null); + this.bytesParsed += event.getBody().length; + + // writeEncoded would be an optimization in some cases, but HBase + // will not support it and partitioned Datasets need to get partition + // info from the entity Object. We may be able to avoid the + // serialization round-trip otherwise. + writer.write(entity); + } catch (NonRecoverableEventException ex) { + failurePolicy.handle(event, ex); + } catch (DataFileWriter.AppendWriteException ex) { + failurePolicy.handle(event, ex); + } catch (RuntimeException ex) { + Throwables.propagateIfInstanceOf(ex, EventDeliveryException.class); + throw new EventDeliveryException(ex); } } - private static Schema schema(Event event) throws EventDeliveryException { - Map headers = event.getHeaders(); - String schemaURL = headers.get( - DatasetSinkConstants.AVRO_SCHEMA_URL_HEADER); + /** + * Create a new writer. + * + * This method also re-loads the dataset so updates to the configuration or + * a dataset created after Flume starts will be loaded. + * + * @throws EventDeliveryException There was an error creating the writer. + */ + @VisibleForTesting + void createWriter() throws EventDeliveryException { + // reset the commited flag whenver a new writer is created + committedBatch = false; try { - if (headers.get(DatasetSinkConstants.AVRO_SCHEMA_URL_HEADER) != null) { - return schemasFromURL.get(schemaURL); - } else { - return schemasFromLiteral.get( - headers.get(DatasetSinkConstants.AVRO_SCHEMA_LITERAL_HEADER)); + View view = KerberosUtil.runPrivileged(login, + new PrivilegedExceptionAction>() { + @Override + public Dataset run() { + return Datasets.load(datasetUri); + } + }); + + DatasetDescriptor descriptor = view.getDataset().getDescriptor(); + Format format = descriptor.getFormat(); + Preconditions.checkArgument(allowedFormats().contains(format.getName()), + "Unsupported format: " + format.getName()); + + Schema newSchema = descriptor.getSchema(); + if (datasetSchema == null || !newSchema.equals(datasetSchema)) { + this.datasetSchema = descriptor.getSchema(); + // dataset schema has changed, create a new parser + parser = ENTITY_PARSER_FACTORY.newParser(datasetSchema, context); } - } catch (ExecutionException ex) { - throw new EventDeliveryException("Cannot get schema", ex.getCause()); + + this.reuseEntity = !(Formats.PARQUET.equals(format)); + + // TODO: Check that the format implements Flushable after CDK-863 + // goes in. For now, just check that the Dataset is Avro format + this.commitOnBatch = context.getBoolean(CONFIG_FLUSHABLE_COMMIT_ON_BATCH, + DEFAULT_FLUSHABLE_COMMIT_ON_BATCH) && (Formats.AVRO.equals(format)); + + // TODO: Check that the format implements Syncable after CDK-863 + // goes in. For now, just check that the Dataset is Avro format + this.syncOnBatch = context.getBoolean(CONFIG_SYNCABLE_SYNC_ON_BATCH, + DEFAULT_SYNCABLE_SYNC_ON_BATCH) && (Formats.AVRO.equals(format)); + + this.datasetName = view.getDataset().getName(); + + this.writer = view.newWriter(); + + // Reset the last rolled time and the metrics + this.lastRolledMillis = System.currentTimeMillis(); + this.bytesParsed = 0l; + } catch (DatasetNotFoundException ex) { + throw new EventDeliveryException("Dataset " + datasetUri + " not found." + + " The dataset must be created before Flume can write to it.", ex); + } catch (RuntimeException ex) { + throw new EventDeliveryException("Error trying to open a new" + + " writer for dataset " + datasetUri, ex); } } + /** + * Return true if the sink should roll the writer. + * + * Currently, this is based on time since the last roll or if the current + * writer is null. + * + * @return True if and only if the sink should roll the writer + */ + private boolean shouldRoll() { + long currentTimeMillis = System.currentTimeMillis(); + long elapsedTimeSeconds = TimeUnit.MILLISECONDS.toSeconds( + currentTimeMillis - lastRolledMillis); + + LOG.debug("Current time: {}, lastRolled: {}, diff: {} sec", + new Object[] {currentTimeMillis, lastRolledMillis, elapsedTimeSeconds}); + + return elapsedTimeSeconds >= rollIntervalSeconds || writer == null; + } + + /** + * Close the current writer. + * + * This method always sets the current writer to null even if close fails. + * If this method throws an Exception, callers *must* rollback any active + * transaction to ensure that data is replayed. + * + * @throws EventDeliveryException + */ + @VisibleForTesting + void closeWriter() throws EventDeliveryException { + if (writer != null) { + try { + writer.close(); + + long elapsedTimeSeconds = TimeUnit.MILLISECONDS.toSeconds( + System.currentTimeMillis() - lastRolledMillis); + LOG.info("Closed writer for {} after {} seconds and {} bytes parsed", + new Object[]{datasetUri, elapsedTimeSeconds, bytesParsed}); + } catch (DatasetIOException ex) { + throw new EventDeliveryException("Check HDFS permissions/health. IO" + + " error trying to close the writer for dataset " + datasetUri, + ex); + } catch (DatasetWriterException ex) { + throw new EventDeliveryException("Failure moving temp file.", ex); + } catch (RuntimeException ex) { + throw new EventDeliveryException("Error trying to close the writer for" + + " dataset " + datasetUri, ex); + } finally { + // If we failed to close the writer then we give up on it as we'll + // end up throwing an EventDeliveryException which will result in + // a transaction rollback and a replay of any events written during + // the current transaction. If commitOnBatch is true, you can still + // end up with orphaned temp files that have data to be recovered. + this.writer = null; + failurePolicy.close(); + } + } + } + + /** + * Enter the transaction boundary. This will either begin a new transaction + * if one didn't already exist. If we're already in a transaction boundary, + * then this method does nothing. + * + * @param channel The Sink's channel + * @throws EventDeliveryException There was an error starting a new batch + * with the failure policy. + */ + private void enterTransaction(Channel channel) throws EventDeliveryException { + // There's no synchronization around the transaction instance because the + // Sink API states "the Sink#process() call is guaranteed to only + // be accessed by a single thread". Technically other methods could be + // called concurrently, but the implementation of SinkRunner waits + // for the Thread running process() to end before calling stop() + if (transaction == null) { + this.transaction = channel.getTransaction(); + transaction.begin(); + failurePolicy = FAILURE_POLICY_FACTORY.newPolicy(context); + } + } + + /** + * Commit and close the transaction. + * + * If this method throws an Exception the caller *must* ensure that the + * transaction is rolled back. Callers can roll back the transaction by + * calling {@link #rollbackTransaction()}. + * + * @return True if there was an open transaction and it was committed, false + * otherwise. + * @throws EventDeliveryException There was an error ending the batch with + * the failure policy. + */ + @VisibleForTesting + boolean commitTransaction() throws EventDeliveryException { + if (transaction != null) { + failurePolicy.sync(); + transaction.commit(); + transaction.close(); + this.transaction = null; + return true; + } else { + return false; + } + } + + /** + * Rollback the transaction. If there is a RuntimeException during rollback, + * it will be logged but the transaction instance variable will still be + * nullified. + */ + private void rollbackTransaction() { + if (transaction != null) { + try { + // If the transaction wasn't committed before we got the exception, we + // need to rollback. + transaction.rollback(); + } catch (RuntimeException ex) { + LOG.error("Transaction rollback failed: " + ex.getLocalizedMessage()); + LOG.debug("Exception follows.", ex); + } finally { + transaction.close(); + this.transaction = null; + } + } +} + + /** + * Get the name of the dataset from the URI + * + * @param uri The dataset or view URI + * @return The dataset name + */ private static String uriToName(URI uri) { return Registration.lookupDatasetUri(URI.create( - uri.getRawSchemeSpecificPart())).second().get("dataset"); + uri.getRawSchemeSpecificPart())).second().get("dataset"); } } diff --git a/flume-ng-sinks/flume-dataset-sink/src/main/java/org/apache/flume/sink/kite/DatasetSinkConstants.java b/flume-ng-sinks/flume-dataset-sink/src/main/java/org/apache/flume/sink/kite/DatasetSinkConstants.java index 3c67738a49..af33304c23 100644 --- a/flume-ng-sinks/flume-dataset-sink/src/main/java/org/apache/flume/sink/kite/DatasetSinkConstants.java +++ b/flume-ng-sinks/flume-dataset-sink/src/main/java/org/apache/flume/sink/kite/DatasetSinkConstants.java @@ -18,9 +18,11 @@ package org.apache.flume.sink.kite; +import org.kitesdk.data.URIBuilder; + public class DatasetSinkConstants { /** - * URI of the Kite DatasetRepository. + * URI of the Kite Dataset */ public static final String CONFIG_KITE_DATASET_URI = "kite.dataset.uri"; @@ -34,6 +36,13 @@ public class DatasetSinkConstants { */ public static final String CONFIG_KITE_DATASET_NAME = "kite.dataset.name"; + /** + * Namespace of the Kite Dataset to write into. + */ + public static final String CONFIG_KITE_DATASET_NAMESPACE = + "kite.dataset.namespace"; + public static final String DEFAULT_NAMESPACE = URIBuilder.NAMESPACE_DEFAULT; + /** * Number of records to process from the incoming channel per call to process. */ @@ -47,7 +56,68 @@ public class DatasetSinkConstants { public static int DEFAULT_ROLL_INTERVAL = 30; // seconds /** - * Headers with avro schema information is expected. + * Flag for committing the Flume transaction on each batch for Flushable + * datasets. When set to false, Flume will only commit the transaction when + * roll interval has expired. Setting this to false requires enough space + * in the channel to handle all events delivered during the roll interval. + * Defaults to true. + */ + public static final String CONFIG_FLUSHABLE_COMMIT_ON_BATCH = + "kite.flushable.commiteOnBatch"; + public static boolean DEFAULT_FLUSHABLE_COMMIT_ON_BATCH = true; + + /** + * Flag for syncing the DatasetWriter on each batch for Syncable + * datasets. Defaults to true. + */ + public static final String CONFIG_SYNCABLE_SYNC_ON_BATCH = + "kite.syncable.syncOnBatch"; + public static boolean DEFAULT_SYNCABLE_SYNC_ON_BATCH = true; + + /** + * Parser used to parse Flume Events into Kite entities. + */ + public static final String CONFIG_ENTITY_PARSER = "kite.entityParser"; + + /** + * Built-in entity parsers + */ + public static final String AVRO_ENTITY_PARSER = "avro"; + public static final String DEFAULT_ENTITY_PARSER = AVRO_ENTITY_PARSER; + public static final String[] AVAILABLE_PARSERS = new String[] { + AVRO_ENTITY_PARSER + }; + + /** + * Policy used to handle non-recoverable failures. + */ + public static final String CONFIG_FAILURE_POLICY = "kite.failurePolicy"; + + /** + * Write non-recoverable Flume events to a Kite dataset. + */ + public static final String SAVE_FAILURE_POLICY = "save"; + + /** + * The URI to write non-recoverable Flume events to in the case of an error. + * If the dataset doesn't exist, it will be created. + */ + public static final String CONFIG_KITE_ERROR_DATASET_URI = + "kite.error.dataset.uri"; + + /** + * Retry non-recoverable Flume events. This will lead to a never ending cycle + * of failure, but matches the previous default semantics of the DatasetSink. + */ + public static final String RETRY_FAILURE_POLICY = "retry"; + public static final String DEFAULT_FAILURE_POLICY = RETRY_FAILURE_POLICY; + public static final String[] AVAILABLE_POLICIES = new String[] { + RETRY_FAILURE_POLICY, + SAVE_FAILURE_POLICY + }; + + /** + * Headers where avro schema information is expected. */ public static final String AVRO_SCHEMA_LITERAL_HEADER = "flume.avro.schema.literal"; diff --git a/flume-ng-sinks/flume-dataset-sink/src/main/java/org/apache/flume/sink/kite/NonRecoverableEventException.java b/flume-ng-sinks/flume-dataset-sink/src/main/java/org/apache/flume/sink/kite/NonRecoverableEventException.java new file mode 100644 index 0000000000..8f6c0aeeae --- /dev/null +++ b/flume-ng-sinks/flume-dataset-sink/src/main/java/org/apache/flume/sink/kite/NonRecoverableEventException.java @@ -0,0 +1,54 @@ +/** + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.flume.sink.kite; + + +/** + * A non-recoverable error trying to deliver the event. + * + * Non-recoverable event delivery failures include: + * + * 1. Error parsing the event body thrown from the {@link EntityParser} + * 2. A schema mismatch between the schema of an event and the schema of the + * destination dataset. + * 3. A missing schema from the Event header when using the + * {@link AvroEntityParser}. + */ +public class NonRecoverableEventException extends Exception { + + private static final long serialVersionUID = 3485151222482254285L; + + public NonRecoverableEventException() { + super(); + } + + public NonRecoverableEventException(String message) { + super(message); + } + + public NonRecoverableEventException(String message, Throwable t) { + super(message, t); + } + + public NonRecoverableEventException(Throwable t) { + super(t); + } + + +} diff --git a/flume-ng-sinks/flume-dataset-sink/src/main/java/org/apache/flume/sink/kite/parser/AvroParser.java b/flume-ng-sinks/flume-dataset-sink/src/main/java/org/apache/flume/sink/kite/parser/AvroParser.java new file mode 100644 index 0000000000..7c6a723302 --- /dev/null +++ b/flume-ng-sinks/flume-dataset-sink/src/main/java/org/apache/flume/sink/kite/parser/AvroParser.java @@ -0,0 +1,208 @@ +/** + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.flume.sink.kite.parser; + +import com.google.common.base.Preconditions; +import com.google.common.cache.CacheBuilder; +import com.google.common.cache.CacheLoader; +import com.google.common.cache.LoadingCache; +import com.google.common.util.concurrent.UncheckedExecutionException; +import java.io.IOException; +import java.io.InputStream; +import java.net.URI; +import java.net.URL; +import java.util.Locale; +import java.util.Map; +import java.util.concurrent.ExecutionException; +import org.apache.avro.Schema; +import org.apache.avro.generic.GenericDatumReader; +import org.apache.avro.generic.GenericRecord; +import org.apache.avro.io.BinaryDecoder; +import org.apache.avro.io.DatumReader; +import org.apache.avro.io.DecoderFactory; +import org.apache.flume.Context; +import org.apache.flume.Event; +import org.apache.flume.EventDeliveryException; +import org.apache.flume.sink.kite.NonRecoverableEventException; +import org.apache.hadoop.conf.Configuration; +import org.apache.hadoop.fs.FileSystem; +import org.apache.hadoop.fs.Path; + +import static org.apache.flume.sink.kite.DatasetSinkConstants.*; + +/** + * An {@link EntityParser} that parses Avro serialized bytes from an event. + * + * The Avro schema used to serialize the data should be set as either a URL + * or literal in the flume.avro.schema.url or flume.avro.schema.literal event + * headers respectively. + */ +public class AvroParser implements EntityParser { + + static Configuration conf = new Configuration(); + + /** + * A cache of literal schemas to avoid re-parsing the schema. + */ + private static final LoadingCache schemasFromLiteral = + CacheBuilder.newBuilder() + .build(new CacheLoader() { + @Override + public Schema load(String literal) { + Preconditions.checkNotNull(literal, + "Schema literal cannot be null without a Schema URL"); + return new Schema.Parser().parse(literal); + } + }); + + /** + * A cache of schemas retrieved by URL to avoid re-parsing the schema. + */ + private static final LoadingCache schemasFromURL = + CacheBuilder.newBuilder() + .build(new CacheLoader() { + @Override + public Schema load(String url) throws IOException { + Schema.Parser parser = new Schema.Parser(); + InputStream is = null; + try { + FileSystem fs = FileSystem.get(URI.create(url), conf); + if (url.toLowerCase(Locale.ENGLISH).startsWith("hdfs:/")) { + is = fs.open(new Path(url)); + } else { + is = new URL(url).openStream(); + } + return parser.parse(is); + } finally { + if (is != null) { + is.close(); + } + } + } + }); + + /** + * The schema of the destination dataset. + * + * Used as the reader schema during parsing. + */ + private final Schema datasetSchema; + + /** + * A cache of DatumReaders per schema. + */ + private final LoadingCache> readers = + CacheBuilder.newBuilder() + .build(new CacheLoader>() { + @Override + public DatumReader load(Schema schema) { + // must use the target dataset's schema for reading to ensure the + // records are able to be stored using it + return new GenericDatumReader( + schema, datasetSchema); + } + }); + + /** + * The binary decoder to reuse for event parsing. + */ + private BinaryDecoder decoder = null; + + /** + * Create a new AvroParser given the schema of the destination dataset. + * + * @param datasetSchema The schema of the destination dataset. + */ + private AvroParser(Schema datasetSchema) { + this.datasetSchema = datasetSchema; + } + + /** + * Parse the entity from the body of the given event. + * + * @param event The event to parse. + * @param reuse If non-null, this may be reused and returned from this method. + * @return The parsed entity as a GenericRecord. + * @throws EventDeliveryException A recoverable error such as an error + * downloading the schema from the URL has + * occurred. + * @throws NonRecoverableEventException A non-recoverable error such as an + * unparsable schema or entity has + * occurred. + */ + @Override + public GenericRecord parse(Event event, GenericRecord reuse) + throws EventDeliveryException, NonRecoverableEventException { + decoder = DecoderFactory.get().binaryDecoder(event.getBody(), decoder); + + try { + DatumReader reader = readers.getUnchecked(schema(event)); + return reader.read(reuse, decoder); + } catch (IOException ex) { + throw new NonRecoverableEventException("Cannot deserialize event", ex); + } catch (RuntimeException ex) { + throw new NonRecoverableEventException("Cannot deserialize event", ex); + } + } + + /** + * Get the schema from the event headers. + * + * @param event The Flume event + * @return The schema for the event + * @throws EventDeliveryException A recoverable error such as an error + * downloading the schema from the URL has + * occurred. + * @throws NonRecoverableEventException A non-recoverable error such as an + * unparsable schema has occurred. + */ + private static Schema schema(Event event) throws EventDeliveryException, + NonRecoverableEventException { + Map headers = event.getHeaders(); + String schemaURL = headers.get(AVRO_SCHEMA_URL_HEADER); + try { + if (schemaURL != null) { + return schemasFromURL.get(schemaURL); + } else { + String schemaLiteral = headers.get(AVRO_SCHEMA_LITERAL_HEADER); + if (schemaLiteral == null) { + throw new NonRecoverableEventException("No schema in event headers." + + " Headers must include either " + AVRO_SCHEMA_URL_HEADER + + " or " + AVRO_SCHEMA_LITERAL_HEADER); + } + + return schemasFromLiteral.get(schemaLiteral); + } + } catch (ExecutionException ex) { + throw new EventDeliveryException("Cannot get schema", ex.getCause()); + } catch (UncheckedExecutionException ex) { + throw new NonRecoverableEventException("Cannot parse schema", + ex.getCause()); + } + } + + public static class Builder implements EntityParser.Builder { + + @Override + public EntityParser build(Schema datasetSchema, Context config) { + return new AvroParser(datasetSchema); + } + + } +} diff --git a/flume-ng-sinks/flume-dataset-sink/src/main/java/org/apache/flume/sink/kite/parser/EntityParser.java b/flume-ng-sinks/flume-dataset-sink/src/main/java/org/apache/flume/sink/kite/parser/EntityParser.java new file mode 100644 index 0000000000..f2051a2d9c --- /dev/null +++ b/flume-ng-sinks/flume-dataset-sink/src/main/java/org/apache/flume/sink/kite/parser/EntityParser.java @@ -0,0 +1,56 @@ +/** + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.flume.sink.kite.parser; + +import javax.annotation.concurrent.NotThreadSafe; +import org.apache.avro.Schema; +import org.apache.flume.Context; +import org.apache.flume.Event; +import org.apache.flume.EventDeliveryException; +import org.apache.flume.sink.kite.NonRecoverableEventException; + +@NotThreadSafe +public interface EntityParser { + + /** + * Parse a Kite entity from a Flume event + * + * @param event The event to parse + * @param reuse If non-null, this may be reused and returned + * @return The parsed entity + * @throws EventDeliveryException A recoverable error during parsing. Parsing + * can be safely retried. + * @throws NonRecoverableEventException A non-recoverable error during + * parsing. The event must be discarded. + * + */ + public E parse(Event event, E reuse) throws EventDeliveryException, + NonRecoverableEventException; + + /** + * Knows how to build {@code EntityParser}s. Implementers must provide a + * no-arg constructor. + * + * @param The type of entities generated + */ + public static interface Builder { + + public EntityParser build(Schema datasetSchema, Context config); + } +} diff --git a/flume-ng-sinks/flume-dataset-sink/src/main/java/org/apache/flume/sink/kite/parser/EntityParserFactory.java b/flume-ng-sinks/flume-dataset-sink/src/main/java/org/apache/flume/sink/kite/parser/EntityParserFactory.java new file mode 100644 index 0000000000..cfb734943e --- /dev/null +++ b/flume-ng-sinks/flume-dataset-sink/src/main/java/org/apache/flume/sink/kite/parser/EntityParserFactory.java @@ -0,0 +1,82 @@ +/** + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.flume.sink.kite.parser; + +import java.util.Arrays; +import org.apache.avro.Schema; +import org.apache.avro.generic.GenericRecord; +import org.apache.flume.Context; + +import static org.apache.flume.sink.kite.DatasetSinkConstants.*; + + +public class EntityParserFactory { + + public EntityParser newParser(Schema datasetSchema, Context config) { + EntityParser parser; + + String parserType = config.getString(CONFIG_ENTITY_PARSER, + DEFAULT_ENTITY_PARSER); + + if (parserType.equals(AVRO_ENTITY_PARSER)) { + parser = new AvroParser.Builder().build(datasetSchema, config); + } else { + + Class builderClass; + Class c; + try { + c = Class.forName(parserType); + } catch (ClassNotFoundException ex) { + throw new IllegalArgumentException("EntityParser.Builder class " + + parserType + " not found. Must set " + CONFIG_ENTITY_PARSER + + " to a class that implements EntityParser.Builder or to a builtin" + + " parser: " + Arrays.toString(AVAILABLE_PARSERS), ex); + } + + if (c != null && EntityParser.Builder.class.isAssignableFrom(c)) { + builderClass = c; + } else { + throw new IllegalArgumentException("Class " + parserType + " does not" + + " implement EntityParser.Builder. Must set " + + CONFIG_ENTITY_PARSER + " to a class that extends" + + " EntityParser.Builder or to a builtin parser: " + + Arrays.toString(AVAILABLE_PARSERS)); + } + + EntityParser.Builder builder; + try { + builder = builderClass.newInstance(); + } catch (InstantiationException ex) { + throw new IllegalArgumentException("Can't instantiate class " + + parserType + ". Must set " + CONFIG_ENTITY_PARSER + " to a class" + + " that extends EntityParser.Builder or to a builtin parser: " + + Arrays.toString(AVAILABLE_PARSERS), ex); + } catch (IllegalAccessException ex) { + throw new IllegalArgumentException("Can't instantiate class " + + parserType + ". Must set " + CONFIG_ENTITY_PARSER + " to a class" + + " that extends EntityParser.Builder or to a builtin parser: " + + Arrays.toString(AVAILABLE_PARSERS), ex); + } + + parser = builder.build(datasetSchema, config); + } + + return parser; + } +} diff --git a/flume-ng-sinks/flume-dataset-sink/src/main/java/org/apache/flume/sink/kite/policy/FailurePolicy.java b/flume-ng-sinks/flume-dataset-sink/src/main/java/org/apache/flume/sink/kite/policy/FailurePolicy.java new file mode 100644 index 0000000000..47b6a256cd --- /dev/null +++ b/flume-ng-sinks/flume-dataset-sink/src/main/java/org/apache/flume/sink/kite/policy/FailurePolicy.java @@ -0,0 +1,105 @@ +/** + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.flume.sink.kite.policy; + +import org.apache.flume.Context; +import org.apache.flume.Event; +import org.apache.flume.EventDeliveryException; +import org.apache.flume.sink.kite.DatasetSink; +import org.kitesdk.data.DatasetWriter; + +/** + * A policy for dealing with non-recoverable event delivery failures. + * + * Non-recoverable event delivery failures include: + * + * 1. Error parsing the event body thrown from the {@link EntityParser} + * 2. A schema mismatch between the schema of an event and the schema of the + * destination dataset. + * 3. A missing schema from the Event header when using the + * {@link AvroEntityParser}. + * + * The life cycle of a FailurePolicy mimics the life cycle of the + * {@link DatasetSink#writer}: + * + * 1. When a new writer is created, the policy will be instantiated. + * 2. As Event failures happen, + * {@link #handle(org.apache.flume.Event, java.lang.Throwable)} will be + * called to let the policy handle the failure. + * 3. If the {@link DatasetSink} is configured to commit on batch, then the + * {@link #sync()} method will be called when the batch is committed. + * 4. When the writer is closed, the policy's {@link #close()} method will be + * called. + */ +public interface FailurePolicy { + + /** + * Handle a non-recoverable event. + * + * @param event The event + * @param cause The cause of the failure + * @throws EventDeliveryException The policy failed to handle the event. When + * this is thrown, the Flume transaction will + * be rolled back and the event will be retried + * along with the rest of the batch. + */ + public void handle(Event event, Throwable cause) + throws EventDeliveryException; + + /** + * Ensure any handled events are on stable storage. + * + * This allows the policy implementation to sync any data that it may not + * have fully handled. + * + * See {@link DatasetWriter#sync()}. + * + * @throws EventDeliveryException The policy failed while syncing data. + * When this is thrown, the Flume transaction + * will be rolled back and the batch will be + * retried. + */ + public void sync() throws EventDeliveryException; + + /** + * Close this FailurePolicy and release any resources. + * + * @throws EventDeliveryException The policy failed while closing resources. + * When this is thrown, the Flume transaction + * will be rolled back and the batch will be + * retried. + */ + public void close() throws EventDeliveryException; + + /** + * Knows how to build {@code FailurePolicy}s. Implementers must provide a + * no-arg constructor. + */ + public static interface Builder { + + /** + * Build a new {@code FailurePolicy} + * + * @param config The Flume configuration context + * @return The {@code FailurePolicy} + */ + FailurePolicy build(Context config); + } + +} diff --git a/flume-ng-sinks/flume-dataset-sink/src/main/java/org/apache/flume/sink/kite/policy/FailurePolicyFactory.java b/flume-ng-sinks/flume-dataset-sink/src/main/java/org/apache/flume/sink/kite/policy/FailurePolicyFactory.java new file mode 100644 index 0000000000..a8b2008126 --- /dev/null +++ b/flume-ng-sinks/flume-dataset-sink/src/main/java/org/apache/flume/sink/kite/policy/FailurePolicyFactory.java @@ -0,0 +1,82 @@ +/** + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.flume.sink.kite.policy; + +import java.util.Arrays; +import org.apache.flume.Context; + +import static org.apache.flume.sink.kite.DatasetSinkConstants.*; + + +public class FailurePolicyFactory { + + public FailurePolicy newPolicy(Context config) { + FailurePolicy policy; + + String policyType = config.getString(CONFIG_FAILURE_POLICY, + DEFAULT_FAILURE_POLICY); + + if (policyType.equals(RETRY_FAILURE_POLICY)) { + policy = new RetryPolicy.Builder().build(config); + } else if (policyType.equals(SAVE_FAILURE_POLICY)) { + policy = new SavePolicy.Builder().build(config); + } else { + + Class builderClass; + Class c; + try { + c = Class.forName(policyType); + } catch (ClassNotFoundException ex) { + throw new IllegalArgumentException("FailurePolicy.Builder class " + + policyType + " not found. Must set " + CONFIG_FAILURE_POLICY + + " to a class that implements FailurePolicy.Builder or to a builtin" + + " policy: " + Arrays.toString(AVAILABLE_POLICIES), ex); + } + + if (c != null && FailurePolicy.Builder.class.isAssignableFrom(c)) { + builderClass = c; + } else { + throw new IllegalArgumentException("Class " + policyType + " does not" + + " implement FailurePolicy.Builder. Must set " + + CONFIG_FAILURE_POLICY + " to a class that extends" + + " FailurePolicy.Builder or to a builtin policy: " + + Arrays.toString(AVAILABLE_POLICIES)); + } + + FailurePolicy.Builder builder; + try { + builder = builderClass.newInstance(); + } catch (InstantiationException ex) { + throw new IllegalArgumentException("Can't instantiate class " + + policyType + ". Must set " + CONFIG_FAILURE_POLICY + " to a class" + + " that extends FailurePolicy.Builder or to a builtin policy: " + + Arrays.toString(AVAILABLE_POLICIES), ex); + } catch (IllegalAccessException ex) { + throw new IllegalArgumentException("Can't instantiate class " + + policyType + ". Must set " + CONFIG_FAILURE_POLICY + " to a class" + + " that extends FailurePolicy.Builder or to a builtin policy: " + + Arrays.toString(AVAILABLE_POLICIES), ex); + } + + policy = builder.build(config); + } + + return policy; + } +} diff --git a/flume-ng-sinks/flume-dataset-sink/src/main/java/org/apache/flume/sink/kite/policy/RetryPolicy.java b/flume-ng-sinks/flume-dataset-sink/src/main/java/org/apache/flume/sink/kite/policy/RetryPolicy.java new file mode 100644 index 0000000000..9a4991ca41 --- /dev/null +++ b/flume-ng-sinks/flume-dataset-sink/src/main/java/org/apache/flume/sink/kite/policy/RetryPolicy.java @@ -0,0 +1,63 @@ +/** + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.flume.sink.kite.policy; + +import org.apache.flume.Context; +import org.apache.flume.Event; +import org.apache.flume.EventDeliveryException; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +/** + * A failure policy that logs the error and then forces a retry by throwing + * {@link EventDeliveryException}. + */ +public class RetryPolicy implements FailurePolicy { + private static final Logger LOG = LoggerFactory.getLogger(RetryPolicy.class); + + private RetryPolicy() { + } + + @Override + public void handle(Event event, Throwable cause) throws EventDeliveryException { + LOG.error("Event delivery failed: " + cause.getLocalizedMessage()); + LOG.debug("Exception follows.", cause); + + throw new EventDeliveryException(cause); + } + + @Override + public void sync() throws EventDeliveryException { + // do nothing + } + + @Override + public void close() throws EventDeliveryException { + // do nothing + } + + public static class Builder implements FailurePolicy.Builder { + + @Override + public FailurePolicy build(Context config) { + return new RetryPolicy(); + } + + } +} diff --git a/flume-ng-sinks/flume-dataset-sink/src/main/java/org/apache/flume/sink/kite/policy/SavePolicy.java b/flume-ng-sinks/flume-dataset-sink/src/main/java/org/apache/flume/sink/kite/policy/SavePolicy.java new file mode 100644 index 0000000000..ed47898548 --- /dev/null +++ b/flume-ng-sinks/flume-dataset-sink/src/main/java/org/apache/flume/sink/kite/policy/SavePolicy.java @@ -0,0 +1,125 @@ +/** + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.flume.sink.kite.policy; + +import com.google.common.base.Preconditions; +import com.google.common.collect.Maps; +import java.nio.ByteBuffer; +import java.util.Map; +import org.apache.flume.Context; +import org.apache.flume.Event; +import org.apache.flume.EventDeliveryException; +import org.apache.flume.source.avro.AvroFlumeEvent; +import org.kitesdk.data.DatasetDescriptor; +import org.kitesdk.data.DatasetWriter; +import org.kitesdk.data.Datasets; +import org.kitesdk.data.Formats; +import org.kitesdk.data.View; + +import static org.apache.flume.sink.kite.DatasetSinkConstants.*; + +/** + * A failure policy that writes the raw Flume event to a Kite dataset. + */ +public class SavePolicy implements FailurePolicy { + + private final View dataset; + private DatasetWriter writer; + private int nEventsHandled; + + private SavePolicy(Context context) { + String uri = context.getString(CONFIG_KITE_ERROR_DATASET_URI); + Preconditions.checkArgument(uri != null, "Must set " + + CONFIG_KITE_ERROR_DATASET_URI + " when " + CONFIG_FAILURE_POLICY + + "=save"); + if (Datasets.exists(uri)) { + dataset = Datasets.load(uri, AvroFlumeEvent.class); + } else { + DatasetDescriptor descriptor = new DatasetDescriptor.Builder() + .schema(AvroFlumeEvent.class) + .build(); + dataset = Datasets.create(uri, descriptor, AvroFlumeEvent.class); + } + + nEventsHandled = 0; + } + + @Override + public void handle(Event event, Throwable cause) throws EventDeliveryException { + try { + if (writer == null) { + writer = dataset.newWriter(); + } + + final AvroFlumeEvent avroEvent = new AvroFlumeEvent(); + avroEvent.setBody(ByteBuffer.wrap(event.getBody())); + avroEvent.setHeaders(toCharSeqMap(event.getHeaders())); + + writer.write(avroEvent); + nEventsHandled++; + } catch (RuntimeException ex) { + throw new EventDeliveryException(ex); + } + } + + @Override + public void sync() throws EventDeliveryException { + if (nEventsHandled > 0) { + if (Formats.PARQUET.equals( + dataset.getDataset().getDescriptor().getFormat())) { + // We need to close the writer on sync if we're writing to a Parquet + // dataset + close(); + } else { + writer.sync(); + } + } + } + + @Override + public void close() throws EventDeliveryException { + if (nEventsHandled > 0) { + try { + writer.close(); + } catch (RuntimeException ex) { + throw new EventDeliveryException(ex); + } finally { + writer = null; + nEventsHandled = 0; + } + } + } + + /** + * Helper function to convert a map of String to a map of CharSequence. + */ + private static Map toCharSeqMap( + Map map) { + return Maps.newHashMap(map); + } + + public static class Builder implements FailurePolicy.Builder { + + @Override + public FailurePolicy build(Context config) { + return new SavePolicy(config); + } + + } +} diff --git a/flume-ng-sinks/flume-dataset-sink/src/test/java/org/apache/flume/sink/kite/TestDatasetSink.java b/flume-ng-sinks/flume-dataset-sink/src/test/java/org/apache/flume/sink/kite/TestDatasetSink.java index c46d66cdf0..58aa467616 100644 --- a/flume-ng-sinks/flume-dataset-sink/src/test/java/org/apache/flume/sink/kite/TestDatasetSink.java +++ b/flume-ng-sinks/flume-dataset-sink/src/test/java/org/apache/flume/sink/kite/TestDatasetSink.java @@ -18,6 +18,8 @@ package org.apache.flume.sink.kite; +import org.apache.flume.sink.kite.parser.EntityParser; +import org.apache.flume.sink.kite.policy.FailurePolicy; import com.google.common.base.Function; import com.google.common.base.Throwables; import com.google.common.collect.Iterables; @@ -29,6 +31,7 @@ import java.io.FileWriter; import java.io.IOException; import java.net.URI; +import java.nio.ByteBuffer; import java.util.Arrays; import java.util.HashSet; import java.util.List; @@ -37,12 +40,14 @@ import java.util.concurrent.Callable; import javax.annotation.Nullable; import org.apache.avro.Schema; +import org.apache.avro.file.DataFileWriter; import org.apache.avro.generic.GenericData; import org.apache.avro.generic.GenericRecord; import org.apache.avro.generic.GenericRecordBuilder; import org.apache.avro.io.Encoder; import org.apache.avro.io.EncoderFactory; import org.apache.avro.reflect.ReflectDatumWriter; +import org.apache.avro.util.Utf8; import org.apache.commons.io.FileUtils; import org.apache.flume.Channel; import org.apache.flume.Context; @@ -52,6 +57,7 @@ import org.apache.flume.channel.MemoryChannel; import org.apache.flume.conf.Configurables; import org.apache.flume.event.SimpleEvent; +import org.apache.flume.source.avro.AvroFlumeEvent; import org.apache.hadoop.conf.Configuration; import org.apache.hadoop.fs.FileSystem; import org.apache.hadoop.hdfs.MiniDFSCluster; @@ -64,9 +70,11 @@ import org.kitesdk.data.Dataset; import org.kitesdk.data.DatasetDescriptor; import org.kitesdk.data.DatasetReader; +import org.kitesdk.data.DatasetWriter; import org.kitesdk.data.Datasets; import org.kitesdk.data.PartitionStrategy; import org.kitesdk.data.View; +import static org.mockito.Mockito.*; public class TestDatasetSink { @@ -74,6 +82,8 @@ public class TestDatasetSink { public static final String DATASET_NAME = "test"; public static final String FILE_DATASET_URI = "dataset:file:target/test-repo/" + DATASET_NAME; + public static final String ERROR_DATASET_URI = + "dataset:file:target/test-repo/failed-events"; public static final File SCHEMA_FILE = new File("target/record-schema.avsc"); public static final Schema RECORD_SCHEMA = new Schema.Parser().parse( "{\"type\":\"record\",\"name\":\"rec\",\"fields\":[" + @@ -127,6 +137,7 @@ public void setup() throws EventDeliveryException { Datasets.create(FILE_DATASET_URI, DESCRIPTOR); this.config = new Context(); + config.put("keep-alive", "0"); this.in = new MemoryChannel(); Configurables.configure(in, config); @@ -195,7 +206,7 @@ public void testDatasetUriOverridesOldConfig() throws EventDeliveryException { } @Test - public void testFileStore() throws EventDeliveryException { + public void testFileStore() throws EventDeliveryException, NonRecoverableEventException, NonRecoverableEventException { DatasetSink sink = sink(in, config); // run the sink @@ -222,6 +233,19 @@ public void testParquetDataset() throws EventDeliveryException { // run the sink sink.start(); sink.process(); + + // the transaction should not commit during the call to process + assertThrows("Transaction should still be open", IllegalStateException.class, + new Callable() { + @Override + public Object call() throws EventDeliveryException { + in.getTransaction().begin(); + return null; + } + }); + // The records won't commit until the call to stop() + Assert.assertEquals("Should not have committed", 0, read(created).size()); + sink.stop(); Assert.assertEquals(Sets.newHashSet(expected), read(created)); @@ -509,6 +533,376 @@ public Object call() throws EventDeliveryException { expected.size() + 1, remaining(in)); } + @Test + public void testFileStoreWithSavePolicy() throws EventDeliveryException { + if (Datasets.exists(ERROR_DATASET_URI)) { + Datasets.delete(ERROR_DATASET_URI); + } + config.put(DatasetSinkConstants.CONFIG_FAILURE_POLICY, + DatasetSinkConstants.SAVE_FAILURE_POLICY); + config.put(DatasetSinkConstants.CONFIG_KITE_ERROR_DATASET_URI, + ERROR_DATASET_URI); + DatasetSink sink = sink(in, config); + + // run the sink + sink.start(); + sink.process(); + sink.stop(); + + Assert.assertEquals( + Sets.newHashSet(expected), + read(Datasets.load(FILE_DATASET_URI))); + Assert.assertEquals("Should have committed", 0, remaining(in)); + } + + @Test + public void testMissingSchemaWithSavePolicy() throws EventDeliveryException { + if (Datasets.exists(ERROR_DATASET_URI)) { + Datasets.delete(ERROR_DATASET_URI); + } + config.put(DatasetSinkConstants.CONFIG_FAILURE_POLICY, + DatasetSinkConstants.SAVE_FAILURE_POLICY); + config.put(DatasetSinkConstants.CONFIG_KITE_ERROR_DATASET_URI, + ERROR_DATASET_URI); + final DatasetSink sink = sink(in, config); + + Event badEvent = new SimpleEvent(); + badEvent.setHeaders(Maps.newHashMap()); + badEvent.setBody(serialize(expected.get(0), RECORD_SCHEMA)); + putToChannel(in, badEvent); + + // run the sink + sink.start(); + sink.process(); + sink.stop(); + + Assert.assertEquals("Good records should have been written", + Sets.newHashSet(expected), + read(Datasets.load(FILE_DATASET_URI))); + Assert.assertEquals("Should not have rolled back", 0, remaining(in)); + Assert.assertEquals("Should have saved the bad event", + Sets.newHashSet(AvroFlumeEvent.newBuilder() + .setBody(ByteBuffer.wrap(badEvent.getBody())) + .setHeaders(toUtf8Map(badEvent.getHeaders())) + .build()), + read(Datasets.load(ERROR_DATASET_URI, AvroFlumeEvent.class))); + } + + @Test + public void testSerializedWithIncompatibleSchemasWithSavePolicy() + throws EventDeliveryException { + if (Datasets.exists(ERROR_DATASET_URI)) { + Datasets.delete(ERROR_DATASET_URI); + } + config.put(DatasetSinkConstants.CONFIG_FAILURE_POLICY, + DatasetSinkConstants.SAVE_FAILURE_POLICY); + config.put(DatasetSinkConstants.CONFIG_KITE_ERROR_DATASET_URI, + ERROR_DATASET_URI); + final DatasetSink sink = sink(in, config); + + GenericRecordBuilder builder = new GenericRecordBuilder( + INCOMPATIBLE_SCHEMA); + GenericData.Record rec = builder.set("username", "koala").build(); + + // We pass in a valid schema in the header, but an incompatible schema + // was used to serialize the record + Event badEvent = event(rec, INCOMPATIBLE_SCHEMA, SCHEMA_FILE, true); + putToChannel(in, badEvent); + + // run the sink + sink.start(); + sink.process(); + sink.stop(); + + Assert.assertEquals("Good records should have been written", + Sets.newHashSet(expected), + read(Datasets.load(FILE_DATASET_URI))); + Assert.assertEquals("Should not have rolled back", 0, remaining(in)); + Assert.assertEquals("Should have saved the bad event", + Sets.newHashSet(AvroFlumeEvent.newBuilder() + .setBody(ByteBuffer.wrap(badEvent.getBody())) + .setHeaders(toUtf8Map(badEvent.getHeaders())) + .build()), + read(Datasets.load(ERROR_DATASET_URI, AvroFlumeEvent.class))); + } + + @Test + public void testSerializedWithIncompatibleSchemas() throws EventDeliveryException { + final DatasetSink sink = sink(in, config); + + GenericRecordBuilder builder = new GenericRecordBuilder( + INCOMPATIBLE_SCHEMA); + GenericData.Record rec = builder.set("username", "koala").build(); + + // We pass in a valid schema in the header, but an incompatible schema + // was used to serialize the record + putToChannel(in, event(rec, INCOMPATIBLE_SCHEMA, SCHEMA_FILE, true)); + + // run the sink + sink.start(); + assertThrows("Should fail", EventDeliveryException.class, + new Callable() { + @Override + public Object call() throws EventDeliveryException { + sink.process(); + return null; + } + }); + sink.stop(); + + Assert.assertEquals("Should have rolled back", + expected.size() + 1, remaining(in)); + } + + @Test + public void testCommitOnBatch() throws EventDeliveryException { + DatasetSink sink = sink(in, config); + + // run the sink + sink.start(); + sink.process(); + + // the transaction should commit during the call to process + Assert.assertEquals("Should have committed", 0, remaining(in)); + // but the data won't be visible yet + Assert.assertEquals(0, + read(Datasets.load(FILE_DATASET_URI)).size()); + + sink.stop(); + + Assert.assertEquals( + Sets.newHashSet(expected), + read(Datasets.load(FILE_DATASET_URI))); + } + + @Test + public void testCommitOnBatchFalse() throws EventDeliveryException { + config.put(DatasetSinkConstants.CONFIG_FLUSHABLE_COMMIT_ON_BATCH, + Boolean.toString(false)); + config.put(DatasetSinkConstants.CONFIG_SYNCABLE_SYNC_ON_BATCH, + Boolean.toString(false)); + DatasetSink sink = sink(in, config); + + // run the sink + sink.start(); + sink.process(); + + // the transaction should not commit during the call to process + assertThrows("Transaction should still be open", IllegalStateException.class, + new Callable() { + @Override + public Object call() throws EventDeliveryException { + in.getTransaction().begin(); + return null; + } + }); + + // the data won't be visible + Assert.assertEquals(0, + read(Datasets.load(FILE_DATASET_URI)).size()); + + sink.stop(); + + Assert.assertEquals( + Sets.newHashSet(expected), + read(Datasets.load(FILE_DATASET_URI))); + // the transaction should commit during the call to stop + Assert.assertEquals("Should have committed", 0, remaining(in)); + } + + @Test + public void testCommitOnBatchFalseSyncOnBatchTrue() throws EventDeliveryException { + config.put(DatasetSinkConstants.CONFIG_FLUSHABLE_COMMIT_ON_BATCH, + Boolean.toString(false)); + config.put(DatasetSinkConstants.CONFIG_SYNCABLE_SYNC_ON_BATCH, + Boolean.toString(true)); + + try { + sink(in, config); + Assert.fail("Should have thrown IllegalArgumentException"); + } catch (IllegalArgumentException ex) { + // expected + } + } + + @Test + public void testCloseAndCreateWriter() throws EventDeliveryException { + config.put(DatasetSinkConstants.CONFIG_FLUSHABLE_COMMIT_ON_BATCH, + Boolean.toString(false)); + config.put(DatasetSinkConstants.CONFIG_SYNCABLE_SYNC_ON_BATCH, + Boolean.toString(false)); + DatasetSink sink = sink(in, config); + + // run the sink + sink.start(); + sink.process(); + + sink.closeWriter(); + sink.commitTransaction(); + sink.createWriter(); + + Assert.assertNotNull("Writer should not be null", sink.getWriter()); + Assert.assertEquals("Should have committed", 0, remaining(in)); + + sink.stop(); + + Assert.assertEquals( + Sets.newHashSet(expected), + read(Datasets.load(FILE_DATASET_URI))); + } + + @Test + public void testCloseWriter() throws EventDeliveryException { + config.put(DatasetSinkConstants.CONFIG_FLUSHABLE_COMMIT_ON_BATCH, + Boolean.toString(false)); + config.put(DatasetSinkConstants.CONFIG_SYNCABLE_SYNC_ON_BATCH, + Boolean.toString(false)); + DatasetSink sink = sink(in, config); + + // run the sink + sink.start(); + sink.process(); + + sink.closeWriter(); + sink.commitTransaction(); + + Assert.assertNull("Writer should be null", sink.getWriter()); + Assert.assertEquals("Should have committed", 0, remaining(in)); + + sink.stop(); + + Assert.assertEquals( + Sets.newHashSet(expected), + read(Datasets.load(FILE_DATASET_URI))); + } + + @Test + public void testCreateWriter() throws EventDeliveryException { + config.put(DatasetSinkConstants.CONFIG_FLUSHABLE_COMMIT_ON_BATCH, + Boolean.toString(false)); + config.put(DatasetSinkConstants.CONFIG_SYNCABLE_SYNC_ON_BATCH, + Boolean.toString(false)); + DatasetSink sink = sink(in, config); + + // run the sink + sink.start(); + sink.process(); + + sink.commitTransaction(); + sink.createWriter(); + Assert.assertNotNull("Writer should not be null", sink.getWriter()); + Assert.assertEquals("Should have committed", 0, remaining(in)); + + sink.stop(); + + Assert.assertEquals(0, read(Datasets.load(FILE_DATASET_URI)).size()); + } + + @Test + public void testAppendWriteExceptionInvokesPolicy() + throws EventDeliveryException, NonRecoverableEventException { + DatasetSink sink = sink(in, config); + + // run the sink + sink.start(); + sink.process(); + + // Mock an Event + Event mockEvent = mock(Event.class); + when(mockEvent.getBody()).thenReturn(new byte[] { 0x01 }); + + // Mock a GenericRecord + GenericRecord mockRecord = mock(GenericRecord.class); + + // Mock an EntityParser + EntityParser mockParser = mock(EntityParser.class); + when(mockParser.parse(eq(mockEvent), any(GenericRecord.class))) + .thenReturn(mockRecord); + sink.setParser(mockParser); + + // Mock a FailurePolicy + FailurePolicy mockFailurePolicy = mock(FailurePolicy.class); + sink.setFailurePolicy(mockFailurePolicy); + + // Mock a DatasetWriter + DatasetWriter mockWriter = mock(DatasetWriter.class); + doThrow(new DataFileWriter.AppendWriteException(new IOException())) + .when(mockWriter).write(mockRecord); + + sink.setWriter(mockWriter); + sink.write(mockEvent); + + // Verify that the event was sent to the failure policy + verify(mockFailurePolicy).handle(eq(mockEvent), any(Throwable.class)); + + sink.stop(); + } + + @Test + public void testRuntimeExceptionThrowsEventDeliveryException() + throws EventDeliveryException, NonRecoverableEventException { + DatasetSink sink = sink(in, config); + + // run the sink + sink.start(); + sink.process(); + + // Mock an Event + Event mockEvent = mock(Event.class); + when(mockEvent.getBody()).thenReturn(new byte[] { 0x01 }); + + // Mock a GenericRecord + GenericRecord mockRecord = mock(GenericRecord.class); + + // Mock an EntityParser + EntityParser mockParser = mock(EntityParser.class); + when(mockParser.parse(eq(mockEvent), any(GenericRecord.class))) + .thenReturn(mockRecord); + sink.setParser(mockParser); + + // Mock a FailurePolicy + FailurePolicy mockFailurePolicy = mock(FailurePolicy.class); + sink.setFailurePolicy(mockFailurePolicy); + + // Mock a DatasetWriter + DatasetWriter mockWriter = mock(DatasetWriter.class); + doThrow(new RuntimeException()).when(mockWriter).write(mockRecord); + + sink.setWriter(mockWriter); + + try { + sink.write(mockEvent); + Assert.fail("Should throw EventDeliveryException"); + } catch (EventDeliveryException ex) { + + } + + // Verify that the event was not sent to the failure policy + verify(mockFailurePolicy, never()).handle(eq(mockEvent), any(Throwable.class)); + + sink.stop(); + } + + @Test + public void testProcessHandlesNullWriter() throws EventDeliveryException, + NonRecoverableEventException, NonRecoverableEventException { + DatasetSink sink = sink(in, config); + + // run the sink + sink.start(); + sink.process(); + + // explicitly set the writer to null + sink.setWriter(null); + + // this should not throw an NPE + sink.process(); + + sink.stop(); + + Assert.assertEquals("Should have committed", 0, remaining(in)); + } + public static DatasetSink sink(Channel in, Context config) { DatasetSink sink = new DatasetSink(); sink.setChannel(in); @@ -621,4 +1015,19 @@ public static void assertThrows( Assert.assertEquals(message, expected, actual.getClass()); } } + + /** + * Helper function to convert a map of String to a map of Utf8. + * + * @param map A Map of String to String + * @return The same mappings converting the {@code String}s to {@link Utf8}s + */ + public static Map toUtf8Map( + Map map) { + Map utf8Map = Maps.newHashMap(); + for (Map.Entry entry : map.entrySet()) { + utf8Map.put(new Utf8(entry.getKey()), new Utf8(entry.getValue())); + } + return utf8Map; + } } diff --git a/pom.xml b/pom.xml index 7d4f01a5a6..1350fa43ee 100644 --- a/pom.xml +++ b/pom.xml @@ -51,7 +51,7 @@ limitations under the License. 2.4.0 0.7.0 - 0.15.0 + 0.17.1 0.10.0 From 62a4cad223aebfa0aa0f9d20c1c37f04a8b918a9 Mon Sep 17 00:00:00 2001 From: Hari Shreedharan Date: Fri, 30 Jan 2015 16:55:47 -0800 Subject: [PATCH 198/341] FLUME-1334. Windows startup scripts. (Roshan Naik via Hari) --- bin/flume-ng.cmd | 19 +++ bin/flume-ng.ps1 | 403 +++++++++++++++++++++++++++++++++++++++++++++++ 2 files changed, 422 insertions(+) create mode 100644 bin/flume-ng.cmd create mode 100755 bin/flume-ng.ps1 diff --git a/bin/flume-ng.cmd b/bin/flume-ng.cmd new file mode 100644 index 0000000000..b1b97a989f --- /dev/null +++ b/bin/flume-ng.cmd @@ -0,0 +1,19 @@ +@REM Licensed to the Apache Software Foundation (ASF) under one +@REM or more contributor license agreements. See the NOTICE file +@REM distributed with this work for additional information +@REM regarding copyright ownership. The ASF licenses this file +@REM to you under the Apache License, Version 2.0 (the +@REM "License"); you may not use this file except in compliance +@REM with the License. You may obtain a copy of the License at +@REM +@REM http://www.apache.org/licenses/LICENSE-2.0 +@REM +@REM Unless required by applicable law or agreed to in writing, +@REM software distributed under the License is distributed on an +@REM "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY +@REM KIND, either express or implied. See the License for the +@REM specific language governing permissions and limitations +@REM under the License. + +powershell.exe -NoProfile -InputFormat none -ExecutionPolicy unrestricted -File %~dp0flume-ng.ps1 %* + diff --git a/bin/flume-ng.ps1 b/bin/flume-ng.ps1 new file mode 100755 index 0000000000..1638bf1a16 --- /dev/null +++ b/bin/flume-ng.ps1 @@ -0,0 +1,403 @@ +# +# Licensed to the Apache Software Foundation (ASF) under one +# or more contributor license agreements. See the NOTICE file +# distributed with this work for additional information +# regarding copyright ownership. The ASF licenses this file +# to you under the Apache License, Version 2.0 (the +# "License"); you may not use this file except in compliance +# with the License. You may obtain a copy of the License at +# +# http://www.apache.org/licenses/LICENSE-2.0 +# +# Unless required by applicable law or agreed to in writing, +# software distributed under the License is distributed on an +# "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY +# KIND, either express or implied. See the License for the +# specific language governing permissions and limitations +# under the License. +# + +param( + [Parameter(Position=0, Mandatory=$true)] + [ValidateSet('help','agent','avro-client','version')] $command, + [Parameter()] [string] $conf, + [Parameter()] [Alias('C')] [string] $classPath, + [Parameter()] [Alias('property')] [string] $javaProperties, + [Parameter()] [Alias('d')] [switch] $dryrun, + [Parameter()] [Alias('plugins-path')] [string] $pluginsPath, + + [Parameter()] [Alias('f',"conf-file")] [string] $confFile, + [Parameter()] [Alias('n')] [string] $name, + + [Parameter()] [string] $rpcProps, + [Parameter()] [Alias('H',"host")] [string] $avroHost, + [Parameter()] [Alias('p',"port")] [string] $avroPort, + [Parameter()] [Alias("dirname")] [string] $avroDirName, + [Parameter()] [Alias("filename")] [string] $avroFileName, + [Parameter()] [Alias('R',"headerFile")] [string] $avroHeaderFile +) + +# when invoked from cmd shell, array arguments are treated as array with one string value +# so we accept classpath and javaProperty as string argument and split explicitly into array + +if ($classPath -ne "") { + $classPathArr = $classPath.Split(";") +} else { + $classPathArr = @() +} + +if($javaProperties -ne "") { + $javaPropertiesArr = $javaProperties.Split(";") # properties may have embedded comma, so using ; as delim +} else { + $javaPropertiesArr = @() +} + +# FUNCTIONS +Function PrintUsage { + Write-Host " +Usage: flume-ng [options]... + +commands: + help display this help text + agent run a Flume agent + avro-client run an avro Flume client + version show Flume version info + +global options: + -conf use configs in directory + -classpath,-C ""value1;value2; .."" append to the classpath + -property ""name1=val;name2=val; .."" sets a JDK system property value + -dryrun,-d do not actually start Flume (test) + -plugins-path ""dir1;dir2"" semi-colon-separated list of plugins.d directories. See the + plugins.d section in the user guide for more details. + Default: \$FLUME_HOME/plugins.d + +agent options: + -conf-file,-f specify a config file (required) + -name,-n the name of this agent (required) + +avro-client options: + -rpcProps RPC client properties file with server connection params + -host,-H hostname to which events will be sent (required) + -port,-p port of the avro source (required) + -dirname

    directory to stream to avro source + -filename text file to stream to avro source [default: std input] + -headerFile,-R headerFile containing headers as key/value pairs on each new line + + Either --rpcProps or both --host and --port must be specified. + +Note that if directory is specified, then it is always included first +in the classpath." +} + +Function EnumerateJars($path) { + return Get-ChildItem $path -Filter *.jar | ? { "$_" -notmatch "slf4j-(api|log4j12).*jar" } | % { $_.FullName} +} + +Function GetHadoopHome { + if($env:HADOOP_PREFIX) { + $hadoopBin = "$env:HADOOP_PREFIX\bin;" + } elseif ($env:HADOOP_HOME) { + $hadoopBin = "$env:HADOOP_HOME\bin;" + } + + #Searches for hadoop.cmd in the HADOOP_HOME, current directory and path + [String[]] $hadoopPaths = ("$hadoopBin;.;$env:PATH").Split(";") | + ? { "$_" -ne "" -and (Test-Path $_ )} | + ? { Test-Path ( Join-Path $_ "hadoop.cmd" ) } + if($hadoopPaths -ne $Null ) { + $binDir = $hadoopPaths[0] + "\.." + return Resolve-Path $binDir + } + Write-Host "WARN: HADOOP_PREFIX or HADOOP_HOME not found" + return $Null +} + +Function GetHbaseHome() { + if($env:HBASE_HOME) { + return $env:HBASE_HOME + } + + #Searches for hbase.cmd in the HBASE_HOME, current directory and path + [String[]] $hbasePaths = (".;$env:PATH").Split(";") | + ? { "$_" -ne "" -and (Test-Path $_) } | + ? { Test-Path (Join-Path $_ "hbase.cmd") } + + if($hbasePaths -ne $Null) { + return $hbasePaths[0] + } + Write-Host "WARN: HBASE_HOME not found" + return $Null +} + +Function GetHiveHome() { + if($env:HIVE_HOME) { + return $env:HIVE_HOME + } + + #Searches for hive.cmd in the HIVE_HOME, current directory and path + [String[]] $hivePaths = (".;$env:PATH").Split(";") | + ? { "$_" -ne "" -and (Test-Path $_) } | + ? { Test-Path (Join-Path $_ "hive.cmd") } + + if($hivePaths -ne $Null) { + return $hivePaths[0] + } + Write-Host "WARN: HIVE_HOME not found" + return $Null +} + +Function GetJavaLibraryPath ($cmd, $flumeLibDir) { + $flumeCoreJar = EnumerateJars( $flumeLibDir ) | ? { $_ -match "flume-ng-core.*jar" } + $output = & "$cmd" -classpath $flumeCoreJar org.apache.flume.tools.GetJavaProperty java.library.path + + # look for the line that has the desired property value + if( $output -match "(?m)^java\.library\.path=(.*)$" ) { + return $Matches[1].split(";") | % { $_ -replace "\\$" , ""} # trim trailing \ char + } + return @(); +} + + +Function GetClassPath ($cmd) { + $output = & "$cmd" "classpath" + return $output; +} + +Function GetJavaPath { + if ($env:JAVA_HOME) { + return "$env:JAVA_HOME\bin\java.exe" }ss + Write-Host "WARN: JAVA_HOME not set" + return '"' + (Resolve-Path "java.exe").Path + '"' +} + + +function runFlume($javaClassPath, $javaLibraryPath, $javaOptions, $class, $javaProcessArgumentList) { + [string]$javaPath = GetJavaPath + [string]$fullJavaCommand = "-classpath $javaClassPath -Djava.library.path=$javaLibraryPath $javaOptions $class $javaProcessArgumentList" + if ($dryrun) { + Write-Host 'Dry run mode enabled (will not actually initiate startup)' + Write-Host "$javaPath $fullJavaCommand" + } else { + Write-Host " + Running FLUME $command : + class: $class + arguments: $javaProcessArgumentList + " + + $ErrorActionPreference = "Continue" + $x = Start-Process $javaPath -ArgumentList "$fullJavaCommand" -Wait -NoNewWindow + } +} + + +# The script will terminate if any steps fail +$ErrorActionPreference = "Stop" + +#SWITCH FOR THE DIFFERENT COMMANDS +[string] $javaProcessArgumentList +switch ($command) +{ + 'help' { + PrintUsage + return } + + 'agent' { + if (!$Name) { + PrintUsage + Write-Host "ERROR: Name parameter missing" + return + } + if (!$ConfFile) { + PrintUsage + Write-Host "ERROR: ConfFile parameter missing" + return + } + $class='org.apache.flume.node.Application' + $confFile = '"' + (Resolve-Path $confFile).Path + '"' + $javaProcessArgumentList = "-n $name -f $confFile" } + + 'avro-client' { + $class='org.apache.flume.client.avro.AvroCLIClient' + if("$rpcProps" -eq "") { + if (!$AvroHost) { + PrintUsage + Write-Host "ERROR: Avro Host parameter missing" + return + } + if (!$AvroPort) { + PrintUsage + Write-Host "ERROR: Avro Port parameter missing" + return + } + $javaProcessArgumentList = " -H $AvroHost -p $AvroPort" + } else { + $javaProcessArgumentList += " --rpcProps ""$rpcProps""" + } + + if ($avroHeaderFile) { + $avroHeaderFile = (Resolve-Path $avroHeaderFile).Path + $javaProcessArgumentList += " -R $avroHeaderFile" } + if ($avroFileName) { + $avroFileName = (Resolve-Path $avroFileName).Path + $javaProcessArgumentList += " -F $avroFileName" } + if($avroDirName) { + $avroDirName = (Resolve-Path $avroDirName).Path + $javaProcessArgumentList += " --dirname ""$avroDirName""" } } + + 'version' { + $class='org.apache.flume.tools.VersionInfo' + $javaProcessArgumentList = "" } + + default { + PrintUsage + Write-Host "ERROR: Invalid command '$command'" + return } +} + +$FlumeHome = $null +if($env:FLUME_HOME) { + $FlumeHome = $env:FLUME_HOME +} else { + $ScriptPath = Split-Path -Parent $MyInvocation.MyCommand.Path + $FlumeHome = Split-Path -Parent $ScriptPath +} + +########### Source flume-env.ps1 ############## + +# allow users to override the default env vars via conf\flume-env.ps1 +if( "$conf" -eq "" ) { + if( Test-path ("$FlumeHome\conf") ) { + $conf = "$FlumeHome\conf" + Write-Host "WARN: Config directory not set. Defaulting to $conf" + } +} +if ( "$conf" -ne "" ) { + Write-Host "Sourcing environment configuration script $conf\flume-env.ps1" + if ( Test-path "$conf\flume-env.ps1" ) { + . "$conf\flume-env.ps1" + } else { + Write-Host "WARN: Did not find $conf\flume-env.ps1" + } +} else { + Write-Host "WARN: No configuration directory found! Use --conf to set." +} + +########### Setup JAVA_OPTS ############## + +[string]$javaOptions="$JAVA_OPTS" +foreach ($opt in $javaPropertiesArr) { + $javaOptions = "$javaOptions -D$opt" +} + +########### Setup Classpath ############### +# flume\conf ; flume_home\lib\* ; cmdline ; env.ps1 ; plugins.d ; hadoop.cpath ; hbase.cpath ; hive.cpath + +[string]$javaClassPath='"' + $conf + '"' +[string]$flumeLibJars="" +[string]$flumeLibDir = Resolve-Path "$FlumeHome\lib" + +# Add FlumeHome\lib\* to class path +$javaClassPath = "$javaClassPath;""$flumeLibDir\*""" +$flumeLibJars = "$flumeLibDir\*""" + +# Add classpath from cmd line & FLUME_CLASSPATH in flume-env.ps1 +if ( $FLUME_CLASSPATH ) { + $classPathArr = $FLUME_CLASSPATH.Split(";") +} +foreach ($path in $classPathArr) { + $fullPath = (Resolve-Path $path).Path + $javaClassPath = "$javaClassPath;""$fullPath""" +} + +$javaLibraryPath = "" + +# Add plugins.d into classpath and libpath +if ("$pluginsPath" -eq "") { + $pluginsPath = "$FlumeHome\plugins.d" +} + +foreach($plugin in $pluginsPath.Split(";") ) { + if ( Test-path "$plugin" ) { + $pluginTmp = + ( (Get-ChildItem "$plugin\*\lib") + (Get-ChildItem "$plugin\*\libext") ) -join "\*"";""" + if( "$pluginTmp" -ne "" ) { + $javaClassPath="$javaClassPath;""" + $pluginTmp + "\*"";" + } + + $javaLibraryPathTmp = (@(Get-ChildItem "$plugin\*\native") -join "\*"";""") + if( "$javaLibraryPathTmp" -ne "" ) { + $javaLibraryPath= "$javaLibraryPath""" + "$javaLibraryPathTmp" + "\*"";" + } + } +} + +# Add Hadoop classpath & java.library.path +$hadoopHome = GetHadoopHome +if("$hadoopHome" -ne "") { + $hadoopCmd = "$hadoopHome\bin\hadoop.cmd" + if( Test-Path $hadoopCmd ) { + Write-Host "Including Hadoop libraries found in ($hadoopHome) for DFS access" + $javaClassPath = "$javaClassPath;""$hadoopHome\conf"""; + foreach ($path in GetClassPath $hadoopCmd) { + $javaClassPath = "$javaClassPath;""$path""" + } + + foreach ( $path in GetJavaLibraryPath $hadoopCmd $flumeLibDir ) { + $javaLibraryPath = "$javaLibraryPath""$path"";" + } + } else { + Write-Host "WARN: $hadoopCmd not be found. Unable to include Hadoop's classpath & java.library.path" + } +} else { + Write-Host "WARN: HADOOP_PREFIX not set. Unable to include Hadoop's classpath & java.library.path" +} + +# Add HBase classpath & java.library.path +$hbaseHome = GetHbaseHome +if( "$hbaseHome" -ne "" ) { + $hbaseCmd = "$hbaseHome\bin\hbase.cmd" + if( Test-Path $hbaseCmd ) { + Write-Host "Including HBase libraries found via ($hbaseHome) for HBase access" + foreach ( $path in GetClassPath $hbaseCmd ) { + $javaClassPath = "$javaClassPath;""$path""" + } + $javaClassPath = "$javaClassPath;""$hbaseHome\conf""" + + foreach ( $path in GetJavaLibraryPath $hbaseCmd $flumeLibDir ) { + $javaLibraryPath = "$javaLibraryPath""$path"";" + } + } else { + Write-Host "WARN: $hbaseCmd not be found. Unable to include HBase classpath and java.library.path" + } +} + +# Add Hive classpath +$hiveHome = GetHiveHome +if( "$hiveHome" -ne "" ) { + $hiveLib = "$hiveHome\lib" + if( Test-Path $hiveLib ) { + Write-Host "Including Hive libraries found via ($hiveHome) for Hive access" + $javaClassPath = "$javaClassPath;""$hiveLib\*""" + } else { + Write-Host "WARN: $hiveLib not found. Unable to include Hive into classpath" + } + + $hiveConf = "$hiveHome\conf" + if( Test-Path $hiveConf ) { + Write-Host "Including Hive conf dir ($hiveConf) in classpath for Hive access" + $javaClassPath = "$javaClassPath;""$hiveConf""" + } else { + Write-Host "WARN: $hiveConf not found. Unable to include it into classpath" + } + + $hcatLib = "$hiveHome\hcatalog\share\hcatalog" + if( Test-Path $hcatLib ) { + Write-Host "Including HCatalog libraries ($hcatLib) for Hive access" + $javaClassPath = "$javaClassPath;""$hcatLib\*""" + } else { + Write-Host "WARN: $hcatLib not found. Unable to include HCatalog into classpath" + } +} + +runFlume $javaClassPath $javaLibraryPath $javaOptions $class $javaProcessArgumentList From 990776427d0bb12e8775c31fc78d47bea8f6501f Mon Sep 17 00:00:00 2001 From: Jarek Jarcec Cecho Date: Tue, 3 Feb 2015 13:32:28 -0800 Subject: [PATCH 199/341] FLUME-2443: org.apache.hadoop.fs.FSDataOutputStream.sync() is deprecated in hadoop 2.4 (Hari Shreedharan via Jarek Jarcec Cecho) --- .../flume/sink/hdfs/AbstractHDFSWriter.java | 51 +++++++++++++++++++ .../sink/hdfs/HDFSCompressedDataStream.java | 4 +- .../flume/sink/hdfs/HDFSDataStream.java | 4 +- .../flume/sink/hdfs/HDFSSequenceFile.java | 2 +- 4 files changed, 56 insertions(+), 5 deletions(-) diff --git a/flume-ng-sinks/flume-hdfs-sink/src/main/java/org/apache/flume/sink/hdfs/AbstractHDFSWriter.java b/flume-ng-sinks/flume-hdfs-sink/src/main/java/org/apache/flume/sink/hdfs/AbstractHDFSWriter.java index 043ca6c53c..e367e1271c 100644 --- a/flume-ng-sinks/flume-hdfs-sink/src/main/java/org/apache/flume/sink/hdfs/AbstractHDFSWriter.java +++ b/flume-ng-sinks/flume-hdfs-sink/src/main/java/org/apache/flume/sink/hdfs/AbstractHDFSWriter.java @@ -19,6 +19,7 @@ import com.google.common.base.Preconditions; import org.apache.flume.Context; +import org.apache.flume.FlumeException; import org.apache.flume.annotations.InterfaceAudience; import org.apache.flume.annotations.InterfaceStability; import org.apache.hadoop.fs.FSDataOutputStream; @@ -27,6 +28,7 @@ import org.slf4j.Logger; import org.slf4j.LoggerFactory; +import java.io.IOException; import java.io.OutputStream; import java.lang.reflect.InvocationTargetException; import java.lang.reflect.Method; @@ -43,6 +45,7 @@ public abstract class AbstractHDFSWriter implements HDFSWriter { private Path destPath; private Method refGetNumCurrentReplicas = null; private Method refGetDefaultReplication = null; + private Method refHflushOrSync = null; private Integer configuredMinReplicas = null; private Integer numberOfCloseRetries = null; private long timeBetweenCloseRetries = Long.MAX_VALUE; @@ -110,6 +113,7 @@ protected void registerCurrentStream(FSDataOutputStream outputStream, this.destPath = destPath; this.refGetNumCurrentReplicas = reflectGetNumCurrentReplicas(outputStream); this.refGetDefaultReplication = reflectGetDefaultReplication(fs); + this.refHflushOrSync = reflectHflushOrSync(outputStream); } @@ -225,4 +229,51 @@ private Method reflectGetDefaultReplication(FileSystem fileSystem) { } return m; } + + private Method reflectHflushOrSync(FSDataOutputStream os) { + Method m = null; + if(os != null) { + Class fsDataOutputStreamClass = os.getClass(); + try { + m = fsDataOutputStreamClass.getMethod("hflush"); + } catch (NoSuchMethodException ex) { + logger.debug("HFlush not found. Will use sync() instead"); + try { + m = fsDataOutputStreamClass.getMethod("sync"); + } catch (Exception ex1) { + String msg = "Neither hflush not sync were found. That seems to be " + + "a problem!"; + logger.error(msg); + throw new FlumeException(msg, ex1); + } + } + } + return m; + } + + /** + * If hflush is available in this version of HDFS, then this method calls + * hflush, else it calls sync. + * @param os - The stream to flush/sync + * @throws IOException + */ + protected void hflushOrSync(FSDataOutputStream os) throws IOException { + try { + // At this point the refHflushOrSync cannot be null, + // since register method would have thrown if it was. + this.refHflushOrSync.invoke(os); + } catch (InvocationTargetException e) { + String msg = "Error while trying to hflushOrSync!"; + logger.error(msg); + Throwable cause = e.getCause(); + if(cause != null && cause instanceof IOException) { + throw (IOException)cause; + } + throw new FlumeException(msg, e); + } catch (Exception e) { + String msg = "Error while trying to hflushOrSync!"; + logger.error(msg); + throw new FlumeException(msg, e); + } + } } diff --git a/flume-ng-sinks/flume-hdfs-sink/src/main/java/org/apache/flume/sink/hdfs/HDFSCompressedDataStream.java b/flume-ng-sinks/flume-hdfs-sink/src/main/java/org/apache/flume/sink/hdfs/HDFSCompressedDataStream.java index dc93e4f421..f1287958c8 100644 --- a/flume-ng-sinks/flume-hdfs-sink/src/main/java/org/apache/flume/sink/hdfs/HDFSCompressedDataStream.java +++ b/flume-ng-sinks/flume-hdfs-sink/src/main/java/org/apache/flume/sink/hdfs/HDFSCompressedDataStream.java @@ -139,7 +139,7 @@ public void sync() throws IOException { isFinished = true; } fsOut.flush(); - fsOut.sync(); + hflushOrSync(this.fsOut); } @Override @@ -151,7 +151,7 @@ public void close() throws IOException { isFinished = true; } fsOut.flush(); - fsOut.sync(); + hflushOrSync(fsOut); cmpOut.close(); if (compressor != null) { CodecPool.returnCompressor(compressor); diff --git a/flume-ng-sinks/flume-hdfs-sink/src/main/java/org/apache/flume/sink/hdfs/HDFSDataStream.java b/flume-ng-sinks/flume-hdfs-sink/src/main/java/org/apache/flume/sink/hdfs/HDFSDataStream.java index 6fa12eb315..7054bfc0fc 100644 --- a/flume-ng-sinks/flume-hdfs-sink/src/main/java/org/apache/flume/sink/hdfs/HDFSDataStream.java +++ b/flume-ng-sinks/flume-hdfs-sink/src/main/java/org/apache/flume/sink/hdfs/HDFSDataStream.java @@ -128,7 +128,7 @@ public void append(Event e) throws IOException { public void sync() throws IOException { serializer.flush(); outStream.flush(); - outStream.sync(); + hflushOrSync(outStream); } @Override @@ -136,7 +136,7 @@ public void close() throws IOException { serializer.flush(); serializer.beforeClose(); outStream.flush(); - outStream.sync(); + hflushOrSync(outStream); outStream.close(); unregisterCurrentStream(); diff --git a/flume-ng-sinks/flume-hdfs-sink/src/main/java/org/apache/flume/sink/hdfs/HDFSSequenceFile.java b/flume-ng-sinks/flume-hdfs-sink/src/main/java/org/apache/flume/sink/hdfs/HDFSSequenceFile.java index 26089872c4..a261cce6b3 100644 --- a/flume-ng-sinks/flume-hdfs-sink/src/main/java/org/apache/flume/sink/hdfs/HDFSSequenceFile.java +++ b/flume-ng-sinks/flume-hdfs-sink/src/main/java/org/apache/flume/sink/hdfs/HDFSSequenceFile.java @@ -110,7 +110,7 @@ public void append(Event e) throws IOException { @Override public void sync() throws IOException { - writer.syncFs(); + hflushOrSync(outStream); } @Override From a7f9255a83beb222e1f3e871dff2d44568fa3f43 Mon Sep 17 00:00:00 2001 From: Hari Shreedharan Date: Mon, 9 Feb 2015 23:15:02 -0800 Subject: [PATCH 200/341] FLUME-1734. Add a Hive Sink based on Hive Streaming support. (Roshan Naik via Hari Shreedharan) --- bin/flume-ng | 12 + conf/log4j.properties | 1 + .../flume/conf/sink/SinkConfiguration.java | 9 +- .../org/apache/flume/conf/sink/SinkType.java | 8 +- flume-ng-dist/pom.xml | 8 + flume-ng-doc/sphinx/FlumeUserGuide.rst | 143 +++++ .../thrift/ThriftFlumeEventServer.java | 18 +- .../thriftLegacy/TestThriftLegacySource.java | 1 - .../flume/thrift/ThriftSourceProtocol.java | 18 +- .../flume-hive-sink/metastore_db/db.lck | Bin 0 -> 38 bytes .../flume-hive-sink/metastore_db/dbex.lck | Bin 0 -> 4 bytes .../flume-hive-sink/metastore_db/log/log.ctrl | Bin 0 -> 48 bytes .../flume-hive-sink/metastore_db/log/log3.dat | Bin 0 -> 1048591 bytes .../flume-hive-sink/metastore_db/log/log4.dat | Bin 0 -> 1048576 bytes .../metastore_db/log/logmirror.ctrl | Bin 0 -> 48 bytes .../flume-hive-sink/metastore_db/seg0/c10.dat | Bin 0 -> 8192 bytes .../metastore_db/seg0/c101.dat | Bin 0 -> 16384 bytes .../metastore_db/seg0/c111.dat | Bin 0 -> 20480 bytes .../metastore_db/seg0/c121.dat | Bin 0 -> 16384 bytes .../metastore_db/seg0/c130.dat | Bin 0 -> 12288 bytes .../metastore_db/seg0/c141.dat | Bin 0 -> 8192 bytes .../metastore_db/seg0/c150.dat | Bin 0 -> 28672 bytes .../metastore_db/seg0/c161.dat | Bin 0 -> 20480 bytes .../metastore_db/seg0/c171.dat | Bin 0 -> 20480 bytes .../metastore_db/seg0/c180.dat | Bin 0 -> 69632 bytes .../metastore_db/seg0/c191.dat | Bin 0 -> 16384 bytes .../metastore_db/seg0/c1a1.dat | Bin 0 -> 8192 bytes .../metastore_db/seg0/c1b1.dat | Bin 0 -> 16384 bytes .../metastore_db/seg0/c1c0.dat | Bin 0 -> 8192 bytes .../metastore_db/seg0/c1d1.dat | Bin 0 -> 8192 bytes .../metastore_db/seg0/c1e0.dat | Bin 0 -> 8192 bytes .../metastore_db/seg0/c1e90.dat | Bin 0 -> 8192 bytes .../metastore_db/seg0/c1ea1.dat | Bin 0 -> 8192 bytes .../metastore_db/seg0/c1eb0.dat | Bin 0 -> 8192 bytes .../metastore_db/seg0/c1ec1.dat | Bin 0 -> 8192 bytes .../metastore_db/seg0/c1ed0.dat | Bin 0 -> 8192 bytes .../metastore_db/seg0/c1ee0.dat | Bin 0 -> 8192 bytes .../metastore_db/seg0/c1ef0.dat | Bin 0 -> 8192 bytes .../metastore_db/seg0/c1f01.dat | Bin 0 -> 8192 bytes .../metastore_db/seg0/c1f1.dat | Bin 0 -> 8192 bytes .../metastore_db/seg0/c1f11.dat | Bin 0 -> 8192 bytes .../metastore_db/seg0/c1f20.dat | Bin 0 -> 8192 bytes .../metastore_db/seg0/c1f30.dat | Bin 0 -> 8192 bytes .../metastore_db/seg0/c1f41.dat | Bin 0 -> 8192 bytes .../metastore_db/seg0/c1f50.dat | Bin 0 -> 8192 bytes .../flume-hive-sink/metastore_db/seg0/c20.dat | Bin 0 -> 102400 bytes .../metastore_db/seg0/c200.dat | Bin 0 -> 16384 bytes .../metastore_db/seg0/c211.dat | Bin 0 -> 8192 bytes .../metastore_db/seg0/c221.dat | Bin 0 -> 8192 bytes .../metastore_db/seg0/c230.dat | Bin 0 -> 385024 bytes .../metastore_db/seg0/c241.dat | Bin 0 -> 8192 bytes .../metastore_db/seg0/c251.dat | Bin 0 -> 20480 bytes .../metastore_db/seg0/c260.dat | Bin 0 -> 8192 bytes .../metastore_db/seg0/c271.dat | Bin 0 -> 8192 bytes .../metastore_db/seg0/c281.dat | Bin 0 -> 8192 bytes .../metastore_db/seg0/c290.dat | Bin 0 -> 8192 bytes .../metastore_db/seg0/c2a1.dat | Bin 0 -> 8192 bytes .../metastore_db/seg0/c2b1.dat | Bin 0 -> 8192 bytes .../metastore_db/seg0/c2c1.dat | Bin 0 -> 8192 bytes .../metastore_db/seg0/c2d0.dat | Bin 0 -> 8192 bytes .../metastore_db/seg0/c2e1.dat | Bin 0 -> 8192 bytes .../metastore_db/seg0/c2f0.dat | Bin 0 -> 8192 bytes .../metastore_db/seg0/c300.dat | Bin 0 -> 8192 bytes .../flume-hive-sink/metastore_db/seg0/c31.dat | Bin 0 -> 28672 bytes .../metastore_db/seg0/c311.dat | Bin 0 -> 8192 bytes .../metastore_db/seg0/c321.dat | Bin 0 -> 8192 bytes .../metastore_db/seg0/c331.dat | Bin 0 -> 8192 bytes .../metastore_db/seg0/c340.dat | Bin 0 -> 8192 bytes .../metastore_db/seg0/c351.dat | Bin 0 -> 8192 bytes .../metastore_db/seg0/c361.dat | Bin 0 -> 8192 bytes .../metastore_db/seg0/c371.dat | Bin 0 -> 8192 bytes .../metastore_db/seg0/c380.dat | Bin 0 -> 8192 bytes .../metastore_db/seg0/c391.dat | Bin 0 -> 8192 bytes .../metastore_db/seg0/c3a1.dat | Bin 0 -> 8192 bytes .../metastore_db/seg0/c3b1.dat | Bin 0 -> 8192 bytes .../metastore_db/seg0/c3c0.dat | Bin 0 -> 8192 bytes .../metastore_db/seg0/c3d1.dat | Bin 0 -> 8192 bytes .../metastore_db/seg0/c3e1.dat | Bin 0 -> 8192 bytes .../metastore_db/seg0/c3f1.dat | Bin 0 -> 8192 bytes .../metastore_db/seg0/c400.dat | Bin 0 -> 8192 bytes .../flume-hive-sink/metastore_db/seg0/c41.dat | Bin 0 -> 69632 bytes .../metastore_db/seg0/c411.dat | Bin 0 -> 8192 bytes .../metastore_db/seg0/c421.dat | Bin 0 -> 8192 bytes .../metastore_db/seg0/c430.dat | Bin 0 -> 8192 bytes .../metastore_db/seg0/c441.dat | Bin 0 -> 8192 bytes .../metastore_db/seg0/c451.dat | Bin 0 -> 8192 bytes .../metastore_db/seg0/c461.dat | Bin 0 -> 8192 bytes .../flume-hive-sink/metastore_db/seg0/c51.dat | Bin 0 -> 32768 bytes .../metastore_db/seg0/c540.dat | Bin 0 -> 131072 bytes .../metastore_db/seg0/c551.dat | Bin 0 -> 8192 bytes .../metastore_db/seg0/c560.dat | Bin 0 -> 65536 bytes .../metastore_db/seg0/c571.dat | Bin 0 -> 8192 bytes .../metastore_db/seg0/c581.dat | Bin 0 -> 8192 bytes .../metastore_db/seg0/c591.dat | Bin 0 -> 8192 bytes .../metastore_db/seg0/c5a0.dat | Bin 0 -> 8192 bytes .../metastore_db/seg0/c5b1.dat | Bin 0 -> 8192 bytes .../metastore_db/seg0/c5c0.dat | Bin 0 -> 12288 bytes .../metastore_db/seg0/c5d1.dat | Bin 0 -> 8192 bytes .../metastore_db/seg0/c5e0.dat | Bin 0 -> 8192 bytes .../metastore_db/seg0/c5f1.dat | Bin 0 -> 8192 bytes .../flume-hive-sink/metastore_db/seg0/c60.dat | Bin 0 -> 16384 bytes .../metastore_db/seg0/c600.dat | Bin 0 -> 98304 bytes .../metastore_db/seg0/c611.dat | Bin 0 -> 8192 bytes .../metastore_db/seg0/c620.dat | Bin 0 -> 98304 bytes .../metastore_db/seg0/c631.dat | Bin 0 -> 8192 bytes .../metastore_db/seg0/c640.dat | Bin 0 -> 98304 bytes .../metastore_db/seg0/c651.dat | Bin 0 -> 8192 bytes .../metastore_db/seg0/c660.dat | Bin 0 -> 12288 bytes .../metastore_db/seg0/c671.dat | Bin 0 -> 8192 bytes .../metastore_db/seg0/c680.dat | Bin 0 -> 98304 bytes .../metastore_db/seg0/c691.dat | Bin 0 -> 8192 bytes .../metastore_db/seg0/c6a0.dat | Bin 0 -> 8192 bytes .../metastore_db/seg0/c6b1.dat | Bin 0 -> 8192 bytes .../metastore_db/seg0/c6c0.dat | Bin 0 -> 8192 bytes .../metastore_db/seg0/c6d1.dat | Bin 0 -> 8192 bytes .../metastore_db/seg0/c6e0.dat | Bin 0 -> 65536 bytes .../metastore_db/seg0/c6f1.dat | Bin 0 -> 8192 bytes .../metastore_db/seg0/c700.dat | Bin 0 -> 98304 bytes .../flume-hive-sink/metastore_db/seg0/c71.dat | Bin 0 -> 16384 bytes .../metastore_db/seg0/c711.dat | Bin 0 -> 8192 bytes .../metastore_db/seg0/c720.dat | Bin 0 -> 65536 bytes .../metastore_db/seg0/c731.dat | Bin 0 -> 8192 bytes .../metastore_db/seg0/c740.dat | Bin 0 -> 8192 bytes .../metastore_db/seg0/c751.dat | Bin 0 -> 8192 bytes .../metastore_db/seg0/c760.dat | Bin 0 -> 98304 bytes .../metastore_db/seg0/c771.dat | Bin 0 -> 8192 bytes .../metastore_db/seg0/c780.dat | Bin 0 -> 8192 bytes .../metastore_db/seg0/c791.dat | Bin 0 -> 8192 bytes .../metastore_db/seg0/c7a0.dat | Bin 0 -> 98304 bytes .../metastore_db/seg0/c7b1.dat | Bin 0 -> 8192 bytes .../metastore_db/seg0/c7c1.dat | Bin 0 -> 8192 bytes .../metastore_db/seg0/c7d1.dat | Bin 0 -> 8192 bytes .../metastore_db/seg0/c7e1.dat | Bin 0 -> 8192 bytes .../metastore_db/seg0/c7f1.dat | Bin 0 -> 8192 bytes .../metastore_db/seg0/c801.dat | Bin 0 -> 8192 bytes .../flume-hive-sink/metastore_db/seg0/c81.dat | Bin 0 -> 8192 bytes .../metastore_db/seg0/c811.dat | Bin 0 -> 8192 bytes .../metastore_db/seg0/c821.dat | Bin 0 -> 8192 bytes .../metastore_db/seg0/c831.dat | Bin 0 -> 8192 bytes .../metastore_db/seg0/c841.dat | Bin 0 -> 8192 bytes .../metastore_db/seg0/c851.dat | Bin 0 -> 8192 bytes .../metastore_db/seg0/c861.dat | Bin 0 -> 8192 bytes .../metastore_db/seg0/c871.dat | Bin 0 -> 8192 bytes .../metastore_db/seg0/c881.dat | Bin 0 -> 8192 bytes .../metastore_db/seg0/c891.dat | Bin 0 -> 8192 bytes .../metastore_db/seg0/c8a1.dat | Bin 0 -> 8192 bytes .../metastore_db/seg0/c8b1.dat | Bin 0 -> 8192 bytes .../metastore_db/seg0/c8c1.dat | Bin 0 -> 8192 bytes .../metastore_db/seg0/c8d1.dat | Bin 0 -> 8192 bytes .../metastore_db/seg0/c8e1.dat | Bin 0 -> 8192 bytes .../metastore_db/seg0/c8f0.dat | Bin 0 -> 8192 bytes .../flume-hive-sink/metastore_db/seg0/c90.dat | Bin 0 -> 86016 bytes .../metastore_db/seg0/c901.dat | Bin 0 -> 8192 bytes .../metastore_db/seg0/c910.dat | Bin 0 -> 12288 bytes .../metastore_db/seg0/c921.dat | Bin 0 -> 8192 bytes .../metastore_db/seg0/c930.dat | Bin 0 -> 98304 bytes .../metastore_db/seg0/c941.dat | Bin 0 -> 8192 bytes .../metastore_db/seg0/c950.dat | Bin 0 -> 12288 bytes .../metastore_db/seg0/c961.dat | Bin 0 -> 8192 bytes .../metastore_db/seg0/c971.dat | Bin 0 -> 8192 bytes .../metastore_db/seg0/c981.dat | Bin 0 -> 8192 bytes .../metastore_db/seg0/c991.dat | Bin 0 -> 65536 bytes .../metastore_db/seg0/c9a1.dat | Bin 0 -> 8192 bytes .../metastore_db/seg0/c9b1.dat | Bin 0 -> 8192 bytes .../metastore_db/seg0/c9c1.dat | Bin 0 -> 8192 bytes .../metastore_db/seg0/c9d0.dat | Bin 0 -> 8192 bytes .../metastore_db/seg0/c9e1.dat | Bin 0 -> 8192 bytes .../metastore_db/seg0/c9f0.dat | Bin 0 -> 8192 bytes .../metastore_db/seg0/ca01.dat | Bin 0 -> 8192 bytes .../flume-hive-sink/metastore_db/seg0/ca1.dat | Bin 0 -> 98304 bytes .../metastore_db/seg0/ca10.dat | Bin 0 -> 8192 bytes .../metastore_db/seg0/ca21.dat | Bin 0 -> 8192 bytes .../metastore_db/seg0/ca31.dat | Bin 0 -> 8192 bytes .../metastore_db/seg0/ca40.dat | Bin 0 -> 8192 bytes .../metastore_db/seg0/ca51.dat | Bin 0 -> 8192 bytes .../metastore_db/seg0/ca61.dat | Bin 0 -> 65536 bytes .../metastore_db/seg0/ca70.dat | Bin 0 -> 65536 bytes .../metastore_db/seg0/ca81.dat | Bin 0 -> 8192 bytes .../metastore_db/seg0/ca90.dat | Bin 0 -> 65536 bytes .../metastore_db/seg0/caa1.dat | Bin 0 -> 8192 bytes .../metastore_db/seg0/cab1.dat | Bin 0 -> 8192 bytes .../metastore_db/seg0/cac1.dat | Bin 0 -> 8192 bytes .../metastore_db/seg0/cad1.dat | Bin 0 -> 8192 bytes .../metastore_db/seg0/cae1.dat | Bin 0 -> 8192 bytes .../metastore_db/seg0/caf1.dat | Bin 0 -> 8192 bytes .../metastore_db/seg0/cb00.dat | Bin 0 -> 8192 bytes .../flume-hive-sink/metastore_db/seg0/cb1.dat | Bin 0 -> 16384 bytes .../metastore_db/seg0/cb11.dat | Bin 0 -> 8192 bytes .../metastore_db/seg0/cb21.dat | Bin 0 -> 65536 bytes .../metastore_db/seg0/cb31.dat | Bin 0 -> 8192 bytes .../metastore_db/seg0/cb40.dat | Bin 0 -> 8192 bytes .../metastore_db/seg0/cb51.dat | Bin 0 -> 8192 bytes .../metastore_db/seg0/cb61.dat | Bin 0 -> 8192 bytes .../metastore_db/seg0/cb71.dat | Bin 0 -> 65536 bytes .../metastore_db/seg0/cb80.dat | Bin 0 -> 8192 bytes .../metastore_db/seg0/cb91.dat | Bin 0 -> 8192 bytes .../metastore_db/seg0/cba1.dat | Bin 0 -> 8192 bytes .../metastore_db/seg0/cbb1.dat | Bin 0 -> 65536 bytes .../metastore_db/seg0/cbc0.dat | Bin 0 -> 8192 bytes .../metastore_db/seg0/cbd1.dat | Bin 0 -> 8192 bytes .../metastore_db/seg0/cbe1.dat | Bin 0 -> 65536 bytes .../metastore_db/seg0/cbf1.dat | Bin 0 -> 8192 bytes .../flume-hive-sink/metastore_db/seg0/cc0.dat | Bin 0 -> 8192 bytes .../metastore_db/seg0/cc00.dat | Bin 0 -> 65536 bytes .../metastore_db/seg0/cc11.dat | Bin 0 -> 8192 bytes .../metastore_db/seg0/cc20.dat | Bin 0 -> 65536 bytes .../metastore_db/seg0/cc31.dat | Bin 0 -> 8192 bytes .../metastore_db/seg0/cc41.dat | Bin 0 -> 8192 bytes .../metastore_db/seg0/cc51.dat | Bin 0 -> 8192 bytes .../metastore_db/seg0/cc61.dat | Bin 0 -> 8192 bytes .../metastore_db/seg0/cc70.dat | Bin 0 -> 8192 bytes .../metastore_db/seg0/cc81.dat | Bin 0 -> 8192 bytes .../metastore_db/seg0/cc91.dat | Bin 0 -> 65536 bytes .../metastore_db/seg0/cca1.dat | Bin 0 -> 8192 bytes .../flume-hive-sink/metastore_db/seg0/cd1.dat | Bin 0 -> 8192 bytes .../flume-hive-sink/metastore_db/seg0/ce1.dat | Bin 0 -> 8192 bytes .../flume-hive-sink/metastore_db/seg0/cf0.dat | Bin 0 -> 24576 bytes .../metastore_db/seg0/d18e0.dat | Bin 0 -> 4096 bytes .../metastore_db/seg0/d18f1.dat | Bin 0 -> 4096 bytes .../metastore_db/seg0/d1900.dat | Bin 0 -> 4096 bytes .../metastore_db/seg0/d1911.dat | Bin 0 -> 4096 bytes .../metastore_db/seg0/d1920.dat | Bin 0 -> 4096 bytes .../metastore_db/seg0/d1930.dat | Bin 0 -> 4096 bytes .../metastore_db/seg0/d1940.dat | Bin 0 -> 4096 bytes .../metastore_db/seg0/d1951.dat | Bin 0 -> 4096 bytes .../metastore_db/seg0/d1961.dat | Bin 0 -> 4096 bytes .../metastore_db/seg0/d1970.dat | Bin 0 -> 4096 bytes .../metastore_db/seg0/d1980.dat | Bin 0 -> 4096 bytes .../metastore_db/seg0/d1991.dat | Bin 0 -> 4096 bytes .../metastore_db/seg0/d19a0.dat | Bin 0 -> 4096 bytes .../metastore_db/seg0/d19b0.dat | Bin 0 -> 4096 bytes .../metastore_db/seg0/d19c1.dat | Bin 0 -> 4096 bytes .../metastore_db/seg0/d19d0.dat | Bin 0 -> 4096 bytes .../metastore_db/seg0/d19e1.dat | Bin 0 -> 4096 bytes .../metastore_db/seg0/d19f0.dat | Bin 0 -> 4096 bytes .../metastore_db/seg0/d1a00.dat | Bin 0 -> 4096 bytes .../metastore_db/seg0/d1a10.dat | Bin 0 -> 4096 bytes .../metastore_db/seg0/d1a21.dat | Bin 0 -> 4096 bytes .../metastore_db/seg0/d1a31.dat | Bin 0 -> 4096 bytes .../metastore_db/seg0/d1a40.dat | Bin 0 -> 4096 bytes .../metastore_db/seg0/d1a50.dat | Bin 0 -> 4096 bytes .../metastore_db/seg0/d1a61.dat | Bin 0 -> 4096 bytes .../metastore_db/seg0/d1a70.dat | Bin 0 -> 4096 bytes .../metastore_db/seg0/d1a80.dat | Bin 0 -> 4096 bytes .../metastore_db/seg0/d1a91.dat | Bin 0 -> 4096 bytes .../metastore_db/seg0/d1aa0.dat | Bin 0 -> 4096 bytes .../metastore_db/seg0/d1ab1.dat | Bin 0 -> 4096 bytes .../metastore_db/seg0/d1ac0.dat | Bin 0 -> 4096 bytes .../metastore_db/seg0/d1ad0.dat | Bin 0 -> 4096 bytes .../metastore_db/seg0/d1ae0.dat | Bin 0 -> 4096 bytes .../metastore_db/seg0/d1af1.dat | Bin 0 -> 4096 bytes .../metastore_db/seg0/d1b01.dat | Bin 0 -> 4096 bytes .../metastore_db/seg0/d1b10.dat | Bin 0 -> 4096 bytes .../metastore_db/seg0/d1b20.dat | Bin 0 -> 4096 bytes .../metastore_db/seg0/d1b31.dat | Bin 0 -> 4096 bytes .../metastore_db/seg0/d1b40.dat | Bin 0 -> 4096 bytes .../metastore_db/seg0/d1b50.dat | Bin 0 -> 4096 bytes .../metastore_db/seg0/d1b61.dat | Bin 0 -> 4096 bytes .../metastore_db/seg0/d1b70.dat | Bin 0 -> 4096 bytes .../metastore_db/seg0/d1b81.dat | Bin 0 -> 4096 bytes .../metastore_db/seg0/d1b90.dat | Bin 0 -> 4096 bytes .../metastore_db/seg0/d1ba0.dat | Bin 0 -> 4096 bytes .../metastore_db/seg0/d1bb0.dat | Bin 0 -> 4096 bytes .../metastore_db/seg0/d1bc1.dat | Bin 0 -> 4096 bytes .../metastore_db/seg0/d1bd1.dat | Bin 0 -> 4096 bytes .../metastore_db/seg0/d1be0.dat | Bin 0 -> 4096 bytes .../metastore_db/seg0/d1bf0.dat | Bin 0 -> 4096 bytes .../metastore_db/seg0/d1c01.dat | Bin 0 -> 4096 bytes .../metastore_db/seg0/d1c10.dat | Bin 0 -> 4096 bytes .../metastore_db/seg0/d1c20.dat | Bin 0 -> 4096 bytes .../metastore_db/seg0/d1c31.dat | Bin 0 -> 4096 bytes .../metastore_db/seg0/d1c40.dat | Bin 0 -> 4096 bytes .../metastore_db/seg0/d1c51.dat | Bin 0 -> 4096 bytes .../metastore_db/seg0/d1c60.dat | Bin 0 -> 4096 bytes .../metastore_db/seg0/d1c70.dat | Bin 0 -> 4096 bytes .../metastore_db/seg0/d1c80.dat | Bin 0 -> 4096 bytes .../metastore_db/seg0/d1c91.dat | Bin 0 -> 4096 bytes .../metastore_db/seg0/d1ca1.dat | Bin 0 -> 4096 bytes .../metastore_db/seg0/d1cb0.dat | Bin 0 -> 4096 bytes .../metastore_db/seg0/d1cc0.dat | Bin 0 -> 4096 bytes .../metastore_db/seg0/d1cd1.dat | Bin 0 -> 4096 bytes .../metastore_db/seg0/d1ce0.dat | Bin 0 -> 4096 bytes .../metastore_db/seg0/d1cf0.dat | Bin 0 -> 4096 bytes .../metastore_db/seg0/d1d01.dat | Bin 0 -> 4096 bytes .../metastore_db/seg0/d1d10.dat | Bin 0 -> 4096 bytes .../metastore_db/seg0/d1d21.dat | Bin 0 -> 4096 bytes .../metastore_db/seg0/d1d30.dat | Bin 0 -> 4096 bytes .../metastore_db/seg0/d1d40.dat | Bin 0 -> 4096 bytes .../metastore_db/seg0/d1d50.dat | Bin 0 -> 4096 bytes .../metastore_db/seg0/d1d61.dat | Bin 0 -> 4096 bytes .../metastore_db/seg0/d1d71.dat | Bin 0 -> 4096 bytes .../metastore_db/seg0/d1d80.dat | Bin 0 -> 4096 bytes .../metastore_db/seg0/d1d90.dat | Bin 0 -> 4096 bytes .../metastore_db/seg0/d1da1.dat | Bin 0 -> 4096 bytes .../metastore_db/seg0/d1db0.dat | Bin 0 -> 4096 bytes .../metastore_db/seg0/d1dc0.dat | Bin 0 -> 4096 bytes .../metastore_db/seg0/d1dd1.dat | Bin 0 -> 4096 bytes .../metastore_db/seg0/d1de0.dat | Bin 0 -> 4096 bytes .../metastore_db/seg0/d1df1.dat | Bin 0 -> 4096 bytes .../metastore_db/seg0/d1e00.dat | Bin 0 -> 4096 bytes .../metastore_db/seg0/d1e10.dat | Bin 0 -> 4096 bytes .../metastore_db/seg0/d1e20.dat | Bin 0 -> 4096 bytes .../metastore_db/seg0/d1e31.dat | Bin 0 -> 4096 bytes .../metastore_db/seg0/d1e41.dat | Bin 0 -> 4096 bytes .../metastore_db/seg0/d1e50.dat | Bin 0 -> 4096 bytes .../metastore_db/seg0/d1e60.dat | Bin 0 -> 4096 bytes .../metastore_db/seg0/d1e71.dat | Bin 0 -> 4096 bytes .../metastore_db/seg0/d1e80.dat | Bin 0 -> 4096 bytes .../metastore_db/service.properties | 22 + flume-ng-sinks/flume-hive-sink/pom.xml | 189 +++++++ .../org/apache/flume/sink/hive/Config.java | 41 ++ .../hive/HiveDelimitedTextSerializer.java | 109 ++++ .../flume/sink/hive/HiveEventSerializer.java | 38 ++ .../flume/sink/hive/HiveJsonSerializer.java | 55 ++ .../org/apache/flume/sink/hive/HiveSink.java | 524 ++++++++++++++++++ .../apache/flume/sink/hive/HiveWriter.java | 426 ++++++++++++++ .../apache/flume/sink/hive/TestHiveSink.java | 424 ++++++++++++++ .../flume/sink/hive/TestHiveWriter.java | 321 +++++++++++ .../org/apache/flume/sink/hive/TestUtil.java | 228 ++++++++ .../src/test/resources/log4j.properties | 26 + flume-ng-sinks/pom.xml | 2 + .../apache/flume/source/scribe/Scribe.java | 9 +- pom.xml | 46 +- 323 files changed, 2659 insertions(+), 19 deletions(-) create mode 100644 flume-ng-sinks/flume-hive-sink/metastore_db/db.lck create mode 100644 flume-ng-sinks/flume-hive-sink/metastore_db/dbex.lck create mode 100644 flume-ng-sinks/flume-hive-sink/metastore_db/log/log.ctrl create mode 100644 flume-ng-sinks/flume-hive-sink/metastore_db/log/log3.dat create mode 100644 flume-ng-sinks/flume-hive-sink/metastore_db/log/log4.dat create mode 100644 flume-ng-sinks/flume-hive-sink/metastore_db/log/logmirror.ctrl create mode 100644 flume-ng-sinks/flume-hive-sink/metastore_db/seg0/c10.dat create mode 100644 flume-ng-sinks/flume-hive-sink/metastore_db/seg0/c101.dat create mode 100644 flume-ng-sinks/flume-hive-sink/metastore_db/seg0/c111.dat create mode 100644 flume-ng-sinks/flume-hive-sink/metastore_db/seg0/c121.dat create mode 100644 flume-ng-sinks/flume-hive-sink/metastore_db/seg0/c130.dat create mode 100644 flume-ng-sinks/flume-hive-sink/metastore_db/seg0/c141.dat create mode 100644 flume-ng-sinks/flume-hive-sink/metastore_db/seg0/c150.dat create mode 100644 flume-ng-sinks/flume-hive-sink/metastore_db/seg0/c161.dat create mode 100644 flume-ng-sinks/flume-hive-sink/metastore_db/seg0/c171.dat create mode 100644 flume-ng-sinks/flume-hive-sink/metastore_db/seg0/c180.dat create mode 100644 flume-ng-sinks/flume-hive-sink/metastore_db/seg0/c191.dat create mode 100644 flume-ng-sinks/flume-hive-sink/metastore_db/seg0/c1a1.dat create mode 100644 flume-ng-sinks/flume-hive-sink/metastore_db/seg0/c1b1.dat create mode 100644 flume-ng-sinks/flume-hive-sink/metastore_db/seg0/c1c0.dat create mode 100644 flume-ng-sinks/flume-hive-sink/metastore_db/seg0/c1d1.dat create mode 100644 flume-ng-sinks/flume-hive-sink/metastore_db/seg0/c1e0.dat create mode 100644 flume-ng-sinks/flume-hive-sink/metastore_db/seg0/c1e90.dat create mode 100644 flume-ng-sinks/flume-hive-sink/metastore_db/seg0/c1ea1.dat create mode 100644 flume-ng-sinks/flume-hive-sink/metastore_db/seg0/c1eb0.dat create mode 100644 flume-ng-sinks/flume-hive-sink/metastore_db/seg0/c1ec1.dat create mode 100644 flume-ng-sinks/flume-hive-sink/metastore_db/seg0/c1ed0.dat create mode 100644 flume-ng-sinks/flume-hive-sink/metastore_db/seg0/c1ee0.dat create mode 100644 flume-ng-sinks/flume-hive-sink/metastore_db/seg0/c1ef0.dat create mode 100644 flume-ng-sinks/flume-hive-sink/metastore_db/seg0/c1f01.dat create mode 100644 flume-ng-sinks/flume-hive-sink/metastore_db/seg0/c1f1.dat create mode 100644 flume-ng-sinks/flume-hive-sink/metastore_db/seg0/c1f11.dat create mode 100644 flume-ng-sinks/flume-hive-sink/metastore_db/seg0/c1f20.dat create mode 100644 flume-ng-sinks/flume-hive-sink/metastore_db/seg0/c1f30.dat create mode 100644 flume-ng-sinks/flume-hive-sink/metastore_db/seg0/c1f41.dat create mode 100644 flume-ng-sinks/flume-hive-sink/metastore_db/seg0/c1f50.dat create mode 100644 flume-ng-sinks/flume-hive-sink/metastore_db/seg0/c20.dat create mode 100644 flume-ng-sinks/flume-hive-sink/metastore_db/seg0/c200.dat create mode 100644 flume-ng-sinks/flume-hive-sink/metastore_db/seg0/c211.dat create mode 100644 flume-ng-sinks/flume-hive-sink/metastore_db/seg0/c221.dat create mode 100644 flume-ng-sinks/flume-hive-sink/metastore_db/seg0/c230.dat create mode 100644 flume-ng-sinks/flume-hive-sink/metastore_db/seg0/c241.dat create mode 100644 flume-ng-sinks/flume-hive-sink/metastore_db/seg0/c251.dat create mode 100644 flume-ng-sinks/flume-hive-sink/metastore_db/seg0/c260.dat create mode 100644 flume-ng-sinks/flume-hive-sink/metastore_db/seg0/c271.dat create mode 100644 flume-ng-sinks/flume-hive-sink/metastore_db/seg0/c281.dat create mode 100644 flume-ng-sinks/flume-hive-sink/metastore_db/seg0/c290.dat create mode 100644 flume-ng-sinks/flume-hive-sink/metastore_db/seg0/c2a1.dat create mode 100644 flume-ng-sinks/flume-hive-sink/metastore_db/seg0/c2b1.dat create mode 100644 flume-ng-sinks/flume-hive-sink/metastore_db/seg0/c2c1.dat create mode 100644 flume-ng-sinks/flume-hive-sink/metastore_db/seg0/c2d0.dat create mode 100644 flume-ng-sinks/flume-hive-sink/metastore_db/seg0/c2e1.dat create mode 100644 flume-ng-sinks/flume-hive-sink/metastore_db/seg0/c2f0.dat create mode 100644 flume-ng-sinks/flume-hive-sink/metastore_db/seg0/c300.dat create mode 100644 flume-ng-sinks/flume-hive-sink/metastore_db/seg0/c31.dat create mode 100644 flume-ng-sinks/flume-hive-sink/metastore_db/seg0/c311.dat create mode 100644 flume-ng-sinks/flume-hive-sink/metastore_db/seg0/c321.dat create mode 100644 flume-ng-sinks/flume-hive-sink/metastore_db/seg0/c331.dat create mode 100644 flume-ng-sinks/flume-hive-sink/metastore_db/seg0/c340.dat create mode 100644 flume-ng-sinks/flume-hive-sink/metastore_db/seg0/c351.dat create mode 100644 flume-ng-sinks/flume-hive-sink/metastore_db/seg0/c361.dat create mode 100644 flume-ng-sinks/flume-hive-sink/metastore_db/seg0/c371.dat create mode 100644 flume-ng-sinks/flume-hive-sink/metastore_db/seg0/c380.dat create mode 100644 flume-ng-sinks/flume-hive-sink/metastore_db/seg0/c391.dat create mode 100644 flume-ng-sinks/flume-hive-sink/metastore_db/seg0/c3a1.dat create mode 100644 flume-ng-sinks/flume-hive-sink/metastore_db/seg0/c3b1.dat create mode 100644 flume-ng-sinks/flume-hive-sink/metastore_db/seg0/c3c0.dat create mode 100644 flume-ng-sinks/flume-hive-sink/metastore_db/seg0/c3d1.dat create mode 100644 flume-ng-sinks/flume-hive-sink/metastore_db/seg0/c3e1.dat create mode 100644 flume-ng-sinks/flume-hive-sink/metastore_db/seg0/c3f1.dat create mode 100644 flume-ng-sinks/flume-hive-sink/metastore_db/seg0/c400.dat create mode 100644 flume-ng-sinks/flume-hive-sink/metastore_db/seg0/c41.dat create mode 100644 flume-ng-sinks/flume-hive-sink/metastore_db/seg0/c411.dat create mode 100644 flume-ng-sinks/flume-hive-sink/metastore_db/seg0/c421.dat create mode 100644 flume-ng-sinks/flume-hive-sink/metastore_db/seg0/c430.dat create mode 100644 flume-ng-sinks/flume-hive-sink/metastore_db/seg0/c441.dat create mode 100644 flume-ng-sinks/flume-hive-sink/metastore_db/seg0/c451.dat create mode 100644 flume-ng-sinks/flume-hive-sink/metastore_db/seg0/c461.dat create mode 100644 flume-ng-sinks/flume-hive-sink/metastore_db/seg0/c51.dat create mode 100644 flume-ng-sinks/flume-hive-sink/metastore_db/seg0/c540.dat create mode 100644 flume-ng-sinks/flume-hive-sink/metastore_db/seg0/c551.dat create mode 100644 flume-ng-sinks/flume-hive-sink/metastore_db/seg0/c560.dat create mode 100644 flume-ng-sinks/flume-hive-sink/metastore_db/seg0/c571.dat create mode 100644 flume-ng-sinks/flume-hive-sink/metastore_db/seg0/c581.dat create mode 100644 flume-ng-sinks/flume-hive-sink/metastore_db/seg0/c591.dat create mode 100644 flume-ng-sinks/flume-hive-sink/metastore_db/seg0/c5a0.dat create mode 100644 flume-ng-sinks/flume-hive-sink/metastore_db/seg0/c5b1.dat create mode 100644 flume-ng-sinks/flume-hive-sink/metastore_db/seg0/c5c0.dat create mode 100644 flume-ng-sinks/flume-hive-sink/metastore_db/seg0/c5d1.dat create mode 100644 flume-ng-sinks/flume-hive-sink/metastore_db/seg0/c5e0.dat create mode 100644 flume-ng-sinks/flume-hive-sink/metastore_db/seg0/c5f1.dat create mode 100644 flume-ng-sinks/flume-hive-sink/metastore_db/seg0/c60.dat create mode 100644 flume-ng-sinks/flume-hive-sink/metastore_db/seg0/c600.dat create mode 100644 flume-ng-sinks/flume-hive-sink/metastore_db/seg0/c611.dat create mode 100644 flume-ng-sinks/flume-hive-sink/metastore_db/seg0/c620.dat create mode 100644 flume-ng-sinks/flume-hive-sink/metastore_db/seg0/c631.dat create mode 100644 flume-ng-sinks/flume-hive-sink/metastore_db/seg0/c640.dat create mode 100644 flume-ng-sinks/flume-hive-sink/metastore_db/seg0/c651.dat create mode 100644 flume-ng-sinks/flume-hive-sink/metastore_db/seg0/c660.dat create mode 100644 flume-ng-sinks/flume-hive-sink/metastore_db/seg0/c671.dat create mode 100644 flume-ng-sinks/flume-hive-sink/metastore_db/seg0/c680.dat create mode 100644 flume-ng-sinks/flume-hive-sink/metastore_db/seg0/c691.dat create mode 100644 flume-ng-sinks/flume-hive-sink/metastore_db/seg0/c6a0.dat create mode 100644 flume-ng-sinks/flume-hive-sink/metastore_db/seg0/c6b1.dat create mode 100644 flume-ng-sinks/flume-hive-sink/metastore_db/seg0/c6c0.dat create mode 100644 flume-ng-sinks/flume-hive-sink/metastore_db/seg0/c6d1.dat create mode 100644 flume-ng-sinks/flume-hive-sink/metastore_db/seg0/c6e0.dat create mode 100644 flume-ng-sinks/flume-hive-sink/metastore_db/seg0/c6f1.dat create mode 100644 flume-ng-sinks/flume-hive-sink/metastore_db/seg0/c700.dat create mode 100644 flume-ng-sinks/flume-hive-sink/metastore_db/seg0/c71.dat create mode 100644 flume-ng-sinks/flume-hive-sink/metastore_db/seg0/c711.dat create mode 100644 flume-ng-sinks/flume-hive-sink/metastore_db/seg0/c720.dat create mode 100644 flume-ng-sinks/flume-hive-sink/metastore_db/seg0/c731.dat create mode 100644 flume-ng-sinks/flume-hive-sink/metastore_db/seg0/c740.dat create mode 100644 flume-ng-sinks/flume-hive-sink/metastore_db/seg0/c751.dat create mode 100644 flume-ng-sinks/flume-hive-sink/metastore_db/seg0/c760.dat create mode 100644 flume-ng-sinks/flume-hive-sink/metastore_db/seg0/c771.dat create mode 100644 flume-ng-sinks/flume-hive-sink/metastore_db/seg0/c780.dat create mode 100644 flume-ng-sinks/flume-hive-sink/metastore_db/seg0/c791.dat create mode 100644 flume-ng-sinks/flume-hive-sink/metastore_db/seg0/c7a0.dat create mode 100644 flume-ng-sinks/flume-hive-sink/metastore_db/seg0/c7b1.dat create mode 100644 flume-ng-sinks/flume-hive-sink/metastore_db/seg0/c7c1.dat create mode 100644 flume-ng-sinks/flume-hive-sink/metastore_db/seg0/c7d1.dat create mode 100644 flume-ng-sinks/flume-hive-sink/metastore_db/seg0/c7e1.dat create mode 100644 flume-ng-sinks/flume-hive-sink/metastore_db/seg0/c7f1.dat create mode 100644 flume-ng-sinks/flume-hive-sink/metastore_db/seg0/c801.dat create mode 100644 flume-ng-sinks/flume-hive-sink/metastore_db/seg0/c81.dat create mode 100644 flume-ng-sinks/flume-hive-sink/metastore_db/seg0/c811.dat create mode 100644 flume-ng-sinks/flume-hive-sink/metastore_db/seg0/c821.dat create mode 100644 flume-ng-sinks/flume-hive-sink/metastore_db/seg0/c831.dat create mode 100644 flume-ng-sinks/flume-hive-sink/metastore_db/seg0/c841.dat create mode 100644 flume-ng-sinks/flume-hive-sink/metastore_db/seg0/c851.dat create mode 100644 flume-ng-sinks/flume-hive-sink/metastore_db/seg0/c861.dat create mode 100644 flume-ng-sinks/flume-hive-sink/metastore_db/seg0/c871.dat create mode 100644 flume-ng-sinks/flume-hive-sink/metastore_db/seg0/c881.dat create mode 100644 flume-ng-sinks/flume-hive-sink/metastore_db/seg0/c891.dat create mode 100644 flume-ng-sinks/flume-hive-sink/metastore_db/seg0/c8a1.dat create mode 100644 flume-ng-sinks/flume-hive-sink/metastore_db/seg0/c8b1.dat create mode 100644 flume-ng-sinks/flume-hive-sink/metastore_db/seg0/c8c1.dat create mode 100644 flume-ng-sinks/flume-hive-sink/metastore_db/seg0/c8d1.dat create mode 100644 flume-ng-sinks/flume-hive-sink/metastore_db/seg0/c8e1.dat create mode 100644 flume-ng-sinks/flume-hive-sink/metastore_db/seg0/c8f0.dat create mode 100644 flume-ng-sinks/flume-hive-sink/metastore_db/seg0/c90.dat create mode 100644 flume-ng-sinks/flume-hive-sink/metastore_db/seg0/c901.dat create mode 100644 flume-ng-sinks/flume-hive-sink/metastore_db/seg0/c910.dat create mode 100644 flume-ng-sinks/flume-hive-sink/metastore_db/seg0/c921.dat create mode 100644 flume-ng-sinks/flume-hive-sink/metastore_db/seg0/c930.dat create mode 100644 flume-ng-sinks/flume-hive-sink/metastore_db/seg0/c941.dat create mode 100644 flume-ng-sinks/flume-hive-sink/metastore_db/seg0/c950.dat create mode 100644 flume-ng-sinks/flume-hive-sink/metastore_db/seg0/c961.dat create mode 100644 flume-ng-sinks/flume-hive-sink/metastore_db/seg0/c971.dat create mode 100644 flume-ng-sinks/flume-hive-sink/metastore_db/seg0/c981.dat create mode 100644 flume-ng-sinks/flume-hive-sink/metastore_db/seg0/c991.dat create mode 100644 flume-ng-sinks/flume-hive-sink/metastore_db/seg0/c9a1.dat create mode 100644 flume-ng-sinks/flume-hive-sink/metastore_db/seg0/c9b1.dat create mode 100644 flume-ng-sinks/flume-hive-sink/metastore_db/seg0/c9c1.dat create mode 100644 flume-ng-sinks/flume-hive-sink/metastore_db/seg0/c9d0.dat create mode 100644 flume-ng-sinks/flume-hive-sink/metastore_db/seg0/c9e1.dat create mode 100644 flume-ng-sinks/flume-hive-sink/metastore_db/seg0/c9f0.dat create mode 100644 flume-ng-sinks/flume-hive-sink/metastore_db/seg0/ca01.dat create mode 100644 flume-ng-sinks/flume-hive-sink/metastore_db/seg0/ca1.dat create mode 100644 flume-ng-sinks/flume-hive-sink/metastore_db/seg0/ca10.dat create mode 100644 flume-ng-sinks/flume-hive-sink/metastore_db/seg0/ca21.dat create mode 100644 flume-ng-sinks/flume-hive-sink/metastore_db/seg0/ca31.dat create mode 100644 flume-ng-sinks/flume-hive-sink/metastore_db/seg0/ca40.dat create mode 100644 flume-ng-sinks/flume-hive-sink/metastore_db/seg0/ca51.dat create mode 100644 flume-ng-sinks/flume-hive-sink/metastore_db/seg0/ca61.dat create mode 100644 flume-ng-sinks/flume-hive-sink/metastore_db/seg0/ca70.dat create mode 100644 flume-ng-sinks/flume-hive-sink/metastore_db/seg0/ca81.dat create mode 100644 flume-ng-sinks/flume-hive-sink/metastore_db/seg0/ca90.dat create mode 100644 flume-ng-sinks/flume-hive-sink/metastore_db/seg0/caa1.dat create mode 100644 flume-ng-sinks/flume-hive-sink/metastore_db/seg0/cab1.dat create mode 100644 flume-ng-sinks/flume-hive-sink/metastore_db/seg0/cac1.dat create mode 100644 flume-ng-sinks/flume-hive-sink/metastore_db/seg0/cad1.dat create mode 100644 flume-ng-sinks/flume-hive-sink/metastore_db/seg0/cae1.dat create mode 100644 flume-ng-sinks/flume-hive-sink/metastore_db/seg0/caf1.dat create mode 100644 flume-ng-sinks/flume-hive-sink/metastore_db/seg0/cb00.dat create mode 100644 flume-ng-sinks/flume-hive-sink/metastore_db/seg0/cb1.dat create mode 100644 flume-ng-sinks/flume-hive-sink/metastore_db/seg0/cb11.dat create mode 100644 flume-ng-sinks/flume-hive-sink/metastore_db/seg0/cb21.dat create mode 100644 flume-ng-sinks/flume-hive-sink/metastore_db/seg0/cb31.dat create mode 100644 flume-ng-sinks/flume-hive-sink/metastore_db/seg0/cb40.dat create mode 100644 flume-ng-sinks/flume-hive-sink/metastore_db/seg0/cb51.dat create mode 100644 flume-ng-sinks/flume-hive-sink/metastore_db/seg0/cb61.dat create mode 100644 flume-ng-sinks/flume-hive-sink/metastore_db/seg0/cb71.dat create mode 100644 flume-ng-sinks/flume-hive-sink/metastore_db/seg0/cb80.dat create mode 100644 flume-ng-sinks/flume-hive-sink/metastore_db/seg0/cb91.dat create mode 100644 flume-ng-sinks/flume-hive-sink/metastore_db/seg0/cba1.dat create mode 100644 flume-ng-sinks/flume-hive-sink/metastore_db/seg0/cbb1.dat create mode 100644 flume-ng-sinks/flume-hive-sink/metastore_db/seg0/cbc0.dat create mode 100644 flume-ng-sinks/flume-hive-sink/metastore_db/seg0/cbd1.dat create mode 100644 flume-ng-sinks/flume-hive-sink/metastore_db/seg0/cbe1.dat create mode 100644 flume-ng-sinks/flume-hive-sink/metastore_db/seg0/cbf1.dat create mode 100644 flume-ng-sinks/flume-hive-sink/metastore_db/seg0/cc0.dat create mode 100644 flume-ng-sinks/flume-hive-sink/metastore_db/seg0/cc00.dat create mode 100644 flume-ng-sinks/flume-hive-sink/metastore_db/seg0/cc11.dat create mode 100644 flume-ng-sinks/flume-hive-sink/metastore_db/seg0/cc20.dat create mode 100644 flume-ng-sinks/flume-hive-sink/metastore_db/seg0/cc31.dat create mode 100644 flume-ng-sinks/flume-hive-sink/metastore_db/seg0/cc41.dat create mode 100644 flume-ng-sinks/flume-hive-sink/metastore_db/seg0/cc51.dat create mode 100644 flume-ng-sinks/flume-hive-sink/metastore_db/seg0/cc61.dat create mode 100644 flume-ng-sinks/flume-hive-sink/metastore_db/seg0/cc70.dat create mode 100644 flume-ng-sinks/flume-hive-sink/metastore_db/seg0/cc81.dat create mode 100644 flume-ng-sinks/flume-hive-sink/metastore_db/seg0/cc91.dat create mode 100644 flume-ng-sinks/flume-hive-sink/metastore_db/seg0/cca1.dat create mode 100644 flume-ng-sinks/flume-hive-sink/metastore_db/seg0/cd1.dat create mode 100644 flume-ng-sinks/flume-hive-sink/metastore_db/seg0/ce1.dat create mode 100644 flume-ng-sinks/flume-hive-sink/metastore_db/seg0/cf0.dat create mode 100644 flume-ng-sinks/flume-hive-sink/metastore_db/seg0/d18e0.dat create mode 100644 flume-ng-sinks/flume-hive-sink/metastore_db/seg0/d18f1.dat create mode 100644 flume-ng-sinks/flume-hive-sink/metastore_db/seg0/d1900.dat create mode 100644 flume-ng-sinks/flume-hive-sink/metastore_db/seg0/d1911.dat create mode 100644 flume-ng-sinks/flume-hive-sink/metastore_db/seg0/d1920.dat create mode 100644 flume-ng-sinks/flume-hive-sink/metastore_db/seg0/d1930.dat create mode 100644 flume-ng-sinks/flume-hive-sink/metastore_db/seg0/d1940.dat create mode 100644 flume-ng-sinks/flume-hive-sink/metastore_db/seg0/d1951.dat create mode 100644 flume-ng-sinks/flume-hive-sink/metastore_db/seg0/d1961.dat create mode 100644 flume-ng-sinks/flume-hive-sink/metastore_db/seg0/d1970.dat create mode 100644 flume-ng-sinks/flume-hive-sink/metastore_db/seg0/d1980.dat create mode 100644 flume-ng-sinks/flume-hive-sink/metastore_db/seg0/d1991.dat create mode 100644 flume-ng-sinks/flume-hive-sink/metastore_db/seg0/d19a0.dat create mode 100644 flume-ng-sinks/flume-hive-sink/metastore_db/seg0/d19b0.dat create mode 100644 flume-ng-sinks/flume-hive-sink/metastore_db/seg0/d19c1.dat create mode 100644 flume-ng-sinks/flume-hive-sink/metastore_db/seg0/d19d0.dat create mode 100644 flume-ng-sinks/flume-hive-sink/metastore_db/seg0/d19e1.dat create mode 100644 flume-ng-sinks/flume-hive-sink/metastore_db/seg0/d19f0.dat create mode 100644 flume-ng-sinks/flume-hive-sink/metastore_db/seg0/d1a00.dat create mode 100644 flume-ng-sinks/flume-hive-sink/metastore_db/seg0/d1a10.dat create mode 100644 flume-ng-sinks/flume-hive-sink/metastore_db/seg0/d1a21.dat create mode 100644 flume-ng-sinks/flume-hive-sink/metastore_db/seg0/d1a31.dat create mode 100644 flume-ng-sinks/flume-hive-sink/metastore_db/seg0/d1a40.dat create mode 100644 flume-ng-sinks/flume-hive-sink/metastore_db/seg0/d1a50.dat create mode 100644 flume-ng-sinks/flume-hive-sink/metastore_db/seg0/d1a61.dat create mode 100644 flume-ng-sinks/flume-hive-sink/metastore_db/seg0/d1a70.dat create mode 100644 flume-ng-sinks/flume-hive-sink/metastore_db/seg0/d1a80.dat create mode 100644 flume-ng-sinks/flume-hive-sink/metastore_db/seg0/d1a91.dat create mode 100644 flume-ng-sinks/flume-hive-sink/metastore_db/seg0/d1aa0.dat create mode 100644 flume-ng-sinks/flume-hive-sink/metastore_db/seg0/d1ab1.dat create mode 100644 flume-ng-sinks/flume-hive-sink/metastore_db/seg0/d1ac0.dat create mode 100644 flume-ng-sinks/flume-hive-sink/metastore_db/seg0/d1ad0.dat create mode 100644 flume-ng-sinks/flume-hive-sink/metastore_db/seg0/d1ae0.dat create mode 100644 flume-ng-sinks/flume-hive-sink/metastore_db/seg0/d1af1.dat create mode 100644 flume-ng-sinks/flume-hive-sink/metastore_db/seg0/d1b01.dat create mode 100644 flume-ng-sinks/flume-hive-sink/metastore_db/seg0/d1b10.dat create mode 100644 flume-ng-sinks/flume-hive-sink/metastore_db/seg0/d1b20.dat create mode 100644 flume-ng-sinks/flume-hive-sink/metastore_db/seg0/d1b31.dat create mode 100644 flume-ng-sinks/flume-hive-sink/metastore_db/seg0/d1b40.dat create mode 100644 flume-ng-sinks/flume-hive-sink/metastore_db/seg0/d1b50.dat create mode 100644 flume-ng-sinks/flume-hive-sink/metastore_db/seg0/d1b61.dat create mode 100644 flume-ng-sinks/flume-hive-sink/metastore_db/seg0/d1b70.dat create mode 100644 flume-ng-sinks/flume-hive-sink/metastore_db/seg0/d1b81.dat create mode 100644 flume-ng-sinks/flume-hive-sink/metastore_db/seg0/d1b90.dat create mode 100644 flume-ng-sinks/flume-hive-sink/metastore_db/seg0/d1ba0.dat create mode 100644 flume-ng-sinks/flume-hive-sink/metastore_db/seg0/d1bb0.dat create mode 100644 flume-ng-sinks/flume-hive-sink/metastore_db/seg0/d1bc1.dat create mode 100644 flume-ng-sinks/flume-hive-sink/metastore_db/seg0/d1bd1.dat create mode 100644 flume-ng-sinks/flume-hive-sink/metastore_db/seg0/d1be0.dat create mode 100644 flume-ng-sinks/flume-hive-sink/metastore_db/seg0/d1bf0.dat create mode 100644 flume-ng-sinks/flume-hive-sink/metastore_db/seg0/d1c01.dat create mode 100644 flume-ng-sinks/flume-hive-sink/metastore_db/seg0/d1c10.dat create mode 100644 flume-ng-sinks/flume-hive-sink/metastore_db/seg0/d1c20.dat create mode 100644 flume-ng-sinks/flume-hive-sink/metastore_db/seg0/d1c31.dat create mode 100644 flume-ng-sinks/flume-hive-sink/metastore_db/seg0/d1c40.dat create mode 100644 flume-ng-sinks/flume-hive-sink/metastore_db/seg0/d1c51.dat create mode 100644 flume-ng-sinks/flume-hive-sink/metastore_db/seg0/d1c60.dat create mode 100644 flume-ng-sinks/flume-hive-sink/metastore_db/seg0/d1c70.dat create mode 100644 flume-ng-sinks/flume-hive-sink/metastore_db/seg0/d1c80.dat create mode 100644 flume-ng-sinks/flume-hive-sink/metastore_db/seg0/d1c91.dat create mode 100644 flume-ng-sinks/flume-hive-sink/metastore_db/seg0/d1ca1.dat create mode 100644 flume-ng-sinks/flume-hive-sink/metastore_db/seg0/d1cb0.dat create mode 100644 flume-ng-sinks/flume-hive-sink/metastore_db/seg0/d1cc0.dat create mode 100644 flume-ng-sinks/flume-hive-sink/metastore_db/seg0/d1cd1.dat create mode 100644 flume-ng-sinks/flume-hive-sink/metastore_db/seg0/d1ce0.dat create mode 100644 flume-ng-sinks/flume-hive-sink/metastore_db/seg0/d1cf0.dat create mode 100644 flume-ng-sinks/flume-hive-sink/metastore_db/seg0/d1d01.dat create mode 100644 flume-ng-sinks/flume-hive-sink/metastore_db/seg0/d1d10.dat create mode 100644 flume-ng-sinks/flume-hive-sink/metastore_db/seg0/d1d21.dat create mode 100644 flume-ng-sinks/flume-hive-sink/metastore_db/seg0/d1d30.dat create mode 100644 flume-ng-sinks/flume-hive-sink/metastore_db/seg0/d1d40.dat create mode 100644 flume-ng-sinks/flume-hive-sink/metastore_db/seg0/d1d50.dat create mode 100644 flume-ng-sinks/flume-hive-sink/metastore_db/seg0/d1d61.dat create mode 100644 flume-ng-sinks/flume-hive-sink/metastore_db/seg0/d1d71.dat create mode 100644 flume-ng-sinks/flume-hive-sink/metastore_db/seg0/d1d80.dat create mode 100644 flume-ng-sinks/flume-hive-sink/metastore_db/seg0/d1d90.dat create mode 100644 flume-ng-sinks/flume-hive-sink/metastore_db/seg0/d1da1.dat create mode 100644 flume-ng-sinks/flume-hive-sink/metastore_db/seg0/d1db0.dat create mode 100644 flume-ng-sinks/flume-hive-sink/metastore_db/seg0/d1dc0.dat create mode 100644 flume-ng-sinks/flume-hive-sink/metastore_db/seg0/d1dd1.dat create mode 100644 flume-ng-sinks/flume-hive-sink/metastore_db/seg0/d1de0.dat create mode 100644 flume-ng-sinks/flume-hive-sink/metastore_db/seg0/d1df1.dat create mode 100644 flume-ng-sinks/flume-hive-sink/metastore_db/seg0/d1e00.dat create mode 100644 flume-ng-sinks/flume-hive-sink/metastore_db/seg0/d1e10.dat create mode 100644 flume-ng-sinks/flume-hive-sink/metastore_db/seg0/d1e20.dat create mode 100644 flume-ng-sinks/flume-hive-sink/metastore_db/seg0/d1e31.dat create mode 100644 flume-ng-sinks/flume-hive-sink/metastore_db/seg0/d1e41.dat create mode 100644 flume-ng-sinks/flume-hive-sink/metastore_db/seg0/d1e50.dat create mode 100644 flume-ng-sinks/flume-hive-sink/metastore_db/seg0/d1e60.dat create mode 100644 flume-ng-sinks/flume-hive-sink/metastore_db/seg0/d1e71.dat create mode 100644 flume-ng-sinks/flume-hive-sink/metastore_db/seg0/d1e80.dat create mode 100644 flume-ng-sinks/flume-hive-sink/metastore_db/service.properties create mode 100644 flume-ng-sinks/flume-hive-sink/pom.xml create mode 100644 flume-ng-sinks/flume-hive-sink/src/main/java/org/apache/flume/sink/hive/Config.java create mode 100644 flume-ng-sinks/flume-hive-sink/src/main/java/org/apache/flume/sink/hive/HiveDelimitedTextSerializer.java create mode 100644 flume-ng-sinks/flume-hive-sink/src/main/java/org/apache/flume/sink/hive/HiveEventSerializer.java create mode 100644 flume-ng-sinks/flume-hive-sink/src/main/java/org/apache/flume/sink/hive/HiveJsonSerializer.java create mode 100644 flume-ng-sinks/flume-hive-sink/src/main/java/org/apache/flume/sink/hive/HiveSink.java create mode 100644 flume-ng-sinks/flume-hive-sink/src/main/java/org/apache/flume/sink/hive/HiveWriter.java create mode 100644 flume-ng-sinks/flume-hive-sink/src/test/java/org/apache/flume/sink/hive/TestHiveSink.java create mode 100644 flume-ng-sinks/flume-hive-sink/src/test/java/org/apache/flume/sink/hive/TestHiveWriter.java create mode 100644 flume-ng-sinks/flume-hive-sink/src/test/java/org/apache/flume/sink/hive/TestUtil.java create mode 100644 flume-ng-sinks/flume-hive-sink/src/test/resources/log4j.properties diff --git a/bin/flume-ng b/bin/flume-ng index 4b323a6f1c..380169fdcc 100755 --- a/bin/flume-ng +++ b/bin/flume-ng @@ -161,6 +161,17 @@ add_HBASE_paths() { fi } +add_hive_paths(){ + if [ -d "${HIVE_HOME}/lib" ]; then + info "Including Hive libraries found via ($HIVE_HOME) for Hive access" + FLUME_CLASSPATH="$FLUME_CLASSPATH:$HIVE_HOME/lib/*" + fi + if [ -d "${HCAT_HOME}/share/hcatalog" ]; then + info "Including HCatalog libraries found via ($HCAT_HOME) for Hive access" + FLUME_CLASSPATH="$FLUME_CLASSPATH:${HCAT_HOME}/share/hcatalog/*" + fi +} + set_LD_LIBRARY_PATH(){ #Append the FLUME_JAVA_LIBRARY_PATH to whatever the user may have specified in #flume-env.sh @@ -417,6 +428,7 @@ fi # look for hadoop libs add_hadoop_paths add_HBASE_paths +add_hive_paths # prepend conf dir to classpath if [ -n "$opt_conf" ]; then diff --git a/conf/log4j.properties b/conf/log4j.properties index 3918511d24..c948bff618 100644 --- a/conf/log4j.properties +++ b/conf/log4j.properties @@ -32,6 +32,7 @@ log4j.logger.org.jboss = WARN log4j.logger.org.mortbay = INFO log4j.logger.org.apache.avro.ipc.NettyTransceiver = WARN log4j.logger.org.apache.hadoop = INFO +log4j.logger.org.apache.hadoop.hive = ERROR # Define the root logger to the system property "flume.root.logger". log4j.rootLogger=${flume.root.logger} diff --git a/flume-ng-configuration/src/main/java/org/apache/flume/conf/sink/SinkConfiguration.java b/flume-ng-configuration/src/main/java/org/apache/flume/conf/sink/SinkConfiguration.java index ac11558005..b34a3673b9 100644 --- a/flume-ng-configuration/src/main/java/org/apache/flume/conf/sink/SinkConfiguration.java +++ b/flume-ng-configuration/src/main/java/org/apache/flume/conf/sink/SinkConfiguration.java @@ -141,14 +141,19 @@ public enum SinkConfigurationType { */ ASYNCHBASE("org.apache.flume.sink.hbase.HBaseSinkConfiguration"), - /** * MorphlineSolr sink * * @see org.apache.flume.sink.solr.morphline.MorphlineSolrSink */ MORPHLINE_SOLR("org.apache.flume.sink.solr.morphline" + - ".MorphlineSolrSinkConfiguration"); + ".MorphlineSolrSinkConfiguration"), + + /** + * Hive Sink + * @see org.apache.flume.sink.hive.HiveSink + */ + HIVE("org.apache.flume.sink.hive.HiveSinkConfiguration"); private final String sinkConfigurationName; diff --git a/flume-ng-configuration/src/main/java/org/apache/flume/conf/sink/SinkType.java b/flume-ng-configuration/src/main/java/org/apache/flume/conf/sink/SinkType.java index 0a1cd7a9e4..edb9f46733 100644 --- a/flume-ng-configuration/src/main/java/org/apache/flume/conf/sink/SinkType.java +++ b/flume-ng-configuration/src/main/java/org/apache/flume/conf/sink/SinkType.java @@ -99,7 +99,13 @@ public enum SinkType { * * @see org.apache.flume.sink.solr.morphline.MorphlineSolrSink */ - MORPHLINE_SOLR("org.apache.flume.sink.solr.morphline.MorphlineSolrSink"); + MORPHLINE_SOLR("org.apache.flume.sink.solr.morphline.MorphlineSolrSink"), + + /** + * Hive Sink + * @see org.apache.flume.sink.hive.HiveSink + */ + HIVE("org.apache.flume.sink.hive.HiveSink"); private final String sinkClassName; diff --git a/flume-ng-dist/pom.xml b/flume-ng-dist/pom.xml index 4d2ebfec86..a083fe2e0e 100644 --- a/flume-ng-dist/pom.xml +++ b/flume-ng-dist/pom.xml @@ -77,6 +77,10 @@ org.apache.flume.flume-ng-sinks flume-dataset-sink + + org.apache.flume.flume-ng-sinks + flume-hive-sink + @@ -92,6 +96,10 @@ org.apache.flume.flume-ng-sinks flume-dataset-sink + + org.apache.flume.flume-ng-sinks + flume-hive-sink + diff --git a/flume-ng-doc/sphinx/FlumeUserGuide.rst b/flume-ng-doc/sphinx/FlumeUserGuide.rst index 7a1dfce394..912342c9b8 100644 --- a/flume-ng-doc/sphinx/FlumeUserGuide.rst +++ b/flume-ng-doc/sphinx/FlumeUserGuide.rst @@ -1721,6 +1721,149 @@ The above configuration will round down the timestamp to the last 10th minute. F timestamp 11:54:34 AM, June 12, 2012 will cause the hdfs path to become ``/flume/events/2012-06-12/1150/00``. +Hive Sink +~~~~~~~~~ + +This sink streams events containing delimited text or JSON data directly into a Hive table or partition. +Events are written using Hive transactions. As soon as a set of events are committed to Hive, they become +immediately visible to Hive queries. Partitions to which flume will stream to can either be pre-created +or, optionally, Flume can create them if they are missing. Fields from incoming event data are mapped to +corresponding columns in the Hive table. **This sink is provided as a preview feature and not recommended +for use in production.** + +====================== ============ ====================================================================== +Name Default Description +====================== ============ ====================================================================== +**channel** -- +**type** -- The component type name, needs to be ``hive`` +**hive.metastore** -- Hive metastore URI (eg thrift://a.b.com:9083 ) +**hive.database** -- Hive database name +**hive.table** -- Hive table name +hive.partition -- Comma separate list of partition values identifying the partition to write to. May contain escape + sequences. E.g: If the table is partitioned by (continent: string, country :string, time : string) + then 'Asia,India,2014-02-26-01-21' will indicate continent=Asia,country=India,time=2014-02-26-01-21 +hive.txnsPerBatchAsk 100 Hive grants a *batch of transactions* instead of single transactions to streaming clients like Flume. + This setting configures the number of desired transactions per Transaction Batch. Data from all + transactions in a single batch end up in a single file. Flume will write a maximum of batchSize events + in each transaction in the batch. This setting in conjunction with batchSize provides control over the + size of each file. Note that eventually Hive will transparently compact these files into larger files. +heartBeatInterval 240 (In seconds) Interval between consecutive heartbeats sent to Hive to keep unused transactions from expiring. + Set this value to 0 to disable heartbeats. +autoCreatePartitions true Flume will automatically create the necessary Hive partitions to stream to +batchSize 15000 Max number of events written to Hive in a single Hive transaction +maxOpenConnections 500 Allow only this number of open connections. If this number is exceeded, the least recently used connection is closed. +callTimeout 10000 (In milliseconds) Timeout for Hive & HDFS I/O operations, such as openTxn, write, commit, abort. +**serializer** Serializer is responsible for parsing out field from the event and mapping them to columns in the hive table. + Choice of serializer depends upon the format of the data in the event. Supported serializers: DELIMITED and JSON +roundUnit minute The unit of the round down value - ``second``, ``minute`` or ``hour``. +roundValue 1 Rounded down to the highest multiple of this (in the unit configured using hive.roundUnit), less than current time +timeZone Local Time Name of the timezone that should be used for resolving the escape sequences in partition, e.g. America/Los_Angeles. +useLocalTimeStamp false Use the local time (instead of the timestamp from the event header) while replacing the escape sequences. +====================== ============ ====================================================================== + +Following serializers are provided for Hive sink: + +**JSON**: Handles UTF8 encoded Json (strict syntax) events and requires no configration. Object names +in the JSON are mapped directly to columns with the same name in the Hive table. +Internally uses org.apache.hive.hcatalog.data.JsonSerDe but is independent of the Serde of the Hive table. +This serializer requires HCatalog to be installed. + +**DELIMITED**: Handles simple delimited textual events. +Internally uses LazySimpleSerde but is independent of the Serde of the Hive table. + +========================== ============ ====================================================================== +Name Default Description +========================== ============ ====================================================================== +serializer.delimiter , (Type: string) The field delimiter in the incoming data. To use special + characters, surround them with double quotes like "\\t" +**serializer.fieldnames** -- The mapping from input fields to columns in hive table. Specified as a + comma separated list (no spaces) of hive table columns names, identifying + the input fields in order of their occurrence. To skip fields leave the + column name unspecified. Eg. 'time,,ip,message' indicates the 1st, 3rd + and 4th fields in input map to time, ip and message columns in the hive table. +serializer.serdeSeparator Ctrl-A (Type: character) Customizes the separator used by underlying serde. There + can be a gain in efficiency if the fields in serializer.fieldnames are in + same order as table columns, the serializer.delimiter is same as the + serializer.serdeSeparator and number of fields in serializer.fieldnames + is less than or equal to number of table columns, as the fields in incoming + event body do not need to be reordered to match order of table columns. + Use single quotes for special characters like '\\t'. + Ensure input fields do not contain this character. NOTE: If serializer.delimiter + is a single character, preferably set this to the same character +========================== ============ ====================================================================== + + +The following are the escape sequences supported: + +========= ================================================= +Alias Description +========= ================================================= +%{host} Substitute value of event header named "host". Arbitrary header names are supported. +%t Unix time in milliseconds +%a locale's short weekday name (Mon, Tue, ...) +%A locale's full weekday name (Monday, Tuesday, ...) +%b locale's short month name (Jan, Feb, ...) +%B locale's long month name (January, February, ...) +%c locale's date and time (Thu Mar 3 23:05:25 2005) +%d day of month (01) +%D date; same as %m/%d/%y +%H hour (00..23) +%I hour (01..12) +%j day of year (001..366) +%k hour ( 0..23) +%m month (01..12) +%M minute (00..59) +%p locale's equivalent of am or pm +%s seconds since 1970-01-01 00:00:00 UTC +%S second (00..59) +%y last two digits of year (00..99) +%Y year (2010) +%z +hhmm numeric timezone (for example, -0400) +========= ================================================= + + +.. note:: For all of the time related escape sequences, a header with the key + "timestamp" must exist among the headers of the event (unless ``useLocalTimeStamp`` is set to ``true``). One way to add + this automatically is to use the TimestampInterceptor. + +Example Hive table : + +.. code-block:: properties + + create table weblogs ( id int , msg string ) + partitioned by (continent string, country string, time string) + clustered by (id) into 5 buckets + stored as orc; + +Example for agent named a1: + +.. code-block:: properties + + a1.channels = c1 + a1.channels.c1.type = memory + a1.sinks = k1 + a1.sinks.k1.type = hive + a1.sinks.k1.channel = c1 + a1.sinks.k1.hive.metastore = thrift://127.0.0.1:9083 + a1.sinks.k1.hive.database = logsdb + a1.sinks.k1.hive.table = weblogs + a1.sinks.k1.hive.partition = asia,%{country},%y-%m-%d-%H-%M + a1.sinks.k1.useLocalTimeStamp = false + a1.sinks.k1.round = true + a1.sinks.k1.roundValue = 10 + a1.sinks.k1.roundUnit = minute + a1.sinks.k1.serializer = DELIMITED + a1.sinks.k1.serializer.delimiter = "\t" + a1.sinks.k1.serializer.serdeSeparator = '\t' + a1.sinks.k1.serializer.fieldnames =id,,msg + + +The above configuration will round down the timestamp to the last 10th minute. For example, an event with +timestamp header set to 11:54:34 AM, June 12, 2012 and 'country' header set to 'india' will evaluate to the +partition (continent='asia',country='india',time='2012-06-12-11-50'. The serializer is configured to +accept tab separated input containing three fields and to skip the second field. + + Logger Sink ~~~~~~~~~~~ diff --git a/flume-ng-legacy-sources/flume-thrift-source/src/main/java/com/cloudera/flume/handlers/thrift/ThriftFlumeEventServer.java b/flume-ng-legacy-sources/flume-thrift-source/src/main/java/com/cloudera/flume/handlers/thrift/ThriftFlumeEventServer.java index ff32c455a2..3bf632a2e1 100644 --- a/flume-ng-legacy-sources/flume-thrift-source/src/main/java/com/cloudera/flume/handlers/thrift/ThriftFlumeEventServer.java +++ b/flume-ng-legacy-sources/flume-thrift-source/src/main/java/com/cloudera/flume/handlers/thrift/ThriftFlumeEventServer.java @@ -209,14 +209,19 @@ public append() { super("append"); } - protected append_args getEmptyArgsInstance() { + public append_args getEmptyArgsInstance() { return new append_args(); } - protected org.apache.thrift.TBase getResult(I iface, append_args args) throws org.apache.thrift.TException { + public org.apache.thrift.TBase getResult(I iface, + append_args args) throws org.apache.thrift.TException { iface.append(args.evt); return null; } + + public boolean isOneway() { + return false; + } } private static class close extends org.apache.thrift.ProcessFunction { @@ -224,15 +229,20 @@ public close() { super("close"); } - protected close_args getEmptyArgsInstance() { + public close_args getEmptyArgsInstance() { return new close_args(); } - protected close_result getResult(I iface, close_args args) throws org.apache.thrift.TException { + public close_result getResult(I iface, close_args args) throws org.apache + .thrift.TException { close_result result = new close_result(); iface.close(); return result; } + + public boolean isOneway() { + return false; + } } } diff --git a/flume-ng-legacy-sources/flume-thrift-source/src/test/java/org/apache/flume/source/thriftLegacy/TestThriftLegacySource.java b/flume-ng-legacy-sources/flume-thrift-source/src/test/java/org/apache/flume/source/thriftLegacy/TestThriftLegacySource.java index 8e08f223c5..d8a68721dd 100644 --- a/flume-ng-legacy-sources/flume-thrift-source/src/test/java/org/apache/flume/source/thriftLegacy/TestThriftLegacySource.java +++ b/flume-ng-legacy-sources/flume-thrift-source/src/test/java/org/apache/flume/source/thriftLegacy/TestThriftLegacySource.java @@ -145,7 +145,6 @@ private void stop() throws InterruptedException { @Test public void testLifecycle() throws InterruptedException { bind(); - Thread.sleep(1000); stop(); } diff --git a/flume-ng-sdk/src/main/java/org/apache/flume/thrift/ThriftSourceProtocol.java b/flume-ng-sdk/src/main/java/org/apache/flume/thrift/ThriftSourceProtocol.java index 7f966b0a7d..67a64ac7ea 100644 --- a/flume-ng-sdk/src/main/java/org/apache/flume/thrift/ThriftSourceProtocol.java +++ b/flume-ng-sdk/src/main/java/org/apache/flume/thrift/ThriftSourceProtocol.java @@ -228,15 +228,20 @@ public append() { super("append"); } - protected append_args getEmptyArgsInstance() { + public append_args getEmptyArgsInstance() { return new append_args(); } - protected append_result getResult(I iface, append_args args) throws org.apache.thrift.TException { + public append_result getResult(I iface, append_args args) throws org + .apache.thrift.TException { append_result result = new append_result(); result.success = iface.append(args.event); return result; } + + protected boolean isOneway() { + return false; + } } private static class appendBatch extends org.apache.thrift.ProcessFunction { @@ -244,15 +249,20 @@ public appendBatch() { super("appendBatch"); } - protected appendBatch_args getEmptyArgsInstance() { + public appendBatch_args getEmptyArgsInstance() { return new appendBatch_args(); } - protected appendBatch_result getResult(I iface, appendBatch_args args) throws org.apache.thrift.TException { + public appendBatch_result getResult(I iface, appendBatch_args args) + throws org.apache.thrift.TException { appendBatch_result result = new appendBatch_result(); result.success = iface.appendBatch(args.events); return result; } + + protected boolean isOneway() { + return false; + } } } diff --git a/flume-ng-sinks/flume-hive-sink/metastore_db/db.lck b/flume-ng-sinks/flume-hive-sink/metastore_db/db.lck new file mode 100644 index 0000000000000000000000000000000000000000..45e0595987566b3f1fe9e6fb22f32ccb9ba1bcb5 GIT binary patch literal 38 pcmZQjNwhFDOExe_)ip3QNzyepGDy=+OEm^^K_JO2)zH|;006BL2@L=M literal 0 HcmV?d00001 diff --git a/flume-ng-sinks/flume-hive-sink/metastore_db/dbex.lck b/flume-ng-sinks/flume-hive-sink/metastore_db/dbex.lck new file mode 100644 index 0000000000000000000000000000000000000000..720d64f4baafc33efdf971f02084aca5f25b34a5 GIT binary patch literal 4 LcmZQzU|<9Q00jU7 literal 0 HcmV?d00001 diff --git a/flume-ng-sinks/flume-hive-sink/metastore_db/log/log.ctrl b/flume-ng-sinks/flume-hive-sink/metastore_db/log/log.ctrl new file mode 100644 index 0000000000000000000000000000000000000000..bc4ad4cd28195e8506bd37318bc51ca4ac21bba7 GIT binary patch literal 48 mcmZQzU}#`qVBiE|W(KaW8bF#0h&dR9*t~!=3i#1)qy+#^*#rv! literal 0 HcmV?d00001 diff --git a/flume-ng-sinks/flume-hive-sink/metastore_db/log/log3.dat b/flume-ng-sinks/flume-hive-sink/metastore_db/log/log3.dat new file mode 100644 index 0000000000000000000000000000000000000000..1e8b5d6825dfea49843854c5e72f498422372f97 GIT binary patch literal 1048591 zcmdqKcYIvM)jmE~UAAn?vRrUoxnjV!swGqJ$`-b{QpI@j%Ec>3mFDs1!SeeswRza@yA7Y?Zm4{sZzY@%flx{|4O-n4|{t5dH1Q8$>(Gm zXz&wD18*$>j(Plq8wDWH;9sIadj<_g0~Q*1;{bCGG-y{J8V!^xAUUN@HgAfM0rcV& zXsBCYl-y-O5691>eRc@FMQGCuIRZ(jfdY9OAj`BA-Ad`9k`FHtfR}_+n}V`1Gzd{l zntPinD2##X`LU#hBYyUo`;^M6+V&~+Rqgdt>SuIS60owe zeR^kCRZT;vYn$Fj}q zx9#d$v3}>eS-ZAwT;I{Qv#WX6#*I5P4((VJz=VJ4zh!%8*Y>W?nOipQ+O*j~LOu_s zf?X=>JG!c>XF$JbbwRfcQgZge;JJTy3|^&jyi@(>Re}arwaR}!Cu_nnc;p}smZrVN z(9c^P((l4w;;`WI{wh(1pVdsSe{QPo4lm|Z^B>5Z3Vf}PZtI9 zQOa}G0bxzHV#xfaP*KL2#W3U${|1#9E{43p7&1GDv<)e9KmR7s=iF~oDRW=uSN@CE z{4&w{UpM{XDab+qiFNtE2Ng!^&J3|mqIJOlz%W{ODn{#8I-*gN(Hd7N|0BT?x@bzU z{ue*o-!=8ZlO_p_6|D;{urw$*odDlIW%w8IW@%7xmuS$HK?8}u1+N2!(Yi}V>sC@? z7{rw!S{E?>+t5%qI$9S_385LSTR}4-S{JSWBuA_(b;Rm{1aB=oRY-N}Xx&Oe6q6x$ zWL|;tVrsDWJ^Uh-hF6IJ!iHNq^h*z{G6ps;L8z#n>QkBq76!%JV zQ(5+6{^$CfeB>jpTNKRyPR038UNQ3G{BP#`&xK+5{~~n&7?Y{Y{^FUTlAQg!ku>J` zdsU)=V}LNa*UbLx#=wYaXnzLy7~|p_9Qs$NlDz(LM4u6)-=^XlihD7}#j3D*euBeH zlfZvt>L855P*u=4clO+7`qv=5^i@>B3qMj28PPtKjxc)>2hR2zRiZD2@s;10`)>KRH7)7%ir+_|bOM>lJ0QwBqovGoP*$0;BH_1v*>MZ zht(5~#9wR?-xm?rXAp5cV4-XOL4aWmuwK^yTR{>Z&{{k3!d1%u%JDkDH9-GMEDeTV zK`u+bb^PD)W@*rWpJ;GW1`VVJ=pW4glXMLL4SK2p2IR_+1kg}7x&|0f6+*KHAnk}6 zV8980q*I-gI%2bd1VaxvQ%J4XzOa>qf*Rm3Qv*zJY5+1T#)4~^{@1LU@y_3Hk>pi) zk?i+uvZ@s|L5{x)I&m0JQT{4Z6I4#Gt*Wb@frV4_dLgFDRO!rx3l_y^w#;4FoLIc9 zX<1VYO#Yp^fa0|p5lc@FMGWeKiasjAgru6hKDn9L_Nq2m3_`7Vu@ZnuN`D>l3u=S0 z2Xp=FRU(Jufhc`_j*imVjR)%kjHRZ1 zhR}b`q0jnYKzt+0NYYQJ_(oG7WJkYH<e^=iiS{kHK5Orn;f6p{}PXp!%xfnJqI-TY!xG-+%;}2%|xbX^du9dHzF&(L=pw z6mOj0q#6BAn4E}rXf)`KG)8kOJ^x{RIumc%Ga8`!QKLoiB`tF~tu?dXgA5r-omY860#%JC^p8uHP^H{I>1OpW>5guZ@!7XWg<`47y z#{mV~J)S)u*e-RP;{8B{d``d%wj2C(8m|RIJ^xXB8pm7qy!xrWDZ>=+D{Q0!8FDCC zF`sQn&1Y-?LY5({f9NwhpXr(1>_4dzxeP={EB{GTF7`eUl`x#?HX{ona(c+&#FfGH z#yD06_mdS?2JxH%%E{MbHoC;p}mD0501&N04b|hjrTOo`G_n3>E75aMVu1qr5TCQ4w~pNqtO}~XToR? ziqU2UMk6AT&ch2vJ1F5G(r9II)Qw|^w^qfs^u}lmrHhkc+ZZk{%5dP2^RK|+C{CM* z^pPVH4Y$~TMhu5G`!l^UTn3R!sGnvg#vAU|KQ!Kq##u1lLt?yHf$@k)q%)1A!-Ms| zi6=DPpg3ya81Y(Fd~0uvXNGubOT0xqEx!TiyhwB1dBZ^7HbMDMnqanO2HAYIEb`Zd3>8vdclrT za9eLIhWd(gbk!m;Si$}cMlmD_j$Gx4LN{CLKPQ_DXcRry8-ux`lu#wRQ8e-ehZpU2 zMB}lr*CS%DV*`5;(VCtk|LGvoUW4P?l^RPtOwaATv6q<(D;6%9*R&+8pbLALo+``- z=Yd@tUb)CbS2`lmZY|uD(Y-57&*yt%w+tecP(OQmj?Qry(s)NU&W7n-P&n=Lq5!2az@#8b@1YJn=eJ zd}nWL#+Aw9#FAyriTF}uG+<}Wq9h-B8)aF87ZnNiKawTea{ooKTBGt`?2Xk@){Z0CR1bwd0H)dF$)8>E~TP8AW)a&y&CerH3o__~EJsxj*MbS{*(B08l zJ-x@(Se6Z6EhpnL`3ExmcbW_*PVoGDjSTl{8S2~Gx@x;{1fYlhRWEu*gM4OtONMd7 z(_~mv;rVwP8Sd6HOmC=bsO+rmdC$E!b~q(PhOec`kcfHy14f1ij0|<%U6ozkJ+F*= zBg1E18OD!Ilc9CA=kGHz>=PMk>M<7e2W%`D*sZY`AI$%xPqvLR-3a)k=|)ViZ0qdk z>OOEj`8ijH3Bmd&DMR}Z&%ejm;T~;=+RCoBhR)ia8%n(yi_=nMxI1kuItF?EokoT` zwG1^~9n&j2YYv=0FojuTF|j&LhR*(;f1i=zJ|n~Qs#^5#4&2;4Cq;%}{hu6*t^(8C zgFl4Lz5VwPdovd2xiU;TJk1W>xt@Q!vBT}!4q0`rdLsi@NF0lScp+9#dJ;0QC9^TL zB}2y{z|TcUlQhNG!BTZ(2cvo;Ix}Kav6sb)z?}CjUBwh6fEMZJHs6Ja7%i zOFZh3Qb5w9Hl=#hB>;j)9TMz+X^#Ra5aOm4&3zz`8EXAOlQL4)J!e;LTP37yeIc$=xyRm6R=AJin^0 zvbu6cHU6uaKAk5PJ+F614QZ7BlcRE0MzrvG7U#Q{wm zs#5vtE`Oa$Or_gk`mZz7KO4d(ykNle?Bu%`6xsi^ZMxz(OHNs5T^IfE=WVgryc#~O^LR3X{7UG4Gn!3$4na3b>qrXul zCQ+vv}5%TB3Wl@D`h(%nM-HHSR9d+H`1M^<7af1Ek85}{$%jv!pgQIwi?ti!be zN?q(HB+Wvpgi$IxKd_WysDsU>@%(4FYvEObd}{?lBXwG)DtcqFtY@zxZ&9yug2O+K z0CAGvtP;ay%$j>6L9WVy?3%RXoJ#KhG3TsL%{da+E1~~f;8;gn?!RRDXYdy*$5!I} zi_WoIGUQmP5-Pd=WtFf+A8ok;Bte9enNzYEZC;Tn7E(AKcEc zG^oP)7p=k83>u6EEL@@LUck^5w(2SY8uU~pRQ)Uk8tO(@3DqSbG^+&CW-iE3B~-Kj z#Sz<@I%4%k%xa#0rI=gv(UvPnh*bhxDjAQr8fGo{SRZd?{RC~RgZQV(s8&S2TpzoQ zg&elCm4AG`4oU5G4YfGR(%o|e9Il4xPSW{%?l_}~bv9DvVmoQ%kZHv0fjvi9asNoHTRO^3z7e**mgt>#_Dz3>Ax zcbm~S?d2I*N+PZ1x>`q?y_iye@lR0+Ee-a9PC>DO5zS896lqGRmfdQu+v_l6&-9qa zCF7$8Tc4vv$%e2gEny?=*Rq7J5Bgum zw|NM5ufIHb2zGgIMmx)io)pAdLe~f9e~chJ1p94PPWtFI6qU*;JGW!887L}gxS=l$ z$)LU^HK-4UMKB8+4zrS&IRtxF!^UrNxbb4vB+z0|;Zk>dh9pM!B+M=uvh87Z$nFqq z1N&bTaeI`AuUaAwXQ)&!?tJJjyxAgtEFzwoLBuIW#2LZ)r&ATeIqrz^)QQtoLZ7)W zQ`2*FCL{S$D3n7B5^(}v&~L^v(eG61r;*n*s6;hqULVzvQxOZ0h;X@!4mB3ZH@9>o8o?DKL6Gd8cxmc1-m_8LILbcbINvWM)&ZLoW7h1h+V zs>n^Mj+n;_)-S~y(MA2kgY&O?=!&1QNtB^lwnzvv5l3iR>In8fShyJEMHq!Cq)+vE z5lloqBG~^lpcJ)kteAP28k;Jr7mT?j>Jeu-+$d^DqD+lV5jF6#K-9+_WQsZ!Af7Gi zu@DzlJ>u66XdHY-l&eI4f*z{MZBd7+!8^CNZQilIYxB;;#{jSlEuz)Tx0(w8ahUrW-#;E0e! zqi8WJL5E*6(UxC$Hu#qNAwUsb?~STTV= zarPy-=L-SxkE3}0J9MRjZQxv!(UnIeU715{`NvT~{cEC7g-yb7WvzLNe;oCu!@m_u z9G2=IV&S6;Y|x^C;FCGpIef&zN6!qA88L|5L$o`>SorAR{Lo0=mEJc$Y;gKXJKd~e1H?2|1mgJED?z#5zW8U5t1ozL4?QRbg(tK zia9gFh!!1tWN`gY9OT{EAulk-hzI*$TCw6Q1zA^N)r_U1)yR%5n|H3?j0k#Ee8>8> zX&qa3ZQi;4w4>&3?p)s%#tU6JFsP8QSS^0JBiUXMuL`17F`|{WG;s(oR*O6SuY=WC zg4mH%LQ^gm%O6b$C!_>HB9$?>$7bXVNxD}EbgZ{r!{~53eNtCH^$zVJ% zK&H_})W*#Yf2OQypsyQQWJfR&wee~Pv<-af$E$=sZ8}ViPZd=gW{{xEFgO0xAwf|? z5@l+9il{`kM4fq%gG^B?0is*4fYhQ&cAYuP0gZ#t$iY~Go9sH+7PY7BS^}z8cAfbZ zhw46?YMC0GB0Z69>3`@TGrNugh*KMgm1YI!zs!K{1)u6MB7K<}ljJj!0~+v|0Dj1> zvyKV*1lctVUwu$(Rm1`$BGd}f&Jby%WY=*X36IPx39_pyag*!Fe8Cs@nB~lWom^`( z7GEPHJ$csOLNU6Kw<@4`NWv+;wBnowmVuiZY~+Gg7WAO0ZwrKA?Z@lI1+nj|IX&4onm2x z$IcmSgX-iO2DIqVBZE6kAU^Is587CI&wK?xOk}=YSYs+E;E1g4#LwK>;T*v=)avcwlOo4RnJe#~8$c{Q$X_TqD zWFXawb5Ti)$vmHXL5R$G?m@QB!^4)r;SxxRL{>%agF^3IPW0EMYlfh)hJ0bzac_)w55_8nv*Rm$3QdzYadb2dP9qs#T^2rSM_ZX7dT|f6#pLpN36F zq!xCf&%9v&+lWLyl~^JRA8Uy`oW=wI1p4C*j@P=_b;d5Ztd z|8^)5TuA5hkc2a)+6F|nY`wto|AJPiGd0!!#MTRH96p>XndiB~ho*zs7HkcXVQalu z-v2ybfc?+7Dcgg2zSx@QVLs6D=5jmlC>@@+l^v`gq8 z(IY9oyQI-Z*4ZOW`^Cr}x~ih<5MGW4EIHZ1c}p1T9?D4LkOgfUySDG#5ymmCBFT`1 z*k?(y{$VitFgv+r{w(>cErU*A6{K%h95gA$bLj+ta@*f2M~qk_>MM7qt`NTfQXJ zZq?K{70t!Z#R4sli%7SniUiZq0xd!R=T_dyQp^@kRjxM(9~O9na!mH1JyGQL(~&__mRTu;K~Jg8tX73WsqZ zJRU-nY?Nsz`W7jp{}(!62^OkK-M-?f@8FF9L+~QdZ$kFl#1`D^;6(^ADIU*g9R1c* zlghX(p9TPq@*Nq*MeMkIsj#|3o1ML<(Vp(_6nG`+hFKou|0MVr<#8q!=J(BDnB~Fx z{|;@KN&u!BMs!{tjQ<@t7Nd$ONDR=tES*>6h|YIJ=)7W_=zLcOoyUO~%)4T?=zN#f zxfHA>7@hm%qo-wxGJq01Ksh}PWuno$5A=3X#3xn+_0L^ebvjRk>MMf&-(6aD$}aY0_8`5ea1RuY2mmMiD1_uhpxhZubOQ4~P4r-diB^vR04Ku3 z(I#3wQ&>H$y-4lHM60)_f%jyh)xr6(iB7Bl z6ixJSgo#dM{G*8;&0wO5V2>DiVz;n*RGY{LtL0{*^~<-K@cDoyMt~Lcf09@$%zW#Y zZ|#exVtqG_*Ge<(`XRiA4T1nm;?tU-|M4h23#-XUv#qHP>CjN{Frzl#Ngna6RmgR z{ScCaM`5M`nwK?rt?Ykr@V*|Qck6TjaPXeWp!dNbhQVt+URXV)^^SqnSd$+IAU{e= zP-lC<6GuF;HOPNYX-|ydlZZiVeIn#QWyoP1pn2KK{8E&8DngmITv6tEt&AbS|5@|O zO0#W}uzFr~At^7$Dlv=W22()*mT2pAklAdWSR^-x3{pioe3ISHt{pfrpj*`}eTgaS z)VAv#Y#&;tbxH;bBlF0~D2gDOK4(59WXB z@!bsL8WTPKC9J;7+bJ}w(IuYm;Zt+Mp6@Zd#5p^-{$T?CE}c^=(I2|!5DXWgWGp&X ziFn_Q5U+#vKgD}JgLn^u5X{{1HDUKU50Yx}#+2YA9(el-H_KyTe`H`qOtq=R_O??F zpSE@T`jhc}0@F@z+di#(%SL=@z>aCBZJxHaep}5c+bcIuZvpW$-qqE) zt_^1fV@RTT%a%4Rfpk=Wkp)V0b~s9K?t~kH|G%!M9%s)a z&K^MENPFb$nIX)75i;Kd3ljU_VD60;c6vu7BOwxz@$8xmLUgy+Q^!;0R@ZUX+)@`( zybHP>{RQ5vxz)7=R5<8A%`oV-fQ7xfg82Va9t0lT3XyZYLVPwd4_`r%v{nvpgP|wz zqQ??Hke2H|O8e6&Y5!tL>%B@6Q*QqXTXdGR-OPWK_U9R-{VZT1ZFd|noDo0QXSpjO zNqClfy(%n8%IjtP5wj~9{wgXsB;G0d|Lq#yo44)Z+kS?>18Udi4V$-|0zbnybu8L( zTF1IgZJVcV+|q%4U!^F|;doJ|A2>3_jZ6uTMjE3Eb2AF!X{NL@sfh2cW&NAfj=l@+ z45e~aX!$PmfaF9cs@y5Ec1?)Rp`>xlP&Q#GOU%BF#%r69RQ6Zaj}s=qgt8>2Ve~ex zF^Q+Q(8x%}d@c9CIOac(81uE*{|=*n>0`iDXMX!{)A44F`8uBe=9s^mVa#KIg?-o6 ziCXV6YU;|l29Kejc{4`MbzLF+77fp5U1yA%>n;}^-c8j(g8I69MZC8&h{rmV;sxj5 z-=<4y@oL%i<6Jp9Z;-Oae^f;b2cRc!*}isa+t#*@bzM`}wRLXUvUTdZ^(S{t-L`S+ z`YltpZ10%5aC^tnE`0k6D3FPG$NHm0FT-dchBrJ5SB|rANk1FKQO=Vf&*?OR>?Ma$494CFp7FcgX_GUOX zu>VWXe=oyuNYiISkpJJ)^}^E_4%7=V$e{1XIu=^q#I3p2nvUe!t=KfaGAj zmpT}S011P!;Z|F>d@>FzfdQxlyrj_>4u~aI*}(djAxv9aA3+F9*K(ZFw-7m4Vfqy( z-N^MXt@6hVR*8WPgxlC8EdHpSR6)4PvL52W#$6#iofJf(?6UDXK+-OMOtni~SU)YK zj4zRORT)0wT%U;3Zuypa6}+tqXIEmlMJ#vy=AHc2TqMRg2wgS>_rDl|oYUbUp%FMu zZqWB*kkreOa&saSZg$w-{!a}*0mO3)$+(QI>Y{#TRHnuTP_ zam$KBCOzhD=J^M@)BDlxv^l8%-`DBIhxW(!lwLMJFV=cruNZv@8!^o`WBsGu>HQRU z@+JWZhTk&4HhckDpCGBj!_!AqaZ){mbZZQ&)U<4p$lT3ackMh@-*mun_nL$M*QxIR zoILIaf+?j|4hahePAN1$Qz)aZ=>rY6WvG1 zvGj6vM5Ipz=)w3Km*kXP!=#Qkl-`u{`ilCAIyq~RExCR5zswoLT- zP&;HRpduV{+u49*eETrPA@wl;IOMh<|7+a}$T&>GqtcSFgtt7(g^NcCB&=-<_Wul> zZFbHbk(zTgK0Q-sSSze;pXv-$g-s=kf@eGTe>wR-%3$djgoVP}uM`#^X-mhT@c5oq z+G5cNV?7^ zsjjmWNbuTS!Tzt&j;zD-tO2g$X4j}gRx#Yq->0g!x~8s{kA(thl9zIP*X_0==tEqg zBTSW{05mSxKbiw3HAjX-#YIkL{0j?2eU2+6oJE0nawE_=GJlR9nUjP4k3Z{4eGbgi zV}W>bQ2+c{2hejMY{bZ%%=(Wb^XJr&nFl0{%*k)r+7*y>SRhuW7l`%=GVR=UMFom@ zm=wSER+B#ENQd;5aOuk6-0WDKnNH3vjEGAZqrj4ngk%&>xzHx2&w7Q#%5jX!AtvZV zog=b*(?-{7h85$Kw2BeE6ghR4rX#CH(mcY@JvHXgrH)m>x!f4ei#lT2WF>i!jA(f3 z2Ah=DFeK%MBA83iiOTg12J%$;KS%oQ)RAWQf&l1$(5=l=g7)Aib=j;he->|6d*f7| zf2Fs6n!#JeWlk#>K|j?EICgFFbZ4g~*|nK4L2J%d%|3@W0jZosWD{Lh`2BzxT_Y)>^ONrAS6ea$aK?y1MPvibSXZ7Dy7m(Ql%MChL zBhdp3i&wjch>Db}F_MstkJE$kH)IyuWXjM03(0T@G%^uo>*Dla{cDhE$b8WE=wrMH zCWblv0x`_rA`El-ZI)p?oof1&pG^0%40HM`Vwitu!<3SVzL-TO;{`W4{Ub5VKe%26zwfWl zOUa}mQhjqe@3F(kz-lC8E!J`;w&IOVH;n%Gl?_cQm)=nsn+{m-EP83w{WFF-`K zc+@SI;?0^WpZQcqe^iFi=LuG*b=D}c%BZlBS_kLRqkoiBa*VF%bsv>HBr}{P>%XCf z&p?Z?O zl=`2xD9%^s{eRP;97IcRy#N7qr^$KQh(*Q#u4B!F2{^(YbvbGTUq$ zj7ar%S<<3Pqb{T%x}3YnrlN1c%F2=?Lm{~T!<-b$5S64Cb8g`OF)k+bEtv=R^I{;7 zsTXsu)c>HkMMFm}CC=*$-ps18)VK;n3U}UNwr-m7wbF3w=ADa)sbSs0KqCYU$ zcq_7C^nI)Pp%6(-3~@k&37xFPwCCO7u&=-XjT(Y4xMnp~u7+4%2JRRcibm7`UHZSq zNsD(!qZ=m;&=-LOl>aREzr(1bAA3RWx)V^Rh9DSbtzwcsX^VKDmGzH^w-@5EuPoZx zePuHV0`bIgKFjqF9cOH`<9zl#QDLlhoJE)@)62aWWXR+==MN8|x4>~IYQ!4!{8@mc zz?-EivV(S|b>M1t|dQ4uqvN8#V4u>fC)DUqUfRGBO7)IgzdmKt_uu{VW zmC#=&HbPBs2}MrQ^FMK@JY!QSR}<1aiL!E-M4ivoI_R`x11bw$7L377+DQ+FHuyqO z@oZe1sd{b)AqfK- z%|T)giOR@aAm<-M_I)eNg2jhmi!k6_5Y->fG^esd`+@|x0n29bIFg=o~uM&oXWZvu!O%jOa2SnMTtof zN?drJt;EN`R-e9i6xYU<5*JGUbLiv@N=PAaq4Yli(*V}7t48j0mlm{^jlb~FXd zT8Jd3;02>zG)TO`kYO^CF$$!xqVwlMh zhPmh(%P^XedUx}-r(DBa^o)$|q1rH|_|#k+eOdBf^efTm&~VA$*V6<`p;MHJt<_!!}SbcSK6g9NbW#T~-^=y1uOulo%tOa44($qz(($$v4&pJC$Y zR7qqiT>LHDSbE9-&b5Vq2NlbUF6RCxr$Rh~xU%HGq)fz(hfDsFVj005s<-L1&u zCI6~oy;}1BtU|;0c(lt}@?UbVBOQIbpjf^wb&M*uM6!N_bnsLtVK4bF;rSO9G)7Ol zc}S^T73Vk_2|B}?Ecq{;5uyi6{(hQ8Q(kf8lK&k|R0oQ8u;ee!yyX8`+LHg$o575u zACDOQOTS@_zS;vZ)%W-PsRD1-=wHhEhoe6`!{|#OysT8znjHqh{#vb?q*`_${FH$Z z3Ulu83IpLg{KW}`mu(OwW=AM-*(J6TIuM@M|MZ!z5|`~0CFW*OLIUAsKM*D6hJmm! zaFUU5k}!fLf6{@F9%2Q;%Y*g5Awv%d$>6Sylt4&iD-d4J^S^X2gUmoUAecsgw0obI zbN$EBof~18%Wtv_Q$z{XSH^Ur5V1%1@|VOg^R!_y2ExlZ|0&eGFc6k{dU#9otwGcS z=>)2AXn%aY8dLSN$!y4Pyvp%l+S5)8y_m04ro8_W(=o;sY%AZym;z z8;s4@DwnGPmfVIhgtF!S(!mZo?cEnCwvUm!)RMc;ftCA9dmYA<8;ld0ak(nBm^Br3% zV(7%B9MH>D`cZu8cYbxeqe2`i6a^`LjOt@8G+h`>y0BBbON2c8x` zC=O)J`PI8Z^kB_drdc%Q^@&<@-mxp`$h+XkeKM{&@7R*I=DddeZ%)Pb2$#O*2+O6_ z7onv(`QB4c$D8HS*R+BPr{dZSE`1DO;bGTs{X>^t8&1W3dMZvzn~G=ZlQzy=T&w3I zNy%Ki=6fRe+9<(4wgk^%^ij_aTv~`XOYmzC62VW(Ah^uMYmXAaPtvWDQZqG-Zy(sX z$b}^=b?x0D!!Ah6xtJRq>Tyov=;bUmBBHeBuKm4@T+YY=WVbaZ3+S&6u|ahwu{iMf zk?{C1#^i-RX^!+)YXSYW6CE;i_mE5(YTvO^elA-^jCEJ`TnXJ#R`;9O)-!11dootxy zDG{by!|p)2r{ha0mM~$9C%rqcr%AY{eI1-|-_tGJ&&)8HI02#Zo{NP0nPL3^UGZ5; zo3y~GAMyb}0%C_fuZkpRrrLo6KrSEHBc$7;pFisEvG~v$p}m7e+_N)?dzm4$cZP_2 zwx0WR&%sy|>Abc>^({%}VXT*|WET9WBC>Nu3}U9|t;g~AUYv~D%lm&BQjK&>S$%@l z+!)o@N?ERSU=1bgly!ZDLy2>klw>_?uM44(pflS^%DSHM-v|<G43aQ}nz^ZX2> zF9Gs~dQt0qI%_dwX+=Q^=#0T(?;KM*lDZnt-*w@re;tGWNkZK92JU}S;PWE{zTvtw zfwyid{t_tI0(1Y90)M^-fjR#<4xbMN9$VBObjIXiW6}{Azefsu<1`WY^AQ5yxX2Q? z=(k{?e)HvTyyOae<1P{SybJ=%*|Qt37J<*>{3<}JOihQ*x?y&dv&lTB;y*cib`#b= z+AxOCK#XL#qwem7kD-6((C+l_Id~zS+~nKng?nrg?G~Uv}v3v&EX?Z1h8{ zp3a_8Udk#J=uL=!NpU$DU^Tcn8R{Tb5b6g0|KRMI ze!9Fll7V&he{}C0Vt{@2?3QyKlQO_&Ga|u&-tvr%uJ4^&ca)q6DOX43IKyE1ulU2Q zSnekWRG&SIKzH3=;^3{^|4jNrtL^?HQ{6xM?AfiKbI5G7ZQvejffRZ0W}R#FT(r)C z-1;4xiavXml_fWW9;WhbJpUEyqK|hSf(WTI`ckK*HV8swdAdvdCYTELtN>SXmVvnK zLWg?=OfY(=N(}S>Qm%$t;|T5;N5*cV2I${((050pV>Vjoi$DUp-(KjTUn%JNHnBQ2 z6!k*%>8=zNlk`bj#Jhd1Lw_&m^9E7TZk(l^akD_X86qC{Ka)el6GtGmZvc_JN~2>KrQ1*i_2ZBug7JRz#;E zq@Qp{aQ;^ZQGFcj@Szw64)qvq?7&@sj`LGi>>&uLh&p%R{EKB5s&mA_DlwK|BhUeta!OT|Y&OC<-)d{JTtAo?jAv)Drf){+}&Y2E64=9Zb=1slo?+OupdMNdL5~;IR zBM0n%T5852U^q9XrW!1ogSb73id)|KPlqtGY)H7LNhjsoU0naOd^<0q8oFzZRSl^< zV63`JzI7ztn3iR6bQjM5=<@Bt4Asy)z`+7{-6Dcss24|EZ4A|AbWO61wp*5Ob6Xw- zHrHyrn2UMr;Et#Hi?c0%H`l*Z;=%|e?jC0=@iDN~?dSd1xAA5varYci;^GWSNV2#) z*#Es)bpe^f*f&^|jKrcOEEtmIr|v3nMM4s?+Q0jLv4tVSgh?`bMN)>Hs*nsVr;TKF z748n|AA?Lo4g-B1-8E>~ri+=elPa^w#7|B`YK9E2z8G4eO7L6+S8 z4y@ew1^a(Vdk+Y{(WV^VXp^+JVGN;cx$oQRsQnt$E`zV?LZexgTXOe5z;Xxse@VGZ zgRj7K?{YSbIe{!cx&Poaxd&mE4>0x~Y{@;~z{-99UWYOLWMJ^+g6`Av3}c2KTkc;t z=(Kk^$`O66zF7^i5O@2P3lTg0Wc*wdB_I z=7EbR4-QO|dr;;&+%RUix9t5O`X4EAaL6$1^%*}IX31T4VD0_jw;aZ_clj`t(2UE~ zFl$~K#t_Pu`=KHSoqjU(AU||Kuw5@TriSnfr^W znVS=s0O)_s#gAsA`y%|L{(f<^I=#2TLPeSS>V-~v zN5jqTt3NVH#JxH~+((fAlj2?lZ1wi@4|7sm;y$uT#JwhixJLsPdOvcdh~>j`8$tBZ+M`+7kFn~>c0FVtbb_cYa--+6!|}t+w9B#`pHqVUAZ5fAaY-q zLGCsq_oH(~?(1{`^a0F_v*o6+JXRIuCG#Q*pp5rK?{|$2Mr1}GfT13J%x0+f0wZo& zHWe4wltjH>&_8w1Dwq9a&@T64%t{`KG1A_n2p_BZZqs_6Z`{9{iz zD#W2eNpSRQj4H7fo~{bu4^8b&j>iieN?cNel-xsP#+F2Vfak-JOEPY9+}V`2$?^DoU`Cg|F2bcB$NqQd(q>kb;E-m}=PXylePe{+Ph4UN zZuaH>@Z{npuHaAb{0jxYC4=BH7oXtyACAW@dT*{2!x%-{=c(r%ggI@kY3S|IDa-hv=!T$d(ihStt{Q{3436D?C+3+X5 zf~LQcL?G@bHpjtq_mE5(&P;~LT3jL%QH~KjS?wS*aEt_q4jh2eV8Z9gVEtM}!H z25U2X%Tx*<26>xLu>Wt6Ujv^J0MRSTfR7~8CtrX_oD#Q0c-E7@wmeI1BKldgK4wa` zveQ%S|I)KEediQ=h!}vUxc{LD;}xEx|MKLogLjw!4-(1O1aK#_%JmgU+82pPdVdP} zKb)lATZDRL=*Y_iK=gX*TG1=&>zJMj^8dXVCPTo9`JW=+t3Cqz#b}$78!iGg;|0B* z4*I|M(y{QUfE@2M|9LsqS*`r%bMU1U<16bsx~i&YR8GMY!rQ0RSGCtqsh`n>UwO4p z@9e6oX+YO>kbk*QUL(}IAObzyP>-Xjxe{kab!|Em9YTczSaaqlT2?mCot2o|JgaFX zx#{PUgD}Wku&4=~W> z2CYj~!;H?(>W22Lbs1@Nc_vu@8eL*2k#C~D`T#6I@ErVbd-oaoKmGFF2)}&hEXxaY zw^LoR^ojRC!Sc&zZU7bfrTH=<@-P>RKpEs`z5y5x@;(`4PdBdxAOL#L=2k%>)0Y12 zC;<|mdzR}T%AEGaLPF(P%>U4H`}{AdJhSue?+X|s>Z{xvmD?NM~3^kn1gqPD#<&DwDmb{fN3~!^XKL} zuvBQ9f1KRT+^GEHvR4RQFetPrzNBSt%iM*{TC!E(Moyy9b3y%Q%twvljZ>k~9?@v3 zXhb|)qmLXZsnPH_VDu4efNG77$2+X{{AdT38g2LI%6`ew%AcFPMtxKfG{OMQH{-)? zesU3Qp1;81L~R-yr$C$QM4Kt14e=~(o)6Z4#tPJCL>w^AbPTkC|EDJDjM^%*I2Z&9$P-`Qi3%hh+7=%VM-L&mtsPpAY6 z8|DTdU%o)%9aZgh)%Eq+5GP@&A70$<5MPA-y0q^zVWNbJFEam`2(b}=`2f#tJ{eyk zqf|xSSXGEE4qUQL$3L(?j-r#1kAjkj*y=^(|K!NyYgRpVB)*zmHyJAV)$G~ZDwpLD z$qMOj#2wNVLVAc91|glDr9#qTsc&54kfupulfy%I& zxoB5OcHbHUQ0CS9bzTKThT`d4vmAO=5cUm!w@TELAPSD%TG+O_y85c>`WZb=7Zy(! z0gI(fCtk4rx2|`n$I(2vkTLjRQlFz^@PB6gfG7D$3H4HN{?|m+#h9bt@z<-wp`y|H z?9CrgCuPa0F9rMGhB`~Ao2hj!sq-S7NA|5zG7 z5WQx5`M*V+2PEt`<$OTsaUt}OCgkocSAZ!T`epV%>2VKnXW3)2mG9{)!p?GVket9= z3{Agxn^iN8uA*gb)i!^766g?wF;5> zu?U%8YqY2oFn6l@(H++b6&b+S){4weWRO`x{c9Hkh6DJ7>H_v&tg!|lEtp$*v3$Bn zm*+K|?Uhy44L$cDuzZ@Cx*$9OG_B>+YeE0paAS!+tFaQH{(3q97KD{jq7%<5(O(bN z{|1jG`lvWy!oC!vN&+vqc? zN==Q`Q0Ycd314TfS1ZJ`RQhgk{wt}{=r~~XVFG~a^hGm($9F3o9vq~B{>9QLI!5^y zXRi{Th)tD}OTyJ)5m=k7zUr`|I>$CvL7f{!ohng>c(yu0|0k)=m^fhcqv`SJWHML1@&K2omd<& z`VQbgoptyDCi~ti4i7Bcya_lOl}`=}c*1J98pcX#ep3s-Z{Xjro0fqfc}gPx{;>{U z)=l&MYo)`7M1HMK}4)ax%qg(0^fhlfvrNqI|5Tc zuk@-^wU(o{t&Sn2YQII(#X^@Q3uym<`9C?G;}Bq-|By=PLm()c9?ITXdQ&t}3Q8ch z{=t2Y6pK+9Ugj^C#W`%fyf?NUgt_b3`i&s}8*gLLbSq)>={kVw$s_sWjUyomlgtyk zN6;?l-Zc*u5L@GE@!##5Xh|E7c zL1ca^L(bzQhfDqNY{1Z^o|2r`*IS5gvbnn&Evd`3OLbOvVOd>Y*Q3L-!G;MvXW`No z9Y&UcLl8>HDLDURI5GUpX*>ks=Nir;DbM+hYxj`+la?t`d<=VCWtP? zvvv7{gGXJ)#tB0-UJN9%?EdjV4lH$9>|ZJmpuw{H((FeD%X8{7KfbgjF{cSV-NvT4 z8Kb4bk1|59A7AS5q+Z80;ykzi718Tp(TjMNUO(pkmpew|;(!TjS?a~4PaYpSu+(b_ zYZ5)d(RDX9NmgUTt~>Nv+5(lDOdph+wLx_OWX3&F9s_sg}*t)VMTT3H{yAv zJ|3c>=g>q^hj^AcZwC8cX717NCd2{b*?uPx;ms=@9#p5rzf7uNM1;$-_dAIQ`Wtz< zwb25ptcdXD-yBZVWqKTX)MUK}%RgFjzd z?&u`X*U2i&PIAr|2x9f|e)@z%oHrh{5hSoGqk166yACM(qc{5}sl?%gg&ZgKMvi>0 z{Ybb2FL?R?1pPlfyt+Sni@!vgROpW`$=n~U%(_#B7A9Geo#^vF4?Fs>KYE3~NE&p| zXOY$?E2kIYmR)~VZWGu2(NoFawkz`-RR7sA0987Fah>^=TLMVM^0Q8d9ygay_n%dX z!$=Sn%d=Y8>KSd*vC&u4W0GSN=>lMpvIOd%x$~d>(Tli^IEmB|sIiThsXv;%^#h(( zfAr_b|5kir>%#Z* z=)LY4Du%rs+^V*+xp!SrrWev@>k>SEG2Y?9>VAR$RRjuDWO@L<+8cFq(PuQInUzRB zIR9dzV=K~DfrcSvyTdF(OHqmZZD89(()s)A#F2pBKtf;J?++ zq0X-84V=gmNgV|~CNc{5tY%ej3Vcdo75Hym0vG!YT z{-<-@u3>Cz%k2K;C`gi=-LJW`TZ-jhHUfyn@>3B7{x3VMV%Ze1 zBJ(p*GGqM{w)1kYA&*5HPrO&C1Ttg&qh)?BgUl>-Fc$9w@&7qRjrYOG_8DKvd6fv*PUD%7!*PxnKx z;K+XJ=3?D#)RFM?t7eB4BW1Jywd79V*K~Nw%5v<3kh+x@entOJD#AcnNSIvOikB5A zfAu2=mO1wf|2ZkVkaM5Yl2mncV*99}=Y=JM$R(goNf@T!A-@jtpD`%|<&D_LO2jA$ z0%c3~9+JJajTDqXto`et{xecAP`<{ZPM`gNj;5%~(%MNK#l_wo?pTn4@($WTuPY#g zwzf>*ciI2qEPqb79%2~-<+~dI#6bC6!~}l#5^DmRK>2L{2lBy+Y-9qzyAM1RsjyG>)jWh6>btJe=|m8em+9x-}n}l0e>Nt2Y&eEWkN+{ z{teeZl=;OBGPeR2F7=xW0Ym40QG!m9y4qhOose0|U!z?rt5bXCG+}(^;$?G}pv@_3 z?RDToUR=qLU(o+I{1|x_HsZkv{yp?zM4qE0@(_IXDiOVMOCaI37L& zkKYFU|D;Y|@mESe-96QmrM8_>L(5Zc*zplbo{DX=!6Ebh=q| zsui7rJt~<(?*->Sl0$V^95A|%4)WUX$Ok&T=juk&2(Wd|b zs#RiL1|EN$?Z8r{J^pG*#n`)DoxMufyA4!oS+&UY;kv;kr~t_==>H})THc5+jQ8&n zjgAnFh-YgQ^na6%H!ThreOM2mwnpy<{a*u1jrRJ-%g!J)I=;6Wxm&kvCRiH1|9QuL zoJuPiahlV=Lo}Kp8WGRZ=>4Z0Ji1+F95DJe#6TnIEWZDH2bLOL@2`?a38S;PDtnF4 zS;QD6cWyht%IYkBu*~5^ZB{lmK%3h|n+DN_c$PLF1pV*i_*BIKqj&F8wE5sshX=K} z!CxUa{zID;z1Ak&xNUQ_`EZ=WiQ24coDOYn6>X-AHpH{E`H=fR$?(Xrsg46iFFXQm zB)xvXiOidgx`Z!vkj5H@dYO#5EZ2E5W7pTN^vOsfkfnIxbRCU#0?XkbJ$vH6~Sb3URKlhqn|pUyc24ffE=@WBS-#t5T_jU*<7XmG{YgzL$SAV z38cS@4K)OoKpEPO_1U)_(U&DlvJ<=hDOmsM>ETsC^-EYH>!T;oXN_6YWpiVT5V!35 zaj~tBZu*`E{_qBK(XO(*__!9JN|)+r`rh`fkT1A3boS7DB zS9jeEY{l>YP%Kvzq>R``>wg@q|4pP|)AwTDYB8DA(e%Zw7XO*`x#%TN@)M0dIn>c; zF>Ev3zg{Iqh(_0EZ~lNfDN6zIiM#&ca`R3~;M`~;k^B=k|FNSn3HMX!U~vrZr^2(8 z=FDB*qzi-IwSKvBAs5m4f4Tk(XPaU}W1+t%xy0Vn8=Z3zpABWv`G14+pJo(W(a9(x zjI)yzNOGGaNfo@{hyJ@LEDr#m?+GR6Ob z{wL-&SL2-GmF^Um=D=TA|Ffa-VnlP`FIfN11yS=11OB8=s9#bcY@|8x7o7jl&4F)Z zXbuG8@cbvm`Gyt;86}Uza1kH?`nJuDT-3LgT>3Ks2xMmdqs-rkkonVrHWe-NSH8S9 zpdvDV%KjG{8sChT`P1V?=5LA#Gs*lZ`kzS)X_=3>Yx3!?%%9#UGJi8l<`*m~xnHH` ztNt}|k5CC@{+-DDQU;l&4Ebwt{{1D@1?;^DBWAEGZDb|9T39VO%kcScaU?=%Wm|&jSFuFTRu^#IwqfzXtn129IS(Z5%K< z1_M+_)75wj&bLLJOflylf3l*5ssI=DSlJ_u z!cF4jKZl9T-;R*^pQwMs$kp#OQaSnkN#7AFBJ)3a{)aNZoIz$ekn~T+f4bDm5_59A z75-DwS;T>)r*zDzoL*a1ha&<#?lxf~XXaw}HsDPV$U2bp&-Wc>j616v5#am>=)j0O zjS_c=XT_a=O>*!Uck1GR8AUW+^d0H?{%ffNyBBe1rT?UK7t!^7GJA#4^@T#xecLE3 zB!vF!F^3IRIk7PgRqhv6;-U)iY*m8$e~rDV%Jev3^pOF8YE{^pz>_NQ)YsIpS>-7>f?aX zOEG|IjkxJZje_-W(%X*rUzC0#_BLP4UL&4gUz}L7tT}wm+F z>LeO*dC-^T&D-m6g7qV9eJsqf{ z5A$hlc0ej}5p8lNrwvYPBaYelkBK&OL>uB++T`&3Z*qFm-DboAGcJQRIYIrK)Mhmg z3h50!X!Gp9p^fxJx29ICmWE#57aXCeOh@CfQ05U)=2%gNc$PAG zFFANr=Eyi;Dr{vWuI2IkN75;&OoER)(VY~;wdb=}#>BN13zy7mS`tL+?U0O|C>>%> zKJ$N)8#U@|#B&GyM@6IAq7m^djqj#0iq<#rVkV-nBWj+&#eO>klI`X6FCYNIiMauZ}B(EX*|fW zu{Uz$PY}Xvf>DmU9OCS`-Nx!u&p!0rP<^I%eWv3nG1=CdXp500*@-@dF^6RZ^tqIc zS*<(tNoakts!oKsW!J)>|6!sDpKowF_~+VoHIXAMdA~MuMn|&Hc1p9o=l}^P5gF;^Xq$~ zxuI@pUhL+-R`^^WO5pT@M3O$i{)ZWNt|-^=40j-jIm11@x~m>dfu5h!!!fv0bTlhU zug`8r&p7OQlK-es$BOdN>_c(3vpkR@xuW#?2K_&yBmHcYf1?a9tbJo|tewdfo=Lu+;B{Rvp=!|#3N z6TFd)S)F-({|+i#oxPG_btdAJF#b`TSM-V!ZfJMjgiuM75PDS;DkVF8@$=DmLFAJ4 zfMF7RmA7Z>eVfi4uL2+DdlfnP`2rb0e<=;R%kyFyI+lbUgFldVQ2!)pcPoz{7UYk? zw;7Q*e;Y2~ABV)TN`NS|N|?;7`@X$?!TRS_g-SvT&GQ+&hTtWm(Qg*$aWq~{9SyNh zKaM{Q{dxxbi~}L)(CUG*j@4uXp>meJ!OD4#pZKwyq1C%GyP|nQp#%q-EDJU|W z{jmNGSMBOkfU4mijsFbZ2q1L(m#5MFjOX#gKHj(#y5t7w`v?86r1WPgZ!A9Sn;$-z zhvE}nhN1r%Asa5}KESIy+0wFyq5n-GB!)rT=4`z4vWB5Q`oDS@UQZnc8K?g53BW6< z0Aig1{cP*#KFmS8e!{>^02#~y)u6(`{9g2|8L%8MoHgImvu3M1YXWZ2KRFhh32NG& zzO)$uMCJiJ|3csTUX;wwSX6RvC6&Lw@zqqBe=RcqAcM@Zlq~H77*5b1s4fuUso%f) zr^q8guv|Gs2jcd+hFbJtx_c}hu#{{@g&!XJ-35BqQnGZX!-@59S0gs>{3kpBq6u>x zCL$4bXQ&>*jB2 z&SnjteIm*AsTX-lvK$!HzlJZ<_7XmPuz@lnk(h@MW@SYd1*C4J?SXeYmTN`Y?n@YF zlQ59vCk`x&!TWtY!$%j(kiD3m} zwmFxTm7dvK+el%h?Xo8wDcA`9E>|16fP;=^wULz+wvLjv2aR;7Gi~pq9rQjc%%S~_ za~1wJi08jJrqY@Vya$F3}wNkQxw_|p8s&|HfN;A9x~B3Mxt#< zLz_Fo2muPDv z(SrVWau}P3cs$rhwC+f>zol_oIN0L}Poi~2p$%jG82?cv|PT5BX)QySW` z0UnPulUpJZ?aVZ^<^4V0#zeF=k!Xznu5K#;&s{FQMWX#K4Q*wK#|PsOZFLmd@QO6F zRee1^@rYI?OOJmzn$KrxklV6IwDkDbn&;`8`GM9FiN^Tn zdP!TZ$EQM&+tMhsk)>&9?KvKwy+E`jk!ZpCFFCtAJde*fAll+IG*AT#pXG%kq{nod5KeA4ho07@mKpx4fCbTPA`SS)C?|ys5o~ zi;Me8JiW}~Tai&j*VV)bu zy2M~z5`k3_^#7CN^%y1*59vU{i=(g-X}liCJi5qWT@;1&UujrRU@Bc`ur7?kV*iUl zF)88KF`F(hSQn&XNzFOdOT&6ndHRU}{qpfc!?m*A74f2CafaDUg-3s0gIc#)|r3dS94Eec}a<)^a`x%^E5AVWwuj`^P& zyp=%%>3fe0>YumNheiYRuu0CGf8#qL1L(m218AsQV3gcNP+WwRAshF5@TY&g#UZ10 zU<7W^nu|~a@q(n|2LqCW@mA_!z`7b>x{&&X!r}y^K!{=%;WaU@KzT77zWE-0kxMfx zP%Y?Xwcy_}{KG|JVw>@t|Fq37GuVc$drEbOF#RQMql5Ec{{*iRKdV4pf{ME1gZ|eq zwTHIilY>AegSr#)MBQJes>`_p&iHyJV>zv^<~#&9>=}JG=HbWu7@LM&4Q<^WmDRy> zN2)8kr_^+I;Q{%)Iy+``)^s++Fhcr?EHP4U(RkTBBwd5E45LgfB3Cf}zfde{wt-@I z3=e<7&uWGj`_CT!(q?!}4C?=1DE0-&V=Z~mj>pE(rp<-={D0|x^0XN;TPzHN^B>08 z?4T~?R-8Z613)eJ;(B3hBcCBIzBZBZpT72X24AZ%LQD$kpSQKIv25&)pX~YSX?z}f z8-c~!Cb9p=$@RAOwkz<-I7FEY-Zp8wc-z~l-o|Yh_{5~E9B(72a(u<71cw{nb!a&^UKCZiB_22H~Zzq6%L4k%EY^4rc$4#(GB^tJFZs=UiVw z;Dcw0!0+f%hL0>;HpBCGuWzWk!?Bw4M0%Rek zn|yW}o0Zc&w6L}5HfGPpoZbshi23&lkATm+X?#}Ic|IG27^2O+lP zpW|yif1Ba6?cdSDdWaLxb2X>Y#ODsB z|8rg^eQtHN=Q9+-=Qd@}D8ux9nNrKpbtC;RzK}z=_TuFykQMl*A7a4&oD2x2EaUIo2#~;1d^S&qaJ9e2mC6@F5TJiK#zI#@t zzegBJMw)he8l$yiJpXJ!AtRk_7&UZsRkrQt3;Ns}#Qi^mfR&Ld2d1%EH`>EN0G%n! zA%L!I*kFM8jDX;-djEY;-~fpuR0jQzq`R#j<@u)>KBr}`j_?V-Sj^|zfIK9CSH6{| z&q*Ua|8zhgfS;Z{9|Umd<2=Q||48@kkJ~$ta@S^vrPwiK8w;>c*~xd^<0_cGg$mT z3a5bE7t^?H9_IO6_}s{b%^Jm0|-pybFyb>CUOt~{l5qcsU~WI^>1>5Z5!#}E*>H9(a>p+H#Bs|se{F__Lo?PnrU4TF+?8=@QYuS{qSHZVE7(Qab=_}KCor31^ zPC#L(c4}UoU3D`m+xGMDhqSc#9!QXh%%3j_^!y!wfId46n}*uXj=Juk{ge6Yantm9QJ6?xtiLji(dp=E zUj`@?NdMu}zsVy|=+iVlXP^ttel`@k*eI0E_YOcI7MS6ZFwQ;>nMl#x(4NNV%skU> zN6~yq_KY$V%}a!h6wM9n|GEzT8Fbq(1r&CLgza}KtV z68iih?w-6d_uuVImFNx^*-RE$u69`QkZ5-!?qm1A?g5}gHO|YaU`xv&5{Vg`N_^Kr zWczGg9ObJ=yfzg-r8nI(+qq2xEhmQNS$|k8Z{{4mEe@*Y^4LAAz|Ry!^5YQ+M;IDx zH+JjH9gVor>iB<~SOUJ^oLLI@!w;gMcQfHWWk2t3j#kKgw@bMCz}yL;tb*u4LL z{aLT1opa9joH=vmOue|W%KfHTZh>bx5|ZdL4RnTU5z=xS<9lFK(rt3$d$O@y+OfJe zV16bBBqru4sL>hT_%~>&``sJFfb$*qhHMNNimQ~WDv@Hx6TJC9EtYodXv8r@_o&!$ zp=U=Dl8ED1;5~usf3^pn8OI<^T?N$Y#Lvyfj>deEwq8LD^Yq=}f<-IiOOxB}Ua#E} z3ezRPnd!~{2j)BA-Y6T3aVf!#*_ba0PATXd4?B!Dvs2(H40|BcWlm znZf!GtDVlbDUK1Kx)%gSJMUu{jl&6Tw8CtShC^Xy9kS6LbZ?U00Yl$h*s=PU_v}>Q>PbHb%K+#6;{Eu#9 zJfQF>CcE`RhOy4KF`6-BI&2?hB=TlxA^%@n9>OCEpLKCVN9Lhh`?rRchhy*w<2mt_ zwRM$<85J?ek2!Zn_D323vj_^8bZlvBKWF)#ZSDA_pmF)qrY8K5zpiy#MT8UmG)l>t z?R!pc-*U#T4gQ}BJ9fr*wgyE8b?jWRrDOZ<_ES1`ZJ51#+oq1zmR;=&cW>IXQ_+x7 zu~)|D(tYcWw)P$EZL_v++P!&;E(v_Zh$zs8>ow~us=WE7kT|CdF3tbMR?Fy&b6?_Z zRqEIpn*W))`DemdhZj_s3N37KsMHgmwD|K}{vn>vo=CSus&_KtT!`JXifR-~k)qbz z<1Dd^YCpk(a-xA^q1Kjcqnf9d#ErSPfj{HMD{10J3^JhlT<`qXE2^L71qyqy@~0sB zyEIrA&KezMx4H3+4@gbjE^Ll4HTQwdi>bmiiREZw$U-gMJ`?jDuhR z>sHsdR##M1d3P9NwXlr^s~0$+ygY^Dx4R1?LR=h z&7G|}5p+PBb4`0o&9rF&=DNwPwbhs&*4qJFs6siiS)D^TK;lFz>?D>MWE1=2@C`O?6d$ z%d~*?w8_&N+Ht0tuXS~8?UmK_&_CKw*m{_~^Pd6c-3^gSg}G8OZmpkISJARAcv5tF z%+W&7kf|_U#Qpz_qXFZy>LW80;~AQ9?fSO*sO!v!WqWkMUzEjL)r(oS+z=pc%K+w6s>W2aH4fqL(lgms?Vn!uY(Y5w{7YVC5#o zx}gp2ucamMj1blXKt#44>#ZrQ&z~4^8B@UeB*nVEt-iLt{TM5l@aCaOtiPAS`hp1& zmx94sU;A3$Hm$C;eSPOQ_kuNhBNB?GmwNl31Ao47d}JC5R79p}%q^|8mDTOR3R{S8 zlnPUiv3LF{V0_Uy)#Ft}Dm3Hz^%WJO0|D>|znMstXrm;EV)@%~{EW&stEV$Ha6U42c(x`1(rok?TZdAYa#6;$8>?07j5 zrC{e173;d%+VSH<`(;vea%bEY%apT~?NR`4|r5RUGYeNI~rkpC*rNE)WA@x$;}*oLhNK|ArXa%B zR1qmsj4RsO+A6T&@ECchq9IeU@~Yh_jIYI(0G2=n&eN>dwY9h6pU*nP7(;}uhxxx# zSRc-d)G5q$!nkE!TSawnQC*1Bbp;KXGC6?Lt>*bZjy+=-u@M}GLP5AneQGFn~z z7PxRc_IjGpY6r@J(V9LeU(#qGFte@~M@6>8;v2 zFi+a);VgrDT7jA&YQScDOka8-H)KKzyYaZEhj(KwTG+I3 z`LY;AEYEI^X{mraT?0JLi+`^>?fA=faNJPtB!=Q=>{u<&gIgsD6POtN zv4smM6wod zXnQ*6zu4Ba-|^bG_p!-TKa2>AzUSn9d^Dw0O9>h;!o2aX7RE*2O^r*Og!?4nVN=kO zXgSr;ge1CH2)*-9nvgcz62Cw$PFwH9FUZDb+)pygf6Rb38|pNJ9R@_xtPv&lq}c5P z?0(mF!w#1{+1M==Q7N@kvcz}79mZyH4X<* zm$GA%!G+nFPTY;#VYS_$WqP|2)!uTx1d8}uGvmMQZy#bef1(=+;BV(TaegL{};yvwBec@92v->p{9XUO8t~DG2S{a{|_4L)>dN1}J5eI5HqinHj5(1&CeX^SBouf9jKFoU@P zqP1fEH1w?Ah$C&!3jWklI*L}S|5w*_f0?%zss}`(qu3NxP%>dmpXfsyDc;=O2PbTO z!K(2%mR48e@x1ER;nSR&Z0bA1o;ZL(2%=_Ns|8~Tpxx7`n-cYN(*PcifUE)Dt^S@bGrSZ{dgl^)!QGAe(DY;B1XR2-U4*YxZeoRSKAMY zh2Bv4Dm3UUUv2mLf3Gyu4bNBGU-o%3UnOsvAxar|anhw1-f)-oT~U(x>Kn=VDz_Q1 zz$9U%+9gXpJj8Sk8wm%%jUnnig9~?X{Mw^=0zH8J)3D-TGRSSO0 zu`24q+y1cLTmJ|G%eUQ|p&NK|`@_xu*1+<7+SSdPirD{uwm;Cioo|1r!4xvKd3k3h zL!z8AI49IlGaJ15e{J+i`0DZ1PC`9ozS^nQjXx8%I=G;^aEplEi9qg0O`k*pP~u#T z^$@+fi_H%U5arUWn$zl8#Wae=AkB+u9n&najA?iANeflTXS{q8Z>ZIA zv&Emg3oagu+9CVR!GqlGH-aX^<^t~a3(+uR zM-Zu$C`z5_&3`NBj&O6qb+`#dUxSQEpJmzjfuR&PLP4oB@ABPX6aGRs*bl#ToO13k z8LeJA9gOe~a`{<43h+fHoTecoj(^w-y&2vMZDRaqX8k66Aq3w%XJj|Eu##?kXPT?yzNtxWik??!b-OWOcoW)@_QShnM0jx*7GbqaYa3zVcjf`qC@Ox z*6cHbj=P@7o~(P}4Kr^2mDu_9WIHoEWr6pNl*zdhe{`mQZT92%Fw)m;F>qnvcsq@+ zNCLiXHp(yoe_LHP{t~l@AOXjZJ@HnND901V+pzy>9$2d>?=GiO${%LFN-sAx_?>9* zP8tn{p)5?X-8=vEj!M8^!e8Na>Xl}~czX>vvTb;!p>B8rzTL}z-eJq=`x3Z@B;eaI z|E&V$JIOUR14M-Rc@g|Ja}ltT1e`4+?SwIfrA!##F3jFDnW16lMFzN-`-rmgE^^$p zI_%b0R^bU+)!s&jkR)S}Gk}xDCpRr`n$2U?IhZ;o%ncgqj!*kg+wm7Z%wb|zJOm0q zI}0LR)x#XZ#All3!A^PZ3NWP1-FU;^JBRvmH{&mqJBe}`)z+}z&4a--U5-Hz0AnaY z-0OotrQ^h2B(lQ&jFXtcdP1uC8674vt0xUv^OpI&T~Uiar>eHYyY<6~Fu%9AL}z~) z`2mh7`UmSJG09{;nNUv08|v;lY{9t)fBO|KIZ3D|JmbXVMR}h9PAL^rlE#a)k1T1N z;M>-SC(5}`)4G^YUJ)&)9-5Fu`!(V2dW(=&+Y#S~rT79?8k*X~}i947eoW@EWDkV>ha5+=qw3-cc^DC0golg8WLi2c0o zcWDF|Z>48E5|U^y-avYmxBjaMX}q2Bi$OxVHYa{@HpcT@F55X}ViBJ^yTDR}mV3e7 zhf5BL2R^qi8_Ol*DWztn&z-%~l18)bXv9dxeMZc7if1+w(rgFthS|=3$RecKcE#ZU zYU+D~6W^bW*^)1e&`y^h=5&A-%qkPlI*sWZ*TO{8{l>jmOgG1IFV4nv=`fX2LnTeT z?wl`L5^27jjjLe3?}_$oB>(@`NX>G%V2R1rXwA55$$X$^|2r32uy!yVB1&ukaT+vWnRZ~&z?NUqI|AN0U zt-R;QmY$X9wa(`=fmQ25hw)TYLCJ(MouUtIq&lNL_=FptTW#@=ga4WC*PMiU+%nG9 zeJz_R&v@+Z0U-$K8PEN?1!M`p-RKdZ-8A`UhcM##H)hc9sto$C|7RKWdFcP<0fWk* z3(vz(PC0Rahj1k4|LI)#=pN+S}}xw(Z6oEZ(XokuhzaH#U&(g-g=y>LURYMb&yE}Em6s$js|p+ydn8QZ~p6D*=7-q zOrfYKzFxrSkq5Kz|CP=smrI$8Mecw3DtbxrO82u)q8}>&P5-mH>1S2}8l{yKU)0~? z&#rcN9Nk`b@<&p9KzFs7@kd6mkd@0?*4=C*Bq;{-V3UQB6kqfi3(h@w;9DPDYm-nH zt|LJ?q(knm2N6V>)CLE8yNe{bsfknt3_B-ya zavuSD*jtOU@N4u8nd#&8uz$^>&6z!HT|-@MYX#5cRMxDUJS}Z26&=O<_SIT?@`<4OFZSlYbevm) z`R8jmsXj`Y*(upIvp#eNrKt3<7hecP8L!_>j@Rsd5MM67+vuO13-L$i)wQp)pECNV zi@yyn>|y_rrhlS)lF_ST(tr4YpZmfiRHQqaQ%p@S;urJ?I%^b&`^yF5KSChxZ!{sQ zX|#7PSpAAD#ppcmVR*bHB%F zG{rk42{^#~mrnIwa*fHX?g7q!lhwcI3t(kTqgCl7V4|4>C}E{ohW4IaumUTP){_j# z_z>Qx#e+*s=1wzmf;RGjK44ksh}h# z1}8J<<-sWY{}ksFi={|53yOUXVi^kFS6-@G48GCdFOV|-BhcPg*VS`E?G$%*#JwF~ z<7)5Q)h_Y2Os>8o{$-#dV_|;j;uPkq@Zp}MVVTCfE@-QIRGUoMPK z#wWD9)SLeekgvuihxcgS_bA@$+Saw;IqaREwju9Tpdw%4eOXBg@HMz6^ghk|KFvFm zhyRi{-Oc3Pp2GXo<09@|n)h9rH|Cfds_HvGhD+WnlXyRo!uvE_)yX)F%DhwYu5Mdb zQCU@gtcTp{`CkL~<=*^nVDGg!4Z^j0@V;5|uB)u6X>a$&=jo3R6E~M;Oy1u9&w%&o z(TIDm=6$c?T~UV&sH(x+Y!S|TweXfc>vHaYX9RpdIRZ)obNS!JXet7#Ipu`=fSgr8 z0`q|G*E0)JR1N`7_3O+e;iy-vwD>c9J}Z9Ean$3NF;jI=r_Y)3*H^=;%aHXgz6>7- z+L`3g#M6p5F?stRwa)jT!~e9~;v@#L!#~k+TXa(O2{4sXMI}y>jw{z%W?&+{yKxP2 z#qU!l66rP3aw?+QB6D&-$ z-)G!)PGXSQZ{5eVUx2BUDw_7Y>P|}}?RQq=sj%M<#D1rG_9G$f#|#m5ebvt_LfY?~ z_@!7HB^|cNUz&~muu0gv$Xr_vil((3nWx(#f2hquL(6^EZIy}nxZ}2FW4RNsvmmZ5=NVy-FO;o_CvARX`anUNSkqwJqWMvViD42d*YX28J~2`PW-ZLZ06k= ztqqnRHmeNI!BPfsNS<=F?f)iO|4u-t=9y0`&XJam@)L;ptAk1RqzPFq& zfx@aM9PEEI^q|EaaNA`b9zNEdjm47Slu|F#$FB9pzglBy%?^^ zao1;KxHOPTsh<)?JD{cxbN`F&apyKBV7%wWcnQyVBqY&3A$-sxr18#=++?HV0X+YO`>wiIy`k;%A-D=qkYd>dXW4d&JN~xZbCbqlIJO2|n z-g%AKGUNV;Hb95k?AeZlB-)ENjDDSW{wHXy7sRi^;E!}zal9%U+v!^NpU$97SA1;K z5lf54bN$&?UDE3waKes5_y~bmE7yv0S=GBxv)2OWHmaAc8)Xkk$*VuUN2Xz|V8- zM`a_}?%9!z0mERGQc)#Q{O`somQ)753mTCFxjzvruJf!&LJ~ity7C;%UVjHG3oWe;Jr)!Y}atO)nwd zn2ib3Vk)JIWr*vkbKEf8e2b3-K+kTw17Z zhOerrU*B5Q*7-aDI%6&#_>cME=#w%$Io|j;kn^8-C>{rTkh56Jslrt!we8-~r1axR z$}t~_@eNzfEu8hgQ#epnp3i#02j*P3KnI3n4;4ocq-l!d>h9{=CbVbmrB$(MGWV~AnMi)DWV=7 z8Bvo0n88`3L{+v`x8T`@olhm$kEw7!5B3xx-f%1DzwBE2QaAlXZPe0oEvCM`s-mva z+tC;54GU*2QA<`zo!^KIQRlX4DRRDy?ZKP|fI110gG-XC>O1%Dpe<5Nyu()Kw!2cq z{8v}~bFaxo+VN)l;z zdihUKWseu=$J#^MijO5NconO9{l#v9Bl zUDk;Jsj9oapJKcxA`zG0qN+~MTu4eXqoi#BAM%n(*1LKBgOjWuBu}zR1HAiGffEdm zzQucg)tJM@y}#MyS3#U)&Qi+UM>Rl6Oz(c$lEuX8qDD-ZxIdG`s?AHRNN6NhN4)wE z21fhh*E){Mqmfu$n|)&CUD=NTS;mQ#nC!^amKsc&p5%#Wwe12X+myY@JT#@$$~4&# zZ~uERBHG)C858&CVzPG6WF(}?et|c9`5tfon~grRIEytptj=dP z<6I2pc=dA>^UP?>T@J04tqoXk^m4!qb~)TT*s2WPTm4@yZUa^mf>+3VYB;n*}jKe4qwlvva{OdKp@60hx@Spt$IogWC?%_;|}}LtWYQ_ zJM1&Fxv6Le=;1K;asJmqq|gidh#zFk41E{gsMwzPP%o}%=kR|1`+j3A>_>>kS+&Ty z93RHQ{->fq$HB~pYA~a4sQ)Us@hA-@Y>SDk2 zR`@r0(+p9{z-tiR5^+Dp{m+cJA0|g!jur{?2f|8KBv^q_A-Zf>pi)>K2{psEhGP}JLJM_N_FIQSJ_ zbg$-a5FX^B`%D=NSJ3spU}WY8P|rYOOvvhMHLg7)ou^ zN`0=yV$Un+H;zOsqfRTFkf{pp#oJBl@_~&b&ueo`nZR$%9Y-sS%T&=+XJpBUS1TQ0tcxV&Kz|&CX zZS?feptCIEq1%#5L*4K!;vrf81aD>$-o{8ix+Z~FXy7L5??bKaBl9^T1Z0=sJf|{yy~C$M9il@ORN5FO3E==KJCx zlwq{aQ_&h4bQZ0@$oOxThPvU=`ip1zr5UZsJ0x0v@d!#XTIVI#*fbEq&|iE)SVdK| zt{|gAe8nC%-H9#{wj5up6J2zEuz94|WaBhDd6B_xvm9O7>A1}WDmJvWRAUyU#@nTu zHa5g?prBNp=u&g$AOA=fp~=M9BeMPpcVYzKcIwKWPC_|RJkm2SS!5RNU+)B_9C6)8 zrTs$|+dpMhQ5M_(Lbm@r7>56^a&jP;LKowWI(Ss_UkK(nWgj|?X7_VW;siDwwEoZO z)}L7&XqcM!1;YO?RvmK@`QrFt$5B@c;K|U3OR}irU>%Mof&YfAzT>N4^_@VZM!zKc ze??h7EcFuXEKaDz*u`CRc^?~0DU~v(I=}QsOL5K)?`y;!C-)b$EVj+8kCsylO*kHZ z`vu;pkuMLk2$@NIk6WwNHKv&Ayt7z=%;b{hEVNnzM9Ze3_(!G8K z>!0&bUJr&Y#7|C-s;`X0htWBG<$7>o_gavqbE0#?q+gNqpD4o|ol!rt<6K|)t3WIW zf%w&4CPW(0S=gS>O`itDuht2~qBIaYG{moR{(~7nk;(vEc5(kF8NgS6Y-rH)9td^5 zaL%T;@nLH4PtgGTL|MXrF(r`<;A=ep&OW_ZWdJVAuxrKZEd60#qyM#&l1f9}@C@K< z=lG?W0l457k^y|}UX-L$6(`r&i6DY6eC{I5XaGU&It@;08Xhwbm1b%&8Xlu{~WZhwDV?mrSq%nUAW#1>xnmlTD};EZUw z&J0M%;CnINfcTAit2St@197;f+Q5nn^{>yyTG&1XYt4=?k2l7bp<&QwsLc$nKO&lD zLG(Qf5smf~Cufz>W@B6KnDy4QxmUKbiJkF9w)9r7>7GC#PV!92U=}1T-b|Ns{ zH+NcuG~L1Y4T!p=!^wdgvN0VtHou#3|o8fNTv$0wdmQpHZy4$yYYYC&pE@{LT9QQY3u`@l3k&qUX&frP8|HJ4E zE{osnIOU{6XK-^i7Q;}HZFKcwF<9ZT4MrF;P<)d4PtfntV9&TakNp1nwDG|z9JXc5xGE|1@WTPaD0=6Opt_R?+PltoMD zH!b!1(4w&ZfD-nW^MxPw&3I+slElls5W+GwCopM4OP?`_0#y2kUg zH~*37v+PR@|di)9Gay;7=W8 zu<+fVTY6SP_g}gPWKRVK3kRYKN+yiy5Pj5L1?0U4Z`kn}+5ay6LmYFYTinNGFoG$a z$Fr&Ktfq8OS>}|^Ge<2&c|*^&oLkDHBS6trTG7mGZYtWL75!ce{4wZ+qVJ#vmpBd? z(-OPzM#c8Thk8ZuO%C_2eeYId7DKN(|Kq{e--W2Yc$eXgYWUtafR0l|CE>Fe-}{|d zs6@?TK!eU^F`n(2R2u4r&tg1#f?t}m801YeL@5KWL3sDV8%UqsfRdcWC`q2hkX7er zFBeuSfPfVkaTbH-&gH7J(jPpDkywMBYp(iBx>e`zk2dO};1QH_u0H?1mH0qisAqjZ z!|!|ZKRN<3w0z&aTi=Kq!NXPV{Wl6H^rPs!WSGTG>s zx-BhiXI0oEf7Zk!8c8>!OC+qa)J9DOv%t~(EPunn|~&Z zb(}+msn(5IRr3GgaEm|prd{jep{DBcY3xl~ggBRG(-rNWRaB!`iWIeec#|cTQSC4$ zm3*~UXB*W#wVo8U{%-N--n4`0B~{S8-x-fw6Xv)N+5iH#TpzWa9^hQrm>Rod46kc!Yb?&K!jRNbW0~ zQQ255_Xv5 z&qM!t_|e##Htf9eC-E1fkNwdIaA6u4q;W(mx& z)9!X?#ux15@gj@c@us$3{8Mb4W<VI?83Otzlo82ZL$49D^W$B?7{Wf4ZLbA(>g{K7?`JEVeK7?+@t+ky%A) z%$oUyBH$xmOLo5;Mqd?&_0`cAOg-FQl7%B+ufvYaS;9DmzOoV1x9;y~TTIt%iL;pg~e<7&TH#mm= zm3z=hsLOibdk3}eWx`nJmPf#aeS(ake`MJiwGfE!bcT&6EP&YG&riD%&Iw zf{<4B%Y3Usxh>*noXB_QNGscxT`TKDZ&1voBmQMO1Td_3O%Chqr4Vd>iT&SxH>R(- zz2n-AkK#+C8~Y{i|6xbmJxw=8|0JX5M3wG-|B%_fh+nb)BgF0@5PyaHAAE=rLUR^< z`i+xph`(AW5C^4!*s3A^O71@zFvvkbj~qrnxQ>Y`QdX|NyL0k04%;{_zHr5?lbcp7 z!^GXJdGlz2Iry)HT_VNF_MTsP>mOR=J?Ie^@`R1+Qf4KN*r@9k+bgQq*HlzalcoIh z7b6IFau$VtJxgq^xjzhrTHSdv1p|fVse{LvgadmhRZ&$@RohTiVJb8jVxi=(zif%z zi(y$Fb*4Iev6%SYuV0pm9TZZrFJS%8uh{%MNPfnmb1*+zig#peezBu_y2{qdy84RRPNy1UPO{@=5%BAr znyov|n9|m=t7XgX)=llZcTU;4YwM2oDLdNMZQePhrLAS#uJ#>MX1DKHw`bXsljrQ- z(z>f->y{YRoa6l;fgMkaJc;@)XxZ7?(E&B|iw=PO?M@%32v-_3o!GP#f9S&R_>Gzs z;E$16D1$NFthsR(P%llY3r12$_0^R%l@<7w))dG8t=#_wi?-t%w&YgTlF1L(738ud zZvjhK;J04>JA^w>l*l$EG7sM`rFn{r@gD*}&ZJYaxY>)`|H247Ou3mF)ZknF{d{}^ z;fqVe&4%&bp9o{kqDc{V0luxohYBE==o}EZK=k?4XH{I zh{#r&@{3&mWsDw{+zb|p>>pSzPT7my#q#7Aj3?!+Q)WJ{Oli#lyfn#T&p_Aa&d2l6 z=sK7Kn6EqROd?%~e@O+ZlR2K&FLC{oS`SIqS~|~{u>Z>sV398Yi_^66hrh>%(Rse~ zD!8!fVyO{=-pi_cxm5Iv`TpGQp6Ddhd;n%JPt3e$>gDwCdZq>O<+J8CEu#R{H?dMF zkRCyy`z)cXvn8BiQ!6s)nieUKsl@e7^%a#~SPJc$$efI&g}(e-i#cno-91U>(a<%W z^ohErYSlGSB*jVB^gGZ0wGr7h<)et|-BHTyntpda(6MG?AvOEEJB^wxs3CdRN4I|i zA4bjo?pxr(nk`S$C`Y3#jQqPli5<#Sqa>(;NQB5sBSj_pvAOskzms5$Ci=aX{{}>b zPK4cTw}L{)q9TR#YQOjNFZX-3QoUaqWpt^2_ynlmd;MQIMz!pausE;EAj`+6H2U9j z{7WO1CmTtE((gU}N2PjpWS5BY^dIH-eO*d)r9@UcCA#aYufb7VTnyFuPR3Nv{{N!$ zJES_lqRlP4S~s-s6sviQ8prJZ@|WWV9yGL~#qs6Ko0cv_XE@L)S{YxO)E^FaJNd?w z1~aCOW=P!dG($jZ#~aoC$Bp8Hqx|TyUPTvt?aYhhIJ#^Tp)dMEz5enHC6xW;@)&Rx zBXx92igFpfiz(P~?9+lN*v#67vZmE#zC!((bn1r30KDEkS)TEWF7xD1 z*kx8#NF7>T<|}glJEX2h)(1`aqDdGx;ft?e{?G3+2d&|^Yj2!CP6|lfy~_BZWtkEAeNF6r z!?DjnX_)Rc@BD+Zdr2I2AKnjVv?zX}(+`e_ANcC+^mqF1+R?IQXGi;%U5QOCJ9o`) z+qAr6b31L@4JF7?Eb`}WK0a8aL{ik=`CL38mxzv$y`MA#pKf6!{YpUo2LQ zH_-oiw~(q#1_-wnQekI zDsYcs$mz;KUh$k(?J!}ztzJkx+F6>qqH-;b+SpU`m@bme^!G2Oa%wlHOqT6 zdo3E)!I_~nQpImQYSI+43v3Cg;(<>{cJ0_RW8Rjwjuzj^edc5= zR{N_r|EZ(@Jz(y8ZV6h5vNQp)yjbn8-u$1YVX&ISB4*H+-u3Oi7WO#G#QNi@mMl8a z=^wP}KArZi-BH}=UEe;_;`*@372R~utoyK8lXx=Tg8uFME#eBGAB`ebvzF%eHqQUW zby|h+QPKiACybSiAOoDSCi;8`U~NS?;553dcluc@o-kSTcE%<_C9whZon;m=yQ~=~ zqGnMjjv5KncP_Mw66(xxPGW@aUB@LuRn^*G$SDzi_RbR)3qti-^mfK2K_#&X^?xm5 zLOlUR@=Tu)IsYT~U!&8YRquhE<3@^4^u|3#zP}DPYj3m$`9JG@Iq0So@pLcY>4I#f zJmwLRphk#p_ldmTb#jDmF(kjc*Sk91?ap<0fzW%Xt@f|_$4zN-T!q@pJp}c;LHqYQ z*UjNM?sSd%U2py)=v>#SjA*3FasAE}32#d0Ivgy>rb$@c;TxbGdlT4Tj`OTG!Aui} zch>t|G*ZR?=w;Fry3nUJq>BG>yjfIru6{QW z9?-=Y|JY*@?*q2<#H#=EdV>W}4ngl9cK>%+h}AW@RqUX`JJ*Hn+gdjH!OCY&#$vU9 z_OXyL()pva5(Fz1g$an|#cKcb&i@3R>y+f~NfiI5xBjcWEdusir^}O1dOOpDK&r6y zfeaFk#T$zM(_8-uhTYp-SLO`5I<70Blw>4fG$ENc{}}|OHms51h)sSUW8lqSf>ghk z#3x}R?d@Ncrge(wQ+4I?EUy7hdC+BXi)6H;je)&4Sdd&S@ULAai(*#2&!V?eo@4o4IIUrOAng==~OsEA;-sYWW5G$=hdcpdhHU3pt zeIC}J(yA@N`rcX#0I}A9zm>FjaxOEA7CaBEL(1gh=iW|Pl1)klpeN@ki})zy#8||% ziKKS#@%{%tIV#$2%9-etnKtQc;)L9d)b9N`CfDL8Os>a7|Gqc>sTxF4b2{`g`St!6 zOgV*zA*X7%Jh5fCGu%XvGemxR?fD?Nh`rzcr^%_f(&W_J8J=YCBp>{s)+8?6o6ZO8 zAqy?8kT3HLJ;Abx88V_fY;Wti(_sKJQ~_ml=`t;Z-T%8H%wI zwe`Q;E?SWK_Y03hhRC^>aRkS9t|0K=O3Fn*_nNJ}baiIp1`o+3H$> z3C_sCUxs#;^HD7_6r237RsSmI^O;(?yJZt`o{vUYh>k$H2J$z9i^%yXsDFT0IbRlX zt}x}OoG(RAwKmd6U$TmB2Nqw>0NY6UpkURixK45ps@?v}O%}&8Nfh&cMi4D*4mU~E zvD^QW%;b#Gm~jE0AxGtW+x_0Y3wwK&i?B9G;Za%RU**J|QR!-qMoLUs%7@pdujXi^ z#wuC~oigaFMroZQ+f8B>BmB<41TjKs#OBZP+_H_ji+tLmd07OO(gla;POGqGQ~Qox zJN;&*x?{2-3ti{Y$X_frZ)b*E>=U)dzbZ_PC4aLS=tZ|^bUEa3(e1hP%TLkhZe#f= zas*tQi}p-z#0L|2Ol48&=xyMF<)?we(=0y?k}}`W8VwvC!Prq%+8LP1B@Vt=%ER=k z#sqHb?Ikf0m&f|2xO5vtL|oov5f}5tZNS(zjkvs1L|n?&22b2MB5p*uc6l=X9XP@l z*MYFg&X_MlyS(p+xDjF6y&~e$ZIa-y8wStsiZV!z2CDrIC7fa!6HyZmg}7r>cQh=@ zI%09$f8wtpmU2^ub5i5NT~Z*1PZmm~@Eq$zy5Y;Bf^Lp5Wm&Os{xyDi>)jJQ6$AdqdJ@AFj1p0! zxIkUPS)!SF5l$r)M*16u@ruj*7g&155vT~NFw!qgaq0h!;_*m?MlOu>HHydZ7wBCe z`M*)T(Sgs?qGf3;TJW+Do)%qgL&cY*v;NBF1Iw!&)P*Tv_@hMnZ^e*~|PXWClB3y5o{{&#@t-!Fq;LFgv znEua3G%iH%V&;F;dqNt$^#QABafg7KK)nl<@A;>_G|&a3~DzP+cY`Diq)Bmxc-` z9jXEbTiTuzsDk24Q}AV~!1zZMCWNSft-xA^NoiDwff=fj^6kt6}Nuz}LU&-4dY6_Jo+zU~i z`(GEyf5FrvIEMdqVgFA%O<}u(h)}(|$oxM_LT?2ojsJD|l;}MrMDH$7iQW}y^p>dH zY@TH=jhHe21AH{ zcRkx@PuDmSZlbRD_$BEY#61J0`7->w>(eMj*Qf|FQCH?abd9QT6P1>Vs47*{&M{0> zIz6R$XC^9@{ck`(KO*-|O;pPKk6u)zyod^iw4Ty$`z7f`$pP)hMmP1NmizdRdSbGV7RG5*n*HL9TzcN!CQ z?}kz|QB8=6x^w-5CaOzgqA?JUDA}F;KSk9k6S)wzQ8%=11;(OJ0ZNm?qq;1G>P+3< zx*@Icrwx*+y!=0qwMjR-ZfJHXRMPmmzw67Q*<7Fq@vI)h{F3x6>arQ7`7#<`kJ%_i zv(<%|tw%cw&}= z2~*~EUjmh(gz+fNm!VA0UZPAxh%!B^M49PI8O?zIGyjv3X3v!(YP!>oPMM?rp?&3% zGQFL?NwalK0!TOoZ&;$|^%n6y5RWgGUHE;R;wbk??`q$PodasZn$)C}aca-!EoxR- z%3N?S5%d9yvfidAt13gBUPFx`PGmW#oPWMt`QP|3#;Ls;!G)E(Bu(W?^4e>?fLg-J zEg&e#9J-)#(o~l>w3Y2$bf_sMlWX|ETPeWMw69@7)Cj zs7G^}dS$PSs_8B3pD42l`R89zH6x?kYB(GRsG_4|{`A8j{vIr)&|G98hVPB>zrxa- zj71FJ8|$C`*fI+J9p{lxzxOCU48!++OaQm10USeF0+#jff$cn#rVKw4&qLOe7Q0b~ zuNdCan)=B8AC&j3&cQ&G#qeS&(VKH8+XnH7VTReb8z#;j+V2Jo4RdB$5}1A+NYk?vSaG{t()4~ckG5Vy84F7O|561eb$zW%AE~-m`O#&8C#|awQA39qmFAd@q|)5Wyzy_FiWO|`6WH8Q0w=3On|lMk2D7rGzHmh50#Z-(g2gZq zhhnAa;|7^XefBMp`fN+CPr4jf^qpY%q*HC;|#Dt|-F)1k~Xal?HetX@5lP3gUHnIjbacw)%gEDj%7@ zqsJAE_&ZAN-g4%at!Kl}@KBk>JNLA1*xa&Z%BHQYSg>{+;yDg)Vv_mqff`dUjY*#% z9_|$7r3DgELIGWI5#R0ik)<7iGPTp3%A6v@ccBN6lbob$@dl9oC-~&7N*YI3WfQ8h zRIhs|x)vQ9m9k3$y_bw04HL>3o`z8&Q=25BR(2>z>b(Dh5XL&+5mM*6OTc?V01vnz z1@N>HUCy=v58(U<0iWLqV9tLs1fA~#9$i?f0GC>T$72!A06YpsjDBgLm;at00vO}e zpkLA!p8N9pYi+;-IsZq%7li{J=#Bp_@&S)2VqhGOsBc>szlU0QAoE{Z_@Xet=>PnV z6dSPcT31*9O&jo_d;z>K4PcqR88l8TyiawcN6{ssBaMNPf3%w=r5hdO<$wEBH+mG` zgmj~W_KF7kk~NTsI*8+6TJx+l;3j}En&BXh|5^QKscw|u{BE=d3>pan0!=_FY5#+~ z_OB_QM+5c~)iNZi1|i$A-**YP!EFBow?7TI7{q{N@HzpvpO!-a!CtPiPEwG;*ZJkQ zs`8<>8~k;YWbN-ywq0CA|3+9T?Ud*SB4?*E2HV>U4E4%+M+f$+#2oAe>DaQ1zv>s6 z)!_|YhIsj(X2`%>@2$=b#=@h)&u0fR$|MId`jA~FEVZY^=Te4*(&8fIB%LiF9x=(4 zR>z@QBN2Q(-szj9l-}7wmW4Za?ZB8pE5^d)ClPOGk;RjWHF*2@hhU4k1<7LxXmlsC z5Y2`zG}$Q4d^Tn3LNTvD@2lL2W$lj!`ILgu0y)iKTBj`9(YmaCM_W5MP>9s~@Hu9^ ztA+pLy;DMEIN}qy&nQ*z1Yy+s@PktCSA?5!_yZ#13Kd3<9LtQuf9aRsDl?2BW{l;b zBwgW(WHYkoKn=!52`k;|DcU#Vlq568kOieZ=R&Vb|F|Ew)n*UhY?E=g z$}(FgeAV`K@|5a`{nX^E_IvnzGpLU8GLW+wUKw5;<@F-sO6AoqY_g>@uP)!ze|)e#x1tyXTuKd_2ATg9s*rKpY)%WU*@z7TLKovE4l_q;9JcJG>_ z4kxKfTj>42!CC)r9)3>MyBKtc?hzrM2-Ef8#Q1tXp6#&RV>Uz}AOf>4an9n7H z05F>4A6okAG?tEOprbkeLwCAbSvm%VCv-NmG`hntPg|2qh^wXWu>XQ`PMX;rF{j^tNF@7;VwhC!wjjPReqz&X z{j{0#axftvG5MHz7N|ptdrkaM4L59VoQIJ_DtFiKT8^(Cpy2#n$S zm*2RHUKZ)yNIu-@t9lY&6;DCkOv#B{1jty z*q_>^vZkuKwr1M22DUr0muz9&QcJ{k(|}W?Do`4)P9*>=YlcjvX2vo8`8A_9S!d*7 zQZwTi|5-EFhu6&b?jquP)u{KvJe`i!#(VQW*Q;E0FTjR0>hbLVSu@us*Nnb5Dmwmh zQ@cWn_Qz@!sbe*Bd$($*=HLb})V)z$;)5N>|G{E?7&TJkZA%`3BMU*>LXFsrK~3|K zk<`M3N|Tw|i0v~g!~R8!nUIrouEsK9n@O(J^apQ~QwDFC6O=9CDLS%cljn5d`(2AK zb*%NyI>d0+!GdM71w@W`$VN1r*vDk0H1t{7p$O&@a+3DaNG8(%S<`1F*R+hRChjyw zR?ZPnIgz=4z7ZcRAl^Fh7I2}r-k8Q)#bqXP{KE!uBZq=WHzH2HnxU)?W*zroBHbLckz?+{i6e}!H1#5q_v{N z&1sa7@$961qQuQkJCx`JCGxx(*ddt9RlOS7n{(k9N?_ct#4D60VcBlI!8SWFBvxk3zcbBJLkFEI=gQC-cS<8M9+4u)YS zpCyL5RT+jm)AWnAKOS#rG}+t#a4XjgRqYn)JF9vEoL`BoOd;|h{#YR1nha5VYx27y z{gE`)Qmds;8GthMtw)q^@d2yaw}#q_6nXsBH*OTfWFSbUu>RQ;9!W+*TR_Jt7n;WU zCI0AazwWt__%Mt$U-seJH1Ow8#cl(+{n$O%V>Bp%_%if zlQ3&UO}b=KBi@`z&0QwBnsnLeHp*QVm~?40JQx)r9$X3eeB;*|tbYX!O`Qb_Wq>$g zFqozp)W%E(h0SIY7?RuslBPAtf`cP)tv$nHv)%Og!O0$<4;pkx0)`yKnzf%dxu|K4 zjI1gBe(hgC&d78$O=OY@q`ICW;wVQzYA1YEk4Ov{(oG=MdE=j>9RH{t_95e+I&b`Q zG�{kgC>s;~&-dl;G+f_=@dpIl?w(OpGQclNw^F@YC+R00A*$6?HQp zJo5J7{mFuN5BT#6GSSXlpU{nf+=)PPl857#mJ6IkbC zO=>iZwM4|t`kkUG$oH$xBtUNhGbS|}xc=>D0_w`WjQmb&rs10)XU#m7re>r`H~d~i zJf&)8G0Zc2KqLkn>6-L$-u%~7s$S?sA@y<`_kXiqo=UD4Nqdhw&C-p6n+AX%O*K4! zKvKP6Zy9fa(%leBsI*tY=W(C0IJLn1?sz9LlHN7g8E;o`5I!l)2%pC>{sp#v!sJrs zj8AFfBxfrSb*4|W$f?epfs(WhcLv%Z@Bny;J%mG?1Ij$b@ofV1X@8!gM2$MCgHeYuPjTf-oTtEeBzd0V%Cl1DDUPTA zQ;Da;lz7Qh;$157)Qp=C+Dgpe_@7EVlSTcVwh(_ z3^U_jhGCQ-r+4*Kh76;2PZ%bKc~%*wC%)A$h98-yIH6Gt^Q=El(KVtN_Jn4BC8o|( zoN%5%d^Q=P1h*6J5a}7Ar`ihOIHA`ir5AWdN=>z`iq zLNXHC0#as0;Q<_z~zKn$eY9XbadO?u~!6yU>h2r!uFDt+9}k zRHl34zwv>1(>>i)O*Q40giZGxnyI{v(>;e~V!Ed!?R3wf>R`G@gf*U&BCJQ0`ZVQt z2^BUg5H<_Kx}+C2Ef6Lxw$a=F{9;IpZAAQInx*Hb-^WkR(Fbq-E-G?L}2; z#`${>?q4lAY?~!s+Z=+n>B|Xzih2#xfZH_9Vnet-o8D-cJ{tv>t-yfWDS-^ha=EJ-(*uF>NSZgaf;|FvY@r2n7e_5UBFalaZM zpmeYQ|A5W2FHa<;JxtM_?l47%=7HzB=yv!4JLSGdz{5Y{u+S+oW*#Cj`odN}Xk9(u z`mwuHYM+-$caDKVCA^<_fhG81EjZoj11-2nNIl$Nx5$Y*59y%lB^Nry25#NCf*XQ$tg^Eb zEf}Co-18<`ISoKS-8G(ZbxyfN%O&s+Zw?2WrV z@B|zbTj+E(a2Mnj_epD0a6_TdZ2uhK2Dpn#v0S2Ymm0Y7 zV=mj}eD41W1k>ISoKS+xoKgd~77S1(?)hH+7vL`LhK)NKcQ*rfQI2u{AO$xBCzRka zr<;LW3kE0?_sQP+M}WJeyTb=_&`-J>xQla(`{aWuxFI;91eZD84cuBVKv}rG`M&^n zmmZjo*0_5ZxJz=2d%=tp+z^~lg3Fv925v1FpiJBgy!B9tOOA2> zEd@6O>qm~2IXw;BS}+p^Bc3d*vB((*OVOuy)%NaX;O?4R+zY+=4{b2EO9)OV!MOFy zz^w%{(J*m)^M8T8yWt)kb&c9Wr?-K-G`F}HwOa-wZU{~&!DUWw1Gg5;%Y6*oi#}zM z)85_t;3;I<-hB+*-Exb2G4p>r4nlB32`+Q`7`U}yfHH9}KFuO0?jA_()LohjoiYP= z_Z;K?sU?{9hG2b(XBjS&4f-7|nE9q*?=H*ojLG42m7!Nd*0`Z*6}cp^jKCt5H-nYfo7Z;=ysAFLUuRkelA00VdL z+~QvLB}*`IL$J4;XD(K1!QMhf2k>LewCr7roVd%dIM7Gid!T{4Pi}E9-)spcZU{~& z!DY@s!`@o3w_+inPb_lc9x&8#`fJ=n4cz^6i~E!XmSEzB;Di!f z<_tA(Yrz0z;y&dni=4OzVnJqr#y!lyJs`KZR~1--i5r3wN^qGo%)qS$bCt;OlU1`V za^fC@Yt#m6+`|pr19Ob~aZ509LvTU~F2g0}LHyQ&0m{PtfkjT-gJamBpmE0x+=Ft9 zd-ZNhFmXe0LJ2OzW#$2HEf}Co+^Zk8$ccMMIc7OE?s5Y+xaM+Pyk@c`n7AQ0p# zoKS-8WAIuqK$*BtyTKwS?$~H-8PT{$8@Pw(7Wdk2mSEzB;Di!fh9{o{?NSQ{C=>VE zlPz-ME+6AKF^zkSfjgFC+)r78i5r6TLn`g#j#@B4nYd5Sx5$Zm#8}5E*SNzHSTc+?h(1gov5`06E_6w$Cj1hnPY*U zXu$ww;!fagR4}kIXUdH!Q)#4Z(UFmAwH<3kE0?ck?8RoVZ6%z(w^M z_XGp?sNCXi{# zi@WupC78G&SU-cj%$Z`~)`9`b!u@TFoVX_-Js+=ePc?9l&n@n@8J1w;hTwz}T;@zQ zaBIOFGw{v%Hq1k`oey#@OMjGReV1blmN_CKKH}!pzDKa>66a-N_%hw*)i$1g=7i-( z+`Nj?SPfPGSYB^E=A1D3V*i6$SpO(_@p+>z&^}ODd6UIaCsK&7it%2QUuf<}pO6Eq zQYuXYXze`zZ_m0;tZ)(&Se<1~g<(}CI7(C`?GW-GS>&`>$ZGb)9N?xqdB{*7=gq!<+1GQ=Ikk#xdxy8K!=bsJ1j2csG zoWvB3yT-t+1gjlUhP^j@(IO}Akk#y|xy9Y#?f(w!T~X^KrfS@^25u!d+SRsqhj;%= z;3pxg*%dj)?d|^$a97qji3*Lo&cLk%tB0By_TD%m#oi&S*_FA)z43NSF#V)znv*;^y@6W^j&`^G5NfT)J>9^q1V@><81~-$HH)0MLsqlv za*KOQwI!IirycJk>NM`-4ctnwditq>d&>@soVY_)v!~@4_v@Bm;;zTDu%~I>$G_3A_3Y_6MxHwUJ>JC~b<+j8c}JZRtq%Mf$ai@69|bWy zWJUY<+#=s`X^PEf@Sykcg4{gltwcu$z}_Zu@BWtndB~dfjNBsM>Fs|Hva1u^Sx#bx zAfIJ;ixRDl@fpZ>UTmpOKMz^eJ|V}*-?s$Q&u6-`ox}-(e71pHiH;7kZNBSxivf{` ztZUEAE%IGovIG;k>oz%wnS#8@K(0hb2PYx_yTyRWLsquk93ig)0K}Hv-ubViXX-cP^ke3iP=KbR|2==_dpF+1~l@AiRVuanH&v^s~MBufSnu zyYm2jmO!6pKv%M(!;_$=_WxL&O}ENDJGao!N$vle+>-!(wm?70fUacYcgyPM*z2Ec zrXdU6O}T}>r=Qix=<9Rb`GDRe(B~V_mF#G_4SmllivyvDtaZ=HG4$sx!GwOIdorNU z5$Goy(3R}y2pjsjcK$=?An;HF69xJL1GHA z+}uJx&%6I4Xr}Yrg@8U+pf5C_E7{ReHuUqn`%ePsA&cJgatr+{{kfPRudUu-~EvZG^?pnqg>AoP%>@A|Ly1^N;Lx{@6oYeT;vb^IT)`h9Y4p!GU^o0U_xdC0tj!sB|ZqI)ZddM>PqTE8?+m-@-vAY7$7YXzg26QDmI?;x{ z*Sr5E2#z5u;fr$%ec$jD=u6y{fWBCuuQZ@5+0jWh^nEQB2Ummq1*G{beN^M zmLcIT5$LBF(3R}yWE=X$LsFoJtcNenE%b|Tv;-6SGItfAFBRyk4CqRBbV?F*o`1KS zX~>fJvfM)7zt|E?=*!*JfWAziuQs47*-_4Uai}4y;`=%Ok5zFDwjaP>gBQP-Rq_2_ z)>2UBZMDvw$5rtPmfbA`EXnaY2A%t5{|kQIDy!mo{JWH7sb0F|0nY#6xHRKoeram_TkW0>=&J~KqJFiO;^yK)L7q&>gI=9d-OO1bP+ytPn7U&5B zx{{s8b#k^8@%77+*FW=k_&VLG?=`uFetGKp=c#Tppsx|=%?5NOI}f-_=$9`@iH;#l z<)`Kt`gbhB>~&6aTLAr3f!<<3SF-cE+0d`>?!WXhth}(*^3!q){fd1l(AT=_0R1$9 zzRrNIWan|g+;EsH?DgLx=q*AP%-7}?`jz(lH?4lU+Y0Dw1$wIiUCGYtVMD+2LQ8o< z4_PxmJ;%`hVF@PmgxiMq=>omYfUaccaoOCk`c>ZZZvux2SvF7P7W!3ROo86)wgY-X zptl>)mFzqwFedbOEe^DL$jW(hZlNFAYzd~-Tio@4-Yn498_<>PyxunSL*D)Gfz?A6 z&s%Z}{pxWk(AT+V0D6l+Kf{2oWass9=o<~_N_Jj9fi8oQYi|b!-hK5!@?eBIB6+{|>&D$z&K3Bh^Y_hp zEk>&0kH|pa9_Ichj-B#C z;2u8HgnK2N(+2m3q1Dy zT90V*>m~o!c;THmK8<8m4H$26x|RlbZRi{U@fG>FTEqZ8vSN!?SLg0xoP4eU)c%NT6TrY-4ys`CFEXMtwPHsOlH z^-V@5J@_s6iO~LOgpL=XPm9q06sp3@z=(=3Jqr+DRM?RJKtLjqeybzW`-e!swOphR zPScp!0HN}&O(K1;pFcoXe3x_yb)l6%6rcbZcu1}gB!iQYumULML8FFn4KiMD-81&* z_%Led*8di`!_vT&{NYv^{|p-D2dydURLdCLx^CqU*Gxy)Ew*A-O8#(Lqu?GEg8Q~s z1Gml}&RyBF-o|~~WrDjr4Q`29w>^Y1j9KM={(u|Cy_i*;A9Q9&EkEA<&j4lQ54R7s z1Ttp9LdU-6k{vKA?Do@4a^;kRFk>Y7LuqM%w;=d268!kFwTMA_O8#)o?DFjMhdX5c zSDRc#(^Bsy#NkefF$+Y*H`EcaYwE4p|vs3biJNrTygJpS$OW#>%xU{44ho>8V&5qesa@iK8qrB?@Utl}T%(j^R!3pZRW~SgqhkVzc zEaIaeF7zS?N}Bn@-SYgCLF1TeD39;vdHfJ~yx;K$kC9XP1Kr)oAMRddN>uLdv+0d1 zM13|2E{P4ecgy_0U_i)lhMIcyz=cg>lJcyy`NI*J|3?`we<;$= zlPHQ5r^z3Vd`pl`N=7F6!;#;M&?#wzO8#(95z4S@nd0XULnA7_^ejMp3C|zyaYg!+ z5b5`{i1ey7jj0yqK}T`VL6KhN=MTl{@`oZTf8g>a8AuJ?^F2XQm5hWHKq*-NWQfv( z9+f}TR7CdUFGdaB+f(4yrhzN@!@V^Ex7H6@m3bxLGX^VrcK)y#VYkGLSy!jz5BGZc zUu_8Pdy)V7xOM(8;)ib%zlr<1g1bHqZi!j<{zY)t`}qUznD_FBlKi0OMZcDd{NcW} zmO#d=kf&fkAo@JokbR!y5BJ?+lB@h-SU16Yi_@(%z*`vn`2U!D5BR8x?tT2;1VW2Q zQBiCOMMP0IjU?8)3!x}dgI&`j*y#2GVpjx2>;+UnQ4qTV0v5!EVsC(A@9nj{zP|c@ zo|!Xy=kDE2c76T+KL6#jY?7T*o-;FN&YYP$O|{BTEA z076GtpuIVfcvojDD}D$}6 zX?A?{=>@9=R=>FCXp_uyv<`RMlh7JKh6ZoCQi}{Lry|@L~19 zh0K3a-2q9|y%J@i?n3&1y4eBjfvQF4wU+^BlGEO5(XISu?D{`CDf(r8@)r9D{+_`# zy^C@K+Oe^2ZH}Sd)oE`v)}VH1tN>v*)QS#0W@~(Ww9nVpW+XYyC^^SFH^#|LpYNugwytP_n;MWOSE+KCzsAp) z&s^vC`E}^??UVQUNBlmIhRiy<3-OOMZ6mC^5;m*}VbA-*o=12Ot#qWpHc=DLaL!!E znpHk{or?G=QZU|S;=cjP0ic-EL#;Ab;Nrtn7+>dfUKjEnaQVeF}d z4QoQ!Yre48fDu|H#rRSj<4bXj;-41Z0kL$a15)^>#cM795SQ7tq2+E=o?`|&uhsCYAo2tYjzY?wGm6>m$1^KlGy-htm4~$?*wubfC5bIvb zc16DJLagJ)vRJ?7vljc@{j}I;SPJ{x{fT8Cec8`-%l|_^ZrSIawql$sXRs1J4Iih>CCP(*739w?k_O z`hQog16%y=;2)T@7=~q#pM{S=8eatbZt_j$l3Fo(0#L9KTn&Gw)T7W7?MGyMTbh65VO5bMs|&y0YsPNzj;*SgWEtnb>}Xom`B3ISbH{NfXE$F?Pb$f(Az@Fv z@#}qN{(l4m(>5eNsF$2X+1BYL?U_U*C=sIl;}X%H)s@x74$JHem(F=Gnlyi66ONu6Umr|CQp3yveS(pGcMcUoe*GiW5~=?5?_^^u&Dr zA|22|02X9Zp;_I}{D0i;t=MZ0G^X1FSDX9D5%6y_sbS*Snv=$!Su=U;xKs9S95?ag zn#rsH!XiVd%=@pknYP|yGwp=>u$T(5)oAX2-6n1|4~-@Y{>d86{eRg-b7=jz^3BS5 zz^{K)T@J>eupH^~4;&wmGr^X#eb7kpE>DwK{pf+kHjCEzHjCy}{mdrLsZOSH_&$R@glWn7lS<1-Imj4T6)BXYJ)sf9Uz`9#9PsU6>_9%=0ehVLs#Q4oeLTF z5ay2un9m1u&vd;u=q1Y5q+xln+Cx7CXgZ)Jl(2~HQN=Yqypvs2Rpx0PjuXjaSwsm@ zzZ)v;FrT-hgonBQp&f&X5>Ds&66$0OL<#-HtCEaSg1DxKP5#$+O_P}_aJskH&WaNF zi|>=HFoA5O*F9|FKUM9m*t_>N=Vx#4boX7;ws_VhYivCo5tGJNuX`jf4g!6=H7h^P zqLb4-!2%>^s}Wm2a;#039+` z>La~0)JI;lc_>}uJUTf&5_Bc8rR$?@Y+|MB{wR{nX+CQ5zsjsFUxBW>m}_b}Iol@4 z*(yQK1Y1taHLyuf7O`zI0KWs?qc_<_TTQU#?CET4+oY)|kKD{?1M}TVrI^>c&ku zW%9%`OgkhG(RX>JEmG+l=hMmQmY{D!J6t~7CRX~UYKO~L+C^Kj9hN~~bx%`3P7f-% zl_=O#z~cUM-#aPda^oA?qGRXh86pzMa_tH;|JA-J560}{kZ~_$k=yC0Nljz;s}7vv z9k+)H12rpguq)1q>(C&vUn&QyAS5{06+V7C*p9%Tj)PtCueekb`%SSNY&`a(gO!&L z0$Ax_EAWO*R~{H=1pq4c!&T{efY`&?&2KS1nu`4hiwvdttej&rRk2^3X&{h~+k}MD z+*fk{OT4eB*e^!@wtF#h{$m$av0wUf9yjaX$_M5+XbB-XG73G8{V%>sITN5uT+a4E zBej60%CJ<~8igJ=``?r_75l|mG;5N_x&9e9Ne+EhNL4Y(DwF@xs7BbTrj{d7+^R!t zIVxN@BR<|E_Uo_X5kez{p$gA`nxc|wUKN=C=6Jw#YKm%J^`Wg+P605bFdG#YA^D)yMbh#)?}e5vrb2($U3b-{2O2WaW;dC(>QhA=lG!O=5PZ; ztr-L^%sQQuBpDK>km_*|Ux1mVnv z#1~75VV}VMA641sCL$HXK55qfrzHV_TBE2ZP5d`afnZW*u;YPo$9%ugEsKTTmCj)EyLKWXMa(;Q-^BFwh)Wnq4j{*MiOS_;fhwLk$@ z-1H=vyMq}pKefAV7yaJca$A3di(RI2ivE$Aw97~<_E5Y}!NFAWGNof5)YFJ2N56yIJb&6fpX zZ6ivt30|B6!rF^bfK_*C5(s%<1_*0e|MX{Pl0<~e-vQl%}9bHjKNo`^eP)-i z&s=BnpMLTFj>{I+s^xY`>U!O}ID7WFd8sk4Gx?txs?Sl83Y6x{YOv3C5zI4EV1AbI zFEP(djrrMQM5&pouAOEuKYMwoc$1i)T@f$OdM1Jt?fcnJ;w4$nR3r)J=h~qZJL$|6 zn4jx|0>nH!HRk7ri>TQOb9FPApF1y9yh+T@-5W1Y%+;weKle_&Br#KwWSE)%Cg#~G zFh9@y7ctLCf>}cN=h^=0KISOQ9z>m?+xAv1tp$XZP?i*i>dX+;nYxK@g(e=KdKxP6 z%+D{2Q_%okY6Cnk=YK#e8h~1zh0=Uk4gCcN{AhqVDGcxe$3GfiZW06R4rcJoFI0$9 zbCm(|AZl+txNHs2EQtU$RH8g-1k4L&{F_S;os)-eQncU~7R4nH3?=M?(tO!q)(e=q zsla?IU~W)gGz0$6@m%H|FPi-S4Ne0_G|mBcHJ3Sea<)sz-f#pZ;of+|5-$!DJvA}Y zrG3nHk?2o3lAPT40S}O>g&qui|Wgq7EWjHCh#EVV&Wf-Wd!C;`QDv! z9T>3Osjw83*VdFpit5tF!ta=QLEf!M#uv*;Gk?Y8zvF;h$2-fZ zQW>BaeoP+_b_4>@@aV|O!IguD4ON<*Cz1rwD^CW@M?^DXJQ(sBK~yP-NM|GZJwQi9 zS4L5$jg9Ci(4euux?7+us>pS{Go7mTEPJ5iotZwOwoYpxLIn-c^>Z#bP>P^=byh$U z(Oenh37u;N%_u=bIt$IK&j#p-W@Z#+0^QM|LHiFjI5*2XT^0_2=Jbrw&|#^%IXj5v zwc`Skh-PMtr@THTXhsMc(phL;WB$wU5=1jAiZW`;8H%dbb_hOEH?IZi->t#lig4}P z&Xf>w3Y3$pDM!zV6gD*J)7Awaa5MSr`r_F&C3+H>MrE?K{0`e3sc5_#aWA`-nb*6HBAx?{}Hzj%$?&Lhzt2tcvIjMDOHQh>Pw3wg@|6737xJ5Px7r7-ggFgM-fQR z_HZ_OA1XV@@eci8ZO?FxGAU02sJC|t$Q%KT^LTKQx)BTk)R^>*Z@7~)M@m@h?Lhx` z7(l&D4YG~-Sx!dOw-*L9nE?6|u$HcTTtBX*BXC62Sk-UXp~Hs_tr%K4D(yDc!pX*g zmRgeFerKD2G)!XLI&Xp8a0T28GQq96Te#nu7~ozFz`Ia`c(WsJqlujVB%x6O9N%N! zNlsNq+6bDQq%^6EG?pSXs&BSUT_c<);}bRTPO$wm96uHpF}5Z1Bc0;(M?p>r(hbt7 z1LX_Ccc%ud!8SJEJ6>+f1H$7ov2|z9Afyd}@ZHS;?rdWlSSHJEW75Un&6REfabU-2?{aYC8RZnNz>Sx!^SLq$HVZ zY&QN+OJj6CE5^g!pLJ0PbHT&lCpZ@*ofVzGUmVbq(fMqaX-6`?=zXg2M!S8#I#8B~ z-uKRTs&-=8evWs3`iQo6wj!e7WN^BO5=ifx_|JzlJI0gKpQk)PIzk|k&VuwotALt> zbX62()RYxP6{Mr_2}mCl1__{HsHG z{#93Jdxl9vDk>}bGu8HYgC-_bC{qUV56%2X>&T#~-diZwI%6PTs0Q-1x-LU9a<_t} z4^94CbFYRY-<4%xFa2VdGn$z7KV7x}h0gWXO8NzwtW}ys%8Lq%BSnSH22Ixq=Tq?s z`+igqux}LH$9cWD+(}3u+)3KjQuu-%&Dby3NDEpsGaxcCc-sJXhE1=qObgrEGJ<>@MOh9r%hWLQfef`^WQ}?sH=+`7c7Se7Sh>R%>Hj5%QaD!v9T~}gN)C13CNg_A^tM&YUyT}bzhx67Gy?< zWk_^XW!0bxBy?jHQC&x8Ko;c*uFtLt$Rw_7Vncyzjo=z8xJYN=`t11t9qZ`YD9Qxp z({!uE^-G{Eab51sajH}VgOvN6^l>re4qT%u0jWYKK~IMe3KBq{GyeCRHvwH6unis6g_RO2N`OxLQ-;=8m!Vxj%Ia(B32Ub%O?WNBbdLP#3IJ3J4$ghk}tmw6vgSo z&SLLmr)qnag&w#G9z$}PdbDKznR+l=3Fcp!_(vJUWpeah50i;}GDmXsu%rn(tX!rW z1t5P(krXF({c2?ZPd?PRnisL{C3r?EJZX&^!rZd!*IfVen=5nfmr%ZKR~b&f?vJ95 z8o!vTxGxw2NW}8>*Z@DKvX*-%I#s)qAtII&m9q6^<@HEq6*oD^F_}0QROBp<`s-l- zXL|H1-g>zwxx-Np`CqMXWKy0W`Uc~_WuXyB_rB$wBKL#?(JASh-*6{q8FIg2{>QgJ zW3X3=I^ZrUQ66pYo3(+m6Y#ea^JI=$?&M)%$7undmRVAio|XKEX&K-44#;49-{9Q= zM-i0@ydx91*&f-GwZQ%Mk^uW^3|o1s47u~ga$@*z9}JXbQ(EESX!(6alM}Qik=k-( z+OaF4nN0iD!kIAujQL%s0QV@2x0tIOs-Ysl$0`T9Y+Ai4-LB2*rxw&eFn)JkKnpgm zx9O|O1*7&=X=!1&TaD|x;P}tR^_tSawzl-{@4G=0zjx0I_HIdQeLoO|7~nsUB3S-@ zj1?^F0RN%V_8jpkW*fX=`tP~_hr#lLNrGjg&V8cJgDSwkFKJrqd(8ji2`MKZ+@0UQ zI=Lr4$i?XVAJO^26gvNa^$(v5e{vpPc*P>&B0B%T`44q|D7DT%G>Fa*DV^sf)%l0Z zY-vj8r)J*73bAzl;a<`Cp;S6!{u9?Z`zGq#>-MpaCFqR#kJ9;(Bsxn(^`r6sk2npW zo{yal+jui%H472d4Al*a3rfncd#0qxPKJnTC_}2D6-VfZ_X04a7>THUH1i)ViZRFy zG2WT?0?}hua-_r{q_bj>AI#;}QS_9J-Xf=JTcW|Zut>S5v@FL=2)SEb_g`%P zJ`g&}e3l7>`9_mXfwBdNNvb_W-YU$8CWk6b`j+=Csf!dfJ+Y({xn498NBNT(|FlV& z2^sIL5S(z7E7ETv>04WCVL8g5O#G*{7=gjyD!S|4#M*~vVr?Iz3`;-n8n7T81X8sqjZ#ys9yekQb+l-$^So+q67at+v>pDQC{5Yru|`hiV>&t^TQBDr}Jo% z4s6sh`Ts{non)DipTqh8^<7t;CDO!E{?Y@|=-eMoq4O_x{(tM=K<#92vtXof5uJZA z`Tylfbgm|ell6`CV`^L^Bj&(VEVN-P%42!ZnJa{)1ga#M`g9lk73juj}Rv!MLu z2IvUo<|xW^A{k%o2}j}$D8KC$C`%|Wb2(Z$+yk&2ojw%o0|u0Mo@$l|5iq}96A(l& zH^+F#*INRnLBNpC2J=#Yj$m$yqD)sCj7->m`#n&WU^Xy?svaN5gl%m4U^+V;0VYXm zX9m=xIDz$VlmGXd+buEPb@#f!suftIvta$(x2~EZv2M4C{AD6-P!%!(9kIlqHycXZ*wQ$j5YRjQ6s>D432COr*0g z{eDJ(j+o{}QAU{`MQu#K2l2D2UiDx%wG9H8YX26d!9(MiE`q#jyam&LO#a))G%v;r zTh|MwqXiS`EKL77G=yn>6lIj{QB+})g~9&>@e|W)-1wmS9To4+X@C4C+P5#$M^oEBE`?g_OAZpAMQE>70GLa;R{(D8h*7SO}$9Q+@D}snO zPjEe&bQYrjJ{_PVqB~edX^El=5p#a1qyK&%D9bu}(}QPK^@+@4L;8r2SwtQAYqys| zf|XhP^QeF%qPZi+OKdj?n&Siw=`1vVn*6t4KMSHLqgoM)+Gze<6_7zRZ+RHMRb2v2 zUB+nQOScyX(fqZ0KoZd`i18ximjul*f`)V!n!mXIOhW zZKlYSo5%2aV)|(G@ETA0&V(cik%smc=YPIZ1hX)Pki>gaz|;#E()nPr@uZJUM=*<` zD6<2}_+r;#4w2W3A8D6$ne@HJ!%~yl5Q}8%a21yqBHf3%M{^sKv$_l89fdjL-#7r5 zN#B!E#5H=bn9*ZPpeU2RcQZ?;{4Dg)S-Ld!Xh~y`ah=w~JutwWN#A82wp(Uq*y&9Q}ZY&d6Niv!&4U}bjdYoBN z)tRA*&WhF*)RiMC)%({lZzc&1YT+j|vRekn zKen*$+(^@toe&uHMw+x+n87G5tYvWgXA67NLz(#gPL`7{o`d|S)y46J-NkisAUlWg zKa&H?lP3qVb1?snXC!q(W78k^jS?|BIgp)${U0i!u_8%wz^KFL-{^5xFrmSld0K!8 zUo2n>ZLp=~^V^)SIx}Em0i!d=KkB?9h0d+Ev$;@v=Y@@ZKN2pYb1UY*n1orGTIW_( zqVr0nGlr-nI=4dp)3=b)`P}w1w-;%mbE~^V=as2+#`=GpOEz*;nD!2<*d;;d??mTS zNp#j>NOo(p|96$s0BWwKUx|(R`l?2RA=j#IP+wDoPFCFXN*%(Gp+o$IlPnvP+}ol%t8kz{qS2~##`)E9Ynp*a~CT=nU>+` zC&}INlWi6UkkC(dXPH1MatLUEy3Lz`vV0QdAnfc{dEWx3Sm{w*Sd0f>iHaXddatFhrqLu;&Ix=P?m0KmFnV=q+YV!@hC)(vnoaJZFh;)d$pH*_}HC~ z0%(d6kJD}mxX|OQPSSggI?Vr2ozyv3 z7MHEgAeZ;|IhJFDrE_kL==?+yosUIXG}qh^}J7dUFc<)jgyv`W$I;69_&Q^N_ zl%&_WD~dAeAtV%4UPtGAv$r}dP?mt+Wt_4HMIh&Ua{7Rf^93NuyT3r2xV~;Wz?p06je}k3M9LIen1AHyyuOVofSxdj87j5 zx8#pV*uOp(()^@OcKewDX+*R*h6Mm`qaZp#5RuM8)ZXm>@Dbe|MVVbm#utZ#qwz)q zX#cN(3?ka-O_4|Gu(CNNeMH>9KB8*m=%H27Q7SXTXp3T`t!-`QKYm-gI|eW1Z4o%* z1P~gMPZVW(*l_3^0B7qb17!(klQ&srLCAqjP9F|3I)J0L@+mY7A}B@B zbm$Y1#LBrRhHmP8AZShzG^Df8bf^!|5zUe)%5+bHro*g2S)$qOO^|KmKr`WQpplH| zH3?`;{@<^gB{5`MyiJ1UL_tG33r$Be{`pO9X%uDhlA!4rtbd|;pQF@1EDJQ_GDYKO z+pZ3x>G*8GZmgT7F|1~I?+Kc*f`)V!nodrDj&*Zy6lK)Y4=AcS&XM>;OY0P@f1>%o zo9I+UC<1}u#Prc1Fhom>r*^Le0}7Fb)@g1)DT28-hIJ-yvw%5Cz>v-cvpzsaFw3GS zlTR|f=mo^Db^3RpEWvE?COK6FEDOIjDSa^7uN^*YVXuhtjL5hhnKb89o>9h)A}LPrY}-8`k{P$z9u^wRGxaqJPg>Cl zGHwQs*mc{fHglD6y9`CLZM!m>0`zUo{PzhJ*^LMWnlcC37V}@Lp%tKp&hwses#G=w zbC740vXR2_0vzB|-ef4n*#B0DCg*ys!ghNHxHIE+HFxsvO?8oR8}`3i-^ipqLA2d% z0h!$3F^4s75FMGm`3-j~((e-Ve~k3sCTcZRf)dJQRO1sB&?V^qIG5b7tte9Aw9UrR zvK{znRkR<^YN;AAx?*%iMjMH0AcwL9d6%04$T^=J>)q`?$aHrm$g|O81M|r)!SSye zZLSB)1TsOQ$@b>?4?X4;z#1F93;c(>E>N13*VJJ%SaH)w^so+Y5luLsgI%{r{=;gs z%)3qW9v7ULQ9ho&E6!v_NiC>>VBDc?Knu3Bt2wV#T@gL!h)k^QW8{znj5~}8aA!Mv zhjs`&!Y3VlhpSPRQ#jqxyd>S1JB$ zocTDV3N};tgPHYkc=nEF{__-1Xj4j01P{Kp!&_=&$ARF-+IT9lHpD(Vvi@o4wMpzl zP_WOASBO$;X&*HfT%PT6z-a6CmQ!t7;x`sIB&$OtS7f;iAlkaSryw!&Uth`$*X^Z9 zdL@vsZlKssJwsBibX-0S&<@WeSTrmz7T(FM|E=ZY_W(nC#dB4*g?E}B=MD=i*jQ6k zFU!I^nf&M4L<>tZ-O0{>I{BoZb>i3S@xcO8PdhRH$$EM+1(%&WSiMH)VcHz>&6mPO zDs$%&aA9SxOWkXBCVuvsb*k5RNi)AYpKD7~I?wC!=L12VcfMV8UYAPer!6koi$U#- zymi!82|9lvIzN|0XPHXkc4^%~pK}^O&CKsiJog!2PQOk$;>OwnJbqt-^Z1(BSdXdX zP(=9gEx$K{pEZ@-<+Ok##>4l;kX7(LbWspVnDG)1lg^5Vcj5kLx)7})Tvau6M%N(ZV?rP#+ts{r+ z6TEwXtsj-dNOF%FNzyVS^T1?J)-sapdVc^`HOB3pEE8BGlre4B&4IEUw>Nng%KjB- za-q_stf;UC725Q93LLlRK?`!0aXSzBKdUmB3BHGsnM&JX+(s-8Gp2S%t zX7oamfr+C6dQvyz|6fo86MNkQJze%Y91sz`F>qEdNetOBV9n0zlRenOWgd%UH>4y3 z{sgjzy1Psmk?i`EWbpq9WDj?9Iawyzb*ad@6Ms<2NAg`}4M=uvN;32OqmS*;JePSO zl3kOMY+Oj%W4pS{RFZ6NN-}f)l`n1iE-v#lB)d8#nc4sEldag z_I-$K&Gs($Zj6&bMN;1a3L9!QfT^?#fY3HXT zGwa{JlIz>K(P^4&S}L;LdH#P;+DmO+{6$Hdnv!g2i0tJyZuC4&c3w&{v;OBRxuLZi zWpx4Dxhct3hNQjH%8j0*$<9eh_OB4xt2u7;Y)y7{Dze^Y{m)nOwU%!5EKPP+h)iZ2 zy=x$i!~OFqhWp;|e}loqU5j|7)gAcBIkR*}F1TAwwD+Qr&aby{qi3QNEOcf_xHQq; zCjRqvek0qBo}tOkNJX~CwjpV6X1USRHQDJY$%cf;-g4dOX`1Y`5EGIQ`Jb=k z+m5S_mHBvg_K`JTf?)Or;vF*@I^m8)X-Hu#7iRA4gk zS5yu@)QfG3Z2|3Zug-DqY%Q$9y;Ye&YuT$DrPx~5r)VvE9fbmHEiWf&EzF%z6z2b` zQeIZAg_Dc-a$Rp8zU4=24Mb(eNMWeV{wJSmV>?&n>OpmDDylC-vNpAKRW24(x1^xj zJ1<1F8P#&L*6`+3RA&E!ui^WxU3DD*WZjgC%H+R%st?e2IL!dnjj5>2{LiP_g6_jv z2B>aGMfF<<*N5mqoMeFN`V>_AnDq}|)<@_>oMV9Mx>QtV{oAMd82yM-3{YL0ipu0a zeX38;l{muy)itT8%=(W{^(lH2Cm5ibn@A<&L?kC*fg>}Z2K&^@Df(1|`+wP|URQleIN*aG zdMw`PQ;}Cu#!+Q{M2{VhkriJI$9po~cy-`^B*?`#t^Z1R@yq63dtZ72t-u$BN~~Fc z`A>WxZ~qj(-D6rQgBTW&F{EI;Ab2AQf?ZG!2nv}0r>lO$`9UK<78_sTb*lbQIdW6l zw0~0?aC0=sDOPEzvH~;zd4p9(>%a(Zd}A+5b;KK56n2Fctc*7jD+AV51x)<=x^ui@ z1x8e}HQu|m$Z^~}Y`=L8zsO4o%fXoWSVGN8c=5?08GEGO)E)PxF?hR?PJuHnq2sG_`!KL&=V!!&|DU zF=~7_dXwupYw&%%v9Yl9tr8RezNH%5EPT@wP9{lXD>+pf+gpi^jguGPFA3tO?DwKK z;X5|Anilx6O$G*tpQFdBLWj?DcEG;pE>2Et@IX8Ti$8V7TfJK6^MxO&2m>l8)3-;U zGUj8|e+8fG+|$MP*f2LH`|s+Kt1wYaZxJ;!Ad2e(_s6jP%lnA3a-G5%y1~$38>5I)T+-Dz(VH`|mwg1BHW`{*7nls&P5KRrvIE9nF70Iu#?TlBD;*s` zL4&RIuov}I$;22jh4R@(2E9K#V1IW0T1C5q#5+olzQb7suC%7+_xeimUet7hApVx zr((y+#*QQ;(Ro_mea-yG@3rqo=Q)nr+f(O6=Vf9?EIIO&AwKk~t%vcrD?05CPOHPB zBNO}WRMT&%%CTCV{mlBO-~Lwfd<(T*6Z_Cl%EWvL>f{oa5+p|3@63?V9*$ufviF%7 zZHh4(2`!`T_dtMRCS^w2F#Fq9*>{J--b1~Yf#%Ls@(PY5i|8r=`0srmP1O5TIg5+&A;B%7l zg*V^zg3)C@n96{EfSLci9iRW~iJ>$LCHS(U%L6$7VKDu6;(*UY6ngFfmjtXx*HW@Tj0mms_Ui~C8iyQk~?18jnBX@xK4ha<2s6>Z*fs!U2#p*&u=>pK}uwIpvnL1 z8swQ&H69+?Rb3vD-O#MG8MDE5Rz@Q4Gw_BX59ISN{usi|WRtvA5;-30c&jpLj%hAS z`-)Hr0eTSgKfWR(pzN(49@JLn2m)xe2AbYc`gt%HP6M9blRS#G~lM5=X!6)OLLw}-ZVp!t{KfjNp{V561ygs>3}!dU0A6W z3szvnc`nU8!O3Zr2d)?9`4&$3Bg(563dm1xp0kg>Bt~@P->D;YtyCLk@QcjS@y#0i z<~Y<#>LQB$UxV?hT0F?%j2O?aT=J4G;5Ois06w+>IP@)FdAETG2;9MrciaD|Ynhe@ z9~`_Dtf(wf8p(tFJm+S2x?2u4TLEyJv8U8GoL+gx_y+vaGFCaVq5?m(sI40xarZ$P z8l}yN4QC8#IOX`sW8#0x89OODsm?Fbb?l_kr;MF2rQz_glgIR*GXA8obv2V4hE6%@ zq)Cc~j9QyAaW3u0O{{O2*iheZ+(}bTK1G*=e$)|OpuViO0Di9NXU`mEaOwX2%>Fmy z=UO5Z!=oQ+a}2uwINklzVXWN?DonW+3HAD!^>58TieY!IH(X*BgnGlXQ`$WvuST&H zDX{uS0&R(2ZJvkC@CH^*re4j!62H-Z7Wi|Vem9BV$b%ke&;3pQ`&~6o^JwB(d$^ub zF+MM6SsAALoA&>%8m7O;H(fxJg-^b$VY;F{q|+C?o9GLkMnQ>=ooLL zEGdn2;HB}><5X#!CJ2fojdUR6e>T$h5*z6RutS9od|g;+>=Zn?C%$4MJ*-7fM?+7q zKGXJnDz7ib9~8)cxKTLV>v%AEOIA6IKQa>ykgcWiWZe;O;2XsLPXoN3XaE@%2fbm9 ziVm|L&O>WQ<>G?{qzVW94lb<1O-ZUyVywZtpbTTIO=<_-EDjxhjD;UN;H?<`!)pnH zk^hU2>+nW+UG~V`+vCGRFqr!vh+uOP1kwlxoBbb~Rg47$O~qIT)BiDMHKhSJb&PfJ z{_)aivfGh2hv+0t_TWa8WQ}c3tg!z*11t)H{RBp3xVWgq1xVETiFCDsBG7lo$ z%FG-uGlo>|IIZ9zQ{reF@RteTcuGDl(8AA7KR5rMy3X=Mk@itsnn~V|qcqmjaE=Js zFb@Ae=AXGqlVXQ!(L;`li)IWl4OMWGw^UB1M-a4BJA(9TLMy6ypDX-V1^6=+vL$*O z+AM3j)`{NMCWD%0!UlJd@dP+MPP!?wZ)6_NlT(vp@kYBl)a3tlyW^4B+j>{Z^RdW; zT$%m%waFzcI}>uKiT|`PWR-67&4xNnBke|QsKvyKU#q6uTkqJ2Mlzg*D10Iweay|InWx zkACxgTx_)ji;*> zmHl9~MK#>4|M^s_@SHpzmseEwlkzszF(D0C;~)o|=b)(Uqa19it3p(3@DzKDLozIH zKgVuUne`uE!zXZ}M{JCyvd{Fess0SfdJ>0c;0z5V%RWBCq8d>gqIwEv7R4Ghm3?ZF zO?5(u>S>&3jAMsXzpvQM?Ps79Ln zr*Fe`IGQ+CtEue6iES#A|MjV!#Tl%z8ck)N!fI2U9@6kRoJAX})>PH<_z$-{+f++K zRL|pdR2-73tYM#zYEzl{pReHyIE6IE(HJ(g&mpy_Ms*FzdJ%_d<4kQuWgn+)QwYsrEc=vDo9dzv)k`>R7-tSED*L!$n~L#Y&;z`TW0zw`X)62RWt+|0rFs^>#guj69P7^6d2-o9GXqQbX{HhjZ-#*I~Ks!C~xTqLro%=({?>rHR1 z8ylsmMhO*T1DookkgT`7r`;H%LufcssLn~Cx-CTYw)d198=`UQns%0UnO}Ke2Hbhem5o>VIu}$@9h-x$L35*@0sqC8qEvmzJ3Q@i9t#o4t zYpR1q)|8;E!$*dwKJZq!F;3@T!@)u|If2To|M-^Q;w^V$gEZA3p_-IHW#V6->O=1_ zH#SgH4HT-02~>=Kf_~#8?@>24KvNA6stG}=Bh3CMU)IOoBW|ohQ&k8R10>6aM^uMo zed0ar#`s{MD?lnkQ?i#srm^O10P&URaB@LAlX!9A*wID``wtQsXU=#fMiph5~BLjyU&dsq^S-PDh5b4)g2+K zue@b$>_AO*pinVDvZ>w;QGM;*>&6byR0jwZ10;*;s16~jZ@i^$Y=2F)zfdtivZ)4y zsJ``#EplTenyN&osuQTzgs6V@7P_%wO;s#Z43KPDzl5lM@fNtTB285!R1ApHq~_@S^xHKb7K)r6%i^1NH*0= zA*$c8f*;#QQ|%*E43I3UV_Sr%{^QMaV|#0=y@iSal1;U5i0TjTRyVenrrJxW7$DhH z<3m*c^=@%veKb`cp<;k!Q!NZp{psE8#`e@yc*H`@^cf)8R2xH7e|a~#F(&BIQ}z%l z21pjwah(#WavbkQH^vkKsCo+(102ef@7>t$nre5UVt`~*og1Rc@~(Gdy);!X zp<;k!Q#}}>%J!~vV=MvM(1){<>o`=b489N<$m|I*FKpy7wtbCn^xgEcgh_NIIP5h$?nY;QZdIvH~q{B9z zJKE$yksj&g#w2$YjSZ>LNf@L|k|$xveQG{mq6)h!)T!qFX9fj+@~VqBOU^?>@@f`1 z{#|u)iOWu2Ri7FVhU`jCPV5XA?N%|`8OCTNq|v71O)bs-AKz#nM;Gu(A$)(l6J3yr z(Qs}Xl2`F7Ike$gG-0@`zl`CiPK{Qn#>{_x!)@=)7Q=)>f8tLwRgMh_gv|Cx3^ZBrnj7!c3Wr-;dpH~ zD+4pwTZ{^7i)zb?Ynwi3T4QA2MI)(wJ%E#EpiTAG$r*9TIT{j}v{nO6&xIKN ze;A zvMt`*;Eigo=kuRT<9?8O`K#VM|M7uZ{sMxgmcJUz{!ddHa8oaTHJImLKH#Mm~*VDUxo5VrK`$(x=^qhl})X6!^5dOnn;pllN1? z|Azp7?%=r!VYFHszSG&WmC`;^+_wy4PLpTjtAgpd))Z#KI`V)FjD_MAIz|B!1&3)l zm~8CRHY6a)a+dG}Bt*1oyTy2Ivh+Y(*HBI-ojA`&NtiHy*)Mr;WdT-;k66iUGYYB zo;W978siFGHTIMzeP;1Qj{h82wj_=#3x(BYVWp~03GUq%=QR&czbHS9?mZ>5TFxO1vNr7B3b4o7f6fdK+f>!Y5U`hR zs_7|@M`#UuvGlmUV(CqZmS%j4su*{;6*TD`XvYqm^(MZw0@!iPf3mfFoTRl#2tIBe z$}j}~SnWVt#yG+c!SQ1Uyp=Edhj*Zj-@!uAb_Asi_-XQ2_^=R+KTr^Sk^})=8cZ?X zN7Kd}U4VK*KSW zJfU((MJ4XB<`ZuGZP?I?p_QZZU|_d0vlxQTmTHdCs!lZL-+k;rLF`4#at~;St(y4R zIn-^ZPRUFqKHWeMcFOjS1w+c5f;XDQ#53Y1IvIbV+yg0>e(flZyxB0Aru&DLpaR21 zi9h2?Fa({3Fz{sWUO!WCul7Re)st3P^OhXnq~ikondAE;ijhv8`ivak!Zxbz(&Nv| zM*hYIHLAkoI&414_FWy zy_?t#N5y7hwX?<8_JI5G}gf!X(c*AI?Wd#Unw9lh=@mw{0KgNmPm5I@C6*t@4-^wXdpK8$w`7bpJ z`Wtz5lAVY(JO$H7mu#y z6kA!o9vD0Bpr`MS_QiBk)xu}3M%>H*>;Y=w{@z%&dgp~4{OsYJi^a~ZC5|UFo z)2#n%MMmJkyxx7}3e+N}PgVvNH!#Ur;^s5C|AA51$B9uGgG$)pOq~BQ7(c4Sua29) zKkAQa{DlQ%mUZTX;KI21vm{aHSd=59w^4?1^Jgkw+o6mYSp)@*R>bUu=U*e@1Hwqc_;nk#;YE?yE9nkX8ykyBvZ4awRY$17T2x;e_^nCP)DbVt=jS2 zj@bzkGm49k0}qNBgEtIuj>&(1CPyvd%r}b4!q)SQxdZm`f~t?g?=lGS=oe`$crItGg+|Ep5V?>M99dhK!lo;tY%W}ne> z-thrx9AI-}=fS!Qh#t}Kd1inmp*6ss$N4|KfUXt}u;1Y}Xnm6g2H1re4zTIrF$3%x zRd4kdT zX8rTaxaZGmg^oxd?ePpxp2{oY7YqryO^b@>PZkxwOrhfWb8QuGCX3t8`)w*dY!#P_ zieD#DQKEqJH=_)rfUi{)utBTBnwX3?ULE)!!-{`6dp6z9|8<{1DL3zX$2xpi2&S9- z-#1ARNE9&L^#5;E6aWNGMFG>z{x?$^mQNi8O#cJcVW0R$Ent&(iYVX$j(==a-z3)9 zp&&wAxWKIcey!TV20IEU)KS2mfhd5&a`F78ISFz!0?@#@kYr4|k-6Fbs+c>~2nvcl zj3H{0F~RYU*RerqabsCsaa}XdbLKj22Ms=~qKfx@|Ly!472_sCD0iWWf4+81Xb^3W zTXjtq`hkwKO7KZ~VEiMB85 z2C;Mn->O!sdJMkR-?Nrrgo_4l+G+%9P&R zf=j(+(7LoYY(f|KZK4YtjzXyHOAi*KX)nb-aGH0&T)&D4=6*e}r{|>zj(w24FA)BV z1N<33e;I|@w&v)HJk{Mgeol`+o@yU7MESrfM9^xIly(qJed@GMmmL_;X+HLdJi&|G zyHlruytrNG3)1q;Np;F4L?uq5j?3-~h-4(aRctCE>4lVuNP22kKAoW^B$3$fGPD0z z*AMOYRrDT&0Ajy;{-*tKfNI=+sN&`iytM3h`MCiwwBM6FdZmZhZ`|LspRZFcA=>u4 z{D*)@+OK_V8tk`7>^IHWkA$=zLqxRiEA|Kw(tck@m!N-;4$s;z$;5uxuZ;F*J!=o3 zNi9d_$)2^p;?aOMwA@on$f;9huw8k4CYDRkCYP`rM#R$QW|;iHZaTEt*0J+pvpdCR z=Np@mkVMS;z-BX!3usB3eG^^kIGspW?L?PmVzX9GOW3S`jIA5NPhN@*-uTYn;7vUw zP?-^I|FqcCoOr0ds%}=#ps(7xJXa_bKNE8nr_TRLJ~=e;K~^R zLYnK_=)KtfLOP@f?#;wpQmOv(MaxtcBloHKe2FTodeq*`odW96VrxAdth*Zt(8ebI zjW%Y~$t5nkjm^9?AdE)q7{e92ct{Ba;nyxOMk67OHXUzj`DB2QM*A+h4B;8+uwJ$- z6Qf~y3T-T5`68zY!zJAkKWlVAB^vG--mR_-SHZj0GiYi_RLUhjrv`6nakHKY2&C~k z#c5?gwOL$6}*zPK`{^dKj&M{+E!7tBa1RtQu4i9a$Nxh*s*B2QEp+R0#3MRI1d7H`Ka1 zH-Lu*e9pt+x+)sMM$c0-F<>gSatTyPl=gS^`2nf4;D8Nl2m#L|yLx z*Yjb9%Iv%Uiav;A(@2Ma_`yu9sDsaFzY(JckHj_`W5Tv6Oh^&&u!Q=^WO?o|=)}qL zcS$D8d=wC7o;ykO{!UGnY3ApPT#v)_f%s6Drk2*!G!!?~7s_-t`7A{q`r?5_2nlkd z5pbXkBj>%4oEI>C<2({_j>%XKhb|?jP-Ku3@Lba`B5oz zG@1!^@@fJs+=P7iu#tyUjO5kAtSU|{DTk9xnDE+*LMD6}H#*hf+XL{CzN|J*E6Ab; zRbmbi5z->AHT!>jNE>hkR5ePWMO3FR1}%?b;w@qjI9M(3Iy3(Ja$douQ5@nRr#4eL zYA{uJ4o;B6{m=9OzMNNa%@o}k;WP``s)ooGNfsOdbFJjNPB-?D~!ldMlZ-gDm*smPe)JefC&p?&5bz{yV2 zyr*g2B_*W=g{5XwT58^x317q8%>RAfTlRImb2RUBH1Fcl`r`VcnkK_Z>V0aE_q<&~ zdVh!ojMFr4?3_fU*Ay2=>JSGt%bSBKbr!wnof+c&QK<{JqU?Q!=3UsgZ*kv(ro&+B z&80G%_nRT!9~ZmcRLy&;<{hamC~jydZC3B|gS_V#hIoI15R~o`?c`j=yS%=jsJNl7 zd3&Fgz}t-feqDaLkLz)Q0p1sA-Zgbtcquk*urSiNS5ZA-@Ks+|USCwt1&q$>W>3o| z^G*KG=lxk97XzT;eY)mdT33X+Y;JL+vzn|}yl*q>e?ITecXw55g5IYJ?}CQ1k_NM2 zl)8Oa25oZNZ6SMqv776$R>1pw&AX&HQeIeQY?7Myj0E0)hIoJ3-9^$s@t&f27nRjU zN*c^V8mW1Y3i7`Fs1Wb3y15tw6z?+)@4j_4WzF{;hMJbWZ+|$%`|EtyV{3v<&Jx~* zb&;AHtZWq&*X~uCTJIBryzl4~;{DC8u6MTPeYWOZP}@*eU(me0Pfp;?{hu78{dV&0 zHm-M}=6#{!T~<_B6N!|VIaTWRJuiW`8UKCW-*tAq>6-U+&ATvy!K}D>-Zu$vi7glG z2E7?uexDdy4o5*a_yvLe4-Q9(e~3OJHt@$cd+HQ1#ybTJ<6tkzn|o9>YO7m4BY z5YPR=r+QB~Rr%zP?MqK&5>8~aeF@=!y?tq6B%lZ%&%PaVVx6A^(0UCty-2_YI#vT+ z$o!`ROY0pwTeu}fFN03TCqjn~_)sryRrv>xoLGeMe=r#9BPXbhGjAIttphuD7VQn~ z7)bn(ddJS9kz%1A)Q%k>XllpKqNxd`0XOxIokdIIrMYjHylI9k%D}5D-qK<@`8_DU2&i&|v%KdKf5R$!EauW0T`|MRUj1@T*L6n-U=&ocAK3IAT34fs2C&;9`a z>8-%q;uG54iTO{wHac)t<nBy0D0$Gfxj|7G~P zB@aoz#CvTDB9S~S{O36<f)IOI;mIWS~kGGymyN!gwS_w6Dy1ik!aMIi$l@dqGqt zc7?^WtBaR_JGkqq`_bygRx zQ<_M;+vGoeiP!Drdej>dhbEO+l9|LCMVgWLPzdAo2<158fW$+SNkovI5yoqS67T61 zl6b?OuE&rB5{D#}7|BfH+ysf&ha}#JOMaNqgv3JBaECZ+7iDDNxY?(>kZNp z2PK!78OBS35|`G5B;Jbfoyj|39GFC6NpV?b5*a>P7?-{lk~puY>oGG4i35^K%&;D( z1ts2F5t2B+hwD{ni4{pDX6OMpowqRF`$$OQZJ4<9*An|Dm6)Li;AFy*xQzSX=-GWQ zcRQvt{j|h>NhB7R_06mvITf`f-WZa22PQc&EislfE&lyq6g*LsIJ zRcf0#PEI;3lXhT%;=l)+Jvu(Xo_7;Y*$q>A7hjKdcGJw$vsl^_REM$+yzc|@qnI9F z&)#LOCLEQC1~`-rx+~VwyC38Hm%aPz#NI85jmLmLp4iY2xPS3w!=B*60^8sXBRw`3 zTsY7DDM^}Xtd!yNA54_}q#kYi0XD=J>~IWfTL0fA%9dmOBaQ%R7QCu<`s@>YSO}Km z`8S2&=OhSrL0O<&eiq6wqWoDU%7CD$MA`DY5=sMZ>O|S{x8kM8AyG!&DH3HX=>J(` zKPT4M6z@^Vh|E%N`)t+?N_%{>20sMOW?iw$s*7v}0nP)H zj{FWEs0+6SpRn_ak8A)A(`2~B0Br;obg?h;uGSeiLv*_Ih5)oczP9@z4=GzdY@#;% zR}CK8zv75Im0-%`>Sy@i|C_6y*)Edh=UnIrQvq&4VFL~esWa<8Nyj8`p|RQfVDQJy z{&&@4uo#f#PV=6XjV2gMpA{dI24kHDLuD!649&=&=U=en{Nyy?FHxM)aU325?2Mmr zeI%a!3mH740G?|fl*B+D(pLHHpZQlxtD-cv)8R%xX7X?1V~78{Xo?j>uDUr6lzxUW z$pWuY9@>T>8#5V;sG>1sM{pxF?tft4|0%KWi??2l_3yzbYaK^2;EVgO!(S|4x0?As zdh1`4_`0!L((2<;hTi&Ddh0G^i8o==TjSM%|Iz*b!w({_{=h=86SpNG)O!*>(MGVQ zr6Bk%2?FufYs~!rH+pL#Xv$l!q5o$SHKhSJwYOe#VZ1anQ@m4n>owf}$r}4DvBpZF z52|#{24VFpy){^gx28Kvb{vTat0WwDR%jQjI9;cF0Z?nL+>0 z_NqmWG~Fjn|Nk}q*l~Z#V*ek?{+|uQ@c(M(9}rBTBk)EYJZa)zRh4*R!*x&h9>BIH zbT+l9`hf2K=~aT(Q!{5WjRO7%R5CZ&{Stk|anzX;HBR)AHW^ehwzUl%lIeLQR79%u zsY?UO@H~>)-VJiXl;?OiAlk=hlpffiGPwk0Zy0;(=YTA37~3Is1uT0%EsJf5S7hb0 zW;J1N{OxVLVX>#V{>g5wtAwG%zxb%Mx_$#2#um3$N~SljN*j%Y5?!1OEMRXKD=n?X zY2I~aw?fihDj4a3fphJ4z)6?>Z{-kGzrr`#LvzuCW%dhvV`V8qmZ z0~}{oURTm!{6p$J7)(l#?_lA5O^A10iAwTB-2H?zmKsaT{M{XW_o^!?!sG9H7f?-6 z4bl|ZPUo$KsbOsL|Gv`o#VR`%aSzap8|ur8YwG>2ODPz0SRhle#xsLL8aLpfhCwI+ zYYft?aVku4ePQ!Et|kay!`qzy;49sT<*q|C??W{2nv%xy!g8}IFtw#;32#!P7T4iX zFx8@c-pB8*6R>!O!PC4;%WAQ~s9BS6{2*Jy`sfhrF<6I)q6An+HS6NWqOwS1(`us{ znV%(Fsm682|NB~x?V*xn5w~0z*Vji%OH9;SP@AF}bA+jw`q`~RI-iiISENvlgEi}- znzBfJd=pJ-Q=bOjU$_CvU09=S=?J zugFt6scd}2JxFt|Eo!Kb6q!>^QdZ>d!qhP4{ttR;pYga3Ds3Ba_Z7yGhDd3tZ|W2+ zq@6I9T714B#CSZ;Z{W%ktkGYyF0U<*)Rg*-FZme@oz;U4mZhIJ&;R;LPsr8l+TeYt z@a|huhs9pm)|T9vGuFQz8d;Wp{)>>_6WgggeZ<{gb1ti^t*fasoKsqQ`=CKyh=mwW z!b3F!v_S?4>(ahuH8m!$nY!tn6jb_!TSB}iw^WNK5%)mNxo>%Wd8E*HASr6GM}pG7 zh8R!D)@$9c^ud~SQE}f0+Fr982pPp#wfN$w5bsm7^j-n*9-?^{Hk8&EmN!2z&KBMs z@d(&S~Co$)01%I9$h!>4XG&Zj0zf&C_Hf-uj4u(-9EQ z5nJlvGK@VLIbkCnFT+SXutCA}LM66mDncbBL|M=LA48PilY}VT2?A@a-z3WZu0j-^ zHMY^i8hACnMpaa+s%WgQt4G`PTTrUb%C`S>!l0Os*UW1Ruq&g) zw4v0!bzTDVYeSlE_6E4#5t{iCnt546ZE;~^^YH{H6t?C+g_yt31BDLP%n#3uIUAnE z{N)iL<{x-5{oEjIUay%KmzLq&9}|(L_6J;qx0##$f4;VnDO7Y$!B=tgy9ExpCY`|Iy0-YH#eyFtHmL` zKgSa%Cu`m(XU6;L1m3ekyuT=Oy^)&tNX@$_Qc_mf(7eYwIDxmh|IoL|mzbj*t$81< zc-J>zH(*U!^MPDLkoRk5{^Rrh3Wgr1d5_b)Yl^s^(&(Mq8Qz@0+suD_-d|$|QLA~^ zYTo6Ajgk7I=Jif^!sm4}|M7W$gQr=J)Vz<>yvvH~@l=Q(7^Uj-^v-PGG68!1m{47Q zi>F!mE!5ZZw))cAddw!wNukO6B_=@Ja6m%C`jHUp@37006%N)qeo2>gxv&*a^@e%= z!MDiw*v`qmiwZqnTcof7&zqDsAJ1I^Dyzr8aY{(-A25?;-v#f+%y?g%z}r0k;OqS( zwrGykyvHiubq(c3n1DAQc3l?a{U)D(;B3O@{V#0Bq_2eDV>07?X#(%_L%e^&wo7gc z0q>(U@4j_~*iF{Fy%VNxZ*C6p{&_q7_%V1_Dc-g9jU^ZvnxCxDl2%>56^?&?!6A7k zYQ0Ys-nBIdbN$I%s-80|fw#&3`1bx4&&e>3f!-rD@4DK)b&-}TMCckxd z5hE+!!!zUU32*6^Zx0Re{&y$6`5N7FSo*xJl!sw$)_;AI{N7Q|4Z&K^4KeGitBW)= zZ4$8FA6k>GuzsgSNbCP}&=VH0))SU=S#u4RY{i+pW1fHawf+MKJDh|PsK}GFK}yPs zu_eB-*|2Ah@HMz$z0K0&j_p_O=^&y1AGM+Hubt`M~i@0SvJ zyEup#+gFrH4pY2q%Ib>|Jv6O0c@GWpes7NuZ#T!qm4u4-=*)OC?5575_j?zGcxSbA zy-LlyGBe(WuS}WW`w+ajWcGWCDf7l`5w54Sm?gT|Jm#z$r?i+dPt;T9^j03sOqK3L z6xdiN3jCU=fUL4^yuezc)F)EUUA@Eo;9#w?Ze;)GeD{weYm~iF7L$yP>jlXl@x{Na zR#*ZgkS0&0-jpLM{*gk(O?ducFt}I?!H@Slwrx_TL|8S{8t=7huffyle7Q%tRlh|n$4(FL^`Q1%=~W09iP=%foT(QG*kblS>k8*0@vqTGv6+DUtDNYg zZB%8Y$KM>4L#v=^r#M^;U-^fNQ4PkzROOxkRxS}=<=rS-GjW3FjXL+3xhlB?q@?uL zl0M}6r>;_3l?%jOV>4jY2Z$OA#51z;^#U;oNpwEmQ1QcI0YX~qx9DSzvoq-$o#!mre(8gFW}C7=nC>H!|I_V-rrR-gB~16AnC?nrIueqIp%)bW=(2#KG~K_W%W*R>>FS*5 z@=Q!;9nhui29#;ojjAxU1$92&A)pTJcC~jiE~242QOCPE6T2m-lS^DmkQnV_6aV-| z+bK2^M#JuW)!t?rqmhtA)A6R3s{>lnXun5SIL@x5!{MhZGBH}hp;_8+Em9c{5|SQ% z`pL+EN;KSDoW-FHhwYZPWMa4^D&-QNQ-ik{?~|tj0@>nrj?IGc9v0)xGR7kziO$9w z#{0zVfAL-0f1)cLCy#X1PIP4^#?w`7oqLLbunT%>N2f(%H+4-nr5@7UKD{IWf>yf* z5876#lakTgZq3AM3A*GGl-=AuHTz%u=C(_0HZ1mtSZubj7zt@HdSO`Xv%Ldavbp^c zecW;KNr%DU@k}g+O*%Y8FMir-PV)w%8pnt#T>tT1G!1sGH&5;e#D1Z9nHVgnN-iPU z`-ML58IZ*mwrlJv*y~ZT*Hy+|B(%Ec=f?#IX|MlASK&fk(xH2<%EVr}FT~GL)ZMdH z6W!B4%WOKt)!>aP{QQT2O0?T`-h8J@o%oCHIX@G-B~dAt`0VcaMQuPJjh7dj1LHj= z#+ze|M?xC!T&?LBX8)VtJ^zfZcAV~{!+~b2GcjJgON>Nt$>5`Pir|CH<0qR^6X~E| z_71?H-LCg;^E>Emnb<8sn_R+Df(Rd#^rgvv`5iPrb~S9aTx@o=u^9g1y+)rv%$J=)R##LNv&#ttpmTk#$Z<1}vU6yRgy{u%*1siw3R9UuVW5B%- zYJgDE14$?ekeDQt6bL1dga^bB0)d1Yl0Xu8K&WZ-@-Rv1wW5J4ei`Fm|6@D^nl})Gu%NSLjMwD;i+~#D#xG;M zc32>r%NUeRHzcV8p8@zP#}@+fHJ<;%G6v*g6Nq`4AO7)+_#(kRBMOsi%)}rHf(q~_ zx;uwY9~t{0lNgCLxTHrGVWOTsdVQwT7O7jnsm_~o=wy6DTO>O09R|GK?i!$&THfJ% zq4+xQ#3Ggc(f?^`sjk9#nXF5CCe~MBouEn`%o9j-O2$Pw{8i@7J6*ZcC;JATxF?W+ z5m{ivdhC5Q1_!aC_l=FirTf3}2hZtzUGaWCnL3_fgYLh>bpNghHhB&Wrh7MHkd6B0 zWKTXX*|`y`)#fT)JfplCc`nB92bI;RmMX=pH?Q~9GOMNdas*v|!>rv0o7JpYzcgmO zRp!a(Gs+i_hr#sPb!t3nO`yk%uclx_-0n?8wuUC`T5^qX6PERsYbjkjAO`YQQ=$N> zqB+a(g_e5jS=S`2geNf~VR2u(43G25;(phR!sEPfhZc=&Me%RX@x)`Yr*2IjD^F-0 zqHz2Be;x>;l^VsUw{P{d@)_mJaC*z!wg;n*IG8wajM{FDdZ*G8&u5hPjzj`v7#yZX z8sS~@SeJXR`;78CU-$GkBh7r1y~{p=z4~Af;2C9L*PfFx=bg78o=fK+XO1}>LeZ%2 z!v010(b?>{>W3RYh2J=1?7Pz-g=1_oVvLn4;oW0FgDL?uZ({JOhe!xvXJ(avPfz}* z5??uf>g`?Rzrccmebn;BN51h_d~+@MKrG0MuwV#i;fnW$f(C{0Kop9`g3hXh_YTdV zhPm-o!h5F$)T|QV9k?g5N_g*TP_jzM%WN?|azL2(iKwKm66z_4RRSyQ$fw?nveo#r ze(LRVe219#CI6wln0418^G)H#Y$2QN(<*&JN~fgt>o-op=7v3|7n1=8sDApXx6B<5 z{j#+G-6jWN9xqV2|HFn!Di#{tyuHrD`Z2D-U-UA4D8ruD=xPTnG84ZfE7;-{1AnTm|>; zPA`^MLt|-hF~%6QXxFUGK+}z8VP+MH*#S;_e{KVF$;jFACZ&yTF}7w@;q90i~rq{1n-h zA>Wxoe<;s?NbGS6{bm}kG^nGUh`rC5*z1a5Q(D4DI<933{ozHPA324-krjoGJxrll zQFPhVE_U<~h%<%$Xn+TV#byC+h7R6HzUSaNSuX~IHdKb;k4}YrCiP@yQs)?jQ2Yq% zf59MTjzv7N^^+$2kpvtyNwgnQI1HyE1~G;wXFOwb|5CV zL!CP3Pt8~0n*;GL63JTf`?g{A7_UBG57!I=H8h$hzCDO3cSh)c(*|q)O}Jb@G9v+4R3UJ{E6M~ z^8D_qpi2LUf2lszRY;L*cebBAeZuygoA z*xj*f_vWpe>Nc#a!j!8}{;8iZKYrKoJ3Bfy9N)gPeQOmuO+MfE0X(!|0={U4oPQo@ ztXQEEs;aMSOjWABh-XPBs!tW@leCbmb(%e2jSZZDAvc9lM~yF1I7Eq0uS8MOK}7B{ z;hZMin*;7G!d*vhHt`Tqr@0~pQHbd``RZl}VuMy@HmK=0`3mPB!hZ8PaH>_K*1v{t zVNm9)_aTKrIW(fx8$p|=@)Lsu@=$dWaxBZUyhI`XHYmSPB|>i{Ss+>0nKT%nO{^8P zLu1e;&UVly9)WbVYx31c;+umuagCrI7J>F$&=Rej{~b7tG1`Z(U=MovW>wfN#G8ox z5#1{csp@JrN#bPv-<_I1v3O^XuRe(1gKE##6Srllysf6g|Uve@W47FsZtd> zu?stdovdc}B0u?Zzz&GoIi@LF(Uje4^hfix*M!FY&iS#X1};qLk9JIo$c;-9sgx*^ z2+dn9Z9Xif&GYs+ZQhex80yk5y~LjCw0Yh)rOn4gw7Hp{31HS#VDXxN#BSsvv{l*#9|y z7)Q?`*pe!Zli}15`mPE;i|T?x zT6i9lEZZwMnB=SE<_%R?iptx(bvOTMJ`^??Ule$+C$JpunpvaT^cjeOs>bPMaF>Wf zs-;d!5T_TExKfIk-UBI>swT_nWGCx9d;vs(t$#zST9`8F(b1}JhEcF)%dSl{v|!in zofy)LVI-!!DI)Vq7g8Y&n^h~D7=u);Z_v<8_3QrE`w`uy=a4NhClnkf3%y8lu0mm2K zBcil*TJ?f{_!DiKo!?J6UaGCG!XQ3++s;jO?c3YeAKy`TeEWuN+qTyozj<#*-Hwy$ zHgBujwsU>mvYqQ!b?n@LQC`0RPuy5!$A6=Ut&tuj$Al}@C?`N_wHpb-abwX5z)3eY z#JjOb&VPc2y>j1y+*oAuUjs85=wsbj^oqE#A=8a472w9=V%LpYb>POj3^!J(KB{t8 z$Ii{|Cv85pefQ>VTkAG%+qtD3-4jDk0|Zj&LJ=(c|45aafbk~RZR6D_w|P31o1{M} zQ<|grb{B9F$JT>sj$-bAu{kEjH%AHUeJq#DHIdCxkOtV80j7lU z�c5(Hx0UU_x`0T;^H_QAu@%M0KJJ%~2dCNkm%_nD{|{Yfuxq65orqZr`(ee%NSB zb<^R$o7JZO>)L%n`$-);ckePOucS&%RP|kfyll^I2eUP~7CDwalc*?t)brblu0{oe z7H;n*_kWmR#SMW8bt-P?FwX+?R=B0Zb2gGC8->hPz;aiA5tBr$fG|yvS+Z?6&i@&g zjS+DvWluM|{?(}Ca;-@dRS*(Y_H_HJKx))cxfWD0QCY_IPex^9MpUY)V1j$f~LFYU}jFx{0vX+YiGnRyTg@IRP9RXLP9RH z>)%Zgo~{B|7~I{ie>Z7QSHa-=PUhwL-R1o6z$Wc$N{R7x_fy2zO_{zvPgJx1W%4n@ z4OKVTIrj-DA4|PR%wPS~m6!OxffyArH5PXU{fA5iwnyTP>4!EiZC-#^KL{P}q)eI+y7ib_OofsUFvR>`eL8-koLBP}N5d zC4ni{K;%S(4kbC$sUGinlHTa59nwjAa{rf|bZUI)^gK#bOx0sK`$}g^I{7{2{4->3 z!#KGXfMY@j?*lU3bZTbktO66Fw&(9X>!>=Ic}zv)7dZab-c z*Y0^6PFlTrOGg!!drNR3!K9|fDO0Ag_fou+3*}Qi73{%5Ak;}yfEu4pHAu@)sDwzl zUH{Z!Uv#%CxC?Wia^`=w%=Gw{DSunAPuHFLEV!rBxlhH2fF3PVa2CMEv`huhKQVz$ z&ukgaPuYrU|T!@rot8m!GA5$c7S-{ChI32+rP{#E9A4uGDW^%d^W@gRl@5 zRIu(sqAX)k#CQ>7+uw+J1C17lslX~&Acjh{lgKzwnE&o!-~wT)P-f_$&}8D6K6d?Q zc8p{COmQ5O&_m6=qrXJgI*#dcj5sE(9n+Ia^oOlK8eecspR)v1npYI*)-6bA#V2nOZC8=7oUS~jYpL?+cqzs2HAgO1fnzFpxT zs*0TCJmxDR7r0CT(#v+98BD3odC{=kK$*HHpkCV*k}di^i+hQ3AQ0n46eq%pJ(~XRw)8NyFSTrBxE6RsZ6!_rY65;Nb^D>VLXU zalShD|I<3fgrEeQ)x2wSJN~I_m063nZrI$On?;O~2xIzFnq-je|Hh{)x*CBE5P^;| z$o7BJq6x5{hWi*z1Y;3nz7xV|2?=2`i-5ts-8R*W7_kDvK0sy}?g!iY&ls@Q?E`}{ z`#{Dr0Yz|<4?_mq^=~6)hwF!dnIWGK9-R(#fK!5abFf|iFz%>!rF0VDwniH=49;L` z+7OYNWI<-gd~w(Y&M1m|!rrMp6IH+vwIDNOcR;;OQ^(FHhI|W@99lPICRZ@hLqmQc zqO@&V@if*M>}ad=dwLu52vxI4Af6hc26Z0Uwzs++7jmguTsw5Or=G4I-XHHq;DB7I z`e(%%p^??HpW(5e-BtGG5CP+d+Vvk}{6)g9mjjw~ly%ue(+voZvISn1-TyZ7uVUa} z!C$O%7yL~pu+~WdR>l242F|94z^O7ORDD`Jut^6FkJ{GZ>K*zo44o?cJOE!Xx$4IO z{WgS7J$@TA6|H&-lnk9snV}<#1XceQQN}u|oQfOq&VT~%{I3=Xs*m+lG`sCv z=cR#8H6TF|n6pSweVZrvCRcE0aVITP{SMgKGMgh>M!NK{{-R>DZkgq9&yoR&D&UCf z(!=cj*JgH);^m-D%IGmK2COaATeinyT?v* zmSRGdV2BS@1AP#LLI)gC4|X=WM{+*r`%Qhz@-Wv8~6#nIpyw%y^z8HJTW2w zW!^u+uK%7GRPn{Sik}o#@$S6;9?ttw-)GMIzjl7synh7apGKS*V?+(+zd_Be-lh@9 z-FDeR--wzSV#JmRBRGn~T{Uf>Veo8G9Vq*rgSS(5PBaK}A|2HfA5Psdr8g5IijaAK z4bT6CV%~7YRA53Bh@n#LBxl}VW5<7kf%Ej@c69m|CoV2aXxu zkL+`jOJBq{r*&&ri(|HF$1r!957WflBlMWHO>kCG?S#0%lBb zH1iqh1#$&+sltu$)m4^_ERqODa{rrcxR>!IRBhv!ZGe8EpeLJy zew01`5~81*BF!)il@9uz2Sh(=XBPUiu!+Iv0WoIu$w1G*bkL8o<9~?$?DQgeunp)J z33{@*=wAwSTHntrAkDD+audM&plW{xhuz-xoKd6-V&0BcdP6`;9nE@sl50 zGB7quKtYMpqNC6Eq;fnDq*kbs&?h*oZ*u{la-**WRrySUmhuk zkoje*O+Vk!PnEgo^>|2aWnd+{KpoTIiQxDVh^SCy8BodWKpnHyV_pvCDv;<0>_F*4 z9rF#3lu#GowPOfarTXzrW1*wGzIv*`xRB4q9~2hK3K$A#^0`(dljfx~qUloVLae*~#7zz-1=(<-8V9w6wDV>K;! z5J#u*F$&M#gnTiFb?8@`MSbg_(@f569c}Pwmjceeg*& z%xk@cr}7@5TSg2WrQI7Xs<>uvpO#5aSqrc_?qJ64s3ojuj* zs9OqYlqepq_u9YsCRb{ODh+$PwwTOLZ*Q39p|1yX1xTWXvIgw`grTh`ZhU4?rz)=( zrcgETBe})tg$>twBJPH^K!n#jyn z6-ZlA%V?{-LZpQLw5?YIDF~>E?*+_u)SW*@Uba`F#>l8kCNjvVJ84|Ngwacny8ZG< zGbq0_7m8P|Kt0Ls|C#ES3G1xRYWEq!7Ph|~ zbrZX6#BJ@cF=!SAiccEP zb%p9tHxTMYjOP=Hn8v$ZWs(!W)Lj_U%#(lTk``A(K`>avr*(12 zl%5{**`5VdgLB zbcpQ+X_P4LoNCvt-gl6`4?VvsPq4y)_GJAfiH5WI!de$&MUkqb;3wQWclVrPW?AE-6+XF6M8ngQJ~c1GC#z^&8MBE7&g zTrV&MLo8OCvW8gpS&C^DQC|)G)*%zyc{Y^CxS?Fy*$_*!C{TPd<7HQ<9%2KbUT}Lp zk>H*=!e!P&>~P4BD#~Uqb4m3Odk}iQ=rMDM&C($`tOizeO^*zgv%c*z>ml~sARIBoE;4DKrjf&P z{odm$hsw^lq?u=zx?DPjkwYV!dG=(NR1dLTNbBN|*}FXEvw<#}S)4B+UAn;E5cjIv--|{UMgZDVRehgwHASgfrhAVpCQzw)%7~CJ*VwYjc`Co}Sp3`87o7K;~PI?RQ-0LLuWt7T*qNqh0Hq0X_hl>Y84CB?^T@gOR(vP>nvG*+{> z{;RPDGQ0|0Hx`-6C_&MDcox>-Ac+c!=CfVW)MIdKm0jA1uU4&|yFBLoVCEtO0@Zv- z#%c_)-TbOcTDZg2^S@t0X_@0n)lVyQ9evB`E^{H1XcqdZ27Sx5E@=U~{dBu=G1%R> zNU8AHJ6p93{FEWB+VYw!N1y8td>SwCdb&{*N9|a27xe0*M!aO$P7WBV0Wi z?+m+pfg6OwA}__OIyem=SoTW_5U_K-`d5Pugm~qbmoFqV=04>iAlSi>9~FZ0@?FwG zhTu$qm6a4{pQMN&S*{RZ4ZaXp^TPaxyrh|D;o)yxffUGN1$z1Z=5BC#{Sx67;xz7de`1-j^&pLR)g!FLdP?D;pdAfpflS$bsj zneWg4nZ5!&-gaP7Sn$y$&ge7$?=DomrV@xapi35va+wR(MnM&qELi1{u5h8o>ygrJ z!G$if9_B9yhItfxhnOOSra1-Q0)PLb94b5Gl4f4;p{uJP4X#6Dnt9;W?w#Yc;(TpT)6aRtR2VOf2{%thS)#hxi~@Ixw#>AwZj_cAe6 zf(V!nQKIhZcy))^`QO!nx@A~Dj!<{VQc-txoVsnI?wU^2eGN3s$kznwmLsu5sk>9u zT@$B{>tA%{b@A#J+w=d|1?pBHlSimq%<-QMd|iyX#a#cRx@)yM7KP?RXQQR)@Wneo zL+h^P!6%$xO{mHOJ@Vtfd1h=QsU_c+MczA=C<8%3qAY^^ku35)`FdTgAkobsmxx^a zu@jM%??9^Bcl0!VBY{9zGCWIW-#lFg6eLO=nU;KI7J0vL`BHl_`hJO%=`!4KlwV97fI@dvsMxCB<)Ezw8o?BnPZiw?E% zACiwOP0&LHiQW<9Hvfm@qe>DCfr3P31o=5xU`7`wdTH`r5#%>zk&h`#aE-4ZQ4vA@ zY!><0!UUaJkSNb4U)m#!d|ZK^8sWbYHL$EWnHuK;(jX0`QFJO2wkQU?2q}`qwton$+Vu63RB;B2xP3YEAKm_=7`HFK z8w7Ow4H0gyHf~@3qCmZYZZFhzW|8iZ_-_D|#KN$mPZmQ>q3)AKiC&Qm3$hq$(H|iZ zeKV55o_`G?jZEo&S(GS`WU%W$A;YMA-7||4JtG2HfCx+ z`cSWYtrh=Gq(DSMOU(L*qislBnWu9CIz%S%s%}}ty^}g4}M*zG@>R)9@Q(3}1Bt2x$1t5r#{`TJ=MLce9RSPC>-*$WmaHZU2y5O!rFxk-oY( zOS%{zIm)lL^WRXqnD3AB;(*ng!~r+QIAHZ>#Q{HwaDep6)jtx5KcNHk`pXbA^u=_t z)lX+J3^jvaOeZ_6dlo~L8UA8A*S2Edq3cG&hT4a3Y37}LoPv-v;NFx(7+ znVn3$blBrr8b+95uqa_4YBq^Z7!J?NVyH0#VNt@az8D5+%)=YOz-azSOk*B?7zh~6 zw?#Cj#M|Mgh}PS5H19(+*ObH;RfpU8e@I?i9A8u&{>LooBayn17zz>*Mb!}nS>&S% zb^4Ahs*Y&NA|H)%N=uI@s*bSp-w^wlRD4l&gq{C{f2;$pSMjFTSWc;*Biw z@yYn2YE4xZd0ir+sFKiFvkc@6joV^EV-4e)|SFLY#Qp>dQQ|3dQ4 zLgOet|HQF1B=0OVjyeJ442_?~gvL={00Bee_V~~^>c^t8`VV__i127L9+hEko5dx=(s3_=ive~=On|i&f`9?`i--Ulg^3t`3#v~qyDw1~|Oa#3yL&z#?^=p6Yu`>ho#8C|IoRYQm zJ+bwkOk26E2Pp0RoR!hU6ZoZipz#&1$&i2slW3MW&u;nSDz=qpROK{VZv?r_00e534e%a_$JUIIMyE} znEw#mEtX1P?MY?aEtc=yEk=(}fWTesSbH@B@K4sbUoL<2=}Q7 zxaWZua5vcb|5J=peYaRQtitO!JwX+B3mZ5u{4o*!sH)uFSzXOm$+}YP;uZM)fvyxI zAZhmv-x41hP}~J92-;m0J4l=#6IwNQ5GX|gaKkH}ByIu*l9*aNLx7y@WM#v{I~}(E zc`CS!piI-W<`>6ZDRwdDPUzdbd+_2lxKgax&c5lG;9A0qS7co&*1`D?WAv$*M(@D> z-y#6hSBe$4;U~55kyrSxdZ*Dl{v?h5bVQ?%1})9{SiJJIPEPybmA+W;(~aI;bx(v| z=?exji90vi^N)t{F)&s~ze=XBZVj#?k~NJl=-s&2)r(G%P~!ky>#b3}Q;y#1Ds1RS zz5#)cb>_Dr*F-t;8b^VOWeJ%p@@A23ROC?f7q;wasl z8M!y1VFOs>)fm_OPUUL6lqvDH`E=K<`ixE>w-a-5t(*?}&Hv@H7o=hCr=?Y}{4=1~ z^Hvdf_(A*rr#H`?s5bwf*C=vD{zUdxJO}?D-jP2FebGr?C%o-BZiU;Po%kJ&(|$j3 zlFP1-=zNy%Bh+Dv>*W?ZajLtMZg(TFTHLu{vlC(Qj04^AFJw06sz-$EhnArvZEYMDm;X0 z>y93!VHCrc8MnOiHwIRB)y;nan*82|J=}=0!`ya?pgPbs?_2tKpc>$r&W5+lYIh6Q zzk_sJ_!!vd;Jr2a>1+RmGp2cAulA;#ecEEzKTW=2)kE!a_Ng!OjcbcF(S#aBQH(Us zmjCvUywO!dRzd^~e?koQ{cYK{1 zY};{=!ovH7h#G7$XyM~+=ZFVh2ujQDy0n}WmX^*vR~K)^*;4++Y_Nv+TVK2ezkx`K zwMKCLmxbgDaYlUX8u2zb)q)RZT#Rpy5!-qGheo^y^+71q zgH{ZG(x?84p^748^qf7x-27=~*g1}kLY6B6{Cb`lw7`oH}*9`mDM?g9C_s05WJ z6US_S7o>E|OEHew(bsW|zLrWQUijUIkm5LI2iJe;m{+u8{NrAuL5tSiu~tC6!bKxJ zt?FrxJoHRCjmX9X1DOQkj&BIWS27{W;=_*HMf)ca&C(>=pAqe!1ar`Ws5xlLn}afU zQh*@Y$@ZtSKFLHvU%-T^0Iec@RwR5w9J1FXX;TM9If~5Wf zjh%z?CL`T~dsK-t2h9)jLL&STj^7_>I!bJuNCMxk z@y4BkkBPacL@xRnid!{8BtAZ_cH3X=!ynWM zKUw&y5JAe z*n{C**nZ}%pu<*`ys#B z{Sdx6y?Zb7KOIx59TS-f_ns)AN`q9`+pJ6X#JY6Qh}o%d@Am{^X(mL8t-Zez?R_Jf zr4dNcclW+3+WYF*+7FjRzRa~F>SeB*jf|6R|L?2w({{*6a`7U8KQC2hS*nU1+(^B-#C?j? z|Bk<*INXQ`8`>xaC>QrBQvW;tMuaQ=>nF2)KkmO_{ZyF0jxa*O-)8$89t4aMeC$V%Mk7~(dH z?XvS`0K*t?`aeB(;x6f-l)i{-skawboLk&yZ1fZpHx!#2z$$UCV2Il&1}N9xXI$;E z6L%?wSbgi#Qq|M(cS&w>pE<-+Ox#dxu6wG)U0xw>qnP8qlTXe()?+8`Zm1jeRj*4` zxr4hj$GC0%6ONZqZ0?w?#9b~SZljpRtb-dja=G1uxXUoJ?`Gn=!ol4wx41to^IvD& zfnswvOeHRM32__6TyAi2Ka_>LdoQKR4DMbI?y}tCK5M?Gm~jHdZCY_9ZZ-&U8^r+S z;y&vcV z#l#K8Q7?8kiUG>Sea`hBJ8@U^#Z=VzyRU=0Jh!+%Q|l=vZYZ`3x_upg8^r+S;{HsB z$4=b6`r)=~gS(%DyCTQ9pYaqEHx##N#r~-cqZpuE+~*GV*onIm?>5r+nJrcQ9o)Tg zi~HP5JjKKf#rDl3?t4{?Vu12+|K4LK?%voLt2F)|;NY&zE$;J{dWwk~irci}N;SZV z6QkH}U`l;*-j_Uf;_idRO?{WbQZ>-Q-8;9q&oA&46E_r_OB^cIKnJ%`%oSHBpPb+7 zu@iUSL0J1R{vPDu?vrEOcX^758;aYs;z~8h!EF=+l!yBxkDa*t4aOd+!9Cc)t@j9W zH!r?mucw%}q1asdU8x2;xQ$|fa&cd9tH(~<{qfqCeg^jt2Y0{R`unqWo?_yL;x?_g zQVnr%8^r+S;{NOwkDa&&;BgRrCEQXq)WO|9$GBhe6caZTo4e>L)ldhwQOsvlocR7+ zjmJ*h1FP_yz43RIgL^=3aewYxo?_yL;x?_gQdK#)jbecEa6j#_6ZfEMto|9?)ede* z&E>rK^M`wii5rUBwBkxt?cg?w0m{Yw`HMYv;vPH<2VM;BVGiy=xyAj3?w(@ehT=A@ zxKa&sa2v${<>LOrp&mPN4;hXFe+Kt(2lwC{Qq^Qyx2U z4;`VDey#9QHNwF?B)7P~c&4YAxS`m*&)9znu~7_A9_~9mcH*w8!Mz2>-!%?yeMmHS z`TI+gJ;lTg#cf(~C2lkc>u;kNpj_Nv+T*bkcXh2&RR(vhgS#rX{{GUxJjKKf#cf)# ze=xx)1}GQzm&beT#64`JQq>0cNC$UyZgGG48c#8CLvfo{?4QgtiUG>Q{i4TC+`~uV z3Os{*l!JR%ZgGF*I8QNgL$Nu&=byVXiUG>S{go>{cH$l}8aD$N+@l@b!*h%K!akm2 z;)dd=SHK#@0OjJo@Gy^^xNFAX)b8y$@7WY?Y zdy0u0irche|7CJUF+jPvzj}tpPTZr$<7PI4d%S~tWR7va>nSE~C~nh=E7f=>e;dUB z<>LNYlgCcnqwDazkHKB%;2xD*++VxFQ%u}Y+@=**syYX^Q4CNX?tgmh#61RO?r4L1 zf`fZ>ZgGEoqoyT4eRz_zZ&e^T^FiLF?jbQ1alwfXvP36F>;X7t8s_ zV6!i&5(W5E5x$n?7bwT8V{+hCYNcza`3^B6T;(2HuiD12IV)AY<5jIVNmOLrEbJe6 z?DSa7ZuZz5;HFZ+eTkfZbym5@HONi2m8!wPtrZiLi~EvWvT(=jW{=A)?o0Xno4;!| zexkfNuTo8PaBIa$Mu6k*OYQjwy9=L;+sz)IW85!f;jWt`@64-IlN{VyakALQeHqXH zgl02NVs^9Za*O-2D?G)_8WS2-Tb+sTMhCZ6tY3iU`1`Uyd+fv=vzt93x46Ht+EYy2 z^-Zd6g2CP7;MR)u#itJLZ^-=*PC|*<&92Wa?r+NTFHXN~n5^3B4erSfZml@k&5x6B z%JVM{?wH-|h8*L*H_P7>r>M3DgL{gDTPseM`M597^Vk_DF}v9lbBp`((>=wElSxxm z+eCwVs)JiAPIk}0Ezf^A{*Kwro|IeMS2SkfZk(ptCK=q*9Nb!Qa5`Sbohx>E?DTic zZgyjiasR_pOn)~`S8a_3_jCuhR-EkV`}@kV9y@Wz>}EIR7Wb7r{}v`-;+{N1wKW;s zGaTGnak4xE_g_7B;*QzPo}63U-#W%qOx#muspC1On*2JT;a?DTicZuayX;cfr` z)ZbUl^b`~KjC5MHO($?X2$pt`Yt_j<_yd_G6JC`$|4qj2XV1tn^6dHV%yhGAn<2=X z9pqYdvagT)JM#RmGxo&nXwS?o^6$v~A5M(VO1G%CnS#8!12F?-sx za*O=Cy#L70;k06=cJ`Z7;3kvFI3tG0AnpwD-pYu(8~;3T&={c1k{AI2M@$82sl=N9@k{`g1e zE$Ib-jwKwO7Z*6tweI9#{K0{KP3HbzGH!pnCAZM8&EEf;n_dX$EdqU^16}J*a{0-D zey!YpZosHV%o|jwb*JY3Y zt$gP}dY(W(#DT7LUX0LmGj-mfM3;Kfe5mPbOX1jYqZlV9+)-31? z(}x230)c+016}J*4$pvY*MGycxR@R9g}H_P!|hqn7p0d1`a*%e)Pb&bCr9|ue<=4K zI{_23>Afho(0}C5e;MnCq?ZBuB7wflfv$BYYkcTG+Tt0{fQi}nJ|xG`U-A?a`r`C* zKtDvFFL$78-N{-X`t@@EmE-l8t?$LTg?{~2Sz6 zYWI6dZlT|hJ^w#6y%Nxu2=tW>bgerHT<&Ok!?!%+889&$;D_cG`j7j1iWx9V)2jge zP=UV6fv$BYNBhu!e55CV&|~(%m*yC{-2W_*O6beds{wtfKws@Z*SeEqeCRj&>mP(3 zvkks1x6p6=lIJr*U!Fb;(3c7H!yM>ZcXDh7bb0@Y(@kS`!k6b3`b`V7psz?D4(Q7T z`r!_Atvfl+hkn!NJqh%B%x3tC+(N(EU;n1pSEi2u^c4dA2nV{>ogD8&zqvIFddz!G^?26yQ^?&Y)4~*XxzeVnU0nHnFpF5Ac;`I;`wJZJ;d;Vo$uq&R2$!L<~ z_zb{TIleGG{R!89xGO#|W>@?tYdrI|gDGxTe02^Sdjeq5HH4O({|-wZ1;?%y#~$T4 zR_o5=#yGiU=6q{r{(}p!2c00l|Csk-Ie<>30{zzP{C9ZzXh1(qpdal(*Shn7%Y&Yo z|ImI#kxzObo?Ga*W#_*m(#HV$;R5{_2fEfxTrTw6GV>qWvKTFDyzV~ieMD}d|1>-Q ztw|pX=tl_jV;$&PcOG!L(0`hl|MKE?$k*f;dUpOhGJPDNuMz0SIncH4yb>S!&oc90 zUfd@6k-3Hbv+VqLRJsk&j}+)_4s@+MuQUUC=Kg11+*992Zl>%`No*^4Gr!{kZgcKtEQX zuXmtp-FZEHum9Jzp7DepvuA!>ZlT{%Qf!^Uj*Shmq-8j(i9PCLT^q8IVwYi0U=OvzELT^uR1oX86 zeWL?i>&~mlfc~T>fzV?%&)ahg{jL?BVnSb+-UR6F0)3MMUF*){{<-7zyE4x|<;Crv zugfj;Uu2(uTAw~1(ANp{;~nT)cOFY|2l_9TddAc1FCa}51{Pcgl|A-x&U*9-K` z4s@+MkGtp&^!>R1%G)l=i`zxtkXz{c&+!x!dPn*MK;IzHPjH}X-Fe(bCv=&N?7tTh zI2q}iIT@jiC?og3=v-l?K966jEpIM-2j55_)2X{lAcfPZAraH5CeT69-F1R>2rrSs zo;dD%iMmg(z`IW`1!f8mxOemX6X6btfqVDYUASKZr~2k;U%Lt4T)00MxYZGGXM>h- zpBA{)`pzq!_a3X^)&gGvE>F|XDe)$}OHRa%aD{wnBhM}C->lX*5>k@jzK83-#9bYO z`<^8Z?&MBzs%y62|2=$jaNn~-aF2+XtA9oGu{k`5$BBa{yUlZ&4bGcV>rhe2|!$v$UoZ@z%Y4Jac*ZTp-E6 zk10_F?`n3k9*ZwH``*I>c61YcJz-x>)0$rzcRk@Hr)zjR^iD@GmPTGrc*)6G*Aw1* z50o)1N5q8Xy;%QV1YqW-*019y^~r{fSF|@Wp*o1CbqQV)myzdm~+iDhzLG)pr|mm4b{sT|wqO z9y9wlm<=9KXtF6ny3ZqJOqbf29>n*+a}|Lfow)Q*S|bCRA}rjOJuwXEKuiTL7z@M% z%nsc9?fI9ning-z_p_a!8flg2xqq3b=WLK}?6O<#|Dwmd9n9l1-6FAb|4%{6pdT9( zJNKjh3xYmz24TLqa*auE^ zT1e?pzVJD|_l#haPqFx5**8iR<%h#n zva6%~gAaQmnU&y!1K&4!v1yXu*!6FNh+#JXD>oX^sj6GI5K%$+Ve zZb$inT19Jqx7bnsTJ0@8%3q6M>=r%BeEB{5hj)eu(Qo1l|$>d@hVcLV4&n0=y*{<%6~VgY8c;oIN}w5ZNK4Ji4gq zPZg>t95$1T4-Uo7u=()O9y3Rd0^7eq>5lRb+vlHK^a#tCF0(N`2r(TD&jrCxdZ1=N zJEQ!=4|y6H(1955<#QS?nH{)~aQ?@1WH7Mv4+4qq{1&db<&h?jl%xEnF1zIsj(?$^ z?O?8lTarwBR*#G!k6Z~-27OCR>^y??{~&hEC_nd>8fG&m=pRA-t7B)rj-AL+{?S4K zH9r{T3lq9Q3ll{Vqx_?G|8IUKG8yF`<^CT7c43548RZ{6U6d{iMtQubkj>H)FGlQS zlz;SAFw!pzz z(Gw$A+#DHdDNbgmkN?CI$x%M$U2YJGyhJaN^EQ=oHAI=)(pZGGcvOBGe3Ckyb z?1ZIR$-Cg`j(-EP)197p0#cZJj*1A&CeR`Xo_Jrpa#Ya!OQL#zvDf=K!clR!ZFsKVo*deGRT zx;rsj6b5P{{1LOcxV{w~v#(bl)SEH>u)xE|aP46)W*(}#JK^f%L%6~WH(dX4ktd1a zN=f<-a{r2F*d**uxc=eMfc+$yt5@p>XIt}e={_bydOifc_L?w8-*bLMxOR^Y*Vn^v z-DFzjk8FR_DpuJrt!!B*%038{eSrEAs5ps6*+@{~^6Pu7baeW7PnVQbDPi|V=0Aqr zu`yxy$KN|)r}lwUoipV}_u`uqc7Npf$6m57BJ37}7Gd|Kt^d{qoqxDCa8hWXyNdiV zyc=2s+}4HgxCy9%A(*WU0gCSo#OJTF`gI(CdKH$tiavl0W&?Pqk239$coof zn>=Rr-x0R|f=ZJ~>pt~WkkT<5VjT0-Ev{qqZblC^>KI6I9P`wZkiynIA;K|DphfFG zW#@k<1UaxYDhHN$Igq0%1qj?fH4EGmV&ML1odeh8z;Bm6kN)Dq{nG^kcS{6Z$$@|R zA!yi+TY?-|(*uUEaNeVOI61I5Oniy($Ais*mrklas2up`#U4J!S-HLBeW>@cG#?+L z1!lN8@Xr@}k{D`~R6tAp>*7rgB)gLX|IG2vILfA_T796PHNQt(4!m@Fkd`i;jvUy- z$x7kNYA>CXl>`66_V<;I)XJ81qHI>EY!)&GQ0WwykpnNCZu7^b(=&76Uw#I)>?K=b z!tO6mIbo;v!9%Ls{E64&o6}4F@-d{am+XuPJIR4h4-*4-203t)HgHmCppyeHZDtM( zEDCerrL(ef;L~S_5j$gyc=}te5jqFnHsF$8z7bDz{f|cMjxa)U;L|UQ5xavNSYcyx z5Mnb(Ud6#rItMbcoE-R!-TyRV7?>2J1G56}F$kD-4kWXa1D`q3!@wLk8YH^^g3`@_ z&v5;ht-CwMG0$NAdl7i_o=(wk$$s6A;T8DiwC*#15yzYs;TZN8KVD|+=rOM{29ti1G5Ial8(kS1YapXSjmp53 z-hK%$=_^(!(W2-2hY3prMbX`^)GYLx*+Kt&FPFJ+En;&l`l*Lr^gKV?Bc+$p_C==^ zsy6h}C<~H|HOBPk?fSp&JNhlD%{FpN5O&qN<3O`!kY$kP!}e##sRFo3dibnG&Mh^) zM)w}oqg2L}B1qt86#tFylq9|XdFKDnPvyWL*ZZ{v1B6w2|6gtWZ+bu3JMaB7z>kjb z*FAw++xySfy??mw{liM-nO-FMXcq-av%b*V6Xnnub;9#5D9v_Wnmh+5ipl*)+&UBaBrqeb;q~&J)4#RE#{K zO=6lL{Z!%kqsPMtrb7LOlfg`@;j2M|VAkV1w&ew1Yyi{u3CQxDmfpPd)hB;X+`g-j*e`A)(8fqC{^`s?^V?Q z+Iv^34rpXH7;fL+Qla{Vi8zQ|GRolg>YqL4v%x&keTkvFA_&jxZbw#$-oFj-*x9|a z$^fzTZ>=DucdyK}RT}kg=>I{ZRvZZ>>gXTJ6Ys$R%)fmBQW(A0MmR)l`Wx1NwL`9@ zLv+3UMP7?CoaH9U_4%Po`p_m`p+i;~u>Z#WUk*6es!hWFC4F&#zViU(`k_*FkdHjm z@r9uLdu_mH2kdhbq#2gq-!0esJLS4WcS@v4z4!O+fg~7E+>AyUY2R!!R`n0Z>pnPM z+I#oT_N}`%cWm9=c2fJU-Sak_w0iTF4%)@O=G1$Czt59;lPk4C_0Mn$nHeY2(ErHu znCsEd^(3K7`5~DMr2nw&V_o!d1L;3@c-roEMO3J<8Bm#yT&OpA z%*z3F8A+m`V*W2pO-B>zq4sL(erjAMR6U4TOQrYybC4&3Py-PaYFq|XGCNTJ$^LIb zjrs|7IY_+ZZ}pt(kzNHoi!m?iMXS6y%}CqyWURp#Li3+?{-;B_n9NJ8oh1NSU1gcb z&?HRn1{x91#XSLTx?-CT&hM!=6EOqjPfVr3_&72p0;Dhg%Z~r1FP1}dTwm0{Gy3Ac znE%4QNcPVA;tYfRUpEJQwlAKi`{Hoj7lYy%9n*{it_cu9F|8u%qd-)L6weFoHTPrG z=rBe`c2+#oE((;^e9hLs#x2jmu0Uvo8XZPW;1nlnUi*rNXdfD8G2}-@&1<&))p~UC zd=PrL{~JVSfr`>2eg1X3{$=#+aP8^mJ1OY%bcs_uzi#VatDC^uYtOvYkEeSV0gox|1cmEVE%GAHlTPu)Kr?ZjVzYUu74OdhHYS8*^ zTX%2XiXm!N^RCV96V`9rvvv2*Q)VsNx?yvBkUI2O%=|*ZEYsnEH^csq&ZiSskX?(B zxO7G)59!5gZ~FP4d22@IP$k*vO)vlJO5rPMT>UnT;87{Gh8xW-EmBL$g{U@vx zE=(^~ayt?#g{6{N$!F)}upQs6(r%<7a1%udqr-nu*x#q7k$ znrB}!(@)ie*{P4RD|`BaNvGqz^^7Y*ua*TOD)62fzkxDV%id=FALNKa=7_l@aR*yj z*?W7wM@p!zSO=;iX@6DgLJd~-^ielwW$*3JyD~J^K!&%Lq?@ZRBs0V%Fy6l3W!6~d zfg~z#zWt_4s#o@g+jme_s6H7!$ynKYXM)GPA3hmF64xhE>AiE5OR87)0zG3@AJ-?H zReF@}RC@2&^)Hid?uO<->VeVQ^-s-Q$W%Hm6TMWt?}qV@;45I)pK0GHHB=3E(U+>? zAPKP6V2n_TWT1Dk{}+Z)QG+X~LJiOGcSa1p`)!xG;H;<^)FUE(MJ!$*}Kd*MD&tR-3>q=w#UUL;GPuQ8`TLQigp$yZ-yElVOAU zj~P}zL7J-t1ZLP)d?94s&#wO#*$s&??uJBdXZ0WLqCknC|H-cZ0-;_tA2`LyWB-#~ z|1F_>H@{dtw+F4+f!3t|4nj|M{g zUj0|lfQIfu%9kqZ1F!zmiK5jeiee-RU1c!`kbVlj;E4|_-N-6@P8900Z;%Drjprw?Cn zR-Y2zg%tJ3n74c2o3nQM5ubnN>eC$&t4~~GfMY%yFQD!S)=ocq385V8R!WJ-Ktvk7L5e13*c)-j!*Bi2B&(|2x$-t(tW;0!lul;%E+#AKU(ar`jPP zCkhoQ#AhFK{f`Ro%v30=h#&vRX~)vHC}HkBZC}GTryW0j3{vRW{Sl7Einqq|jyPez zb{f}aRwNRPyn@6)@fk_04EZM`vdCAe1nnzK4T$jZC#wT`I%YWtV;u9znE@poLm;a_ z&5z@lPre5#I%a>SVM`4tFg^F0wZ4~HtFZi24+x2+mS+rYRZv&bOTfPEKu_k+L6Nf8Dqyi(RQTNm&A^HBkZV#FoL>C zRNYHE3KdkfCK4>eiV|`{0zZ>@m@qt_#W3F-D=1F&QKmu=YOy5pqur_fuzRdIW|&HKS9 zl-7hl3b6ZV0;pq6K!-I=cORQh%N+-5ejF2$BgKUKVoXRb6cc_OVS?#)sU+K<6Utxf zrdtaW#wQXzps66yBdgn`l9y+Z*BOfoVR2S!OC^63(6iarLT*gBCT;)!HEjlEQWvIR zaBa>T2zhL_U&q*-XZODzjIde4HE*qedXP5j2|?%Knuq-_?WB<50O6X)@sDnOkZlbW zv3BR#`QL-ITZ4%R*L_y4n$WAd1&RI+c@+epGvwR-zmR;HO0a$^O!bQhjr^Aa@eGZ9AdCqOq5@n0 zhvc2PsNh2}BqZ2GBzXX7M;tH{>BA=j5pw<|-p z%hDxdQPFyF#^XA&(m)XtSw$TGnFSu#`H8lC1=RdFov#Sz-?iHwk8xYk6XLcfBHU(n zf>TBBi>fEI+p3^yiW!S~q;QbTt{j;9nLNUW3fn*o`KFpdsRst72tIK_@$ms4W2Xud zW1LX@)qs+*LuAuH&5z@R;-7(vvGYWX6N=gY89RT}PA~%aU)GFfhL$R}>mPqq9df*2 zMC2jIV0uT!I(#AUN~Q}=!_1w9Nk#H(a4$X^j8(SX9lNkOunFXP0gIewewQlY{D)0# zxY^WMOOfTGi1S~X`o~OD&AcF0a<4NlNZt-EwY}n&NAWG37oqNGQD`phP5S zI1@n4j}r-{_kfD-c{0;IW_FM&{VND)`O^`RATI1y2pXkn-h}iw6ihcC5-i0RjO}Lg z-_r~i%rldTbJD+7Z37bc*F=^6HSP&lBN`@lY^-lwKXrV4L*u&fQzq7L96xD8$CUB) z{A^f1ZNsDuQ?X&uEB$?ue4I$P>;Fc2vno_;@qCTm1)s~uba4C8!X<61*DPH$uWixN zd97=xZK@VqUk~vcCXQb>Y3j!Djq5kGQ(Jp|dt<}I_4W0EHfplA*#2*{tpLq6Y22_k zN^IGvZJE-zaq9ZU^^>}6%OGP*na%%3dX?&xc#rmC8_Il0C*fUoj-OdrjZl*6owB4iMH#~`xNoh0 z`YZBM(q@(ZN*<$N^2GX%^^NN~Z>KUGv(RTd8XK6lP17k37=cJz71G7vgc21Z5Yx%`IWxV~0meg9fB8lDCz% zsPsi$Hwcdo!l32ND_1XCy=d7|jqGZmq9$w6e|y?!QBBLKuxOuHbgEcHKF6Y-eLX%} zlxzk~v1`#$;6W4hoa@ojqJilP84orzbu>1% zZ|tgYDG(~Y;R#=kiAD|e+W;gP>mnFgf$?7)f_*LZIu?m8QDg8^b^aJtXf}kZ&;t1I z7qaBTe^VHRm`A-*y}bUPAQz1){3_EpKk1AwTRXk!isL_nx_KM z;;CN049xG)<91&5p0u=BFMIw`qe$RdMRj9(yA*9`vF!(AVrneRqu}-ULNin@c5x)i z;qlvedwngvyg;Ro&O_~4v1-MVhNk+7_0uLcG)FQNoj=6l5hwHA5Cb7G( zUH>q?#>6LaTKWdn)?e(t;b81G(jB||+VwvpeJu>0N(4dJ(Qb?6PS0)@6G_-}^vlheOYZG$NND3$(2*GIyGDb$IA2*k#I zS9vxv%C6(J@1 z5OFs4Y)oa`AN5~Q87KZgEiG^UWTvQ*%DDd}kiyFNxroZx!r}fu;XEff$E)V8AddeO z^t=|-lj`(`f3LI)E-z zUH}Q^K~=6c27x+1byXFoUf=L-;+JI&3U4PFb0)FZQv!8ZRH zU7T%uB^Z(F!G-!0onK0;kV$hjl5cwQ~hK%s|Xiz~jXv)aO z50m7P_(B5>Vg3ts(4aaz)Z3e+v-voBR}I3cLK+m#1+Nn&(mF$a?kS=%V_MFDG2azq z&JbhB=Nj{d$46rdn?a+i7LdBe3?1yz(wGT2f!oIzgA=$1YfLZ|yv8?X==q)^8Z)-# zG#GQ07;~B!Lq5ltp|<`tAxdM4nn6=SHh$PL$KwkvGxTqs4$RgPi|kFxhw>)n0jeJh zq$RDZTjz0^?SES?v13xF4B%DkJiRQJPEY?>4q~HB{IMRuyVPk}j8Qrz(=u39WzWAD z53EH3pGF$^%X~;@k@)f7fP(&R&VF=}3?yduvwm6w)-TshJ7BqVwMb^M0wmRkcy_W} z`c!(q6k{lt_IEv85LR7Y*CYxekV!`MqaKPCSVbL!$ANS}H1L_Fu7)g2a>}M$@%6BA zo@^FN|0a!|ZGlYF$+$6W4QM&IyrKt}8)bk6kuz=#I|D=<>t2Z&H-=&VZxMLR)$ROA z-JgEs6Okfg-7wC7Io7=zF>Y8mFAJPk1y0mhH@usxt(w{@&K~!OXp`}Ic!OYmH3su= zoc|0kmvVGh_wBr*QKSgw;U^2`zeiw}m8jvDgNCW}@2UgbS7A=~r}P%pHin^%m8dNy zpw`t-ncCRCq0?~Xx*9t4!ey&gqkitzd=(@(>uMuXo+hTBaV>k1ey*S@jC6Y?{gBT| zKO<&&d`v&Zl$|0QKMb5T_(Jy>vDu@gHGk&IRmPHbrb-{*b!%|tF0GMOwJSsiaZ%4> zo*vpVzGVk&xs-}v%MP)He2y(OcK_RWo3@lRlg71$GePL6Y4UW?mcOJ=RBhu(i`n*x zUAF}*BD7^m^QzTt3tO?2+tS)>+UVPmMSWsd&G$W>w5zUVH|+YR*tJ{iBA;v5a~>b< zDs2W$Q0TIaq#CKU>tA8SJ)Op@SjJNY7TZtgx?Nmshh3{y!=zTT{P`_WBnH)T|33_G z8Z@D07YzD_7_>_aBA;VWE$Ux~j|O#X1`VjqhXx&uFEmu`GafA)>X|fNi&ID1IV!!m z>jt475rbBnwdpHGjhHiXk*A90)VJ(}IhTn!JH;IGIp&N!+vB4-WzC>TxaLSk7-`qP z!}j^_^hv610%?&EPU^Zjk`WGD)w)t=gnf|UWQ0+6{oiP!H4QCTH%?zJ*6a~$$mduy z%0B-P_M-01pz($XS-cx{nrnHY89nWCz8=~xAFnj47hPKvP1XR2vv@b^Nl$n^W^E^v z#CnyBcY*`Yc{%|FzbCPFwoSF^^UhdA+;%WHQe1bY;Pv=I1dd+o3Fo5dPt$lCsa~SN zqUceP3v*KrYEhIbsm|K-J5MDSMZd`xK3l?j(DRmE>uLn8QWXAa$rmkyP-`Kvc(?O0E z%Xa{Uk|j}(@%w)>`v1*oy!X8?;UZBl|4+SdbyA;LG#2B(lb^U?_5Cz|)(rBgDq8vH-@Mo&6xun%apmo8Qd?O#0F9pK+lCF^(7R24RKR6cUr!7YX|&?sN5 z9hdfWu)04ZeZ6WMN)N-@>$@IotnMk@v34BlKgXAAF=|cMG?dN{y<(9(=+QR7@1IKJ ziA_B&0LfB~WOC!AiHM|*14hzSB3Y_?%*tIPtnN=q-zBpcRQGpvJ@p<;bx#yTAU2M- z&%c`9$JqKodaVr5uyO6dc)M5H+95UP2+l7I(OYv(1x9RyWhE`R<1i1 zM6B=s9#bpVo$1ud=4ivyc^|ES!Biu)a@}_!h4uYEB5Gv|hx;Fd^AE{2QT2Tt=6^wD zuBxFy4Ouhs3DG8)CzK23f5c#(FwvEw=Op7#`(UU@5zG^g5X}FK$2@`KA2I(^W8NOQ z{x)Hst4(8m>pRE)Jp=Qv1oJ=RFys7dV0j)Z5cT3`Z++dzTwf}fUyHyj1y%h7(3}O# zuc;1jUxh`j%hTUbZF(mi1=TlnH|S_=Z^w?q`c69;3aX{7sFt>_F$M1rAdqULpsN3- zr-~)W#FkH^1i3l^A{6qUmJ)<~P6<-~xW~tWsz)8Z>nGo04^k)w5}w%hCrA=c)9k%;4LV z!+PcrVe{M-hb>x(WlouE{{(udOHxmhz5meYW9pgI@;UU?A16SB)blw>J>+vzPt&0u zPNtro&7jc>j3CvW@CbZ|O--kGw49ng#{eBj6*E+NN7rpCR4E*coY#VCpj-0|V5bUe z%Cnvtn$p6wgxW zQ@d^o&J4qpV4dnVQ6k1nw)5YxU7A|H0Ap?vW4<89kk2*dMGqj2sb~g`jvtU}V_15k zU8ZpS_hap^X{?3m!vvVH?d`fTy;M1jiCEgX1*)h{%$mae&rl`XXmZPEVb%>|)@Q{m z@;PQrxx?e5S-qM;qh~iD)n={5cbfHzN6R*PA-z?#4WSBjp{-pv%XFb7%jPa_Jpo#uJQEIrYS9F!KQ1(rnAH*@;Nq5z0l*MO_j}{=|MJr7p zId{}^57@Nre`V96rGZU95?x}`G`s#ChTqhdvtiSUacizh@9w%y*b-#;v7N9CC#zOBK@C+|lkEA2(4=WCCV|({+=uYimtfedKbp z4tF!8P?K0R-LC(J7EN!t02W;@7F{3~klH_nr<~^EYnvjSy?LAG9^W#_jdKHM8b;nrO|emUCdubz;pq zVh#BmYi4o(Cu~!;&46am=$VJLMuyi}e*4p!zo&8PxrQn*yq?l^Ys~N(Ec)IG0aQX@ z$UU>|{6FkEvs*q7V{Q~*akDIXPaP8MFJu*3mqQQLV*WUO7CstR@^x{oD zy?D-_?JNcs{%Swn4_IBhB(BRcR&*y|vZa%IUk?dShy^p;x~U^*DF zayVA7JfHD6SWW-8oFhgr-|zspHAN_1to00 z`Cx3$OH;aIbIYZk^tD*Wm`NIMAIT#5rAONUbZ@6I;!k3D0?7p$NqYwZs&V~+Pr+U* zNcI{#=T>{tx$t{x`fjOSu$*ys*RyX|dpBc%D2PC8oO`W@f)Vzk^aeRw4;wcejJJ(+ z$J_H#o^(doCq$rk;s8QNm~`-Y4WLzeF3yL@9`!r8EDg*%97J5scr#{cU>?T5U_sI> zXPn!-W+PNljVuk!y8=?UH1Jl$(tw4tU*NnY`6g;PV;<&zp|)yT^v|0=<_Jb9g1NOs zFuxUpxpk~7MKApR_p)z4FH!_^EAt;Qza5Xcb+=%CTVuwHaPlF9AHosCT5m@2U=PbK&=j z^f#p&pa}VrZ(Yu$L+J~p9b$@vd>d@Mo+HG@X4r+`e;9fR-4UGx7RbMFBlRgtZa*PT2tzyL!|Gvv(V zprYM_2uMx}DE4F+g(0hm0n9nBu30eWoG^l}Sy$bxyX)#Eci*PxzPCYr`}}=hg?p>I z`;Of*`~Lsdo*t&_ob#Pir>ah!3ioeLS@x3i3iSeMR1{`npI?DY`^w2vnt9)~kU=?2 z&O+9#Mn@L=$(@nMpwA55Jv}CyZc3J8-k0pjTB|*a{vXr}JId)S<76UrCo&7`A6D5i zm~lH-L*>wjPGphlL~5FF7_7PTSHOjx&OeegU<2p( z8qPm7oMfHOkv(l`B?Q}JSn203<|BFjg_!>lg89g07MC8UkW1T!`stdB#(X66pTzum z63nYn7H#Lqn^1<%{kh-i+!VP{2esIbbfa{sR=g@Jar(Qn1Ls)CnX%YC{JR%ID2Sa1 zgw19D9|&W|6RE!x4fSz0RCGL->W+tWq|rVxx89*+$1^O7GBRO^VzS)P_@s;ur!0HX znqo(Dl9#t70 zMH!jMMNz4W&g#rF@n29kw?uB%YiG#n+?={9+>5_BZtME9kY;6d=9%~>(5Sxta%l9V z*64Dr5$SA=F#cJUN{vQDQAVZ+jYh}d7gWxCu73s&z{=T;vp_MK^2XZcZK-R-&FhO} zOO`K)MVHFKY9B;UjIN;hCpw~7K{M*FhC0t`b*|Rxkj_$PzFGfQW}rI7QIwI14HT6X zq^AP&KXhbJojuqoqRvVnrE_cQ>L8^9b>vbha{x|1IaCqm7QDOo?4Jx?fw| z+bK(B?uhKsTgK3@?MYo3)vq1BaLL@6OH5Dw0+3n#+G39XL7zisn_YhuG}@~*x=L$A zI!mL)PdRkdXiOAkM%o(b)X?Hz*=0SZ+-~B2K$99$`vEJV= ztMz>@NV%;}n)WGAE9FL!R?2ND*T4O`^O$nGF>2gS>gtoI`&`sJBJ_lS-#ACr;N@K7pr_{OLHxbpSR#P zov*pO=YF6hXPtCg#`EvKDFl;lkJH8E0xObk%MR4~oNrL3m2_M7j3bkgem+;qWD|f! zTT?f`;ZBV>=D_~T-Tse}zDESkA-)D(*6sf|ot)oIUqUcbbj!r;x`jMh6`jFrT4IZr z&s;t;9#2Pg#on4y9@W=j-vFmOln|V3 zkanx14s*(u!E}F%VCh<}nxPWTmq==B%dzRQrSt1~SP5JVCTBgRJj#v#Ii-Azd31SZ z1XD`n(J@s_sfDF8r6dY#N_q4sM@KqZH?H={ehWHoNyplOj@r?V=KhzU>Cn+KXa{*> z3qYhp>Xv>q$G@PZ7rHH7=Y)>t{x?p~J`b6m9sOU{^ep2o%J{thbvyAvF?xEog8t9x z*%wKsXBy6mLo}Q(leQtjyD?HigQqgHuzGWK<6NWdx<#G zWP9T>va@wbQ~I%l(mOnlOF`M**r4JPNVvi8KC*4;Ik%bWEP+@yxDAp_#EEF<%NC@Q@n z5_me!XGWe7u|ugE29hV_K$4Ob85NMb6|t;-)8XELVf!$aah3`7u)X^4PFW7y ze~(~;qdB8;fke_+UEYFdu=9%)IBY+lk#Nw2eUC%_&+--y+mACAmvw<*8?iX7n3A5U zTU((Z0Zab1b3u~*Z&r(8R>!^qASgoTYhwV(-!MR32H|AIi%A&;s z2?KPDDM>QHu?tWL-53gC87vEW@WqyHxZG zZ2V|1x(_8lwl9@f{Qvgbm{^!*PO*_06;6O|-;Hk9o71dvmQdrZ@i?3qw9)BgqaHf@;4 zoHfbzhLJT*N|07M)MI{_WKV~ZasRt($+96Hm*zk7loo3lY7@e_FyQP&i`3t2lw{4Z;sL)2qpV2aHYB?{ls~&B*t7pDfAl2qn|; zzg69b=Xt!im1MVvl6^lx+RR*eA1}#n3nN?8ErINa94~sSl6Gq-Su}xc7HGL&+XWAe)ov@jeIYc1tK3&i}Z%rWC&0%C~hb55B^*r8A zK(d<>$aD<9*2I6o_%cs;@-mH{+4ykiNTcU7YcGa4hCmr1A<$a1e}7h#b2%tQ@p+$2 zzz4e|J-=Mb{D*2toL{ok*dQ#k=a*~$$1k}_mMqAgk59h(G(kJP9#<5B5QseJ|t zFy_im5-REGc>5YHDqDt149+9Do($ph@!?-jXU*W-uTLO}=6FoRVY@-3Jt@gM2_*H| zp4>75lC{Z5FhNk(3V_YX@*-=Llr>37DiTNzhdPK_39KzCNm~NROypBr6-jGS5;Oh> zz>e^|NQ)wAaY#nvkBCQ)e>*|SEa63(@ol$keE2jhuf64jK!18m1KvYXIrxw-mZC3! z`w0_$?(`N`;cQvP@s<-7p%i^MJH%T~*n|S~mfR%XGFFSaT8qk+-onX6R)Hs{S^UUT zipabeDGb$H2~^qLWzrh%AGtq-YMqxrmDA0WnOIQW7e-Z+K$VNi=O72Fd&8*KCs5_( zc@d6rpt>iF>b3-`e6$@7aiF?8jOv2~s%~gL9N|E9R|wVmUI|p)(Sq(&RCk6^%}k&w zKqKP#23dE6QSC^e>VbB2yP~>1jOvjDs-9>{w<)UI!l*t?pz4Lz#K8>+Z;hwYVOodj z{{ufnx(chI=>N0vVUv+Wx2X=(|L5|7FQORl$-y}dy9e$#-G{|CMMr0^%a_#ycifIr zY*V=*ZK~rH6kwYwkZnpa;7|6@>@d-MJARKcLYCnqJc5)hzF36!R=n{lt0hYIG$GEi$ZlAy#e!|Iryi%!>s4a{PxEy+kd7D=rLr&d0K-WNi@)T?+y2% z3A@E9yxC>0_yDkgAW*5tJS9M!iWi6kB5}q6_Mh(duS&G5gTPEpJ zxd>1eUjmuLv`phN^MEtjAq+g7i5XXDBe z^xxv@PuUBjFX1~juEs3ALc7VEo#eJu zobz6gj@=Rg6;L@TOIz-wB1ay*ZC*Vtxrw}}Eq9%<90^Huz5;sEe20*hTM~T*bx*o= zBKk@?mP>jwq%vSuDh4Dbc9L+slYZmiq5-!@7HY?vDaJ0;(!SYCE2MtRt3o zoL`SKBawaDj@KJIl8{7aD7+_|`H!k_+Hq<0RUxuS*DRv1rejCzrbA`D>|*rHJ>i16 zM@5&!x7(ej-4e=b1!wzcM-Q6s6rP)qn~1RueNj5*i-S`@y`&&*wCz_o!f3Pw_1GdF z*{_XugE1NjEu(FJ(;=kMmPKFFCqZDe*S>+#coBy(T28t~J7uK`p!3QM3$e z@@Gfh&|AF_*}ajGGGjWdG7@=j#~X$`^>+?`u1P-;IZm%huNINx(s9QWSEU0W1fdl@ zjrpHoFypdxLF7c;E}-a%O3~En$yT&oDSF!b;2+F$7Yo#h6229msMtb$$QLJg-)Nru zi?)+5)jTM)AKtL==|@`gTyG5vy#7j$TwOr7x6?NO9ot*)B<)SlTu;ABTd233=R$+d z=DDZuk1Gvz!{@oD|HUuOc`kX=3`v^i8G}%gO|y4=)4Ui&(D#gkG%GnsffX3#NPe7c z<>VHF>p8`Ng+qS@jeMbi{>>{E)6}D3G)DfN7>RY6+?0cV42a0T;G2E;MF`^3b#caz zth&hIj9A<_uWYO;;I-nD3VdlRfQH*QqX-xC%h=ePgNyn9tzFCFV)VgrPXogN?b2e% zFBUg>XL)sinG1#U+dEoXPhPftV=MlVUB7I}%$fKltEqWosW%N}Xq5aDTDQ+@?O3yQ zt^cQ-_ASvZ%|VgD?OT?2v~SwhdUX5NwTExpxURjqack>>ZR^%;ku+qKXUh0odT!X< z(z>~|WyXee+tzoelE6od=mpBlYg%h-TRXp}LP&$_=D%yBV|4a&=S1GrSBIndzp4AV z6d0@S1r;W{7B)Ck>MpbXtwJa6#D0QLM3cK-5b7;QKNn*6w0bp)rAV#TnX??V*sJx3 z;8}(;M2A`@rQ53+wRFF6=3U^=expy4exq2aex}KP_mSf?uO{9-)r&}o{=A%J^)P*= z@&7(@nEp?EGYB+E`hv5@Ksx(^KJk44CjhYeSw}ePQ}mvx_=*kso~)??odK-d+EiVI zQL)n}OUQz$b^amMh|JL<$tC5?(M%QM0jr=s3-N|dXJh?87^plCe>)~JOT?b-8uU-1fp)_4%>Ix5GRA5kryTxJCm;C9lDhQy?Ee_En$l1=JjObIsb3l{ zJD>r4I7BDuSI<8UC0S$r<7;d>h)|{H->g~n<6r|;U__X7*^z23VTiMq(10(p^eVz~ z{Eqvvgt!84GBf@!nTZCZW#;1P2{PV6(~GRjmfdDkCGHWeZSC~l+hWvkPhMUeJW;Y3 zBD`hkb-c_N65MeL-~|`^+A>7j1ddxvaPuU77c;z)u6m3m7bjqj3%>BBndJR8N@G1O z;)sw5K1iRkP3Y1A|mHh2yhZ-7(dq!*HXBXd7bP9&@GhM_Mq`G2MD9>B|CBIhLKfBIWO3wW!b7I~ri>D*p$I47Z)f5S zlo#hG*zBn28+!YAn~1)Vj?I{44IZ_w%!WEmA?-#q4I0r64kFquD{`8)8`8d~rDL~5 zL@*Qoun~mv6NFtrCyyS3)knVPL^i4Pe>5zGQGab|E z3g#owmEE9aa=VfH&Jw;f6zy-9u>WUo9r#;zDVm}P673@{q53-BaB8{ zQjc@_kyo_QZZSq9A&qtx-mvziCjL_>X|xs5xAaviYee*|bc}|R*7m~>mEp2FF!(H-uX$(Xu_dR)Fj@N*OH2Nz$D_GZB+t^&& z+WE6Ll#Y6!xZD~4@pPC>@s9{@q?8dO28s(aB$QMb(~yD}>%^#@O4%mDWwE7*X)P+G=5{BW{@SK0HQe%<-)yup9oix1R=$qIgp^B-RU z!LFr$uw+-Th019v;G^fe)?cf?(qZa6BO&3ma@9?8j~~#Wj#u>or6!HzK-j^>}&(p zduzPm__EX+t{9g!VNs+hz?{4qy?oG+sbOr!zku;GRUSA0m3r8o2XC)#ZE3A6ZTwos z4Eo4a8{_)Y1jNt63T*WJP^_z}TAN!c%yl23#vmNa&eva*!1}pq9&Xf;tkrEg)fF`@ zm5t4TcO*AQ*1$N{rvD4f@%$8z$seL#Cs9|mAvIrK7c`gAH6dW!)!^waNbltoKdy z5c*1k#1!i~EN+$hi(8?rySuD!H2Z%8*83-T5l&8F>(z>NZEa~)McdaoUpAf3@{k)J zNbr!C#(PL1OV-O3Yg|E4SyLVKO(7n_t&JAzoAMG^zdTkQHv#Gv#kv9y25+nlSciDX zfH>AB{}UMGJ0rcwDusHLWL;U^+FVy!9ynhJ>bNFz(+dfxUn%i$sg^XzX2m+Si@oFp zRF*+*=K8nWHN84q9od0FS|#g>meQ))ma?zy^%y|gthXkxer=dKrvui_igj~Ub#ry8 znJb1mA7>FZ>vt1azdppnQ?sQ()+^Sv%@w7k<(+R%LFYWyWqpe|{}WW>8-qPWQIfT~ zU%#TZwgQhS{o3v+&O>hD{KvKRn}gJeDj1|qp)PHzY;Gz4+D?_@xn+=BK2Bi$RzD9< z{*kCVBYCE>wy*8^ddH#uEP?fRaRg?QVy&)0OXsl<7V16o z5?H^Bqf{#t>lF%hC1SG17Q|ul7(i%PHb|q^_|L~2((f^DvCMJnF$t`{hZ9)b)(zIj zXx5EQEu|G^suXGtMv^vblm7}_@B27#b*y52tYTf;SY1`&$NgcfvuO~E^=)SU8?gQX z&R1H^&V&svr_Q5o3(kXPEt z+tjc+y)`3^84UztE@Usc--h|W#hy#<&obL3^P5<5XSOS3dXw@jP)lvolJU>m&HWEb zDK5F+jH!xR=!a4(l~Sptwk)OUl+A88`@i}LhI5<)-X^%$poIG_#>kh-2YdX3J_DMC z0(_B)o_XET4c6hzYfyL+>W-<}f`gcZ%3!Td>4hbHyteTNUTg$6VY1zVD)lQo1QmkQeQGQ%%8EnIio}~FX!anXPzp4pSgC1 zy9Q`y7@WiziXj2o-DSqV!E^?15fY5!G5(1LeC@kKF@D$XaMuYM@L;+F%JYz8j`9m# zN_P}@asNBrVQ{=V>_9<^HvOM!QxrX+C%&RPG-UM@S!uXKhImhwPR&#(4H<-?tKn`l z|4~pH(WxACkbeQ>LFv_T4wbO#_--fv?`;C}Y14GqQ7)z_=BWY4fj7fkTlwxUwUztD zTbV&6tbET~Meg~?(x4{=ie0b2&&jc+!M&MUgJDTDV0Q*n+&d0sI5G{B+4r6JD|}AA zlAJxgcX?cCs2iSrzxOP^G};BN1M?)!zTeCFAKS&S_!^rIBEtNxW;Imq7uiWhgvoZ1 zbic?>e37tUw& z5B253FjDR!%H>$Milc5O45sM17z6>(p9$h+9|Y4-yO0q*C-St2am9jbXHTncBDH!_ zkQHyey#K%`hd)=US4Q8)of71a<^8wwW&fEPe?A%D8*C>b6y4!G@F?K0!x=7{tL!@P zt@@r3Fii5mk2K`r@&buH@P~%(UgSIY7P;K-$ouz7t8vwKd0k_tQGK#2%$Z%F946)< z>%sa2S&f+Wyo_(B<0Exhad*FO(jqv0q8PLO7pT;P8z=VR+e7h@x)|(e;1%=SU;)=G zh<1PjWoS7MnfO1D(~LVRUQ%*iN>@(s7lmk_dl=_Gq8uUuh1zg+#(tE5&+SiLkn%bA zk*F9Lq@3mWgi;Ti_$QFH2J^oc@GUa)FQhKZd5Q|>4-{~W67q=Y{{tawGrh=*_%@1< z)P-R05j^hjD10uuR5=}cON!B<$Rj5HA09uj9jCFF_+Jief(^Po@_O)8g4Z(-pbsiw zZzeY(PEgVw?We0!N#nkhW%YOlLgY1edDuj@C!>VDyCNjfS(=dhpXmdNkX_}f=sWt> zbZqK*C!Ma6e(()$n8|+!##>&G z=dMIv*T%cm7>|S`+KD%u`Z2Tq5g6~7=yzdn(zS@_chfPR8E4YnM@B5#=N>oyA6V`L z9#xS~v&V7uH`B3PJe~q-X8YXZ7dWm%pF64^Pv42Wq0M%iF&hbKwzKhu*&ct%A*9b8 zi^pK%hBAD=Rz%-T$830JHhs?Td{pKji}72s6T=V@>?9+eCWYyVqZ~|ZuIsp-FHLs@ z*7MUbT{27s)KE&(UiXAq{|vnD==$4XzBje`Za3y5A&GvXpg!q2P`Te@0G{J12{(Ldj>A8SD&K*qAtvpJtz<8yQ{^eA+YAtb(273w zJqO5Q02v**N5o{fig5TIrD$sPWcZV}9;==%2Y*h&hQ^Oo{gfr1#{T~~_*EWKP}Ao1 zZ+`YC{DlQ{6#F#qKV%eJoMh73t{HLtlhfv6dCcPS|FdcHUW|Xf201%`Mm)UovO0WN z8tmoyPijz-L<2o--g_L%a6BxL(`IPU*|d4@#c`$4bi=33drkhUgz2OKl_=FWR$7nh~c6X|3mk?C;mng7t%Ef%f# zE9b~~)>DjLEh1j;%s7c@O^e&Kn6>zXJ)brCk79i8Dl!3Z>$$XTr}-arkqnT^)T-h@Ko{FcW@H4YjH9 zM?|oImCIUI!>ku_uma43gB66L1C-~UbKva3uC_l$b_7B34%Ov;GdKlQ%nneVH|xL3 zyBVOYsJ{av>|4|xLGc|KCDcO^PQc&v2>X1q1Ct}{Ks*6fJ@W!1>|=Rp_%%j`)bueU z?B!{+IeUa{ZES2pTFWeDCvBx#N9}zt%ysnSBc{H?=X=%AKJdQH%KK7btVUSa$R1%| zxX!V-uz!oFDP09lMme*in6)O|2ebWM8AtS_BGDym1= z7rzDm?5~UC`)dwA=wDuZ!Wy5%>G-2q{=H@FU99ox#Sg)SaM9$0UK+ zj^JRytPBF6 zL1#h0{wu(dA%iIm%ZCR6`(N}+GYBB>kRV_`<6k<}*!UWo4v6+ho!~_c>KrExEg$6CPc#Kp z;|*xv$=A@v$&@j(=;v`$G5e-+e8`OtDLuWL_!*Rtv0>hp!2JBkOb-vOlguBN%u{-f z8=13bu$i0ue}Mdg$l;zm>jUHuE9RArjo7kbHrOTgENOX$au)MfiW87u$oU_5vdDC^t*YN%z zf%nDO8u)p%ZsY z9B9WYP!TqAos*ydlZi>pGi<2Y!+4w|-zZSHAHctl7$3@>4LOzNWxvKY70G%3t96_Hw zPj-SagwS%5lm(_xB zk&yOdjEK5^|Dp}1`+*1IyS?46Kr;PJ-r*zPgq_V-vvjzi9<&!Z@K)3i^5hL$vbKPUiMM4tk9KqX*6U@~V{h?6zzavNR!*tB0 zD>Yce$PTj@QSB_@OG9DRqxRnZ*nvZf-5Wu*%fJ&pc6T}!i-S`@y=)(QXTBqhKDN64 zUKs5?ZM1uh(MU+6ouyE|!}UKu_Lk#9bM!}u-$;jbvmd2nG%QlV$KsYSayl_wu!KQ4 zjSA9t&Hn$OTe}aVureIh&F)FZa7iE)P(LY5d)#*~bZkK59an!JjQ74a-hIY+BqY%; zjnu?{3Mm_COZ3N>rjZW!75_LL<6*JI461VJUNzL*;;+5unP8d}pm&!!n$YL&$D=Wn z>9G29Z#t$+2B?7Q*{eVAn*BdPYmL?458HiD+wOj2I}*}%JMl*Ic=sy@Ds9&qMczv8 zIL4aePtviSs%6f~V#`)8nyE~e_w}YDmPUo=y~`XtG~EMu7Ku6s53jo~9n&SjQ$Xcx zuS0-gw>)}XL;VA=-1oKR9x#?8AuV^FLi&E4LrBZDML)nCoOCCM=m+UouCM4#uZu2S z7MndYx@1}X%;+*@K5$7orb0+CrVN18smWE3dSJjZsa+BtYAEzm);6SYIDgv0KjX& zZpl0~8hZt!vb?$!>x|~uQu36K(s2Rgz<+<-!_l)0iPytxL`?1^sSq_85=bg+*?#)U z>=ro}Y3|(r&n(>dBw4tvx{MkB#>*@mFQj=2XP~)+xe6b0&sj=IxD?knB+h%mo$@t? zpP2qH!0_}`b-MsCtW+49%Bo9o@>p*kO>WzCFzk-Q@NHca0fxOeu+4r37><<~Qpy*m zgW*0G!v|*nQ-I+aoa{bYVK`c0sBLU(E-mkLEjx{bao@Pagz_7}~@Y6{Nc6b4&&zC9;OC^Tp*1C$8@|xg0W3r=?>0r1Z z4#PtU7+xHw4%eeDRx1ptWTVr;z?`vVho4m>VAwZGox%r(9g8aFoJORa=P-?!hGk2U-_bxfp(aZ~}&xioM7(lmdoj3PWiH?ggne z?c%`gaIwbF8=n}le$M!h8SU}$8Evjtvw)lb;DGXSq*261bI8Y{d1E^9MT$-njD7^6 zQ|lkKYip_CgLeTHMotm2VgkXU`6+62O>LMrqbqs0;|)bWdIS6gETPtkA3-8q3#aQM zmMFxBd~t!a3vW341@gahz*uf~^N2<)I`Jls68-Ql!W-y*!Szp${S)Nak3Z@czt`); zzqm+SXo6fPh6bIj6aT`*zosQ4YSMxwjnfa}Ub==N?G?rAVHgaNCKa#puVLX_ZbU`4@Zc5>H=CY zH9#*>QPEg|(Ax}y$-_fc9lCq?<@rvPlwoE3yvW)%~D4W!JMtRrxP} z=kVtZXip)Oiw#4SDpPV=soyyJUp!uda7$#Bh{@CHP->M@Dzyr*l-j6l_8TYviwg-d#Lfxm%V@IY zVnR9=OS$z5%GT=r*2({QQS=?R@P0v=4260}rGo%BBmlc)-CL>m+dEKa9FxYxk4gP> z0qF1d`fAJhGXuWhteFM$82sBmfD6aqNlC`wcE#w^UMRyTVv^i#wjVac7mM)zH%Ael zqW|+XQ1_%(_x^$_9hL^4KA<(2oJ0d1MSS``%5Zp|ETaf$&{-7m=~r>3p>B8-@w+L0 z=?#b?$U7v8_}$Sc$r_s+Utp; z8(sr;;|H`A8T;_VxhM7aMOIy89?Wk$zG}DzAE*nj5uZ@t_a^@@6%dRehTu`O^3rb) zD-}=XrO_Fp(WTb(O-^ioG2&Tn|B%b?u{jG4pShw~j=t&Kk%~U}-}B@oS~gkm;sGN} z1$7&ns!J_15?Pdhz8zX!Uh}ybfI0TTe~W*XiE+kG%htz4cVtTfaxQQ&8Aj<5kEX?Ebs> zUX4FhTN)J5wxZ^bS3HFeOM^chqcxb8L<8Mh|LF{rVK+EU_SVp#v)=ko?Emf3P&d4{ z{?kwW((J9tJEXV%Gtd99x1JVXV>KXxxBhvoW;Ipz*6)!K=X~srk{%FJ!j|D{c|hng ze3H$2n~i7?nchO)tR-(b-y$L#vt{2<-B?~;TUTllc_HyCR(ItAA-UfC%})ps8q>;v z_m=`69(w@p&Rt$8Vukn}C%y|au_fLSnMV6pJ7OtEyY631`=9C`5ju*p*!~x={b#~3 z{JB0sF7V)*YJ2^e-_9bv!1pe_ul!LzuUE1S(aLeilijc&LvHcL8n@!w|t6Zpd} zTt)k$!l3T>NNFc~I(E3t#qc-N{sRnW_V=*uT>9j*(jRbD53WqE=yXU-%MRDM82+A{ zP#0(Q@i?DhyLeJ!sAy>_Ei11zYrY4*mT^%W29y5|>~MBZFY>&?@VvxOUfxvIjN=ep z);4c;G5mw`fA&!UhI4Rn(sK&Ka}q;Jrw`KchwI}oT$O;~T%3SlZ-=^gN@1w2YiTL3 zHFh}gy12x}@XrAW7|zS^aF$qN*efxlbb(YlcDO=g(DR6Y-T(}YN=xITQZ6f@GykVK z|EmTR9tb@j7bvM)Hn6t!N;;!^ia>*rhamLG_W4Y0Ed_)JLLWrnrtWP;Otm(>n5wU= zYx35X_}t{brp8-B&!|3k@?WQeQXKqn17k^RKJvNAf0ZYhk1!sANxv}uU+xF`sT~OY zVz#cV@(_q$VEu0ngjlyic*nf;z64OHHP5QpPHLf(m!thDomuCNG zmEfe0yhA2^Uw#3;Y+hCIH8u-G@P+>wq6Jq@Dj zT>cAxCQB}#lkH}kt!Tg`a8YCvbb?;uBoW!9CV?$wZKaq5c6!FESo9Hvyw@-~W7(XA z3u23x&s>g01*El>r4cftIQNkdUIX5M_^*CG#8&);6TzdqVgELM7c;$XnaRYaG5V?# zj16GM)DPvImhplR;j6O1dr*PBFUk`riy+I45kokVU)#gKq3F)(1MT z=T+yGT73{C>QeIFflny+zfBH*P6$tkeuTUM`Nu@`BQ+sRjla3CjYffIhwSXdSHaF+ z!eGvqQ2yjV;bzN6aZJ=dKJ%~|Fw%!oK!xn%GhZ?P%{W4~9eF~lt^P4g20x(Im<&Fa zQ9>OQAv@nQ@dm`NE_VoNt##2~2qD)g@fgluq+>1j;yGnA)MoO#tRO$w)JrrCc;nVw zG15UqqdkTPf+?dNjt7FJW3)s>1=LQ;q7^95tp5e3TT}lyO!uKS-Q&h|BqY%pnvnCq zz~(gF`sl~lUO_sXJo-2t)8VL(xg}EB4O*sPH$qvh;CQPXIJDd2xHQT?-S%iYc8h~k zK)s|OZ8YySM;Mb2YwMqY(SE9p_JlDS2}yJo-mrFtIsc%nO`~;0KM`UC>2UJslXQ%R z{1}WDck(DJ%y5v9?Br2ShKYX_BpU7s979)z!^+BI=@>2vqyp+E8u8Y~%QWZz1CML3 ze-g&~8I6F(`lK-)2}yJ+-Y{O~bjRv6-iGKeg(xOngNXhz9pkAgwhkR(92|g@BUY$q z%#Lz|@?TyFU-&0+Xm`ux{-?nEX|*S@138vOEI8=)csf>#!%{${q#$jvEabl}VRX0S z>+t~R$j>Pa7JJHAjD#dogF#N#28WOq+Zg?o5GAC;VDPJSEQW*}+o*rkDCg@9Mi?qL zn3Ki$$2Hhf*aV~=P8Y!@pmYr8LMfn1R&FpSyUmeBd!0~^D~cl@X?s0w>_tM_>nyzC zVcEAhgtXVD=&x~^HR;eif1QrKR9iTD;gY#Cm-xq>b319C!FWJ8jS5muxdVxg_B0X& zF>am`T=-1KZb={&P(QnQ=G^88r193(<5sK4FSPOY8sm|W#^VGC#>@GcLrCLoj{Zi7 zQKZA+nBSyhJikdSL2Q`?26-XD2bufFiikuv(A?u4G_>1ZO!WK)`cyi0i$haDwWJ`b z57l#j>FP`$cge@VDvM%)T;M84c9Q zM^VDsX}UFHIZor*?ckx+p5f)*vRlI?q)(?~wRk)Q)J#g!X3OvCNTb<0>T&-=ak-_OSzzSoWGzwB76WsaTqfV~o#tsWhQYfW0^ z6&`JEHArC9nB{}y2`nxypQjlc<_{+@e>DuN|x&xCh=%l6PxcU0rjl|8UoE-t02T*YGy`p98DBevo=_B6zRSysPSvVGW+) zlx)X3jl!+CHt$ywaKBNjp4kcB>Y1Ib&23d><#pyBnQ+`!y1WZY5_rD}ro7399}0JA zQ(a|gdFOY<5%;&^c$@og0-L;rQ$CzPg134W7p{uLPLURqbxytlK;HMq@%|tI_uDwV z)1r8{NZc*W)lKjdgFD$NP!| z-rvPZ7w$0t?-LYmWPfW*%gw2RaGP8m$J@+*1AD)VRiAdnTRrTnrM$ekth~|KJKQGB zmsmd7b4~*8_pp-Erg*nW+|5lDFNovK{HNQaygx`i#11x553y^mt81w& ztuT)>2)D@%E^l1D6VLm51J(1sz*{}5yyLV0`KqlQBNZTZ}l|7 z=CWp-oT})&52l}RmB6xhFSGwYsLLPpR*y6VZ}mvi=F+wr_{rDv=H52)?u|Ek@IvOl zm>R5(PYtS^QBI*7|Fesi4=StXFo~Rzeo$E|;|t@0z8orL&PZkUa|*A~hEp>#o?7`% zq$xPH(v;4qpH^m{K!VJEPVW&85bl-g8#zt*r&dnONMq==6=j=)rF)+N{zB%7n9Tl6 z2wIR<=3+6%cYlOs7Psq8t>pCnri@o^&9ROy$pTTDUrqcECTwu#j5$I!dUfls4&^OnQ2F* z_B-9-&r>N+M^@+yn{g^-MP{ho)9TeImLiQ>=KPcLCH87Xk<)`yDW|8~s~NRkR%-QM z?x@96DR-1XVK%nAMOm(_jn?X#YHT3t^mu+6ook4DVsU4YGIA$x8rKS|}{4j?+;lTI^jxgXnhdpwB;`s=UX-=lea4LnzG($8@ z*%7ozcdQz^S~*Fda{QgLkKm~{NWQfi}OKk!(GJx`_VnurdN+?R`q=~xU( zrqam3n6kBc1D*Vj7ezO-n`gepYW0pz2LVo{0J~(}8=t7NK?g%UjyaX_V-AN<7J&Y9 z@T;}tJ?GLB?!sTJF?P@{aN!tRmt>69QNkd`zl;*nyc%qRh`LjK?syleM4 z=L|;w=WCF2Go@rdUHd9NEDZ+F&>9?+M1$cd3sVeUgEEZZ4`P%+4LXYw245Xl8tR5e z34>quOCw5jslKgmWkpyykRo6zO?Qmj@*HfPH&sky4MytJ;Y(~XAAmkpHroa5KI58$Z? zgt=Bh9ctEp>I4G~EI0bDh8cc@^cd z)>g3x&V<4E^LWw4AP9g>hagPac<1HBh8s0y~q6ZK~Ck$lZ0C?NK_ayM}T`YOuFrizm`mv&Uw)|IVmKH;PjJ4(y8)RoGP zVNJ)hWz&w|*3rJTrmn8Grmmv0qO78}tgfo8x~zKI*48as+dI}wZD}gTgezD7vzW(w z)^1t5xwW-rZR6&~j$$;LEZ&9(Jk($s-q2!{S^rd8EJlnlW)M#9(aJ?)5d2wpCSqVQ z%4bkRl1|m+n|vc+z|F>(BP#Q7KO?@y9l3cK79vvj)Rc9a`XQehTeu{36{#tFxE3d= zk}@I_-EKx9|0CPYtoRDm?Pl~KyWLy>N^ySf?;pm8)ow-~3NEbmB}r<%5@lhp(PsbG z60r_R9G)G^^m6gt(5zhH1=h-7cw|gReMqm-j=oc)T@r>C`JXu`uP$ME7r%**GOj|f z?4afScSO5930f`%0qq#G{<)k!I$$@@`mMZOYy`;Yi+oeiMSS8*`^5&8bM_TD}WCG&xCi1Tl7sPtK~Oaa2{#L{%1y zkuOQtI3`f`M(fuxV_?E;`I3goCURvGFFRWc(se%edI)3BzC5JP$3AP-xnQ18Jkapr zBz#yNI`(65VVxhFq|SBJKhEs`IhG!p|DRB6LQVnrjL0ew8A=s$7gPCfiCO-W0|QxT zz~km>z{iFF9*6$VchZa>lgFj+HNA%q3-CC`KLoro31CWuLyUU@W#|wqWkBy$A|%#)fh({Si-O*qrIgVxu4ATj;;KsdPVS2jW_w7X6upL8~l!#po5elVuIz^ zfTP*LDjU zMKXcnYDRLB>M?l&^B;7j#w4z!ks_y|v~To%e_|d7oe^0*<9r|g4)y(sv%rRpwK2r^ zC$6=8Kkp|HAP&0d(&hM|@2|ic{xIsBvSGuz^U~JMEv;IHw(G>dIdWJ%M@|wk zIV~#^ldNViL3#`@NW0EfFeg>oVUpDUDm(H*`#G+pu|k zBiaM1yA1ru)9}66;roaUYl@g;tKT_a;w4Y5?xbJaeDglB`F5hNHx4XNE0agr z#iCr4rUH54wyhTCJeY|u^s~}r+Uirx{$JTGM)zM`jqQztLJ$jzcR8=J83@= zS!j8w>a1DoBsL5_bQsXdLm8mr;EslY?i!|=^B+oq1~Bkj5uI~XP$}v8LoBQWS7er^_DsnveecsHH5->SCP-)r@2U8g^ zbvhWcSEe0n9df$a08=$Jcnn&g2c!=W{kCFiwxzwbV{2?(DWz^@mE{#xl{J|1fit-?*rEid|I4KZ`i7pWRYgg>I+dVpNT#|9%BI`U z^Y+_}lZ8{J3XTK|;3_-SCePUk^09rjQ%)0ovn|a_iUl?=HgD*_#1*PbKeOWxFvK#E z5YzD{=N~zouK?#m$LI%Cjuc~Do4SmlB``=vTFY{$|IaJ4nH7mKmZK^6AUR2AD=g)1 z`=^%U5TMBTYKka|(-PC7<_+6Awr<{@pd(`}@;q4cb^JevIvzGw@613J&(+ap3k?|{ zk&V`{BHK2X)X-;DB*rFaNOF?)#v9hII2_~*epZGAKNYQ3@FRAEO5FCH$>-pM!HiPq+#93?%dfgUgC(z~H^mYm~N1QW0^W2E26S=EP_lJOfg<-!aL4OxD1 zBmM%GWUG~^%-2dBAEHDh@?Z3QwVL(c_)UzO|JQr*VJT6$P%E)6i4uBTsWkT=tP_aK zL4fc(SXEd~n!WaIg$hh4@$2Zih4slS1mn+Mj?o zG^#T3Uk5kN%IS5ECy^HdoL`BoOig4UzM>&^#6#4+RrOmf{k$aAQl+Fj@27&V|{=>io-tp-ZFex#;WG}ADhC3 zN#I_p8ST|@FZ2^Nx%6(<;1-c*C2FkZ>jhn^uoE@1uTkT6Nip5!B>6K)*<4?SzkglI zhRF2jT4T=tDRhspp_)A12$5%XPA)o#rUO}{Yt0@_^efy@-^%>F+mr#GxW zce36=P$VW>)}JV?4-EF$S9hEZGnd_r4|9qOv-25U8TiiDZ?vMiCmp_YFCW$vG1*qX zbH2n&zIq@$sHe?W&J=yVov7;_4iC=0EMoE@fkk4l zRg;j8p#aDvsUbJ(|I&i_yTZwpBN*~UU;{@u)U1Ch@+&mCoG@02!RT{RRx#jD-WuMa zjDP*kBYzM0FPP$MH^tIUxtW2dr;&%g;gFYt+|P;EU=N8?HK|M^v5*LG)->|4Nsff5 zZNI6~0Hz9I{%{GpVytQ8VP`vxAF&yCRvZ*wUUc0DRDWecSX=*THz_J1fhqc(mz z-sB}v8ZVIJItZ)hB~EfN>ipCJLUk5S6|wP3ovDfHc(TW{FMkaNQ$Ut0Z^t3>?? zT8V3uD4_?7`pdKu*NRpcf=waY#LYtyH~al-FulXHPX*0nq34G6CjX(>u+n@seGoMJ zY-l-YBx`OsqpyQuH;B8TWy?+;BJ8>046c8(cCQUF%#4_2n2f)Y{FmR|kC4@_-5G5E ztljISVG3DFzO1?743q!7-k%$G_ayQ{Xy#X9;@oh?CmQ1Q@euXg@bC;R{kA04QVbmM zt;5Hn4EyHW^v04*2Hg>FCt+^m54?);9*j53oAs~T{Ea33@b?US%NI-O_Hp=54ez!v zczZOwTa&#|_P9SUfCfrnHjmv{dig6bRws-s+Hm3-vIJ!F}gS^1&4ikqc%nAE)fX5=dJljka~ zNyt^q=K2SH^r4VSpEKC1^zLH;Q+!lCe;YomN}qF>uJp%}RQd#zMWxSa*7ki&_D=g? zl_BWkWdgjXDD;Br9Rvar>GnM5Mu+fy5QgkDlih*GfrF$GyeZejKT>Wk+)PwLf^5%Y z#Gq_TZVBVTuiLGGC35AwieGQ=n{8Ij-Gw~B33h8xDMn;ksUyw)H_1irY4EwY3wcT@ zl9QBEVkmW_IsYFhRfXY1UI#Hj431OEY^~^gx5iwye-0mkVslKAXQ(r^1SG=}?hCp0 zZjHHfZ8lOfpN%`WRGN{TnypZp%{B8Mi9)8d$_DD{n(nMpO)5E%l-At8+Awoijq^;D z@5bkoirGkkLBgD^V9qPE$#Zvu+=n^Eh1q%PsSNzc)9}5(;mcmghc(5Sd;nHwn<|Kx zJhi{gWBlV+u1sn9d^=Isn+z84xB103c}@hdj?=Z`^%ev3gGK~cd7$BI(1EA4I>fqv zdyFz|?S^fe*KKcU+}fz-gWV2r2gD@wF|?aD{Q{hS3QR9|K%`=NlX+Ue@js|J52pVX z4`0gpC{kBEd}&!+yDHs^Z`9oa?tcmx(9=U11insUJsVj#%w~|&00zFV4~_Tr@YL19 z7KhCyu<`BS&Z~hU3*y!2NJq9h&V~2dT=Gh6E}b~CI3H|ajD^3q$q^BIVni=J3auBt zx;wVisVmF68rIkCfPey6h`0SJ=`ILj(oV7n^WUJ7a>{_K(?JfHC8;6bXOri?4{~mi z)4Q>G#?UCCYS@q;s;si;D~J3OTly$52mw!=I))pXI2?}L$Vr1)Y~mj^kbGuybGL>E zk&QRH;Dfsh~r1viBS^6&&xuoEI7g|+z4j=J7x|V+7613)X0?n6j z|6kD582R~(JJWnNSim`#oNJTk>;(CEL|HOanIihdjq3Jde%#0ocCz!=djytzJFEv3#Pfgv@dMzzAC{Z_4~lpIX`LJV_M;{Jzw*UcUXE>`UxW%fVp zm4@L|MZK&X*HNbb+w14J3O$LuFx#rc#2nXA%>U2<_r^oiIj*CAt9|RmB-K&{Ah0KH zcvda+ML(60lPr~x?W7VoE|P(63P+p$4==_ep)DxI+5hkn{wPkL^dzg+GS<;%|HDg3 z;9jK}nf(th`Kg57-E<9}EV@~#gsh;K^MYU9R6^W1S5i!$IZ6JsHbK0Mzkgj~f$(%< zVTFnR6vRj1WO=;Snoc$XH(3W@T!;`2P3*+NiaT}PC{FZspHq?OW_3_1bRs!BD4keX z@f)9f9aqdmz7*mDE7ATrbjpoX z`RvNY9W$D<@nN~Z%4@-ez2R#~Twn#t!k1P)r`39mE+A)&<(Mu@`%ZU8^T{O#GGkmd z+L6;6=9_O8Qrsnw$E;*~Q-pKmAXdiq}3W6b)Os+9;}#dNKBQ86e#@Or??gIK-> z9r%p#W^v>H<&4qa`Y@)vYD`VN-UxA;oKsH6-?mS|kM;b~!XGDe%?uN>OKE3aRmZ+y zyMUZwrczaAf_Ut|Y;w6d;%Y2kswG0qFqc>z3u*}c5?zIE$TwQxYOen+Ln`(g3);zAj<|1d9DOq!u zGRAQx|0RtfH%E*zdCoRbXf47FgflpsBd`I;?4*IOq~=bh>Da9ZKnTX@W9I(~PzkIu zVTg$Jk~hFFygKrn~NpK9{h5ZfGbzeEVR@(n}YtjrNx<&a;YrOTP=Y%v6*-+^cy zh@?1!_#H>&9*E>xwN}_(t5(iv5bOD8gH!+c+(O|es%!8V!l1i?y~1aQoy9Hzt#&Fw*A)j*t$72^{{{^)pJPEtUAa|#n5wJB9EY5n4*XR4Xr~#19Quvf zVfs7Ot$~XmD>f!{3yoa=^1FrnMzC8nO~fY9@rHku1CHHX? zl@(ky{S90gT)mMbxYAS2=0O_h8~#+YuZ$Thf|${sYW|&5O+=FkQ_Z&$rkc%s{tuOS zBSeX2#Q%PTDCAUg=$6C&;40C~`5%>dD~S?%cxg8IkGDjto@(Zs2z)4?C1aXfmj9ca zYBGMdrkX8dv@H}HR+`VI55jMsm~|wLGzARwRI|nG|5L;a|BApr*;KQI>)$lYTOo#N z!TjGhjNJVoDt>z-TaZ<|E#KFMc}E(Cot%1EQ_U7L|9!`wYWDLa^1^JZ5)-GIt)&{` zJMj?pRI_!lmi|$aYN-Mccya4Clwtqykw4YUO*Ylcai*HtASMId6k1vTG}cG)NN5X6 z`Ke{B%zN=i@w+p3?ZAg+tTv38(u+Px0#{Ep+f4lbi9gls-(A<>$)bBfFfL>Vzk0#1 z!dbOWX{vd{^pdounr){4R}k6%ck3=<^1;F*M0cxmav{Q9p@}`!T+>_Ejp9UK_c;}b z?p6l{wKO?<-Bp%p|!dE2~eP5&R5*XL9u3S3KQa++7qKNf(r+QPm?rfJ+ig` zmsaaj--d&w4J!g0S|i&Hb2+lXy74314Kst0O|w3JK?3U`(qa{X#S&PKofenZ;d8Z=M%wl<_9W;Zev{GCD5zA5-wYT1$W&M4l&^lGrb;4+UewV-2Xv8 z|9wb3uDjB%$0*4oSALG|E4H7nRC{lOpOhiAzCs4&~|Ir<)$ zPtb4l;p>YM2uFCbh6=N-c3=@|A6?J&KkD{}c-?exu%7cD!u~vo`*Ccibm%W(f9?kd zdC7uoK78NIsxkoiLzm z;QWuy_&*^ou;EL~1@it0sN&HdY`g^@mJ4hw1{b)%aFN6XR-i0Ab|csS&~>298l8u( zz`RWcU(=nL=9H2H$wP0v-;vWB)?Z+z)$W9NN-=vlGk2V=VgA7;&t)g&!<^#6>^xsw z27ctJ`EJ7cSD?PkL;J9%IJ14g>MRdUyyQubfmt`{`M)x&%tQNpJ5kqTvnDy~Kd6-m z)NRnU;^DYTe$WH~D-Se$A+2G5q zs_SJ-g=5&osk$NE2KBOKvrR6O&euei>g*7b&P%OcL}mHt7bG?e9&;G58H6$j8Ul%R z#IWVRZ3a0Fc0(8%-w?u+&RY+5*le=x;AXjif^&2=VWKuxLTIbr|D(OU#OBh8m&Ad_CCJ!#u%4)|TEk}m=q=)O&_|7q!0*yL4WD8{vvRaA~fc;v0&ov7zO zn*Sc~XBx9VZJVrWjCMMmeBv>Vc1$M^HC0n1dRxOTq<^Dy@`-x?iv}3A4KP(2z*J$U z5-?N{y4A!_>hC~*#Ae)Cz1D#ZD&{1e|JAsddaXt_Ob&>{oisOab#@}}q?;WajbOB- zOvEO!28M_-*Mvb6gDks=ouuO*4QId2rAU+|cqz&00y;T<@+b!cwW-0i;qrOiqvNeC zlmBk_OR!dQ{A8#9lgaU=<@!knLqvI^PF^u=>OrEd)5%Xeps172rAU+~=tOer#EM1@ zoy`0nf+Ls&88B%%{iTkNw-2;}Be5G_it*=(*YRNmN86`^3xlKFB*BrM5^rzRKy&@s z_DC7?RRl4gJtcmbvu#9!oL%t6p;@LbuIE3TD_gd+{ZonDFeQF&EAih{qNdNjhg>C2 z(dWO0=OO68U6G0C}1ntKUIsR{QO3e7znq!`#<6q5&mFBbQgYelW zW*tc*Vdyn^| zd|0)+V+gpgcDqZ%B%NdK;Q0@N>h8}mM|%=^VYXF?vLe+SQ$yTwvWD0_9-^Kq?zm1% zACRP4DuEPUyuOJMXLnoYEpw+?|&K|;Lb!-k(*@~h&&57 z`T}@Um*GCGJOMWZt1Fn@3t5z44nb5|<35e~4|+_18#@$Lz_LK(Te#5^b}8=va0Jud z5Uh^(xreiqV1Tl5pFY7Mr=MV@M99rX3q&^ycUITpKK)ilFmXe$+C}E>{ZfLtc4gW7 z^iLde;?6-PO^(G2M0X2!cGu!QL+Af=MH4p!t0f6{AwdZSC>!?~Z2zvGHMeFF7Y&Af%zG6_q1^5buI2)XC~l=;FuI#Bzjtrj1mk`4(|O9 zIsLm^FCp?3Zu?4-{I11)=0T2N+8csnQgD&zW#LwW0m{aG=5~jixVvMB?WSiY(jh1AUI^`aD%?dD z?w(zY`+JUH;)dXu6kH^VEZj;k<4wy?&h71x6E_|Qi%aEEY=P)!;qKM7xX-=N5lq|= ztS+Q268$XPN-!gJ8}}OyIdS*yk0+xk-2E-wg5wi@Fx~g(Dro#0|m8ZXHvCncubA<%Nw7 zIdS*LR9)^#T_A>7xchZ6?hhTo#0|k_GpxP)R|z&*FKzFO207%!Jpf50nfzQJhFZA$ zcP;LVu5<(wHw2rktDRF;f=y=2;C|mBC+>m6kh4+ka+rmCK-c2F_((@EaYJxS3N8}E zto}p^1}NK4F22AaC+F6ys_pg}d0oJ-BOe zU%J^5OxzHxo!@>?G8C{4;zVxU@6=qE!;!9822w7!Nd*0F)6r6jI?ko!2o6BzP!XCC+^{+ zgczo9kFszN>ss8GU+)MeZU~M^!9`+}g$RdDcqwi+{3#T_Z3SV z!Nd*0F)6r6jJ9wq!2o6BzTygpoVbg};Jl>5J;uU4qHA$q+07A5+z=d-g57KWm0*Cf zabG#dAt&yVu|gCp++!`=#a)d1B}XuELvTzAb}#Q&f&t3LeU;~s6Zgn*LX;@n<1E}I zU5opwQyjs>4Z$%f*u79d2?i(!_g;sbxJQi_Vx+=7-oibyYjJ<8+!0LN5FC?&-OB=$ zV1Tl5e`}LNPTZp>;86k!_XG>~s4m9+AC6$+hTxbK>^>$;2?i(|_tg^|a^fB{QHaqB z_e2Z#=&r?m^{tLz;)dXu6zo0(ObG@k2lpoqIdPAjq#r@NKuof5kLg<6*Q{~`6E_4W zd&rg&3{W=iYp!$1iF@2+A>^~$7l_Fg?y+5q``djS!Nd*0>Y?%OHC{?EK-svzz1Sfq zZag+m$V-bBh$$BCab1l2O-C?sL$JCn$-N&)2?i(|_qF*BIdM;zibZeb-%~Bz_cwGs?a4(|O9IdMWi?&}V61QRy|t3!V7?ieMQ zb7H=kZ^L~4y6qt6>hj${v|u*^+b#9&^#Iv^r4xV^F%U;3A>vgBaAS^6!%^afcP ztD*8gmUk{)z3y))$&CYp;}@TgM?tW<-t2!di}%P0sTV_UaQXkP9@*i|y6yNnL=OLN)Fi zW;=rE4U^4I9e0n76iiUI7vFH6L(ZBGSWq(xTRqK?riNRduBQ0^plX)?8+|2ZT3G0{b1EW zB37wzA7tT{g8iq_8{A(d;0{^MuIgIcw{rhm;Gnd3^}!-mrEni?;g*6kdb@sd>r)On z?H#h3UEQ^~Z!2{K6L-xaB37+%A7bH_f;0NKxNqCwkP~;vYIaQ*9}B1!weiao;}9At&yT)$H1?#eMs3M=){M9V%kA3iqKFZYel}(Y4io-~J1SoVY_) zv+KGR_Z=%7!Nh&gVIo$ia35yjmVz_-xw!B6wnI+bA*2}$LW-ag`kQMDix)%9eXFGyv^XZX#5j#X9ueXp((HR5dkn{e-pfe@%kTvb;U5or~bN(e5 zeGZMx5V7eR`3wuW6rC~1MSk}g3CKfMwGZuLao4i~XQHS)tPpv^8IVVAwAxfSh4k}VJfP&;G zK^TT%NRp$7B1RAt11d-o6mtM`&SK6vhg~=7npSsRcisKox9++3R(1CsXO{jw&-ceR zPt!fsbX1t>`^jmyor|0TiEBNn#{(+m9;xBsKji}VSAju4J%7f-OD8`&90;s*=* zEs5)Y8By!oMQMhfy#6;(4*~Qdfj-27Ze(XLsANIEwQZ8sBbK-arWg9He*7zJUIyu* zfId*554E5h*%_@8peOhL&2v(&at}%`^xKmA|H1l1Kp!N~Pqd&L+4$SB`fcv|CmR^7 zH>X;tGuk9T_v62z z)gxBChol$!LI3=JSWSoOk$^r#ppUem8`&9cUFZkTca*2qBNn}frWv|B|E0s6s7C?% zP=P+mf^KAIv~!`~QJ4fhV%_`1^g_SmK}Rs557T(Et3FYn+mCiNvNPHzK>wG+fmV-L z`W}{E=y$Gk1QYsjjVF8PVFKNLw1<(M(ZPj&XY%|Xi_oc7zlWz6`d!KM{}H@0Ru32G z_MNdtb_Q_SjpJR*9OY^Chz0NwX@>6Ke-T#Gk-T_7j}Ykg%>zbuMn@O=-CdKQN34O5 zOfU4iuXhB~VMgikfId>7kGHC+k)6>g0lFLipw%Om!AGSR`aLB{&`0YDfIdo~Pq3gH z*%_T(==WUfa3J)EmGIH&g?_Jp{y%h>F?u4Pj~3_?E$BveMi&?Qz0;GRM=XYqNi%de z{!ND&t4{*-F#`Q03%Ze=(ba{1U$Z3W5$oY&(+mB+%N)UkK2A>p^sxeck_Fw!&ghl^ z{Vj(B9VTK)d|Y~=-_QH6!Z-w>kJpm{eVjm_Y(Y1&GZ?GmR6|z9?`Qm}iStKykl70&~cRIB3m`}<#-^HHQ&LsyuJrhF9X1C0N0RlIq`s`vw49Q8EV1dka_ zSQQ_i2E)z=RGJ1e=LeG8zX^H@3_D&7JH;}rk)6qfaZ=07`GLgt4<^7Gbc%)X32A^% zp#uHE3}{-pij4;8`+uF6QC!yf2cuNk4|yqdvbcAKa$-3P0_`GK3Sj_ThNW{OyIJu z{z&5bXJ*to`IPiRe>8dhbE=*J=u-sx3=6uEomta`{^*HGtz*Pe`P4K+f65WeUgu<8 z0_al(dWi+y$j;H3+6U- zcm0>pBNoi3r5F0+Zv30jr|VKcpC-^tE$BveW*ry$<7*w|2|Z%Xe0rLp`}d!PMNbtnWg9(!c-FD>p)qSUfLDFZ8G6{(CDTI#bU9 z^b&zS$AWHTXL9}As-{mR?tjXRT0fteX6VWLpGx&yK%Xhl=UUK>>`V^DE$C0TcFK)b zk61!4O)vDPcRGUUFthYLKra>O^DO8_b|zQRZRoiF%9@W5dc-RFtn@-ZG|>@E=w*67 zpwANM^DXE`b|#n637ypyc|WaSXWr7O-NMx)jwau2Q$cY>d+_P z!s%4Y6w|5R$OlHxWC+%l<`SvbfVEnx`-Zcf`-VA2Cj)`|4EH|~Zp#R`&&;*qZUv>f zxcUvh#fJs=nTrH&ZVI@^Aur+HA#igIT<-VoY~Z#BzACspE!>G*gzIvvR6rT zOu&Yzs`2nTiCB4$OZe409U+YpIj^QvfBnmoy3~{U>z8a%QIJ2^QjG19?*(}~3_#4R zakr{nGXz~V{2Ftof#r_^?+oKx5z~d`n(_i^(?|G>;g{iZ9Opqoa7{(hBplQyubget?^zOw7n5A zIF6n2@LuD<5L;sTF@ijpR~qHV!li4cya9UGu%sKHpZ`dlyKRJXKmUv6++G`U#s0@W z2UN?sUts&sHl$|?=VlWCwO%-0?AtSFL#n50L#jD#2q(*AAa(JD%>uV)1l$+U{|`le z^Wemc3HYYs?~Y%94;$|50=G{JxNLU_m**cCr|M(sB1@qI2B#TpoKIDCEVs@MJtG!= zRpr&tTvP5zZhINhEAi~i7uo)Y5b3e?{M(gUsXCU&x)9+y(8TuG7k}e$qQ{bx+%IL{ z(*(7I-169$aszT$x(RJ{l-d0{I9?~xW4DJMyB!`|C$-037J4k2L>A0TTatuzG{Q!e zBWy<~3>Ty`Bd=RTOV4T)+@HB?dr*nCn_hq-#q(a~{Exjrp9s%;xxM9i>OxSeJEt7d z_^>?h|Bin)(W^YygHA+Q4fqxGf5s071c19#sc8V#bO7@S9~mLwZZ6ZCi<>R<1aH@v?;_D~z#6`v3obbbEdtE=-3~)pJx2l~)m^My*_7Y2_OQErBtr zl*=1S9m9^PV7ZvBmnfGv4%+1C#A0e2SAsjS@5cUQdm$Lbfi8f4L7(HB48C(1uo^@% z2t9+u(uBX+!)8!r1{eetp+#avh`v4l%{302rM4ZKC;C0h!Rwu=O&`3)nr}X4bIH!N zxs>DO{_Xj1{@W%ux97L3FPFNGb1nru0?yz(vbFrNJA5Z7|L`j}|1Ju_zjeAJo!~pC zzA*^WQIbGr^cLOUddML^E1KMM1fP5~*uW9q`q?4BRLIT6RK3-a7-3ad#gIRF3%s|d zIQ;j4Kc93d_aWI&y7b)4hd0)P_x4o|c^=4*M3y-HysxTn4FF-B|LXSqxBux#D75X@ z%^1Kh!g3`@!LZ#F&EFaCFuucP?A`?BmTNyx^LMs72>bSYR3Tzwd6N|@J-qf^r~NZJP4Lf;Hdnn|c81&jayZZ;*2|cTB|K=sm>0g29p63o3Qx1&^-Bhc*0skMVyFj*3zYj)oyG z)P3)20a|1xmV052PG&;TGn^3EA)m)Fe-38=80W)<&BQtR_Wbu)|EWY#gc9#Jx0Uz- zfYb*WFR_MOO1!T{i9sopkRIdxS)#-sRfZ`P>qC8iF4qEcxuB`c4!)V&^XotqS#W0x zZ1Mgr4jVcm*_er3eO$O35Hoj-#3XDk#Qyrfhk=Idj4Wo#iyZdt`5$oqFAXy&!Z087 zvJ5ltJd$^Q>h0t3VHxIw$zqscDGbvad7;qc(2J?i; zlK~TqpPOhfj<`UxEQf|!Z2BWQ#qNgA*9u)#K%qT6!5<#u2p$9Uv-K$m@|MY1taC>@-97>;Etl03q}_dI6x%7U&Bs=tg#CQy2P21&;Exdc={V zIq8M|(H)LpLZ7P_0{R?*zR-ehWM^{N!7>x8fAm*}1EEKpIhvbZ=pXy%UqY+T(~AIo zu0UU8K{v89Iq$cif9#)s@i-8A#G#{k>4pA@fBq$eK3^{e^mzh(u?5}8&Sb-5LI1>$ ze}~W`P94opGju=x6+%BnF9G!V0)2@E-N?>NICb=?fBq$e9&zgEl=MRXbhqQF^z{XL zDWIPs(3e`!jqFU0J}j$$>c{^>=n=PSE=Vu*&n6{7U#OP>`T~Kz%z|!YXL68XLH}%r z!$HH>BTgMHOfU4$+5d6C7dp%$y&TXN3iRa`bR#>nl?(lI|NKj6_2xM#PaQ2vGj#v_ zO9*|jUIFNf1o{dKx{-|mqhs~o)OD1n)gw+Fp&w0WZ~7bm{7VRZiCziliv{{h3%Ze= znVSIp#U$tvr;e7S7y57g^DiOvrFs>hFA?agEa*mdW@{JvZ_iJH9&zeuX_}$`#}UjH zW|>|M=t~9qY74rNo!Q2P{zaj~fesUK>IiFi>2#Pc9&`i~`f`0Lpf3~Xr&`dB?98?a z&>8;@D>|V^oH|;bUg*E`&%cBYvqG-{^yLD5jRoDv&TQvG|DAvS#p6Ke5vPt;q!;?{ z{q@ff`bvEopsx_t~Ss`Ntt!}X3}LSL;<2lQ0}{d5buk)3%|0(5`=7h1h-$CTG@ zdMlS{npaq8&Q^jiH-Et0IhMy~_( zQw91u3%Ze=*~x|eC$9gvElk9zqcv%U?(hE&t$vz56VTTP^fN8!Ms{Xr7y6%@B|(ok zb#z*Kq5s+6{~bbKtJeeiX##z{1>MNb?2-W8-~SytOvI_9wdsZarN93>gnqg{3((gJ z^s_AJMs_AcG}fT^OMm}&2tDG|(dp@h{#CY9+-zaa&}Res=>q+13%Ze=+0BLimB0Tx zgdTC~=!`T&_xFE?R$r&j0rWEj`Z*SKBRjLZKyQdo#G(Ge^?ybpN2Z8X$zQj0B-1O-)aSyq>%_F@TBbF!aXF~t&|lARI1qcV?ah6> z*X#2D{Y-&=o(0{=Mv&2g{te>4PU9VqI(M`_z0kk8+!0LZXX*0+ zeZ4?G--2#r#}VeXn~`tcaX8TG5$BH1N-y+pM>&EC{cODf(9aU+8!YHXHgVa|zrEDq zKdLy8pEzmbw(2eYPbr5BpV0)c+91>MMwb97^k zZ2y)R|BpwVJKB_H=*jW_&H55R-z3m4v7j5-@%k?GzbD53<5B01Hm4W*-;?A27wSs^ zeX~Hn)Ping#~UO-PmKS^qs|>&m|o~VCCC3S(w71Hg#!ID3%ZdV=V07&n4c2k|M94E zM;E0R`ahE6|6BA{K)*c+_h8#p#9qb8`Iu5`6`rUo6nCu%H{+@r2d%pA+N%@u>SdFG(--e08qa{g>qU|K<8BK;J6R zud<*U+3|$c^j{L=|M94&Q(m54=>JZR|6ieZ0Q%(ueTN0z$c`thrvE!J{vVHeK;;!_ zhMpY%zf$i6^eY7VP7At`9Zy(I|4(B4KOS|5=auP&{-5Od|2Dk~(61EeyDaENc06G< z{XdEE|9I4D`nI$}ud0&c|J(I$K;I_NcU#bn?6|v{UR5Q=|KogMWU41sZci`tSaSUT zD!m8Lw+r+=7IY&!4qR3XQ#FJHDH>4lz|9RJ^?_W}A&fxgdz zZe+*XxX?2b>&33u)gLZ1!8I)gHMR6S{8_U!)OM*P{dWS zW4oHFu4<^7surGoiv_)R8!i(h8=_RjpMU)mF7r z?NtYLl!=^CO6!Orl%>~(1n|>^bDm-6mBrmr3yFd=rW~eD?LZy#t}VF;RX!s zkH$V@?AOJkEj}Bi@a;G2*X?VX!p>5pdKn!<1AG*+*ogR<)hTX6g;L)#ZZc2 z3NIIop;W^ZEE}m9N;yn1N0U+NVM-O|ffs}+Rha^>2vf3|0xt4s^Q#FkkXAQ)m85>r8`q zN>8TLQAdX$^ zm{iE5BhhMyNkvR*iB>#J8pxzpXw}1{K}^a)D<38eW>PL%{V?fxCbdQ{5GI|#q&Dak z!lWTgYKvYXOd86hcIY+2q!XFc9=%AIG>l0d(5r+=!V{q{Od7|e?&#IRr140~tb$%HOq#%? zs_6B?q=`(*MlTp9oy4SS=oQ1HNldDaUNTIY%%mFVHN&JSOsa`qG)$Vxq*~}z!=#g$ zR2#i)m^6(^bm^6z?s2!(-No7nz?N|^d&1Mp6$HFja4wFzj7KKT3nS|Q0I82(yB-D;2VbXji zp>`|{lTKk0YR9rLX#tZ^JC=t@3z>x4u_8=b#3a;?m0{9iCZTq$3X_&F3AJN&n6#8h zs2!(0#0;CZTqm5hkr>5^Be~FzHk# zp>~`ZCaqx-YRCF8=`<#xcAOO^tz{Bw$Jt@h=}bcHI44XxgGs0z=Y~n^kc9m{`n)jd zOeUdroF68wXA)}1hA`RJu_a8}$RyN`i^HS~n1tGKNtm>WNvIu{hDn>5gxYagm~AU zNf$8*wd3+IX$zB3JFW(y`2uuUny;K1DCk3-uzsSTE5_^)kI& zuh1*?D!p2ts@Le#^jdwoK0~k5XX^F(EPb{cI_v(H68htHiEB~9Ua2p(7b>J%ho2>ZO zXi01QZ?fXK1|lUG=KXK75;n8{o2>X1<$seE-xB|utoWAr-(C4zD^KxE$K%sFcqI+?xZ5vBiep;a>(+wapMmFqb;_>FWV~uRy=vf^KBTfy*c^ z(A#WsI1qZo3Eh2Zf-Z}tZ8IU5i>2eES906>`@hG_N^T}sa`T{p^3T3mE4gj`{jcMd zzmkjKk#SH`aJ^StqV1WGO_vy-=n_7-y#j7r^h$2qr^RyP)EraIP_V3Jw0xRYa*e9+ z(01JeQ42*BmP@?4*9gAk*IMhFGPE=m=@%!_BfjiGB|X0Kg~55i$)!J#K%b$mvE_T@ zkd#Q@1;0@KHvzr35{(E;%v`yJb?2(#oM1098wVv_H;)!4m%UvQ%sv0c z_@QoJM5Nh(#bLnG1DKgt?c?K&ydmgFN_YIMc&j&BAR2#{tU0{jx*OCM4px`L*fAebn)eVB)S?T@}YY+|@1IMsS>G z%`JN$b)G{`+!1HZ_oo^6za7EEon1o}XL-15Sh$VgIBSE2yJLTcoVX(nnqQY*+#Tphr6bQ+X#+#O~C!5Lr&Zgr_8TUFYZo@9Kpn0y_PCAS9>{vb_OKIE^sqU|hw9SV zKmRaM&X;Gght+B{eh&Ir=9S%2ogWZL6HOnh@kvcTjXRPJ2A1fL^Dp& zUE+>*8f>D@m*1EMyHYBRL&=Hz-!fHA|7OgF*y$4bgAtrnMM$Nc?(#?yaBNlZ9#7&P zF25-axG7X{ckSZ{ra#oGtBR|8^{=jl+X&7ADBF*_u5`$0aI7DDmvN3%*;PWdZ%#Ar zuN}d}UAvwtuHoUXXW=%2(PuiiyLENQi5ts!sUBx}OL}p4OWgmC$7j`7#Wg+L^)1{+ za27z>_U`t(B-{~a%5P0C?(XjXZ`!+V165qh!`;BbZ3GjPjl28qB-{}P%5O_A?!3m1 zVB)UVP!-qqa5uDY8^KvMT-CZ9r66igXzWH<2Xk!aW`zD zitBl}n^?Gw;H=s%?jC121S#=X||HUCE?uf@$-j!b5y=FUtiMz=Ws<@$t z`v?oS5uC+&9=lrU=X+h}kP~;rGb`^-FYewg9l^xiw7Dua&j@IU9jjqSZ3Ji4cX9Wg z=8zM2!~-kuNi*(u9Kpohtc5CW?Ag18h1&?uYT)AT)5sww?m8ICn&y9`YG{p5?oBW5 zK3g2Y#C=5SCqfv(SsY(me$wYh%ZGi@V@nM=)_8 znWKu2@Nnl?xQ*bfCJDHIbjXQ2;@i~Hy@hn%=0Zd`vj&A2~t1QT~o8&!Oy zhr5l1+X&9$tl4T$jycjHC+>*r)*neP?qjZW1QT~|TUFfB!`;@xZ3JgEPr&`QLr&Zg zcdb8~UfjoybOaN3>vpQRm4~~Xh1&?uYT@EO_9BOzxFarFe=NPY`&V%U6L*{TsyN5P z-QL1&1ZN%T;_g4#At&yLTh<>>Gww$n!NiS~n&Mm!cLxi%5uC*}8q2@?|KgAncf=Lz zPox+3ajP7`#NF;FRovRceUydU2+nHd;y&&khn%>Z=OE@q@{uaX8lOCwUfct^If9A1 zeMeQ?#>3sw!fga+0hDd;0Sg^+;*Pjn{i!tL{=FlZxI1)G#b!aHAd$_w;xQ*bfHZJbMD;#p-j<{3(ne^f=s_h6S?oM4*aR(1~ zR|~fhoYmIFT{OZWC+?hfc=9I6N2+#~pFEpp+|M|IiMw+*ReY3(yPJjE2+jg1+fN2& zIpoBh+a9Zu9`5!Q?&s2rd*C`pFmZS3u8KQ)xVu}pjo>VRGPq?Obl@W(XJmg^Vq~8a z48+$5equeTNo_)+y62%6&cg>YaQ1{ugZ%aX$thMRdP^RE{eQCA$?}?tsT!%C)KndZ zKVrc;F2k^7AaDoy`#&Z}z#WACe<<>s2Ry5-_@)~C^!NsR*l_QJ(p*fLo&xT1$V-*p z5xY(|a5d68*W)ChJq}wq4~gI1rCmGK#C}Np?uNJyK;gJ_fO|0Sf1{bFN8lce{y)H- z`6DRR7i%Z}8y^<#!Se-oNebMvkr%iJZxP%jW+%f8*fLlc7^br0VLM$`-Xo}g_0ffp zMu{8>Kgawd|C=m`4(bOW+&V)BBs$(GFTzGS`%Z-@wUp&&cXcv zZF`o?>{C!$(z|P3ey^Eb`}WA2-L>bevc6sO_?lJPZ&uG)1tw#U9(~IS3d(*xV`-n+ z`MnCrqCAc@tA2_d_KgaT>fPyGWr}8K4xr^=y%rF2RIk`Yv3~f98y8g7MJpGMUA%gE zj1`El=a5(pi4zvhTe`B$N>+-xufiMp4{`PX97H%|=K1VYm5suJ{uM)uj1|*gT%QP} zMH_k?N`WoIw1}2rXtC8Ys4bvWZ})!xCw#DFn1(lW8G3HuE(@@!h!vHD)LF1sA&LoD?oP0C}O-iBm0ncD{R8@W)(*Ykg@NWh9{1m{Y6&}`5fX_Fr@OqR-u&g{xRkd2- z*x|Io`9MaAUGavwhWY#d=BqMz0b60%q}bGGXKxnz1ZC_W3#4|i%uQ@;t zldZ30{ocvSX0D;Xm!XBK9X^ec*OPcJBav%2WFc^e^Zpm!Z|}in8%H+P*Z|eHudN7( zEx5z|`JYEjkM9QlMpufYR=%8ljvQ{R++Sg{49ED#0;wUVYt;lf;$9=FAuH>}{0R3N z(aCZzZ!Gn%8;@ZG!*Z_?qriprVo3`3>W#c8s}aiu)RJHPV0T?&bT()4%tdbQazc!ZgYq50wqeeSYS&Jdn zA4}Og_#hSa7>LZMLB>*}E_TRSF}kC~%|^Q2s$R_yboKCSEc|+Ssu)B`W2u?Bl^;uu zKFQI8jZ7ndqe&}O&l*d)dcY!daBD0z+OPj!F|wT@C)sLaZ&~pSN^+Kk^sS>m3xN22 zYaWhxvu~{z*|*Ly1~uL69JqA7)P3vBq`q|w=fA9!t0J6x%p%LVmFZhQ+jae1z_FZr z%vNw=8?rWqb4%Yk#=rk_Z7{*@U;=Z$722$ z*2?|>tghXB(@`$)v3&jkfv-yeTx#W5)_-;z>w>=Z|8j;Soyb@}{^?eK`qptHovd{G#{Lwd1*Q-|k6JDGX1YfX_N=~j zoFD)3;L_*G#%$B7Z(sLh;F8#e`-X#oO-Men7+sOmXqAUg^hD$Q_^&62ed~3&p-Y}% zj6L1S_jq`_(Zw?3`#YjEcy?`qE{aVoU(Ck322tbZA}c*?U4(~?$NtB_!@Tab`+&(` zgW9SZ<8KESdf2%sJdC{~CHeZF8+5O=QgyF2o$i%8n8-ljPG}-<&y9dPp}>Y~y4RM| zhva~R1$V+Ufx96ETtHCB}(@? z@qk#%lgjQjVxu{vqAL3v>ZE&}__agMy3qrBHq3tS-KxITz19f()L8iS@KiB~lDgLo zxs~r;PvZE8PUD$@?MxGYKV&Oa-|AjnJzx>2Yj>}cl*5TlO~9!kZjEzv7jmn6oirdI z_q*4gW*5w0T$&O!b-gg`UN3}8*H7KOZb<50C*2BRY(q9hIQOKNE$8;S*MAQEn(d=i zDJOCM!!~3~3g?#Yb#h}-YfCWZ?QX`r9g@bpm!o^dIN0u9rzUo$U2|RFlW!5=m!<$NwQ@4+KON)JpnDB==>8uJdfDxu$(@4ur${Z6scW`B=&f&G zj+ntZSq(&1e>LT7hj<1&5X5eOCEe?k+nlU)`=q*pR`)vP z2Zs%Po@~sJvc7$HqJc|d3+~i>hnP)BFJv(zbPBYlu2WBTva)-<-j7i^qYfj7h;2Dn zIm8-r@)AcK(pgP?8d+H{E{$-nslT(_%j;Xe-PPtA2(;?O$<@Jy^uW#L(+v8;y_q3&g zdv^-lQj4eg_%Ic;I@{`U+@a;tAG-WdS#`_?{Hm}f9Ut0^-5_<}dTUbOI)nFr)46v?IQNW>mUDZ3Yv#|Z9{?Py4Vl6D58IG^DV$sS z))~u0t$o3Ox4Rkec1RlV-iy8!gJ8RFU7y&u&UiwA?~4Nd8w+@bnYz~N_m7D#@RF(m ze18hyQY%YX|Jk?h5Bk>N6v_X=h_@>cqRB1s;~)E#-?vuB(Mr=6)VHr0=0zoDyh}ov zpq2RXPmg#8JP^cgXC-~>OvXPw?I?$Ce}q4zY=I$V;Po|vZ>DeMc+%=yXZrCkPa=Ju zY|M$5`t~iv1}=#$xHGZ-Yhhp$($^oJBd0xeo$0Utcyidc-k=v?>N=b44W_Qi_f0xe z*O@;#qBM+`*A$WK6jRqy&i`38_D6VF=^)F)yzX^hkIij?!>Ssk^T35wvas7k1Z;QY^3*+B}?semUv$`bUeob&6 zOo3Y}@+^P+cQEK)Yha9Gda7ZnW;|@+=NwM=nlN=On=jV#q_TU3)Kq77Ar)1bqb93+ zExXAfXWb}hsFXQ6yIVD^8G^1AevO4+4^NfU-RsM_mG53>bNoZ6@yx(>rn!H#td(kL zb+4`-un5$(yVuzdIh@$k1e_Y;GIK|FA-B5M+5Y|i2aWgAp#7>+#Nc?t$nNzG!_;)I zZ@{G+rtV%}PU>Fg%mG$9_rVC~p0m+%Zm$jb?GZn12OO&nnZx-H+mH`ZIJdMRb6yv# zd=RuDbyKw=b(}VYi+ffZGMDpT!u=ou?p&<@26Zv}Cs3;)&z$|J3wJKZKZN^n3b<@{ z(7nxFFK|CLb&;iTG&Ig;JAA5YWVvO|9TMVu`T>Vg{@R0%VS-La5@xy{@efv=&{oGZR?3jyq8g{RmCLle$wI#BUb=YYq7}>5 z^l#Urce}nt^4A&Nxf;f_HC^%FLjE_4|DBTByi;<2-MrLD%sbzo|A*#1(m(syN;OVx zUT6jTKzZA|^Ivl8L-Uf8EEUi-|BPrW)ikv& zp#p3PO>J8)oaJz$Ey+nvuiK|hJzJ8Twv;8zh1Uk;XqlR@i99ZwEz4lsFC8%t+*4-S z5_VL3%Ft0YO4(8EnG|+ZbHEUlU{T#9VFQe?QRN83*aCB6BMj@M%!nx)XKk=WNwB47 zPg#(w%*^5_(igiSNv!@=vGYT*^UD!CClEU)5G$r##QhJRm;0tItj1PlFz)G&GVOv; z>;jm!v88Od^J>qO(6l0K@hM5dimJ2T495Cwl(0piFf8Pv6*7i3mbHYP910VIE`Bdb z*gzv}R5`+yhQgM@pg%iA6)=QIF z-=4&Js44NDVTmWPhT|onUKO&IQeDpY2TS#hZLhP?{e$OgPd z;caZpZAPLbl)%vPzdOY2Q%(ebGwen#lTF~R@b`a(aC80fSqqHM0+1nGv!20b)PlQ$ z@sChz!X+D7bZMLbm&6v_71uk&ggXpbjIN1r{rOJ_x3xb6YJnk83b?$R(2Rd9xGVkf zkH>~^$wszdG)aKl#D%-k-~a6q6Yg+iF}fzeUFpw%Lbz@GQAP`lGE%^e6E53S8}74? zfe4pu%n+l#{Xj0`3?#O!yDIJw6YdCPF}nJ2rIxJfj;yRDUq{rERfzuu6T|u=N#6DQ ze_VnOtCp-<2`;Q9Kc}cA9QnY#SNZE-KeLv=F=J{w9!KY$p+bCbbXWvU{qiGe2O;H^(Ox6o|vsUsYv(yuNbaXG>DHzmXuJRbg~laN&eZdrm9;aRMV8Q0$5PjWH~GdHDFQSeuSEVnoxVz z6gk9%dJM9dyCxyVSXE52#-IOtR^5lJV_{Wu4T@uuPIl?F0}~Xw!m9O*FV!lq8j!c-qVW0&614%dLsWh}5wD`!%uD)WkNXCK5Vcx%@-wpQQmpvbbDp0Z< zsqv?C{1eppDobrSOz&gkG!u@uUql2O(f-2WHaRD)_(L@}R%_-~lC z8f{mBhFuJD#vzCJtVm+hQ~JHa2+ttvG92Pdh1lGg)`J~YOVy%u@gnSvNB7jfaQVEF zAWS%P(X4qTL01qkC*!ul9Os`bwf2Gec)!uAhDOWSnt+&JthR2eL&G+!1F|@6Kid~H zP3s=Dvzo%(sYl5HaSRehB@7ZctR$&4P3wLM$k{Z-AZZsKa2pH~CQ749GLpz3;mlqE zncp;(nOJ0N%$=-3LgVN`f>gRQ*9Dx)&@$1jPW0~+XsSAfEmKo;vl;XFc)i4K_DrsS z2R@d~vJY6)w;wDTjFRXyV(K&f_=jm9ssaFd?81%(pirGXFp+cO-4^* z8$H)Q19zx81H{dc#i=~v#%KN0&RT^PzsSF=rIl)DmuL-DEocH*t_>iRTm zvg~}DQ+@j(xyIfkwh*6vwgY25h&v&RgSY@BsP1QT{VPz;ARg#n({iM0ZzIO~jx1jg zFJms`;p6?$=40>%eCpee@HKczY~ekphfQ3CcsnCYs=oUir~R)=KOBPUZ8jM=b<;eE z*DuL-p0elc2;`t~GQ>l1_l3DDsydzqoSQ?6z42o_{BSXepZA2aeBy)3LmfAJrm5@t*T~ zI#&4{Y6VQ|+mC4uY+`xOc}r|!GxF;L{*Jf%a{T>Y-r$YqtAw0z{)?ik0tqTfj&wrj zXWEd=$S;u7)HPBt@}o&eE>O=u-ezG&egTVLHOcws*u+(4q=M?p@#BAFsw1y*);L6TQ(0RU?H^ZOC<~SPJD)NyoauU;pxeYEaFIDCP}r`$jXq@&k-S~jZax2C-n*O?!RIr zsjZOH#y^4!LQbuYN)d880(oJW3u=0xS{;Rl#(Tk^xtMa26JN|Fm@eq!VFI_=*iL7D z_I_k%K~^k(itAX8xV0j#V<2uG+PPHXt`l(`6U9+OihIU3!I#8y+T=@@Y4(R6zV34e<2T=jVxBM4*mtR^;I)>IE{y{d*09upFpum=6^zb7KjV{{q7BT$_|~f zoU%oBLrqF?iA|ft9G+;69eq70z>ZFd`8VAo_UIa6k4^7b_Q==>F6xSt))nK!vd1Ql ze`t@M#vYXAy;z2A@Pm`>2k~bxqvFtgJA*c+uvu0m))RqR_Vl zqXUzRO$?cz9^_mq%zNP%zL-Ol-7NkOn5lhjGyLD^QIMis#X&az0eKX3El}TM&EW}* zSBxCbAS3Y|crNt)zkr3W@{#tlzEBnSViEK(k66ZC8Msp(gas2wm`aZE&Ae z&?CQ~VD_vE>%!yy(B&f5e+)#0dyc|pZt9Cp#d|TIl;1@&kr(Y3@+Op@l+8ux|ALUY z;y1PA1odxxSk8LUjo?CO?H66h7rlf$EaZMt$T7cVA|qcK;vb>3TuO_z?Ddgk5X@T| z3+8?in78z`xK#fuxg1)%C11EmBe+HOf3-R$1!h?k-y-WD$is5lhjOZ_&yXvy3h~Ty z+_0zG&g|7|Htwe>znp5R8si5R4=NmASX4M}Fr1=h;qzcgK}X?@V%zc;sKsLQgx#Ss zt1KzngH6Lvv)pQ^qCKdmJ{@VKxG!$!&|QKB#g*i3HeVu}$tP}p@fasB70T9Us^U%* zQG~~{S6rdmswNZ~T{w39(D6e@jWoz!1Q$vYjV_k{-_mGl(OziujA*o1G$Ng?(cc}E z)M!;9@&qet)O-rwu-YYUoV?VinqH@hJ5$7Qcm{pNHNy5YXoLb9?v>9AB1N>hFOCFOJ)9JX>)2J@|gX}$ZEW87(P)pmpbJiD*=vUj+TqQ z^VHF%k?UPh+6Q+w%`Uf*!-J-3gC*Gt>&uR?;m4MOb)J7WU7qT0SeMQ0(>t%AtY1Z0 zOILpx*Z+cg7#ofGNheCDUHD?Qet1K{%k~CYF&V@SP+97F)eYZr@o=E3-dr#T(C}h{ z%xU>InURUF&YuECXlODxjxQ>ZM_S~mg34Gb6X77+Qd(?W{cb}g93VP>NxC_Sv+9;CZM4!tg{vjfBq2oS1 zS$YQOGuh}<< z8LQe`W^D2Cze7F6>K~jh|_!-vd9O^m6)aZ&1yeczHCki4E9k-w9prDsE)u+nw zHt2Y2Wvtym`#Or3ZU5NePA_ZCJm`157xPIqzl!UhVKtxOR`Ytu2%}xq4VgHWKPF;a zeAOsxT|xJecN`nAMyqI8 zck$%g^Vc2z{HK4S%|weGnEx1a3`n6xYfpCc76fo3SjfqcwZ|4Wz*PVf5L zAui@!9RCw@VFcz~82zI{u270fO_0D-yFz{J_$%ivLf&47UBZ z!={ChN14)*?;YY`-eZSlufsXus(~(!G>T*yv_x#mi)1AUuhE4nO9PWjHaDjeO^u(~~$%?y6 zWrj^eVGW!1-Rq#h>GxPWeUt2_hK@H?#@Y?EucKJ|nuZQ{4x8FAPjJT#^GOeKjYeK} zGXqUGvtGJd$soPnHD$=e9%Nud4|2`>Gk%2gO^7}E&mYuWx2 z^YJM#OGN5gj(^w>9WT9JT%DoE$m|aBwK2w}Dsq=K?}A~K9x`g&c+Aji7Cs9!lp+n# zwSN55lf(w-)S_KzfSzPMLXWpg8X(eH4bXl+{^`-N0Xl=c%_S7bYO3iZe4+u`@5etw zZPw}GGF>}X>ERXE2G?p)8;Pzx3I3EJy6k_zu^M$*Q?w1bJR-Vm6J1DW>*B}%LS5Ds zG7ri4qI1i<`#RbGV3o|7++Jp$!-RSFsEX@?=m>QgUN~-i@sPm?=@tzx^vdXv$P>M; zyEI9!(~5RLug68N9ikWMEWNIK-l1a|omq%HnYLaMbieLLC+|M^%6dIgp3{V&`^bvx z#h^R%8aEy)4fevIPXm}0bibbC|IoczP-}~>fkC4eMI`I(|R|sPgQE0puqkc-nNcr55?}(y0rx$I9I**Dv+eIDH zS?b)t@o#7bmd{y*$P=^Gkrv^GE1bMk=WL#R$Yox%2%{^mjUur+MH3e6WTl>+Uyi>FE-!M1XX9$`Dg-{@0MIOWe znXzn$rg$H%Q~%A=FnC38P`8*(MhG{v{pWz?cr!v6;|^FxdMe$*_0O=fGMu}KbDVBe zALFYRjA<(bmch>hpd#m4!kzJ-kUPV<53)j-t$pZLCLFG>_%bQa*NFRHLYWM-HPfr5 z$%jU(EAA{lcXE~y)mvym&XK3_aQ%J1IFXq$;Ek)7GdZM_`sGtkG5H+lTZL2U)eZ8My_ z^rX!UX>=7N&wHMf-(x1$(F%TbHgdagmf6T{k2>5L#C?{nPFr%vYy_=NB`RNq_gWBz zH5<9zU;pxYYWmg@dX)^5;9ILIW909kg9^2Ydxo>CuBdpxot=uvj zxt;spII=%JVq|{+^Pga3uQ)bT=Ny0NOMFm_jO-8e0~e0$Pe?Jc7dQu|37iw8t#M`} zKBKdR(GbJP4dS+iKs2!ju16l$-XSLZ`5Kzosxi|D)`(?i2fJXsZGNKWfe%1KHHSp0 ziTGEjW-cVD-P^w0iVwmUE(ddj%Lx%&4i?&6$WMKCV=MEtcb-#-Jmz8E$Y%11I(OD` zWH5MiGA_n7_ql7{uZz(>@tL?0rezh6U}Wk=5JfQ~@rF8gp6ZBV3w@=2K&FXkGafK) zMujZLcoDgivnYM1-~M|_&p-=3fq61%E54XdkldBwAj!v4@H={$YzP68Wd=!ENy$tE zoy&dj6$aQ1^j*SPTIjp{_TNLoc+n>Pl=Sszp`WU_hg96!9tx|4zU%jnj$?rIfSx1s zBj`A%GS&`t6l>q@xBs4wY@yrJ4o-MLn#8-g|DCmbxT)orq->$@=KP0E*YJo6d^h4> zK?U~WML%uq{Ck+*YIN@Y5M07KT>v|LD01-&oWEYhUW zxo4@AD9@4i- z@NbaPw;7kJ(AGw8%VpufF%yQ4ML1X@5+6VgWl76(FV}xUd2D&kEINRe=WW&|v^)o- zgWR`pjM6t$T&Jok3)@Hs z6`ASoTVPKSzA9fkVyMdcBE(JgtD?$Hq6+CORqku=&{3813XvzcE|fK4GTu<-zOhbT zs`3cN!romB{gpmYaaFKi8ma`*ypKePD081b|MUES%A8en2b6hNl(|EcA)T!Z$3J0t zQknA$k;gnf8Cgxa$h7wUR!&|j^QgX6?sLVo_ST9k(?HdOGAROJ??WWTiCXtx=7^+P zXBXWLwcZl7ZWpykXQ_3+zyHZI2i4k8h&*QT5Lt~{Gw_Kry8myE465~*#yXkl6j6ot zS6nTx3Jo7Ma9A;WMhRBF4mp%1Iz7PgPiS)LbWYLr(CH=7>3Y$Lbe2vJ>~ZL*)5b#N zF=sW9)z;|&SAXjCxW)j%nb$UYItcp!uET7(7ah~^F zsPvFO{t0W;c}4Jc{f4M?tEfaeOQnYfJEmYUZ7M{bMz%__#{JMzCoffcQr{wjEUa)6ed!uk92J4Qy)JUr6K$xDr%);G#o1;k%(thh#qzoLx774Ekofl|cd9y!yIL~S+} z!NK)wqRq{s4e2ax9=Xq>uq=eF6KiVi}VcSM5P(KHWY)c`<`bK%Y3rwhzpm5xY>oq>2m35R7i>vPgenO3m*$q9}|3)|7PrCgdBO23i=*(;c8o)zmc-H5Tp%Vw2fkEX$&RprIlu>x2sGs8e zFQ{#b0gY|?Av715f|_D{s4_}tYO*J5wiaa1^IswR83<)eWFBY1ZMfuR{430xkErnR zePzs^ulgDsy?XS?gFF5Dw&AdRZ-9B61kQQ@r;s+H@}sMXAlJu zNR@dS^M6Z2I#>&RnzUDNu+u7I>js+KEeAV<_^-vC4t5mtI0tZ8fnkqB8hOJSzBgq& zl{BzyD>liD%rAGKhNU2U}V$ zxT~K}uku@bkc-4Kp7Hm;jE=zk495RKJj26$%(2s}2^YcqOfk&E7|fVx%+Ij>qY1|t z%zIN}eg^&@N;8<7oPRQAtq$gw!HIFBF;SQ?{|~ri?gX{Eq0f&~UChrm5X|FJV3twH zv;O+`I8_E}27bTcDKfJM6r&LD6j`5=9z6>B<@GK%@QYE%$l+nw_hYcLMj_Ac1sK+4 zPY{Pxn~Uy5v+`aH84-NBQ$``AvqmA$`sZIgI*vjvDMTK##)zz@T{#(_@ZsmMTf&mT zMshkHH)dRnuUJc)UnNTiSCX9o$p-HY_gvJeR*FB>HJz54Vl?Uz@6?4F&}eS$>V z$>%3I+!^=XtPjb`0NTkzrk$*io|S1QiNb0ppZ~-`!MN{1JvTg-G`BL=4t111=>>oP zzh`Z>lO1UX=MbqhOfSp@5?1nYrjoywvYmY4B4lFy9T!o5U%1_>zh0Zuv-9NHP}*uI zUw9K-*yfB+QGb0ne-}981x~8C?~5&LX{OD&=HXXbi8N^^U&Q*C!8|?!^NZO36tp?D zP6M^-e_-3+UCb{s{z=ReQ(&Hkyr`xxvi)b(nW)N;!kN#rI5SloV@rTtTX%R}Z~p8& zv^k~awmDeG892s0D*Y*#P>{@iUh?;Uc*5B3Tv&8B+MN%i-ML%Z9nx9t&P$^mkZgA@ zEkvFpNyZm_=OnzL&`YN~d8yEIIG*F}96`|cV8s3;j7}$e?wI+D!L1* zyf3QUC906lQsw354m_%ISt0V6oeIcmRAHocn&{3nUg~-#2WPGu3*c(ElSJ?l% z-u41FbejPumNxINxJKOOKc;x>gptLC<4jbZ3pErY>b!EUBZ}%=T*RUC=c3L-q7LaS zbzXVcp`$vN7b1_DW+1CkM-sAG0K8f|_DNt7bm zyz0;YLwCER=s{@nnP~H%XhS+nn^(F19U7e4Tv3QTZS2AkZC>@qzo9lSVT9^EObXiE z{atms+PvoXe;z_=b7dj&MwxIYS-Ri z{~uNXI@{$%*c6~Y6^$McjYwx{^u{5Fjv8H6h&-)rjU+Vm2KWC3GGYw5UCVe#d*(%` z>m)NID(lm?4~Bi^PmghjQ6~_QadW)UXudhpVa|}-xyS<4-iz4;$D3=MtlW({T@O*k zJp{*)%HYV#7sjRdBo06RWn3f1klPa)vvhOG1ww9^rKdQqUr?!#8%0u_=<}8z|L{aI zam<(tfjg z&ME+91MFMzW+4oL!TN8Hcepd;_Aoc*`o{()9j^6#nUp6Qz5SdclM#_4c}a_@0?=q( z#m(LC(sH9@=}}PF%t$) z7+k5HL`xxuvLx{K&XbPjoKaq-A3>v`r7$0!7;nEXHSgK%fHNsg>XNRkh4rF-*w0T-0beo9(kfX;DK9&-9^qyGpkPDhSialT-6#2d5($>_sZRA+UTTsL{rOKQ<+fO?yCLbGK++be0QRAO|6xdS zyNY$gk2P?-!lGe*vG9lM17xr;l;JNyxXQNhhc^c~VPT{80etdhS@^@(kc$?cl4xOZ z(+@HJ3BnO7m*fXlUGo_}m_f?vqsGGJqzEn_A^sV1;WxO&o7Bs4_K!{m7Z&Ep(Y59y z>d#7ivZ*znxwS?xe{`KK&0y~MQ+$v}^D%S(3pGDE3Nz0C1ei0gC+6B$Uiztv`Qy5R zd3p-WGW+_t7xJ)zPFH21=JBj^^jL{uV!krgG{mL7d*LRe{QPo95131i#1KE&^!o?! zqZHOic+|&x97!AyUr~f;ss2q284&$_7?qI+JxON`h(Gqne;z^(h<7kgJCgCm=5{XL zknxGX|JBRO{(43o*)}vn)iEc=P4qXh)8-();vGpQ_S8Bj#y>gD(T)A~4*jSqK8_gC zkvwWTk_uUokzjJSMl7HB@o!J78R)hrGmo=KDC=dP`tffsFW1XXD3IdJm#e{W|6re9 z{qoDdbb_7sgTP^}mwh_J(UP6^3%XQBn&`AkEAA^5x3!1D>a;)g+dof3cG|6YDWQqV z!PaM1#?~Hp%hsO_a&%;;-Gz2=u3M0*{2Al_tnbrV-6BRGcD9R|dRYq}!A<4tm`LK2oe9=#N?MBT1EGRBBC$v%xF^Vk1EOYUzGV|18gpG1;z{Nq~k zy5=#i%aQEkNHX+)31rtF5#yd+l6@3O_E8eq4b5WQi$t;yqsYE!kVJN4(-?Pdlk9^e zGO0>mVEh|)ug@X3I(Xz258;Cu#NWT*_}4YSO-*9lrc2)MC-L^lu1_Mnxp9me5=r)6 zB-zJFWVbYmac3sU-i;#roxlDc7VfPLW85rAvUeiMh9yb6twD^t@<{e}B-za<4oPHp)QNE`GNrv9Nj5!+ z?9SS;!q+^q*CNT-{<~gs7sR~kk-Zv8#`}+5vb$@>c<~QGz7j?Dhs-3hduqgZqZ7$q zjwHkWhXiT&R*&&s9+JHjNwzeJ?7nI--n~n*7bD4TOd`8KJH|_HN%le{*{4Zl4^)kr zdyzr*d=iLb7Kg$vFOZr9B*v@lpbkJrhOtC(Qp7$R5dz@#YMY9f~9?NFsYQ zBgXqFNcMCjS#taLSS-eSAV~I964_Ds0i%mQCAWW%t61Ta__cNxJ{&sI93gc6`u_~x z<)<#e`>1eOX3sAFoE_-1)a0z0^#nfovSycm?uJ~PW6enP3mJ3$c@#2m%r!H`P)R1^ ze_k%4W}2ZAhvrkN#>@b|8a{%{{FvP*@%0Z$rdElWP#sA4oM5~pyHE0dl9ZFPVrIh# zNS34``Lbpb$+URP?8F1f;*=x{aoFn6-XAQB=P(I zQ2iOPn7O$WBzyuhULu8aaCpp@{`@B-DN!+VO1*j(J^~u%7jOA0)Suom1Mik79DJA; zbI}*T{i{BK4)hil;Y^dq@s_Vx|LH9=BfRCSGmwGaGAo6*v=>qPMAR(fEu36*tr;`Z zEPmLBpGo0Uy_-bUtwziQ>Ou8D1l3<^B~f*+9y4)RP~9IzH86=P55-!BpSwM(yCbN+Zk|Nd3pMC2kLs={ zsxe7ay-|to^r-HPqVmuGh1TeUdUS_Jbw?D{BT2IQqADHqs18O^{WXcI0JZ6WM|B{P zN_wSlIwn!|Lxs8>RJF43VUv+W>eM&>{C}2dfphRnxA2sP-2#8JDGU z4?Fw$>N`&ZM6k)uTdUxw6(j%F_kTY()UDzvxsqA&Vs0t2?-~TW(G1c$FanQ6crWIv zhd0#uu0L|JFym_1{kh4|@C{TO1y17pb0OX58#9Zy{lNGiEAE0|MA{LDFj9Ei z4?D!$79@Jx8DNJ}{ox_U0F?by;UDoEy{#mUuhlAHfZ(!0mKADzj%tQ|&$#)zXy^$z z1dF5wc$-&tE?@XcAXD)Mm>>Q4pH~{(DDxVI8{R!X*e>@#=6{Q&rqPLFI*O>f9KE?H zaAgq}ynoAruc?Y2$;hP=9wCXuU_XB45Yk{z7yb#llCDG*{;4wdvhT_BOa^yr0+V60 z$-p6ntbxHRMtk`Nn~r1{>~H?~Cp6gW7}a@KY++~7gOxE@BAA*~%j_%?oBiz%j_S18 zwjwMx=r6=(kNP$vA&JiNVE#SFA*9U?751j_JL~0n_cRxU7WDkB^85G2PGiCz1*-p0DM$jPSU3C$H9U^#OMAsIZ2@p+sE`CxGbCIwcADxNS`D!mow+sJM_6 zhc6xg6N(Y1`^ArchlTV>o0z@=IpEw^c%lkQW|!vmDF0v*oSS!w1xMUZ6B*(RzxQoUk6Q3xi ze{=rpwyB@z#PsF(bqGExE-Gc0VJ}NFT7=jup zt3-nM#D8u|((AV^WBN+uf=^soaVg>x!L34T!GSV7pUB4eEzy2o92wJBAqRZoDo;+Y zl9Ga6Gs{0?1fOs(w7Xe^cpXKyIxR`i?;6MSHsnG_vCRmYRf4C8<@M@SZl{#c$I|W; z;fwh^NzCsXd2t(vIlS8b-5Bl^QfeSRDy~=LeL@uE&2h3brzI)& zRh^i=4mk$lqvFD>OQsSoWTGTr+KWlj{!%-JYpsm5{r{D;ge$EmsRVBrGTtvq+Sj#W zxEoBV6Yx=SY31GvMqyHs<2NS>{08xLhHX)i5ARkritxq~*;&c$-?vyfxB)pZmbjtf zdL`XMM*&h)S*wzC`3?&Ww;=}{BP|#BQIgcJtS^$JeUAl-dyxZmISog6C`U@E%IG9n zf6ew5C{Rj=clOC@Zman5->|v;0qY=l0tkA;!#pr%HMdn;C8_hHteim3VIH%y<@oC# zp+0}ZTFZT&K8JZYk#eMWulh}rK7WsUD_rQ^5BE$bg^15(Pf8N>6BcytLJs&`8jizK zj#Sy~r;}v;Bg0z;LX|z7Q^XYD4PmpZ`RhMn3H==NR;Vz9J^ZuR6jKjxG_BPxhdx|Z zSeO|3#3F*paE~Y)#?Jo9BQNHN;V`8isZ0zg;tYpJNYCEJHz8 z6+Mnn(C-=2LE!OmKNLhlhJwz-8wyn~a|js;=a+d{+_Lu*{XdLu=Mypa;qt*2NKlHg@ zxZGeiRKsYG{TD{_;nbvFw$W-{>F7?Q?JUAmw)7vxXixe^BO#6E1^2RRz6wGH_ZCI2 zS!CDzt1w?=)-1TtdW?Q3rri~?TrfQXZ@6zQfB);EU=6oBRs%BdD__hdHMiDCux8C& zlvs1$M<%jb>?lUr$0B;`tyw%0$JgvitaMd)DJ>fk@M>DO5{nQqm7Akt%24#f^PKa_ zekBO06qV#EG8C=txBs5Ia*)U7UbqyJ$cg%DXM1OKOrZp=HkSv zd7%fhUUgw*JQl3Lh->vU_fl1*Y7V%bmlIkz@+UmC zKX&k7e2LF2U6N1|PAOY6yzKum_Z@Ij70LhaZ5~))S(c2jiwdFwn}A&qd^4~p z3QHD6aanekph(7y0-~NdojHLy=YW_nfjR4$&O7hC^E^Fs*3;kj+gKVF6|dDYi?QA(mZ`;|ct2=)>Y_)Sib*ZPAB+%>RKmVnxluwdB=KXYoils>LX(gR4rRdYjyfw1M2%oklQ=dluEbBoKQ%fVp#8#j0Qw7s={UOzey)Tm#m$di3)ad{ZSRLv!5pS5Y z^S{JAN+*{t@=XtS^r*+!t9VCG21?<82HJzH=W6y{_ORIVz==osY`VJUXfHmS{`(uM zNTozks>_3xRGv9;wReEipq{sfQU_$xP7S4`nRe|3_H3q0Q#4Z<+`9Vxe<_OcRpl#!$gID3I2MI-Hc0c5k0E=_EtOTi8m+Vwt5cM3jv7yQIV zx=+q7PA2u4w&%k@7#P3Xa0@^b?)EAVr(F0gYTrx{V80o`vtB#m4gI>Y|I+{q5)B|@ zVz*7knCNif*ZJ`2&79`3fK*|(zkmy?aCwR}Hv-CzG)p&~43&{~Am-Jk#XrJ-(k zgw>-A@;MD%!6w^{yg5LpXtF){{1a^ITuk1Em|X ze~8S3oqj|z3vXC@yGjcUvqwKuT|W`B8ABZ^GCge*&=gWFC1`*MyWA4SM93vETp#7( zu_cvqdn~JrYG^_dtrfyIEJE7s>FB5O{QF!d`e`OMbvSvR7Ytr7J; z$U;QB{o)}JrtF4|Yma1Nw`4?x)K19~*Wdds$fM(2sT`+r;wYhqhJN@&UH1N2Fs)Sk zylCD_akc`ESm4=4*69kQ5UM`aNmQ5NtU?@FsHn`-3Qa0s|G2EnaV8*+22@n$semTc z`XpKFaU2>>MN?GffoLYx>q%4_a4Z~7hf`GM!EgpuX<-u86*%V)$NecP^Q1qMDwafb zB@Uv*d6bIEJc`n!Iy{N$DxAWH!}t`Hc@CdRbxRV})i{w92a_r)^Gs5c%3uGATmBjx z{)sa{6_t7Xr$JTLBT3e^ICVL8ho&;mT{fxw^{=?B>u@1c>~>9MUdd!qoscB!dfYP+ zyG>J>H%*vS4HjtGOv*_sr=_3;<9eWWjeT1M^TyA>6lbp|F99>g6myy%Zs8iFL*Jjo=&2=71yZX zHWfu>UZi4BZSN#e-GjSZjC!|TM{l! zQdH)BNhZ}5NwV(5y-~P6N>Q0NN10T<|BuVM3s*(r#z;kFUKVLk?a)3+*4^IIZtN;e zb(Nep^8)^9QjJNXy2pFUja{j!t`w^06R3_%qPo|6(v4lAsjd*J=Mt#={V#F7HhNp# z*al6tL8zWhp!#Q$toytzZfw1#S}#=3*i<`m{%5ztP2Ll3>~c+YxllcwKs6VEHW zH+Gq(x=g5^N}&2f64e9VV{YtHO?9bIJ()oDP!iRH-lJ~p5>0i9P;E`1^4GuOHhc&d zzT@6^)i%s4-;G-Cw{w!LhrP{i>>^Edk;r<&mep@|64hq!VK;W6rn*q5n5i&j`RhM% zgpYU+xv>j0)dfQJSOV3vNwOaG9&}^OSz|OmU#K3nsrqLnQ9b58;Kn!)1J!v#^+*C$ zO%m1PSVW4QtEtWvs?7;h{`^0#*Aw0*H}*$O^+%z4IDyKKf8$hJy!+f3w@(7$IYRYN z0u|#QyY+7MHoCF1HPzWd^`K3)Q&AGtlis~%Ww#!lB%rwbJWB$LX|f5mP1taqmy zJ55uaCR7ZN45|UWlVss!VmHRrJgjl5P%%IfPwZj@MK;5K+x~7$BKc z&nHp6=H1}N)@iDBLd5{dpxPxTiRyLldN+2QraDfj7$BKc!;`4~i;8Dh5a<)iFs_fA+3$V@GSM zqlJnAl1X(}64krj1~+z;raDTfu1TQ!cM{cKy!CGENKJL5P+e_P4J=Kf`m1-j8#_W% z9U)WVk7b*rwCe^7)RDbg>bz_HVs>6he0g_4eXcE=my-VEK zp_=MYp<;k!Qhk?1^}ctp8(XQVRtnX6n`+RYB&vUS7rC(&nrekmF+eh84QfuJ`lolH z8(XfamJ1aFB$Mi*B&rX*3*6W;O|?v@7$BKcFD6m_%RAqVE!9*@g^B?ZsXF3~Jiy>Q zP%(A6GBI^IKnnC1wmQefM8Su#$B~D*%tZlgeR@2rOy^-f#lK0!PDajt{G$n(hj}LY z88S!{cwe5e-}ezxHbqiV$e(InOgwq8U@bf>aOLvQsQ67GxxwqvGBc(rh5@Qy)$9EW&i?poG_+sra8d_k~MR9-r z`p+Y4@4^S_!rx&ubfDD|=~{Nj##t=Ap$v8yJ@h|yEzy))2!_JrxbFWaJB(PDzp}%q zx}p+(uBz21+N>Q$!~Fc89tgO@Xxwl)pSj8zt~-Bq8f>+5VV0mC?|FEmP1ZP8m2!vC zL*7jBqS#?H6J9RG))ZPjBcDdG6e&Kf<}gbvecJZkadJTkeA;oD`ZV$<@5k__T7Rri_*z7V+n=C!0u-OIXQ2Q;!&km1(J(tT4OR-$ms38pZ`~P8FE>qXa zJce@w_$!|IYW%&|ngCPhaK!2)fm?9Cz=2xefhh%6XC!b?g22y{1a8Ga1DqeA4riqhSW{7% zk-&krz}@`(PrMGF?C)ZmmJ)bCN`d7W3EVY7;44WQKGo0l_SXXUPbIKCv#XDGu?3Fw z*FWMKK8@oCW@>>mQwkiGk%l`b2=t%-i3@yY2NxSdl}YzY7Fba`q&6dg18jk#`X!n4 z+3j8I_EG}(O(`(5Zn0B>!1I#?KG)avW@v#a?kumUtR4zJh(&x)u zZyzmipHv!FXVyxYZZHKNktFbiQrDZV1x`;8SYBCPH5BK-q&w(haLC;;L7<=iifi~H zjwzU?1x`yPa7bpYbO&1?dR(GOU+V38Q?lT!#BQk|KnDN7J|OOn7>d%7M2A2{zxDFtR$ho!c_ zSf3<;uk~;}h9nR;F{QxFI)0x7foCQOe7(EtP0#`-q!gH0$L}oyJK__O;%FBFnff~{ zan?3h3iK63D*jeM&!70DQ4bftV{0>?Gztm*(vFZN9HT^u_D1{bA6hE6HeZ85x?u>V zBHEj!qrLPZQzP1|6{Wrnlv;?tFh0K1sl>N3?r+~wy>zC`FTGVv6(vU$IQVn}p^Ix|S z@Tx1FhVHtKD|P2i4+%~$r4nQ;zGIJ9Ej@9glXC_i#_)IM&(ZADgD&jJT0G{SRbuDm ziFRh%4tCxX>)%0MPw&h>>>u|o#g9f_Z%@WQY%52lXe$eZ(WNNks4`!m+EMqk51YUW zyzwgH|JeAp@y`5lT?`F6t)Z0N|JZjeJ`4@UjSvlvPN4xu8JJ>R1In=V9<4HygpO9H zUR!#I>;jZm%ut%RT=L^)I1E?+d3-3N_Y>P|5@!gr=wpDkZg^5o)Q00#HL%@(SVFucJ z??VH*{DpGoP%gdNOpd$RFqo#>#vllQl}8Xi1|U%BIPlbUFXe{pa@G@aaF1v=kzPG% z$eOn-eUA6zf7Phew*BX#=*Q|VPb__Y5z%R}wAulW8~hFX_6g-ct?qb#{-fb+aN0Oi zJx>y;IB?d>{vB`%shE<)nKO(#VPDI*T)z7_wiOw>4YV%SKDTC-v7$90iH;LO|M};* z)t--j$*mXo9c!OoW@0s-o%kDf?h}p->-9T87rN?I%Fz|!7yaVzoFPNsdVl}NDmgK= zjWey<9gZJIy2I6>Nwq_V4Qpzy@Xt%Fm@{nPu-X=!Ys%N0q0KE7RW+y+`@q5ik-&*R zrTFuoxRmKt?lAl~${m)zl%(Sd+2JXI85D?_=s*7&7qd^Li&VMexi#sFNpV;qeLscF z#+!ot{V#DrGb+@B(tEhI=?lX2gRXIWbn3K*J?o-Vrp4-_`t;9yku^=^eIl(gE^ps* zcPM@w;SNn-o_$auX9eUhEn(7>BpLe+c1NJVNOwf~GNdI0#}r0H!gPGXa+5YDNtrpw z-3~wQ=5D8@45@0wHQoO6xfOGU4s04yg#~85nwuJ{8j%u%l;HkN%#wBhJq3s=lebF} zvj0FeaT)2BYavzDHBF6k{Cnb43$ac@%ytULDI?qT zKO_m6+t%F;LPoi}X(5$0bDNrj1AkMGpJte(qKo$6KoPzefB#p!ht6x`j#6SqWiCef z0R}OQ2`FNJyiqIr`1>E@V&+HO-SK10-93FVIAfFHOR#rKX~bD5#mp2@k0cpxL4nJ8 zP1GHkzNmyqNlD_gno?xmJ7X6};w)-a;w-9BLzyvNpu{80gS?oWD~3H{u}mV&1VB1g zL>#Er5G2T)@t`G;LFRvAPa?>?LW0aE{UDQsBwCF(>Sx9`79n>fz7YKiaVhDVo#LX1h|%_LOfn64Gqz@P^s;^Vh%QvH6S9uX%MJexL6|zs|&LiH%#EE~gdK z#WyAr&m6(zxBqyfTj-4u(~Wh!F`1Yy6{bRJsH91&o5}o7kSuU$zE5LM!+ckX`JVR8 zM?w<4p`p(7_djW<+&<*t2stN*bl92raDkqIsR*r{GXq>lFsG_!?!fA%X5RyDU^(+FS9O|{Kc%{Aap9c5y@ zKk{FOfhv$(e*^RRhMlM_8mvyCfn*M6ZABT@#cGvjoJ~&QdB#~^ zCzOV|;d#ab{rG3K$}^I8NS^UP#{aCb)rmDW3`A(T2QCu9$ErNzY%(HDYTQeo0Mgky z0c19Q;08v#$l`Ro$;RLRpx8LGkzt&N!;J8O-U&=2q2Iqv!gQvy?rz)ImgB z&ttARgXWnKSdf{fU4`o5zvhQz{f5XGl>l9XpZ`48X~AEZ+-vISG_X}8f6y^IL1IRE z#6{piG4t`JsD}c3`gz1=ips*z6X-j#VHW;>k@NfIQs!hWclOXguAe%*jE@m`b*urn zxJPy0Pp<|vOwIgQ5#aCt&^62e<)!F1xb2qwkvse*l0gk4AP8bFO3EQVd0hM4^wdp+Q%J?k_@wcVmUTJ6einNM*1ekABX{^r zRvGosgahz58DJZqvj{oBM!n5)WGn{Q%>@}WYzz(Q>BG*8SGU<2REasjR?e+x!a_i+ zw*i8`F~H8T`+o&`*csjnPD8DM>3%Q{1Yt{h36bN26(jSoYEmg_}><5OrLQ9z5||BqKu05oVV3TW}yfBe$0 ze0UTvcL3xw!Z=<<0puML1uS*sU8<$@ny3{*uYunT?=ezK5t)O){1`-eEd6sH|>msc9MNr&L2?gF>gx zp5ykZYZyOq^xjiTf&I6H1_UO_ig|wgvn~h?TH0f-h7w4_wMPQd6qr$DuvnyUJcfsJgB|59P>GxwkWj5A;FdHS*oFuzAJ zZyGwcqPm82xbTht$~XrL^YI$_0)PG=XTD)~mruYr;Qf&1J*2v_ytcw$c1YQ<6z^rA zB43~PNl8|@qDJ2U0p9m2-gBy}=hoD;w0=A-jj}L-H`o8!cjLIP?4xhX0Pi~#?_pq# z>kM0eFonEX+Xn7~qDj23!lujnPzt>7)4a=vHaFGGZT;as@@9Bz^5*^z8~4@NefWUp z{ea>yQV@igHLofjTtVcLLc&>y|=PJ;>o_HexMI;i>` zm1VUbsEj5gk=XB$E*2r}_e%6TC2n6{ZPffe(|eeze8@YFwuVJdJ`n=1N%+*jrNOUDx`|0{TBJ_pSqdSeqY3%gZ-`* z`#tB|kA$=zBSh5oqJ1o^wBM`I@6kUM>*K| zt>${^^;Hu~AvID`#9T{OTUxV;eHFu04(G3+Ak6iGZ!Qv&NHPRV?z0GKuGgbKI*!Ud zB17+a2MhU3T-$nnoZ8ca48@aQa>e3THLZPEP-rs-^5;m@op63 zz2qB@gd|#{kuK-@AES}D@!p937n3y7wK&oLW@5a=>jSm*@_+pbb%L2AfVlrX4v40^ z2v56c(_!gnN+zaD1*nkfnM*$_23ZnmyKiGJ!*(}`?Oyh6M?%_eHQwkRE9O~*Y@av`{mPWzjKmQQ#t{3CUq=sDLL93gZiRn_{ zDWq~{t6S;kKjK~XyVxtR+|6RSSA5Hnkd`|UZ&+?6^B?g_rsdv>{(@&BNO!Ok{UsC2 zrMhcTn-5%4j;Rn5A5#fwF5ZB3XpIGnZSN9qnnWXbDrQ5_^NM35|W4k6bYICv90)a3_o=v-FzqJWMV}f5JyK(o<430b|purX%n^y zF(E|+VF~dx;SEfO9c5vn2`}}g#{=T&nV2vorb4P{26Ttn{-5^yA@&;VcdOX%HL;%} zr2QnIJG{Szm96ld7--Z33v;lmBNO}K!d?s zW!^rLj`tjIpG+)gqba0TN{%$SBg!nzX|f+bWtPEGu)9`S+zEFv`?qNBqk|Lz7+Z&5#%vCYzDD$$T`0)XFs3k^cI3+}HgS`x8ud zyO``xzR5^Plf8jA>h;LeERbBkh+#)MvflU|>lcp{WVU{hUVK^r=6H2ml0i&su3t2l zR}X7y8QOYw!&<*MYNe$oHwdW9Sb9-UO#kIFmUI}i2X&&2%=L?-UbFZ|k^DQ=!`V@4 zxeDtSTQaHajMgs*o@bNXCWx zHONYcaHi~0u|Wb7Dduf?bkV9|M#|nL8fWLbM=ZyOk+NTP0MIcRdSZCWew9D}IZ>tT zp+RdY`&It<=a+`M;VJu7?}+tJR4IG%rWx4j`09bTB&?3%^FK^jotT)g;%Wp8eT?P* zVQ4U7%AV$4{<|+ns|YSfQ~E2Bd|s0)|MA0%n7||U?7!lFdM^NtO>|EfwUPT0fAvm3 zm%LEGJx&Vw&wpsIMCURZYt~qMLYaA3vi_esmo=qGktbb`R#RSz?02bizg&+Nfy$}Q z527j3x$i9X)oS|Yw2I&kw8Wd#c=ek8m2&Kh*phEhOSqa1J(wE6OAKos)&xJ;dInin zhonJQuaX9;tE2+H>}XD4c0iTT4^GkUAsxor528AiA7qXi?P9f^|Ks6C4}1GN4Q1qy z!;SXO4y~1pydK3;q^PyV9{=d|&ZAg0j6tpaGxd7DT90Y9*4$;8jSp|%RV~lgUFTF| z5()H8b6P5@TZXo}#*u;Z4Qaes5u2pAq<5Skipz8jnwW%lj z`BC_et36y!*iY(gzf2I|0e@gutYu7E$Mat~Cap^xlf>(t9mL&HDe53 z_g8S?7<^KSF<7FA<8x4kQN&4V;r}eK1QT}V|8W#?Jp5mvfj$6n)cU3^wg$)Z{5NWF zataN4qb!=j@mo-aE$w6#ML>hrqKMUD zW^%nMeDMF~dR1CBRq)~hBTNM!Q`OQq7h_VZk9>m*ja{z|{)fxW7Nf3L-D^Z}lEDb= zVbWo%;{#Nc;(VZR|8Ky(1%Kg2YHr6U!FOEIhJ6u9=B5_u40RY_I(hei4JC0G0UFOq z2M5x)UQB7+l2O7|I?2bU>9&bzilq~yXn*|E?UoVA`5rFHRL6cGB9V(S(;JBhk%lDi zj++0;W1$**{z-{FUwrk+^Nil5w`z|W+Mg9|`0A5S1s8g{Q&V_4@zp2ahBEZkrz&5) zo{cWKM^ei5eZNZ>ZBD5d4Ngm;f%xiEmZ1!N^=ZmiuO}xu zLAT<_BJXd*!rix*6u^p?`oh_x< zC|IbrYOB=$Z{Mp;Xp%`h^_Ku1Jpi{!%`0*mR3nPIMcIiWGwA=>-V%``P4_gv|DWox z|5Mg{%3}YoXaCQJVfg`JX-}Fd}C`%yhmR%XSss)qyFNA1 zUCRJ_)|1BYq&I3$TXe}RAT$Qpv;G4v9AM8#F+8z(!lY++6ivtH6=1+%FAv z!()JRUJgnxMhrmSAu+%?KcOTW)!B(Pwg-sN7XG-MusTb%h1OXGMp056dKa=G-!ghtp)|ri%;2;m3N#(~5bG##RRj(aZ*;Iv1y49`i(k*q0 zog$vl6dgTn+{F5Z$GmfQfC_pr{GCi_<78Kuzjg&R`J30TrTpL zqqb=0m0o=ir0*Dy68_B=f5r!YiNTbLe}fat(eXig{IQ#C^kns*9xKC5CpsXEnRvs_ z=lSO!XjM0$wSMNU5G|S!e7CJT?=% zCBP}9UP_P{?ZT%l-C?wRC-ydscDESqZQp1lB++piBMDmY<2y zFhhgUf&)8rgUbnPaFCF4PvAx8T9DY_zVud$;gDuqor&R6Kq{nuP9xr8yo>WJfi#{Q zdk4n5M~wH5Z#)u`=xDrA#TV~x5z=^nj}l7CwCFVHsky(tr;RwI zxn2B>g@so8%3C9D9L;S_CRR(pQb?uD=5~p{|4X-RS}ZH}XISiBvDlw|i;<8PlfmGU zr4~wB?EP58amq-C!61@}#V}c99}PAE=KZ?C2t%fdmoWZ`k9#!O*WOyELAeV|71w5B zFdIrCRWhfFm-_qv;`Yjpy$gG76nnkv+lz#>*Kv5GguD*FR!ya4aY3&^_B^ zVlUknW=x#2ciog=49#z)d&d15;WTPUmsKa(?Hdo5wbJ@I_$w}n~Cv)E-_`A_G<+p!3UWK z8~%tyI_Ty8`6u!5;9EY;tKuMZ&~=&EEdfm-)l!0}KHAOYZ(6F;X1TGy!e*PqW`Ff< zMnV#;)`aVeEkfGtgIJ;C>_EC^Csvq=&3G2il>nYeliCfK%LpyKP63q}e2;vtgV?$dt}lB-!+7 zr%35Mf!30Ch5-fB3qcD20IzPhWiVqhQ#wOhhL&TF)%w$OT1Q!txuU|-vjQ{Gy?1jO zDrlZ6XSXZ`BptSF4xMNrwIU-4oh#0;xO3;)X&#PERlzS3I$JZT?6l481M%lA=ss6= zvw%zn*xS7Yl37ARXMt8Uy;}15bH23zZ}LAJ{4pbz9L6Jd4g=L%gmUV|$@@0EbnnW~ zjU0wMk0N$GtFY0~)lJ!??;H|NPQuFX1_itIi8} zGlxOmAvuhzHlrkS80RGBFql|^p|5&RSShCqR$#;k81j*V-*#)xN`LUoQoPMIXC_4` zEo(FWXRSG3ZLa_4-pOB=&-n9R_&{B_E%<~2SKI!Nj%5$b&eegyE-u~M@;`MfkEGlo zRa;(~@wziBFk$Z)|CG2HEuC#{$(z!`)}3ADjw2wVy+UsdNjup;E6s%xsld3ywIOJuJwos17^R4uGFavE_<( z&&aD$EJgZiePW5FSG$8pcByy>YRQpZ>3cO_t;e)l*Kz$bK2~#Q+TBB-Fq{8%&X9bK zzIJYNQ_HYc_e-mS>A?sw6HyG$Do!!X!567Id62}}4p)3IX$Kk87((&Beb7S}m3vgkA zg^mOlj(_M^D)alK>|5I$7+3IiebK>a?@n1Kd&BOKfT zw;|v=uDPbV^ye02&>pp>TIr?%A{jXR%$Jvn{5P64o zkqp`aX8Qga^?HPIL-G#boigum>-!cOKKr#EcQ-buM@W%(*cW3A{A_yKCZH*#T4vti zw#k+-PN4H+f5Qa&0qTx*kH2M=Q4LKv0DqGS^lkq62f9ue=kAF*(Pt840=>0Bp-g*E zM0)D5gX7ih#SBbfPN3)33>$*b@>jMrDjlWm-M-8+5Fh?}3+J~q45NL}_72l+FCE6T zV<*~3wQHF|-|nyfYCCcYeY%H3q17S9m_k39No{8oY!Zkug}!5dOHpn*yuia1cIprs zC@NRjrB_d;qTY)QI>#vV@;lx!1~I)SqRS4yjKGh^Aco~QJq%xvVi05V z1fx556iqG&Mu)6?3^uH>Ru8Ycllxx?@q!SDcOGj(9)78`e;oB}6n$5J(tk-hqNZtzbIl4Kz^y{m%&z9IzhUF8N~ zm!qI_&A1Oo;==%Z7vn#MHdm(rd^gI%rgt47fUl-ab3Obmu<1okeqJ1LkFrwu%-PEt z4;wys$+889Hm+zHd}!mc!E+ZcY;IY$eDD#A2G1L|wCb>B<%@>a)K)BPI{2_d7nN5m zuf?stc+Fchc-r8DRxVnwqO!7jXcdm<8j7oEYAS~et*sh1cty+d6$=*48`M0f6mzhA z`KQ`yIDh&4Wi2hu^Bb2nE-FPBlEWvK0iFsB)=Jzx%2Hx7tiDrEr$L<4|75vnS!3v)vQ>-HH#R((mE^ zA6ELUDJs1hWdZq~MFRO&w$g6vp>VcKqx8zjcigzOvKTS>rqg~gj`j{gdutflR}8f7 z3*heD^S9IKm`t?X|4OuXq(Hk4Wr6nIUV`=x)$MssVbDYuIQfNfyl%>`j@hKj%2*}t zVe0?ORC#yrWgXUEhrgrYtz2~QqQ!@yong!Qq~%96%|E1Z(V&Hkn;Ky!b%?=md=eA) zKeN&`CN>#h(g*YOaq{8Y1G8CQA)0n37iqiq{$O$9!x#al5+~niyU;_(Nh*sO8*2k{ zRwa$2tFjJNS?J#&3c5z^e3Y`c#`V5+#~oopxmc8jQ9enV#LdkWLDE7uUISrlp?8GT z`NpkAoja$3+L_byU*F=xsPl~`zG}uiyil&fw`e{pWubIXZ*u)bx~% z=sP++F*Eaf4hNhZCu>1-DMFU)1&dbbup4X~5AUIPgWvsr{#*0w2!7)SIt~2^r>8T} z=!t+M?9k33QgINa?)URQic10ebHJrUrAh)+rR;?yC+Qfx3F2YXCQ7N=UYIiz!~myP zf>O}tkmd8JXx{P_%P{O|#aNj9qzgRI$>K@HCJd6pk9#?TY{h(*fCjmWg=qG`Op}e$ zEMQZD;KZ=E(v0LJ-4}0Y_P~WEIp*T7GYm!4OnRi#%Rn)>tkl`X8L)WSyg`ji8k^>~ z44U8Aym;}FLGu?J+A?VA!a)lb4_dseY0$)FO;cNzHMfWif%zcgKlUG00eRGRN`=#~ zi*8jF2F$@0%sn;C2R)NKpCeiTbD#~g)al}sEN@x1pmE`XBO6yNSiES^+{McdX+-ma z^(w%RJbk_wSbT3cVGVRDO!ZsmOS6)vukL3i--7o{z9mkmx^924K&?D9*d)*MKz@+a zid*RPio`1w)+zuEUugdo&fs+FXWXJCD_4w-8>L>E2LE5+%==%*e}fwrwk%t*Tz82! zrB0<&o(|-RD_0npeU)KtWiqk)!*eaOO-|G5Fq^Xa!~XsU?Ic`~U&g~*R3q(-!&{72 z3S&!OB(cwc`LCdn7qF3sGKe>G5{pmR+}~u7HyaFsX5K5YnTxZ_0Xbh}AwF}nAOGpj zw$!vk&%{=(=mTxYL9Ds?E)z^anaQP0O=9D=;-%zMy5%iiU-a%U6tPUN~*R zAuXlIGPS9wuEbPtSWP(_F}X9?q6EwT<+B9_h8`rUq9k6eN)Q{8sZ_yZ{{9c$M;|d6 zr;1MeD&T4_a6R@1Q}evlAm6pCL??rtuDOQhWu*?bu{14Sgo!IuSMAIk&cP688sPRo zyvg}*7U#3TdCUM=^6Tpiux)BHhL*sf7>SmTTmHYG!emzB3@{p+ZXP5j>7E+P<97e2 zmIJ#g;UmXtITdkgwQ? zhEH@f&80L9Sd}l6KTco|pu3&Q$IRiKCxD{+rE4&!yMno3r`o^Rn<^%v7G> z{s+doxa{`v5=6F@{3D}N1s&=DhRUcwb#$WW53o|TqWIZXv zU%--VMTsq)MTspTN^BWwD)AL4oktGX@d11oN^D{Nk4ik5LJ1BEP+|-6Uus}_(rLjg zat)L~l0;8udto{o_$@OXfJ#H8tHzaq!p5(E&4%?Kuqnn|Hef@cikO5wHSyML3j+z87-s7VG0fA-F!TiaRf3NJcth>2 ze*E_|56M^4>pWK>F9Nur5?Pr-T4k=)7o3z7bQifS36%RJcyWf(8KALMCt z+uF{~(`56XfpOW0$v}`i$@*tkcs~&dZ2=@t9&H-yP5jaMaHZrr6hW}-&<8a-~8m4bUx%4Wj|{QcKbHbf?) z>r+h@I<|4-WObgdk5gbcCmS6^vjG{S>r*#doY>3*P9;u(K@N?CoW4m?Hcv7C!CL(w z@E8S3L>;(yM(r1T0+`M&*RAS{ z52N;PMg%bkudnfCXSZ%lcedAo{3F92l%x+q&CJh^7$W4E@~Q?P6Kxqfeozr z!f=y3QUnk?b&`mEs?!PcO21Y2L}Sp?NWs$E>D7AM@hM?M|7#COrDSOeFKo1=mqGf# z-cEx$&$z_tZPX;BqauJzk^1D{TI3v)LdbPP@X04>-Csof7jFoWvxHnt7>78$;d9bf zG0va71>TE{e*@%`m5v;Syn--!zSUX_zG; zYT9p*GJs!itxAxJF{Y6(ooO+qHxH>8yIy5Y@TC_l@<|}yt&gM6mFVsCu{Fn%CLNb{ zRWQCi7%iSzSjbOAd?wB3S!LFyeW7beG-o+rx>|!r-{X1TdkCNw6+ZvNkm%!(km!}ujF8A#4=U&8kFNU^A4W*@ z3fDgw5`CE>B$Bz|E3b>XUj~t0L`8a4@kq~@8*Vs+b3?>{iF3mZ(WJTIs}-Wemmx~L zI>l5%%?)4r@apNd60aU6N_?F{2^lP2y;zj^+G&xwVXhxH_d?tpxW@SO4%0pzG?RtQ z4PW*1ADRt)RluegL36-{b)<}B%ne`bV_{eV;x=gbspf{Sas8XM`*nz6UNg_XP=cJ{ zyPkEaZJ5{C|5>}=D#N6l8@}e}Kfev;hHYKNun3w3m6$v?eC-m+=5_kSQwEST<%1SxuM|%-1xYQ&;fJ3 zU|&d((=97yb2&`?f0-)IWXgs+@%IeohJTX$N2x})2FA>b$f@jooQMI*_zH9o4JFLE z;h*IBZ{fsd9&jpgB8G2*MnX>CGz)K-=?$*`vQ`TMkI_zvsLP3j&J8!l_1=iNVI=k3 z@Xn;U;TtQZ_6tI4{|$`)LG5ct_)E>Ij{%jK2#gm3&s)M_87)n2Jp z6<5o2gwN3tLg6GwxG~8QzRCEHO}KrC5^o~^5m!Pv!UsOt$c|tr@g~=Q(ceppQYayg z@J;UjV%-!4jxgVEB)!l`0uL7drW_$#h~Ws|{K#U%780;2Mhgkput6vz8IJI+oh)KH z!Vch{$`QUb6-8;7q7cKph5T1wm=2h;<1p{}7d~qn<}I%O(lEu!Fex43TkQWdOmW}{ zi(G}gDA%aOWJmbcHv+LZ5u&uMx4Vk;-YKf38c1R4w?~Nd-l}cQhDlOxX6l@3Gt&a$ z;rIXE$~zT-N3g}H-r5D{s$RP|=y6vJw7%joYJ0z0s(nNh%05r<-v!`N0o}^9V4Zli zcmHU?`9dnV-;wyYG>$tPg8a6_<8rr9YlEtpX%RrYaz zYLV04=zy_R z?woCl``uDYFmXe0gA!cgv@wPiEf}Co-0wD6B&Tl!dB#r_jKSuiLVn{6*$}BAB=#SRWl<;uIRV zwP1j!Q< z&f(TO?G4-o+ZOkGP7-biZs38=h%?$7xV2z_GI77RyG73S-KK-gx9gn_2JXl<#=X%J zOnXCcgE~O6#OYw*)`9`b!u_p9PTXxV#J17?q{zVCX4~Ta+Y(DKaYJx}5?taG8Mw7z zj{AoH{@bka9@AHyy zw=0%sTk4%+12-ll+tQ!>{jZi_;)dXcLfyWL4cuBVK$*DT?_-hE-tCb|Q_qsrTS?A# z+ZOlxnEx3Ln7AQWKg3dkM}y-2TMPCxfD3>h;m!MhvdD?M1429Xd{@0wV%WR=w#EIA zNtR&ZhG6|*S&36(;MRf}ZyN37A7@zP#9f5hLI>S1I~%w=Y+Ky_EU*Lx?i=4Piy7OTds8{cFH*j~_#<>4w2_|j`Zcu_t zobCp0Ef}CodwsrDF$v?0moGOYarvuj zbD*U#dUpu?xXcn5)dGJ#QE;b_&$h^yfV^WHfMdtGz-fb^!^dkw_y6ktRxL^aJ}EtW zMXW5v&fQe062g~NsQ+wlsjvnr6!=r%J~)myOa;h?re==zKZjVHn1Ks8*^_CgE##~Q z$t`$=~Qqvse9(xg8@{xfLO z&L08O`Q?%$?!$-Cq(5l@7iRGWrD)Q-qbyq3C&!6egMzNuRdvPwNnLTW+QMk{#TwNY z$x8a-Coc=|L1DnL|2qJlMepNWdT=lHNdx$&-2`|=3gFTgKiy4$SE#-?Np}t1wzq9x z90+i*)TcKG0$b1k*cY??-ICTeN0t7~2N4N-3b8!@R9fMXn4SZ~b_Q1x&YyArC!M1% z$`&8-TYN7bK<)R=oZy?1$mVV&oIg9jV#DShuqj6FD4E!R`x*1UaSU`EoqWduF-j{j z%V&>UMaxjMOWJ1nT%P|CU5h~6EzvB}vOeDxa@Zv*LR!}66O5MSt|H7`mhD1|7%l7b zW#GatF+{a2PH^>$gKa6^Fz@HG{{dwVlDf*#yFc^Y{2a+=v#gJd`(KG{NFp*B!9V{% zgbqs~bbuE6MO&1i#fJsi^b%M3mm;M6f+dmUZ2A{tMEbB0>0h*n^bsjK6FVSO{^BH& zJ|c)8peue$SXRul;)h%mAOo?(7tafl5s64x0hIEoQA6rHdZ*!4JjHF*(3d3wcT@_v z5-eGf6AgDH;C@zc zM^oSypY`Quf;$?-4_GtyeO5tEd@w^uoO0na!}#H=MoS=l7A*AZ344hjzVhQ=jhAkq zGjg>OKeR23<3%3VcS~;k>u=Vg6wM_$emJNs^Z4Ow+5cOrRiHO*|KP0+%|Rm za8OeG@D2BW(j`Shn)EkE8ckZq4-?++w%Bgc-&_tZbe44~nzY0Z-^l)7l%ZPm;5}4V z?4Q^bZz)&XT&%u0Coz8bwpf7Ig#iDy$^?E606RBce&@Y5@Nf4O;CrS3EK7t z9QSwCf_v`(cP~Vs#N8$VH|I`mTF3p}TEV?{2=4E$Ov3%<*|(pVfcr_oUEd0Bzx~$- zxVKZ!_V+{kx9vTOO}%y8-}~$T^&z;wx7PnvJDESqZL)EHzglol4#)kyzy3ctz}-8q zP}#dM0e2m67yb(FFT~!HLva7lDaqb{%-c|JaNIxm@!zxncOP{$K|dT# zVB4EhlXk7+{^16}JuM9PGfB9|Zy4E-fScnVanA_H{UgVJ20}9e+@%HWmA%_1;I0Sm z_O0XoagN}g5rX?i%zqO4<;!2~^A8*MkJkzAnXTaF_)pw31Keei4hnaN1l&`AyF=@^ zf9fH)XNKVZ3Hu)sa6dL`>i`?~Pn`eL-m}7S|FlkU&kAt&ZNs>@Urtd1?tOr}=vQ$2 z^S@bPxXt;W_Jdc>=ySS_`@jDDZ+1BD|N8U4*#YkD+cFaEm(wu;_e|jK*gEe2o-FpB z9fJG6HzwKpyAE5oCg6Ti?A_D~?ytn&O#$v53c2vpFQ+&G_blKpZXNf}F~Qvwg8OI0 zKMC#RtmzlUY}`NpL2%Cv$Nlq7f_rX&d&hR26z)z5xSN2x)34zEL2%Cv!Tn3mB-|y} zo&2JW`xj4eF9^r|OS9l!5a90Do)fcvIVB0W7XWuj>$rcpLvSw$!~IGU?zMk^d`$xG z?*#Y4P}~uxm*8F);O^gnu|mI`&Iz~|19#_EaYrzz)B)kb)VL$gDM`5dt}fi)!X0sL z6Wohi!TqY>UL4@wsfY_d{c^e_;9d#bU0TQO_7&WVLvXvJlW@PY_r|kr-0pnAy(}EJ zd#d1G7T_Mxv8%$}H39cgz}@v%aK9zEmxbZ}F$wn_eeSZe72G$f{rAdn+*zFebKF@O z;NH2I3qSpGx+UOV3*6mW$DMVy;9ePmJL}FQ+<$+eZMJRitTzPr;jQ5QNpK$?;NGPZ z7k>KXbWgzj2jK4BI_~Vf1oz<~xU&yV!hOJieG=@QeWu_(DjawAU4r|l0QasXT=?mi z(<1@*X~5m1b=)~wg8Qft+&SBuxK*O}wcD=2bC$-uA(FGF;9ivix1Oj)auy2iRRQjS zArrMfCQa0CDNmTFMRM*J;H$!bzheN`iQad5Y#MI^&t?9P0m|wWz;)6jlFRikHt^L! zqBq#>IM&&gNfVvXjO6mdgp+KgueK~&qLz{((-mZQw$$FF{m09xsZOLJpU=YlrwygScbQ&>|&-^!Y z7pp^BR{l_RzR%_621BQQI34KAUV++ z$;bMqLbfgunNIXZ^8NVlgcL#_$o z^2sSW(-1AaV40|Vvho7!Q4a;F6219WqL=Q545Wq%xc`AQbaEmRRseJ?fd7vNoyuuF z^zPbY@S{;f1^*SesV5F{n1`2@Nb-`+^6Um9^NRffu2ASSS zo4qZ8%=CslfdYYWA~YhilRDEIX~X$X954MqcO*F!-V2<&=<>7_r*=~lKbM7iG8t<@gJQ? z(%nJgMA~ha1pM-yM*i0Z-j4HM0#9|1g8*+g7iHMC&kcZAsEld<#Ej`CJc`M`8QV>6 zNnGNNw7Xk?hn##JX@~kxu(&g`#7>z;+9Cc|z|T+NtOa=cJ_7vw0C=TJnf6ainQmGJ z;7FsIz(*xoy!|2petwACZhuA+@T&_p9Bo^?J>y?m{K8g%GybQ=FARWJshnwl^Z+aO zxCwJW{>=nlmk7KA_kXdeTo?kp1LEHVi?4rk@;x^24u=Tvi^G9;fd5y(FAjiLtE6fF z#H8t_!vP$LR1^4KNx=Uqz%LE~jxCyrz|ZNi{ns||qJ9GW(s1BKod3|`mj=LxsH|!K zoP@0DCd^*>Hxqb$BJd)v|I^}^h5_G{1pNE&x8ETF_+JG0<>A0PW()Ak1K>kd+O&U8 zLfUlGDgejdOyE-zfp?rDz%LI0-f?*n@LP*9N?Y!*;{^hILo2`^5a1gE;KNkjw0~mW zbkk}8M=sR_zE2|X;3(aoRsI zak>c!1OCkfJ~IjU;{yE3FyQ}40{+?NhI|`1c9&?clIqZUafh7-3Gk}};I%4q+CL{D zbGqqx0LR}<;Ik5ecUmRDLk?b#bh<1Fc)Lp~-cA7im;k>v+~S?M|AQ94HUK_crB3_j zB&1F^A=SaZnZTP8ftO(ZqdMocAr>#0kp%pZdp54LftMUDz^@MnUb0?*UmpM;p>n7F z6LY7VP6cq}Q%&Ftl7M6XhXQ_m2=LBDNx(}U`^jD;jdUI&z;6r(-kIy)wD^qy@ZD7M zw0~mqbkpeoj)bZSd~qW1&esX>8^eI(`R{}QXw18{C)pP7{D}a+IUIPGVgY`00DPp% zp7u}7o^Cn|z>!fkfv-#i-i7-==*Mpk0p11k{{-NlUcBN58+aG&e^Enzs>=nWb9Q+~ zfa}WzIKdyK(x?4%64Ixe&H-@z%>;f_BJi#~1bE0j1Cg#Hl7Rm+|Htob;9X}6@Y};J z-gTV-zdf+{?ka!UKQVv0>3jf3PSpgyHVHWMKeYJmAr|k7`G11NOUjPs^udV7yLA`f zcZLJ+Hd27!836ZG0=0i)0(H|x0FI=p3H%RChB*@zxo{~+*9K?*f^@Os;3BP3VYi1h4Y5i@rZ zft6k6PWWO@Qc|crqgGMQ`P=z({$7~#$EWQKnG4`Ol$Shd`tPm_?nO4MfT;lA!Cg zFN(5dZ3<~wy^b(imQJBQ-{pf1;9@wTUWk8G%X&z)tduF#UXKc>hk_Jp8+`~wz8guA zLhbd1AbThgnWRv+>m))qrw}>-0Fj^BZkPz&9HdZtxhlg_l#2`tyhZQ=0g1#e+xhvw z&0*3x|6^x*JVj?3BGNhkXJ>j`d4ct)he)avYJrtPWxPuUK9V26gf;YdA`(^rbnM;J zs3E;U?}&Xq`WZhOHPqV^xT#JmmVvo9@;?f$KB<_^W=u%-<4$;6JNvQekVL1|`%VEK zayoIO_p1hQokBgaY)qd7;NJ`Ir&8EioK_#?e-!Yig3YpR{XOVooi;h~d>+1Tixet$ zp_l@hLJfI91Onldk9S}M?;w)f8w=;O3AQmD2HFa|U=Q>dkW{##cbJ!-(oK9NA%lH^7TwT$r(``A;e zkI|OBm54gzTtYU>Zn|IDk}1?p_alYcE@cXJ)5fF}YMHBw=SZ413c1{LVXXDh_ zHTW?4V%f8ROxKa>WH|PGbm}tS|Lc?C=sMb@avc%NbuevB2I4yUa{fct5po7xr0+gP z56n3VluqsYf2SKX;PyRA;C_>$2XZ?W;P$;-YT+Bz16d1wuoF)$+AVk58!fsZ-i+P& zS0|)jmZfa5TmFr|r+<1^WP3mU)#%t*JNSo3^l{o7jnzhn#>(x4M!tXh6D>X2SjkC_ zSuu~HfJQ>j#(E3hFw^#1EOMHuv;%%|@$)%Ohty`m)((F;BFs#`y4OFo89yoebu$t< z4FL2z`17B*8H@b0Ec!SdQkxOzVFxH_nsJBcEj!SR4vs^t42?X^mY6^N$mF7v_l;-n_)Ws6c_swbnIYAjo;h0 z<&z{G`yH4hYzHN5Vz{uaxq+~)xzG`)QtJ3qpyN-0j^dyCy#}%LPv3;N(|(x$jf2km zsplW@&3XFpC@p08r~Zh4lso+;g@4)`Wl@R!cNYbJ2}X$>RV7x%E75qCcxwTkERNS2 zaQpT2J#++1vh`VCl*GE9vRzf&c1f%QW5ro-rS*iZ*r)&3VxM0^?6VW*KY@MRD?#l% z_|-#yz=vUaeahSv_L14qP7PWKlxe{%j2^P1?;(4^Lk1qGAo!+cN9=UQ>}aQJ zEH>=dWTV_;F;3|Th*>Ghw+;jY&eBjvJN?}vrgQ8M{?^71fjgkRRg`dxeNWU2o+ywL zhpV=lu>Tlv2TZrv5H8s$H&hHa6tGcnNo>F!aJEHExI3Xps!7&>2T>I59M+wi>J-h$ zfOn00EUt$jXOHN+U*N;2$DO(U!Fp_$q8xS64`@qr*G#=lE!zeoDwvM-7R#~Y}MiLnqcD^(qgFnvmV2XtW zWZgGsJ2=oXbSA!VntXhJQV;Dc*y71$EZ)TbLBI+4BT;Y~!JTyHnI7UK|ZgR2l5^}LJte<3ahaVhw#MgsP#^;U_Tt1a}6C_2+W zjJX>QV>aaEkd#n4-66RT^QfO^3(+9~ z5!$mlGJZfV%k+{}7&K|UZ9U$e%h{4iFFkSK?Ew$8IdukqSWi^A6MEu6iT}~+*b_TK z=~jE<2#tJDy#M<>@etJ$w^uz;_ro2__z&rSh6Z3kHg&>knW;cNJIyrz8jYQCc)wxx zkoyu@9aM`9r7j0;HJPf*Dgvg-bxC5Q%?!>liSs!pszniX1{_M7ZD#Obv#7c)G6OlX z{}r}J)#V}lNR}hrXt3=6k)lgYBXzTl6!ds%MRGC8V444F7V3I~^r4n_we*~w3aZE{ z=l&P%+o{!L6Kc5W`vEArQTeeZu^Jez2nL2W92k^`plYP>sqVI@*oa%Bk~pCJBa=#v zCW}I4+IEpL2ndFYaK|nNshDcYiL41FRwE_5KM5z`4TDsiX%U|##OffeN)A-N zau#2Ti=HC2u`*{jOD z((|vWM368e?ot)HUXWr036)&`u!DrFyqWPJVWv1$k{LllrR0AB(05IX^I4{E4h~0- z2MN8xg9P>(*uJVT0DzX9&t_Hc-{{*7H~Jd3Rd=JTQ_DzFb+1*FP$v$OJ9@k0j$T8KzM+>`X%y1?Sgr9-VGVc` zZ18eO*y;ZYDYs(*qtA=5*HcBQLHq?YX1d~m|T<5lh{B% zMCQK&iEgwfidg6+)*T|}U+ALh3d6Ajh)!$h0HuJ(TmmxZ^(yMfdP8UEB_rE*f%>QYO%9xT+U#>8BorS zTG?Sc1#%EKDdGw64cqB>Z=kp)RzC^!1OAa=vjQTzrcy}UKAUCuU~5|)G^t4pZk^R( zQ=sc${#!VSdmVOTz^z3>w=nmrG+=i(-0LuT{s9dP8mP#xRrfjqDL=>k{Q& zw^!~pi2UGdlOjJ#C!0E9WyL=#@|*5owY@WL|BWjjc2hPhhH8m;WIkoZmFRCuj0mvjdG6>aj{V(3D z=*0p#-E1QTksnQB3_`V7|BACvkzc^#S4}d!qe+}cmz4xnOh0@#tLS7CYPcMU;f7=V z*8u5)`08M^N8~qCMApgY93+xzY4%=Ww7$y zy~I}|CYW-Rli=vDoJ0hpzlyI&!}5}H8nMQrVf&>(74Y@aqn2E~9JFK=Jdur&avJfZ z#pd($aEpC*%QcCWpV|TZtzk-1hr5-dC{s@DLQ+n@q3suKb}o~h~$w&_BkCNhCN0x{-Hg*6m{e? zx*260RR%fCZf7BhOJ;bnz<9i&_bBfFVnup1no|LHJAQ8u&TLb*uZkWnwx5ytD0+^9H*1_nD9MN^WJ@s#XE|?cOLT4>KNG3 zOlRXL$vww6Oxw49+?a-O^<(PxrL-Eww!9qdn=1#-sj8Vfu)3)k!pqAW%NwgJDx1p7 z18J1RL@hMg{Sr&sWGC0T&^tg5)vt8C1C%b6)r~DREklQ-r3*RJrIPy(7WYydsy&YS z25AqLlPdQ5p)7m77t-XtJ&Y<=cda|`|EJTz;9yj-$MK(4oa%fssbY`wUsiF{(N*l` zfC*nL;1|ODKWI=+8Gr4z{JPeAxQOeBmI~(4 zDVpUhl!YauV^D@IZnSD~8}P(yw)cqCQ`GSu$!05PnN!tN(=w#>dTQsioi@5*Omtc_ z7M)s$Mo}336hbIyClTi7|Fp2%(eDdlk2?+CWiAS|bz+ZWnDOaIBh?-K&;ofTR@FWv z?;zG;Icesx&Q@6}l!Kpd#>C5&>sN7L7+jZ*-P zP(kB${d^`;M4QnV|BS}K`e_q;0@_?I+B_lJkj~I%H2q(^5mB3uq9|i^N6}`qpZ|($ zljm)gTNI$p=8UzW!%}tgiLDKGO_`{uP1_h+f_JHC^O$HuIzyW=Gb}o4^Kleqf_WM1 zMix59SoQBVqP~V8A>5r3%9)2p=c%n6I;^^72t0Hv4m;b)>mY0=;1kt82J2sOqq<8m zUYzO82?hgij$&TX)Y4eqGNd9M=432GWSu|%Rkh$w!ay^a@`;--7OTY@IO_cPe>8Jc z&S&uC_c)dKR+dxg($G=KuR!96krq*dF)wZh5#8zq| zO18p!ET4Y~8emqLuvY$|+|jJlv{b-aEH$>pVoj4&BfeD1;X_$UYHTe1e}DqvMz%A@ z8z&b(R5;$aOjOL}q=<~0@J2O^_4A*)#yY~{Z+Xp5Ll2s7565fH#x9ZKsguW73@NWH zudS>YGPJs4=+N@C2k~r2iKIBuXOG#IK4pM$oj1#A7)p`QXO_|@t+7Fv8!d2;2TkUx zS=|ejVKx1t`DMu5gX@2e8lyO_;%kb~Y0q{RKZZ^dhRa-}!WpihRx@5{8B*11XJP2X zIFBJ&Gv2WMo(nAAQG}n5aK5LS?w*+M{a)MqGAWN?4rTt?k~tafr=NG6(=d<*LI8DK z`o{OUle3JdNT4$3xU&e?!Zk~l?DbO8VCNsm)PMHcE5j8f9o;YFB#QM7W zX;af~b2(17w*t~Qc2C3`dXIa}l7`UVy^-UC=ZWH*Gf`Z#Cu>oBuWF0^Y-qfY5Q28c za#96*S?wP!KFdAJdtRy?Rq(t*QZcj{u4Ty3Uu{$m3g<($qI*4LA&H{FzTmY;9(<K5|h&y;!OX-e5h!4N5K*))l&tDj_zT&>y8}v!p{|MT-f0q5#dY zhcbt)MPCu+=RfzL7Vy6abo6>dB)7yiPr}t-!N!j1;FA)dlgcUnQ)@ zQKk5V@GB7`>P((viK05~V^2YyD@C2BL>P)`KqN6&WMp33Pg*ubBSY@fs63>%s z$DodvxjMm&5aFV-ny53S!V*PwI>fd@oeiSSR#As^hB{OH_%B{NpG8q7*qO!JnU6PW zXUe&j3{2bHL;PdOCh%CY9!@t#O5^LM)s11c?YG8FD@CRZtdoR7UPbyg2n>ORL#%%nZ}%t{e%k0e%w#UU??32l}XS<43sY#PV?7)w5=xt*Hzv@ zaw8XPeNZO0?&SFz(uUA*nm_()?hKm7vy9p?j3UYxbk=33XTusCWcF5r%p@6T@gob! z_%VGa6yo?1a!Pq*`Z#0!&>7GV*6uh>xRi*fMc~3HVSMuV7AfvbrRs_8E&Z9P{dB z%xjyK4|D5lCXtYOST(7+Lz*h9TdL=_+FY4lLxA@v_1p%igH``l(pqWdX| zA(7`@!k&Tg?on|+q%+*lJ{soMVALf7t$HJ%(%;ESY1@t(MKXr^xAitC6js;$2k4G zNc7q)dXdi1YhQo=lWs-S>&qz0WSe@;0uAbD-;I_G>ea@(UoNaf&i(%M^0s#A#Mh`!(w9^s+%)e(;{eQMnl8~vFibEo6J>qbR&zH zc9jrdRvwBvYW(^F+0imP1Yi~u^)zc2iyyP2^~k;V(pYtZyt<`!PEAX7bE`ufbBPat zC;5t-o^_(do7vGvxevMznZr%*_j_&c%cMMsz-Iku$zc&Mqm@DwM*`N(f2Vtf@Kdp-YUBke_K^5maTP4v99Cfz@ufB6J0L+nYg5k z4=ovt?Qdp?;>ASWhci((+ar6jrW#=1u&2d-HU_Ols*I_w7`|bJRki~3zj&uP4ZQ`) zX$na#Rv)k`g+>heeK^$sIXSEONkCrB$t&g!e>2 z1ft_?+yAq14e(Yw4Hcr}>P)O1*HNrJ`yR`JY+Uuq4yLuGZ_oZ4%EtRPGIJ`A2B=1O z-{t z2`|DCJI~q6l%_BbnDZ$&B^j9KEEdd@LNL$4`d@%K`%=ni_&?^p15V27YX8n`Usmd( zRGFoUwB2Qa1>3s=t0J&;6e-KH1s3T=L`4CySL`*m#HdL$##mwnJ7U*pOcPDarzV?>+b2bIW^lk3)rv*m?1Nf_X*?%=1td z9dhy8D8txmhHAz)3_$ktuunoKxDZc0WxB!0>MG2jo7?PUh^OipP}S8UvgOu_FTjvu zB%W&Y^Z!N^Bar>0d*E7r5JDl$N%lwtLOLq~Y4qcNL&tdPOY+uFsYg-mTST44+Z}b7 z5?R?v!YNPT3QI(>#HX_%m~N!sb#o9v72(=iP~ z_Liq?TI;Z{WwALvPM$bQqhQ9YTNTlddi)` znTXItn*Qw|p{IP+Ymp=iJY`GzO(cDD8x%w!IyP@}P|#Bz>}^-|<)Y*EOspN(Q9Na{ zKmRipq^F!pJ2>-R>A@`}K*DZ5LwEBW3-aHw9&8dJ=kMIH8VXa4 z^x&3_;6kr6Ger;f;an$hW(u5CUZ-WJEv*a^)P*-c%@51#w7f5vX9i$ig7`0nxx-f^ z|FW|CY~do9moWY#=GiGQFG5)~sU=fUhF)j3c%A)lymGcOkISB*Jmi|tRMOhAXSNd7 zjGsPx;tXUt%~U%IF^aOp^(=YZkw@2aK=haBt9P^Y!1esnxE@Vsxt=9oI&^eBU$IO# zlJUjnb1dG_X=y*F?9FgBw|G+}U3iG{rlzk`fy%@7$O%#HMgJ)xM3trIIbx_vw8oYB(B!G(C)IOklq}qrkV9Fb(+Ocm9(C#& zeE~Y%COW+!I+4!O>4b3(9d-J54a#&#q0k? zXhb?oqZRi$bkyjZ8kE`JZg`Y~N?P%bQ11oe zkq>pV+O2=q&5yk~vJ8abbx!))nBmpW`9>*>LZoU|`}6;}GJT_OLYbe6GH)7XG@Y%C zJN~n3a#%*EIKf^wG|9ED>0v7#V$Sz=rsM`u1agyebzI)KxVZu+Nxp4ka(F0+teo!} z#=o(iqI13%p-70l`Ra+ckK-1nD0jp*ddEmU2RM$&1V>(#FqRl%%{LBn=6rwaVFO)H z0znLc4Rk4X#8zjN^QB0N6MfeD^FLj8p)#anE@PTmhd9t@v6<7QbE*Y7Utb>@3@vc& z%eFo`=lcs3L2EEC7L)14TIT<1;9n&JV=9q!EfKIzR<^=0Iatna+XZ+@RUHR5~$_FwPLe>mNoMg-0&6Y+iP-SLkpja4{IrT4cS!NXJ~s`%)Mb82;9 zkXi0`BhXNa)Z%*1|6;vOF{5#JEEfcwpUOn%9FN>Boi~hkxGzE`V>Qb-3rT`x!z!mN z_jUXh$q%!L03>Dsr>VSZ4BFGTZ-?C_NF<@L;YEjg4J>-Ghm$lq=Y@pEGwJ)^OcEMI zVI?$9bjLroue z%qK+z^RXK9>XevIsK36q^`er!oGa1I~02yJ33F8|p#7#}dETy6J== zn+|gXF{bJl{T0HHpM_8m*_U5Q7(zNL4B527p<@`5OWu0aMN!>`=i?I%Ytzk6S$fIU z9c5cUjp~@AFw@Y9Up;-wF!Os7WKfQHr<48pk1-tm+R6-%uK8u*HK2|Q~dQ0 z!<~L|2JPUSj+9P*Y96dj$23=W@|#ln$x{zTA^M!T0Y2x{NmlvoXZ8y@d786pFH&!wlwW?R(^dt8!EZT4!Bc)FPM)H!i@Mg=G*tTB;bsZ8ugI!!mJF_n!YtDDDEwYl7c zm7MX@oio7EJP4s6nf`3%`WGFX5k|k$Kl*F=+8-SmROMeD)}pn$MG|C7`l@h8{&aWy zdX!6?lq4FRj`fd(${85_Ei}4cH2SS*L^?~O)BXH+T%!W=)>}YORBJR3pQxPE{rPXa zwf&o}Lu-%4%`?;2h}+kv*Uy+;S6?#|17%6H1R^L#)Y;^RTG%uBbyg zOPwuq9W$_k3dvi~oKRG&BQt?5e*Pz}&VRTBrrRa5AluVd2RR+6qxbS@ZQ4N+rTE(X z)j`GPc2IOLw0TIh*(=(R&eCQp&;P`G9JMKmqD()M@kOsI+HCdX-?%pa^{}r-Z|{IM zTlYa5lM{^=iWJdi>!S`TYBM)oi2p1bQ=5aMzkxRQi8j9xZAfQnv+YTTj@q;%Z*6ZB)paAG z;WoGap$+1SZE_qDq2adlwLxfzmgXmSqeT#8WvI^Z_rJuu&5-Etq0ED#%dD zybG{;hDP6kM!yh^-Vu#RXKA#3u|r3V_KTuSUt1%|4Q;>3E*q**xLTZ3N9_oZ8!}VJ z>ai_Njm=})o*w6@@Gt-&VOo9Q&nd=RIJ8MHO(Y-VLZolbx};@(pF25Aq<^;S{}}1dq}I++zC|dHws*Gc z|2UgmQ!MSeRu$*qTB(jaSyeNhx3$zypItkMDug%9nCqTyoVVn-JzlR zGwC-x&M3*<()=8E{9A-Jx0Yp`T+n3DK=X6_^$%0;6`1o)JyiB1R;WWYj#15{%MqTp zeMJvz;A!wAUztyyi~NV>8ku*y#KW05Jpy4q`E2@*IFtD#QCRcIbN4tX*vfwHwMxr| zty?p(wb9YC^?8FG?rdeVXb0!~o^d~$xk-0ZyTthrfu z50q7R?3+HAFQq0YKIE?=ZXMuPfXFe&6&My*13pFEb6O|W#KM-%U zq4S5K42Qsl`WQM#kW0cLj3LNj3=8Y~I$6>$Juj*Bj!vccDl`N? z=I0GjrZ|iCkmCOQXCY69V*v@XxcE?JpA0|$y%^&)x+Kz{^Rl)^s!zzFuaV2X%ZMEmH~mxpsgqwr^OXq4P)lZSt?xKHBG|KcQ%ghN%D zq$+^q-XtlHhCght}KUq%; zg1EeDb0I%<@y7c`2p6f$3%`)cTpZkME~5VIHH&qx`Dx0`??p%0(lqA3K3>MUwJ=}Q zB$yWmV7>_RznJDZw-a;s+uwWM#eC77g1IRLW|?|j^b*RjgEpyVP;&wLCp@R4_ZeZb za-4R=m5ohf%A3cGZ8LPhM6wR?y}#!-x*zya3L{&zsEex|NsNYvMc+d-{747|kb-$n zqG8fm(eTC0f5j0p8g5VC14zaf2lb_RQ^xg9Sq|1~@faDs!DtGfMuTI}I`x}4Vl#qI ze;~>6P8OLmXkSw5$mO8@GY)5ZUor-gXZ1jmmK8Z@le;yLT;i|)nmSs9L3p58DWEf7iu_lsGZ$=!$<>CuwLifJ#fSsCl=~kznlz{A%sw29 z|21?#%>pOYuzjgN{qFnX>SvZjF zq$IMQ-9u6OHIRh@$?i@f>(wn31r^ESSFk5Edj3_K!d^KmQX4c~D7+SJe>Y=YeFL|GClz zw+rzOB9i?*knEl$vV)65yh(;+p9PYAkVH15D8%bxN%mONn|p()>&Jg_X;mtuZ_;R=hYyF2G(GS}JaJuBOdMy1m*9PH zJfN~?hSx2OmE_Ejk;f=}@@35ouiK7N45?ZYLn?`uuKNKBFj`ugBFK@6^mV&M)KVSf zFsxo*5YoYS5k6wK%UN^p>wNv=BpdQWI>`o-O{qw(ACV;G#JrH+=>d|HQj+-j|G1Qm zxgouY4kQ~L5}D*)e}x#NInf}Z?DhWoXPoCrIU$cxHF%!r@RZhm{fAa-&*h+^79W4l z27Iu9@V%jZl6srMAzZwq`EGFd`Xv7NABS>sD8%b931z)QBH4%=)+b3hMTPWf@!~vu z#AvK+#0|H{XxQc!;T=I0<3qpLGTEZgI&XL_R+4Rw4dGN>#%Xgme1%eMb4wFVEN$+4 zT>odATb82D4K$*@=kI@7rrR7R7jsHNdYZ+LLZgVzi;=>o^4CA&RL8ap={!BCo(ZD5 zL)3~>9akLEnOIOg9Ypm`64hK(%TquDs;2^|ZtR*wb$nsSd(u!n8ALTHiE18t;1h=G zi6E-2NmTRE33nN)T|rb2BvCCuKip}kb_P-T`#<6~T!^m7WEHIOcmUN+{gPzWqq99` zs2&TV^4I_4vKFB$J!+^P4Wha@Nmc`T(<6rJkwhvPZEo`Ye_YmLbf||xwO<}SY%-FF zAHT`>f6H`+FM?mnd61Gla}C@!fz5@{PEjf!_QqtEAU03;vjFq z{AVKmsc$$aeUrZ)HFX}Rtt=p3;THOTYOp$m29o!_WhTlnuv@J@#m4%*pvGd0e$hem z#gcw$zy15Ap>DoWl5UqTmXs>H#h?GIW|e8)A((-)ExuSv#*kay1`1Zj>cq-`b!q5Z z{wXk5@p)Ka1xAEfi1)zUd=-jd>&@%R^O4Q->H^Jec9oxWGTx-2T{-s;EyYjg`;gkQLJn?l`NpA&W0B&vH3h$tE3 zpRN_Hu9ke7cM5)^zRPp*XI_NKw9z=tGPbg^%{8zHYDizm5&`GXEKQfrLr!$@8IVUz zv+xG4@B8(?N;Tt~HTdqd2o}Tfw;CIYe7Cz_8brqXG2K^)jMFKDc04kIb+R0&&;PGe z-$5XSv7bVzANcXVF*Y-(_cF7n&$v_}vl!06)&`#8EluqQjDOhF)~0A`WrE??qV!tb z)EIk?jC5=?=jp=qtp@j(23yDqekKx6ixn7uZqQEhK(89 z%%}R`wRm(>bH&JNWd6-^UT(MbH9#QX~35 zSED|oLf+ZNXPI8cVjxr6Ipt^xa2hP+FRoJ%s6 z6RW82iwv>d9e(~l-Xi){guERn1>5aNUk+@SbhYviz=d+=;f+Rd$0tcT^)C;37vt9& ze55ancP1u0r|4D@Bj&r)kN@Io4Hyyf==o7i=NM7t4cN%l;&0?i-J=o~;%cUX17%2u zx^q{ON&~xuaGjR!xfg0VRTZPEtIJ2WIi|7ay0`pn7a=0(M+YYf8ic9Dxkk{rTF}U* z#pM;1)&8AbsTE3c5yvM8x-3c1L707s6xu96fV@sem;v#=oh=>W`rd z@inx#%isSTH`q|`a#cTsV6fHn@=Oet2&RN;>8++>v%8r8q3bo$XtNQ~Kf-2@iOv4# z+l+)H+KM+}zT^@>OV!8Ky%aM>objYyWPr3gmZgN^~Tom0& zrW3`k6vlWANxIk?zUhkI9w0VgT%%1jIxh^vIEYLQ~cdk%2K z(vB6;Kf#VW#g2dS?MOlroo?{nGtD8S9S@0iR!ScRZdB3Enb?sL3htfbEeOVXn4VvB@3w-`y6l6|cr%kRT7ITRyVu|U5;t18 zceRWTFxu7oU^E|23H7p#c5i(C%O+2wjf{Q(qdg%;`@lCE328K$*4+CLjK(?O`rtX> zy}uX3uGe!w=B=8%=R%&Fa3}*)nFId3LyTY*zK!5rf!S3UpM0^DjIlo-i;^5;*C&p# zZ=(=A%FmA%R>q2y-HbV~nFBtWi$4o9NF1b{nJ9z=yt?hpU<41TvWN;nTS%#>7~RrT zS?R~5sUHoi6qTHYHx#|kU;i|UR$#Qb+{3Lm`Ytt0_FoOB&6o|gi#8H@Zxa5uI)>yn zrAs|LKtx|!vOsyeGii=#Zd8f`AqY|Q{-F*KZZN*jJ3;nKLD3V8qUmLjY(-m*qW50_ z{z?z~pP>dnRj3mqd_6uCkG}H}FeLiq!tTehNtpg*ZOnpA^ z21>GPZbb(Y>iiGOt4tG~f7JMo87#3gm{AoAZ6@lTzG&es-v`L!qem;#|@d_sW-ng5E9 zH1sVk=n^_M4oyUtDAZjpy_{smuI1ARe6ahN$WawzN0&z+Kca33oe>%fm<3R{sC8vi z^GUNdt!~CY@}je5)YjsU+{KNn%R?hjhDIqmp?TBf=9Npog4+75ZgjT-vW%+tj?axoP~WgR64{~e$Q`ZV3}D^zrk1gzDl7fmXUb7g7fjFj2UH^#6Q*U* zaZ$Y1gm}QJsLxcqVa`W5|Btzc5S`?2xNTSm;Ul?iI0L0{;Enbm>lVZQQGfr3cAp_k zq0td=eS|*LbPTVL`0fo=q*9_N^{DUvja04!{m84AYjvPheJ1VHQc9ZXV@0qTo9V_B z%~XcB$A*brH?o-`hzzUChle7-7;B^~DUI|o|NP5FHc}mO9tesQjr1|b|7@fi6C3Fo zutSADcAHb76nw>S{KQ5&CwI8Yq(0O3efltm82@o6{}HNz!@ZNYW@+CC~B^p3R#m9eWjf(09P^rzo{>65Duz*zI<1c^l`9lHQfBa6R65b>#oZFw4u!JCXmJ2)24$FJ^jH9q{l2x8uXoVCO}m!O1B! zkVd%E&;OsyMo10Xim`Um|IrorrJ-(cjJ2zMtTdXe=FK5GMU&md^B=6SlM`#~FyMiq zclr76O&n~%3XJmc6Ps*bv%K8XUS8J1auFhNVR<$RDKgrDH<|hEUo%sSkd`^!I~6=x zc^i~>YMybv<&_l})7yL=Tm&`To>_(XqlP085t@~mU2Qk#=89n!ldAmC{&G(>NmJz&QN>`s_SfnPMecc(ObunlZ#yRKd?V z8S__mcZ(NJt0oLk!}~O&+LM6FbV zwscA)3k;g44s_5k`&Qvy6VHTPgOLvYbaYzUCZH*yT3XOx5$4B#rjaufQWgDB)q778 z9x~-0hRdjiAtcc@A@ujZ8baFa@F={BK2g-7qFpnwnY*%O%!WEmBke{siwvTt{q=8y zh<2;=t`)l>?R#w|c1uQ7LhZCHG2PS8IQ2x+jf#E*(>*1o`^YyP2}vaB%BR0^2x+<_ zqWhyUkPex*{WCG0R4|vDjNPDRYP*s9mNLEsinO<9=>O@h<8~YAT_@RIWZtgJ#BK?2 zN~o9J-k!n!XG<83Hahxe80~2>+Mj)+k&s3^18;caXZ`pu-rl_E0dlq05*0ll6Qd=p z+8V>*&G;SLIzVhO1TH^_dth1*8({r-ez2d(J=qXly{^jfwsR z#(PGL_ZQ!IBqY(V4ASS;I7m5lU*h4Yr(U9YM0w8_W-t(??RyF#9Ix&w1Ak>_1)CZh z8nG*?vCSi+Mn@Sac01!gvZ#6L9c-F7~%6L?^dJe^S6M1 zJYq$}ObfCABUV&wM||iPL$xes1^3|m-$XFh5v$1xcJFrKE__2L%E1fOu*U;B#@wd_ zN346A#6qX&h!q;N6|wHQG@&%q4USm%?2MIW#7f>YLzpt~vJ&=tK0--GtfwSKtdbRc z!QcO-yEIsV5hGTbJBwLCjgi=hF%0gpte_5oDJ{Gc|8uf}FWB>+BCdp}YyWiXzwv>( z@Uwz1aQ+h?J?L7NdV5e!;2AQ#LOZ(GO#e@ZvvG8&F-7K#uPK22KVGX$3qHfUi~5s4 z(t-=&=u+g9X!wlW8pTp1JqyIX+}#mNxAvg7QtncQTU(i_TO)t+ep2{va`-dv_5^m- z=^5SaYD|GfUp{u^7z{gY-chRyrsrJ)%!JwBkr^1vO}kMYNs_!W*;WafgmJmGpCCq( zL=wd5DTNFb%%NE`35FVZ{QqIfQG zQ6}xcQfjPG>eVeV_RH}No57Qc`NlHlUI*xxi3vmeLXHOIq5xlHA_LK@yTOP9(W$`$ z(X0OXk5f4i!T;u{F`Yu?RxM!X^AeT1y^^%PYlb>&+Vj4`XarCfli&CFG0cd($;9uGa`T)Vp4;Mr{E3L zulI6Lb3m*eCpo`LHO^FgdZ2Su4)rBtpSjcj!)2+Nr+QecCy(ve1pqDOYbPizj0ayzQr44hwBy<$qIWh#`048oA^{Y&I<;-gEb0s8zB z=Llno-4ipb<$S6^);@cG|CiAeSGcGhUZXra4KA@!*rx>^XCZv{V(DK#CYIimXlaB+ ztcqV^|4%H2HF@L0Rdr|JM;4IQ@@6i$u(h0?qP4UdMsFU3GK|7c*LmXzLy344jz5V^ zGt6?{_|2Ow4T`z3QjOm4hCk!O)`0mBYOp1R26SmK#hZ+O*?za^C_I91(eCu?MDt-E zOM0tYLTRWQ9EHDiWUMq?W%mg91w`R*twu@qjV*~a#&j=WzQyxjgn7Eo8%M~9FxfX! z=8YrxAt`Ts9lj(pfB%nWhOV6x+Tv}72WI7+j!T@42w<9kI4h<+L(r`V1MlNxUf)}VVDow7Ceo`X z1KIGFG~e5Qa`-dNR}<|fm-ivf*R4qV(DeA*``0BjiKLNMCX|In-Cz6u-!w~Rt44V@ zsQL(-C3X|vkn^bWh~=*O6$JWFxND}Nj=qaKEEFutRIMAP_< zKmHq-v|2QZNvghB6M4SwnOH4h+qy9vsx@_j6aeB0Qo?BvNdEc9xaCHB-xJFr&-cAd zESCaO3H8&$Xb0Hfw~-{{jg5W+<2@(F`@}aM2}!g?2t?UOm~0N(B8bMq3?^ot_wvF}#E&-o^ZXBL37_U=u-K z*!9#y@eK<|1pO}KKgOTiQpBIF!e|-FFoNEuBk1z~vk3ZKjDIl=@=pO8_3^5qw6mqb zdp!R^4bDiRfke>n4MiCaq-W>|8XB|}LBHqk|Mp9x#)2d0_s)s&W&};%0TJ|j-2cG{ z`i#UHW4;JA_MU(L+cq6Rmy?knLH9B{FSA+D_lp$*_6igb4VDW@X@sBsW21^90?4F?7xN zhGFZ5PO6@*T6v*PISCRoijX-SqL`(4!w`S$7Zc@2$R|=%7(Y*;_sD@+`2Q2tcb7|< z(?srdF}Z$Fd_R5A5v%}o{XM4Zr&j?6r4tnY(O>^H70d{wHrh)mea$I?;$CK`O^-h! zg4pR%Gt7npf?|M=@3pXk;`h&U)a6sl&hqd$Z~fFV1jVzA%iT9PB~;7~P~QK_k;W1B z(CDWaVfWCw2#P-qmr)NxI2hl_2>YiBhma%eGVet>tAi2t#lj5QHAaT?^zld7a?LzF z8heCoXc}9MUfT8vU}ydIfBgKvX&n{l_dFh;}pi#&C_aJ&(OM{R06AjKup@9SeANl?N932EegSLWz zkNo_nUmBJV4gx;95b_ydoWm4zck&Jh0zP8=%SLrhVvS7!5e)s&uY}dvx-EEQZlBqz4*&W8!jEKpA>LsA=TSCuwFtgpn+I2}XBlhoOe7N;G*mY) zZmJylE!#1jr`$%seqSO30y|#7y%q!KM+O`DSY7;zyG+{WK+Es?SX~asFvDLCjY(R8 zZy1A|L`e(`Ce!KVz$pCx#_#?#Uy77q`!DxGEM395x|MdsFXqMHr{GO!Zwa(>^?fc| z;RGJ=jx<-D;S!I>&As8H8Y`=st6JKeb`k$NVJyr)_V<6qnLp^&nw#Fh{0YN+WOY?z z^QdooHYKAbgW6~A=YQkOf8k9Cc{@=I%y(+$&5K7iRyU7odpINV^$E;BNixer9p!M)s+?1)h%rN2KkRwtiasEJS~E}Up_)W~k}#i) zPgLk9^#3mNN5FKK0lrJ~t{zo6wyJs5x7`D8Qo!UZy#LDe5103&5py9SMC`*OYHZ$% zl6XJX-dvFg-urNqK6%Tq_18zhn@;fDL?EczK@=@%#y zvGix*G80RakVIm?Pd7V+wBJ!tjI$j{2m2kkkM_f*STXydic@a+hJ^Su#IoOKog7TG zUx|mO7ak<`+q#eTi(@LGinjee+wO>@{SJ@*9rk-s?Du!yek7#*PQx2@{n?`qA?9AS8PbT)mc44$XOwc($&!vC>%aM7i&GLWu{h#R#wA_AtwxE7)88$C(%fxaC zXiBJ-T7b8-xxf4Ke~r5{BqY(Pc*ACY|1`;F6QdEOI+JdZibgWA z8J_G4n@s>c4VD{}SjKPh)ePd0P~~&C{nKLYap#)pt9V??8JSqj#!^C!v=lMd=l|nq zO>=qCf52QXi@E+0E;C(}ge0P?gSq|@aR}MOj)|5jb4@6c1ZA0+ODZ+Kijfy&F{0X1 z#+N{0)f0}t{u6Iw9XvcK^C8 zgf!Y2cmw4Zvy+TADcTnS9Ow4TEqR1QO;y zdW?@7t|OnAs|{DnC+21_ETn)`LjANbX>nir>tAu>MWbKBc(01_zVwYpLK1B?NWb*= z{~AKJxXICen5U5rcNq1{#CTY3@ds5r6L(gvv0lNq-~B`|iw&T!{QO_sbe)(W(5A!k z&smw6E)}2>s;8xi?Y_FgsVw%^@zJkfyVt~aU-`BpA&Iu&jfVHtOAaAzHznF1m-Lbj zi;n#>v7M>q$y3Kq(ocCcrYrn*(-BL9!SnS(2M--wXI#N(uFXQLJ3AB8rNC1{!*U&C^*i{-xdEk{CH?rekfpM?%mTCOfS0B1BwcY=xz$i#A8R42B&nwhie zkFG_>jn>x8GUfx9Z;wU@sYPJj=)j@@yWkE%6OCZc=Q)`eFbGx&71a`@{r#(}BbDv1 zHu_Ik@eQ%!KYc5bkVM-J+<(QIM&4KllLc&*B8 zKu>|vLsMuHzlIiN~9YHG&eb36;hlob~;knztz(+i5UmSW8=$H>ZPv=8f$h?>j z<&2pc`17Chbk-gkw2`$h4*TPuUmEHLX6=i^t>8w3@(+pwCz>6E4Ov0)-yaFYe zusV;UZLa>={K%DUzzjS7zx#AD%EwPM_qy+VF;#_sF%=k_R}MbR5w!RrG*5pAID7Gj z*$qHr16>aGKTO1*dMilYtNgWJXm|lWP#|;$K3QnGSi?kMbS}%im(5{x93u2y{$F)2 z(<8`|CtXcdT^>OKJfa?!tEq}$&{XHZXo__1no51GmASR5B6j~&;w@IZ_R9Q9IqoG` z$+uS}T#JPsOby^A#9Z_R-m^5f{?jM~hsX)s+&6}y}^)pxD9%h1hnv9Jf zM0IN3OYn&*&vWyCJfZfihnsmcf1FUOhaU{6m5jU|#ZsiGmFJFs^m@Y?8|z^VYMo=$ zN^f+cd-v6P(x{c+(=i*5COuUxd&h5A)dkvUqbkeCVpM8#u|OA04@Q8QuqZZ7b4hPZ zM)6uR-+(5H>P!`tKCxB8!f+VRlf|KIJDl>1#@WD@MdfX1ydaMJfXIi z17aWgFU(drAO>idc0&-UlqgCS(EoG!LVKJLPpJLGYf$w=DHTdJWYP{SrB)lI3i~?j zdH2$lBjE4IeYP55n5Sp4G&>m>Q?{sAc#9*y2EOA*-h5TxMbw+02?88n19r{26FyOA zMcn_-G3o5YF-g3B5%iCFeX{}Z54FvI0kt&-7aavI9D^@NF$Qzorlb>4hEc=?dhNdf zHpCC?%>T<#L=pU7OapU}edmg?wXO!mMWVrlDKwBMqIej}u%%t7qX=lwRuqB5I&NvG z8yrOx`{zF{)KLU^2SgFYT>oW_U6@#77k~(cE`CDuRlR)h#;QjTAW@Bsh?Okq4v0l9(^;p+7dqaf9GmTwu@B3O9&WY zDyUmAW^r{zWmTKUnZSj{X77W+_sh3i3@+;khpzNqm(5rhN?$iVCLP8mLO^9{-V23) zON@UrzNx|Gpsm;`!2liCV_{E5lEJA*I)fdCm@eLvU_(hK;7x7f?SBQnp|oX`#&(Kf zloYQBT~tQfMrKmRu^modo*CY$tCOfclR#*pbXNCVXH7H{32>mTg<7bNz5>0j+% zvHF+s)&swPJ3X-Ft=s&0{*-dX&r8@jEe$$w{}VO1B!vdztvmb%W$3Lh(cT&wwB@Zk(*N6~p>D9Z z?l>k^dKLT~c?WpwjtwZu8oMO1#>{r@;*Q&d)y3LduO=gUYigWwyS5QF3qR}a+85&! z_0@5=%|$Q$DBlw8T5a%SB7$=>2~t_|HZK*jVDtU`+pt#e+~>|_Sj@W zFojOT8+FjxkAHPl>is4=ku)+}n9=)RGQB^&IxsK|@6E#hCov;(9>iSdjObvcda`Tc zktlb!lR^^5MNL@OEf;x}j+% z{CytY6!#yo=MFuuBs|^qXio22(=D2twwpu_R3-LjnTRPjqDuYtzeBf4iVCxrQ?;&f zvZ8qQ3_p<8r@uW0m17#|bNrajqAv)f{~&?BD5T|6Rg1nuOo0Mde%FH%=nFz)Q{a!& z7a973Mts85UE}({hHr>Jw|59|kt`_2v!_aMV52?0?1)@hXFPGC6yAP+JfWu>8f6dF zQ}x0H&j+eLuyC2`tNMXOK_^w9pPGc{oQ*B7sP+>g6^lm?YaBHa4=~^h_lu2e7+cP# zTTPo(JGZ{BW=bva6m(OCnG?^NJ53XJS4HU)^XYxj$=HuXvHK%IM%R|kc`drc#D~G* zcq&d|r^_JmtReb2j~bSw&%r+Esi}LY{W7QaA2ElM$q2ZBeiS0f@`|x-R~gO>jh{Ms z_LRC`{%853$}kgzuo_GCafbAnWNPu1BF zj%jHsuW0yIeG>qesV?c0BZQtgYsSR7qv|J5#KN;_Z+%p0`W&!6Y;%}XGkLad?h)11 zFlk=gQeLsRZ7r~=22n!|x2n)|{nh^I)57L4)Adycq)*7ClB0Y5B%^;n)lG9r=YA+2 z8B2JmAMP3Lo<2SLYvKb4acyLe^clc*U@(jOqjYE6U-itGIM&xnRWC!F#sh8Pg7uS@ zxYnp8cTv65CyqUzRFk$>2c}PoisO+$cdLYE(@FJ7pABmcY_P>=1}^8&^d0#qTlCp< zF68IvAoR10%42ctG&y- zE4?ec%e|{cBe`49M-A49_Hm3oICJ9o7&|C^;)F34*FdoHzQZ7kGyQ3E2pDC(8`Ecz zVwlAxAZa7;$zy%Ln;NF6_j#Nhkr8>quyvpso<6xd&JN9pFg959RF#HsU&q<<^vMIp z*^2Zz1dp?$43m8tXGf+_>yEQk=@Ygw&f-pQ?J(203O)5ei#wXrW{`TM#r4=}b4VU) zy=yY2j`g=vJ`_*O-nYRP4_r-~I&rkcBOueJZELv2vpCb{lXASpLr~LZkYd2avl!DR zPabhc8{*7I-0JiRQ;)bDZI_x6x4)nN@$(5Gb-Fi6soWg=K3RE_a!r0FjR|YSnXyI~ zN_UH=36Mt7G9@eApZkB94B3&G4B>(dX5an)B(ryunBwfatiRmhd4%epv(IUx5~iQ5 zg9gO|ra6+hDQ#l(zu1)aD`pz7Xa2w8GUl5MA&JeVwBiB&`j>&oHJ_DuilD;!q~4VF zYM}<1&YD05+DN*l`|1p6>`iIaV_UHNU)ARA7oy`de4;7_u5k>+O=&rJ*m>-#vmIfh zXBv!Y$1b#y&OdTL82P{I@aN4nSKxM=*v&OBWK!7(qS7kmL zJJ4CoD^g@J**wAMp#EagD`FY?w9eHRAGAOqUJ(HCpwn!KHxS~^?caQ!0P$9VcvT9B ztpf2`l;LX4ReH5%3zsAQPgZLNdzJ>BcR;9m;4hPyw6u&oxLGu~I)w(ZS~J-1|5xkP znl0qS1S4L_*Hab`=K2p;Yy8qsH+Z#X@cXgSNGo^W0-gb@H3xShXehYpm!Z~{LSRlRrKH@Jqo19pDBdtNchnxXxZtk;}C|#2E)P*l$&AaH5@Ri2nB}_Wx&WBN(mLR zV}oJK9Z8H0j){JQ*Z_Bs>e%3$aG8k>NJ!_q4R2U$*bf{+T5DEx2%h;!Iy~TLNG8@| z6gRzoLJiVvH8asMXfxDi8uw5V%_6*kXt=-rtK9&WGs2;@JXx-dHUYN)nNc=9qa`CM zp>|r93_-(x=vaWJn=~FR#rw4}-FOu)Go~XUiOl)U;^BK8LYi)NbSU;klMd%Mhh}0r zgxxTm&QBV$pNXns9J_^QBDemH?-OdTBvov=RRPe^W^` z+T`&e80|M=w2*H!5|Zc)yn(XZ-~Vk0X|y@fVM^(Jj5xnJEEA(~6=!Jw2cd)RzZNT0ia{&aJ z+bfw^Edfgjm9m>#rGNfCZn3)YIk4F8#9}$V#Yjkt$zV{q-9br<9Ty#;R2k_o7>vlo zVp!H^AH_7x1HZZN_q#T%73lJO50^{+{>YCc!Wj1;)nNM1P{IcRn;w(ZC)*tn6|8W0Rkw*7j%7+ppD%_`aw_-^nB)m{Dqa8mqds(ld8NVK__(! z@zFWhorb$U8OEfgQL0pM&C6gzL9!F3YHgCRO$UVd^lpWlXm+PBjJ-Hvy{hI#kt1`Y zDz<-yv)3esvp$$_1WeqmoXn@!jAGOCl=oUD9mH*VC2Ydhd9~tE-2dx(l&Rx$(La7K z{Ug`!A0)Kg@u=@RgzO)4qnKj!B^}&xc_#hC_`YMO&X}YZhD|3aXu}=Hk4ByWaSeE* zpNvi(|4-qDN#7m6o{8O3Kq{erwmTl}`hSjP)5hn)c<+eu@_gfwkjCSLmq_1DGT!mg z3ZC4-?>HY>k%{qQE^x*yW4OYAV3W*ahjEESdfJ%e@&8os4OKsq1aQc2WMa1jG$m9^ z3!?f^eN6KBKYe^YY_?Zymhan)ge2Nx2&*{M} z8l!1%~1wWT-alw*5xc~j-E#Ura*hE+8-VNP3K z-ca69RZ-bkUY^11rx48X>b@ld8asv8QqeLBJ0jb>r4Ib5qf8FRRXKX{^67WIQ{vZ3 zof6hi(qYU7(S(jHJ%sn)AO|8@s~v9E3KiQC<@Af`nk=3xJ>&~3Yh_M^?Dw;0 zMnaU8wK}vX&@mReHaKf_=n-O}YjxHN8nl(QI&?)sX>^R>tkt2{#Y(S2`zCLiAxs%~ zS(_mbeHtY>%wL;0%wK?lF!Z5*{$CICU7+kvl5xt}Vn&6Or z?mawMy#8=D8*FDjRs38UjE#Fig=yEqlp0#)5vMx*d2aMMZ;@Q$k8`7oFsyOKdHUpv zQRO&iSqVpjgLX1Hq?1>Si={|kt-m{B>D2~#KT`F27d6!SQKnwaSL;cm7LL~1aW2n| zK0Ou+(`vV?v4}O{lSemK<3i@DHg7Y~1=Dk`0cJwdcbevsp0Q9|W{rSqpBt?iXsZ;W z6Z$dl13*3 zW6BowqQ7?3tAU&Ov9}<8)^|ZB2yku`*fr};_(YwJ9}D&zbFNApb2x;u0Q8Ufb#oHw z?oYOJif@gv<4*?{jzBlM7L385pi&u)ZtqGSUl!m&&QNn~> zvC@nZZURq^C@G?Z34caO*4Pb+H6|yKYD_~iM6aYZy)AFshd8Dl(?X!mpk@tO^Yy> z3e^9`G;YQ>Tw~hYLDhG_-+1U6Hk-F0o?bj1w+d&L17B%mZonr@bmWOKxhwDu<(@{l zthM7gXyw3QhAzt>2w<9k@co~ur>zKPo?voRU$lpD@D<}A(yJ!}+3=Rk!%_2MN^}54T{9L5-%T zZ33DSs%2*$j-KNP;}m-4_(Dvf-=pp&YJ6e1jMZ!i2je@LLLYsxL&!LH4e}%AsbQEx zzfq`Brqk{+P=_5HukJTxKx0p#o6DQ<{H9Uga^TpWLQnM9KTX?XX!!>`XRv-4?SrrCm@cA6bM4#b*5AG62-!d);o zdWdoTJE#_A*2%iCF=$0)7(Qkv_|vIhpXk&%M!^!tApaK|#LUwr#yNm&1gK@7bDajHBtXI=utI|Sk#DIne_ z5MM+Yj3P1ua_JuyNdo)oWW%Y*^ns;>3ovBU)B1 zZ)#q))bO^YL#(iF%)^{LvW>z1xcdFQZh4@H(^M%Lk%_d+DaT2r-w{yh zQ#M(ZuFeIeI=$q+jrg$I<&^J9rQe&P(yLIGke?99_p0UaVLU&bgM)<(SqfArZmlo_ z5Z`ph&BV~wvHi0b-5Y?mZjgmG^fI`s`p?d+$A^Wsj`>gaqWe>z-Hx(ATZj8Ew1Mx} zy~qRFSTAZ-1toF3p_G3ryGfOmu}VT4_5auE@}Ws>P(rV49liU}B2{ruq<#)CY$wQvwNTp|mr(NZZAkFI&3? zn^UPOu-Y#40CJL(R8|M3UKk^1Rnj=7Dr-@dCH}+ULDw)KuI&5Pp92QKgz}7d8YYhw zZcGx&%M(G;LZ|){!dU0`2h{nro>rZ=>j!EzYuO_YS>u%yMhpyYEloMsMl%! z`iGVaTf-IfmNWwhm?+!i7ICzCyn<5yWa#~le=ReKM!^dd6!EgFd8xGFjhSanX za<^qqHNt8qfFtZs3lOO}h*H!2^-qnekPa%wrBvUi6ho!k2+2u0K|tK&fS^)`VVb3L z1w0(_W98<;aIZI}f&*)WH>Fe5m^UAnG$ZEM5w){`67x2{?_yk*te6%A-! zuwDiDk*CksfBu_kt@qflhN+RZ`fc+iUh))mXTE3SEqvGJ+lIQKL12MenN?D14}m8&y-;Pd3G%RmkyMCSN z62%c!smjxVJaxl*3$w2>YEzp`tUjCLKLZKV$fl=hb(l?CefCv0IXZVp-R8}cM%q<* z^Q=~Cd>?C_#6E*xISkmy0~y4dIf=z5%<=a>83h)BL993SN^Iuh>~cZQ7g>nUoa4v; zhRqt=4m}fFwWik%fZC9QSaZ&8jy7dBmp0n4T+3ZN=Y5+zKLYXros=1XT2-fZPQ%Te z_4WYxBzW^~)R9 zt)I}ed{*m<<_J;>#T8X}zGKxG-24U3RTljy79p2>F_gXgDv&t_ZDRj8&-`nxrAgNy_?!wdX}a&N%v^fH-;{=KpLq<{ZvV z_)E3>?B09u(G~AfyvgMpaA6$%Y>GH~l9c(e0DV@^^gQUCtEUhnz`fW{X`_R1$11v(l*xH_)<+? zct6c&8EYZef7psXNCEdEVf28&{UDY`C}8Vm4X#s#RvN+Tz9HQ;Py#PIji7yv8Mn(K zY%Yljh20DA{kJ6)h-@6*P_urvgN|(+c{r0r>Vu1y;y7G%5X}ZeF5;W&{pX(;J=x4- zoJv)pMGlRGoTLi`h`;~)gP6xC)FNuZB_N@&yCv>HT98mEOqo#Fy&x%}u!#G=S^FOZ z)c&I7R_zyl0*LCqj#oc`53BYU?U364I7RKtOmWeDV&9Kh`+BB$Xdl^UcNp$XiQBib z?7Peq8~puWMouSK|JW*3Kb)!FY%JEG`7VhHeWqcOh?_&1h4?C#jJn25MwPmunkfxp}~sJ z;$!3H3)b@g?D^eiif5_q|3@>$5wS?=u$pzP4fv;yRSuiDvZ=KpJJU!6uiBncC5?Xn zH#VgwNvBHuSzx2z|24UuM*fg@XX_S>Y2MHSRd$>*aw2Akxl;oH^XL)ZHJzTO+FXmjXR_^)-8ME#{v`Fu{qyVd&yv(nfF}MTHshZ$PUK`WM zC4(IaHMaeRI}I7;50)Sm!;nWbU$V_%e2>kzt$H16DrU(phkP2yy)v^g6cD$px3kxb3fVuw)!7|kCe54-BA#<+~)T@149F1FOa|H3r0T{f3eH72Q*lAMMZ zhc}wpQh)v5kW-tQzOww^2iJ-@^@wx7mgfx#7hj#Xlhj#It?DaTtf#7WIZ>UE?lll* ztJCWEzapv=<5H@slhh$O)#-#c)M;G^5DbYv4hV@_Pq#uMbrY!6kX?7*iVrI!YW+UA zFd+UsMMxxb!`A0S-Opo@UXhOUM#dvOdv16&=Z1&@Ii28(1G1hpw(uX$+;CYxQR4Fe zC6-OFmH3=WeBA9p)~Kb#vQ|;zixf)8V6n_!|N25T%iJ)}kDGh(!cyN$#;13fJ({4| znj0?j^B;x{TXKv|X9UeLHmoCUBx`PXLMJh#A@)FAOxv*C@MX;nPdHQz^F@GRPFP?W zMhjBwtB#uvE|y_VVEDnEMqOdTi_O@fP0ZJdR^ca#pZ@3MN)(7RFO3|w46XF z?pai3bHm@1Wi&TjA^DHK2HkYDulyoauXjiH!M(b19M+#ehn5N@?788JhaElH%wwEN zRgvYJpplT%H_6;^1=oLBt3@%7v7Z)En_Cn(H{7e?X-BaabHk$4bHm>x%?(#BfiTv7 zQ9$jl#P}bpeRUIHs;)O&&1PcN{>q<#3%W|*4k>D19O24WMXe4owc2a7M#j~$9pRUB zgix4$hp(6;+{=GBj&K#@KPu57K#5hzf5es0jTg94xQ1EH^(3-M_d+jYrxFOgi)7%u85zMYZm3`drI^?uBx;yF( zMe0-8Bw5apx6d;54|4E!?@;#l60>Bbc}$ zxLyk`#UmQy1C$XAP&V%M(;af+&TogUbOv`j3wK_YaX;?}CT<8e&j2k|?JV3zFhJS3 zH?aTHv&7qXL5WQ6>#%h@t~UP6W;@v+^FI+x+z@P@;#aClEZjygKsmT&{%7GX+)tJf z>(qW0?t<*%K5?95Fye+_b3;_A+Rwsm1Ot?f`$Sp)ws04h{RqZX@Mle9xxHncfmk>-F}iQq{r2Z3F|9 zgZo2=oVeRzh}CD)>r_VzcX4)cpS0K!OxzG$uLYN?juvhsnB%_XzfZc^At&w<%o_9| zwK~OJ0OxzG$uLYOl z97x;&8^HkO;NI*Mdvc{uXW{7@!>7pE~5k-T43{A`I>WEZm*4i+l4jM=)_iu-S4|st&Ml z8^Hi&j)-p2(H(HOI0@uw-F3bHty3J9CG3= z?XHx5dTX8PZsG2dW!%4Z1QRy|o5#kMs_qtUBN(7;+^2VU$ceja4|q^x?;aNJ((K|s z{ZdCTaYJyu7F?=&Sh$T~fO2rZ>W~xn{>WMCgI{&3r-i#~c5!bx#t}^15NuAIx!d84 zU?x&6f3oFVhn%<%=%v*D#@@Xw-1}!2_tvl@n7AR>+!yVhH!^|&%EpbO+4itS+}(OB zrJpEWr+Qns56Cj^haADg4Z-Ff?NZg-Y9~f8KsmU-cF2jlJ9gdb*Mdt`#Im;$3{W=i?PU%*arY|2 zGKj%lX5j|cY{s$MuXO|yHw2qoN=j9kh1&=QC`mIy_EZn`bi~Fo%M=)_iaD8u9bgAlR z;WmN+%Eo=xM2DQX`}CK4cI#As3-^Ip#{GmNn7ARhUSIHAs`^{FjbMPXai1M>$cZ~L zK(4y2Qv)pAeX@)D?2{eA#0|mq`i|XFHNe7c1Ot?V`xg#5ahDBLDq{NOKnr&yySUG( zbOaMO1e>dHO4UFMw-F3bHtuuQION3LcaTzL2KOKfcUhKkf8+=zZV0Z|f=ksP3%3yr zP&V#!2Rr1%-R~f|gR@Q@WZ~|cUEJs1;s_>g2(Ir-gO#d-EZjygKsmVIb;yak|6sZP zvQ7=QaQDkD?(>d!1QRy|*Y`8H2V1y}V1Tl5pLdNzPTT_ymMbXh)WH_+{@KNSerHE8 zaYJyuzWcIN9c21skQ4Vo!;~6ia1XO^ z56Uj?3&%Nvi5r6Jwct`U%xWh_FhJS3FWlsi6ZhcZN*!cy54Uh1lx5tXIf98Bg6p;5 zQe~eUGlBui#(hzRLr&ZWBg`FaaF4KX56&*`i|%v;6E_6cYr*chX(O1iH{Z;+VeR&! z4?)h=<-RG_cR9yEnTz)m5q)`2RkyvoMJiuK@MG*@-7MJyZMTb$3Xa`8H5E@)v8;M<1O(r4b(KoYW9#U;HFR?x19g6R?UW1sQMvnhNY^)>U~-;LD{YN z5;^}ETYU=ai$qeaW)ICS?j3Ufg@t=qrK%rla93Kmwcv1}i+hK^|G{6S3O=EdaJb0DeJS_<#G4u0Nx*9M@GRrr>j)<95vh-GXu;uP7x!g-9CG3g zSj`@hUEG&l?Fc6B@=>aOglQ+EtahRWhubCKe#;>z?ts3)$EGw;=ZELF&OP#>F<7a_bqF|;r(3PR~+S#)7}BA*_Bzw zz0(m)+#~&M&h7?fEjZjh0rxi!IdKQ9W{=D+?ki7n1QU1FSh>-?RE@RTi5490;Nreg z&i`5MBw#hWD!aIGg{kexhOl@UaR;nskIpXct8a1y6Ze=yQD=}798%JfcqVX zoVWv4v&Uo?_cg~kf{DBOFu5YXR2^pF)`G)bT-?`O<&YD1z-o4Nc5z?Z(Gg7CV-J^y zR+g&6E!4=NVL;>oYublp7x@i$If9A&aBsY-KTMF1w~%Ym;qD2@|LQOx@_<$C!?TP0drKX` zM1F)fLGIE=JD*@7*P_FmWn1n1dv`bth&*6j`-to!zp<|)n8-b^R@EONHm|jiYtiAJ zF7g|3|A#g75_!PNwwER3l>h+Wa--aTXT{<*-jTS4*AwVRTF|xZa4%4jTAa&G2PQ!e zSlq73F7%t+@sCE2dPe~|QaIW#9%Vt-vctXcgAF}#{VyD}z8%dj^qZ5{|HgYq1A0`T zA8kR`vcpU&S9q!{oza@G6?-@zC z$~_^=(38jiTJIP@pCHhWv7l?&__t&ATix|fc2lf3r&{Q)%`Ws?&v2@nZT(1Z5}?-# z^hp+UEjtWcc7wU~X@>)$2ds4;nO*4LKim;a=tp^z0sTmUKG}k~KF9`Vaj4Z#)+lu;P7mcA@`3?*Fj-`b4h|(2o}A zbry6jJKR43I-dWOFeC*mdQZ$Q^xNF|FB{A;-c&%JDA1=`(6#LF02lgg_as3NSoc0A zyU>4lup^kzCwbEV{TP8h&4R9FhX=aQe~9~U609Du^gSud(B=L^tHDh6rUUvUfj-@W zu4RV@xzKM_Wd|o+FqIrVh_Ws`sV{^qCfPEjv8eg?@+Je=PMz=NPaCUYA|yckZ7AeX2JL(CY;H zEDO4p9X{BFey5xNAoPG`@Tpmb{%R8RY2Iu=pDNI2ThO)a@DLaJkGeP<2t8mWd|Gy) z|L6ioFdNKtZw{bO6X5^=D!JjhIcHWPZ#LNTF|xZ z@GuwpkIzkl9Q#$ykGFD_+7mJgQ??wvMQd#Rq=8#Nwq3| zm%sm|Uu;zz4@^KqG%wETJK>G#>0QkKa8zgw-0+Cn_cMlB>TVPJUnd9J651Cuo{e(odaC9)$d94e|Vx^amw@F$7L7#&yxM$ zTyG(uA1BZkTF|xZ9N@B{|18n}{5etdSJ-q2K4`zX^Szho^OV3j}(j1zpR|>5u^Z&yMnh9_UI=3P&)ZFZPxI zdV@e;VnNrkbGUwPS^dGp{ZBbT>*tHJ3;h?#`=1)UrGUOzpf9zcYuPyr#VzQ+xYSXe zRu5Q0Z_FD0q@ z;KJ$Dz!cM|Q7A`7YX$2-eTmd+V9if;n_ixCn_e!6$-sxp{ZE8DFbM9iY`B+!Qq6w# ztq4ABxc?EjgHymg9AyFbkurfhm~gvszxNOgcOdZP-#85_HCnNCN@7KCl$o=@=kb5BS zgg9GI7_U~vC2_ni^g@vH71~!VA^@Ls6^H7sI4G$r-ld=X;q=9`bYCPZ>5Gs5OMq7d0N;u6 zKL$KZ@1qXC;*x*h!vemuMu1nP04{xTC(l1HgH)yaVg&UOONL~(FEXhJOYM9!Ca@U| zfPFE?f5?dQbP#=OBJ&?4Rf@!@IQ^JV-4jtl7aJo*{q;{%lr6rKc2G+bZK$97SD1z=0s zW_iM2|I)hZfnb=Q(emX;%X(rqEnW=UnJ5GPtlp!0ip7fKNIPP(+lY5 zy+T*~mT((yz7s#>p#T|(9iIGBkQ|6=(mne(WE7Qc=j)% z*3q%9I6`;DK}lWlRed|W(-)sej31smM1UV10Q|Z67H|_ke0u#m=eoe3+bX~(r2sB{ z@wp!e@JX@wfti23B9Yy`Ci*;lOi;dwv*KS_r> zbQ}lxj)MXj6^U7%A7;zZ@xv0|ML8Gir!vd)%N*jpkW=ngqAy2Ar{}LgQFe(*0WItK z`>mE`;)mXE-f;}rSfkVPzX2C^iK)6}rHmh*_w)Z#WAQ_Qxmdp-RG1=u*fT{*X899pKw;)iY_?O!_P!M`wW@L$k~L}HgcKN0EEgQRo)i_&MO z=u8~`F|zOZOr+0_#Sew4;)en!e&F&Z83>XW#t4$xiAY!hkn#f7KPTc(GwIRs!-L;{ z?M{4S)zAx@1@3Vv;7a`P!p#ErxR}$b$SDG!fmqpdk-W z@m~zLi654f^k*GdxL+J4xaXz7Ek5hTxq^FMEPf~i>`(}-j#Ncf{D5&S8}Y-7&pQI? zvjQFm0D(fhx|ayvt5t~=Kjg=QhEV*^ebz#Q*{J`&Ha>)YP?mZ8 z@X`y83iPHW{!Pk#REZTo{6FTt15Sz}`@ee=EW2buK*BD9iUc>YK}CCeSrr3uG3&DI zE{YN)DvAhV&f(Nk&wx2+J;R-XD1s=#fEjc4?u_So`qTgSTlH$XdU^(X)cbt?<7b$4 z=H+_z>eZ|2s;~+$1~j$vhxI1@>&9b1O;8G-$*gTjbSr;YKQSSQ$sc+t7WFxWY4e9~ z0_DGfBNwL5A3llX59>MqW3W6VjZ3e8-*RbPx%~OjC*Om}mP>D7{gaVnMhcf64p{W* z4MRk&8Hvb|pDJ>+4Mh%4O-Mi@$A(!V?u<0zZb1Ee6yR0l$TPJ7E)!_vW7p@i+M8&`%X(OF4J4s5p%;)X_oPG@Ct6=r(DjQ@d|dUwDt z3{)389a5XgDyOG2)326KQ=4&E@vobal1t6;h2+hty`&Nk5dd&G=%w z5G9(Cl+?lX;d8;Y8HsN7`4>ke&`qE3qn;A4&uO2o&#x^@^!c@A==1GU_W89#f&Dqvd$9XnWDj}5pEi|F{$6%iea6tgi~U{(__ ztAUP?Dy5E92_35vI!b(cxii=@oX$!UpI$Dr;*;|utm7PX;dC^E9iLu408|*C7Nm$z z`v4Y=`0@#&;DUrTi&Z0*2aRYgudLk_)(u(>a+`J<4jn;~WDV&}5z_sY?aBk&MMx)% zWs$B4NQ-?om5O~9q_NMY@s@qu>miiW>w@PV!-o~WHytbXIU$98q`122N>SnjH8Ek= z?r0)pZ$!w%tfo!kFEu%0c(f)*o8AkNVZSCB6&}0bo`M828%4#|p&-Cn58CLJej#Lf z{I;O4W}bks=W?&?9Rj7eolGR^jYyQ>6Nsy(NsJ?wxUXCoB13UWMunj+xSk+EMu|&g zTU_S<9P+fuDDD7&s90{qm3DmP2Y}MgPe{{_ulBOqQKt=mePzT5kg?kF)!jjbetvR_ zcI2oJ8@`J54<*#eiL{|zsNstKKi%NZbmL)h5&bJU(Or8KzRw_Sc$<`#&4PDg9J3rh#qP{JPXUi)u|5*>|{@;L5??~~Bu9nm@7-Gw(=_4S$%I$do$ zOtF=pUhH&AZKnTS+Ayv7*Ud=DC2e^9`;hcBV;6HNML(xgYBTDzfpr9{)c6M1zu4up zyU>iJ)JB`pzTQHck?1s|45@FJ=ig3E3}?INTT#4HXkYWIX8`cHlJzPN2d*?7w6Xc6_G{-Cd%8swEQf<5irQ$-KNR8D-wOnU2zvJ z`UihWur*!rSk)D`Rb5ec#DT^9AL)RW3Lrr;^+M|71Sx3HhP~!Ma}4B;PDS7%Wlncz zt2wQ+r#4NSdVJH|sWWE`n=*6u^rpFN59&pN(wHCaY7=eyu1z%Bm_)Xm=EDUxa+_;` zJOUu%YQV+ zCegOoCh@DHesqqFoHr?zsABq$SpQ7~vb=He_-qeh(T_g*)<#xC!)?LP(2hfciV@U} zD1!F!;1C*{VT))|Lil*Hjiv^ay+)~D-K!g{o zee&m!DLO%*p#TYuR1%#&G4*eyxN6SZ`NZQ;#ziCwhXheoN!J%&TZzJ_rv4oy3U{dI z-%O%#xCB<>S&71@J0~PG(P=?GVfyFvwG)ND=@SL%cb}e;palE0p<|8TNlv!z;p`Ly zr)``ZCu?d8Ls1_vvijYp&)6i|vhfonx;Q%}S%AoFe6jUsIW}?|hKSt&qH;$-DXrzA;MBHOZ> z`iE+4CF|}0L96Q*RWf|c`i}z5-3YaIG#65Jakfd~(B+Gi>C%#Xv3tNXQ5Ium$|*i~B>Mlz1O=VL%^e zh%GS+Yt1v~&6qoT0p;xle9|!T{%F%t@+RnXafT$xOJqyle^lGZO5SpSr1HamnEIz~ zjE#_Y>@c|)tqTs^Po!O?#tq14Al^{+AJ-=M9079>c^qPcx+SI^I(Bi9K_bF#g7k6% ziJ_?oNZaMmMfh)l&YRRk#_3$3Gp9fXl?ZA|5?}whZGu{>WPj=6tI~iy){3wH48}ib zppwTe$Jbr768{Y5ejuj1>rD!wMNKLp!I_E!oLBvUV>KD0GZ+229t@bVp<^1x$C zx0L-+FA@~@`BxDCX_j|kFeB-p)EN8$iuNC0KDm{Ju6{xWEv^+)Wi9sHmzUd}HE$5U^HQGW{cYRqlRR+-ZsJK_|kcF#XgVW z!!p4)oc~a!r&GuzUF=(v|2*v=y5+Kq4K}x8c5!+K(TyIVj^QBUFuWnfw`Tv7MrJm; zo7pGY)oF{f@e}*y4K^{wCBD5F;xRUAY!4vZc@G!XFM^5rzkOWn@pKw{e2e&(SQT`p zlfZ2wSKW&b%O2k`|D`=vDSNOus$Xow8oZ&>cRK*aA&lE*u~zbJ^NCG;d9ItQ2sqBk zPPW$$B<&Z8NDF;8%aX`_iBQjU96vw+M6d5I6}?s^>m_dW9p}Fkc~goeYz8rkZ1Uet zY(n=!oWXM>kAHP($UPlzj~r(^XFzp{ehn=tX(<~zxx9MH(2C}kCj4(nQ%O@rX<2hg ziPO<(KYC)_zV#C)G}cb28#M~#-97M6TTc7PZa>ZP-=iTGn@+P_hK6t88F__dS?YLW zaSE6-ptQNPvbtn=^U#vgipfK(%1WjTEpKV9B1;I-T+>qCQf-Knu;}w|_WywRz0l_f zo|CD{)X-<2?AZF?9`p(0_AMP=QdUw^Ryw@0ytJaU#IPhgamDZZhFC5}94w&9DO~aU zQ2=Gv+?3p8((vCOWi@=xRiNS=eA|B}p2(T=fsIjgq3La{rjTCYdnz&@VLz#I9d4Q%Zg_I;rb7Ccr956al8Kv6Y*Y+ zis$+u2J!wgUf8|L!n+bLpEkqW<%0(KMQoG-F=#wAOiZhRvBfb9U>n*)5Z&&l%R#(saz+*4e{G zw$7fs0KTxtycx}`_7y<|cMtQg>l8VMx+~G%xlMDLr%t84)Qb*){b49Cv>7$Jp?=T$ z@%W-0f8sAPdby5U2bj)IPTgL$1W+%HssY_1EVYMRv8tr9rlJJD(wgGXK3uXi6tn4XvtaEiq2k0xx5r$!`NB zl9lND<5nQeZt!|?Hy8^bSm(##3G2|s$$NiwD5b~7Eq6Q~N*UY9l`S2Iw5JGt>d#20 zyAUJYk=}l?nzy&(?U$WFM@Z>l(t?_!sQpucsQqfP+7iKj8e&B-_d&?&_=B%K4Ifqn z`)M4gFoL}q4G|99m-hQo^6J)X`FJI>WXR%c$KAATuM1Mk+?NB{okG;;%u?0 zJP29UD={|h`@6=bGBq}lB_o22P5<|3h$Y9SLV&2z9Z(8PTK4}q|D{8}nWklb&a+xJ zmoqx&tHUM~;lpa#p9g~qTXu7bUO5o3XxX1l{eQFSl@zMIs~$Bdbn_GQ^jrWFVLL%G z`EP*KK|fX19*JKFix@Jj{cQB#oEX+R1oamM)m`X+AeP$f%z}X^R7d>67a_z*zi|Fb zBW+GLl0>CnjQ;ONJ=>iNLeT#gqyM{!!LOY(ihGcgugAX9a9CpglVa?PdUgh#{tp}d z&Q+sd8!Uh|AK5yGCQ~ma4>88SoQC=X8vxj5Y;8kB{rJ%s4SPCm_OBhEG$8i#w(tv5 zf5iLIhh{X%4XFEL~uSl}W*S3_pzxp^>T7GyF0 zvGcr_+<7=4qfxSsOxVW^nFsMOWVPOj5tC;jb7~N)Ybawn4oT|~vmnc?f9QV4_f@yl7lea3pjyU@-jz%-2|HY0`ay^hjtl-ULGZqZVlcfY=pEF=f8Ydr_{6E_1vW zby8F{xw^Eh0$ETZ^Nx;iIZjt^gHS$PQ_tBkM7`E&;~b5R$y;&wM4F3Js+ES$`lKG)}UPRV~epoJ5x#S^pDV#$n~J#5+?`An0gbUrw*9 z<{%0v0O_B(^nW(;{Pc~Sdk$dO$n&L3^(lm3Ah!L+net93kAT)%=I72JU^Yx z-&<7LeMu^p3>@52vv!y~1;PsI;fD zq^z{1tZd7z)ZQs*sB0Wq+feJ*PN;{S3w>%#HaiF_ng0u`dvPjDgZ~+JbUj681&;q2 z@<)S59BJ%4ng0sj1x{h!VWgc{TO!<{nn#v5RHjO@D zLLE_bX&Y{Ty4paLtGkoK2$ri6ZeqEb=)ypy+J)|J>5|_yBoo`^0RL0yvQl*URCFPp zrAz)kAw23b$p=hg&jQ;;PeuyzW&KCn%WcAVwaJ_BG-i_pCL{Azcq^$Xuc|34YuRFW z!(qFfgs>t0k}Zfk3&bz-a8xM7R>Ye%@s=qy!^_I-3QTkyu!!qvM?t&N5b?3tQdB|y zq|Fz>bj`G&-JuCktl&9cI=Rj&)MXr}IH%HSqZjas&;oez3t5Te|C1PrNaE7&#snb@ zY*}1;*#nnuDr1%4(oME}Oj~6$LXxbIE^tDmi&K)0XQ>2fu~fk!A<_(&HRNtvN?NSo zG#k{#@mS8=-f@9D9G8iTxm=qeVNMniq6((4+oHx3~6y|CQ)d(CfnlAmNx?K%Djqw+X3`6J#8r%~;m zg+?>t+uu+pWd<;4)S)6oo!QeIYOVSsKop#0jU6V1U}pjU50$TTidEK#l(DgH z%zk6XjIJNuFd??vw!pJOc|JbT0v#U;QLj}l z`>g1Bu;V=&e=vk+$Ij6y1tkz2JIVf6@oak8quz9x(?Q4SnOHl}QM{~^?Eetz^s>3M zgBrykgz_@!=bd=|YtYZ}qutLXDem++fS42?mnJFhTxcc5?k4hh^TWf7@If|`6nEzM z$3#BWlf5EN=ZPZD0tq{zMBWfOUI-npvTKf^`o-Ui@kTp$z8Wy>$H%J~I|tk0j!V$C zcrM4BPeWi`6f5YwK7lY3jyV>;Y!f6PR#B{=^A8EgSqd^w4b4EXied#_`T>eIJ3iTF zqD2?x|1`(K6k7BLF=)}HMOZDQ7HFAlY#YNkgp@V6vm?Vk_D$joQKQSH2@I&A+Ok&= zUTihGEKfi}4Mlqw(BZ2)-qQ44J^~cgSeUGav{=_RR?{o%I}^Wq?;%hifKchm{!c1t z9(^w8iv2Ia+)T{`hxgr&DWc^-U1x&|Ew(7dtjdT({prt(lmnrCRF))!4A5gXH#&hc zcEL4^1jxwzj>xr$b2YZ(aKxx4H)|BVy6|~ zFg@Q6XJM+<*;<~3iAYn?+R{{7R=(xh1Ex-O$y33eph!NnC3%_nr+G0&KhXathV0cY z05BE&C!z$Wg2b~@^sW9Gl94HT6JZihl`!@krg`9A++=;W$JtyeoA-G(S6qFtcusx7 zb3rC4WF!i8J3YjV3LWHs35A{&g}xMph-WF(?THW`6`Je=rajU4qCyAZ4Wo5q{0leG z4mci1O@nvC@i?&+!ah(c6if!$Yso~E>D~}xL}d>4{{>~%h%)~YWr$}f)1CE?KtL+f z>;p#a9ssDVO!tRFu=KV;c&cB`8+YU30C8K#D3h2C_5dSGnXQXMjHt{Z{y(71Gos8t zL>b~)%4~g52#?CN_<%_~;K7zT5^uE3)~AMGF=2C0f~%>nvmafpm(zm@(!TW#^&?qf z`>pBHiNaF`@U40NcVKX)OHW~lZR|4bs3 z^ote{NqU6+pXt(v-tkW3b|T5~I$fGvS&8tm<;6LsOV0^qCM&RS583}Ip^53ziuU)K>!%?H%X>ZluB~)bO=rsvc zINB{qF>;7Fy}15Iah9mjt|;Yb*XvfBTQRsef9+m!wQw^sGykW|OVY{wFN;dMA4%nh z5j$@$R7B=&xc*O>m!^>UK)}M>+f)OFU1zD&if87=BuP zz^Ir4P~{H$;}g2{&I!R%m%)9U#vbIbi_<5*E;#y@x=2~A1JKDqH0ga*hzm72%zq!6 z{9QD8Uo;_}rAcqu|16fKCR2RE5RET(fXR4M6Ept{T4hLI>F0YoedB9_O%v2)-`WWc zjeFIjitE?c>h__hk(+4M$BSrnxc?EfS}t0BBw7*A(yGsKAv|hzgb$c(TdPB~R();= z!LogZ;z8nG1oilE7J^bNUt^6OT1{wxLiM`#$+-mCh&p{u{aYK5jnwFW2z8zibv_hz zh-azO*VKPC9@RP02MnO~i;X0CNZ&aTb%qr=joT1(H>W7RIvBb|od#W$?kG%*GH-?$ zQJG2p2Tl<;~M&iK4 z3H9Su9Owc9RvhSeOGqZ_(&T>(T~>%LAB!%;vvqkVghySD@&OYX7Nlxdg!ylvw_A%I zb|)XCQ$C5R9Ui2EHYM?@p;^Nqh*h;Kni?Wrg1OqC31V!Dq}5api~bY>Wi|92?^wx) zAjh$p$dOwqgryo<^i_yBtD*0EM@Z=e)zBkSmgBI;AnwSIf+bm6)zD%y|5csCWi|9} zUQ?&0640kn=@Y953vtV?#Vc%mRFQo(`P+7taklsifO4E&%5fIQ_j10hUIZA8H$rIt z&LMit#6C)tqLHQOQuO4>EfrYOE8pVK#!PIDP?ln7|Cu4`EQUVo;Z%lgDK3hkhy7mb z8=h7%w7;o;>)6CH)nnc?r?C@xLZfN%&2OmFR+4vZTN~1ddDnbu9SRl=fQN0jogRXn zh3}Rk@~L1^!^oHCV_Et-^AKI?YMB8kS)GciX>=dFt z3FFln3dXn_ts;^PF!{f>JL~F~dmAOMfFv80B&DU5Q{YLJzv@Y=h4OrS!j=PM|EHt{ z%;~RSsaLIPAg5mwKlH}!*=&7EK?y|1fxCz7LErkDceK;kQGDy@OsuV`Th<sTd$ZEfk?1DXE^o%@||=VtZ+qYY&JgZc5&H2Lv%82_2cTYDLQ|QqDuRNBuWR=Be*>dOR%i_WO&>X-IDakg5OZhM9y&c_v}9ZLKAW3^Mf}4OCj&=x5ccq)}_zhs$%}-}H z?^%*EQV#9`AV!_j(?p%YHC7MS)sO9u`NqU9iIjt}|4&8AGg9S^cbo>vgoO$yox(lg@%tN|}%x9&Lc@p5@ z+(Rk>!>DtXM4dc$Ywph5o(?+_yYrMw#oC>xcMaCojhi@XJc^q#%jQ0B@{-Uq_VAWE zjXRLVE{?Y}zD{kNT8mJO1az_xRff(Av7st0{&!Giy{Ph?s6sqTm7%DAS>jQZ z={{h(5REVP1eswEeK`b6ReE}7%ZVZ*9PjM-s^CdYsFJ8oQ6MrAWrpn*Vnk(H{r`h9 zuZS}LC(01sQfAov5FVA8;p2Bz(g3JZMkZ{-O#LVDwO-yCa^?soY-hw*rmNE#%A_dj z3M(A_*9i-ObU(He zN^nxM+7@icN>nPnAS4=_X{!GpsPuxU^dC`)c$P|~t3!BHX|@lTZbajY;eq1?Rr)pr zOO^V1r#p?q2G4&UmHEVCW^jXh_Z&);b#7?V=tAN;r|3>HiFph_9m5aDQ*uC2JsqVHYap|XP?#Yr5qimZP)9+b!28ybyh{@RmC2ZA{<|$Zy z|MeZP>w!heEqKGls!jb{OUt6~TGqc6brgLYf3Nj9=_OC{Ga9`b(umW|GG>jc<_?X{ zh;M#Fos?y|SreZBaJspG60|W5KsbqgHR17(C5gJQRDON;>pGD{3K}~<5Tag-g2pEH<|0}j1r79OgsfPz&rAv$ zl)@@#>=GXT=wao|qEyKNI-Z`1wYz#oNAa*-!s99M~1}-$Y4L7c^$B{_!thWfcc@W&NL}-?LK`2Sl7*Cx|#_tAYlO z4$7gxcoDz=9cObZ=C2#pkL4!;MCM&N{!!+$)5-j}O+}S{ThF`bR-qyVja^y)qRi){ zmw7}Rk@*}YvzJol5kqWlO6Jie&x{DmJYt;4d`=pfM_~VF!p_;31KT-m(P`g=Wgc;v z$b4Q3nWbPc0{cHzcR$Z*1vZzSO9sd@7P~qFR41rtsjO}(tJv~79g>jI`vzs-K_F%o zEOzT3V#6e4rvEdNkXKy*qW1E$Bq7AJl91gF4&gBgInD=+TIUC-^58@9i4kJA(?YO} zktcVO6}nm;f{%HID*LL*Q-*cl#o$3cqlMSkAzqA2a;TlqE{; z{(6Wy7s#%}hA^!sqU3Y&9VC+|Nhz!-=}ihr!A)<=IQE;;wqE;=Y3)Eqv9^c)Z*>7i z$rEUY(4&wNalFp}%P4uC>f*yfnSHw#4eaI!X>W_a62HvXO{WI}ubE zaZ+7VA>!1s{zGxj7jaTW$=a)JZpGl>{B-@eM}(V1$y%&`DVft;$xu-HfkmbL0#bSD zw%ytZ6_MG`7nv_i5pfO!EV`*b6fjIME|iFq+&QKWwG!`jTDt&G4@ul1o)vfe7eaW9I}3cks7H^=AH~4NZ0C)U|;+?}|EYL>=N;>eTNR5|QdG z^Z_$~XnZkF$vmL`s1PjGIoW&3X{;bDY9KGgR|gdxsH3*=DQ!xHi_zw#5GQIgw=NIb zyd&D=i8jQuwArIDghy>o^Z~PN3T^hN4#85Jr8q&Makyx+;Wua_713c~ZT7e>#EII> ztILKqZ;3Y9q7CtEZOr%=be=^%VAL!Dpmy8r8E$`Sa|&|;6h~CstjkoJpxQPB6s-Ex zp2tMAnP2BZn>R!oSF|CXrOlpZ|6icZNj_lIb{BvuZCE-&$JsO7{?z7FKG0f47N}3X z5MLXlhVZmRX&1E{yFRtoh=?-B*5yE%&7w?>C__9;nZ4$P@aSlZeZUkGjW2c^iEDc; z3&B#E(|D*!HDM9gUW~7dj%x>u8NW~ccoV6y`fJCvQ5^q+5suDwTwN|SdRsKg6^)2z zX*A00e+`;vi4U0mwnkDJ8fEssDp;2lx4S*;fisn^!&KT=*;^cPa=(35+UwD$Xk;~!^}OYtP4Dpb|;Nkm+T#>c5@>v%Lv zr{l;J1}Z23GMw!sUcqs%BrqyAy}Y%T2i7pDK}p*zed zGf4mA_HrYIHKQDTUq}l2*&y#Wc^m>d-j<2A10BWB#&il%r=OijJA@wFla4-y&%bjD zr#t%Ma7XV9+TdlV=Yb5Hu+CL`zpsBh9C2ChdCOYpuGhAngsEgN)${L(BSw{y$IDE zZwhVZKNqV*9y#z`t%2+Z!N>j!^w=5~C%1;!XB^v~Q};_!*k@bKYFvc~bqVdG#)7rk zE=QwwZjBscLn9H~pd1lFc`nQ4s!lt%It_}M{{~#1cik$5Qk8@<0N=s})m zaD7Tg*+Rb`FKorr&#L&FJZ3Y8k3W!PW5a>wDT8H$$$zyKIcz`6DXiKugMnm&8c1SU zkx2omTf_E5v;M28Pr|T$E@47Tgd)jAv;SAaa@hV9PfF`OMv&x0B}r>Z3!Z)+zU9Lc zIAFdcl5o(3eJ7gzznVIS?W;JsReECB)|1;=0OP+3Kv#%3sfO(X%=oV_%nc9Q-2H)eKETX> zu2jPk``prjuIicX@ew*lnJ9vGV3wFJKs#lC%PJO5k?_mXqCtNZg95Zuw{=-AB3fx$ zG&BAOXs7jeIaMZFNm?}O&xjYGonGv+WBg>J~vev!Ba4 zCDDeaMcX%mc2-}PRT`oViJ*xC9EA8c3NY%K8}~Z=)NC}I6~W?y&VzhoZha_Iy2CAOow)G_Xyg#+qhi4Bif*}XlDOsAmn+yTyE?o+74;a zPKj_kzo*L$OGMi~Et*;X47gp;!}YlZ9BA96MKkMv0osLIyFPm`&;~}(BqKRw5QJnv zxFStJI0WL4b4RBlMd~$ z?IUPcba43;6VZC7MPvLAbGs7oEW+_4E!rg!w7(R(J~MWp^-7EOTm+<{zqIFG+7L9*5yif-KCt6`y=r^UF#10*Y}satoYHpeDTl0Onj* zshlgED#>(w(!Yh(RVr0ta(-SLSEcaz_|TyPd&jXBqhqB5iS?x zy53?C128vILlS%Hv#3uR49{8 z{X0va*5-rr^Val;v<1$RU;+``qNt`;oUzdvfJJ=*~I^= zR5gN4cqyRyvb<$7`#-(qs^n2yyrtOz0KMg!6yCyA0*Q6=08!)`9DSjV4a2@^t{G;J{^{+{|D`L zIywZgjrV8lZ+v za(e$8KM84|jsF{HsGFyioJUZ+2q{f=#HHZRLGwo5ZA9z92wb4mFG6+38+JTmIUv~> zHzqemtpL0$q;7DIP^3VJVz$M5yPQ17EyCuT7x6#xQq1yjqR42x8=Q&w4=)mvh@woopGx<}p(!c_OB zWOdH~BJ6SG2H~vKRh<9qEyZutcUlhq%vBf7;MzwlGdJ5u38uJ%NzP*_0%f;B;l){- zR^a2ok1{pj4Y{V8_P@bt#W%C?-6=(Q`W=5eHEz-GJNl(bc$obE24}kPIF&qT$32Qr zrp<=>{P%?4bJh%*v7bVzqs;yfZERLh*J9Ao+h!_w!b=9W*5YZ};%Y}-g~p_--JHVJ zibaMM(%?5MS7X)dHowC4oaOjE;cNt!zIBwze{WXKb^(4FiXfB1*^cfl&USOMvz-Ve z=znxrfAYTFufT6~wx%5XnV*3H5~t^}snFqza936bXKN?VAGIf*fyGyxf2dd6V!rSv z1>rt63iE8p^OxZ|%2{5$-2Q{@a`oB2RhB}f6WI*XtfmbQvEr)0@pbuF6?oqTK%rBY zpH)mHG$N72VAE!V5NWVG{Nd1*cukH!JQI7_$G>Tl!QHlo$*|ePzF!Qa9k-Y-5#(@u z!eG-({a<@84K|y1Ty9HYu(N27ObnJROd-`$JB!3-(@p(bbEC~psKc%V?*p+}J7Y5< z5@`wE5Oex<5u4rVSIU;2R>!Z*#Ae|W0JYVi=C3;%g_)!Un!)j(gHvF)IlSsj*=;25 zH_OCsQGp7noZ=;xn{j=JA1$}At^k(%P%KwqEJs8logsq08A7DD-Q{BeHJf&npXrtw2YP4COgv6oI=6id~=m4Yb{TqyC#3`g+ zw$Wx!3^Ak8PO8H*kKQL@v<}8-M5NJVz@L2{5INxAl77ISeV-7&MGg3zq@0U$V7zd4 zSq!2w;LkSmpIg*`UxZ)EF_dQElP@cop2PVMv&&nO2mIFn2-ll4ETkBDpNB2!dcgmC z4*qPLL6(Ts)OcxVTgu7gJ%bV4b&87|*U=bF71b3b&1Ke3fJm|n`czUfySZD36fH%} zY~x)dC&(bPyC^F*VLEJAWhBx*7H=4G?y({Iyi9OfrRc zF~&i(8+m-+cJx!Id9bgb(%N^zJv!a^IeE z+<8KSOuOL?`_6mCn&rA2cbtA3bCXu5=du37-gaw>-X`IDG#%Pn$t(wD)AjQpN5Qq~5wSqA?jb9DT&9RKGyluO#;*k-FO@;D!M z-Wzn?C=K9pHV6fdJ;PSOVPrYq+XN=iz#ZjyoBpSPWknGpaMC^W*nM771pYI`_zt^NyJ|wYh0->*#sYrp-|}v~F7ilc3UJ=IoZ%*{v;g zGpEg)K0`x7KFWwFP*pk^QEsxm-zi0uQwEprf876tjLu^Z7JE26Q62h$?%$xhe;mR( zyr98UXpuy3!C@i#+=l(Uhch6yN2B5bIA8q=*H_bNpv4xGgyrJSD7O3|XD5H|~s|*r9LD+1WubfORLARpF-hX8XzL zh_~o5E>`eQ(~PJaja5`)*Ny6Rf(Kb0>N5s!nDd02ZJiJlvYl4mFRz{i#P!Mh>L!h+ z6rOINJxF?rCciK*M4nG;e8h3En2w6;laKxb996iIqbRk|#DC?dE{{mL*PASU3Z*7z z;-{8U;-(AVfKqhR+f%ry3~vj65yfw(n<9nGa;_ zlb*=^k95-8lbv)HNTWedWc@oaiB^oSEyYiC(*1KvolM#@cI;CtWn%mje+UUsi-3EN zhucV0>I35+o{0pg)lztp?u<9|Tg3iP1Kf~o02vh*EwDyK=PF=1m;HOe<@g|gG~pty z|F8+~OwojrV=dzPCv&Vj)&98+96Ev=3x9UPTM_(&R}xOz#nPY`lV|6f$^$0j!_wfS zNut4BDKro#Jjty8-=%UaXt1Rm>m)P(G0;#qeU5d~JK#@0yNhwPCuwttPT{hPx&Fl# zyDPcHMgR#-x_GdVx|4$qNP$ouexl3v*DDFzgjN#BEJxN5CgN{Ad+0b9;!R@4{}nN3 z5+)dz<9st+37m)X&2m-PoKjxif|-kbNo|TKjRO#QaB*;lWD!`n4VeXA7VIh9;ab2Y zJrmk8MVkeRrxoLx0Q~LLxy66f#g*Ls4Rb6xJ;BYC?oTH-w$oUS2-z?W|6TaU+?h$X z<237&vl6VCLo7iPJcsq>MsFnh5z>VhbR>w^6k1YEe339f{;OP2H#I9E_xc!s@<@NO zov)xmpOo)SbM}~fxrFbZmcg!lL8X9pBzB?IjNV)W~5F%}MpTCRj z9-rd)yJTWB)>wn9)wS8srZK8+lxC8a=oGX6MXN%)b@Wz=-B9&im5JS=5*1QA#Y;l} zDVM_n4E=Yjz^Q^bAao?(4xg~nDSsC$-K{oe(7a3VWC9*g;DdpoXA_8Fu)c|4os0+l z>)L26`%!-j>r|tEz-uWUsISY@SoS0JHdbSV*C}|GxGq;?*-sPOSZ4oU!0S{zQ(c#% zvFxX+ZLGBsUZ>$9`MPY4Wj`iwVV%b3U+AI%uhVggLtU1}vd?j_v8p0iXW$X`I=VG1 zZ$HFtV@;1>or&W->VQNnKG?;+z)j6ImZ|>-5}t*VGjM4vM6ggybaAe=!m^LF zwz0VXE!X|CBUml0Gx|iZF2q5!IFD9g*+x7vX4BoRF%p?88xQtTQ867vl(0oJ6Xy>_bRxtS2H^f5J)HI8a+**=K3nSZ4oQ z;8B<0fKZ$hs<7-MLM^N_P5u*LU5b;2ap17Rvd6 zU5V>gaSN-$vM*q@v1UfF{(_q{aiONdvhUKgv2KiDUFEHE{Vz1u7ZNwN#{b$_od1P; z%GKV}uK&5l`dqNOhq2D?9KpKAd&>1c(^#Jg7E^qS*V%hUu&(u1y8fpc>r=tv*k)s$ z7{OZRJ?Y|pKV=R3YCjvxtbYZ*aUHG!!);&+>to^7C5hLk5nk8hk~-W~r?5T}tj=Mq zbNWTFZopk|xDrlb*|))25}sq$e*y__#HDMvdre{4*RI)EXGeJ5gv$Z__cfM%J)n(c z_J0JtZub7>`tND1_e8Ju_+K0A`v|XFyvJPsU5)jwU=@b3&NcJ@fY+_wqptss#(GDv z3X-r){u^N3<~`#2Z)>c#1*=^W7W1F5_ulS3?E0HE)@H%VPr_Os!MelytLwj|vEC9a zCPt63Z*MC)Gy((BtkSwh8yGF3?^X_&1 zS2Wfug2e>M#u^pDy5GCU^;tWGgqsA536hOoM;Z*XNF8@LDHWOpq+B3wMfO{mr}C^`F;R&kGh4Bpa(Wg7tUrCf9#XW8owK=DoE{ zkZi0gBUq1nH@f~>jkQ*=m>}6$8zWdxcsIB{cO*lvX9bH1l7)3qn+Vo&?|Ro?qp{Wq z784{JYeWQVg?F9nuhv+r1&axijWs8N^`y7V^`FsL&j=P1Bpd722-ZsPTGwBtu~rEd z6C@k!?FiOW-ZifOw8nZ`u$UlOSQmGTU_I?!?fOq?tfvHv36hPqPXud~ca`g})L1J8 ziwTmAwIqV|jQ1DUe^O&TDOgO9Y^=XVuvU9ly8a4{wL-9%AlX>|h+wVpu5kV38f&>= zF+sAh{sjw|*MCA|Jt0_3kZi1jB3Ns^KfC_p8tZYvVuEC2ogcw^&b!R@|E{tA zE?7*Eh{bFLb$~xT4J?)}?@lgV4ibQmgltfUdA<$@%%V{NfI3X4tYTeuiihL>g@=-Z0vw-2czi zB{13p{s=jh^$5owk%`f8Xd8-GiTgOT;c~WMxU64{;V4d%h-Bu!LG`LLo|w?5u-D^O z%uEcILZm|Kr|fVVp0O*U3Ou}c-#nZfrr!sp6@t+ zbG%pDswR%No*3H~+JZPORO3O;74ImyUFk3$KbL``uwp!O8$l?lCB0#Avr;9Vi=)K3)G&(ey<3m}F*X-^gU%R%2!eGLO<&vzY%enNn`c%xZ6;X}Q+ z?3394Vjib4KW^zW^$w#eYOK{Sy~C*U(}xTJAx@1s(M2O)!TL9gxc8)A{kr00vCutg z^$QwoY4z)hdy=4`Zu-@)D>f&fx%x%gG(#47;MEIn8A7gX3rG$j_aqM?=Ku(*TscTc zsSXWNAe4unXzo99hmq1BG$~^is0TZYHVC(@<@h7C!{|zkf1~h~Izb}81ApD2L;1z{ zKwFp{Mpu4jE8sA&TvphK$ZAH*Pn%`J~+0O?Un{WObA?@$(2YEqwB-zlW%Ef6+Q_pxA*< ziTy>B5#>^BOrg~?ifLp^mJBOkdo}m}gk#!pUN5N5{zXhXJJXm(`lP*5=>I81pBs2y zKwjojt6Q91+9>Hu%bQ9u;@FqbMA{;Q7@x*WSVrC>17iVPQagyM)ET2yy5;f%*X|nP z%Kba{c{p%&S8|0?&6)UtrBsbp>RL1Zy%P^>==0`YbauvH#oWX-P`z|I=<*ABoID2r ze36KZLf3u(IXMd5n|>5p#`sU4zL%pAf^W97YZtY{C)GxvWt9nNj4KS)IDa?g1B=Tz z{&QTpH+fvS2}tNn%gp>wjVp@r2$pV{k)55>#mUXg+vCL&^!~V8#oX~l5YrtGaVYVA zcth&zs>A?V>dpwAd+w6B`(~HSyg8ftlCU8j>i=2ALcEx6Rptosxbi26i!)sJ6(pxK z+?T=`iogNdU2o?9_o-cBV-dBhI=h}T2|p*?p$LC7fgpUnSmQo!vT+&8dILxKg+NJ~ zaXr^R=nnTKyTb`W$~^zC3=2|uPBn4q4wG`wu`_UoEN6p)j%O-Bg9pWs?tMdO{^vG9 zpI5wNrIFFSkIk%=^QZ<%ODo@SyzN5nEX1tR5~oq!QCH@aXyWli$J4VUPQv^|i`>Bc zpLV`8+0M*P3Gg2)WzyTvuHI(tnfTF4UvKOKD)f~HQuvA_;5XI+h6(rsYWvxj%p!sW z9DjDgTScNjc>CFnk6Ie^97Zl9C*93Lfvo}SU)12i6dLpeEKG4zHeeX~9#jc9G}uxC zep6`@G}KL>fZt^Dp9j_UGty3zfZuc`=(5EgOl~nwkSXSE!uSET{p?E;%EM3W87a4) zky&K>*_A@kV$`JxUJ@!Cm?tZ7Sq z@0MRe^tsRKL4P+n?iuyH-SSlgjYprijQobpX%s~Ur(4bQZx5;N+6lkt7k@txZ?wy; zX8-3yYSR;MAKkXbMPSBF75Gp`zfK-rT`|0@#ykp9I=O0SRZS~S4&`fdWlL*mc{QE@ z3?2PiOS6-Q@VU*jf52xzwLb9&d=_dx6%|vet6R5xEvU6+5B^eEyE=)Fng0iT23G0w zi@;}5#(Y>_ReZSOL0t~T8+NM&X2fn9v2YByj1}}3I zaBVe6kclw5W5)=iLFKMDUo)Do8I_cm)|6G7yUJoRI$W3>fKN2k9T!I!?O5ibXod~M z_)v^uUGA9Ta$T|z|GwkD5jKNM^%-FByOZOyfq(OU<2!X7P$3`Tb(fj{1Wq?>n2Xyl z6|d7X<)v=*OI$KDK=?3~fjit~#=n5qPD5Q>!Onfyf5`RgWJDhXv%5P+j94>w;A;uCt^vogY_Vh7h-ieGoZM|?K8!VP+j^c9mLpp$_#)xA4Jm<-?E^*CXG zW1JbEiFS+MtnXgLZHX$hZO?v$NEPtXH5aGo%_4qF`CygneqySK<^ge zB_Z(s8za1SDRw>78W;j+#^=@1X>XzuZjaOVt$8YO=n`SJYmtk^Gv(<=#b=f> z1bWwjjuiqQSRCOtqMz$=a*cj9^*3<~LZG)yxJd|n;GYq0yY+RwS%5+aoE4v2x)A8y z3Q|@Ge9+8)125UVkL%3^6gu+U_zc4#(7OW!7%LGZvh}M{a6s6SBmM!iNZz- zZ4Y~3#j?#q$z>bynunW27FD)!I9=VtJ4bH2!zo+mWdDvCb|L5LDx3_$Cvt&@*N3=q z&&r0nuGn_5%mo1UtaQyPX6-^F5{YR75Fau1e=Q~p0T21RbITll$DWnl^D__=O~gXL z$XZ+jSv#R#FH&Ts+p_`&!Db(dBf;tsv;GlSZcp!Au^jfSoSTW|QixPY{S-4X-lK&P z<87?l3dXxmjJK6B9ubMO7;hNwQB(g8jQ3aHgT0B@;`m-B#xqqEb|h^%=$ZOiHIiRk z%oqQ_TYT_VUh?`*GlN=VfBKWO)jGPdJI$9;=I(?S;B-Fx&NFwr<93L?qHF zc*AUundjdFvpwwB@)&OXex&2qW@0uB?ew|Cv`w2XXA7naHcC;R$y%nrbqXoN?mEgl zUre`$9PiyYfO?_^x+qJD5{EFIqAP8rXmdP(8$O;LqR+ijuO<$Ez+NdH{t(+d8SN{e zAQVCref*q|Alxfe$)Q4RwuGYRXhq|-ldY(XRgZrG`W&ksNFJ-YX-hoO-da-8i$bT~ z;XjHW2_QAvC(1yDODYeiSW=lPgjoOMqR_)!6yho&+lp#3^TVqX{~^IYcv0vHjDMp* zR_z38cMQ24JL1db_^>tjRy24dg$8{A3#VA#12CN4Jil3T1w#C$?XJj zr(}}Ot+*i}T`RujO3dllhfA+^JhyXp632`deHICuBk_r5f6~;y9+o47v)nVtDhogF z$LNs_v+&2G33z{>pU`HSb2q* z15SXUEpe78lE>6ZYHTk}9vfz9R zt?<P=*0=34#`;bHig{4Y@FEG)ll(0n$;S4ZoE;c*Z?VYjDE{U^}p94x)Oj9;whHvK|odx%KfGak?=sNJbT}p~7b(KB3gA<0HJz z!}`l}_!VW_=i>81%>uk|7bF$h5$I$fp>x&S5hmwj@#T3yU?!Zwg~bT!U}9!;iq&9Y zBiY3>dxI6nsz;N@Dw&l$bF|2!W6&~$aqjGI?7S;;Fp&-!gKsF2ajo(VtmT8JCd z>X+)aMp}I%*eZ(Hr^RGieX;kt99)aE`t=Oc>UjD{THVXZV4$_r z>YCx@O&FTDc%DD>eid&Sa#ox5Kdome2K%<&#c~QWhMbG_kQ0ZnPOD)fMVlEd{Cmy8 zp$4c$b6<^~)R;^ABduPaNpokER#OlwtzPqfNRV+5WJeSb8#_`ED0)X`V{2DBgHlu+ z|5?s|>G%&PJAO9+!V=H6SV^hFjKlfq$(4`bg8(ulKFj?-91{PQA}OVNlF*%^$=?#$ zURE3Ad{m?ZZdNT)n}P5@D%~cM5%D7t@o#BFT-(ML@hXz|lY4Mad{`o`9U>w=oIw9P59aCm3k7Mf%MDxfuy)X8xp|Ci8#p&w!*;J(1jE`vD1t ze(n(=^|=07-;`IA{89;Yz3noPv=LZLfONSJm5@#nkS#GOfq4S(V4 zwn|55Io|1dJ}{-Ss;Z)-h%FV zo6);R_tnJLoBh85@eNp%TCHiXRE&LSMBA)?26Dg9OJ7C>+Ry46mRnnzO3TXaIc#dVS?IUq-Y_U4 z_lrGs?he}PH0_qk>Xxz!`%Kl;v==AQJ~cx7rS3Xk2kqZ*54SD%yAj$iW8LK`NC4VT zDY?rknp&${D}TK$j}6Pc(d7Suy*J?kHKxm;y^P$on`bu@Q*WZraaAfQ>ME@GV%H6F1R(Vu{(9%)A82moa%STR7e#S zJIOg-EDo_`9{qA%59HA|ktg!#9$CdyMk5kQ?DwL{|8)D%evkPhkpK`6_8a*d?WeaB zY5QemXulUZ{_B9wcuV_r#a(rcTZ{dk{f+huWGbYJw*6i*>wnq|wBM$>p0MA|V!xio zenh1Gm>{C9U%DbBEA96;zaA+a@o;QpeJ1v^HmGRJLD7_#-jH~zV;=#U%PB&6^;fTHp&XK?Yzq57E!GXQd)-%eb1=hKQ@wb7)t04@ z8YwQ)MK?Va(wgRawQd`j>sB$>HpW~;B$8AGH<|rUK^J}8-_y~TBcdv}XC~&7Mh%uN zbJJOjQY|j#OGIJQqxD`f>mPxSbw{Z}KN15Udp;A3C5cl=y=)(Q<%fvTUaRX3qunM( z>uroiL>g_Wmhx4ue*`gvKK6va7m_#PVWoVpOpJy)5jG?xEn4Pn!EnK%CBzANWn8{io%;oLYr0IIFN}(RZ!g{Y%DdP3gR00)J zJ$t?P_1PhgwA~wZePO#h#CCm+?TAR*Eyf$&`|GBoEKORTH+BihL(}y{jk&QG<$>32$lU8f@)S}z+w0!ABgBxFd$X<|EO)0^ zuAi|S5ox*8@rLEz_%MV>%RTAu<2Y*AKicv4$;5K0w!UfeflA6T6+#4KDn**2#d_21 ze+c4TFAvv8DzWg?^2SUIm`c6&WiMkw~X# z#J9};w_vnb>0_p$>bFSf_RYkKkv(wQgzeIpkfJ5h63WvoGMV+yz=Ydi*(Q+?zmSOu zQ_56G743v>b2$Ffew*uXRymHX10d{IZ0tuw+D{U?&8Gex*zYMH(|HwYkkIXyiT#3& z&DwAU>BbE34+dLGB1;{c!{Z-&Z*M+XRzxj4JXw~B<-*bwQY*zrT<-1Gh|9fQ*B>Uk zM@-h=n2d;)$=>GrcX+IQ+8^yWD#XBKqcb;|xwH*Um8r?zVf{0Zh9>L7MOxJXV6vAo zH<^*9kXqR$dx!PEfET^&ow{vdvU|m3+ZvM*ktSnlAKm+%jUhxXU;N2?14T*VVfo^X zwi&F&qwF4U5%BX(mB7H1h@JNZ+=E-RJf`VAf7w=sc5`?>gs<;hu zXHoQAt!TV(HmkQ9ZkS{X0T;jc{uVug)Sl*le z)8JBEgbI1&mbRLbB9#7$oX6yrwtT3Z>KG%MBF?pIsc(4G7r2!su3}5R(2Y01>0ikw zd>>r&BDbU);?RSs0iwk4N*p3r(qdO)BRj05IT-iRZXc}(H3?5x8o#^KxFfx)+}T|R zhd5+)6hvdHbL5WD)IaVQqRtbR)_XYOW)R8mgClMb1=F-iMlp|U$x?#*$CrlKGUiot z1M@Cqi{O5qRx5tYBYo0di8qY)@xMd#dBW05yUOVYw>Z0E3JK{;D_hGkCdD{mDb_Mr z8Z%)zY^-7zdv{#D&iJ5-eZtZwSpN(X1}q6@IUjfnoyL6a2;M?%vUp12Fd~>komL`q zl}|(UKbI#geS!zr^rhFBt4#j`96`8};~2crPM?PIKbI#g-S4$Hjk}3bEt&X%rPMJ( zK2-m5dHdNi>*fmfy&fhC9docmvy zL#$M*{-q!ZBJ6Pg;~e61%>NP^Umw2WX}S=U9SG0a)~$nV;`$6vUlOn|9tF&_cQ>GPf`l} z;}}~3hZ)4B-do5sVL=x~qqqL28ANFjvROWuu1+gx95s4m{eeYl?9JqkSA^{U%^k1w zi&UwL3mIW5#F(-v)s@H(>?i(G{-$sl#w9}IbZCSwsrG=}V~Rr~TR z7?OQ|Wpdw_{`KYUR{zq`y6Y{CtATCBx-U0?3ZwO^6tPaC^_P79fzf)Eiq;<8POJGp z-lO#u_CF`0b7}-1?_8T@YO}4!80i|kZAqYLx5p*IaL*4Yz`s*$U=$VLj9_XZr)?e=mNVeEB$t}i5E8wkPA10($sc7wy5XEG*B<0f? z9)9424KI>7A8(Sm!6xHOa`6NwE)Si&q+ zPiLr7`)_|dgQ6X(#ruZ)A8FoI4kG{`&o~=r%Xh-x*ks%(JDFt${Xg4V3?k$suKSJY z|Em%^vd$qd_W#lB|Jg7M|6T0-0gTC1djGd3|5Z(?cALD)&C90T^os8N@!CL(scD}G z9%$ol&qx@NSqzsbR{3LatR}l=ljD!c&!CM_jvc+PI><;*72Gscb&w9K>J(%vydlhY z-2cTOvRZXP)slo&zsuhL?`lNg)$({ozMG8;uBQPC&oWF?+t;fmmsC|(G}+JkMfXEt zp_WeyO`K3a9xYc`do3uC$3b{gmSqt>IXLc%O|!c>AR!hX?ENR7#KR1=R|%6tG^2mF zi!jQ?^Cm~(S8SEn=RU+@#D~T;qwzQpPw8|uh>*=>VKpzpDi4pFaIo16AMtI6IVD*| zkD{r)M!4-S9L@7TL2I?alP5>w*In@upJQ7mj~w;W2DN_yJ~@a_e7|2rueNyZgd;V4 zVtRZo;uGe?o7!czp zS%^CS=^bIyZYO=j3Dh}8s}qNf>_spqWYyj&JPe;xB76#l>4PP}XLiPX%ws{dcMBg0 z)c^S+!l!VkK2--#x9C%KVwo*zi-5A-J>W$?!s~|vBfQ!V(Z{lZmp+y)173-RVa4k{ z;d2;1;SN7M72(wZ$4D`msWXR+#Nw5_ey^C_3o>MMxMucWbN*-0lRDxQshRi{r-00i z-&i_1)x+>;b7ocT9m2_QyC$MqCmbO)Lvx$)o45tX!_?j`+?wzSL;l!4!maZT`iMIm z4>LbLw^WPK%9!_q5t&Ix{&7)+VHcb#bvz)@k@evtv06@dau?9aK$_|&*8iCX1x&UYs1Iy`W9S20V(FqCBk6>!+6RP_ z^sk@H{_lWWHynwy6j112OSMX6IKH*DYRlWd(UHh{q zH0xfZPYOgUogAMRS_!-o2i@fh3kie&cXou$*8TJWl?a0;#b<+XOExA9`b@dW!|?eg z!ly@H*INPz@L7^6pF~38Gp)Dy{Cr4+PtQI{2On0&X(Fu;_{h_d1;R%%q@Pzt`1Hcz zN2dY`8Pcio^@%(kSqKtjB0l$vS^o}vZW|ncv=|WZxyA7rX`f5%dSh?$+Xx@&vcH)1 z?|@ftY~^Q?iY|L%d|siaBXfXG29omqV(PyElRntfzX%XW`4+`zlH%#eTw$XdsSrD! zlcPi59{TtrG|~y0RY_$_bM=(Up(UjilZRHNe(b@1_*#Xb4PZnjy0aEKX8k|Vsb7IU zwicarp<-0oR9049Q(~_&rOb_F5~0|y*tdqkpkJjc|udjiIH;3LHNY;KCapSsCjX-#JhC^uvy|h7Xa9jGa##&Ga-#g zq*n#;=a7)BMqcJ^mYce;Sz>crC1jjxRR-EHPToC-=(K4*jGQ@enD-#)3hh@k; zK4ORKF$_PeWq4N05a&U#Ok^N+t1WU({}1eN14fy(T86bshKe{3jAbIjb72|sPL9~& zMhrrPfCDM<#_ zzZs)~wz#>Ye&`T(Sgm9zYi_A3X|1xCwCxa);TAkBw@TY#m6jpS!*ZF} zVMUS*tbd2?aBIFkvH^B@Uda&ap}I_Dcp)r9e)otBw`IHDliChXDj7@!X+g#z)A&0I0*$3Iy3W+2Ydk@?GGsAA);11cA1A<8c*9)nJ_8kw(QDNhjW6ZJ-n4AUl>b5X7Or>K44&A z=zsw-%g5r?MD-sJ7nrF24_}ky#X@$LP1c!&RP<^dBd+$m9P4;z=Bli?1m}4-m$ce* zp-7Hfe_x`t6>MmLU|SU<782qyk52MkcWNm|4XTO~wn#W!5gI=3Cp;=fjZxJ+!Er_6b72DMX(OThI95-J9LE(ecOTXVBheGEbwx zv*RplQ4H=~p{l0-tyNtHJM#w6#w?10Co*5vQ_^^<+M*OvA?u0E!cO~#l;(u=y}E&z zkls&Ln2-+4Dy9w^k-_(5ydh$z^FxTV)@py85~qx2Vzj74h15>*q7`6)&KHLG(RAc zi#_L0bev-1VKA7OiN%7|1Z}XqUpE-Vkosgd=08E>(O}$&sBff2ee$hL3>FrpkSf{r z$?oR)H|;32*T;1`!d?%Hy>>MABBIqjyPN&bfrmZsW5K;Y@z6c@&%|CyD?Qq1ZMM)o zgXad9f?X5daQdxFBX(oSR3B1|?zuS=yQL7Rkowu(bL;Cu3~9Vi>ITDjkBIRG8{-j? z#^VHtlD-i_q_eH_Q6=A&czAg5fJ}^+=n~@_wBg#ONj^wC@uVsxkq+8pYDgOT+n>F+ zgAV$3CU#4brjTkWMpPfF_t+F-Mw@+F_y3rC4>+lctbMrdBxYa+h9nt!1XNUDGBYS} zr<;)=nZbZz7$zu+fPguHIj%XZ8FK=2-c?-n)iviFFs^BJb@RUauHW-io!ed2VYrR# z|M#`OrpN9&u}+;jb?Q{zouRWwL}xn(IwK%~))>G6Jsd#l?Byu-v_=pPXUzA?MrZbZ zl2IC<)6b$rzov{Y{((0aa1dkw_rJMleI?C$Dk5!y{jXQJp2DRrrCcHTM49eswZ#L1 z{l8bZMnZe88B0;-Y+i{YUBv^3fI8PmuOzRLguQ_ig}s+Gdmt6d!&d ztiTTn_WxhjOB2H95}vGD@HrXbXC~1vRG+4SFOADW#%%gr|I;T2^mDvlr{O;VUufXF z?2J#yKPVypYxsuz=MDC{fLsxt_Sg-9cL~xMJ@9o;tl@gY$oi^nR6o@p=k^EU3D$$v z5Y&tggNDo0wm?wuf$Cy%UA65kb(M>Qvg?Y)wWC{Ws<1dU4_axdYN;;=jV|2MNQE0?{ zimK|0s?NzGm~Yr)N^@*Bo`u#p3tSiU#4e8}9tOO>+9o<}O7rB#*tD4q zGn%JPn=-X|kA@iwD1Sj;RT3UkKh@X9G-LLj2}Ikl4xIW3!xEP#vu7-riHAU%ekkar zdTa@}E^&Yuy;+4CmO8s}zH}ojG&~>-Z*nxwGHL?DgQ8Gp^6Uk8(nym2`>EdHksu2r zNF3GoSBR{`3Xjq&UB%cit6|383sE)+?F~?U!z1Xadd!$TYf6$F1JyR+QK0Zfqj01c zr22)&00WxYu$Mg~2CM$z!J&j%3uZP?o;JT}+N@a9d@5mx8W0`<{0I^3)-Yp!qlO-; z28M^GI{@^o1v6%7;9+V|*1%2k78uGAH8?zQvi~dCRt*Uc&6?Ec3>i6PYN&>+Kq-86 zMRn&ctU%=jBYlJ#79I}q7s{0X#RAmCUxUIUXsztU^(tCN( z)P?)-k$^4gp|%YV3&Wbe@{jlb4E$v@FQ#Xs3U)nCuY zvlaY9`L}8g%eQ3m^luLjn`}LpJ*7IgJp1Vr*N|(c72;7B>E=%<*9|+}q)ebvWf=;7 zGc%|>3p*65&ZkhR4Gx7V-5e?p`pXhNF^NivW+-@f7L`ilP>{*fsMKtRg3UaSN;!Hc zEEy+KDTNJ%B>hY(C1#;8WSUB)dMp%<40EYe?u9~;ZZegkyioAzXHzL84TT}ibn34$ z?jcM#nL3|Jb!jN#0&PvGQq~&^I@u>;MwLq4P~b^Zs#K9?Kc~v`y+Z+}Osev5>rlYX ztm?1VorLPREtppQJHi8JnpZiec9?l}$6)^}SpIuzjXwix>RkMeixXz#nf@N-eA=7;#%l?4>iKL^V%Ks!ci)}fWDhjU6 zV^x^wDQ3z@*TqXdYNrl|8uL-##6}|@^(a#d*bf^WpaEt`_=RnXUMw6pog`f(SZ|Xx@O!kTmR^yGH8jUseF!9AK2yIAHy_sF=Hf9=B(=uK$Xp*X=7)G|{LjrayH^0rO21`#aQIO zr{RBfK~(;|{PpqqZEu+dUOPT1!2j9;DEgLP`C-}GPgEiDnGACoS#sxPl7 zkD%HrqO!S;f=WrcP>NqA-%u6i?yJP|Mk!S~=mjdAodu;= z1owZKd54bfkN#TKT*r>Y@Lwx_E(BuZ=Qas$4mjfH{RfE&gBMo4!W)D1eOS1#stxm6 z2EAwH)ku~kMXXBJza_}htL^M>P|dj_)`o1o8u1hNlXyd{%9|bhys+xc3HXKRZ&4Gv z=-#NoBQtO=pwmkfb;EGZHG`VA(uGyURYCo?wgk*WIqD0X?=c5NaQV@3MrGlI!re|# zhraws;;yQ9ZH?&*tG>p$$!5Kig}7_kRxkviN{T#@f_|!=?NBX)P?ymB^fU{h+On|& zOQ;oseuIOa7gk+Y1qVnoSfHx1(OBrwYh74XTr+${~^MyTt!+7{}-2_U>&J+FPncGJ}e1pFBS>jOe29r3AGQS z4L#1AI!d5{bP^@h{voL~#LXNfjO!b3%_xDmIisYB62?tJOZM2C$vrj>Kv49!MS|)L zx=Ww}B1Q=evD03GVu&^2XMF|A75F5Y!2fA9Y7so&HU63A;z<;M`Df;9$5z$SR*$Mt z`@ume^HBs*PJf+AO)ecpHa7xdTT(qrHV-tc*3FAc+>UQp&3SGQJZ%nttC=3QlR6>K zIW@aV%h{Lp39m%=R%#@NFKRmWF3Q3Vm{G^N zm9{|rz{oSybl|77H67z8INYqmBl!0Aub2D#P;_-Y<{0#7IBAo(DW+I^)%0UU+zWDPz3GY61y?&VnO zS7ZTWub~%LRa7Ii47S|U-ev^+lu_2uyFBm6Spgr>)xQ`DicwG)-^IrG!eBPBC6r{Z zp?3|=KO5t2z{p592xF@Izcwzw8=(1|IT^%?KHC(?UQXz zkUo1mK0V}g!9Zk>b$Z}r0`ET{i*ILQ5%FJqqSb3d_4MfH{{g%MNGE z9(=^%%gQTO)t75$cEGZ+O=Ax_Y}u04b#=I+x2C$TqPDWWqOPuPT>ZGQtJ_ztUb5_f zF>Q-=URC}#0<$YVaMgh;+uPd?Y+2c|EP^;GPhU=jd=6L~i#Me3dH$VBHd4%mRqTuh z;OIwp^ie}F0L1~SVG*>?p@ak-Bj88G!4V|nW1du#3XufEuXy@WVGc9B!V#cziHnf+ zc$0WA{=er`uc3WHvx|ocISrK-5s`>d`m}!<<@d=QDg(V?m}Qje3_z;>3x41Ru{DYs zc>aw>`Oh@HUX8Y3xgog!;GY~t1APdbi^?npfnAiB&{huS(75U-58;JdT#jS7{bwd_ zQU4Vm^WMvVR6qZvX*fP?ZhsPP|DA^0lhGF3Mz=v5_U^y6TjIl2<8En*>QbEG*Gv1S zvQ4V2j9ua#q5oZ}+k3mOEG^l9@6pYNEjxJG@*|c-u;l1IXVp=y2OiS0Y|PT-tu4@# ze)P(Ce3GYN{x>|88J^6_av18=(lX*{qSQ07&;W^v_dh12<0Z0?&S2`Jx>&XgIlwu= z2`ZJWv8p&YyOPQ=UD=4PEFP&}8m`4SM=k6#3AvvcIT9+&*DtLY?K6!^y!?D2Bz+#^ z{r|Mke`o0P81kR-J~w{;nx_{fS!nD_V1b2>$W7Dd;^!xKHDaMn;O8szdDY6WylzQ; z9{-=w&rhCf7zP^T1sGNGMQLP6O@{wVV)*F4OrUNpBb|JyFkH|H!*c$0L_wV4s)AzR z8J1V2>}<_N=JtVCwg-7XXMR2ufUG9Y`u5pp^e zso9%12tRSQn3y+6SuUZfETVQ&qn58cU`)%3mevE?#~j$wwtV@DF$XR=ynW1}OUEo( zK4$sK)-khJw$5)~+13sWBq9!D%1DP$PmK75HS#EwKB~rY7*!njIb?#MBFmIEk)?YY zSt4i$S*DyWB)U`8U9R1ns;4miW7l+N(1P})mNsaa@+n$U)!ma-%~1%_QziZxq1C8K z$JIC{NsSTI3-?*ozH&*+(j`Z?tX{Hw*_e*yD-UT|O}V`aASBYMf_Puy(ICFtR@-P* zV|P!dc(p!>Q@Ug7eKy|0Pi(v)x?}2}0ZwOFnx;F%8BXgbBue8$wWdo6SQwU9;tbUh z{QtyIO$vIa?GH56eQC1p#TiaR{2zCQB~OEZ>aqUa@9<&ymT5fyPKJA>VOY{()1DQE zd+DHeBWz)G4=(~lIzPfswq!7iE}!psfqWI+xDy^gsUPRpM-8?{l?fq{PWp~EE)Jvaay)+ooyYDTi zjPaB7wEp;sdf6_ozsk4tQpQzo&XTqW7P7fZmSHdj^CppT2?SYom-t79DVL@iJNhnBj$f^T~vS8Kfl~`;T!N@W3$yl3ll7o|iEvQt@JG0qfRp|;Apag56t5Jw# zGoG{2XvyMe`sjyQQ1yHnb0RoF#hhlc{*&g^H?uj-i0< zLT|G+IeME5;|eu(b1DP)Y^?uuk88E#Aq}3|z$cXGr=ayUPbdi}8?$1k5I8^}`Tr3H z0gXE&f<*8@V6n^D>%al+vR@jz94cJO{cmXVBkXJuj01661V5&8{bSu2ZF63*u@rKJ7^<&D({sB4oSN>RM$-{63xVeb{;$8*bOf3nU)srpGPhY+>aTT8 zEEy)+xo4myb=^N%*QW?+$Nv|Q^e(0NiRO1uUa9Kl4EwyMcdxqN{RZt+n)HsSGq7Uq zJqm&mi2}cwIFd4@ak9Bjw5$wB$dC*#JZj4t9|aXqd8czY8761|e!LCd&vb-)=2lDdA{`7oTnp!{)B0s+l1B_=B+Fu#j~>Bq2gi0RDV4d67L zL77cw{y{=w5KX5G2Wfd{rZfNYczX;}w9jZd^PfUXn$DnP)0qz-Snd2T1eJ+JNV+00 zojRC~JH?_Fel-$*tCqAcTiv|0W!37*ZA+V$9MT@aCQH|f>dLB`YFy=157b02i-;z! zf9X^V369{nA&il(Kx@39|I8FHsL2?SiitGQ|HTm(K4BwH7bONJB0bYY|IeNolDTIV z;pV8kQoaC5Yve$PRDJ`^_tC8ygntlCzMcKh5#EO8$ZNiXcNy^XA} z!ba9djkX4=8D5M!2|5*Th`EQyHHdhg1uQ9d|e*vU=rE$uG z_+SGOVz2okL|GakB<9~s^8X{sR6B&=RGk~7ULcz~b+P z#o@^;O1JF&jZmMErdw(Z^@YLucY?MpA1X1nHN@Se$mRdWm74=#A`ng%vi~_0CM0u0 zT|mZ#t8Hbyi!aq}w|vK*v6Qv&*TSx!hFz8)6Hze#`|)jxF0^j;;3`#EtOo)9R>~T_ zUgBH$2snlK&R0BGD2o7l%7#Uc{qX(fHY~`QL?`=j|Ce29xTB3@ht%v+4a4d91RpMU zAWZ|ZqLY1YbTHA(5g`}?Nm|F7(R<@aFK>ivC#{QuN6y)Qmw-yr`#HJ+oZ#j#N4 z=z1jO=31k?%p^`pUC4DUz(&#~Q@hT*-?7X7eakpFL_ zx$1oN{J~0L=*CHl~vpL#jv(vP#R{^|JSLS@TQ0BJTcA+IV$3 zIE=X7WS{_B*mSnFT1M z_+K4dO_;WfFmFyg2Wggu=tjiApdWM&?0*b3{D;cNqG&Tki13<<9|*82F;I zEM~f)YH;V49gJ3c-~um6i=i(5*&)w4x*{Sq_kYK?S^Zq65A*EchItfYw`& zbHL9RaGk%XQ4x5=&?1^)K=z0`VDCc*|2p8`4->(_OcOyMokWaneEt^=XJ#71i5L;u zd?7JY8xAY-t2!&TYD4|Ew&j`HbjpDx!`iux`45`T%w*G%tVUZf|Leh3MAFrP>D0k= ztO*6$$((brvKm$7$I5E74;5|=i3YoW$LJ0UOsrE1m}AM9mDOku);|XE-8SOR>`r>7 z{Q>aBo|&DdXJkmX2kZaYx@YD3`hq&o^l(%HISXha$MrK3{%Wp;AENZi}O z_&@Hhib!|?{WDcHEN^g9zo5E|ladT-G$9sopB+~@m|CEvMfIw=iuOB5)w>;>(S3H% z|0Vc&!p71^)u)Vc^2z}u;vBH815R=F7$Q;IFkI8(1O>Q#!J1XcrzOq-PW)FyaazW! z=4vC(co!4J@$_r}V)hQ+0m1s;m=48>W9g&Dr-(yvibD&8&pF^nh{IvknB0ARO@jk_ zS*tU3ExuIOE*SR&KG;AE@W64vLIa$e#sDQzabU3inHx`36zOG0Rbts;B`SJ+#6$%a zB*}@29?Mb^6$jG(Da71NLcDDY@fQm5`IH&=CJFIx5n_HCAtVr35!mQwkLJ}2A4sg&^pwr@LdSL|@N5ekSO0p6aOM>~|@W4bxSCI8+ zH+)%%iX|_LV&-R1%o6PX#?w%mkQ#7(*YknJ>fM7_|IFT9pcPZ9sqi26z9cFRsuzhC z#1j?WJdLmvlEphQHBoU8)_=P179_JMwsp{jLcKLjx72_GSn)v*3iZ}_q9R{sNs)@!c+mVJ{NmLvhA_X1$VyaLPQXie7L`?^qI6zaR*8zGLH^rtR|fkkD3Y_A!*CH_f)DFYa0g3; z5O$(s=`86MgNbGy$J9p^S-uGp2{?U|Br2Ak69*3x6(yQPV_s3FL`CU7nmuMTOZP#d zq9|>mqI6zLqGIX4r1v{A^!_1(tll?{@T+03{s-(@y?+Sf9}bnJX?kBA;UR4z*3!5m z?5@SCN{D4S!qTJZ2qAE?BP`t~#StFzxCpT{lMtJ1A+#gh=Hb6T?+URjUxZkeMhJ0) z%SMS1%i@l(OJF2}VI*-6miS9MLRyIB2$u!%pFu+liKFQS3yGuQ0Mtsd9O1I{4i9vM zJ%V8bGPWaJ_KqlKSq8-{L;d44!003Lh2&6;Sop$q_^^6+IpaU}?h37#w2p8&$3JI? z6>&#c>S@kP^Q}%yb%e{05f)b@vnY;m`ISQb*fiZz4Nl>rmTwg5$7vlPJeLm|qdBe+*%U zG8qc74Pj*GHE6*1Z{o5CeDkD$J_ayL6)!<5AZ#+bZa&r!W{g0Q zh3`X=|JD|h&^LxV`VCEHsUnMS_}i_@_a7a?)Hev5L;9|JHiW^H&G*Wk9dO!7epjXP z4BuTXzVo&&-z$UscS7GF+^h-rQC+QR#SjKl4&R@r@LkXiWsipMZWiD9TbJ)u`#FTE zZxA+j()Ur_EWQn4FlF<-isPT2C1EFBijg=rd>31M0}z zZ$lVN*?g~Va=>XPg(WCqFnpI-e0SN(e81=rCf^`zuC(u?N-Vw&VK8O$eOR#rPQH0d zB=V0Om)$MC3%4%chn?aOCf^`zu9WYix?6l3!eGkb`zZ&Ue0S}Eyqw{?hsAf%*5&)~ z2@YZM4Z_Wua39sf;@c1gQ#Rj+AM1dV?{1i4_1#vpRH?;x*R9O=Ume2a8-$xR;XbO= z;@c4Byiflv>xLt$0nT;9vB?v+tQ(G4V8s^dOhBrOK76e=KCIZ{h@*jpvBlanvwIEN zVuC#4YGHJ3e3?^_ZkdzsEOVGTAcBBh=0C{p+6?R-iTQsT+MCCxU5LNb%JHinz=y@| zk&OSy?#XG`ortzz_sE&T?#X0V2NK1a-R;0%0ixPOw6K(ipb8kzVsd2g-gBVd(IzJ0 z`^YPV?~^m}jrl*$cg_Yts$Pv>orMpZ?@xs9)6($Wg0|rMD8|3^cc<8JsUK!$NGcnOeY-7_Eo|4|fQDVhE-?fd+!`>|F$u2@HbYZ-eV1fu5N2JP!EF zR99;`>9YQdrO?bAq5oZ(=*uI@(v%gZ3;oExWYvy7(h;Ey8Ym2=^kKNeJ)v?}1So^E zJF$o^kN%y5i7SdYCO0_nG~+>VAZyNJyf}EU=Io_OH0E{9wC3!2QbPEXV8~t5t~q-i zk+SAIW*LZaD4v$Vq>nk*GHGukq*QfRzdRiumPsFTAFyyJo}0#`*$yUh4A(zOxA_Tq zs|tk0M1~g|%)*l-RN>T2uC+d99AfH%>7 zD46#**pChJe+GL4&QmPMtfueEiSb%`n8?%_%}smFnugo^XH9P0D~szt3^h^`S&n7> z3&*gbHWy*ic>i40ysOA^u9l?(=b&*hcIP(}+OjYi3{4#u)V~?*yhGv&%By1oG{HfI zxZgT1sDC*(Lnj}H_&@Hq^sazf`;V)(0|u*;j|=Kw&dc1%$Ng41`8<7#4@{q~pD!X3 zOE~URn_3xveXgispfnI)dp!65Xqe|^;Q9C+EG%7NQaEDw>4wFEMHZ~b)Bn>jFG$0) zjJM+tm43QF+uS;Y!9($Y+2(HSC)^og9x~feRn=Bq-r9LT;ey!xC*SS$u`gpuw(VH*dI*Q~rs+&Y9sGu=+i z{DffrpAcx6|Ey}(*NfmL(P#B_s-XnJ`WXW8AsW;8Ku68l1W+R-k?4fG9L^}wA7i~B z(X}E`FOi6FwnSe!aFpn^DB9>)4NbK~3-Jl9t?A>mWfw*8obu)plE8BUp9wD!E_H!K z`q>^@nq7f`R79FJ$2*iL&G)gMkmeeZrl&|lI7^x}%>Q$+8=X^{*Q03T497qk*uT}5 z(v8Xy|{8*U2A!3Ri}5m3TmR|<^=dY@jwUl9F#xRlR8oJ z#bzuiXH(|?P3K}pR=n%)Ej{6v<&9Te5Jl=hQhPdnVX~xP)gU91K%5Wah}tp#^#rl0 zyNIgF)h^j+Dz}%Q71Sqj|0`~QISgNjy3A2a@3dpF|y!=F&vYPi23ku+8A2#iEcP~po6?8H2jUfR5fo)My4o#X)d;dk}mM9 ztFJB(J&EIbKawOlk>})d9Fk>V<37Jt=5WZOP=JdJ;gL+SQ@S0BRTiEY(~~fe;Ly6<+xP zc4C$p^;Bp4D+W+kQtDh{=Zo!RMm_Z+hm%90?bFLtvwj9_jE@XK;E1RZkC~Y{XZEbd zS$I%P$lIh~!@2&+LTNTW(T}I)IMmj{BHriC&^{s=E%>pNCpqeJgwN@2sg65-oacu`&d#9J2Ld@R&+k*FSd{WG>9X=X>`#t|4 zIcB?1`3Hrc)Wa*=a6t~l$WCAGa6u#6)jv};*NCpq%tqG+yJcjjzvy76kzGYSVA?e= z>RQJ5>Dd2iNR07U+%Ya;@fqCzVOV@YhOqdINmf|wy+tYmo^7SOCK(Bf&u9Y{hQ$}A z35$iDGfo$FE@UVVm*a%;0iyMS=pv2C-Hs{)LG<$(uc8gz+(q1u@;K3sS03dl%!dz` zZ74P#XaJz{0ARTWz|)o!p@(j@jkFB%{}JmzUrE9L5q&(`iD8HAIDWB(<5@H1Fze`xa1PG46xOZ{H_;VaL z#L&2XHpj~naehSKB&mDbfu8SF$RYr2Jon7m3fw zaF&BUNRSL!BdK8iGxk7nev0*lIM<6feMKC?S>pWaX@^CM^L7+%^b#CR9dTIy=Gxp$ z|7Aw_Zi9P15n)`>X1MI6Fe;+z}AKSm7{=bb3p=>3U6 zoCEO(RCaE#|8Lszw%zlCEykIAnk8=AWt=plv59B6w>)Y(1@uT$Ci3%m|4BlPjG8{+ zU7+<84bkH3dLj=sbQzNoyET!Y7sS6tszq?*>(~bRGA~Z#0Vlt6IN^=@-}}c)AP!EB z*POIf*Di(=Y1?ch&MGfrJ{)i8_xy;1onh1a{=u@+L)dh1cyq}nY$6xrKx91sCWi}# zO%rIeI^O{q&(22My>TJ2qwNc@|6}zf!=|g*M(?kqsUAKugIrLKwwz`z*VD|OWU56Y zHjo+Q0_=b35a99*Gsp#pTQi8sfWAFyC$dE>GJ{;e{U6SHSEQLi0(MycM0TzacGAsy z7kqA0D+2}f?uJnx2{qyQ!Xn}MiVQqoxTA%oTRyS;t*K*w!6H0gNdHfsuS&!7LbQcR zUC8){L+2{hj&IiC(Y#0a(`8|e^xAZ7QuUP;xWu!yqEmxqLJC8A>g@SVdgiJJK2o7G zA*Bx;N_0Sfi6I5=U&(QV8LzuIAi`M==pwFv6CBb3y&FXvT`_^C+92dm+eHmdTS{}J z->900lfm2Gd6@H6uNd>*BQz zP0IDR7>X_Y%SEnYk&AGaTo*s=z_E|skD^VkE!Tb`*B4G(%5}6qMO-9u?o-0c#hg3j zn%@MG@RVAHOSK?ELL$&5jQ_bjg_x90mG-xY63Y*ijOb zT=Ia^mI58)@1~l!V_Qt(yM-4BF^LG&WVR4$gp7!DX_-Tc;`}3qol5^Q5vQw&LpV#E zOBw$r)If1QjG~Qaiz6Q4(o>wa6z5ofn%va@k1#E~IN}lXg}v;YI&fuqgiC*LC{db! z#!w>TUm?G`t3kw0r&iC3%G`-u`;B3Yi_v zsQ@EZ7Vokk|E(GEcYxQfMf z)#ZyjI-TU0%z6u~5VJVy%iZzM{OD^wE`ZmnfTM2trP>c@lAcI(1?GQiK+lExZ)QGR z(}YAz!>d2QPR#UbFxD%W|4FzD#$eY_YR6qPN!$CJwub`#3>;e2c3rO`*nGi3$-QP} zmlZt;xQSn+y|TYUgYo@`{;P8K3evusjkLKwv0Kt!iTKZwb`fT+>vWsI+^qQi%IloA z6=>Vdzd$ZN1Sc05PFkvANv)NewWK$Nlhycy-mg*)b`~kDz!6Ek(YrTJ=yJS<8d71$ zHT#}CkiO{ps*@caXj!+=m+09H5+0n5t_^le*H?Y&V5enWt2wZBEkpZi-v5yp+IP7_ zTO#GFIsTd8yDCGZe07r*DVym2H+6RxKwy&TgEtg^^-;jWNcrkCk#fM!Rl?5II=X)% zZBFZItpAeoaH{Wff%(#Is%hmU}s|OkcgBSg?qmuVMU4 zJ+Di{^WJESzPL6YZRmW~sdjv`4jU{HggAO~pNL{C4ubM6d?2{FNLhlZYgzx4Aapmv zkAKDbA^flm^y@!vc+KWnTc4+`te5$`>@ssKX7yXQ$dPkWZPv&wZOvRdtI908?f_~uyY+<4%kWOa<2c{rlwua3-3P0EhLNQ8|eSZ^K}_`zG19| zrN`d^tzIg<@n^vzJl_ESuX(;94bO|v7AAGWk!VA8-=Ny@4Xbjt<*uu~{dqE<$IF2k zs$9>M5jMo;E|@kCIZjz-b4!)9#Pi(nl|zr7=YM0^RQB%_&r>Fzhj5nXxiR9v(erGI zqK!`Iqp9{hd*Tx^-5AvWBxD-yzaZNgyD0yK@G{{XIZ7iZNA&`GH=rj85#`2v9hNA{ zzhi?S%B>>GAQ6RdwkW~=hhdMRd=f<)ecc3_YEfiHxT&8*gWWTNo=uBA0iG?qD7Y~T zqQp~F9Lyvl!rZjRAw*&RFE$*)+%3Wk7hwoz33Jm!4jhH~G>SI*ZVNQk!nEQO{c;oX ze@V7BlCi9=YrQqEy5?ny7^J3E&FIY|M#+!fg$w!@M4)EG-Gyb_hvi{F37;eK!>zM zrdxJ$=uxJhWBno1O(Ii&k%@4YOt-W;aFpruDBAQO7+>_ji|~d_x480CrtSUbqyz#o zJ-0$X|m7?Wcu0R4Kfv~*Z|0Mv&b|+WFnj;)2%`MS7I!E5k;F)TPE2Pyw#PT zGVQ?GQ?J*sCunBRP_7Ppf;2z&6Ebj;uwt-k?Frr*#6N~LisZ$HLZmxHq@f}b;VhAE z3+jIoB7GS}8(puBrn;M$?nXD=R_myZB8~E2krEwj+`baNo3L>kh}5+A9Fu}$y*nvM zC*5{~LyZ#U#D+nl^&-(Qk%(}%L_z&uLZUxJ(MHcPXlhGzdyxMyZ7I=?{!6NPC`mw~ zm$H?}-M5_zES5yK2kYNNALYjI_$>c+k!XlWL^w;L+c!9x;86N1iZ*=+#utauLcGyO zx4-AKr9?aVFRJEYY>TAgi{T|gQW1TWsN$U>RLB`9-7(pr#Qw>P;T~rHHj!qqNJBVF znmdkl;3&=4QMBouMw&ZraoSRvo&D!kb3~+hK5J>>yS9z4H0ygglqgMpY#^k$MWh)h z(h$y)X8m*rj?#P+MH{{FKvUf}GQF;M`=8Q`=I&`3Nnmy z_7KWd)V5b)?XkI%Ze6->XKJMJXLC$>dcm5%=&4n1^khB@c`mFN3BuR3j*E@stpLP-+^SuxF zc$W4=$_{zjv^=5a4di=X>8p6Ig?Jv6(C{?i!A!TX2pkwwT++`u-@9+Ig4o=cs!Eb92dKS2TvsQ(1bNo#vc8+^*< zYhm99C&VoIjC=m-U}sUs2TbMbr+Oftu{6B<%_g5gF06dUy;%PzT$`eaO=9)ic+vJb z*=W1BAIK;scJI3mcAD7rY@@w07||vsBm6$*{}Lk{Nr5!^jQhC%!Q8+N8FB;n?QZ1; zOxkyYI+#X4GLjp(Zw0V0pK)WF+(5t%>z~NZjpA^;JTi9Sh z)HKgC&i;%AloU#MzW)N@`KAm!-;ecwoafvt0IAko8exKfSOT8k5uR^J!?Q#o_x~Gh z)_~_*R68Ko;V@Vk5{%}4NJJs0>S2IqJ+$J;WaoQyh(cz~&`%QA84HejD++m_-66y% zq)TiB9Lqf(8Y1mGLZT4DSy9LX%>O4CWEApU6m13(j4#^yzIek4KJa^|E#2frIO(Ss zWDTk`m(R7;X^a!!CvRXI z=U}AxoQJvp!$E$F9^`&nKl$)HG@{SBCByiAc$GDNjh|e->XLpC#u~p5Uj{7nleeZB zzX3bg|IzH+D(s~5lMiG6C+;m&8K|hYuJ7|lp(Z}(kpkiQ)(kv9g8Gj*mXf21<^NuF;{!?TnkJwpG_p>wdWA3DV7cM40c!7}gg+NmNHV0&^r0=7_`q!{TQHo+CA7fFVQy-BV=gS0UB~3?}rzK<@j@xYS-t33Mr90 zQajq=j1qN??Er}$5Q%mWi3n#&^qYGeI7;+G6m4`~15I`3>?1Td|GU=qi2sRfT%k_v zlkgI;c71O1yaltG8|Lddjbc>CKIFE`rqeUFTS>pV5tpi7K{uD)< z5d`CleoB@Czg_RNrL8^ce=I9)q;x(GFAh>V5J#8sX=!L`q$1Kh7Oejh=2jfr8PYr= z((EkK1STrp^s(6vhm_{WDB289Bh6z${F^Yh-(V4BYA+zo2U{SGq(qyNr1`?(ihWZO z8wF_|6lq3@G=#JI=J8Q=$!N-pf{+M7nLBE6wA< z{!gNBy2rMMH1~@%+lw@Wv!(gg(E_FUa};fKiW5z>rAcVG(d~aq^BC^CFjqSvH2g5U zGzbl0Y4P0d0uZu7!;QiIPePa;u^l1ILn6$MA`IayVKzRLBFtZ+Xj5i)jrg^VpF3?S z%;T8i&3#YsYafLd#`v{8XV2TcabDo5_W);>UweY{f5PX`*h*tNK_WyNdeYfRBqE$8 z(GzDnaO|GHM$u+lTOvsfJ@J6u)?>=;et(hNt%B6hA{`RdR8}rV*w^{;7u3nt*HbVl(06cN3ka9-4mTA15ma@Nrx=eB@6M#FBFRU6B9Mrhy}j zdFqe;GO1}m%57Piv`>9`MOG;{lC)B8Pjde|-gjOZb%$BJv7O+^(v9wlb3)GrK6;uu&3GL2AUH4Kl!N5EZECefAZI=<~kxoT)b8* zzoVkMrmnoby3CbTfON?|x=4n!I&)kSc zTvFbiVR`n<|5(ejoaZ5^y7%gHx4{R=$nxwN+CP_PccfXK2|LgB5O(g+OUf2EjU|Yl ztrkSd(O3PkWzhQ6l-4(UUZrkb znP@#saNn2G`WEF?;Kzde3Owm6AAhmwKxzJ2y!mqE3(N5>96PO9zu1bJ0@J@QiMKpd zx6FxaiY&%54bji<+5c4ZooVzlT#y_or0%4C^#05HxgKYXuHFQ>(}r@e zc>r>I3djdJ-mV&C*9;)R`X@o)93+f$yo^TnO)UuNpeFf z3~oW`g;s|W1LB^sF_=+)Vj4weNnU%wE0-t@;r@~)mpLTQmv z0kK;X$%{e$+eozt^Y+bb<7^Yky!~R3|2Aza(6+mOF|-_`y1>fCnw7;>71am_JHJeU z^Y$lzvgYlVg8grUp7ZwItXR+*#=LDR7D5S{&Ad%64i}}Mn_mj*Ukn#>5uDuM^P}lnU`MvP7vLnW?tGJM4{nV zO3{XH;eI_YanSvJq^BKB2|kL<)hqZH%?T2KO-TX!#|V#kEZiv(-76y)OTjoU<7_bGIz+N4m0`~8S$2<_JHDv^2{kKc) z|Au+YR1$1{3YhfdtLIxLk#j#?!u_dgv;x*&hdEHbIh$q2Sj3Ya&@{%TmA!w((e# zNU-S{!5II$9nht($Kp(aP0I-OK?+!5ACL8d1e=-yCNanx%>U49-IwgO#Ejlx{O__; z)Z1g_9`Q~|!7G;i2J#>A(5355Xsy2KnB4^*mSw+jUJCB6y*!qw6ZdW@xC54g`9IP7 z-FkY_Mg!KE5$r!H)QU^J=wt&nITP5MH7Q^vJ-lelfW{>i_!Xd%Q=3VD%ZnQvF~5JdcO%309X8EY<%F$n|(51i{8-0(&Rb|DnD$ zT5G^+GlHf1zd@eI`wd8~CIw8E0`H{yzro7W*Ku^u$A<$)7+GMx6P*9MpO?F-^Y9*% z2+QmR<~y&%4T=j)#4lZj;FGUy@D>;M4)Z^BarY;?I0=>BW&V#b*Mn(7C0UQZyNi%| zP=`v4&Tr}B=@`BUAMs1-*gOCU1jdUNCCM$ z&(r02068rk$a`H!(^oc}(oSGJ7dlnILKf6f>QTDNuebSf55A7_C2pj!&m?dX<|46TncK{cd6-BIXauTXRN zVJ4^}QlQpj?0jILKF9>MJ_Tw6hR^#3>itYmpQS+Ei4pXkfqE|k)Q7?TM?!~pVGzA* zpx(^{73_Z}pzg*vf|q4~y^{&*loSr{!BBeJK)sy_>d_Radoh~cGEi?NLrJ9c;g2a$ z_hCT238<2MeAr+F5fA)Pu>OBod*BFusZJc!us!fc!T#sNdQ8#K??-FCED!t<^Pe13 z4`&!tAN?8)IHn#;v+7*}WEj35{T6MMZo_S|HOSfG2d+z2^fy^NvbN;6UkEfA965k&O@z3LYNHZls z5}W^+vP~N6(}!)@*6Crf|n+S%ZriBYQldD@E=}8Cc4?g z@lV}sOrsk~^KZIBh;F2Av~#}OKTGHDRUodF=HK*Oym{dDM}qfs;{F9KDelH(aZd*w zi2F%jL8Zmjl<&b+*hb!^dH6Fw0uMc+ro5xBy0Y`@Ya-|&eH%*zn7t+q7iO1GH|K($ zJT(blLH|Fd+VRbyz<4GOl`;>i>b=M>yZEKW(D*cn{~uF_2#vEygL>R8qW3P((*K_H zOKzGWF^*Fx`_mI0Wiy5P84G9hQR8Y9&g5jFYbH_IQN%5#_UWg~jrcuoY-}uJ>$4#KeIg!^j)jv+V{D(b05y&6iDY9t4(K5NXRiFD zJuJEqzj1K2_q0Crsz{77`)Y&7QH|7UatF*k2yWM+TE(3h(nNvO3d)URo+#nkrMXf2hU zVv43IiQc~C_)nNL_0}g=0lhsgdaDTZMnD3cZrJ?8AP10o+cvr%`kin~Rdl~>^p^HM zNTa^IFzO>u^!YHlKLquEi7{G@x>>Eh>1u~;)R&Q~VhXAWiWa{L;=jar?Hj9v7B`9( zD+4VOkU%FGZoi`cOQ@e(93F)s=%V6Q72Q7@En=sUxAyR~tI-|i;ei;g15}qi8s;UB z*fqh3B`X{hP~KO6a@e8m)u0koUwrMWQQ0UjiJ4+brOAlOzGnQFker6MO{@wkdqPxJ z6{w7Wmdd`q$N{9vMns$C00>mpyag)b{ToJQ1=*_Xn_&MVVQ;ml(A0B+uXfx5l?BWc zQz~0!-~7svoGR-VtA@&cCn~EBR7OCmOeXnnnE&M@|GUhS{5RhU^55x6o(q!){7*gK zjVQ~2R3`av2gd;zQAY5t!StDfPrj@G`rF;ml9T-Jk|%kt7h!ha9w4ZU4oUk#|5N-{ zi1%;v@MmEbkzwf2li2YZxF(Bv+f!u`GnTvQw9x*=DwTb2wH(D z{6v3&6m=t-TaXhPG0nBNRuXX^g*O!W-CYiT?nB?hi)r+o4*T(9nk+;Oxf7jWgiMH_ zf2?;H;a2pGyp3~L4{admVk2m{C4w#J5+mpzx&Bo;664zTe1$$S#8=}J9b1YI{o;#3hTkE9H%#tgZdXrlVH0*YO2ZIQVbCUy zrTg-gZS5=D+hWU?9(Kqw(-QoVM|gpjj{5q#%Eg^O9l$7jE;z*~`H}S>iOIbJQDG1N z63kRlXr1H#5;Ok8U^e~*J%%&jiU}4?fa5F&yJhoj4&_^jL-$&#eF zwVz5IvUF=V`M7Igf{4{>#0u}$h@ZH{as0Fb_~|&FO5->pXb0W?6vRJI>4}JWgETPjyjV8hauZr@_#vAJVv$Ot#SIALc`*=Xc z_UzIsJ|H6tp>V8$`XK5KgZ?j3+hRN()A-)U@myV|Hyy`ww}PPxRZDRwG2XC z#^F&bLbYdOrD<*;0)3ufhJ$ z(=<{Yb8ZWWG)DT@3xStL`gF399tw2m(7)d2$WFqySK}ud=^lC2Dx3Zc?fi5yUzGp1 zp#DKmnO*}t?gKu`@bsZlSX4_k93VwY=83u|-jMHa9RF0nW627TY4LCOThpSt3Q+2d zho(WTdft>S{2S{Z*oDue=|Tyz{>J@323gPOvbk$HcO-%={8@^(c3oR}1L5ySSQ7Lq zCY23K&+35>iYOBN{YsJG*)$S}5&nH6+Hjh9RtH(v0vchHelcX@i!FoJ^nZ41h?_ac z`bV(-d6p(S@LJ&E9G%8w{}HTzo=xtt2>?Qu{v%lbJfk-dt|cO}Nt4~yY#{3;?JPty}?(FLFL#JD*uged5J;|aobh2hj7KVAp~ z``?BDQz0Xxi(s@Ir;Al|QBf9M&4djeM$L=g%tYyB$h;AGJUdQK_QM;7_rIg#{mPJO z9khBn7q|MXHZCt%fT<1+HJr3b+!RwRO=y%5yWAnhTuA>|4Rre?*+YiBCZ~*I7(fD@ zD1fg!fYjM`(H5y1?@-Z}Y;?v1YvLK}Ms4WR5J@+3v&eAt?{g^1F578TL_1q&H zy`^$gOzAW&QQg1abm&oa17fvM-BY5v+CX&#B+v%6jLtS-u~0ok)28# z7#jzrJx%stZ{q@`5s*qd32%&w|HK?XDs6{otK6V+fQq(eqcj|_wjX+E3@)z|#pV1Q zD30v32up7{EKzZ1`Fn}t8kN6SHi}EbQZePz#Khu$p5hRs@&?7~puGPP<<;esnZZgx z0YX^vGm@l!TH`}W~j4>!Nzyb?b2iKkHaDMHa<7G;{x_l3Uj;2oO6!$)I0oUGP7 zv>6_1S}UvDtL-bds@izgman$f`nIaJI!-j9d3ZF5hrg!q@bPHR=kf}A=rBA~)m68) z*R)%y-(SYVMwf@2qf&U-w6o_QWOz78c&KW_LtQ#QWSos2SW(4(kiN+IzZ4!m*~#-6 zvq2B-nuqqqRjqaHHMXh$GClm(8W4F!+kCf zd9458vLG=oJ{#pBo2_|RYmU_VAR;LqXpZ9{w=G^A9vU94I`LFRrhwvR7BXOb;(6@et%c6ZY`c zwjS;k)_Q2xJS?uR?x?G4-+Fqu%jE&3oO0IX3{(yta%9ah}&#@6gSf>AJt`D z3J>2zJb#JdVTs|Pu6=P^WtDwO_m}m>OTt4>d}7`#90?wnkokS`^4{#ql@xOQPX}UV zcTTL|@~W0?8U0{aZdo?*R!C(fqJYG%TuI@}4kNr%)%VYnM`>X_dY*|g!mZGPC}R$U zpuVArqOriQw4mITyIUcVkw_RJtcu4S%#>`sUAdB?TdahU*_E5M^yCr{Pb7>=ie3bF zjDeoZm@q0S`cX9WoK6@)f=&`fC0&OlwT8GE6GkOnXU1DIVMN?iLk?-+mDn&Z>5BZ9 zwy5WlEsDEeP;}SJ1eLZZpaLQ$jHvFdxhtpT2Tt;s1MG=ixohzaspV|MKlZL%Nw;dN zFN$=J-}B1jv=H=#?8=pN+uIgEF|dEc{{pfBCwJw(_^$@`cciyfg7*Q?3JkF3oNZ^`4biBiY2JTuI4|4q1A&2LF87!GT!kXY17hv7Q7S zsI7Yt{~7nqU9V3kp?@gVEoxF1&3#*2>*ChR)=ux_)D6Qq*9>Yx?sc~;l!fMv>?!l2 zIJ5DlNblGxgCx1?0Dq=xuAxlWqL^t^7EUPa4?!K)rdLUiaSnQJdi~(9ka-51UMseO zAqZ8j%0#Fh>m90mBEW+_HoXF&WYa5L2UtRJhaNgB4emd92IYm&xvHVJ-WE%JUMepObI4Jz=+@y>h1Q@! zHl!PRpY5>H0`_WAFRJ(HD^*o?eOy2xh**3_@2?%?xFJVf$ou^CF$*}vHiJGK>o{pb z^hDhgZ>X`)RMGP@$$Dmd$_Ci~o`yfoef?MAFE!+-?daUB0JhJsfQ7d5QW{&45WLSl zXu}ZvCB0|6k#R&K1jnDHc>fpnY)ks?WJ%CJVwB*QmIl0Fs z00?Yu6V$)Ir1xw$5)s+t$Vj_qyAeO6?Acy{FNrM3|7v6y*ty<)c-VzGgm8lL&oOSd zvZJyCl{KASSsH;C=+AqcW;8b8d5F71Xpqw%+z>2=e0No67*Hq3*ba`jPZ(mV`rsHoprak_`#8`8aubm`R=a^B5_ z!VKJ2W`Ybz1l>QV|1$lw7D3<_yp&nr^Mp|I1LG#b^^;-Q;5IvKCH-0dWt7VRmzzbS zqHS_FB6ir?inIp}ho5(HZ^4c(Sz%rgI`99p!_GSF5p3}H7O#k_f%ne+1?&`4FzYBv z$$&!~YTP9n9IMAB$TM_d*kY^CDPu<)z)|?_B)q}qfSVjZYHi0Tj8k7-iY>PGY_!HZ zjJI%4m+Z6{mIl^4EKzfJ`U^#K*kW6ljpov@R809aG3o(27#N&?GL}i@4T+71@}3pt zjSrMZKmx51z@Yxk08)87MKST{YrWc3v?Ckk1typ6t}b~In;Wz$MRRxg`^e)+VRQRr zqq$^$iYb|GbAwp_oEUIa+tAo9P}}cCZMy_&BOujwGTzYl;DHVx)wXjKV^$xPIZ#Cp z$VP2=a3yUn{!m1-jL4%aPj`9;tR?n!$G1VPp5pLhV_?Y~f|$ zf(n5guYnu0fU!4rJI1wFVE*m=DSbvpnJ|VnI11u@!9V#YCJKg6%+aibK}@6QO&#gU zh*@2j;m{!eZS>fHS@B+O9BVT~YWdAhI4BIvLk9YI+ceKP}S{e&URj1oYH zo4IU3C4wGyCh#*Jdnq{{V|WQo47=TmKTXx+KWFv63_n`&=P<^9j6YvZ6MrrdM1MgW zrQ7@jTY2UGSp*%y{5K6e=rusWG5AY$tLyp?d{`1hE)WS`OCy0q(2-#M_nMBNAwg#m zbmVhjqzepMV+dxBpv!RnMOrh0ChiOoblI+G$sT(xxyN|x5_+s`KSA}Xj-X$$Bj~}V z>anj=^=ODSQ5>S+=^UcD!A7GNA$GnW&mzPJeqV`a5$5Y5R#9J#x`9@E7pSAX3LE%* zEv{{^sH!ZFz+-mP7ojq2W*yv?h0Ek@AH&_YLH+lusvX}jZ2fgl+<=O|mGAWg{|?2h zBIHwngJcfG8;G_I@?Wn?rDKkFIZ5T<=b7{#xljxLJyrd3x}-VB&>hbGFB*{UN5!D{ z%lOkUF!Vn(J}z7b7%rWlc=#y}eonQcqoCiN9gLv(z@jWV7!koNb}oC_OtW5wGDjd! zlsQ^K@rXVSJL|CG`^-OF?r2AVa=3B1;lxd5rRy=e?uP?=%3`woKAU%#5{uVseazQmfJ)bW5; z>boPq#=qA5w0$M8aE5&&&HP03grnMi4^inGamSz21@j0kx;w6Kj_4!`{s%LHv1ws( z`*VfGH!`t!m(8M1qy4+=^{0WwW|8%;Wbv&uEG|J?viMK5VG!_^PNQuytkC3Ge1KOe z|6vYy>mGe5*qXIHzI%BMy?jYH$32Qut4ZEgd2;aqLg|0v zlnxO}SCSF~gBd0;I9CS-r~UE=CX=LWZR|7wWI4{>(N?NQ41P7g&%`HsXr~ro?Hyfn z%Glu#exrwHo}l~jp>Hhjs0vZ&$iRVXVIrO($RA5V{?)D??q%1wU)Q)p6(|w+DS}sU z?;NcE6O4boi|4;+aKC79hbcTFZblcx8{j@Xh3{`_Jv>@e^ZlyEU5}c_^776PS?WGL zDGB#$DY(By;TTIlq32-=n`w!Tr5wt|bKDTkvop@-0)=7{>o}fbS$bK#5<*v^w9dhR5jN&Y2S20dF;UENI%SPS)si9|s)fUk z<6u)~=trqHBYxH^q&$%E!i3P|rxkDTGGi2qFgP=8~isKeI0LVpKsq5g2)SX_VT z;>}+XW$CY+^M8EO#7V1eIb#`<>gQ@ee-e{st-l0M#T3!jUqx4kB=xs#Y!dYMqUdi@ zpg#gqe+&@O*A+1bkoqf+qDEBniZf~lXQMxy(t-V1XVegUr_~&pN^?f7q=NMiuI9@5 zZ0cSFz!|l-O*9`)b4lD3Q!KR~-ePl=!TCR9I@HS_qoML+_{u31&E|C1P^mC-|#xquZ3fmy0GQNZrc0GEpy2j}}YOIn^y+JL7wy}4!(O42Y#gxmovFhs_VzjZ5 zu_%=GiYP4_D2;$r+DUkm%l92X+E`U|8GKbN|DvE1#U=JH$WDu4 zsb-nO5*1g)yDqijuxa*QHi}EbQZePz#KhuiK6VIFdE3QeP~NMeyjY+-0upGg;k35D z14!jnN0AThOE_E_xI7!>VW%aSRPn^&rbaVRyKMfdMKZHk7`oMAh^njRQDdz-Z2i2S zjq1`dR7~;gt)Fp0{X=53ZXcTr-MuEdn;htlfYjX@ykU6bnjBWCyPD_HZ2}c` z&7eZE77t6vPpjc+d{FS z=&j@OX&|NW`Ox<)mNzap#gxjrW42`1?vCY9Wjn>DLS=7>%BBV?BcP?SU4!$_36<4H zkx0}`LS?J7S6OgLB-Sq3s%%&0KND_*Dy#Q5Nem2?ZOUF{0XM~z%2wHgVE@}lPL=H( zn+BD=Eh?K9sEmMAnG^_2IND*7+ZVs}zd}-yaM-^1s!;c9=mLRo5q%+$<27(o7Di)l zUyN%T*Mdxhn2DSr-3`> zLf_4tvG;@Z&$~Kf4+%QU*!#i!7qo`BnKSnO??wCX>Wn>cQw=$!ftPhc!2dH^GGX;@ za>9zc+E8?ZR$DWZ-lIf-w3JY*CctQfoIPWr_VOwd-9&kKTIV6l}+yM zv3jH12mE!rX+HTw1Mehh^tNcjcqKZQ$NaC&9WXdO=zslRbuK?fkReaL6SBU%Jc8tR zL_I2ZLKZ>fbO*qwiZGWvroT_pTo+jpzxXlv`bge4X?`Ug_x8!uYa^5Hqr?cN1n?3q zRWwy>{9NaF`}cl0q`5<5r+$a|SL3t~b1`<~t`s8G3HPi_`m3=~MzZowm5X9AMR zS{P33Td|1*@?Mk7W1)ciG?UP8udK(t51pUtrf<4g@Cxq9!Tg`V-Ft$kGqu2duEE__ zUxyRpo!_KD+%G2KzCMNTKG@{oUxcTdmv`Xt;Brr_>Z zf|lW+(5pG02{8S42L8Qd2c+~u{^<>l=*?({nO$i+Q%mlVE-4l>V-1K+1=+;wdj z^wrMAy6O48DG7Iw|4;Bete=N7uUaSP8{GA{wYRc*^SD(*V%x&*A2|4V zNcm6xp|bLVEgZU4Y{8JVY+=THhb&JiKjh<(e6Zi!k)16h#99IIU<)(ub@220+dFCy z`ci8PR82NY3mwQQ(w=ST&#ZFjHz16B(8taHy4!G)a&a~s;0!Z(*QmvX%)9{T=|bL0 zb|HO*KIET60NHEk$Fe|8k`H%}rO3>MYau)0V=o^F9XrFmO zIXkF-`H%}rjXT4Fa`t54WsiNB++&O*VX?E@h42UTnLuSND7WHeCPsdc%+YWAGLudC zhT6&5h<}{ROy+o2U*zt>Ur#RD?m&E?FT56fLV!8j*#amoCm;8}LDB&@J?tQSQ<#N6 z#Gni3ev#7PYmgT~YJ+7NdRx4C+N{Zqdqwm@ESrl=;D-M@j~Ae3)2(B{wg*)~+?sLq z_{aw!ZOO1sbOvq|7eRr(yT!30r%cK1`6kV>Kh*u2C)ef&{=w9Gi!qmo%{EA zhqmLJ20WnQP!!B@_~C|Pl;))RNC;;rZVz|XP&LGswAE+PAT)c+-U z>*IfVcLaW91M$}LPX!iw>yOj;I`P)?Z$le;>yNd!E~VKSR_Lwq!Z!xtRxa3W8g5BY za5Sm>{Nzu)@L@^N#Pk1@U{e|i#9KElMH~9xP1;*)ewnwzE6ut0Ci;K7HN?&At(%^W zx2Cry?hM|#=_j;gk8MisF*&ufptqp~=2 zU~%qMrPzShY%|)HMLo$o;pX;A4bi{P`2*&@%@pC&@lNQJSERO$ND2D%@uC@YU z5}kuL`e2Wb;zDxtuf?SCCm$O@`auP7qnmQF=mW#1!F`;;zh_YYq)kPiW*;9tOrixm zjPkIeEbwD*Zq|%!kL{2V>Crud{D)y?ohtIS#Y5kl^Js9s@=wd%VrGgdl%_KZpHS>x z;~d3u*K2faI(EI@Au$viPR}W0#~MHa$*$L4!TQ%Q$w*?O|E*jWja{#A3pJBrW|}Od z2_lKoEMV+iul9;OC*8`&s5Ug!;#>mz4R1^DZH$_5>r=uGhj~|I>8nTy*;5ya`M1{6f$b*$uJY zGzKlGOt1^rK~TEukCWXsXCG){A@={K;ZL(1Iv#(i&vO2B8a}M)Y2hD$h11igX{IN! z>3zD1N1i2?Qv#{U;O27m;e zt=RUxTDl}?jlRen1MK@kyftG0;?58Q?E5#gq)~m5++$M!1V!)HPf&fXZ6RXE06UvQ zJ0tBwJDOM#_V3K|N_b#rf&M?BBT6QUiFTQdRxLtI@DP)lde8p|T;ykBg39W;4#WhT zImz2gmGT;zhFH_I*|VDGE@)iP7y*$jIkZFe78w@z?-^$?hzqXiib({&sz$GCu1Z5T ztJvTSAX!8{WioUdbpGt^chPBIc7?EjxQ@dNNJVPz7hPEsJ$XLi|__VEkhlSsI+_i{Y7b$6|N=M0Yx~arE*kE=`<~Bf%Jm? zU(-oc-PqU+sO~*c-HbqW1SF6Yh_^iBFiO=;h#m~JQL2;XXa0!qn z^hQ?p!y9re4$i+OjO{+ZSvBX99UMYv&PH!Z>=aWjO-Pis_(?~0Dy=+*t3LhrMQJkw zr4f)oC*e&lng8Sx$Z$!eO^hP68Ado1h#!@W(y&~D(vk|q^D-G6D5NbAZ{_*l1WPoy z`~5|tI3(H@Wuv$>EEQ8e)q=MuuMJ+$B1q*`#AZQxABgg11tQ;6MWMoUeUExoZAEo;dwZ~bgrI-)xO#4knj19QaD_%4Hq?TGPWY7Kh8Pf# z#9oF>`yA0{g3N>eA9L>kCqfX4XIKO!gNTSO8v#W!A_l;$ zCkD(}Fz0yYggIl*IiH@V;u%jpf6r6(R(JLE%rfiw{eS*_pSf9?H@|Pa_14?f)p(sl z48P(Pma+8N$YT+FS(W0$&MbKtKBoQ;_}qn8HjL7IMzxv`2fO04oA8iDWpjQ+pS$r| zh8jS{@RdFv1S2}Q{`gS5Mt}16N&k?=u z9q4+a0Tstr`n(b!B~Z*pf(#k$j}KyRj_H2{irt6Z1J(F-6u#1Drn|Db38bE`Q{09M zCo$ukmm{j(zr9P5;_k>l#Y;a4sd!a^4EZpHK(o0cBfK8i&h^IN*WK}zKDT7M;uS+0 ziOgge&%HUq@WE|eZ%;sA81I=rLp_YStyi((z#)sh@PS_QO#L5tY-54zaj;{1P&|D$ z=6Nv1WVkSpu$XsvM3;y1U2LROVKFv+CXxN{iW_qdIZ4Kt_hp3J!`QCDUy+$+C~n0K zjTMDW754F8(d`<@OcOBGu^&l3u@1sRJaoX62%kqfyWVs_!9&wEpVGP}yuhU8tzh9q z8*@RxO0);u9bxlmC)b;x)tQh!o5V!>i>M=JJFpxT%hC-Pixs^&GdwYHbX(Ws zmPl|L_ouj7srDz~B&qfw-v41~7^I|U+PEHXJ0R6gOP^cHRQm@=S*dnG?+C+Zb6t;P z5CeJd^cjXzZ7c%>NJLWYf}{yiM7zM-kwv zz0zlrG}SWGs-TdbWe?hpjc?mzX;Yx&wF*!s>k}MF0&Qu%7o%M zk5#y0BqjdBk41OQj3s_?e4pWrua>t z+!of8!9twk)QL!D|7Q?rFM175Z7xN^da|K4c1t2s9`#eq#CS{0`X?}6X?!w__r4f! zvN0YJiL?qI7;lN0|1=_v7mFR?INK2qPmmqa8sqsxV#+ifZEaF?%ShaRT#=H<09|V8 zzrb!Uc{uD--Als&ZETI*5~RtaT8fd_Z0RwfxZwaTi%)^gJ`kHtF*YM2k=Eb?n=QRR zgh-pkW6)xI;^9H#RjsiZPZFM7J7up4wbfI#(SV)2T}JY23V87kd?d>*8y}K~qx5AD z?GqC9!jOmPykExR|wjb%Qr&H~f?&skP5q+>3GYGP;o;y6b2P6UoROY+T6GVS zf=~!i^kDXXg2cjp#Q@NERHN`ac(BNE7-3}2tv7IN zXMCxP_nZF0C+}9^{?}L#R`)ht-lA}Z4xY+Fq6a?G{aV5PC%a#tr@rmB!qh*XtJ`kS zprzYxhx8D88fd7S`nKC4aUYtu-AJ2e$RH0~z3`#L2$)0o{2%M0&lBsSwE__8e}fbV zaqA7u{a0?3RQiJ^Wu&bBb9cS-)xRX$D7kWH%NN;q0nT~zrwez%7kuG1;TsyRUZ(hw}1Vgrk0|7#6;w|yOP3u+}Fx?TC-V_MoD<~jsKMRFkL#2=|vx^ zOSM@NG9n8UmK2xZ31hnlktEM4gUj$g^wE&fnddI%UIFl6~~)XEmeN zW1`ltA^Lo#;#2k{GdbGuOhq-SO~~I+){OTLH`*^*jrcD~T6sp~kN+A{|=vYWkDLzvHgV?cfL}7?NpQ+gFEfM;Ng(ZJU zENHe?EF2M9|F{LnT<0+s%tp3W#zLUhBB+NDUzHnDi_cWtP>w7}tL^2Kx5j8`4`KV6 zidDyl_*Wy>+{JZ@Dpl}IMMG;Oz%vz)-F~KG70*B9#Pdnw#KUmQa1U#J~Ao}RlnqFE~7>dgP+%>5Vd(p~+Or9sYm za=CNOV|U}r*5GT=;L9W$^aCtRv8F3vIE8(wN(B%9ipgJHykF0YkOtaIfJVfmE)~`+ z@S#~Mc%Z{+B}u8U=3GG1V_znEtO7_B)AWD8V9o?75TckYvy<-9_3#5{dR!!N0X`se zq`Cj8$T+i+{qE)?qiTVS_fnmnt+G;iMR@}@05)6CyCzA3crTUOrJJ}dN3Gxg+|C?| zIafrzPK@`3!)k*&Wj1ueo0;%8-b>XzjD=>(CUpR~xDHrIww z^5`}p`dmtnimi5>0nC7CKdf%sia5Z6-`pY5D`_`H`vJ6*wjYi@J0uOa@@|LC(SG~k z1k5tMvm`yw%^bB=-~j>Gydn1(qtouZ{r4#bx= zi5+(vq~tXGRgy`J;YmWTizZ+BvqO4&d5`Z5M9w>3r4aG>B1^=s`%uJ(d;i_TmnGuy z-2X%oHzyHs2w)-N@rR0to0-n>B%SNr>|v`y|DX z)i{6Jyn~u3A7g_Q2MN`6A*y`wyHB;Nlb>J|A``Ko(@l4d5siI~(6*Uqx1^f?*6-xz>AekB=wZta{4RYz=Sunf``<dq@+kpJ1lUREgg_VDe0qkBl%<3elMr1@a?)Ei zi8ci`i7kq{*4+P4$th=cGEs%k+L+BJXL5RccF=mRU2P+)EOu*<#o958aS5;7O%#K6 zM+l8>Xc3JbgjLu6%SKbV@2o^eivv#XX@j<{1BZINsk4MJ(bK64qO?CINnTM0oV;&{ z#6vcTu1-~uVf^%N#pKCc|8Nzi9ME-k`p39QGF6!ouLV$64?^<6x06=M4bVUWW(nav;Nbd1>jR`ri|3dT>|YCv;NV@ zN2Z57UiqfODgOv*xK5DOdE>>Ltok{98Wt{CHg5r@sNJfU&Z`^JuyFZ;WlL7>Hg-Yd zygEN2D`zpYlkoPM_^CA^;u}DGwwVsvVmeSJCl28y7EfIn!r^RCoj6rV&Oh~X8&nOF z+qj#02tABSp*F!kJ%lIfXGCoN>qjYAAgACGV|_95PLM0YS4A&`4<&sO4PlJ<$RuPoMKzlY$x-e zrE@6jHozsm$hwaEKYVsdR-Z;!r#L}YB3rVqJKaWBvW^2tvf;GO-2YXwX1@r1cJ1TT zs&l5hI(-uObV}g!ADd71M%ZKsL2R3dKBx2km##|%oe}5{6%=|}Bj^nC{>ROZ8A0b*Bgn6sID+H~w15a4K{fcm zif5SqhYrhI;DY_iBn9m3lm%f~(o)q#y+}~}ddBa*Ynq_fhc?l!I7!D627V>76{lwo zwUJfTv=>2=#Lt;cHmItaeBrmu=h6@|$8Lm=q?t3H3xO`S_30Tll0RuvWNRioi|ao> z)K%4#zNlx7w~=#ZClggnf7T%((8;!_sZFv2gheo%b-RtMGWzX7Mn~0DucsX9Mihhg zRS2yMc(#Z}bZ|R+I~y&>gI4DzI$D;5XHT#}bLjv-A?@Py38J)ri%UXg5Nk%VX`{s3>!IT4J3|CR+Zod*PQb}2$T{}94t4Z`Z|MciLvQT z3L47GIfpbv-g86sUmMEn)9LCAPLMZY!a6rp|0#J>O<3oK>c1TF@#7=4C#A|dH&p+r ziNvn}+F~M6RTf=Kc)3e(?)$cZa+%@CP?ddsbU^0R&2L(=Y^k5?luJm4B*Z@F4GfXl zoSxhgf#)^Y$T|7&=WdA+$jT4?Jns~Mvir0-Mfd5vo2>4W`vkC@GroV|7<^gXr}KFJ zKe|r?MkVP!^%71$YoP{=a{Egg=sYawDF-h?@g@~yI)7WCR->pM&wc=#3wQbBlFE?c~Z zM7+^Sl-&yO6z@}Cyu~6Ooy&_5!t;Xm0yWw<$@!SAyDGTQK9&p0MEsgW@kPH2IR2^M z=tKZ9>IEwRKo^ZkVpO^kQeSX^exO9CQx82mr4spp7E*7-e~`6ZdgluF!Ja zzG8TS<_oy=f54ZHBKS;4HF9#Z``C^Wn_Sr6Cr>|20+mERh>b5C>qFAUq`WVnc@aVk zbKycj(J*6D80Ny$06@d+k;E|Dfe;LH;opVT9!iPMU=`ER)(+98`e&*DB)HMzb039z zFo;v=ago&j5P*7+cpT8YEIlr&5Iy!tp~potM2~Sv^e6x^=yB2EBGfqQk?Vlfc-Lii zZR2jE>p}cY;B5urh6ut0$K~|VHoK?eV-Q;OzDV-lfN>m0V3AVqi`)9Np|=udBEfN4 zdSA@`7yUObh29rW0s#FtA&K5{&-mhH!fFEb&fW;U3mvx;IAv#da@=hEK}b>IVyXW? z5h?&WwF=MrbfJQxTLfrcwhG^g3KLSOa0&N+Q-w)MRFGkJ3H^@=9x!y>dZ;knRpThf z?Gqh#mo!Jf_jX*)54qXBlMK5{&i3Kyw)Fr^5w4e*@jr>-ssxz{TIcnl+Lrapz{W)e?gKaB6y2BZ&)QdPg6S2POb9>S^t4D zbtdH`I&at{I!{ZX^9JUB>O4J(&I7?7Y3WkNKOHe$DdB`f7Brt;z%uxfd7xx8HjVGEPX<#PWYV$gYOphXJ3 zFE{ybrg9#InMkE^S$bc-R`i~kLhs8t|55K*N%WTcqL;rYYRpo4=Y!P-4i#nUlrowC|DPqI ztUIp2{%zpRcXp^z-7}VFPRj)(q>hw)mmvs@u!T3fPU-5PbnH|&WHgZgHH>XVo zPcm56v=l9G)x#tf)Ht^e+Y0hg17z_HYY>kb48ez5nE2OPaPSQ7C!1Ni zJN+^&CGr8EK@Ev2!{?e_z7aU~IDBFpK7hb`S5pojh76cr!}^~dUd4m16n5MUzJ9_E zJi8mJ z_&<@|YMg3{hKieu-}+?bI&NUC3~r8ODDYy)5?`dfcA`joVhU-mU1Uk?K2H)i{&Uu4 z__CzE_AHThZ4zmZ0W77x8!$BRTE=Lv4;r1REWR>1J(#4_phD_H<|5s#U zn+P)1x-?RMCpR;xAg*FcJClmU?scaBp|n#chWnJdJGoZuLJvqzbfWHu4+VM6Hx5Kp z#?h{lzm>Tr_y@yn~wtFP=Ym-on8PmoyBXxTIlf(~?FcAcssOI&NV7 z$Jp15x**Q>H}{*n<9>4xXJzzq28AH1(dB#5<(wqCAW{E)7(*M@lDb~Ij2Pc>@9}Z2rM~Hg6MIcC`V}$TmBBL^! z+Ii%gZIDx8yQf9E%ATYt?zqX+|B9}ISivkAEVTSmkE3{@0iGgH~+maWAu7bM!HTC={T(zc6YJ(*<~?J_W{my>O& zSHR7-*wv7aC#*8(Em+3?)E9}B;{#o8VgB>SbS}sAj1qZoPj9Eh3eJ3=9IQs3q##DW zh3j9RirRVLQ|a!MM2t>!qK?7`LfqQXMps%*L7}fEu$`RV2}9QqQMrg%c zp(nRsZmkc|q+DGTh z3Wn2#sl(~EJdyN5hEtA*aJsmq>E$-F{=HC5FCMt17-qL|{mWr?VL~{$I|2!YzwLP2 zNQ(6(GIh8ekn`FG;e?dVAmfEdC~}6RGl`5{uxRU1G=KNpR?nCqq zHXR)8@||L*FddO6E?;KJY*a2oG}2}g6@Py)WVgwPinLNMX9k%4@Bg*Y)s6H!_}H1+ z$lF6dc4j!k*fNz8SqR)dGDLvEoss|?XAzLNynPJ_FfK1n5|@(vZ@*laT&&`9HsTU< zaXzF-mi%vj$%kHuD4q?d6j6EmZ-8V}UYrn>>eevKb4OnxrN$gc;ck&BX!=HGV##q? zlnWP^`I6i2ko_N0uPnB`)H`wFRrtQCtlW0Tb+$BWYerhW7E9mpCD3W@+Z=a6-E7yQKVI{@3l z+lPjI*0eV-x-0xIfOC>8jm4ZMidFWITi*E&&eR%TJ)wGcyx3hG_gYW@A{0o5*IoT= zljjt{aV^f_6qu+6eLovL_eG#{ADn7mb;mxqKmk2A3_2G=@Wx#ihR}KQp4_{+__@E+ zSIr#0Dn>y)Oi77Uj3B)K;ZKXXo72@pLg(%QKxgP&nmTmu9xE&^W$5H`tdzABI(M)1 z;TJ0Sl%aF?Wq@SpT$&g&}R5(}eMqP?q#V56!-HC%^TW3hv=V8L|0_#2! z5-;4BW6P=b|Mr_?yIrUm70FpI4Dv!qM;%?ks>-$c+wmubT10g zrH+%a?{sH^|#BjXK@yr1z;PoI|PX|^y3 zfb}oKTEB&2J^peItbf;R){tnu{~%Cdv|gDcS|w!eUnhcI$u3+k_O-H9d#5C5-3%C@ zGNW2dS3#1@fqXgH#RN~4MN<7fPR^odqQT*kgYw=|@Y91&S zC9Y1Qgw*^GaQ_RfeYMjBB{oBeOw$PKgGQK2O(btO5>b|NNk$TK`{M!Qe?>-h!F)1Z z(FOC#&;<&ah;$u1@&NDu1~Sl)*`V(-cz6*^4D(>N80P8}hIw$HWf+%v)meS$RCIW3 z!#udB80K1K7_O4_Mepx~4}^Mfp$K&?Z(u0z=D14o&JfObBAqEnM#QT{#A_2pl-PRk zDdGM|61SjgR@}|{_YuFtt7EHgxWk*t|MZQU0b~*oNjB2|jG{*pC6S@9ajI>sH-YKA z^xDq6MP^0O#zREh$CHSA3SeQZjb{FT+;8#bGP>zO?ANneyw>n_)zAP0F1y96a*bYz z6y$??wAuS8{(Vc$UP>m_$wNb-0pm+c#~9Bt9sd4K8!I`5r2}gQkX4;Lbc{_&g}G0u zJ8lq!G!k^Cwf*pcnI3w?M>iRxjbc#)Dyf>in}Q6o3C-R%$(y~8Mw-137lIkR|9A@T zKRns;e!Hh3qI2iY6Ie=E-hY_)znBQ0N#cF(Yk*o0pD$`X<9Ac{#eztAe>bO}W#96G zRyR}0LuQ6YGD3X1z>l?+PHi!xth-ZbS-+fdlfgw9QODo|F(0Y6(R0^hiuc7F6c#fd zbwta}O42lR+5cjADfLye(idxxQ)#Q;B3&gfX&QBT|IeqZnw37?7Swe|X>}i^|9$iv z59+QLueduo{n`ex3Tw3i4zK+Gy-#tRbIkuoeF{AHmHIKTzovDLoZa&Dc2KEoD4`N*G-AZ z6^M$l+5ez)Q;chP+nNJyTEdNagsN^5h^4mhfMYOP?STurWG#XE=tN+&lLG9iJ)j)}=fxy}ldn~HYJphbMHM)O7$}u3?fZD@VjmuEP}|^B zEJdGs5|9j?7ZXEAngmaMA*8f+WL;%KrxKxKO(?KV>Lq*&O@gP#g;-SE{vE6wWK9UE zIbcXrFsn)M^c5k(x7dVRiaYVl)1LsHo_RTmXJkk}(?(dltUNOb=9$&crQ4DX>1WD) zX!?agl)^91%myU=@^YeIq>Oy#NFk;DLdN@1CYq{$R=)(lV51A4P1e&8AXFJC>GPRq zLX_%YrN&`SZ5c;rZ)aH8!70<{v&?_~1mE@{n@V?QSY(V-Rt_Lh=h;aibgHv|f2rE~ zJN?5#;KX+IaJ5#4a*~Ru^K7X8Q%~SG4R>nGwK~JYl&FrYW&`<_X3pDcEs^Q>~$Sr07d(z_qN8edkq`&<*KFaTdo5`ePq zf6nxOU-j4hZB-Rt5>)Zly1!>XuKQ8nN7ntDBJ2L=82?n_)f7r>!umI8UVKj_EXoL1U5|o41?!&r%BfR*=9}D2Wuj~A`F)9d}>SSNfNT|-^BetJ{h$j z^2v1NjwzEz=p%(}t@}5b`CrTM0X*Ig@Gj$qm$mNS^t%}5^%RDA9{q3MFp81WuE*E` zpkjIV`M4P7O=Xy*>;C8Gh(>Sv>;CqxBG?(4`A&?k`=4JgBEFd@q9nKH9}@1#PD3h# zq=@3@KNap@`Ro2{)pST$_h*IH{h2@}0T~J}(Ep60R3{_7fc{5dEVb_6{O;H9gpKtA z_J66N@J*5^l6C(J-2Y7Xf8($F^V*6BmpW~&b-$Ga+@OGQ@~y31{AamIP{0HcS@-um z8vovkb^nX(e+1HPMttS8l{d8YciLJaS${%0u!a)$y8p$)ZAvQ4eM;S(wpMO}MuN`V zBBQIpj+)w;juT4g3S2z#!@y1#AGb$`#JBkTT`b_6qe|C_`(NAN)kvQK?lDsAVU@tCbY`yZd<~|U|BvU^B;_s#SAV^>NT5|^shs+BwNzyH@?UQYBou_HMD&rqC=77c; z(C{~&PH=K@=hnFaLf&V;0DEBpkfjU_h{53-} zx8&B0Ayi0ij(=npD^7CrumMKPooC67KmW=&d6V^Dz?gD_u|B4~yOU?htr!HDd^6%l($z|FG=cwxh%Ir8q7- zT5`AfYvq3HwuszdT&oy&cRE^fYsL^NB=>hAblSUJCoJN%+?_1B+y1q3zkOhcG3^b; zwTf|fr;{bOW(=Whx!=Avgig8JW3ACn%iYT04(%=_b?H+gDOVo&^#sXK*-1QK!a+b6s3AD4>n+J!Qy?Jf$9b^DH*w+ z61h91k{k2CFL%a`z;xc4ocj~LEVwOJeT;HbMV5g#H20yI{^!b?~`P zySzZ&u;~A@7D|{E>i?Gnz8qPQWS3sD(vRXxSMB{PLn>^B3T@1k-rs3wEi}U_z!;p} zNkwe={9YtC)P1j>&=Ah%1p=GFHo0qm(K0YRd%;@U~%c%rq_()`Zd=o%9BzmTZtdC!? zBFjC3=u>asK4CKb_y@u5@8pEU{>sfnU2#o}tvmHwu z*!wf)f2PR5Oj}_IrrHl@IC;s(FH^M{iB2=h%>R#_d~{BcjJB1$(dD0%8V~E!HhYzIK_zl zVtZ@EI=?~;=lZ*kc^_Zai2Z{3k0Z7+$%x$(un5I3%=}lW$_(0Y8)d_iz=l@Y-)j>NK3w@RTsz1#{uS$ACavNAG~PEu<)hBo zR?IQ>K%7Vh(aiq=Vn0`TsB^Y84lG2`?JG0?4^Pxhs*kVC{9m0U{HPv==f61rI}p3t z`3{bDVMhGGivSbwArSSaZq_Vzq^p0(KVybQjXsjQ-^0xwN0ZRves=!JCi zz7)#)3-wb&)T{B3@>2A+dO#K&L)bBsCs2$TS37OSgt4P)$4(ekGlSgHe6yG4MovZ- z-v17CnT%#>jd!L~JDlvG%b7}-=F*08ypXdnZC&`D0$A$neM8jwdZ6c8$zN{<44oVkC%bF_@ggVT z_RUr{w*vn9@5@FLkN_g{H^W5cSPGfHnPpLF_amuneblJ~g^C2`H{Ab2nUfvXD%0UN zvi}J%jJXYnoVDI`r*?b#soL?TXEF*J8p^9GiyK;wxeiYIY4O@o)p*>mdTI>_<;Chi zgp78?2X6beD+FRtXl8sJSY;e9Xz?B;xAsoF9LdGt5l7tp?H(aKzCo>qv{fS;UV-Q3Qek>GHvX%u@fgK$r?a~oQy`Fgt$?osyKG` zd5?=mm7)>xER8ms^`Eu`H5wfQjOy3|RB1F5->}-|F(Dq*=oD{nr*^ zYlBAWc}%5EJt&ZiXtVj&5GQIgG>+3hyvIZvzHfs5A)c*`ss93Pb|*||IEpskA^sEm zbE>zO9EA;S_G+~@Ojt^)*ekz~i?1GoQsq%DSv zcOyDya__ZzV_FgSUXyr5aT#6?*HDxWaT1nN)GlKjnH!p%!;X^q%|LH-C z>fGVGd_P0jIZi>=2qzb1Bre%B$baAjin*5wjl@VqZ1sI-pHLI#zb^QF5A3>AZ)YmR zez{iIDzlrAHKfh`4;6qJ1wwio?FLOcO-uQt#Zuot9+H|SiIcl+De>6%%>RCRb6G#m z^Tx}*LU?R^YgEi;(~pFiIpBsLCWMId?ZDr8izEx|?s$tbu@ffo1>KdEg|uo~lX#}1 zSdyLS^8@#P1^N^~jJv%i+4c#2nv_0i%?(1_iohSS|1BV{!s>IV46Esj=9ek+N3MSy zHEp1HLSryl@$}<@5WOPU=U7f;1tf@qW4azL^@TVHuA#Wa;lkqSQ(&=_X~YNC|Iyt4 z(HV~K>3-A0OOkgb^)Zgu{J&b?@FYJep?>m08nJ{L?X8pK0*%(CZ+=6alx0r+N%sH9 zT+9;cF>0M@96&gkQ-69oB*|h}

    2B)#Fvz;M_O9XB5 zhSjFewjVSGQue946Kmmot8YWg?(Yq#@A;6tp zmOHZuz(~K2B@!%)S#qPo$<&VuC$_n$a60ctguC5S?djEDBY*BJN6(?r_qb~$t|VNo z-4n4y-Q!wM!E~A-_a}B7H!NZUHy$)=Sb@ zB8_0Q^<13N-W3P1Yx5S&>A5&1y(CT{hIH+cdGiQ;uM25t^lZRi!~`qe1!*D?^<>7^ zkwgw0puJ%6teV=&dCL|qsH@3<1T9esjPdF@)dXGS0RZJiOG}50ZZ8MOlNA{%2&-3A zb$I3CWeck6tLvA|tFE0>e>gRQ-UDas(o8{VLS_+^cpthgXvAV_*DabyD@1jD&HULp2% zk!wS@dZ@6MMCo6+i>s@aEv}qZb9AVex+PF=Ap-ZGRb5qs-F4ZlnmIM~yjJyW zn)enAaKD*LW&@shmzSMnFZEcVGpFimmWKD1Zgm&!552Jn_6AaBZ-D{#i8j}|r5DZj52g{fEAhd!%Q+PvOmj@C_+aYWM;S*u!YIhR42}<`F|iGP zkcY7mu^#{D4yHc40p3l21*00Y2IK(ch2ZW76VN_|;*W0p6XfsC2YHgY(&r*K7>vx( zpY9;>hbRObCV?HoB;X2`6ZM~7p;SRWgl8R@+((j2JT`HMcdSegg|EUq@|5xS?daf7F|L&j&y6K&HDP--#R1*tzU)>kq>iNG++KElv zE?>VyPr)M!;ZzFu{;Qe8C2I}YaqB0JZJa!H?D$C@ZvB+gr%X$YJ$)>FmQP-h8aH7o zU{uvHe|K@~&EI6lt>3BAu=hp6I>xy5_*uH8N(t9%fZ7e10WofUj9Z@zeai#Hh;i#< z-1?T49=Oueo(kq^)YUJ z*1_ei)d4_QLRXJ*>py`0z#mM!kF=0?e;CcJV_5VivcHsFDU}zr~za4;}mH>kG%fN591;4$> z!EY}S{B}UU$N248P9QmvoCoQ_1m!t|fLO-?$$6;geVB{i4m9Dt1i!tElK^#hAAxV( zm%y}OWm{Y0 zdaWjPn=`4V26nxkrHS)U&t?E}B2e>)n6$clM33n!atw&drjF)QnK^L!QUK*<+s(5m zzyj3z5unJ?bJ1hKPz{LN)NK`HlQLTOqOke8mJ|j+cmEyfq+S$7Go|Ya4tw$&Z`A$e z=~oIWl6l{M;F{-ON@? zqxJd>TAV(`+QOXwv}o(Bq~3>vFzr7h%fLD0Cy?y(M9mwi7m(w#(hC=GH0J#0CHlgo z9>GBv17DC5E@3~q{fm<1GAaQTTp=Y~$D%=kROz>Tnyq=QEh{a3 zmk6^ZNn{>SG3Lp=qV=utuN;SN-X^_*<2mBd&)=5#2iT8Z`;Pd(%jwbY4@&y)QT#;Y ze^}xlN$TlIA)1(oRF8}PpGfM4f{KYpf8mMo-vQSja$uPJP~x7!v(cn}E|THk1z|XN zk+WbVy(E<|X~jfe8aqP$3H@`e4#-Fz zT}MLmDmW1;&dp|;KC~?Tx_*OXdz0IVVf#z*|4IhW5wm7iQFCfmSo+ssw$!g_JVAyM z(T(p)tC(eDBD(E&qM`RW9^LkV`2Qdyf=a1vT$2i5YX^BpEd3|_XDS}#TV`q!di8IT zod~j0IR76UasIz<{g1xc3+Mky<9bNy^bh?{(j6$ZFpzl#nKa%qG;7c>F!7nbicB_Y zeS5;l1C@aD6JY#6xyCXIj6@$J-$-(!Cs;;b$y*3M0QWcXmeEfV_2)zr0dy;gU^OS| zYib)%pk?P$ub5?8Zbi?%)~GMD9I2>vO6c4Y#GBO2_W?H7FSo!vy8nYK^Z5Q zVj25L0@#KHvlcADZ?7$5xFliwaMD!E7$HeuA10h18d}DIl4cYaGtDwaOPVoAgI`Bh zFRrPwjB%2NbvAjdWlWSb7{B?m=Pjs&v{NJn>cHD1;L0)%rt&6_vy4M{+!-_24}Bjd zc`Mit9nKVg71sPuF1MCZZOjI9gfWMuMyJ+DtBznlI&hw}Vm|xPTMNX$5H=c2%7tyI zrsj5f<-l0X)6!t_ipl7;B~s9loDu3fS~51UAKh_`_>aY}u29hY#?zbEtXp$b^NNmD zmT|n4wT#kELANyW05_Jyzj7Q~=Tp!DDJg-Cr77ruRpN)Sgk0!=lf}>0(iAkUMf|NC zkEWd>er$wDkEV5qe;xbLg44tg#*;kyJ-B6@A!C6}hbd^%rzHWdY4hmE;+C;N5`2bB zKpW4M1mF_sLka5YE9Wn?jPoVcXDwre45`mi`l)E=g`%^I*pC)|LGrU1F%>QRqWCZ4 zc(m{eNq;5#(ZVlF{8j8n3%??MaDXI<{-UXIb;>fnCd!2+flATr>qNP@x`1+-K4dBy zf1_mm2B$(>H;eyf_M>UHi2qj0xIr3mJH<>xBfcd<-N}A5VvD4_oBe3Sz2c{BH#H57 z*e3PUw!3m1W*$P(lm{f$cJ`ww-xdE3_CxCLiT@$?L*7Tk|0w$*@At+3IQt>*lj8pY z`U)(X5c6pX_@M+q&SxdyISGKEFG#?Pmho+A;!6?*nO~MRz9In-_otHfXA%H;e<1U7p%3+ zJidxC^SfXnPctAAWMq3p7$8^qqst+B+KCI?&#|CGAwp+*i`> z2g|6bwS}%7TJWdU*4*ZK`l}x~$MG&DjxcVU>G79Lk@TW1bAR&y(ie_lg=NhxP4c5K zG`U=wJk2skNs|xa0Nml2VFU@>>gi9H7ia8bAtvOAq!)Tq(tDkuY=H|+AlO-5qR*VS~#zY=rq*&%m z-ZV^D6PV$kM>DIX=V!y5IkIxGBsyFY)mY}CykeQSN}=Q!IrDk>F>56NBWI!H$MpuK zz{shSfO=TQya=pm>{vBsX7kEf*y7hTu1=UoLdNl<$0p6AExcS?)Ym-9Tq?xwP7?0{ zO&YPgOxEc+$4T0R?&8#!p4u{xHIKszXdW-k!QehY8r&!W==vsU-wF=EvEDpU0#-@@ zy1rQgPJ;E<(Yz+r-sMb47o%G(>{jC3Ga+3L%Uo@)DKwL2i+MKQCZ>~5TMARVjria# z^Au(aGhz6k>nC6kc8IQE`A`6cpO`)-nLY@>Xgfnx1Jj2BFrLKrG0FBp0LIe>DFe0- z1z zPw(azIVH~8=H*h>6%v5H`;uh+vIL+_S4)boNC2jDapsJ)%xlcAVNW$-3_(abPY;w- z&8=hVQf*9NPdn#y%e>CqL@BR#r$<_fgn6U0VvOPQq$~po^Cs*r9j&r{Smw?8&wb6! z<~OA}sPR^j@-_*88t;&RZ((B1sjjVF48wC)S>?Q=DjVv`DxtKy2@A@uQ)t(mWiv|xT(z)^D;9QPU$6@+f?Zscu!{>2c5(5+ zE-pOS#YG3ZxZq$H7aQ#2?t)$1R~Tm!I+(?7d7^s|ejJG(fRvx@^ayErYA zOGjaLadu@FM^<)mQe_wCK6Y{3V;849c5#AZ7sncQahhQlhZuHox?tDoXF4>nAEyC! zaR^`+wm-Wtn%RY|%q~o1c3~N_3qz4zr~k+-WWUpoWCXGwwjaAN`PhYF$1cn^c44it z3nPtPm|^U~$YK{}5xcO4*oE=JE({j9eQ5(=7mGK$SXSXqR6Qa={5e1{@dUnS)O$FwLdB@}4i%-#DHi-Q zPAu`lIWz=BsyHKWyRM=yAeYjWX5KbtRH@`V`CuPAoY}|9O>mKQ>Un*XX>QXCDb}NJ z)W?KNTp9MRrNlmK<2uLdNOLxMJ%#FX!+9?X$6gway_#a}J{~3E&Ee?nVeg}y+L!uC z_7C=QKO0W|a@hN!-N#{}{K)-gG!;mYalIFGe<+t}idWywJeMYQ8>t?oJUNBHl;)(fEnK>gIeo5GSRoL4J?+n{(q`AG^?!Vo9Yw5k_uX7SFE^>$$tFsX= zRtoXr*Vr+iV*W+*hs3{VKE?ct=2Hzj%x62z7jpA2Zc^nuy{_yY|044m5>3X$@*?%@ zM%&s+21a2v>^T7=DDprdh-oo^zYqbVa54r77{9Gc2%+LMU;toKR|gTHqDvFx zQ>jIhEDBAIR@ulCasEG~0ENoukO6ndm*)p8`E@Ux{|PV^Km;C3nCi$vwFzoQk7~0U z)Qk@IUv`?bE4r>4)QlcoW;duALsdtyq~#mLq?w;6@ERAZQGtj}xi4~q$BHga@2?2e*1lVfzG4ALHjD%y{(GeIi$GD7X z!E{a~cr!nWKyR@_pf?+YgAR8k6Y;qs@=B(U&t!Vl13l`}Y>TK~5g1O5zUmvH`jwvQ zR|ZtiReM~H*;mmkyRLgppxaOK>plr}ul4F)+eO{oSVlt}l6dF1J0r%Rw~o7$LE~00 zqBqrk&JY_MAjHJJz(!DDF~v^`7a{*VnWXlF+Z*|Q^t?Ach* z_6RL#XS-F5h^XQsXF=P5Zag<2qPysv3%*rZbk4H%2ZB zj1jLIkn*$hh7h^;_nu`EMNBHsM&&Y&_hUj6P)294hZMJ;N9CDGNp zr`+I@*btdJxFkM_sTrWBb;sC~?r>TbRYKnjfhC4BM_)3*o(0A1^SB8h#%lc|lWe1x zy}o)m+*fY#&nCf8+lTB0{R^RW)~_|tP27^2EiSKUXr_Q25^N zRA-=*DZ|?seO{?xT#YXFJt_A0Qmle?A;rGessHL1`$vDo{Smc%=(Xa*EUh3qTVr~3 zoyMVDr$N`HbC&r{F%Atj#hB$a#h69p>K~b%#^HGicd~CA>T0zhOW|{+@Y?jk=XM(P zUW1H9e&LHYV-7?r$2dyLTAE(gQJuyKep$!)WgQpNxw~wkuR`Hiq>RsJsq$i}k}qw8 zy16){8!YrrHyf8jH%8OW>E=sPI9&mU6#u18<66Hx*Z75Bvsv|{#2CNnjleykaYvR$ z+$HtglU~nVp+;Z{J;TKLe@4Xl|FA>v|0s0+Cp<%ir8@mXp;4a#t!nS)iD2AE8oEBL zp<>>SFfU1uo7Gup%#X7)<|opapQktGC!xl?<|_^J^&aE?aD62<(-V6YX47h679rjg zA>I}tjCY)AYP{KLeBk%od!6b-nf=Cl?mXFJi}82P1)=`r3;CzbYWFQzl4v9#=AVAd zKSN{HN?*cE)1CaLnQijp!o}_@YZw(~qB9(EbUmUhiZYXFQOYJ^&?B@4>5+cA@_Wqw zUY-5nEAo6r4R$FWu@EtLUWrLp#{ODs7IvD0gwES6Y(Q~^QY{MD!z*A9_{uzAS+3%C z_lw)Tp~T!LP#H964(Ho^uK_ASVV&kkM}pzVa$q0}Vn;{h813g6O{I>DNHfk)Gj6jQ z@9UcP3uKzOtl>d*YKb|y(>yp2yN)>#9rZ6qPods*o~>j|?g7#j8R6(6Gs|LWF5n?* zF6Ku1tGc-$I8V$a-as>#V4j#WLi0oq`L=l!G}UmQ$vRz<3p_sCO?T+3(s1-<)f`+zvdMU= z$hbaT##2Kwp6SbYCdqiV$Vif%oqY{1&Z3udq|S{}r?*;|=LA;^^8#!t9c8DT`S#qJL|twx^W^BTw7vp;V>~Pr5ttYR&y(dBE_A) z9QgW?(H$LhdOGOLq64vj^XTVW=_b40Q_g|9m@Kv}2A{{!D+cu7S}fQ>>Liwz8H%<1 zSS!zs=0^b4h2S~Hrq7@hX8b5XoYmH((jj#1%*K*S+*3)>qAKT6?qgYSlcr5DHdn#*F`9|by%eln4+HC5wu+6*jHGq#wsoQ^juy)w*7)*8hg;l|GK zW9L9?(k&X`4(=U*cP(ka6y&FVe?&^NhqJY}t@(jYZZe_#z{u?GomD6$9~b1<Qgji=!$E?Eik5fz96!4D9@VU|`$#0|!M_gm({IgThNGyHlz8qB;-bpbu7E z;xVRkV%`?>rQ3KFE;WCwM?0}Z$4toW`9tqEUy^}UidUF_=0pcF|I{t|=jJccGaq=j z`BOLZt5G$-?$-Q9WX-QjW|r>%?>~q1@07+*{o(NOeEz4Np^vQeJAMAoqlIU|_3AY} zQu2HM(m;3zPRX^P}$ ze&`eLLtP)yLAyf84AKPE2fylF%?KFqe4KUgJ&Uy6>3#(C)KgPNz2lY<3sehf-Afsn(rh=-SGj9QqF7O*N zo{BA^)4nyev#O6?wi86fq_qvvTI&(JGg`P)-8(=eT(qQeUd`;9>ct_gX0B(DFq@b} z>ey4I1_>ei)-I@BwxqUZ(UNc%+bS`0!Gd|!m9-><`^hqQXT(Pg2l4sdv!V-of@%(@PxwAjl3+Mk7MEK7c82zEkJZ9j; zuTkk&m3US}4v?70NeC@xKO!fDm(Ego^qoEwE#LTiq5Iu+a}h8Sy=34+{FHJ|BR2;( zB@%rC`89p)3=Wx5cLEv_FdfUp7J;|-A2Lt7Na8}NbY~doR2OsKvYGJL% zR#}D+7Iz2S=jng{Q--e*rPDB?mievBF_btVjl({#N<#y#-*?&CcPV9rv`bVZBPW$> zUKeC>-GmVOxLx@%5_2|sWFXBJ5?GM`bgHA zn3yvr=ETL`m=&qE0C%os%sfD>t!rGFs)8p44Efd;eCtS^-jPtl1=FW; zT?c@mn$IBA>VrmP1HXD7o9Vq4)@VJk8EsfTGfaVeQdVD6^(P4ZTLs0 zj)^&AV$Sfon{AxL#GF~V6Vm|}b5Fta)G;w9gHKkU0#X}+D4gBax@KWx8=gl2loI>` z5g>+G=>5IT0XDSMh3cZd>htPiS<0Oi)LWm^a7Ue7S+>LIX^^vLasID}IRBsV;-+LT zod3s&yk2XthhD<$4+^-KMrwkkE@uo!bpW?;c)p0V;@?_7l3j>k~u! z`FKqG8O2lx7MN3;n^Ns#R<}0d*f3^JswLG16x$VmBwy9KVtUQeW4g()mARtqObDo~ zfU=4Idx*5D9#RhjX;VEyvv{hd9utcjz?+3IaCsK9aXWhFja#v6pHNSdT?S;9>Sjg&!Np(3@In7ef$-)6RO=IPFi{LasWubtd37)gDminmf0} z=d5fCJ6&tJw<&43)RrEm_aF+rJ$d^8eO;N*nC%_l_uNki@Nj)MI{JPjSOHxgxz{xb zeE`W)vPLLXXT{c@f?V9h1nk%npcl^njcLzE#NG)JTqZQg zNk(L>s9Bnj(5t!KcwW$5vMJr7TdD19xd5#3ZQ^f-G1c6@q^0?kbtyy6i4EX)J=(wI-?OlRQKwdyOdaYph6~6>BYhtq_o3$B8hlu9uN_1N+gG z?y_lqL*GQbx0&;UHT0X}zlF|!=*-)Mx%75CLc&HkH?_Wsk9_zMNHbZ?vg6iRU{Sr3 zdiJiQKAKBH)_X*q_a^m;1r;Y>y7I*ne~9NQXc~I`eu?`wp1LPBnA>a=PA5|Dd}!(K z3Z3Z=&Vs@4JwAErhuDvi@Q5`0QTAgPd|$NnIQ!9?Pl_K$Ug2DQTJnR%4Sp!@S!w8V zIhoA#3wXA~=P{o13p!TOC8Vyi^pEsQee{d^$66gXqAqi(2#`;u-|=QMq+}AOr^b!2 zeuV}HXeWdU_Gi-YpR*r5_Np`lL=lLG-ro>E*dC0PQ1vgR2XVecH2lAo=-;F>=YSoR z2jOoyAr$^Qspoz6L*XBYLV1@U_S8R#|IZu`h5uF3|Bd`ahx(yZ^bht!;r|l9ubo2$|UGWm7bL} zKyr<-2{}N{H-N{QVJERN@j&~pq^<#9YALpIW48?I8e@k6rA0>|(!T7yBH$*f!Y3cEK*T33jm`u#4rMT`cqL zVtHp5%R0MQ&e_E>POgDv6>eXe0PJG$vWsB_mw^29@iTeMYWV24$n=6;e))YWDZdDY zwU+rSdVQ53f9i|Mljc@M00BN9#3zMdgAkH@sr3Hj1s2OwOwFK)ks%N}$BA_qF6NTg z_6aZuf-OLvLAV70Di6Xb2#|H0@5DMaT^aTgNXiVmpC`xz@H?P11e^D8*!zrl^?M@q zJK@YmQMp~T-8gvZy~d$AsUeP*{sPimalfsKejiUia^Dk3>h}mI#8`0_i1T@NjJ+BC z$=H_|B8Pz*JF2BQa z5gz;wi-@g(I37?)ibKKgz)2lF0bfPfG(IhP!0%9M??(I%k_sA12s;J*XNk;=i^@ID z^c4HD)I|#P6#Ju^bETku`1)D?>S*rsCHf+W{RPvshZ}YQ9GgflE-BStbmbNj3mYhL z7ukE+W%l0oJ}3n+NW!YYNgcWG=0L;(_Ybjle{Jt!?}_vO%mNHu`FQyM7w`G$+j`;r zPg)Tfpp^n4y?}q4Xa%Gf1^@?0m;nyy1x0phBxceJLLNeFQDQIK=7P3$8Uva10{9xn z5I1O#nDpYDPIbOk0&$_$iJ7o0DuCmzZpia6|FeOKtN&%v3)=4SUVPU|=ou|VA_O@6 zdIae2$3{yLuDTm|>H@9uNuZ^0o8a|M05(|d6fH#}kB}x+YXJK$H!X$EV+&BJzcOAC zp@b5SC-!TGx^=PRrkBIkBXD;rSQqa+EkyqoLG|iE_zq0k z-6KRR^|C;4ro!NVGAhmj9WA}D@g7G8o|6i~%@tguI2>SqL9&7he9`=&n z$E_lYH=_b3IGNo5*VFq4I40^Cvw8pga<@kvkRc@M)T5Z}Vt=>XDDH~kY>_E71m(_f zMGuHM!_Dk7NO&e3P7U;3sRNODPDJK8;7Wph6lmidx40;Ri7`$a_eF7Y0d1Y$q`5$y zmnO}1iV09i_);x*q&gDCERE3o(oEDWz0`%hI?nHpr7mig#s=%Eb*Wo(nKupy$EN|m zf1%USXr~GLgn+pLS?;z)#^(ytL!R>er7as`I)YEeIR_lUxTCVoQTnO4rbJTq0x4ZTU@m#kY8MIw4EO@+Rpbz z+xc!WH4()GSy8Txkm^cz1p2sb`br0mO<(N~tSjA0=6jU{l@}OTR-U(Z=Nec?N0#%D zLsxO1w;H+&s5f?a>Bg%w9%iWix<53&-l;Ko-4SJAMF;U?Yjtad=`&ekw0y5Eo)7DS9ol@*+4pC_{nfg}Ap7B63L)vFNX`jg= zt)1ASUv@j}`A+>)LO#Zo#Ph*|alm!Kki%{Ig|z8)r%m3l_yy84Sgs6nBog~ABshirgJM9O&l8g_$!MkkT~(LV=8XiSvI|#QA^Wx;IKf=YK*_>L&hw<=!EXPl(}w zY7&*KFI%AV{lGxa`+% zN1HUpxkXPfh=nW}#Xo81^FN*X>C?^ge;!>##`ypAt7DgkAjbcvu|)6J z^$8O+#Q6UNlNsaxw=rEuR#@kZpLjuO9*}K}|4)lF-tFFb5b+rQpH!AZXNdYA2mk%g z!t+A;G5&vtEP6SHVFWu;jQ`Iw_iQ~4ZX^zO)dK!CCDiI}5rq=8i$o$Z1(1pfoT>AvQ*XEAEm;%VxSvHae|tw`TL=9nn!(L2RjLXoihl2ZV)N?N-jC9O!~sNC4-%I_ zAp!P-pWBY8SxRuNgVhkq2<)noj#cv~qJTVW+ATFq3f@Dj{j&d1z7pX2 zr#3Pup4v;RVZi6eioK6fl@+RxU;=U@_XA&bZ?#XMDpUJvH8s0Lxl4)h|2r@UeAM|E z|9|HpQe8kfjKqo<{~vI$^ME9heIf$0A)$Z`)(6XwpaE$Gm9>|3Ak6XNDBPS5K&Qf|BG7UqXAzJ<`+ho4Sqmq)sP9=JkY< z1R$8P*vLe(Qi)>!IXbzY%5MQyayrvyFvvs#r!Pey6QyylZr2?YSR zy;ipU@8JK>jyV56*ZHe=d*S>aw3?QQd`tkMaLQJY4?CV0w&gdurpDzYTz^#(S}6p`Bi9W`&n zQs5V=x706V{C{P_9M|gj+_p)&KBwn4t}>EaD#rhx)iWS>Q|A$obUuddYFKMpHRkTB zC^NFVy`>X6S*TidpQO5xz&ytR?9>wA*nUX`NZLNl?X|7UW@2euaB+*U%eS^!x`c@y zbZLzLA7H`BazyVEiN42tHZlHxY$G*B+ek6~e|EAe=qBRGoo)68AXU?#do3N~|F3Nx zBbN*zbo}X-{@C&V=R}c@&7F_ zRmS-LtH&IfYU*fhn?4fHK!@PT3?6l^PA$YWW!0)w)5%1g(ROO%YAeS77hHSXDByJ| zL&y04G5$Yb#Q;pF{sbI1!ti&1_WP3;nEeihe~ka{fwQR?|3Ak6kMaKj^&5fze|W_C z|GkHPzp59`|1tjmR&~KP^##V;1Xo9Mr@Bl)MO8DrSGsVY>dF8lse^5(QVyOay`6$r z140BNIq_g){C^lqc-vfU@?bgxxVUdvDdN8n*&|8C-s_D9|U&Fz)zI$E1sn%YupQow&0!%gZE zEjt5QwtrniUDblxy86YHHMRA1Bvwn~8Ui^?ES$TG$g=%V{=?)r)pPGd%(=b|z)UD9Q(pa$RsB^_xAnSD=-Xz0`l!?76*x7fX}BH-;0` zBjVIm9bP>@)Qdw@;>gOyRfktD=Jm+=X9L^_mJI5M-jJuzm8Yt*zUR`=2<@pf8Ov&4 zURKiqyZbHdopD7wSGPq)o~#RNhWwGB#n!@J8HU8W^YKbg+UK!Z8QjpI{azWi9~*FS z3W_Ooh1{jFOp~qKDkf{FNxjK)Ha1|$po$Hc*nn}#dtw8on@5eH!eRp^gC`)Up4fod znIr7Z8Zd^b=0-gKKYy>`%X{Jc&u8Pmaui6J=_4|eT=Yjp<>iut5^*ajCE1HcuGC1xh70_`>uQq`P#zy$=lm z9GW>U=-*l(JZI=n@9pT3I*?{+tt5eu@^hE&$n2A-vRSbFwc@<*_*l$5l%z;R_md_n z)vfYbL=iMYOrwaB;3MNKASUO{B4$RpEL1X2oP5WZd-Ro|(jUM_Lr4|@6~#rAPScsD7H8w{Ii@Zrq?M&lnc^n(dDU^tWNz{#rP4_KX{uM3 zQbx3BiCU=pqII=E?WX@STet}+=|jay3w@>)Dur5ZzKUwn=5MwdSk}Y|;yA$ z?)n{>+N8K(xh$7iA4WCTS?LcTYu_(?hlZ7xuBtt|p zDsqaX$0EBCttVZHIQnD=#JREv6ey9!NjkILbwr#j>yau$brI8&d4+`R$U-y3p z5*=igNoiw*OJ@vo>5KuF&X~rg#+@k*x1Cyz&qPHB?ZFr#O+is$f=P`LMm!&MIiXBo z$R5(%(%f+<1Fi#!J)am|@>Bv3d8%4h0GxBc+SYb(GqkjKw5=m75Ijh8%G2tAoLns) z&CK+qDg;<|*8kq;jhj*rGb!&SHu&%p}h=8+$+%fx2N6daRN{syomGvUFW^Kp%>2o&T_+0)4|1}+NF>3iGU7P z(}|_wkjTZR5$K4y@+TD5m;uc0up)IjJ@W?HidGY{Q?jI&7IW(`K^Mm45o#_Gd&Jy2 zE$vA)NvkQDTMG|Ipi?zBrP?#lQD?62NYU<=P{-m042;S7sg6~x*j?sE?Jn8CQnQsV zg`t~{SIY|3aq0wOix`IuFk3-^Et+OsV4N>kO++~X$&A!^&=6%QK~PDB!@@($h5#oD zW5p_Ap7D6zw3?WG)=61jDFNyhOP!=nrUhVi7f9G?f+SRnR%4>OC~IvAwN|=j!Sa)U zL1w8oHJO-T0CsPml!gNwbL%v(C6u;k%P8j7$u;!S#wsSwCF)XIT)#+jZi1yQmpOEW zR!NYQ@d`}yC+AFznmt5w^de)K(*{}Bd`8MEDxtn2A{^Dcq60I! zU_^~@G+v{=R;a$JzMfReN#uzZFjaWfxSqU|ETDNXZ_PL0T{+HDn}tZ}X03*)GCp&r6n20KqZcXU%@ihRm&`2A>|5*3cP>%3J5=9sH0%#(8L=g02^)lHoui%2? z-^M7lZi@}jaZsR=VCZz(9(357e$?> zenq?8+aSlxwI!ujlu*CPq#t@${kBlOquvv9vu4ejb^LBKF%NcIuJ_gNsjLq|%q;4U z6ncvK6Ldv4Q6D?U&tKGENs7NQMa)nP!i8mTYF(s(}@!`{=%fI;jhj>T#}S zyruiAEjXv?0T`_?8u+cux}*ky(Cz{2LxLUS12%>(!NTSxcfyl;w}J}Qc;?Vo7J+qJ zh>C_tyLL~iHwr3-{_Uc(M(l_C)ZH;0YcUQ}E9SMft~~;#OklToy9c8&>+&8@QQ82` znx%`gG52}q9+PdA2yUxs5?FdKU4~sk@6FnpXz6`LTl*#Hp%fH2$W8{c^ov^ z(g#Y9jRIRbx*+h2k)$3C?(jgj1>&TSCgd_4HHcPs3zz8RO6u_i6+_y zoBpfS7gk-#;!LshR1tbwQui;Yn0oLL7wt9=r-YJEv5C5XM&XRvHAh#^qG5x3G&F0f zr7I*;Wl|5}plOz#B`vK^>U}9_JZ8n=(o(RwQcGtqSX^B*rxw3hzyO^aZfPLS8I3tx zI$v6vnJcLm6jV(9^Nfejza6@S6qty0B5Qq8A5c&+=c-vF=Ktdjj>GI}khrBueIN&c zo9j4f$MH!$m4nc)jnWPra%j?^SLV!HFu!^+eg}e<;D+SjaG&|HJJS|0dJal2)ybFC zs|qSs?)JS)jvevmf{JrS>^Qf$-MCsD5yC1`dN>BxppNBgmHbk2(&(_XK@h5)|sM9jGV(c2qR~M=<+jo zg`Qi@FZL((xj5N|w8-H*qrZ)ET#{dz)OniwY(d4Xg@=Fhfq%{~sQB@Z_IPL9bm?@A z)Qd&XFJM?Ynj<**l__lN$L3buF47|FOZ68?eU~M59qD5NX6)!Jn_dPm3d1jU1S8rV1x^fKPSO$c%^d@~h>E{NmMr7zGORj{*4F$TR%X`lB{Hsr7R9SlHSQ^6SWKEk?-I5gvJiTia@Z{Q#(?Ol=iw@H052SHcwi_E{rbECjI6*R2I z=iIXXkZEy^PwGDvR6O>Nfye&t5j=>8H0XoBi8Oyt>MIK>-f6q7@{8|dY#tjw)jlK~VP~($^RZ#Kt^S57f zY~^)S)1l zF~S&$%N?T};^KZdAZ}B)Rgg`}=&D6wql|+{VWaU{26l2b>kMpY;{wxz0(W8$H?Z)M zg&0VlG$vwdT)*3GH=VeJ2g^ju5KGm7rTSPw#UC%||G8h!=Y?m|B+EEdhTCCDEK)Sw zCR@f#8E#cc{VWGfv5eU=+~yQi{B@vp->5}b7gQF1u=J0=xfts=k<83oxBBF$YthrP z?UMiF*P>WnY9r47vzsq2>W%ZiH)JE8A7viVGDCKisN9%u)ZzeVEa35h)pU`JqQy!5 zVp6|cP&v5t_%E+`O>;C>)}tl5A*o-dpvhPQkCj1oTvETyL0I`tkU`d1P+4%PKjuYubD zRE_byPW-3RZj9uo^KNWl1w=H)`llrOne0c~&XVM3vmcH5jQG!?>?r*_$qs9pNUSa6 zvwZ(#T#(d%qa0Jv)(b`Ki<0`Ig36O_IrGkg7P4tD1&#iq#9fA)kfebpR(x%Wt7gw0 zw2UwD(=7vrD~f{_uI7hB2A(a!kM>_9{;#ng4Z2nuz`M;9NO-*n48soT(f)6U|0Zex zyf;e&*bvUtT6*|iuD3*_G`(78DdQDBw2*`~;8h@e%=g)Fz zuWM>+UA>xgfM$*uxzsZLYW$7%)W6F^8HulHX-~C~`G0CE(BLojpuyQ%8@5I8aJPr8 z`-kyQY8`Gx`?R^Q!D%%-N3~vqrduXn3GJf>n1-bWDI&o$^CI8eOn=)ed@GkQ3ovrm zG`5{w*6qz>|#4&7w2(yarR~xXKr?}$g_*Zon4%y*~RghU7VTO#YvW3 zoLbq%QI%bsPua!!kzJe>$)y7#yEqlHi!&R$IG?eLLm9g`iLvYGi41D?!-{4XrZc-R zk=TVT#4ZdVc476f3)_ZWSTpRxieVSl3%jsd*oAq*F3b{kVUDm1BZFNS6zsxEU>DW_ zyRZt_g*Ct~tN?Z${n9qher&qzVl!pe(JyVD?8m0bE_O+FvG1{q4UJuFWb9%CV;6f8 zyV!`}_N4`bT}*a%F%98XDNXOO<6rtcVG&#mSe0LXpGw|^FZ3&rzKBlaYclH*iiGCj zB1Qvwy>abLLxD)Fn9_bV%CF_I3Ma>K+o=x6_D~4VYPJB!_@iOw-kkTo7E%|++ z&j_bJ%ZYUoUlR6iC$F9K%6f?W(1+8A=gI47TE7&|{AM`uZ^DV+Bd_144?~F!9ZGEE zJFz0AQ5;S@IGlJ`-gaXj5IP!V<@Xq6X5KbqfBHg+%>42a@)NQ6|=5-ne_EDy} z%@`A|Alw~BMYw`l==E-DHG>#ah9`VnvW^Fd@K5C z83Z#ot3>%drrEH=Omv#KX1kNu1cMBobU72(E5gxIi7%CRs{J&0xNy%?j=?$BRFHRK zZ~PUh9UE=p1)W%4q@LYq+vUZ_@L;n0l^0(}-%Pb2;{5;o;tO{4!ucOYLV59>Tw$I` z`Y*n}_<`c@n5JRa1sKky>bs@t!A|x4VXD0Nc@fMizTfU^7uuHHPkRYoDn^3WIYD{x zFC}6B^5PHWThi_Wra`BykNRUZMuY7WHrl&!%k5J(+JiuRXi91uU;FXbJZpafqr7BJTx$qp*fV@mY65|u%VvuhhT3TFD`FT$8|~q!yGlyi z4PWO3#WMNMjjWr$Q9#A$p{tBv-h&g?7i)M95IjtVkETP2MQuFj`GCl zOO@MY8|`wI!XCHL9)+5Zm9nVAP6!IoRH9QQdS!`jc7;HJCza~et`O@a<3Oad_qPwQ zN7y4t2uT7V^0l2LT_E68PY8?a=5D!kqdh_Np*C%_C!zjNNr^-7_3WT1=alI4O7!_9 z`htKcpDWcDx}sbm83!YsJ;okukF&>_`@Z=GoDxxKd=myf>U&ukl` z(5NXj=5`tj_at4^OWyrxsx7wb?0S0%LYHu;n^eTfcd{PoDW#O9AjK%mII{g{Z`!oc zUXCJ;k%AAvS8${Hidi0*adsnb9wJp!sgZJ}I!Q9`PqXeA`&j!p`*@UdvMUuQC8-ej zc@D&vFG!vd+HSCyVm29)if3OQ$qxS|uI!k6hKyV+E%^mb@=JK~!8SweAGo-pzQgQt z$=YfB#e(HVd$94L8}w0M$1cFLERPK{54M1C{7u2pn(UVYv13llgMu9M$|Z)KmS1iU z=u}4$btiMdscB+nXtg||xE3H?P;tvqTq>_TZ>z(u*rC-9b!w5?KsA6TOPv9`V_8gC z3sw1jF$A`xfshE^C_FTb(Drbr^M*j%Pr9A=oZnSXx?Pn?K-&Jw(_^C zpX^Yrevkdai)&&MBb@hg%vfN3rNM zDt-;ZVi={rGt||puv6hF_hvP{9EaF7Zo#^GDEeK)6W68Zt&*29v+c-IOM`M;m>zH+$}CI zqPQ_mBlm^SBQXey@?;RWO&V#sO&aMI(?6n^X^vC}p_qdsq&k?)$neUf4|WS25>a5V znKL5_oQY=c7i#8Ax43;HiVHS#Hkw(S*38*nGiSTS91u~=e5aX}Xyy_+IHZlRC2lhV zy}2Y2kBpey2Zow{j9c8)h~k1xUxucaIxB_S!^^y;FLR5jh$v=% zr|G++>1+8^fVS%nZ>`d6-Q2SxatEjD84+zd!&N4(v0TB$$lf>Js#vtIva(i$A*YYiF z&U#Cy-iqaj5+U+--n2RY?VE8i&*8F@LnIW_X}UKM^xj#<$}tImTf&EKwM~4q)O$GNaz1{7s#>f&*=c`$z zRzo`vCClxET_{}?Qud>wY`%VSl|2?Y9;FNed~N?Qr0pO0+WvvRqw0r5B7@$Ri_mZm zCA4@=@OQ!BQt^Dmw0a&(1x^8h&ER=gBd13c7o0*bL8C(*jk+`PC2tD7v+z?SClt;)vpc&3q5d9G12szUMXb zJ-3)EBZ>(&^Fz0pfhK(zvG9NB7U&yvk-cf?%w`%I&8!JF)4)8XkJg(b%i-QpgMC@$Ef zVVMKe80Ix;m|M)lUNOP7V~}Gx7y5(T&G49)!S9)%4i0v8;E!m-Gi9EL$Q;x`MT8D2 zpo61AI;e1Ec{ZZBpbln12m6FauQAKh!7R6!7bA+9?(FB-HjKK6{TzXGLlib}j5@c- zm%So`Y8~xZll#CpbHu{h*We2&8x3|ix186#a(rzE6}B`&VM}2coDfpjQn$FbBZ>&6SG+y`a!U{4j#^??V8fV)!E_Oy7 z%(sXV%*irR%yO&*m*j0~`iRwDAx0D#q zW~w4bIrJ2rjw1#hGvE%9jw8kktkRdeRn9Nma(?QU^V6_O@d(hZ;dNB_mZ#FULMrXQ z#dyap>}@~G+sN{sm*u@+7I)6$9n)@2Vf+@uoq*nMr^m}itm!HK2kyQS*h4;u*jGNt zw4jX-Tyry+*WJQ{VeV#6uZ*}$#0Pd%ZqEV;rR+n>k_Yd8c6vUS5lGK=G6Lx#8d(*g zGW$gZ(o>oA?4tTovrwIft4DgP#)eA=%e*aSzuU+fR;6Zt;P3kH+|ZS5{?NP4e)1kb zDc;B{$;|B7v886Ixm$YX1MfDA-OK}{Y98#?JS4K_!IGJ!`=9yekp91sw%juNE9dk1 z-#y=V^ZXz4{{e=+Tl`)zeNIfD6VvDPkg1cNS-HfPF?~);~eMe_7M zii!FESd&7*R;yTdG5=qXUGq|ml9>MwN}x~Fl;JUxAM^hyJUpi*b|)89qk<}mB z|AXS3PZ`_@cEKK!e!eQJ+W!py-=t|1mror#VPfy_|Ls(1G5=r8|Cc&F=KoXwv;2Q` z5$FHChX3oTUO4~9{C}-&>j8H>OH~8(OU>?r2*ZQYb7&D0>Tn;y8pKUafZOX5!8)CE zDjVpifJmpOj`{!kdekW~{~sZ-Wg90k|6iC>nW2nDS!vkc({oS3^wbu3b51AT8~}Y9 zD}lE=HX9INmfC87?$+GWkyM`s9%Tz)G3#3wwl%L|e8GBhtPWjPTTCO92uPc#7(dq&ks8reMr>7vkGe zz!RiG?gsi|TZ(wcn}H;L1_OsPpg3T4Cv+P&n+Ey4YAfyD_X(ke1GAn~ojKswa{}AP z{C}XF8Iw*8)nlaArQIDWnM>Kugg8*tJw#58`Tsl)u$oM5$NYb(_NF!=+Rbe&L~yCZ z{C`>Chs8XK&{)j>=kb=~`G0-H`G3RrZ~JU-od44o`7AxvC68ZE^Tqss^b)7b&=K?h zMI70J0NG)r@#!)DpN{$eV*bCCMA0xl=Kl-cW=CJWSTX-!DpN}l7Ff*x2UePx|1ak? zNzDHj^Z&*Ce=+|bjE|WAZ*61Ks?gTj<&k;ku0?VFUlMWtU-RJ|uk^yu20j|Ix#KyyR`r^Lln;{y*l|(=q>F%>M@hj(cPNKY+yf>~uQj z|BLzmXkHjG|6jK`0e6P~@5qSr|Jf&eVO1}j|6~3?a72NZFy{Y@`Tt`6znK3o=KoW6 z%>QS^{D114PIbOk0RSQqdI#xR-5>zeM7Y#F?C+TWPlyF${y!C9lZpBNfCm)w|KV!WBLoOBJjVQgLGrSg z|L;G}|94cx`F~TNXa5p9{}VU=JImY~d zdBWY~^3cTmfA_@ve+FYi`u0@J|Hn*a#D&F7WikJs5%d4ym5K)S8WCv4{C_e3pDYY{ zb`IXYnE%fwymamWjhO#W)iFot2X@T=M~B){^Zz6N-$}D>c$v=sX&?Wlz>Uau{&(YJ zH`YL`fmj2v24W4w8i+LzYarG@tbtequ?Aud#2Wa-YGCEJcKdC7{{O^wPb^rhfmj2v z24W4w8i+LzYarG@tbtequ?Aud#2V-o4aDdFUQuIQNUVWa1F;5T4a6FVH4tkc)w6o$`U)H6FPX?3@x;cAS`#lk(^JvT6T4Ok&Xy zKE~z0POoRe|N864T=NBgLZXc`Xuvn;-!HvxUsU_^FFz(-{M_F{;QDX;W1!F75G61OHW&!J#G)^=HPJ`9o{*u^{*cMyMT}AlhYc0fJeuM zKXD-#(h*%VMbf=1Av*RYU*|GN5qN!|EY8ufXpb&hfbLVujl=zS);PWDH zv^HAchl(Qbg)SWPiNKE(O*_6Q8mEns_<&ZySGshKHdf%3qBZb~qFfZ(IEfEv9lX<3 z#(XOAAUsbdRzR}B^`GvqoQ@R44?QOU}lK-Ps z+6SK{y5;do{*Tt^0Q|B{Dy^;L|7e{K!54|in3w#&rYHulOKY6AlK&&n+2{z(3xh4d2mHpwlISMvXCLN9}_ z%Oc@?O8(zabQQc#iY4=s|2G191$)`WpS!nGf|8FaL1AJk$ z=OzE|D0;KwtAzXGB>(RObOU^443F1I{@)4d82qBLoKGkDe^=34;GJ>&eL2bhy8+z< z-&QI6!Abt#Q=~iIWb8L5`F}5<1iY^@{|`ed!7uBC*Yi~Jf2`S z3B3zGtux+FQ_25vMDKxjHs}31mHZ!1={ESRF8Mr=O8$>mX+B4TmL1IlmL@>FM)Nyp zP}en|Xi{kg#OpMF9}Rf*WcGu4P{h|+4+Coc&L#djiX&c9JQd1{!ZKowjB{=b>xN_{&kCq~i z&`KP*AoWHMNIfDX5J$ubA%TS8g2bibgg{)7Ktdo;`Tt$7?WQqdOeG$gjq+!A#`DcT z-_ES(`)4+|0>DMTd^A~Bvuv$#@@v!R{BHT!l3e#PN;yUPHGmH1LoTYEuK)Cy*)CVy zdHxwYy#~ulIQT97@%ZgZ#CByV&8pLws*|gd6$6R^#eiZ!F`yVw3@8Q^1BwB~fMP%~ z@c&_;bo0AGl&C$9zE6YY^6~px=;Uw7@Mt8X-M9{;5((J$tX!-r^j#QreQBq<&z5t! zW;c1VgatfX&La&^X@K~ej1whrW84EDjQLO^FcU}wZ~%|2`A3h8`TeO8zkhvf!c{j83T`~3vjp;sV1WoY$BF`yVw3@8Q^1BwB~fMP%~pcqgLC+i|8T%+FXy?}0o%DbCqwb38_f*d6=zm@OjU{|b{KwBmf{T%*`2`v6K4idaiGL~` zPXrR-cp@B>0?^?T@IohmrJ00G(Lo)}R^7DaAW$sk)2pSTS&-Oj(5|6HU@vwPz_eK? z#&TD~DKlG4ucys?Ad@L?UywlT(9kL{jKS>?LmQjLYZ73-ZnV{PFPcwpB=gtKm@?90uN(PecW(1Wxgl@ zIiR6S;87gd0aBiX0Ug~tkU+Of9#`n-v7&qAaWGj-W^xy0dU>CYUV%aMbMZ`?`gb_H zo})cFn`XY4mK!i$^a=E$o1v_|J=AHgq%-D4DVR{>CM%gW#7HXXwPZD^@M zk!ec1j)w#~(N5*V8=JX&(M(ZylY!{hu*-458dls0n!Tkx4ird>qiGkTiX%L}QD=(9BXyE+nmUOuHFg0>@EQRv4}=$H7iAc2$5g-?V$S_&jMNTHO*@_t!h9gMEX!(V%`8rxDh3n- ziUGxdGz0o6%mLc(m?rxHrpUg;I$4|G^QND8UA^fYgvefIrQNh25qbyx4wv%0fiq+i zti8o=pqJLgTfD=dXcYllQ*KUU|FO8mHM?yvzO=9?m8G&ujY?{onTa?X`RK|1O&4f)wV~ z!?iV`eC@3NmoI76zxpCTIjz3_e?Xh>9Z6l8f5}YX`ZSIQ*t5_ofYb{Bn!ynPnm+L? zvS5}{Ir+)7kM=@1svy4hg1M2DX@rkv;m$lZULu*4NUmn26pqi}H~>vRsSP&5wM{92 z9)q>NttqV98xmSBgY~{G{kEXU>MWn7@a+~^PzlKvb!jZ+X-Z9$Nb}iluxO{7%7v+7 zNnzsCSea9hiHcK@Nv1EjEb(nC#B*l8(2%zCHM0cYkA1n+>e>o9@nE7&^t2#3{x}ZR zg{`VE`7zX^vq~Sq;kxK{V8>F5^6;HH3-`l#YzO#;LMFYIuBXS411|5MuvGm3`YI5O z#s^w8R`%3|9Hd6HwOeC(Pcah*r1Er?Z`z=x^XcJoAC}C>@U^hor{}bT(!jvfG}iky zBsV<7+V1}+J9WNeRkC70F`yVw3@8Q^1OF)o^e6B%#R$Gt<8 literal 0 HcmV?d00001 diff --git a/flume-ng-sinks/flume-hive-sink/metastore_db/seg0/c260.dat b/flume-ng-sinks/flume-hive-sink/metastore_db/seg0/c260.dat new file mode 100644 index 0000000000000000000000000000000000000000..25f81fde784105363c372eba91f6175388235f63 GIT binary patch literal 8192 zcmeI#u@b>R5P;#mn}{nEdX-A+0X&0YhQeq(1ZH$RfG6-+YQ4@Qur682oG3I3^Is;H z&F;-@?&~635pjAyhmUg}=lW#zdQRheIg<3p*NIlvMYU^-I@&zg%Qw$kSS*vh%Fi5J zHSdNZuO#5CmTJJ{onf-wck!@^`wyH wy1wVmRobsK&Sb3DTxM0C2mu5TKmY**5I_I{1Q0*~0R(<5Fp_@J@zR0f6#Y9cqTpT@tH{d|bj(Q4xw3HM?<81yW zJlgjPFMJ!7gNPWtsu9_k*cb+14|6(y)}xdDxLjyur&lI2@SBV_*Y(%Y+rF4RM((NL zT{bbk>&;ApCT?FV+Md~|m1BVb0tg_000IagfB*srAb`L>3S6#7BYXWa|AQUbS^D;^ zzTPb)sU8`eC2I}>dS}g3{y*Qy)RZn8ZR={R_p1F;=CZ15Ynu~^Z!SSZ009ILKmY** W5I_I{1Q0*~fu99(*|@z$K1^TR&n;~L literal 0 HcmV?d00001 diff --git a/flume-ng-sinks/flume-hive-sink/metastore_db/seg0/c281.dat b/flume-ng-sinks/flume-hive-sink/metastore_db/seg0/c281.dat new file mode 100644 index 0000000000000000000000000000000000000000..cfed875dfceab65004fe1b91cbefd660cea0ccb7 GIT binary patch literal 8192 zcmeI$Ar8VY6oBE^iER>v#vwtWI0HeW&}bmRfk1KsZonnr0Ml>(Y%uf+c^G zy|wRkYxcQ4S&N9#%Z|jw)WtZ2dYsewvmA}|$K_NjOTBWDPS|Aha9w{Lt;e56kDh-j zdY4P|?|L)C=n%iJ7TuZoww7ao00IagfB*srAb{RQS&~GB8)4O?G9~wjOLRU?Q`!GItqmN?d$`hyVfzAbchAM`1#m1bdH?_b literal 0 HcmV?d00001 diff --git a/flume-ng-sinks/flume-hive-sink/metastore_db/seg0/c290.dat b/flume-ng-sinks/flume-hive-sink/metastore_db/seg0/c290.dat new file mode 100644 index 0000000000000000000000000000000000000000..4d10db46799e9d59746cd4b6c1e17c2cf3a553b1 GIT binary patch literal 8192 zcmeI#F$%&k6oBEEAhn2tp1{%7#YwNAxO5Ql1g`#-IlRrIyyRnZW?ltk@ieG4FLoYKmY**5I_I{1Q0*~0R(<4FqA=2 IV18We?qDStyZ`_I literal 0 HcmV?d00001 diff --git a/flume-ng-sinks/flume-hive-sink/metastore_db/seg0/c2a1.dat b/flume-ng-sinks/flume-hive-sink/metastore_db/seg0/c2a1.dat new file mode 100644 index 0000000000000000000000000000000000000000..8e2ed6afe90b6e9bab82029f98a338a3cf455bf8 GIT binary patch literal 8192 zcmeI$yA8rH5P;#cizFp%Kt~A`6*C}eBo?3sBr0eqk?5&tnSv>p0j7Y9ACVIR5-s{I z%49v}E?KxD-)~u`K-V?%Zx)KyfWW literal 0 HcmV?d00001 diff --git a/flume-ng-sinks/flume-hive-sink/metastore_db/seg0/c2b1.dat b/flume-ng-sinks/flume-hive-sink/metastore_db/seg0/c2b1.dat new file mode 100644 index 0000000000000000000000000000000000000000..2a2969247718ad8c76b895d72b43a1f781e3cbd9 GIT binary patch literal 8192 zcmeI$p$@_@5P;$9mTZ$e0uBoXg~t&L2Ez*=fdq#^F2VB#I9>*g1jie|tzDOH41y+q zlik|8ZkK&-m5qoPy^0apnAjKwUk`ISf0m<>{?NxX0$OL)Q~p2S$iQ4&HaeHpSnoyqnM`C})($o&5?@_{hyVfzAbbTe~jZ7z9oJ zP1cXQc9;F!Dk~8&`s6LLePa8t^Z78Q^Jmd&;g8FSW)}KnA}znkXmMSCZH?uR!&}## z3f^S{y{itU9~5!>D!r|lZ7MMq2q1s}0tg_000IagfB*sr{G-6`ur{*PFY`axk)5S) z-|F-AM3U;2!B(>JBA{_rp7Q_sMh52WqS2|y#yZAYek>iC7P-BS`ISU`aRGV+5I_I{ a1Q0*~0R#|0009IL_*q~ibGNsBpFLh?bT1nK literal 0 HcmV?d00001 diff --git a/flume-ng-sinks/flume-hive-sink/metastore_db/seg0/c2d0.dat b/flume-ng-sinks/flume-hive-sink/metastore_db/seg0/c2d0.dat new file mode 100644 index 0000000000000000000000000000000000000000..60e3e4fc09f7c3c926027a7014a78e090ff9136b GIT binary patch literal 8192 zcmeI#u?@m75J1s$gd&3|vH=AJqGJG70Hs9~bW8w9bS%IGEQORM=%}dSaB?Ivp`f8a z-%&2ko#l&P7uku3)B8D+5YrHw!RpPN&hOQT(jQ-Qt!&C_W{djT+}q1H&#PZ7qrIxn z99=Z;CL*sUE&GO^{?Jk(g#ZEwAbKU1x>>0tg_000IagfB*srAbIO*cd&_%|}MI<$JIe0=ER`ByJn?qAI_j$H8{6YcpV!Ohv8 zl+pIj<>$UzHK=7BM;|;0IcVbYHni;--);C{fB*=900@8p2!H?xfB*=900@AxY2R*?OhF=L<^3tMks@`>1IV@J3GgyXa#t zJento;ElH!1$0OUDo@)&sVPY4SSVRJBriGLQR3BDQs# zvP+A)t@`{-c1f;{>FO?Yu4_{_%c{1n+xcW3*Y!trCxHe55C8!X009sH0T2KI5C8!X Y0D(Utz!&^;ZuDgJFR9n4Olt0x<(HR3 zzqD7jZgjV}8%v)e75n;{$*v+p2mu5TKmY**5I_I{1Q0*~f&UWNpSDhGJ5zYUj{FyZ z@7M8=FCD)b+mk(a%fpR_-)$f}$Mf(7jh+<3 literal 0 HcmV?d00001 diff --git a/flume-ng-sinks/flume-hive-sink/metastore_db/seg0/c300.dat b/flume-ng-sinks/flume-hive-sink/metastore_db/seg0/c300.dat new file mode 100644 index 0000000000000000000000000000000000000000..2053e0105b40818ea4cc1ea3a6c1cb5edb9972df GIT binary patch literal 8192 zcmeI#u?@m75J1s$K*2&3Y>^6y0a$=0qM<`c2CTypbc_HOCr5$_1qBWI zw&I-cEIWQ(WGf<0pQ0sUOvBh5tlrFNe6L!R{y3d!WnET>wrHS-dz*aozWK#6+N=7^ z?5cS;7I_}hvTr!k?^`OQ5I_I{1Q0*~0R#|0009IL_%DIOY3F34ce4NcBt*TpTUtFT zbW>fQxl5hy6k%U-nQPBqdcBAU0R#|0009ILKmY**5I_I{1b#0tkx_Zb>Nq_=05p*m AuK)l5 literal 0 HcmV?d00001 diff --git a/flume-ng-sinks/flume-hive-sink/metastore_db/seg0/c31.dat b/flume-ng-sinks/flume-hive-sink/metastore_db/seg0/c31.dat new file mode 100644 index 0000000000000000000000000000000000000000..813fe3b79dcc587ee2ff25e01832a8fe6e691e71 GIT binary patch literal 28672 zcmeHPeatRJb)R`Zp6}21%0(@+K<%SQd^~pf`TGnUOp65Y>D-c#wG-fZBi2x656EpoHMh}yZ3$H-I1jI z!}vUK&pXfTIrDL5c6WZiIrFeXj4=-X@Y5yPAz2sI^?Dwpe{4QGvKzmi4_7XSe?7)# zl@f=_;d|SM-@4b;tpm5bP3PCjqKwnO4}UsNt&%Iyi)Kf}kfnQimXY-}$ZX(9?Wc6p|ft3eV z9$0x`<$;w4RvuV+VC8|82UZ?fd0^#%r3d)w>@4{5X_EhelBbkDn=n6IdiQ2$UdR9Z zM)nHG{-MQt^-|c zryih&H){hSBkB5y#h?c@{H8X@Bd8&?8Vp$c7BX<7!j*1hHJGqCB?A$PT)L^%V2i~$ z8F+pOeaLMFedt4B`cO*c7B;u`p$~7>eaLf12^iycxAvhAmuZ72RgRF3#b7XG@!NHi zd=V<)OPj%r#lK4iBJ-pp0$UA3==337gG@t5C>tv^oU`~!(}&P=L}D`-jahs(8F(@g zN?2#>7`*fcx=9&OU7iCsSnF&ZLzAx2O>*5*daiAHPkJoAR@YF9NO>OIlC3pNph<_x zK%_Eq{LsDKvYCk^t#^%UD+`{IK< z{uEg_o`4%7$JdGGb(<_+$>UqdLKLaYM7}+AO%|`_@vXX!G!u>}w;k1FaUPF9M;5#8 zx+aSYd3>9$BMXwuFYK*)J&$iUt;&PUFX0_(>G>OY{CRDWNJj<^yh<$=Z{qP6$U>+x zm5JIOz@~l_JibHMp`z4DBKvr}jmLMAh3f~I%!R#GZ|CtBwS^J|SZv>|O|80|$6q1~ zFO+_kBi(Q5`4v3AOV{D13M}m7aV3wxOcpRjU?G!jKh~OBbrp~A)^!Ad$co%Pf>-nS z9&J&;5EghtSo-mYJib@ADoTP}g!Ve#$K(5S9a%2m$pFhHOC8tq_wFu0!DED%GpWnpeW7j%JHb^Z09I;la8%gFDxZV6(;Nc>D-ixH9mvX!&Gswz!?gk7|n`ceBE;?x<#q zFY);6WFbm7lZ9X1wapfH^Y|OOj#MSG2<@}v7>mEDE!?1lM~S^v53=|%vhYGDcZIOe ztFN;7Te^-gRqpmOvbi7M*nymWX1?}*zyxFC4U{}!u-m)#W|ZBFF=zb=Ob@?}>Cx|? z>^w#J7hwLz%_#d%V9w@`G2J^A(`zrn^yYx%$50M0$Nb^TQ1;Iw`5Dr@3$J!Qg6YA2 zl%v0*^uMCy|AklA{uAZ;ttdDC1tsq=h#v2K9(6V!#hh_Kwr8X4UyHJL7-c8HtD^y? zhclF$mtju-B)aO*)yFWsaS+qPqm+Iprq@1*l0Ss9dp+f!Kgo5A6W%d74*zek zm&28h=&Ln3sHN-43jdqDjpfeu<}%fa|DVAgKHY9cA6{VXd;VR6|BXI8VB7U8{BQK( zk@3nxoFJ5L zvJD{`^k?j)TEFl->1I~_XR%))eH8?Oa%+%r@ACTq-G5bgz|Vwot&B*2#NwCh76oph z1Tlndp+o%{i+|1308?RmP0p-oQr0gqtKT?$5{=4M}32?AuL1b3EK@f zfq{D?8ALEGGS3>+Hi16;hW1hfcaNV}tMaD%VG7gxVr@_WwdA(O^?%XJ<5bK3MgthQOLPsTD1g7)<^%Q~=^50ZIr$_~QU#H%4|5o}-=Z2+ zSwsq`5X(r$=OhrhBQX+{9cMtnQ^|i*MGU%_{wW>4EN?aOrnRo{JJ<#{t21 zUxw-4pHa>c!n98!YVH3Jb4G_SJ$ePo;dd$j64JjE<={Oihv#7a=INw=Go}9>)16yL zUPC&c!u*XA%kRIKa<0al!6}&Te;MUE;p*PQnC|`_$$v(<@f6CHgZaZ})71{*z3adJli~ko5QtBYiy#bPG3{56)}}Z0U~ce#pu8w^@FnNKt`O{fMvJ^eR-VoX zNY|AVFTlrU|363qcxgGds|nVbs)~-qhfK*pc&8K&w{a0@N#_MFfp7 zcuaal=u$kw)Zl;I(}8EHt-swZga3mf3Cc1dyhr7Gi&7};K7N+v!>~=YKX>W}3lSB& z^fzY)|A!uIp19WKVRvfqe+XwQ!2P!N_j(5ZhssHmn_B(3&*1-1g_+|BTtTit({)gEQcvxEJ=RSk~HJ^nD1BmT~>jTgYoRs(l zeMZKo`M*A8&!zdWhklVK)ZgRu$L#0!)-M)mj(hXf85~pogq^fY-&+&{Hu~VzjLkdl z%+C52dtQ@%;^F?)4*ePX`6m6e2$7yUzWh1+g_=Ia6SeU{yvt6m>*EdEs^4R$)b;84 z({e_M_t~j+eHh>N@(1h}>-sQ@+x3U+w3QBC3|{ zP4A0Qk1e!*=4QY_Sj(TVmzeedW)$#1YbOZC19paKj}u7P=(OpN3_lo@5PsuX-;d)h z=IZw3D$P7AqMMAFFzrcw#|vyod@^UA)(2pg!5h`m{%MCvtuOpU`en6=*yR7|komg( znXe>x$Ws1n!~&!50&HIb(x&p~V;1W60CkrIrLFz*Ia69cb20$!6~JmLe{0AMuNDBA zlgXqqXvRK&%?+^@!j2{&*tvs;uz)!aax`bvsk0Yk-93^>(J; z2xad!lwGP9A_&mx0OdSISDz$FIdyE`qZ0TLyxQMS@*qlHk_1zLR zY{^di|Mb1zE04qfPtgAVHM?d?(?Rqd_#aUJ_74G=LR$C#pTTK*djF+aSxUFC;h%or z;D5mW9$*^U?+q6jPzMXx-^Aq|;sL<_u=@&lZ=>4(-&?=J4@>>^eFp#Ut&hv`9HGi>I;s0Iw#R8&F34b1^KR5V)5B)TXYV9%Q zPYwRxLm$GpDtx%)Q`Tk%|JU^4yhd53(4H_ucyRoAjlRMEHGM#lKE%T>$^s+m=mTr3 zOd)>W7#jSaIe`!7HK4(*_y3z*76iDTA;i|I7{khu}#Fq4ulAT+@1S*fID& zOT!>@9czDw2LESS2HUpFrC(F|{jtISxhSALa2K}Ylm6V``5cy_$_;Go@AVCy&vOq* zzR;#WoESWx7qAWPx;B2QJ2H44&ILFMg8eP+?~M(fFT%jZN03FoJ2QB`C{j3zV~vsT zjtxF9oic?pCbsg2eS^9fWmabCn5}Y*7w+e4(QC2whVC20wYDZmzx1o55?y&N?5bvk) zc3xx++oC~6Q-inD1P+{uT3Gnr?HP9s-Uj*?Vi0n?-`mO`_YL08T~$Inyj6c_ z@HXt%!Iq%)FwJ;u@OB2n1?Pb(;MU~-@yy_D@P3kcu1$Z^HFz6#12YAP)zaTdg|`bJ zLxC@_`F}DqcstKC2k^g*@0d;v-p;*HKs2-!|MyKiR2c;+oCLD%V9y5ZOj^%C%ycLN z-^TyWrY3$0!kHt`x7}Z}89STGFLPfy0TQ^a^Jm^O@m85r!f7w-iL&{C{jx4!mMQQZ zupntKe`MmbfMj=7=veh37%HX;;Fw0LLeFNfHR16)v`66%ld7yd+Dt9n%Hzv*AAq1z z5v~!b56u>z=kXP~4ks-k+`Ss*W{WR#bF>fQ0pM6fQ1Lv?7WeS@ow`*DLU)SgDb8k# z2ddM6X`187nd)9^ws?rezfX0*FI{C3L^oJ_{xFNL(iSP6xy9eyviRz2EPj{nhg(7{ z7(Bde7LQej6wAy5s%N?9)!dJ7Ri_jSCx=%31c3^>4>s>YI;)KT7Y}H}J|MgbSQK#S z=sH5zPowM~#GK&`nC=|Hbcb>Vgk48(p#1%WW;-Z%&t7l@o}i88PYnK-P8m921MSCn?+lJD zW)PLV#Bb6|X#csw|8R~U){EBn>u6^7|D_kGOh{{gM+W~(nZa)bz^-8@R0# zVykBe02u}jEOM*>w`T$XWa9hMTfT-HJG{j;0U*;90L1b{Uvme}F%ba11K|ZCwXI0U z9TNfIiyZd5maCPf2F*H#0f3*iln}aYo1w#jVE}w7fF{Eu0<8`mRSW=n2Uhsi`@N|> zlfGd9d>`&$VMVz8sfhsaLl1s-LD**8pu=?pfFJr0=3Qe5n`Y3tT!#SoVF+8eweaF5 zg9{9206U1e3aoMA^S{0vqz5Jh6~=(uM%13$}rKcFKCIz}j1$O3O^`YdC3Hs^*j z@N>s^Fp{W!#HIri0|5M?3p2>Jq@Kjkt~h@Lk}*{3YUOzUAM@br8U=TexY)UBYz~fQ%rF2I8WYJ#I2cxuzw&!jUSe zg_}1ST*`GE0GuPw;s0$Osmb6Pt`7=9r(p*dINfT}H5pvXbtHgSCbO_g|0c-We#nXHx!^lz%&>``@RWSCTvt^Lro0bob9u z_U=Ww_C3sByPf1=$|2UEj_n7}!F+yDslh- literal 0 HcmV?d00001 diff --git a/flume-ng-sinks/flume-hive-sink/metastore_db/seg0/c311.dat b/flume-ng-sinks/flume-hive-sink/metastore_db/seg0/c311.dat new file mode 100644 index 0000000000000000000000000000000000000000..f60c260f85bd9b0a9e1ac4fbc0967fb1521015d6 GIT binary patch literal 8192 zcmeI$yA8rH5P;!x5`~no0Ud25Bq~+_(au1>-;g8#yW>(rWk*+^vw7BlSw$}3J(Y^1U z3f^S{gPTq!4~n>Rwb9nhcC{EA1Q0*~0R#|0009ILKmY**{!!p?+!|Tym-!#;$gVnX z-|F+tMB33SgS}+cML_GSddmOj8yT9bi^jb&o9G;C`MFHBFUo8&#oCjH3os&p00Iag afB*srAbVZcRFqIrAO@g9)UwmeL4+bQhZ0=*mU9p~z@+^}kAM`^)&z^QWS_ zTwrk5&5WWZzOJTS&-_q};XnWa5P$##AOHafKmY;|fB*#kk-+J^H?q~2g%5V(tnBTr zA>T}-nT`yOl2?O(R;xM{{`p3R=IW!JewnqS2RjRx%d*Va%{ER7YatMT00bZa0SG_< X0uX=z1Rwwb2>e7~Dr;YVTTNy!_+>2u literal 0 HcmV?d00001 diff --git a/flume-ng-sinks/flume-hive-sink/metastore_db/seg0/c331.dat b/flume-ng-sinks/flume-hive-sink/metastore_db/seg0/c331.dat new file mode 100644 index 0000000000000000000000000000000000000000..85ee72b31368516c6de9fbe3516e0e3229ea2755 GIT binary patch literal 8192 zcmeI$u?@m75J1s$kdl;)Kt~BQRLp=V5hG9o5*-bqVgfc`3T8;b1Qaj@ob5PrLO`NL z-;y2s&dJHIi)AAsM!Ri^8&fyNL&(Q5oj=Rg2!CA8G_%s4i*&;#qs4XowY8Q%j~_k% zR`e|w=->5WhEWl}uQs}x`K}gYfdB#sAbHq)$ literal 0 HcmV?d00001 diff --git a/flume-ng-sinks/flume-hive-sink/metastore_db/seg0/c340.dat b/flume-ng-sinks/flume-hive-sink/metastore_db/seg0/c340.dat new file mode 100644 index 0000000000000000000000000000000000000000..d99b11a3f2dfe2333989898bca9463dfb265ef5f GIT binary patch literal 8192 zcmeI#u?@m75J1s$K*mB8Y(Rx*8GwQbEE)>L8i0goSb&C>ozfsN105s4;lv~|p`f8a z-%*^)o#l&P7ukr2(fc{FFxxQKr&zD&G`^Q3Nq>BuX=PPZ`*BfEo4fe(&GY6K%Ve+e zGe=j=o3Y5NNyEOOr`tDFNFjg#0tg_000IagfB*srAn;!Thtt-`TF(^!@3Rnf-cA|o zNuh0Z{lJ{7bZe@y6)$tWym-^CYD5SifB*srAb%c`MXo7z5=i5{uUv--&9MZ&vF5D`EC d0R#|0009ILKmY**5J2E3fuStizH|39egYK>F4O=3 literal 0 HcmV?d00001 diff --git a/flume-ng-sinks/flume-hive-sink/metastore_db/seg0/c361.dat b/flume-ng-sinks/flume-hive-sink/metastore_db/seg0/c361.dat new file mode 100644 index 0000000000000000000000000000000000000000..b5c8f259d0d1fefc112f60325bfde24724aded2e GIT binary patch literal 8192 zcmeI$u?@m75J1s$kdl;$3Fs)HqM~2|L`_Fa4M=n}luW<|Ou-be0fLqV;B3c{69N(~ z`j&FB@0?h9T~5{_V)U{jaWQo<4xt|Bbp9+xE&Xvh)yh(@T%;W~89iLrUq|Ed$Kk!} zpNih)61|(=%pf|%@2f<2X1=ZDSRjA^0tg_000IagfB*srAn=a@`@_b_O1~`pU?*;t zy?tw_*9&Q;ErXrpWh0<*Rz4N}^NsY)*=JkxqH0?YwihswSy36gj5Z}sK0!nP0R#|0 a009ILKmY**5I_Kd9|T6S@Vl>_%i{&Jf-Z#s literal 0 HcmV?d00001 diff --git a/flume-ng-sinks/flume-hive-sink/metastore_db/seg0/c371.dat b/flume-ng-sinks/flume-hive-sink/metastore_db/seg0/c371.dat new file mode 100644 index 0000000000000000000000000000000000000000..ad11f01b25b6bd995142e4d731203e66b5792409 GIT binary patch literal 8192 zcmeI$u?@m75J1s$kdn|b0v)1)ii&~>5H%AZIwU}%qoHI1HedoureFq`0?u|EIUyj? zqHie|`!3EYuZv|XB1SJe5*Je!;}Gg`PUp{J)Y2c9Gp(%k%0)WiA*0Q8{dF|9KaU<= zzbkr|OZ4t~GsCEff36Z;&wO9Wu|NO;1Q0*~0R#|0009ILK;Rz*j;EcGjec49!A{&P zd;8Wiymj?ljvvODX&o?qKH=k{-imE+&uS5XUFRwoxni0-h8-1t!_e7xbcD6r=>CXGu0G2p(1NR=oQH`a<4)15ciO z1)Xf0wzTOMuVr@0WOqLHzxyMXB=7_P7F|`_(zdPbys6{!J{tc{+JbxKG^NPHIC@Yg zbxGW*CoAsD8=}O^-p|cAf4bIWo2TF8w-8_&XjoTkI-`aNyO;nIU;<2l2`~XBzyz28 z6JP>NfC+3Cf!FtE3m)lA_4~a89h&VgowANE1h(5$d1!sk-1dmZE$G95dY?LUT5tl# zSr%mq6XBvKb2swx&{cs5U94K6GLa=np8?wG;QadPf>yB4(zGfpR1w4&N@Jl7Rv#QK z6#B_9#>AZ8BqU{7l2EW-Xh#dlOi23Efo1>Lg+2=@1HxmO8UO$Q literal 0 HcmV?d00001 diff --git a/flume-ng-sinks/flume-hive-sink/metastore_db/seg0/c391.dat b/flume-ng-sinks/flume-hive-sink/metastore_db/seg0/c391.dat new file mode 100644 index 0000000000000000000000000000000000000000..f2c4a79ec481c2ac609799bb519af085bc822922 GIT binary patch literal 8192 zcmeI0y-LJD5XUE%c*%W%4KbER@Sfsg(+L*H|i8`vw+1j`#?I zg}#AKHpvmzDCq?G5A5vhO!mLKzf`+)MMN66+&a>CrLViHUe9s-S(=jmaTsysf}2IO zUVWtT;ynJ<%%4^p)k8pIw3=1j^EGm{=OU}$msNv% zoi{S4BW}lKbt$t%#s>$c!7l~fXuf|qJD^S4NR!kLMc~=O@nU=8#DSgoZeWYt(q!Q- zb_K1GK8>PW+9XR6@VqQz0l%eyMLk6z491xd6Q`v>kNS#0;*HZ#_-RXlJ`EIs*hzv! zgjq|00Sy%aH+5qng0Q8)u=p|XAOHd&00JNY0w4eaAOHd&00MtZ;E|?$oj;TQN&21K S)0>RnWxPk{#V%(rr*~gcd3yB# literal 0 HcmV?d00001 diff --git a/flume-ng-sinks/flume-hive-sink/metastore_db/seg0/c3a1.dat b/flume-ng-sinks/flume-hive-sink/metastore_db/seg0/c3a1.dat new file mode 100644 index 0000000000000000000000000000000000000000..0040fce9fedc49b500026976e044b54d2bbdb43e GIT binary patch literal 8192 zcmeI1y-ve05XaB0l%!u060d-OCBkjKYKgfcVrD=}LNhP}5?dEWRu(1}o`a|8GqA9; z!TFL@Q3=fs|C4+<-~Amszs(oABO=AaT2H(qzUk_EGsp2)Wn1ZAr{`R`;o%YO)gr~4 z^L(edZPiKd^0>@5-X)2ncYY|{*ra8>?CGsqDoB9<2!H?xfB*=900@8p2!H?xfWY4( z@c4YM=$5~!zTbOfuiBFzUs>0;3y#Yzg9q|9jeyo@v#P%5D>_l@GTVQts_~h{*CmpV{l(`uRpMwi literal 0 HcmV?d00001 diff --git a/flume-ng-sinks/flume-hive-sink/metastore_db/seg0/c3b1.dat b/flume-ng-sinks/flume-hive-sink/metastore_db/seg0/c3b1.dat new file mode 100644 index 0000000000000000000000000000000000000000..f970c9eaf3740ec1f584aec8f2cab70e19d03646 GIT binary patch literal 8192 zcmeI1zfOce5XNV@1h{{R4`I}XxE!#DiO*oJwlFzZOte-LyK5{>tZeKo?0o}$iF*d$ zLWcn^=7@L2w_Q_j=Q#eXZ7coba>11=9zM}dBU0R) z*T0(ARh>F#2UWiHEJ^Ht@T7FBNymQa>90B}NPz$dfB*=900@8p2!H?xfB*=9z~3Tp z|8%S9n!l+z-}_V^wWr^{vZ-$uT#v5|?#SO<1awB5UClXP(V<#b*}-dFEw3INevuP8 ztt%^@%f`+dJI|p#+O@e&Qf=ZuhjA9n!^{ME5}81kX7gpVJkm5E<&iJ?I4%m-Cw+RK zhN4f+tk8NM_UVT-5`7-eY^oF6rytQ+^jVmjT&G2!eoPb5M>fi|Hff)J!h~QT00JNY s0w4eaAOHd&00JNY0{@A?3-Nz%WyGJ0cS}!ei613CrppS+`^Cfa7g7XhtpET3 literal 0 HcmV?d00001 diff --git a/flume-ng-sinks/flume-hive-sink/metastore_db/seg0/c3c0.dat b/flume-ng-sinks/flume-hive-sink/metastore_db/seg0/c3c0.dat new file mode 100644 index 0000000000000000000000000000000000000000..4d061cf0609ecd4b111d8f763b15c5658bf6caa6 GIT binary patch literal 8192 zcmeI#u?@m75J1s$K*j|HB^?!_WdLS~XebbC01_PwFagWZ(vl_U7$F=^4k8l@DkStB z#kt&BzW8;Kt%x|iA0vyik8^XfdNZeSS&t=aq3P25}@J%KlH^aKu0ZZ4idH*cWNmXd;K zoXuatqkXUN!nc7eM8xR%8j+2ujbZThFsJiJF>2|L%aK-QdgdZ+zsYEGU4I>o?RSG) z$K4e?%O<*4y_jCm#OV^ z%iiAA*XxBe(_aQ_$;%%BjkEGD|DA87Z!RtypNndu*I3(+?J+5;2b&d1?k+(@009IL bKmY**5I_I{1Q0*~fgc5iGIcwTt^MfxGuieXjZk45o7=7~Aa(&|ZxC{9>rt@dfYT=K|v1aD_WFqab$!Kw1e{GHBkAr*1 zpNih)0^OSqrWY0Q`zpPwnXM}^76>4K00IagfB*srAbheKZoe|J&@T%=*ey3p z-@Y~E>xm@QD}#+>K-rbX_gV}2!JFWS)%KmY** a5I_I{1Q0*~0R#|0;1_|R%>4F)Jhm^H*DqcG literal 0 HcmV?d00001 diff --git a/flume-ng-sinks/flume-hive-sink/metastore_db/seg0/c3f1.dat b/flume-ng-sinks/flume-hive-sink/metastore_db/seg0/c3f1.dat new file mode 100644 index 0000000000000000000000000000000000000000..08acdcee31e218c5d9c3b4f62ef38fb0f42c8e72 GIT binary patch literal 8192 zcmeI0F%H5o3`LzHwH@IGY>Y55aR$W95tsqN#?p}!a05=kDL6wJ7&rkqNmHezAhDun zDX#M!*UH;s*@}qKWi4?&bv||>A8Q&vD{ILgw+qdzb>$-caL8zL-G8;t_UGws=&PcC zIbl@l$4sIo{#?Vmp827nv4H>tAOHafKmY;|fB*y_009X6BZ1>-Z)Br43lHqXMcLb3 zL%y9zGaVTelGlTP&Zw>m?|dU;bM2TM$;ub(?#18*9@Q?- literal 0 HcmV?d00001 diff --git a/flume-ng-sinks/flume-hive-sink/metastore_db/seg0/c400.dat b/flume-ng-sinks/flume-hive-sink/metastore_db/seg0/c400.dat new file mode 100644 index 0000000000000000000000000000000000000000..1e8976f89fb63a0868c737d9434d15ecfdedc47c GIT binary patch literal 8192 zcmeI#F%E(-6oBDZLa-qY-h#=*19*nUK?mdR3Dm^R)xi^ZCmCL&Ffo_&2!d+4ca%$v)#Ylsj)009ILKmY**5I_I{1Q0;r2Le+WS6z;m_3Z_e Cb`~Q5 literal 0 HcmV?d00001 diff --git a/flume-ng-sinks/flume-hive-sink/metastore_db/seg0/c41.dat b/flume-ng-sinks/flume-hive-sink/metastore_db/seg0/c41.dat new file mode 100644 index 0000000000000000000000000000000000000000..88245f03ee1ea1e2d76b88887d765eb26a209a34 GIT binary patch literal 69632 zcmeHQd$1%|SwB59cV>FtGj}&jl7%c75=e4Qzow^02nZ5A_eV&_cvg3M`7!?>jv`cOE^P zd^26vhJ3SizwJ3a^P6*i=k(+Eo$q`tP75Iv_`_F9+PJiX#@|0!2HXGJdyN!Ne0vo3 zyaxV~LKOTDDe&X){z5P1<3W30{q}1=>Xi>*$%3oD4gaZ{_al*@??RdPrNSF0?B-%- zfEi#0m;q*h8DIvO0cL<1UB>Y z4;U3MkBNCv1i;0W`r>A7bEm$xTsvtJ;3CC7Tuj^$6MIBy^Vo`B)(ov`XnNJG8oD-s zr(yMsg{J9r7pzvh(W{=N8!cJAc)E`_CteW~mxw)3z0Iw~t=ejBZL3#9V*v4K8=UJS z#>I^>`Ay5JVg(m8-D)mW^rpSgDw{Ue1!~u7n2jo>&T6**3kl)6<0O=gvO@t(?E)r+ zyBGjy8YTrWSLoMJg%QOsW`G%B2ABb6fEi#0m;q*h8DIvUjSML77aLIThr}_E^D(gw zV)_Bxj+~Gso8l;xe_u4=GcOmv;9qcvY#20 z|GmA5zRube{ksYtRK9yCiVD#E7cbZ*IZ-xd8!>uiW?Q%=k^j2uVdxYEdfZLj>>Fuz9 zRX7R#yacXTPFd4Cp#iJHeU%5?vS1u{Ljz6=_tzz`W_KEn(QJiu+RPM%`+5niw=LT- z+F=1_fr{Ufz-HHg?lZ!oA_r95?bX<7wk^{Hmv1Ou*IZV(zb%1{s?&DNGB|?60nQ8e zj0AS{N<}k6JGdhWh+h*Gtn4h@tj0a4>xPPIzpOMHF&2G1xuHSpJBR{0LpOeUT!!Vqx<`1~p zk^fS0KQECDtI=^fc6i8tt+-$C*f3f!I4V|XJbzAczv$Ju)vjor{&77!8($FaS&wI{ zV^<9$JT|^4+%I{^nxVH1!@segz0O|}?h_K(>a-g=%qNA$^OuGDWe>S+J8i=eHd-}KAeY6 zy%nu`wJhI4oB9_liTwz-Gjgsv^MEY5A4_6&w9)UO%|@_1dn?-LyJh(+(84q%Vty01 zljqPTifBQ3qdkpQeIwfRf8!?;pOM=xZb$wZx8skZjl30Y_7VA+J7kH8R{55+zlkh|k|QIOd6A9w;vswpay^us$OVE>=v49|AC;jLL!l4UNRPZIL?;AL zkVZ*Xyf7-BCoTbMH`cee>T9*5wT;zYC9aJ{ zIwIS1s(7(K05{fG{8$rycu6u@C@3YD!7`bN~dUbKyFFz2CWB zLx@bkm5*P{05iZ0Fayj0Gr$atGZ6VFaTrwkABda4r|^Dp2<)xW{+p;l`6+4lpjDm} zOW-4TT-*%yw6xcujinGF_IJ29_D*Rx(9T^hAZ|YXFmA_xS=#HR{UMes_X>#NSJs6K zHZJN=v&W=8jlaID|Nroj+wLEn{}%_O{yzYp8Gb!O{jc^_f3&|fS^tL+aqnwo5V}i8 z$T#Rw6=i=(6yOfOhd|c3(IhJV9>)60RQHmxc~t$lVgTylV~}eSE1>cZLZRGS$BHQa zgHR~9*Rc|+f6og6E!tjPBX^EcLhbL35g&rgiIPF-9{@t3M9GSx?;ToNUput2zFOOW zbr|#}6+|g2E*1M^KMw-Y8iYoT{QWcom+}c>I<60G>wu-3Y zk^#c|Fl3e@aj|bA5QU>iLOgFG5QU*gQe5gE7|Y;3T3e>GlM|QuW7>zHGm{aQ2ZE4T z$%_5{8F+bnb@i0q@9CDNo3IcI9j{$yq1-H+3xY*NP9-wMX?Or|5(~j;O|0Y0sQW+{=fUF_kDLr|DO<-!~U1S-|#Co{I>sYWd46Q z>Oc83`da)U9#v17~{pzDkPMC9Kdj0bV!vz z;H8MpfbqKMv=)IkArOVsqGSSH0poQsNG(dL=miKsF(|ww)93&QKp~BnWEQo5017GN zLo$!je;mc6@lr(HAAmyY>Et|${s0sbFKQeWe*g-J7d45Ze;mak@uH?sEeD{Gcu_Mb zm;+EqXG}FG4g{jmQjQcbCTF}ZWm2*gwS@6F0Vt%NPN^7+6M#bECFSMd?O$SfgBf53 zm;q*h8DIvUEes@9P=SM<0~>!_+Sf^Ip`H7#v=2zzly*Pb*f*qoyR>vA zBqRDCX8m^)0rn6k&;N&$($2f%^5Vnmn_KlI@`M`1Sbzs$w>CW#@aq=^kM<0M!n*kJhZ8GQu2*#nzZN;+a2METY%zVb=f&g!6gKQVMpA?8E@Ix(n z2xL~26#4+XaE(C>3L{DyJpet7jryTOwGDDxVo6{B!&vb#$gPRa`}%(rh0-3#;@Q*x zqbQ{QO`!hw!sUCavz>^)!xk&|4izxb2BDr;Nh5Fr;l=}Ve(j;gcr}eWUy}ee41qYG_LfcaboY03Z z@V>sYya))850wCsP*ISBQ?B8(Eyw7Dwk|)761`tF=-RFNR(*YqP_=9U^?9EhIl2g6 z&!=&L479A-ZFe2J>aWM0HEU`yjtgesz=5%{b9AGAoH}5NNn9=i32|k2BLYNH9I8>< zS5$Gy%;n-z3FC!aO%ImE#WQ#mL8I^}nyO(rTtFkoQpIlBU4Q-T%#0UsMGfG#EL&?q zW`o`u*O_rkxWoo7ao*u`)ArgDafp@@N-sSNG*`D^)puyiZ~shjcf8?iSzX8J_*o)m zUqJomirbLLZO70ZryCyfN0nacm$u&3VA;NZGMT;3k1B4<8v1CDa zmRp9^32_1SClt3M>)dIYEvHisZ*l!&N-tqe*L18_r|f69n4Rax758?JXT8#e^a$aN zjnDeIa3FV2vuc?B>3H@!e?_>zAnV+4I?c9WhL>vK+_0+RcmJ;pq>tfB_{<#I^q-J+||lEQ(s6-C&>~i3?hJv9A4SkI=NO~_8!lG#PuHbaN#fn%q{d)^c?_1UpP?N z$jU4NFuT`B*pV3?XzDUm>_rbC%GB~Qn?>(|DiWIzK2EGa23-dJS83u>#tERpU!JZX z(?xUeN8cA~Z{DO_7zg~b>&9x_=B5qN>vKY(+5ab>%x*1cnZ+?0d@s|H3V+Rds?nrM%!7z{CTN+2~PTf{&J7>m271lXdxoN@^a zg@D8&yEt%a!EUB9lat;T@RDHW4R}mUed`p}4Uj9MqXilT>zocQu1xI#$j8y!0&cAO z@=0h$c`yOUg-_Su49TFYW%s+3Xr_D?-6{R=l6-&p9C}jV1SLtE?gUkcqZ_4notT=v zLJIvSy}>z^^Q4eQ=Si;~Q@daad32S)X{bLmXLJk|3g{!jD|g3gXm%5ZLMRy;i)qYE zk`^j@U8sjmDTz5rz%z(n-r3kDp8HEF%tr#lX8GXM9bU>|LK4UvwR~`@?k?pO&EwT^ zAl+lx&j>f`tz@aF=-w@=X*f=!={p@}#hu@y7(HCeX*HU@%VB2ReTwPf+LcPHt^49# zZ&ns6)efF{;MKI-9ax}7x`Vt^u{>a>({eP!*8sEo8{VaqC9qRAs#d4pbF%~g)dbb{ zoG6YzjC0-b*T|^*>!tRVtM2E1AM@eQ)iED_?0=;F2&&@Pov5W_*Wu@*|A{g1(MQlG z&Z8Pv--8-EdKY4<-;#C%ZQ=&}ZsLqwv44&H#UDvE{%y3W1Nd7I=?KWlTTt^S-zDwW z&_+Ip$jK5Sq~Vcl@cFai2H3twxL|8i+uw$MfyjPDgD2*}FJ`a79DbQ6Wj?>xOFzZ0 zN&7;y%2jAnzl2ttLz_N}HhZ_s{C8TG_w|4Ip0}JI(*K(i;{TuF`u|igX8}zq!{Xq>d*g=w20Kx znUp{OKY~J%F`0}%|38948ZVh#aQuIT)YF+laQweR>Z7cT0~kN19a0}XJkGKF&*(#%>9t1OiY<=ek@Rm;MK!koqW>437V= zl8&QXDlq=v)MzM~!1#YtBlS@(_nf#r<3=(A%m6dM3@`&ffebu5;{V@v<~QCkr2o%~ zYoIe4V;zzmEAa|O=KptB{qH^rfS~`Uc$j;@9V9%miE>Yzn-A3gWtwiu%QS#vceC)g zdrDpMsX+Z-rscrP zB=u9s1nPfVBk9~iE>Qp5G_6o52I_yCmI0%f2-N@ZU>VBTEv5qXzYS^N2nw~B3Do~K zZJm2DAGrSCE|WBOu^6cTZJK|v6ukc5rui33a`t}gmVF&PCG=5Aru_@3 z3u(NR3cH#67FX)9%7DCLBvQipcQ3oA4p`3kDeiUvPL+}$2XP)$@DZ_N^{G;>N{I6j#XS*-L%1#>&cg~mSvH_P z#Qyr2;+`7Efk)L6jw!6qCl!1k?Pz@fhuE%9Delh(;t<>QX$7BS8&Ds@JsP&_^TPeP zaU95uL~Pet`S{z>`T!2$%7FEGLf~qO030Ge=LJ4BcMJ#806p_FGS=s*2{-6m%nUFC zlNd-uFfJhen6$6M+4%GurM(et@(0pBDDCaiUV%38 zx6aiGbd*q0F$1DQVgzbHqNAZ?f^2}q4os05pr?nk9Y;l)hi%nywi4g??o0SG_<0uX=z1Rwwb2teQ;37m?(k*&Tge6SN|Wp8f{ z`F0}BbYyUpydDH}TJ@>$&o?qQrH}RoRo0Fk>?~j|t14qR+c>#e3xNOxAOHafKmY;| XfB*y_009U<;3onzS^N6pvV6P%AQ>&m literal 0 HcmV?d00001 diff --git a/flume-ng-sinks/flume-hive-sink/metastore_db/seg0/c421.dat b/flume-ng-sinks/flume-hive-sink/metastore_db/seg0/c421.dat new file mode 100644 index 0000000000000000000000000000000000000000..65775ee7bfbd1fe9a4e81f326933d49be9138b53 GIT binary patch literal 8192 zcmeI$Ee^sk6u|MxGuieXjZk3gY7=3JuT%Wi;?0i0q>HJx=TKMB~qM3z0nMm7jGFn{MUt44O4K00IagfB*srAbheK?yxqp)GzZt*pZv1 zZ{Oc{L6;HW?BpOap`f8a zza_u!EMNR}k*$b0?V)9HEaKQ5EZ>dk`rfuA{Bb+i%(}@A?a)Ard)s_7*M7N5?rJ}? zx@p`^M4m-@?u|A5z9&Nn0R#|0009ILKmY**5I_Kd{}MPHcTP6?Ci}n7;;MXmrR9@E zUg-IeyS~D?{&hs>8?kdI?JewM8f{y3d$X02B)(hZA@7T5XL)>{5De)RoQ z(Yst=aMzm|MMeC&TJLJ+`&x_%0tg_000IagfB*srAbfaii?* zUqil`NHaY$I7nU{1hkH-r@}tp$k5z;w6iF)_UOTupUJY!=my?>PIC1DdIS(a009IL YKmY**5I_I{1Q7T^U?MBOx_2t3FYvf6QUCw| literal 0 HcmV?d00001 diff --git a/flume-ng-sinks/flume-hive-sink/metastore_db/seg0/c461.dat b/flume-ng-sinks/flume-hive-sink/metastore_db/seg0/c461.dat new file mode 100644 index 0000000000000000000000000000000000000000..e6d98541fb79659dd82e6c495be7eda4795dd219 GIT binary patch literal 8192 zcmeI$u?@m75J1s$h#Z%UKu3#&M8ynr~rwMhLQ=`fQo6D0XjB-!;T{-1VoL# zB|G+=lT%)2%SJ?u_E*dNX#JQDF`veC{wP}`{Bb$e%u2fx>Bdb)i|hJpYb}2o-Fo4z z49z7(^F^7>_1V$#Gnr_gmsvf;Yg8n^hX5l22q1s} a0tg_000IagfB*sr{3tM%W!UMgJwE}e#Vuq2 literal 0 HcmV?d00001 diff --git a/flume-ng-sinks/flume-hive-sink/metastore_db/seg0/c51.dat b/flume-ng-sinks/flume-hive-sink/metastore_db/seg0/c51.dat new file mode 100644 index 0000000000000000000000000000000000000000..daff727ca7de975b438b4c5f74c5244485d279ec GIT binary patch literal 32768 zcmeHQZLlO)Sw3g(z1=_WruczcR#{tWX~p-v=bV||o$cv! zs<14j@7BI|_Vn}gIsNvTnSP%0v1C6Xgu-9;V_`HfR)aiVErZw3il1YOYyWv69Jv<$ zLNyZ!qEI=nR+g;%rvJQB1O-o>cIG3zymBrq*!OMtW!jZWIPR#3bv9Id6rue`n}Ie1 zZ3fy5v>9kK&}N{`K%0R!18oM{473^eKbnEZ2KOH!*Foc@zQ3I??f>eP%aO;AIk0Jf zbnlP@#9BB3$9*MeOQBE63+dCM^dj-Q1HU`al2?*TMaik~o7LR;90V!VJ0!VagX|^G zk8Sqj=TqCx?P-}u(bOyQ82)Le@MrGKY@2M54c12fvq6#zks3s0Zae90YCHaX8aeZD zngw3SsBp3@_0k!mTH6ciCP^+rs%r;PW;=7B+TPSIyxDXf7ZImISqXoEx@9e>TO@fE zQqws1W4Fi}Y900$RD&cJBQ+}1EDrLZfjTx9R8w4Y`>V}Bn}Ie1Z3fy5v>9kK&}N{` zK%0R!18oNWA7_Bh$(e9@kFZGJkFURjulM2WvrzK`;Gsib+Edm4^aW(bidRyzXQhM} z7$!*eTdVS40Jj!GbSe6m8Xi==*JWF(Ap$BEb&w z(svvX^lPmF69|`T5r!oBIZjA}neVubHH=8|IxWJOB(LX$((!!9Yph{HlFPIR+a!4d z5***oeaCOCVTUAd)FSMXB;|yeANfwuRD%T;$y|$Ik|g7VB6EBvY^=c|Nv=iclH^(> z*lPEo3u(PhivVfe&zq!{r5;)*|#saswx*-MC(#BsXdi z;643zEdso!-=Rf-_w=nuu)`va-FcH2IDoEkK!-4dJN>J?Nx2`!Zr)VG5bndTX%XN) zyj!bDkfrx%5g<#q=`?8sz4h0%2+&&(a)Lk4ea}l9KJG_w4e!+=jA(MZAY?@xg-zD& zZkHzSk*vk*HXd2B5n_JTJFfSZ0CYxk8c{xq)bvFgTd|ID0Zs1WjJR;^&~X-xs$;x?ChzAAFL&cKUS4+{Bd5s+_;sZ4Rh_%dbzDu8 zyY(2?(By-%Ri$4P%llo|s&~=k9?pnoIa~=d{P|cx9pg7>a<3laH)--AJ;p(rd>9#a z2w#rwtob^AizfH!F@BpShk2`J^E9?S`3d6M<JFORugAEPCJ*Q_ z?xM*@_;o~4>c@+|Ti>cbAjyNgjv({`*O$4fXMCL`AJt=gqvpA|Ep9W=X5hP(fx`sm za+Ccmn(rnKAzNo-9I&|gw-_H}apQR`cK(G$;|myn2P0*1>#Ho9-ytp`z5N(phH;0* z&Fk6G&2urH#Nx&?7(c}#3_-JakVWehhVR~p@qCQWVUaw|j`j!ab?}dC47HP~%sL7)F8CF8hA}`{%=AS6ud6Sm{ugD- z;pgR``&#|4F@}#H%&K^kbSwCPjqI?<3dqq3lIrm0)PGZ_B(UJ*Glo1L&t##;CtGSR^s2HDoan`Gzscz0SvGk>N1 zBeJW7ACo;=_z875KJZewwaZ7+<4+VZ_7E5GvG31p*KV3v(j5{{3qK-0$7k`}cAdua zk4d0~pO8?eJ%b^MwD2Pm>$GPuCNnMkguDpxGpCGWH{5!7&w76H7zSLhJN(4?Q{gu+kl za1Ii8tx3BL4$v#L2;cy5-#AvR0epcAwQ2xw<035rxVo=GLJBTI7M3vSZ2d(;U%FU} z0DbAzI)ojtTYg510CvkIS_CjdUW0@g{Rbp>v%IwexBD67a3fO{GO|~G!D_^ z7xWmvN0ZBWs~k79ZBIV?)Yox4O|IY!aHRZV@n)@O{ApD~w6DJ>Ghh`=ky<{+4OFLJ z$9NBmy_=X)?OlNJ6jsvxM~tVjX#E?D{co|O{g1KOd4{JdN=%i`J9usC9_N!An_7HbX=jh}nF^38E^QLD}Ye(pRRs62d|6!0hwv#oUe~-}riukS}^nVBgJGPTI zoxf`e{jZ9*$my7$$d3ctDH_k;C1+^idt}P-nIG9s*?9gw`7tg0fV>d#iv6MACAJp6 zN4TC>><|6E(DSPJfzb1+_@U7As`!!6^NRhgKNfmk6+aPrUa`ORw}qZp#19Oi=N0?s zz!Z936>kYWuZr)Ivv~hhd%p~NRHLE{{4`q1!aBwqsgSD5 zjFbwk8pYt|rL(0ltgGV+DOQykS4y=S#z71-shSncTcuo8X1q=6)hJC1zsQ@F*o{)K zDl^_L6>AuH5!BH<1>qf1vW7uf*hMMTMg2X$MQT=M#;sDchIX1~McJ&Y4oKA+&Fv`m zQ&>}oN~~VN+#^-1GUHyUT0_?^GJh_8occOGBvq?2K?sq4lwTUsaejK`#Ejoi$G^_bGxsAqgZs#ayj6C}Bk zf57=gICI_RR(+8qzo^If3zEEvGr;E0T~|&!slSddk>n~p#+Pdz=G)>n1OJ;DFi_7M zsOPsXVS1J8?GC=~>}4h0h-uyK7K;`kMC4RQIYcP}S{K1>XBz#x?K9cpZz@Js7WMaRW8^25R!ER=3V(NBghG zcsh&SJ276)qV)|d|2)QnEN-Bl-#CY5hI<*@2dQfQswYAaztsP3ckkCHzq6S2zqxp|R{yKc|A*D|_!*~)?+X3z#$g7l_1WX}y4Cud6QTcA@!LZG ztKxTr{#V8C3jOcS;ym^O`8-lL{4pAf@qbnPM2!Ed;eR2}Vmo7L$Zj+xf>XYYa z;Robo#QXLvf>|KTC*b1`J&n8M`?T;q^8Gsai9voq3vZGin zs^YhWf9m@V46iq?McaXymY7df{80F(s`!!cPgU__;h(DFCsa%X@ro?8^TnWMz3uQ8 zDkgR)Gp?m#W`{E4Ix41i_%^Ht%6%y)>sxibw9LHF@nIc9v#Ik=X_+ZAZjzRnGUH}x znRz*UTDWnuopqbE%wQEm3~LPJmDkJM_eskPG;xsGFc#lr(R@H!X3C7arDX<-1&X*> ztl6n=)#s&Urp$O;T4u_OKa-Z3o8^%W(;*wL{7Jd46D+yZaXooI>aXK3<-$%s598eR zn~kHd$(5bH@4K+NLfWMDb$qi%L$t-;lNm5j0~@G;4b-Iu>R97Mre^!!Wcn7yjUaA4 zgI>U+7!P2)1mj*7SGD~hW-JTw7UC_$TZlK0FczQx&*$*-@%gHfuj+HIvAdOd{(Rp4 zs_N!SyN7EEI#_Q9>*dac`2?%iJcO~t7-BS7?BN=O>b(8_z3eKg4h2PS-3WjG8iwz` z7-MA(0$(X`{PF+0zIO2|s`{VWxH8~9D;-jC{y%J)x7-L&oMf&3*VqydpFdRb1EK#F z@uRNL|El<&(C72m4{UhAZ}J!9{CrjXQ0Q}2{7C3?Rs2}!^Ld$nE z6#85hZwY;_ith@2u8NoX9K0Y8)<@u;k!Nd1sHeDb0jD&q3T)t=IuF&U-_)SCT zb5*=4^tmeD68ijTe4fwYO5zOcE!ln_+us=reXfe12z{=K-xm5@6~80&c?Dm_(_kLi z(>Yi;Vtt1*gZ(?%`jpMS(5r>*l|C2%&_A7E>cP~0xZb>+mhe-!Q80HS+-dgZ<%C5I7^rtGmC-i6L zhjs?D^PAqkJ3@ad;&%+8KUMKPp+8meeVV)(KhW9K>)2oHgHZqUc$g-?gp9I?0~p$D ze*S-uCU4<&6h-0tivjTZIzCFp#vMRVV?0E~&K+4+Mo@+`ZyK4cPKL+p1!6SD{8wQS>CrJMx6xpXmXPtE(ChyW?{0U8N;jQvwKPv(`OR4_KKS#xWCz%tK zps1UB$)C#&PhcYEeCB%c(WJhPr{s<&X6*K7V{_=?=}M49nda?cZG##dLKq1%Es z18oMhGGO2;_bt@Yd`3X`5YxBaoW!-~A6f`b8|k75=L;_BT;QTX!*x zbp=NB3%Uefe}zR8RkVo;9A@}KN!Mku>QGc&iQZ-GXz#r&s<;Vt-odJCuwMVe?q^V= ziC#&6FSg-dF+R>BMQ5Qp7vR|O|8E_f-c;59;FZ91EejvvGRq;bH{;!H334cKqFUS8?@pAlM6)(sC`Th$w z(=)Ir-U-TRnlcf`^(XwF3*d%;p4k`*e?b*L6661>__6R8s(5c&xUM^mBM-djWBw0_ ziSQRx@!P^*P{r?v@qdm-y|-?^3D}K;kN>OUE#WVy;=96MP{mJ#zaa4JKhap*emVZH zith`5K@~6k1%YP<$J+J`gukGQABypRRs4<^|L1t)Hx2{U0=W~*)9O<-GOHPKRjYQ`QPYcb5{TW literal 0 HcmV?d00001 diff --git a/flume-ng-sinks/flume-hive-sink/metastore_db/seg0/c540.dat b/flume-ng-sinks/flume-hive-sink/metastore_db/seg0/c540.dat new file mode 100644 index 0000000000000000000000000000000000000000..b4955609b98341d56cfcc2c70b4f4aa6fffdefe0 GIT binary patch literal 131072 zcmeIyKW|%A7zg0vCUu+=iYkGswgVkNY~26$b=4sQkw8MCN?ReZP$zK`w~^c0j-yIQ zbYWs)p@M-GJ^&jV0~-SpLSo_@@EPDbad1V`il~{&b1a|xo^yQe{hcRQ3F8n#t@=*d z`Tf4W-#^|Ru0K9k)qhRfzowr$yjHD@4ptvdF0WSKrRw30$>167$o1>jK6&F{&%?v5 z2Qh1_UsMxwaR{dl7mm0#maEAlZ+yg$O%WhKfB*pk1PBlyK!5-N0t5&UAV7cs0RjXF z5FkK+009C72oNAZfB*pk1PBlyK!5-N0t5&UAV7cs0RjXF5FkK+009C72oNAZfB*pk z1PBlyK!5-N0t5&UAV7cs0RjXF5FkK+009C72oNAZfB*pk1PBlyK!5-N0t5&UAV7cs z0RjXF5FkK+zzZ$#%Qv6?5q7I1)~3fi7Z$7Pl_w6}cw%OMV!mGOe7*MTZ1wbVRoBCf zaPyx}Sg41?{k3pDy!K9av$MN1y8Lc$+`YWf8Fkh=!)~|~E^PL8x>uszVRsO1S8LH; zXVBfgTaBw!3mXT!s$13X?XK8Fiy^ zXAo`P-P!04hS7t5wAH$w?G567vur0j>v#6X{Wux6<1((=RzJEC-QMl@MoAjC$~x)sn$c4`FX9Lkt&^Foe7O8R{B(}mu9(6pOjWvHq)ex z+j*Q7Sy7}#o*YwaX_l&fQd-YQ^`Es~p5_3x=009C72oNAZfB*pk1PBlyK!5-N0t5&U zAV7cs0RjXF5FkK+009C72oNAZfB*pk1PBlyK!5-N0t5&UAV7cs0RjXF5FkK+009C7 z2oNAZfB*pk1PBlyK!5-N0t5&UAV7cs0RjXF5FkK+009C72oNAZfB*pk1PBlyK!5-N z0t5&UAVA>%A~646_%ejK@57_2zZAX-;nZi7y7uR!u6;PE!;kxYC)}#)tKrQ9n~#3_ z`C{0u>Qe{fQ0!mNO-Hj!ZC0uX5FkK+009C72oNAZfB*pk1PBlyK!5-N0t5&UAV7cs z0RjXF5FkK+009C72oNAZfB*pk1PBlyK!5-N0t5&UAV7cs0RjXF5FkK+009C72oNAZ zfB*pk1PBlyK!5-N0t5&UAV7cs0RjXF5FkK+009C72oNAZfB*pk1PBlyK!5-N0t5&U zAV7e?izcx0-M96yTb-&lJy|`>Rn_7X=bX$Q(dO%+9vahy3$?Jk9y){0-j(Rypf~P} zy3x2Zh&J!;Y;*_1=s`c)YTeKF264Yxwv(OpJA31PoDAD>8CPwqAKi#<@AiA6R=Zuc z+F70@S(&tpq?t6MQFl1%^|#WuH`cD zE>^kL!ma1em8N+!EAq72Znny_XtuM~VXjH6#jr4q_0&|2S*lAXm8vYtI8U-xoR(=` z=5d~7$4hm@-_;ZW0t5&UAV7cs0RjXF5FkK+009C72oNAZfB*pk1PBlyK!5-N0t5&U zAV7cs0RjXF5FkK+009C72oNAZfB*pk1PBlyK!5-N0t5&UAV7cs0RjXF5FkK+009C7 z2oNAZfB*pk1PBlyK!5-N0t5&UAV7cs0RjXF5FkK+009C72oNAZfB*pk1YTT$+MV#I UI^A#k{nhaG!Aalx{Dw^aTW+nPfAI+4ul52lmf7-(+*L-&SYf2mp{S9*(3dnT@obWi1w$i%~%l8A|qbtN&A63!~AFe&4lk}8t`6D2h!AvTmG zOd@P5X)uYgrKHKEf^8)&CROMtX%i{9*j3VD;^DlKE|c^n(o3;n2F!pNFau`54445k zUcMk^{*H$s5Tl$qUI7$$iPbTLt S@=o$vas&2;|1Y^6^nL(h+Cdcn literal 0 HcmV?d00001 diff --git a/flume-ng-sinks/flume-hive-sink/metastore_db/seg0/c560.dat b/flume-ng-sinks/flume-hive-sink/metastore_db/seg0/c560.dat new file mode 100644 index 0000000000000000000000000000000000000000..b2861f8f03d7875b603e1ba30e59d2da6caf6987 GIT binary patch literal 65536 zcmeIuJx&5a6ae5E5U{YcF(Kg!#G29r7^1z0P-_}`F5v~diit9C@LTkwP zHuLx1OlIdxuPLP*t%yO{kITN@HT8Bb=KDH2=|7kEQF*Sat75p0b`_Uj7K@^TZ2Ja4l7V|nMPi~Ent5+Fc;009C72oNAZfB*pk1PBlyK!5-N0t5&U zAV7cs0RjXF5FkK+009C72oNAZfB*pk1PBlyK!5-N0t5&UAV7cs0RjXF5FkK+009C7 z2oNAZfB*pk1PBlyK!5-N0t5&UAV7cs0RjXF5FkK+009C72oNAZfB*pk1PBlyK!5-N z0t5&UAV7e?nF+kj@22S`9+m5d4a(W%*E5@Xr*JWj{WtmJCp~(K`7k}C$M^&R0t5&U zAV7cs0RjXF5FkK+009C72oNAZfB*pk1PBlyK!5-N0t5&UAV7cs0RjXF5FkK+009C7 z2oNAZfB*pk1PBlyK!5-N0t5&UAV7cs0RjXF5FkK+009C72oNAZfB*pk1PBlyK!5-N v0t5&UAV7cs0RjXF5FkK+009C72oNAZfB*pk1PBoL&jO1ytA20$xqd$YB2^Xs literal 0 HcmV?d00001 diff --git a/flume-ng-sinks/flume-hive-sink/metastore_db/seg0/c571.dat b/flume-ng-sinks/flume-hive-sink/metastore_db/seg0/c571.dat new file mode 100644 index 0000000000000000000000000000000000000000..da6611056afd030f0971522e2791537cc6096f75 GIT binary patch literal 8192 zcmeI$p$@_@5P;$97RDrxfWv}8;jsuXC>{Vq0wg#Lk|)3;cpw}J5-cwOw{~5+F$kLc zO?GSVx?T3URn{V6^vXtLV`5_%d_BzR{F#qR`r~q>m8D*(NXu_B+FaLPM{WC4|K4^_ z1@E$n&P{Ko8#HnIO40UAx1}5l1Q0*~0R#|0009ILKmY**{!w6m*ce&qm-!#;$j+K? z-|FktLekVDgPo+sK|t-Sc*_6h8|j&|%QhByHP(C9ek>E2=Czs4iNsfzAR>SO0tg_0 Z00IagfB*srAb`Nn0t1=5y~D@l_yU*jFE;=H literal 0 HcmV?d00001 diff --git a/flume-ng-sinks/flume-hive-sink/metastore_db/seg0/c581.dat b/flume-ng-sinks/flume-hive-sink/metastore_db/seg0/c581.dat new file mode 100644 index 0000000000000000000000000000000000000000..ad4a0d47bc034a370c1cd78613cfeb37f788ca4a GIT binary patch literal 8192 zcmeI0KTZNc6vn??Ki=S$9KU7F8oobM9*BcBXA%tRa(xJ(xr z7H9mg<$s9F-#(UOzbkW=6Rv;wF^xB&AOa#F0wN#+A|L`HAOa#F z0_RBJef4B8=grdh{Vj@&D!IG*zCcb!DP+sw6?WeUI6B+!OW*khb+h&A#Y>WnOT@i^ zdvudb+bL{HY&Z!6lo9s&gNOMfLIEZ;)=5!+7ClaflnNH*fIn{&sTpfWTL(N(CITWL p0wN#+A|L`HAOa#F0wN#+=SAQPRbFU~EBa68Z!q##ysNdoe*qmJIqm=e literal 0 HcmV?d00001 diff --git a/flume-ng-sinks/flume-hive-sink/metastore_db/seg0/c591.dat b/flume-ng-sinks/flume-hive-sink/metastore_db/seg0/c591.dat new file mode 100644 index 0000000000000000000000000000000000000000..34255132cb5dc0a46bbb1b61fca60f510c99edba GIT binary patch literal 8192 zcmeI$Ar8VY6oBE^C2W%%0fz*I;t&WF3XKL590rL%a|2F+&19+$w7kF?!jNxR|;ahft4mI)9d{-T zUD3N-qJPty8AeU~bG7Jt=G$711p){lfB*srAbby^vl8zcH(B) z+qZ^#vyf(bWU!OGItXZ;RlCA}zL9|`e718atM=%@wx7zZtjzUp8z*NC3tku%~d<2IHJxYTKeO1qLqbSxkx8$GJ3eKzmCS^&%;O8 zKNY>pC3<(gnL%`j-&cw5%zRtPu|NO;1Q0*~0R#|0009ILK;Rz*_Q#EprG8oX!A{&P zd;8WSBR4vW` literal 0 HcmV?d00001 diff --git a/flume-ng-sinks/flume-hive-sink/metastore_db/seg0/c5c0.dat b/flume-ng-sinks/flume-hive-sink/metastore_db/seg0/c5c0.dat new file mode 100644 index 0000000000000000000000000000000000000000..aa4bef93f63c214b1ddfeacf13566a2d7fb86840 GIT binary patch literal 12288 zcmeI%F=_%q6vpv4vWkUaO*CL)Wff8g7J^L#8!MX>fm|U6NCLqF1k%~%8o5DCW$z_& z3DGxicNuXMTZR9^K4-t@%&xy~R(Tf@r>8%fRx4JwOVQuXbiB@wo%rD8qjp}iZry+Q z+tu{u)hs($;|J2OPA8v#*$i9D5A(;o``2-)r)SpRTdO0DkKK6hOhN<@KmY**5I_I{ z1Q0*~0R+xVU^t%L%bPxPvA^#~S+5$=uSlZAf~2A|uiY|q%fC8y@+2+|+xnLE>mj)D zfx8UBtp!&hXxkJD0tg_000IagfB*srAbH>qs{AUA&KpGhU literal 0 HcmV?d00001 diff --git a/flume-ng-sinks/flume-hive-sink/metastore_db/seg0/c5d1.dat b/flume-ng-sinks/flume-hive-sink/metastore_db/seg0/c5d1.dat new file mode 100644 index 0000000000000000000000000000000000000000..c37d1b60d37e743b216f8a1a6f0827106570858f GIT binary patch literal 8192 zcmeI0t4;$^5Qb;Aba&gmbX({hNkAY#p#Y6Up!x^|2_%>XgCxP@5Ig|_i9#SqY6v7e z071YDkig7rx3i%O0!ij1`}dr0x;@=*)fqSh0HlkDBk4-!Bdg~b%|FNE8~$;*O_hDR ze1IhT64K_h{&kd#s!yBu?KWPLM&L%f!=X`_}d`M|| zG_G>%fqMZvFc?>Egiuq0Cg?enE&)P`*61s7nD|&xQeslUs*)KdRScArnFJUrahZe| zDe;&@7%K^gI4;(dgiJh4ltfH?tSgC`RIs6>#-xg~N)jdk&MB!g39+f9$t1#-k`|K~ z+e+F*FvgCO4wD*mm2{aT*i+JDQpdiMK9ffJGAzx2888E8zzmoHGhhbHfEh3YX5cRd zoEYd_g&&fyl8=(Nl2?)!l4p`b$vw$|%ju@>=pzavgTlf9)-9 G48H&{G(e{S literal 0 HcmV?d00001 diff --git a/flume-ng-sinks/flume-hive-sink/metastore_db/seg0/c5e0.dat b/flume-ng-sinks/flume-hive-sink/metastore_db/seg0/c5e0.dat new file mode 100644 index 0000000000000000000000000000000000000000..0bd366af1c01ceb82ea63f45765850f9adf196c5 GIT binary patch literal 8192 zcmeI#F%E)25J1s^C|X!(@D8@NCMF&M3#hz?dIvl0y?__t0#mEcDEsxA=Feg@%HHWxJ8Rb+n#)+vfo|S+HE;a5{p#}A zxfS+VCN1up+)O@oO<%Noe%si$m^uOoAbHV|dT$Q_5&0|R=k?egWI!jweR;?mK009ILKmY**5I_I{1Q0*~fu9S^ LW#OK=KO9FdV4)ip literal 0 HcmV?d00001 diff --git a/flume-ng-sinks/flume-hive-sink/metastore_db/seg0/c5f1.dat b/flume-ng-sinks/flume-hive-sink/metastore_db/seg0/c5f1.dat new file mode 100644 index 0000000000000000000000000000000000000000..051499658057a3bebce70da5e57adaa0c9a4bf3c GIT binary patch literal 8192 zcmeI$u?@m75J1s$B$AYjKt~A`6$4OGP%#2EAkiTqN+w_freF$cxtdM0|$`cN)Q4G3B(17D@Ytb2ysA2;k_z*ws*%<6W7cI)IFNo zc6GOZey?7=s`no+`2r!N2ETY*QSH2H%gy+Dxew0Y93A&`ANz6_u6#~k-5B4r3h>48 zi(~%%$fW+C-G1(}?mv95(Z{_07wDg%gj|^P+^A16PXZgBuP#*{s60@4pz=WFfyx7w z2PzL#9;iG}d7$z@<$?bd54`Z*jbD?`!>1kB_m7dQ@ala>KL65@9@Q~B4^KW)`(5ek zhv9sVbjSdH@rNwTk!$2pN%3IRp){jTkp+R{=9z?-G=`sqg_J_MzOi?(*XVY(+ufb0 zSMNuAWQNrC?z{objBfxp_sC)yh=k*TD`W|P{rXnhinLTFGFZ)M zH|jKPQ0oenQszR~tB4JpEKA@_rbxz+7mNuAc^wz^cB{T;6@uqWRtfXuSSE|$z5Q17 zw2e(0C6MPEhe^*Wh`+{78`6DcQeq_O_`A{VMXl|QWn5e+;i$?_%wb*;IB^o-%Ta#v zV6e`UHSi$hi+;2d*+u)sBQ)2^Im~DoxH#8J*-($yjh0<*x?h3rj=_|2&k>U1EE~lj z2ea153YgXJwYImTp3R88O|c5e@==P$e&pp!?QFZGcsX?oK^eG?G0$-&Paz{_g;T(h^T11ZK#30fUe_+{A6B4H zG76~|-kx#hq+ZUQJP5^T$b_2*OpyR+Nvj5FbsxOXZX3w$nU0y`Q5?h<(rQSGQ7HHW7L zze|Aw$-I<;3r%2{XPxL)Uol%vbkmeZzFV;}=7o~t`fo52b^=ef_M>6DJ6blUL|suR zk7u!@DMX(E1(C1JLe$X8uB!?br>+DTPP5|!5;pZXc=blV)$I&nvJD~&M?R{6q36aB z>dXj)Xi?VPOSQ6;yhnlEI15vnW2QXb(m68I+)|l;x}hW2Gy|5elonSYD&Le1&qesTde+)u;2!wo`bd)S6`d;0Ms@72iI3kBn5B)GAm zx7|~kRjNYI{B%7h86j)`AfJWyW%3zlFJsT@>*R~j3i2!=D{qizp#32E1R=|RC!d7& zdAz#3NIp%-(yJtg_HpttLKc66cP?glzVJ_cw}o%u{R@}KrwEz<5&1Z@TX;VA2kf7F z2Jf7G6MJS~!FQXzhWE_;9Q$W_NIdsv)!rr_h4T%(r}jGb)E>u{{1)#gL;RfN30398 z$+_2leDSXO|N1fY|18`*;eh&Ik88nP!CdpW!tNOY7Z!~+#8O??ic^lT))Js_$^;W; z+9H|Xi+WAHu1|^43IYp7Aed*y;Bvdwwwd%=JYads1oO=hm=SwcGeWBd!gB-Ib-@nT zki1|?-x@S-MzG7<#xb%nNh=2;aYNWHpNNC>)o-aqVO+UQIZRqP@K}(*e&3A4LZI90 z+Zv{|11~5t%CJ%}`0-)#(9wiB+^@F>R-ve+1TRmZ%1F#)SocO`oMMWIn~C90eb*vk zYhl6nQbt*72E)z;h$w6g_qhd1OACHRUCJ^u3dgZ&zwl{kArR7o3d>BwT3;`G)?=rY zg&-9$`wBA+c4Vq{v^Q-eEC>-&js?CUL{VndreUCT>N{4EsRag^%7k)e24ba!x!B%1 ztuRnOk0npc`%Oq>m|0G>&S;4tWQBwp)r^DSKTKpcIa6y4EEB%unHdG+c8IcfrIs0D znF+~rGY~d^cYu(jF{#+tsMQ9y;4sCxIaEO6-3Jj0N(GBVqE!Zn{7kqml(>dDfXd;h z@*k?1XMxjN0}o{;UC&(HEs(iGMPtfUPKykZ1p#$WSi@nS!Cy|c>h0lfueEP8Qd(#T zia3>CW-cOO=hJH5vVmx!LC`P&WyV~9gGH@QGrDCT?pkK>Gr?1zqpEg%{WhA)*3Q$y zf-7@Z_~OLyJ4{lxW2Lo)q{s^Cqqb$R43+NF zZ9xWp>K7om8fY-BZVWnFMV@k6X+a?sK22d_87YvHkTUEIERuxQ4}`#K;)KE%G{WYV zt=no^M353R!oV{}pS?z?l>^~|4jK67NH`0PP^$(^peX5^)6tn~ghxx=_3vOMzy283 z)oX8HCBOC|ti4zNh4uF8E2Mz-3fAWq`vvyT9pL%wUzG~*5>jR#z&mHYho5cc zD)|yr^gqS^b5G&3wLhx%dDJOt7m!YVNbsxlPhdYD|M}&?56bm_?d$5R_+QgkOYQjk z0i`ZHo!$Jh765#a`yq$FF6r6IY2mop_-`ax8fI=L?WJk5`GS%eeAsDboR1Z<$&@pQ zJF@u|#S?iJGwzP#d@}EB18?(1MTV*?3AsDY>XXT*Rj0}3Z3T;Eo@CrLyY-avG}-*B zc4_9zjEix?oopDa`lr*U$>vLn#ysEk_~BYSnRX@uHQ9Vw+b0AEBK#>=EnAa`XRA=( zGbgK;l?N&hR34~2Pi^MysDJ94-+KRl0S>$?aR2}S literal 0 HcmV?d00001 diff --git a/flume-ng-sinks/flume-hive-sink/metastore_db/seg0/c600.dat b/flume-ng-sinks/flume-hive-sink/metastore_db/seg0/c600.dat new file mode 100644 index 0000000000000000000000000000000000000000..83367f789f249b7cb94a069813adcb2198c5d29c GIT binary patch literal 98304 zcmeIw&uSA<6bIltsl+ja2?T)Fd6d(`%+D8RvHw+He~kaSeG*UJ z%}$T1rFvaAzPwXy{&97eoo&3@{yv)@&5vhavv~F)4q99Bo*yh(I}hUUe=lB=Ga^8M z009C72oNAZfB*pk1PBlyK!5-N0t5&UAV7cs0RjXF5FkK+009C72oNAZfB*pk1PBly zK!5-N0t5&UAV7cs0RjXF5FkK+009C72oNAZfB*pk1PBlyK!5-N0t5&UAV7cs0RjXF z5FkK+009C72oNAZfB*pk1PBlyK!5-N0t5&UAV7csf#WA|=f}5UIEcTisDIo2a5{Dy z^)4?bbGz7`XvgGg@ioudRl5>igqQJCoLvc>`fRfZy|6i%zA5{A{lR!tj{C#OWUn0W z9FEHUx8=^HoJkXrqa&)U<^m2~gXc%43(VNYp`A?=JK!5-N0t5&UAV7cs0RjXF5FkK+009C72oNAZ zfB*pk1PBlyK!5-N0t5&UAV7cs0RjXF5FkK+009C72oNAZfB*pk1PBlyK!5-N0t5&U zAV7cs0RjXF5FkK+009C72oNAZfB*pk1PBlyK!5-N0t5&UAV7cs0RjXF5FkK+009C7 z2oNAZfWU%4Yd72tp>?aO#m`hfr}{qChpBd}8os3ZG1cuy&)->M(| zG_7m(E-yuHbp!|yAV7cs0RjXF5FkK+009C72oNAZfB*pk1PBlyK!5-N0t5&UAV7cs z0RjXF5FkK+009C72oNAZfB*pk1PBlyK!5-N0t5&UAV7cs0RjXF5FkK+009C72oNAZ zfB*pk1PBlyK!5-N0t5&UAV7cs0RjXF5FkK+009C72oNAZfB*pk1PBlyK;XCu^j@FZ F{S98?ybJ&U literal 0 HcmV?d00001 diff --git a/flume-ng-sinks/flume-hive-sink/metastore_db/seg0/c611.dat b/flume-ng-sinks/flume-hive-sink/metastore_db/seg0/c611.dat new file mode 100644 index 0000000000000000000000000000000000000000..a8a8e86b5cbda5523623b1395c94dc0583997a81 GIT binary patch literal 8192 zcmeI$t8Y_b7{~GVcM9$4y*<6(2XyPUZf#dCf*=qGBq0P*gFz62ge4#dh6Di#8WIEo zg+L$>gb)M*g&>3wG$9By0s`;v>E&ry;SXT%N%}oKpLgw9pJk%2**3<&K6rT2uyj7t z^?9?;e@%`*_^+3z?8;^P@Quk#e+c&G_WGYkY|(1#`Ol+cpL;KX&KG;aVD2UhUN86b zZ!B1GqsgB}9oLrm}|;LtJGqC60tjn36zJB|=FerQ%akNUKCCX=GGllnk;eaY`0} zN`ev~r;?=PkXK1j@+hdJDFqZ&GFH4WN-9}O1Z9rGOQcHl>J; z$}**dRh1P=8EYyXN(JjGtCT9bDr;6y#D>Z`rDQf$x|Fi%sccXxrmwO|shWXGk5a>y zN}p24P-Q@A;GoJDrHMl-LrRPPNA7SFa1(G7a1(G7a1(G7a1(G7*bNg1y)}U`p%>D} z(tFZd(rxK^>1pY4X-`_0##ryW^t1Gx^p*69^nvuY^qTa7^sMxRbSPbtrda%tewBWZ wzLq|fK9t^-UYA~yj-)50N2G0OAT_M{CjBIRBYh@)Bt2#>j{o+OUwtB4xk&iCYXO-4Yr`O+&wK#IKeye_c>)O3W{pyXaqL|FFHfw(}w|_lt zpW@~1r(d0qS{&*=9Q1Y`w_E+zg0sr&b9XqE4|y})p5o<){M{YON9X#V&i&s_`=|I{ z`{(@z0t5&UAV7cs0RjXF5FkK+009C72oNAZfB*pk1PBlyK!5-N0t5&UAV7cs0RjXF z5FkK+009C72oNAZfB*pk1PBlyK!5-N0t5&UAV7cs0RjXF5FkK+009C72oNAZfB*pk z1PBlyK!5-N0t5&UAV7cs0RjXF5FkK+009C72oNAZfB*pk1PJ_J0>!hq8`WZCIL7;E zr`zAp>&vk*l6djq_RAQg^~zXgGtcYeXf_qIr4<1J1PBlyK!5-N0t5&UAV7cs0RjXF z5FkK+009C72oNAZfB*pk1PBlyK!5-N0t5&UAV7cs0RjXF5FkK+009C72oNAZfB*pk z1PBlyK!5-N0t5&UAV7cs0RjXF5FkK+009C72oNAZfB*pk1PBlyK!5-N0t5&UAV7cs a0RjXF5FkK+009C72oN}E0xM50JpBoq1z+6& literal 0 HcmV?d00001 diff --git a/flume-ng-sinks/flume-hive-sink/metastore_db/seg0/c631.dat b/flume-ng-sinks/flume-hive-sink/metastore_db/seg0/c631.dat new file mode 100644 index 0000000000000000000000000000000000000000..695438d365616889922343e8e5cdbc3086575b38 GIT binary patch literal 8192 zcmeI0txm&G6vxl$wyxXywZXn2L69I2fI;y9NCawv2Ovlw!4M3RgrG=p2ow^5oI>yb z1i=Fk1W!N$=bW}X4J!yFIXCIQy}z^ew!f)+;1B?i4jzuAE18e1o@aFZnT~Jx$LSVT z_UZ5d;_OFAi_`qqQ7)?PTvl!qo_m&nrB^!PYHpE&`(;h)tPh&5K1&hPl?0C!gdy*r`P04l1Mah;VaKgLfi{zu^z2uGL2J9wp JJ-&$MzW@LXKFZIP2x7SQ^!t}1*k{} z3BjTdfDjMB6R=>(f+f5KNU&go!~$j#-=F~^?3Swi#-4Ne&sa16r{A90IqZfIqIgfX z#}C^1gLb?btdGay@$dcZFX6wPUXE9~hpV@eE;-$!@yQpG$vUD?L8_hjAmh!8Urumuog~FSPHGMbyr<-NiA%)d!4QQdbK-2{=;JuQhC=7U(<8X$ zVR%xwPr}Fl3*5n%E2*s2Y+*SQzw^u4DJU(6Gb5$cgQ%35f>LVknXzBz^rXMeQ@J7K za)n}krI0IDiluUHrC7$aV9@HZYIX+%1i(fXwM z$G?ZiU;NM0(|+ZuzXi@^M^A!(bASHdW9s_5oh!cvubQ})z|vQLhwj9iOu4#TGuWB3 zVq_Dr3D^W|0yY7gfK9+AU=y$j*aU0>Hi5l0fx8c`!(H>AY4rcT?>heM=&nz>wo3*O ziTW*rTP_%H1WbK2e(~skJX{AKhgUc=T;b#UCJyDt87x{a73j?X%2o)R-1rLz$Rc5unE`%Yyvg`o50?mK={4OI~RT_eImUt zy)E6AUXY%ZZb{qHhBVHGen`JaKS*CopGqG}??`V*FG|l#PfB~zd1;!(Pw6-5C+QpM tGwCDgJ?Ty9Woch}N_tG%lIEq3b>F3*rEjG#q>rU1+@--cpK2d}{s&wgS#AIT literal 0 HcmV?d00001 diff --git a/flume-ng-sinks/flume-hive-sink/metastore_db/seg0/c660.dat b/flume-ng-sinks/flume-hive-sink/metastore_db/seg0/c660.dat new file mode 100644 index 0000000000000000000000000000000000000000..d8d7104ab5cf8202b27b2c7de7d5e7407cac8bf7 GIT binary patch literal 12288 zcmeI1y-LGS7>2*3B^K@AAZo0(f`ioG>ReF=2M3{og5csHh>5OgpRup zw_IHpuO;2kte27fi<7&gx9@V(!8RA)bRKr}N_c73cC`L!;nF85F@OOKU;qOczyJm? zfB_6(00aNaz}DVvTds7^>G^(J#`JE!@D?M5#)@jBPsYKsr_q_tD{>^q(vx7a@^!*) zOI5`)U;)ab@y(#6EFiHl>$ zj*cChxHy@Pj2%0=7=3xT{D?*WfaX2P`NH|UaDaz|gd=Z71ZE#^0c}XTnA!TGnft$| z+h6?8!+}|OZFX10n>`5T#m(`5x25afhA)0ip0#i;h1Qk1L2uzjmYiMB=?<5ySh5qa z6R;Do6R;Do6R;Do6R;Do6R;Do6R;DwT@(2D`90)~`AoC__xsW{M-Oj&%FYd$Kqzc3 z8GI0Tej#A#z4?=8|KlNj{Ft8M)pXS{w*m7;p2fWn2;HGa5Tys{QN-wX=rP3UA$lAMdYGO-lJ3%zNYNwo6g+yA?jcQ&(bLG#dc4SF8y^i_HRE&3Y0h&Fwl z?qh@AqL;8qZ_~@@&^P9e0=DRz^rCFjJ9J-m=v(xXbm`mlvh?UX^a}d)F1?CfdXHYi zeR`i>#{>E9%={LJ#AC# zj_X`&FSX~|Q|%}1JMDq?skW!}wK8ttO8Z^=Rr^JItbM6{u6?3yYrS#tQ#;aLXwS4K i+M)JPJJ9ZFn_72Vxz=84&$Xx8$8t3JnvWkfSN;Gwb6d0k literal 0 HcmV?d00001 diff --git a/flume-ng-sinks/flume-hive-sink/metastore_db/seg0/c680.dat b/flume-ng-sinks/flume-hive-sink/metastore_db/seg0/c680.dat new file mode 100644 index 0000000000000000000000000000000000000000..0d8f4e36d2b465b0a1956ea29e2abfddcc0aa187 GIT binary patch literal 98304 zcmeIuF^dyH6ae5iA(uSXv&Uf{hlZ=z$~9ic4+dZpxp$rZM93LSrwwZ^q?hdAoF_?b_S$iNl4~wI`>v z+cxfR*mG~8b^V?*MH2x61PBlyK!5-N0t5&UAV7cs0RjXF5FkK+009C72oNAZfB*pk z1PBlyK!5-N0t5&UAV7cs0RjXF5FkK+009C72oNAZfB*pk1PBlyK!5-N0t5&UAV7cs z0RjXF5FkK+009C72oNAZfB*pk1PBlyK!5-N0t5&UAV7cs0RjXF5FkK+009C72&@rE z$8j?vJ&jv=yj{ngDvs}QBadIIOnj=x7j^!ATyM7eb}jxynd45gj)(O;oUPg^wV{Xr z0RjXF5FkK+009C72oNAZfB*pk1PBlyK!5-N0t5&UAV7cs0RjXF5FkK+009C72oNAZ zfB*pk1PBlyK!5-N0t5&UAV7cs0RjXF5FkK+009C72oNAZfB*pk1PBlyK!5-N0t5&U uAV7cs0RjXF5FkK+009C72oNAZfB*pk1PBlyK!5-N0t5)0HG${H&&q!*E>IBw literal 0 HcmV?d00001 diff --git a/flume-ng-sinks/flume-hive-sink/metastore_db/seg0/c691.dat b/flume-ng-sinks/flume-hive-sink/metastore_db/seg0/c691.dat new file mode 100644 index 0000000000000000000000000000000000000000..d645a25567df1e37d63a5fc08924c9b005704689 GIT binary patch literal 8192 zcmeI0yNeT16vodt6O+k%@_wxAu8%dkDrjXR*tv~`g<6PRP*%_c)9)}RQ9<%ZLlG?FRx z%HSSF^A`b2)!aOLjt8B>?sS9)QyQvcpcZnGF42`K8*`5+{G^tc3g_<$Kv_z6`@O-9 z-nHI#5kfbwZ5wXPfs4+?grW|hxWr=NX85IMwi{2oKs>GHzNTFt6&hbX5OKf}7 zXc(Axycsklw!K-j3@kX_9NH4w-aJ+eEIQr-_DXDfi&!-S>hwzr0Z2G$*K9fu^gy$x&_*j)6Yg2NJ9UR0@LU~AzvsL>G##cfc> zQA66!lnrc3Y){$5F#}f|ZwtpIw!Q7ikB_fD0iS?Rz$f4n@CoVILP%bjyvJKWD`9e&k1e9!th>xZmwv%bvwH0!q3!4Iv2kE~y_e#-hT z>+7tySZ}h9yf1cg}KM=YFU)eb$=ZvVO+;4Bec3mv7>iTfYI? C_0+Wh literal 0 HcmV?d00001 diff --git a/flume-ng-sinks/flume-hive-sink/metastore_db/seg0/c6a0.dat b/flume-ng-sinks/flume-hive-sink/metastore_db/seg0/c6a0.dat new file mode 100644 index 0000000000000000000000000000000000000000..6ff147471cc4155494759477469f29bf66b0c192 GIT binary patch literal 8192 zcmeI#u?>Py5P;!>kf()(iFOvo%GLoKfG5 z$^Qt)aR(fH9kLUVOn1L>i_T&*#D24;`J-N~a`Lp)&L(vG{xHIfi!00IagfB*srAbg-2d2p;`t1ldXuye~g zZ=VqRtw!PW`cZaymEA$}LKd==Qa2U?2q1s}0tg_000IagfB*sr{7PUhlQ8-;URO^i Cy%qQX literal 0 HcmV?d00001 diff --git a/flume-ng-sinks/flume-hive-sink/metastore_db/seg0/c6b1.dat b/flume-ng-sinks/flume-hive-sink/metastore_db/seg0/c6b1.dat new file mode 100644 index 0000000000000000000000000000000000000000..1b9a62ea62b20a63b031c4a745e22725bc9dcf2c GIT binary patch literal 8192 zcmeI$p$@_@5P;$9mTZ$e0)}A0pm>N06wg3{0TLVriN#>hI9>*g1j`G+tzDOH41y+q zlik|8ZkK&-m6eDXy^0apnAjKwUk`ISf0m<>{G(}Xo9p`PXl;KQ-Mj9o z;9WM+zv;~kf+lWXE!v*hrj}!Y00IagfB*srAbhjGXH}e*;)Gb zt-jtYB&i-5Y$dA>0$OL)Q~p2S$k1F|HaVBoSnoyqiR3aZYe$BZ~pq|Dga)7$-LXC{qHF6#!8pK+FTQoC6|;lf-I$ z^v&O$5FKE%KuoAQ^%o$v0YnlP3AU4v#ke$&ijRiCXb6mkz-S1JhQMeDjE2By2#kin zXb6mkz-S1JhQJUCfn&lezcZ8q9SrMWGl5y`7+!$Mqe-x^0o6+~UPcJB9RRYK8A2FB z89<%=Q7{?;qaiRF0;3@?8UmvsFd71*Aut*OqaiRF0;3@?8UmvsFi;_2!ypf_IzHp| Fb^r@n76NN^Y=4#y2p9Dyrf&`3B1ytV7njX}`l zZ?d=cy>2i2+$t*(F?y9FvN5qS489)bbpEVHEB$eqYh|HVCerhpj5gQx*U{PjIDYiq zQ^C7zVsO`+83s+I% z?OT1lT}VZkmFzLAl+x@>Z(sQ~muYca| zzw0y0=4ibyzpf&Y$4$0zPgcqR=a1|1fV(>EbKmZkok6KV?M(2oNAZ zfB*pk1PBlyK!5-N0t5&UAV7cs0RjXF5FkK+009C72oNAZfB*pk1PBlyK!5-N0t5&U zAV7cs0RjXF5FkK+009C72oNAZfB*pk1PBlyK!5-N0t5&UAV7cs0RjXF5FkK+009C7 v2oNAZfB*pk1PBlyK!5-N0t5&UAV7cs0RjXF5FkL{4h1IZss5|!?qmG}{Gb*I literal 0 HcmV?d00001 diff --git a/flume-ng-sinks/flume-hive-sink/metastore_db/seg0/c6f1.dat b/flume-ng-sinks/flume-hive-sink/metastore_db/seg0/c6f1.dat new file mode 100644 index 0000000000000000000000000000000000000000..785d62c5fc90e698a55fa93c1d408526403b63fa GIT binary patch literal 8192 zcmeI0F%H5o3`Lz*YCAF@W;Rw>m^cIj6K7xqBsLaC!~wVgN8ljLj4*NnaFV7)O?+L=yPn0irm=tk1Rwwb2tWV=5P$##AOHaf{3C(=abskqCkq$s#6|hr zSwp^=$Z~x$*hx`61hhugu5iycGBOvhW{)zP>TKK3^qZHttJ6eM8(e1|A`4&MJs)~67q>3& literal 0 HcmV?d00001 diff --git a/flume-ng-sinks/flume-hive-sink/metastore_db/seg0/c700.dat b/flume-ng-sinks/flume-hive-sink/metastore_db/seg0/c700.dat new file mode 100644 index 0000000000000000000000000000000000000000..646e2f5b451dd35a05e287333ace6c87f19d7d19 GIT binary patch literal 98304 zcmeIuu})M$6adgWE4!?O(u4&>LJYmO#)8t45GV)B*{b_xWVj()iljz2y;n^r}@<@ZZhFp4oewttD zosT{Y&rVBIiq)oRwe6Hxn`>x4C9OA2Yi*~br*jSWoU+k0t+$@?Y_6gGl!i=1fB*pk z1PBlyK!5-N0t5&UAV7cs0RjXF5FkK+009C72oNAZfB*pk1PBlyK!5-N0t5&UAV7cs z0RjXF5FkK+009C72oNAZfB*pk1PBlyK!5-N0t5&UAV7cs0RjXF5FkK+009C72oNAZ zfB*pk1PBlyK!5-N0t5&UAV7cs0RjXF5FkK+009F3Ng(aTc0_s^TX|lt;`mj?@vX#P zN<1m?+Y-Ny=k?WJk2ZhAILDoOA8*P!nC_Y-HKB+A0RjXF5FkK+009C72oNAZfB*pk z1PBlyK!5-N0t5&UAV7cs0RjXF5FkK+009C72oNAZfB*pk1PBlyK!5-N0t5&UAV7cs z0RjXF5FkK+009C72oNAZfB*pk1PBlyK!5-N0t5&UAV7cs0RjXF5FkK+009C72oNAZ afB*pk1PBlyK!5-N0t5&=G=bxb!|@F_l{qH> literal 0 HcmV?d00001 diff --git a/flume-ng-sinks/flume-hive-sink/metastore_db/seg0/c71.dat b/flume-ng-sinks/flume-hive-sink/metastore_db/seg0/c71.dat new file mode 100644 index 0000000000000000000000000000000000000000..e639c1cc40f4634e80579814fd9dd124be6e601f GIT binary patch literal 16384 zcmeHNOKclO82)xAe&qdVlIGQS`nXtfl2&zi#cS^-UBzA-uk9vCXi1y)1_)FNQYn4# zkPtVdUJ!?>5+`neDkPA&02d@AgajNafz%5ZKuFv;!2i!Wj-8}+CKW=eW~0omXZHX8 z`M;eR&-Z(A5r9Gee6-0jD$B-Vt-Z01uK%eW8>;X5u}Cc!=synht=(jhbNYPWGy|Fe&46a$ z;bP$POGhr_C5mZH-(P@9pVFJ6Tx;LZ&=w`>ZsIM(>zRNpuWDxNLf^oY@k6clh1yxr z*`%IS;NhuS-7~U|#QZOGv6dw;G1rDJbok4@m2)lM@o@x`siZts6UpRiGI4(O%1nW@=i5&}j~9fuw2Lo=Wm&Gy|Fe&46Y=GoTsJ3}^;40}mksALA57 z^%!0bWBYmd%kLZYzi|bJsbyhPw-96txkeeEWEnr5I5U$vpE!PYb)_cbk55gXl``J!KwKaKk+oIKjALjQoI48af5WFv1`?r|j54dC8;r z^TmSac;)Sf!(f-uCD4yv+bUZbN`(YPt7PT3k3k8MBj}b)+8HIY!|0Jro^q>BkSk;t zw{OiIzk*)Lg6AxiW$=xB2Cl6{>2>r8^q@ABSQGUc-PSgV-;- zJTA6RhJZPYU2-nL=kHk!=c7_P^{lREIr zmcp1OHiocUAcjb(kZa29wVa*f7?A_nS6ULxc8m&iqTRRECy&{OF&VD!&_dv4oq(oc zZu{#2g*%3EUM!=&Q?k|KjbTELdTF_Xz7JxLeBAz`bDCBmepsHBSL?$r?3L5F0fRNL$jg>)FoI4u;lV zq%-veQN82)kj^%*BFeGvN7|Ol$b7}iDg}QCX-A+PF}4g!6@|fPP9i-oKzkEAqaHPH zz4W0O&Gz0%X1F-;)1F^lZfarI~Q~3_&shmR&i2Mzk%Fp=M^2jwjLggGXRGMtn z_6<%_`7UNj^;R)Q*F|oP-$0hGyVzEI&1m8ct{3xog32hPnP0I-^A+|OAL8rS9mX+d z87KBW-_t%oDjxBl*l)YX9!--y8ecHJ@gmzA2{^>!AB>DoxfK`L7g>@OA6-ALe9)-> zaUJ`pdh-`iEB!Cgs1I9Y0`9H$AC7nlpcH?c z^?kVBkf2>ya7s$%gDmpHu~T)O(@M6zEb!^k$xiP;#SO5yC*sI{XQ`YhD|*tO<&2u} zjkAVlyQQY(_5K+k&MIDFy-rKS17uWRWyMZwM3o*CCGr|p_xro21jZ;1$#FvAs;j>P zm<-7#V@|~=E8c^(FJXPs3}^;41DXNNfM(!7W+2v&M=2xlEvxS64OZ!~oXi@$Ewcts zBSrNSd>y&Nx;=7}b$eumm3(AUW(w|dYuhbZuE_EL-zo0OjKEDs7OQL}j&W)J#vaWN uS;?Ev%bID%G5+HFjgPqAc#iW6Mp7#OKiL;P;d@B`b1tB+|0jQc@7%vK&DatE literal 0 HcmV?d00001 diff --git a/flume-ng-sinks/flume-hive-sink/metastore_db/seg0/c711.dat b/flume-ng-sinks/flume-hive-sink/metastore_db/seg0/c711.dat new file mode 100644 index 0000000000000000000000000000000000000000..e63d6e9b4057e7f16ee278c475b2de84906a865a GIT binary patch literal 8192 zcmeI0IcQX26ov13Z!-J7FG_~TeTN#1fzwWD2SNYxIn-ng<@qRb|SW7AFvcG z1+fyu#!9plY$g8xotbwoW)y7Ye=zTTb56clP7#&lo$>Ko z{Ku!~Tyxt!vLe;VAHlu3d;ZsBYX5d? zIB0VU9W1fvBm!DEWb;Ho8_O)Z6goI;a|&HqVbP<|lOr~#(8p00eF_5{vpI!X9A`10 zFo&VdDa`wS2;m3|gayI^VS%tfSRgDA76=Rc^8%?S(i2JDldd~oQ9i4DLbm0u{IlJ(JGH?{|tDGTp}k*DOh?2t@Q1#OGG!?2KnAH zfA7sOJ74-tDdlKIw939)_NQH4Kh4E_e~h#ApUZhvKC0@n7>3aXae2R3T#3f($D8f0 zI@I9J7rbRp|A0O5#XT6JO*7dVOryKhp^3QL&_Z;(fdQ0!|2?7KN5FkK+ z009C72oNAZfB*pk1PBlyK!5-N0t5&UAV7cs0RjXF5FkK+009C72oNAZfB*pk1PBly zK!5-N0t5&UAV7cs0RjXF5FkK+009C72oNAZfB*pk1PBlyK!5-N0t5&UAV7cs0RjXF t5FkK+009C72oNAZfB*pk1PBlyK!5-N0t5&UAn>0BCh4jAz4dDLasV=Z6!QQ8 literal 0 HcmV?d00001 diff --git a/flume-ng-sinks/flume-hive-sink/metastore_db/seg0/c731.dat b/flume-ng-sinks/flume-hive-sink/metastore_db/seg0/c731.dat new file mode 100644 index 0000000000000000000000000000000000000000..85e7867903432ff4690b867a1bb2b60a08b1aa38 GIT binary patch literal 8192 zcmeI$p$@_@5P;$9mTZ$e!V(0Q859zSL4e^IkU)aNAbA4bfVaV*k?<67YuBY4gP_UZ zWViON+hw0yWh){^uX02-CN_q_*TbC7pVerkKQ3olS?iUF^!z5H&2{~CbhbZ_9=Ure zc$ZBK?s_xBpo!bph_+|8Z{%1YfB*srAbQ6Vc9y<< ztFN~UNvcN%2g&M#fX-R{l>g5+QkbjDrbSha^BZ~pq|Dga)7$-LXC{qHF6#!8pK+FTQoC6|;lf-I$ z^v&O$5FKE%KuoAQ^%o$v0YnlP3AU4v#ke$&ijRiCXb6mkz-S1JhQMeDjE2By2#kin zXb6mkz-S1JhQJUCfn&lezcZ8q9SrMWGl5y`7+!$Mqe-x^0o6+~UPcJBI{?|t3?V=W z>g*#EM9zXi- zso-5UF}Ul^41*?aUnAO{*}joufdB#sAb z_N~6&E+nZQ85|_54+1)8^;7;o-^j=mE}PD(YOMFN{akWcRJEziiNrUTAR>SO0tg_0 Z00IagfB*srAb`Nn0ux!fy_eP1ya20$E<*qS literal 0 HcmV?d00001 diff --git a/flume-ng-sinks/flume-hive-sink/metastore_db/seg0/c760.dat b/flume-ng-sinks/flume-hive-sink/metastore_db/seg0/c760.dat new file mode 100644 index 0000000000000000000000000000000000000000..ca7d1c98807f49614d0bf1c4fe644924d0bccc44 GIT binary patch literal 98304 zcmeIuJx&`z6ae72h8PQ>Kr9e}NXZQ%n>gY}6e&`sh$3i6L6D$80vBOPO_4sFC0F1G zNF0KOk`l)2wH9FlZiBwDcIM4{ewvvtenmuXezfB)_GoD5p~*UY}1rbMlkzCq!Ewecw>pE~$~J}CkO z2oNAZfB*pk1PBlyK!5-N0t5&UAV7cs0RjXF5FkK+009C72oNAZfB*pk1PBlyK!5-N z0t5&UAV7cs0RjXF5FkK+009C72oNAZfB*pk1PBlyK!5-N0t5&UAV7cs0RjXF5FkK+ z009C72oNAZfB*pk1PBlyK!5-N0t5&UAV7cs0RjXF5O{t9Ygd=Mah%?jm2Z0+FH+PT zdg=I3rbvv}Q|Cf5E0Y))w3{4&q}M6%~U;$qxH12S5?oOk$T!o)iXr|2oNAZfB*pk z1PBlyK!5-N0t5&UAV7cs0RjXF5FkK+009C72oNAZfB*pk1PBlyK!5-N0t5&UAV7cs z0RjXF5FkK+009C72oNAZfB*pk1PBlyK!5-N0t5&UAV7cs0RjXF5FkK+009C72oNAZ zfB*pk1PBlyK!5-N0t5&UAV7cs0RjXF5FqeB30y}fUH7b*{}l80VqTAT{by`#zyA`) zDXx{zY8C5DxvQ3}3Pl765FkK+009C72oNAZfB*pk1PBlyK!5-N0t5&UAV7cs0RjXF z5FkK+009C72oNAZfB*pk1PBlyK!5-N0t5&UAV7cs0RjXF5FkK+009C72oNAZfB*pk z1PBlyK!5-N0t5&UAV7cs0RjXF5FkK+009C72oNAZfB*pk1PBlyK!5-N0t5&UAVA=` K2~2k9FKz)OpGJ28 literal 0 HcmV?d00001 diff --git a/flume-ng-sinks/flume-hive-sink/metastore_db/seg0/c771.dat b/flume-ng-sinks/flume-hive-sink/metastore_db/seg0/c771.dat new file mode 100644 index 0000000000000000000000000000000000000000..cb59123de8ce1bac425013c90fe59fc1692cecb7 GIT binary patch literal 8192 zcmeI0Jxc>Y5Qb;&V(#+&p_fm4ksxSq=?}1oA~qT@MKA_5QLs`G8^O}b)f(ZO0~;hwvleJ;EALW*5D0su7EpjpHyF^ph5TrCH}jt2a^c3sJ`$8C-xnItcjjY4mci&qr9o zXTR#~4@Yftx6DH7um+pMv2CgW*l%Wujy*ajKq!LOJ7{#et!AgU)oyfqD?9CaYp)5m z5z%GU5>*#ttlFX)!#JxEQH^7Q)hSU;V3O6Ss3tMRsw1i?OtYGhR4a-ZR+FOYV3yUC zsJfVAH7%+!%(I#i)i_SGnibUq&aj#j)g*eXdZL=b0;>g4O=FSOqNrxD#A-=WF%4x_ z%c7dW3ab@S&0>|+s;K7tf2L9s&;&FAO+XXS1T+CnKoige{+vK00yVR%eI-5;&xsqv s1)?>e^+J3go)OoHH6jd%ed0awgjgre4+tN`C*mD(8QT6*bNPPZ3ktdT4FCWD literal 0 HcmV?d00001 diff --git a/flume-ng-sinks/flume-hive-sink/metastore_db/seg0/c780.dat b/flume-ng-sinks/flume-hive-sink/metastore_db/seg0/c780.dat new file mode 100644 index 0000000000000000000000000000000000000000..c298dfd2b6a1efdcbf60a61c7c88486c1b013e08 GIT binary patch literal 8192 zcmeI#F%E)25J1rZ4NzE^Xk%q*>j5}~g;rieZES2kiWl($R(b+YAccj_&a#o98%tyI zH(?lNfPvSoY(yl{!>!a}X0aJUzgg4#QLk1xdO6e1%6I$jG}NP~!wX-F#*gEd=lkA2 zVefKjec$Bj@~LmeqRsp^IQB4g1Q0*~0R#|0009ILKmY**{z%|BI8?IM4-Omb)Uy2T z7ec?)=w^EVC^@~#`A+kW%w-{Nv(G{R0R#|0009ILKmY**5I_KdUkQ{l_M^Au<@N+F CnHA>% literal 0 HcmV?d00001 diff --git a/flume-ng-sinks/flume-hive-sink/metastore_db/seg0/c791.dat b/flume-ng-sinks/flume-hive-sink/metastore_db/seg0/c791.dat new file mode 100644 index 0000000000000000000000000000000000000000..352a9935d270d27331d4c0416c4d02aface3b126 GIT binary patch literal 8192 zcmeI$p$@_@5P;$9mTZ$)91aTx#Y0S>cm^bp;AW5rG;hE|z*~^uknj|6YuBY4gP_UZ zWViON+hw0yWi28`uX02-CN_q_*TbC7pVerkKQ42vEcMDndVZ79=DPkmI@=#7kAZtC zc$ZBK?|L(%po!bph_+`|G;%BuKmY**5I_I{1Q0*~0R#~EM}gg8V`QaY=6|pwJ4@fb z)z{mFB-JB>tz`8%DA0lU(Lit*|+f_~sHs1Q0*~0R#|0 X009ILKmY**5cpYODhs!_f4-hxWDqYY literal 0 HcmV?d00001 diff --git a/flume-ng-sinks/flume-hive-sink/metastore_db/seg0/c7a0.dat b/flume-ng-sinks/flume-hive-sink/metastore_db/seg0/c7a0.dat new file mode 100644 index 0000000000000000000000000000000000000000..9c5169f5fe5c39cf36d0c0357965dfba6c13e8ee GIT binary patch literal 98304 zcmeIvu})M$6b9fKmcSY*35uvt*a(7PX-xDH)Ig9>ZK5WdC?vR8SYcuBBUsp3_#$@3 z!dKA3N6@*mcQYHZj5XxDn>};p%ztzDejDC~5Muf^*LjLf7|QS_{{C& zbTSy8o-|AIxViN3ZnLRebCb?ip6&l0&QC_i!=D+SeoTX@&Ga1|Ojx^*((r!|o^YQL z0RjXF5FkK+009C72oNAZfB*pk1PBlyK!5-N0t5&UAV7cs0RjXF5FkK+009C72oNAZ zfB*pk1PBlyK!5-N0t5&UAV7cs0RjXF5FkK+009C72oNAZfB*pk1PBlyK!5-N0t5&U zAV7cs0RjXF5FkK+009C72oNAZfB*pk1PBlyK!5-N0t5(LJc0XvemxJz>Ahn6ZV$ud z)UC9;vdortyqlg$cP_^7Wj6aM^_{RCcG4#WmgH7F zxm=R>s>$-#=?D-YK!5-N0t5&UAV7cs0RjXF5FkK+009C72oNAZfB*pk1PBlyK!5-N z0t5&UAV7cs0RjXF5FkK+009C72oNAZfB*pk1PBlyK!5-N0t5&UAV7cs0RjXF5FkK+ z009C72oNAZfB*pk1PBlyK!5-N0t5&UAV7cs0RjXF5FkK+009C72oNA}E(PL`uo^=A zlI#0icXM6OwbRt_GuN-Ve#mt{*9T#F_|o5Q^*@K>)K9etPx89d?#dENt0O>w009C7 z2oNAZfB*pk1PBlyK!5-N0t5&UAV7cs0RjXF5FkK+009C72oNAZfB*pk1PBlyK!5-N z0t5&UAV7cs0RjXF5FkK+009C72oNAZfB*pk1PBlyK!5-N0t5&UAV7cs0RjXF5FkK+ p009C72oNAZfB*pk1PBlyK!5-N0t5&UAV7cs0Rk6IU~TWx!9Q|)cZC1| literal 0 HcmV?d00001 diff --git a/flume-ng-sinks/flume-hive-sink/metastore_db/seg0/c7b1.dat b/flume-ng-sinks/flume-hive-sink/metastore_db/seg0/c7b1.dat new file mode 100644 index 0000000000000000000000000000000000000000..c288cccf65da317adc3c152c784ede74a0e086c3 GIT binary patch literal 8192 zcmeI$D{ND76vy%NYq$3H{eHh-Z0nwV3`h_Y2m}H#0STEa2}?%yNXA&g)C>+11cC&_ z1OkgeAP@)yBqSiv6Nm`}0)c>x|2=JY8di=b7^Yag1NbehwI}G zroTV9W*#5)jz5J&j`>dg-}lSZG|$eT z_?ELLR=Nv5zqw>^kDTKR0Ye`hzqtQ99@NI??yCPqZ`W(?J?4X4rE9d@+uP?gBJY9u zh<@+!bplY9(zkDJt#;N{cRSmwn;VUl&8_v$HhGPMBhsTu2LX{@O$HDYIikrRLLx^s z8A4d3Pm^IpMEW%uK~$up$tYqX1DcE>E;6XeI1(a5noJ-mGOWoYQX(UoOd%~Ys>w7m zB4e7&AS*JiNRJOWkqJ%ukr$cNq=SOUlqLfxicD)Vh?2;RCPOHT%xW@>ipZQMBdChZ zYch(O$bu$gI3=>E$v94nENL=zugjR}!8O=d7Da$J*H)J0Bc zGKVRVlbXz9T4Y_51QAWk=#->S7#1JmDwp literal 0 HcmV?d00001 diff --git a/flume-ng-sinks/flume-hive-sink/metastore_db/seg0/c7d1.dat b/flume-ng-sinks/flume-hive-sink/metastore_db/seg0/c7d1.dat new file mode 100644 index 0000000000000000000000000000000000000000..8223520e0c45b8efa1bbc6df39208d3d7c33c575 GIT binary patch literal 8192 zcmeI0yK59t6o>C`f;*3$ot<4{5)%`nu1PjgS1qgr8(Tq95YpI6v=NM8Cx{4wV3SI* z5V5hb)I!k4#!3XSwGgaqENlclk4(<&`uGFnT$pd}J>Qwf{@CVDxE4YH9XujoLs&=U z>ouDH+#Dn2zb;Rcxk^V8LRmF|9!~3jj^AI1+V4ExxTIIr?j?oAFLc7;+C#K&i1N(j z#!$@{o`5Id33vjYfG6Mycmke)C*TQq0-nJCn84lp*Adp~msR`wvtgN5v~wOA{gHLG zY{c}*;8sYsJ_P)6R&{Uf^AQI4pw;-@rsYo$5xvN{u&R~DJe~Q)qE~c|quSMDv_=vk z$Rss2#+3w_Rkm>@MN^wovvYrO_WGJkXaHWa1HfJbJ<*agaRORTJDp<9!T~!O? zrb<##13N^~WWrRJRg6hhF{Vt7vr4d2Rf1_#6Rc9qs7kTR)D)`>v#K)8nVM$R#BNnh z%$u5Jm1B>p9D7amShcWERSP{+3#Cd(j+929RvupC`oM2VVdjN)I5#kf(l2uc#|p8M888E8zzmoHGhhbH zfEh3YX21-Xf&XQobKXX1(KGeG?`NS*XOrKavh=MI7d)Sq3{D|P76R0xq`Ci{k1&sq zanKXEnEEot7)Cg*HI5U* zJXC&q85QOcR<+(LvjR?NY=T)a{-ar%0W)9*%zzm%17^Ssm;p0j2L8%`a{<%zMO)!? zc43O<>2f;H{O#@BNuEirNiIrOIPp!gCwVJ*CE1bOm#j%Hae`bAH*$O_xeN`e)7ZSe GKluP|Nq#E; literal 0 HcmV?d00001 diff --git a/flume-ng-sinks/flume-hive-sink/metastore_db/seg0/c7f1.dat b/flume-ng-sinks/flume-hive-sink/metastore_db/seg0/c7f1.dat new file mode 100644 index 0000000000000000000000000000000000000000..02c55e3803673f1dbc044851b2b68d0426c68634 GIT binary patch literal 8192 zcmeI0y)Fb%6vxk*WoCBgbC><3P)HOaR4NZZr%>%9phTh(jYcBTsB|KFiGoh0@BpIl z0HToSC7g3+XV0ufp_Frz`QN#}duQ&Q-&FU)4gersTpUSPvKV=Pkp}IsN?sW(_l72F!pNFau`5 z4445kU|f_7ZW8R6Ax1*5fdM)N@6ButSL#DRB%j5$|S&XB~>ON)|J$m zMA%T$U=m|fNs~!}EhQ}?7(-i0n@NHlB^@RyPAHjRQl%eY#SEAMGhhbHfEh3YX21-X z0W)9*{%F97p+ZIN??DN%CIuTJl12 N3D&cPugqQEd;;a-K!X4P literal 0 HcmV?d00001 diff --git a/flume-ng-sinks/flume-hive-sink/metastore_db/seg0/c801.dat b/flume-ng-sinks/flume-hive-sink/metastore_db/seg0/c801.dat new file mode 100644 index 0000000000000000000000000000000000000000..f5c89e632a2285634e084dc7b995fb617f3d1abd GIT binary patch literal 8192 zcmeI1O^B6M9LCS{&i#1r_xIeHJL8PsAJwGU$CQqFEI%uCtg%un7Y2ghLP(=lMMQ1d z)uPvKJ)Z{NKF`yX$o-O3y8l^Rp;hu{u(_y2ko=UW}Wcx#>dbM~UK z;t%(RHFFM`XP#%K@659jt=K?pAT|&ihz-OBVgs>(*g$L`HV_+#4g9|v_~_(YFvs1` z)c=0}x!LFL4&L)q?!CuRBbRrN4Bj{C?1O-L)$DNpdp^u+{G@K$KUy9=B<_REnEiM6 zw(eb4lW*O7KAC+=05jn3kp@nRLc*mKMHP9Jd?t~p0?L6?M8!J;k}098<0aHo6j4`E zK*NVK5-y_|I2E+Kla)*r?ZBy_6F7AY22KNAJ+y)$6=e+jOip%gq8B(VjA*!pQ58)r z@R__!wJ{br9gJ(ZgM}*ESmZM$H?trC1&~1+36;0V`Fc zSmiSnnJVJGz$xK=4VUnMiXtX_rYciqJQz3?tk!S^YgCl+kk8a)s*1INQ^Ug=uHg|C zRXpl5b(yN;vA}6yGH{xBT*n)DLPZnnRMfHFha0kC3sZs9#)iP@;7J{CW21@=tX0*5 z-Q?pP*>Vu}sle*OKCOADHmh>6Ej~6VlS8mu18W#|8}q2aZdX-@-Jz-h`;4k8?6ZEf zE4%ezpW~vaovPe+yHvTecKd-LHvspErUR>M>>h6oOKQm2=L2il*cSq;XY5R1jlk~J z59>;JQI$LLB_Hd_{-dxj2i5}EeVTV_zbY3y;A0~)IR^VmV2#7Rs(GhgQ#I!Pw#6$p z5F3aM#0Fvmv4PmYf6+kZka7P&%^W4)6b{@V|0d6yf^*K1-_UuT`T%*9Y>}hFDGhfIV6Ka-cp&rH)fACf1?9pogr zjD#@xll-3iiu{Cpk32~3Bv+ALA#Re_$gjz>sc;Liby@Y&V(I&dDx zH2Hns`4-OL)o)uyS-%(9;)Y%yUm1J?gWE3xw$0tvO@HTi;8bUG;=MfirY7iJz{~L3 z#PbGw@TJ7pnCW+69gbT;Ns2%SCxXhwaiB|Q@ls^?m&-&5Sb$C+2D3Z?jr;(HvwZ2} zhp;rmE9K*tU^L5DK7It_8D1woehkaAytn_$u#@>hNR2UdRDT>emt)QrB&T>9DNP;E zov>@Be#3lI{CC013?D|m{;$C9Sw8piyJ61^FL*u3K!#NuWO+=Nll2hR{oez7x9~bB z70#TEHBHBz>t`?QWB$(g6hei8v4W;!5+7C9zYq2^Us+jB)@P}MD3U4&vxJ79lseOS zh^4eX_rvPl^;b3ssq1-}X{sNSOmJ;m;#c7S^SL!RB8lr+3ZzP08M!UvDK9GY6Mp~> zGOuh@;23a1oB4xqsOCi&BaiWK<`2PP=1ZX~BP4b}$}rGY6}Wy}6;zl?6^FTsxj%>D z2=j@R0@oW-QpunQOBIxHVkZ4Jx{M`{_Xr%_o|i$Us&ET`6xNv6nM(>$QvXdJ#4?Yt z5l!*OX?gGR|>Zc=<7invW|dWaj0ULDqbk;}*)j{7z8JM`0m_EWG?K&^7Ob7Seh7 z6)?=l#)i0kz5H$nYrZfT|H{knfrxo)Od=E=XwCDr7vh>PY%Y}a^83ItFQPKDDyRM4 zv|qaWAz?m|mBmBE%df)Y%u8n?tDKiV08cQVS!b*&z5GEq&AhG@&Y1F!_Ygc;_s8VU zXr1}Sdl;T#UM1F99pe48IX*|=Y38krW1;=~>nNlt3*ss(Z0PXBY;NElNS|SW5lM{a zG9H=Df_ou-mIaYBs)!WbVj8FT7^Kg!K-*HA$arIPAEakkkXNp<5#1%4Teu(6=UHG& zlZhzD$AD(R1CX9&L1yEqh$_6JHVYnv^aU0aCK3hCgimlB(id4^jKrhYdn4$sb<*!x zpiEZUGMY}D5>=qx^iJ=J#A#N2Fi)@>N0P=Sw%yhP@=@iG2bY14lZSU3y#pV-BS>?i;U zFn$#jz`_OcbZ(JdSS7pkC6z90P=5R=F@u}z|C{`a73CMdAUn9i{u|^UtW&*mt|Bjnv$A41kkn7)3`>$~6IkE$`JJu-=r{S%7ey)b+|M(Z#FW&qB literal 0 HcmV?d00001 diff --git a/flume-ng-sinks/flume-hive-sink/metastore_db/seg0/c811.dat b/flume-ng-sinks/flume-hive-sink/metastore_db/seg0/c811.dat new file mode 100644 index 0000000000000000000000000000000000000000..40992c68a47cca5d0c45542316693c4d93247c6d GIT binary patch literal 8192 zcmeI1OKeqD6o&VIdLQS$-@Wba%STIFptRIS6{>6$5k#P1Q2~_!CMG5fNCcxJjWIfL zrUP|g;y?x(qXQE)Y8)DkabRLh#6%~C5e}F*A+p!MXjcQ(nG>^5a=-JR@BI6ob60Mf z+*5bfIfw9)Eg^H{TCAL3>q7qf>UNF($HS#?=9%zHovW{u5H22$|8*;_cXsTH%iGki zc`u5|3t@-3H5XZDJ+GO*u+EvN;soLZ;soLZ;soLZ;soLZ;soLZ;soLZ;spL*6L{_I z=ittS&$ROU{zG>(9G$rCQy#g_sL18RBZHS+y80ks-L$%R<##^ZEPhg(Hm;tPM-NH3 zk!5#GP0okC+*tB;*yoegrvz|);TWlzL`;f8!c!@V8uBPvr%yTsl+9$s6j3pg6;nc0 zmzPk}P()or0SznXq+CYROkPX{EiV00Lm1bP z!-RF3(n+zwOiN4wlUgodN<)fi>$Ig)#6~k6F(qu$atWI?6ft9+e(994#Y|UB1+!YN zU`|6BTdgx7ohs(d42r4Y7A@CstA;9WvrbPsb=+=dNK69@W`@NyafdE%;7$!qY|~K3 zb}NraxrIeDqhi{)%gmUV4(`_FZS2s{f%lqP@H?$OF7cZcv{g8HP3bp&J zwn1tG@ViY-iW-E!pZ)OV^?Nkc;rD84z(1g=3ct^mPRY_9{DV9xX}_lMb_X~sl4fegd7k`~e2-joPwV&Wz&jro FuKvFDn;?Kp+qZLI|QE1c4v~2}?i_3<&}fG$aVb z6as-jEd+r;AqXJQgdore5_q3q*Zvw-_ygc=(r??(S=aViCVI@4F$VU^izf|A=QBM% zZ}$Gr-OF$M$A_owk!$wq86E|fq!d(`;TtGT(_TT`u%=v2KLdBJ)g3> zMl&2m%`$o>1@kD|b?JeJ!_tM^myxCa- zn1Fo@U&W*N2&)7rVMJ8+Q6h+{?59K#QwdUHh^vGsaU@iHN&-ohFeQnUN`#U^S|v(J zBcl?dWRO*fQ?kgZBq%xLRg#oE3MwfpUJyl5ClEjiqg_6RuN|lnv0hJmhgM%t{N)|1Z1|^3Tl_n*Rwn~dq zKu2YTQpBoCn^MA>N{3R$y2>i0g09M%6%??cvQ8HH=h-lsXQnY*HFHtTLiB?Z1$_xCPt-ZUMJ|Tfi;g7H|u=1^$Z#0&h*mn81$o zsq~@ru5?R!NqR}q_?D3rDN$y=}~D*nvojTe3O2XzLCC=ZcC4uE0aHd@5A{EzW{MWSs4HT literal 0 HcmV?d00001 diff --git a/flume-ng-sinks/flume-hive-sink/metastore_db/seg0/c831.dat b/flume-ng-sinks/flume-hive-sink/metastore_db/seg0/c831.dat new file mode 100644 index 0000000000000000000000000000000000000000..37fd871fded433033709ed32397858f326754749 GIT binary patch literal 8192 zcmeI0J#5oZ6ov0`z;T?!v7P_4&;l)iQho;p#K3?A0|Q8ip*s>njI_`j@ z%nS?+3>5u|t||s*1`xb^-Q-2LfgN<8X!CO*iw*b!&`ji<{ z)_^r&4Oj!#fHhzZSOeC8HDC=`1J=O*)xh(Yk3g&LJB{A&Z_v=4o!<8?7x$S$gk`s6 z@RXwQLcr9c@x`O}e9&2ZDQt9pZc+=oaf>~u6IOI*rnG}bW$uQ?l#+e-R z$~1Arn>J@!II2t=$Gquqrh^4#x>)q4=Q3#GxH2sodei4jn@%Xx!AWlhoatgonI2Ag zGsBramX#T}zpM>wz#6aytO0Al8n6be0c*e-_y-y&U7?()bVfKUjG4iA;f`=q_)+*) z_)7Rp_*i&Pct?0uctJQXOqlo~{3_fMeiFVDz81a^J`p|;t_ZISFA5igxsaIjO}H)m TEPO9~BfL!ai=QI9cK6nAS$)0w(HWtAi}B&T`R`|0R#|0009ILKmY**5I_Kde-tKXEG0&{G@l+IJ;%49qP=?X=9gs|QzpCi4)<&-!$QGdH0jfB*srAbBfdB#sAbW&^<`-x0LCco<4HO|f&7ziMM00Iag ZfB*srAb5t2iR_1zHkyd=jXmeeE9gXb|{d+s? zO5WuYotxfFFKH5XMR?^ zeQT`O3wfbO1{<-}K|tfI+7gmfhK}+1=UikNd!J&arnC3Q;H&N|B4=YK4fBODKdwas`P(Cs8QW z3WY);3WZukp%x)gD+urZ+1q_L|63ZJ{NH4LGxK|&*_nB=&Ae{QIS0SVh{!BCjq&`n z^T)q-hSGmLJ@03(`7L&?Iu60#+@Jq-6lTpXy!?4;M^JsGM_&XnN4ev)bzOyqcOAnEMkPWvvGCA4FgGI0WJ{L_^3E(pR9I-Mn z^1(!s$)li50mWc4k}0C3ObO*+_DH6TM41GYVD?I;f~qoA)Pl)MriN6R6!lm+VWD4jhQ$#43!yREttM!=CH2Jy!`)dSOeC8HDC=`1J-~wU=3IU*1*3v zklA*rbD1sj5_y3uFBs&uX;NAD2_DEcLP?z4(yP=DPkmI@_N|k235^ z-sKbhyWY$&X%e1mMBmeH8#xvTAb(B(NHo06)jUR1v9`DaJJ*f2@i=D z{g!gE@8X>Db+K$j#OUQl=40+-8e%=o>HJZTPWt0=u9ZqJE7FS(8EvlXuj6g|^Z4Em zyOMYL#NehkGfJ9-=UUPCw7XW01p){lfB*srAbT9cIIcr z%eTgQw~!ZlWUv?890a_bHM`<}zLBB1hO9l+Rd@8@+Am}oGR0Kyu5otJz(4>21Q0*~ Z0R#|0009ILKmdW?1SYZy8>(4ZJ^_e;Eu#Pc literal 0 HcmV?d00001 diff --git a/flume-ng-sinks/flume-hive-sink/metastore_db/seg0/c8b1.dat b/flume-ng-sinks/flume-hive-sink/metastore_db/seg0/c8b1.dat new file mode 100644 index 0000000000000000000000000000000000000000..acbc48a97704daa333e9c8bdceb954a3ae848ea6 GIT binary patch literal 8192 zcmeI$Ee^sk6u|M=iX)(b1cyQ5aNGdL4LAj707JL|yteDojgJIN z{!RAQzPIjWKex(KM2uc;WIpCTrXkkToX#KRXrwTlg??H5!Or}w zc=^^?Zx-@Gj|?_qtAl{IvuaoT&o|OF=a99>vg(c=T>FVkL#7z&-8Ifm8W;#5fB*sr aAb-@Cs8PXJ+l^tQqE20f9L-0ow;{@Q{4;O0DyFHaU@;IV&wfrM)x1R@&*6cT&A8) zx_p2%AA~eHwSO(c3!>o%=O^pgQ-ya4n0cWJ=L?f$;|5aZ^!EmsHOzn+Fau`54445k zUaN*pFWmX(y4l(C{@gh>ShC8JCN z43)S{LX4DnOd^bx1VkJc6D1)N4^t%(6CbNeVkTv*DM^@Aa7;=w$ur3#$z90}$*$y*c@=o$f@k4NE}>(D_}SS2+jbn?Yea1Bf*k? zlfAX?t$W$et+Esmqn8_*kGYR&i1jq5^G7vW>5t2?R_1zHkxqQbXmeeE9dFwohPQ6m zmAuO*dRM)fLDD2V*NDESEgLx&2q1s}0tg_000IagfB*sr{G-5Tw=%NOFN;6enV%Ie z-x}-fLSE>R!CGv65b$=&z;L}{KtoKl`p{1EJm+sA*s#O57sJ^yiZ>@&Yhp!3<@U@-F{bDl4+ z>0g_3#F1OTE#MY#3%CW`0&W4ffLp*V;1+NTxCQ>L1#aEF0(04ZrpfpFk?Gqaz#=MvIHsVds z62OG)WB4i_#Ya>nM2RA%vWF5wTxBmMj)Y2>l0Z@=LP;W};!{#ct3)YjWK?3546-V5 zN)~}if)XI7lBDF2S4mOwD5#_<1r${>R=hAuDp^VdWtD*9qoR_dL{U}AQ(~y86ew}j zRf?1Z8Y(495>1sdC54tsg_6dCN|lnqK9w3Ji~TBfN`M0@4N4Ael_n*RMU@t%fF+eS zrHGEoBBg|7l_g3UD=HmI1*l(OlmtWheaud+_5nt@7> zQp1KypHjzAWk6}*pvnfNi9;$wO3VHWxrHS_(rxK0=@aQa=}qZX>3Qi{=?Upjx*$!l_%8h- xeJ_12eJXt*y)E66UX+fcC#6TEZD}AitobVaAblf!CVeP9W-g5X_*>qi?O$)DT9^O; literal 0 HcmV?d00001 diff --git a/flume-ng-sinks/flume-hive-sink/metastore_db/seg0/c8f0.dat b/flume-ng-sinks/flume-hive-sink/metastore_db/seg0/c8f0.dat new file mode 100644 index 0000000000000000000000000000000000000000..4bc03d265892852a4c3b6c7a41b1946e72b8dc63 GIT binary patch literal 8192 zcmeI#F%E(-6oBCeG05OxB1bS`b=R190vvGi8tNT9ij%kS0#4k((apg@Ut6jKY@8fS z{x&=w??DUShU`Qn(!-3{#>B>E@cm{@^G7{e<>2K~I~&&>np397KsQfaEf_y;zpn0k z_k>-`r1gFI#pDy$3`M)=w~a#&Q%3*+1Q0*~0R#|0009ILK;VxAPNTz3w)(>U2RpV| zzw-&c-)i&{y?+{=-{thGc_yV4vU<%zugcCeeFg#uAbthDFc<K)1Ri+gd*I1;R=y%`gWqBD`zyrb;V)7F>_5-^+Uu*O35yHC$OR&{zuLCD39sA>$x4cE1>Dr^G+V2U>m_(Zg6|Nq+Q!COquK&Y zt3^m`L!|lOX^~uQbn5NqW_N=;6cxM1V#$b$tBxGIX=ndve|YnF=V*LvaL3MJ|9JnF zBRB1A9v+QvUmf2%Ja*{l@gv7H&Q!iOP^%(GEb{BRB+|_eIK0zsSMl0mbBwuo16`uL zvBbBqLoC@ZsA`yTpN4sc-2iqU_ue$qW7CpT)O(F;OjZ6IyM?jm z6f^PA~(t8MaW+}F&- zd^0c(PyHJBS+biae2rXUGm^`BcJ$dO<=4v}&4-3wPC1rNFMkXlAKDZ6XYWS4c|*0^ z+-RxxwBSENTxvJ6?dcyGO3blYS@84GWqkOVk<%|(d@LWC27Q%OSXY4IoX7EPL`B#Z z`@9FA%lWXgZ8G8%AutrF2=RD6bWB8`w{|HgI7S}CRpaFqd|dFhHD1;?wx-fJ&oSpp zn~nU+X>W>G@!_4bM86h!wcUVkPO;k9Y(UE2>N0!M6Zq&54J`EC$SWxC4~bnj$zkql z*CBazd~|$xgrnZ>LqGwJ35`>+wcu+qEYsi2a%n}p8|IO}DxLx5b44G@i$qU|^!vmJ z%AU9$%CLBv5UIC|15h^cc=Er+4N%^VXC|L0#z2SvN&GI9NAb)=3XjD9THFNXwOGc! zDh5z)iMxb|-Y4DxWdn~yzJuS2ya=sF9*;-DpGJi6VR0jrXCVH<-=VFAF4isdKX^`P zw|E6{lf&ewJ%8de!cCBl!##_n+Ks|gaM_=b~aKXSjZ4YC9=#%7?h0E2Nv{f@ONZnw)BuB2NpD>!B4I3uh!S9osKmk zhQ(um7kpQw*FajibVk|`CoD!!>I2ITkXGp(Yix8Kz|=rPY5^A;tKQkH)_H>%=>#ln zAbp*Koi3V_V%I?#ogeEOovXJT>fhQtynRG) z&cS-Y%SKR0Y+zLna=p%Gqu$)ztXs_^;h#;~^@f7%k*>1iOuFZnNZQo~A7qG5`yli!-gXbCt+7_1gup>yYeN zPjKy6?eX^b*!b{pJgOhL<+f)ZzE$X)E4vvKlf=iaM-&IA-T!u>!3Tc6OxCV5_!xi- zysvR^qS-Q&4ZFnPV?heoYmIKBy|qSh-m&@HI+4vyg&yDbP1)0xl*l zhBWN4Aon*RyXjWfH=%7`0{MG?t3R1SIO6X9#o7QxqU&>(D6MTSYGi^VrZ`Tt1p|Q?Z zUuDQnb|JwB9oo;HMhoXaC#qlASro+psRUlq5~dTuVJ25xE>tooyeQ(7gBXUhPVMF% zm?dCRCp0t9Yw1p8+Dn&9=Ko2l0z3=(^Ws_JATa-TNEKi&a_rQ%@qOw}Wb)zX;dd9lF8&hA=OMzvrC8h0$7Ok=xEbCXXx;c5S*}U`|7!6r@WeHU zhOfIH_2U;$@&7Afj$Vjm`JuhQ4(RZL`G3j(S=SA6Q+3^}md6++NCRLQnC&utql6P# zcWHBw%E7FKY|@huh;zYV*T{(OQjyV#ySo^Iaig&xBsfVB{04p};_iYADa-C=8-pQ4 za0%|GuR2k87jXL+8gr9_6(MzE>MpQTa|?pbOOiTq`;l{xeGW1)@mJtstrEn1MkGIUEp2WO8VEtR&25Wk#<(6hia>+ zGDGMVJqP=VbVM|sWecc_N$8*_ZLCppwB)}Wgc$rBHd>7Xva_DyZ-aO|%LdHELgj5g z#R|wfTm%(tv1$PaF?p^X_yJCK&h)oKlzzLC9jo&h{GbS* z603pW%v&u+44wt-u+YLU4pV4OhgqS%&P9-*2=cg3W@RVoy<* zwr>mmM6X%UkR*T*VrkIsUie?1Tb3o{*vT*B`{eUvc?oj(!~=LdahZ4{u>A)RCw?<>^7xsk z5yak%tUlJlbD}@MPou9yr6BrbFFu+WX0qq2rzLaWT(#=o;C_c@h$GnPM6GIn3gC^m7bT8(ZCG3l{b8$750ySO)a_Iw=ZT z;QYQ$x&jwm&Y>&5;5QzVvcN)y&@FQm+w`knR!r)`IFsKlWU`P^WwXWQd?h!86(ghV z$AXN7V$u{ySKW!-fU@HPGb5j3qW$39S|*K-sYED5xZZhWHYd^2pUH=mt7tf z$GPQkd8MH8@IroE&Q^p>3a{JJzBzye4oQBeLEx5Xz-Tf==2;eWRa~O;ENED>ZC3s_ zeP;lxzToWoRvT{fu&tGkI8UV@Wm8irq!X|V5M6EC;G@p)*oPE6sRJx{s6X_c=K9vU z?^=JG0n!QB2CCOj8*o|WZ!1d>=Cfd-&1}@33Y^ngK!^yL&gna`XDH+6f%9xS_*^Za&}OHF)5GYEWAd;e!jmV#o6xlt)7*)_}mL?5YJ~p z<4iC>Pwp9T6=#Vj8$|P2*vP^OyWLf*AbjuRHHhW2@Iiw|d|YbL+uML!lxi@&D1Glc zS#A*BXPJTCgl4utYLVdn&C(#guYrdJ*G#?!MfLb zP<`*VV;lO=&gpn99Y*H7v_o?Kcgb=euA)yrglqECFU2|bbOE_^>Pz@(>JB_VbupI7 zkIHg{pC-d{<-fc;AhCgG#(yZwyHQ7oKUvQFe-1f(>u}9T=n0VEy9j=ILsQG`60o7itAm+w4 zu)$CWHq;UT4m!%)_1-4*)X557H_8DPLUgaWL6kjPF69cD0o?89(cU4P)%6;@L42Ks zj>+U4{wLDTvS8L<=K>IIXTjrrvh!R4;_u~3wm8hd2<3AHS{N={=K4CrFC8|BzmKxT zl}ZNh^YFO7Aj#}q>*2lOJ;3LWh{WY}O~9Shriqa6uOLd#G6B5|e#ImzyNrt1$%ru+ zMCn=Jz=;8TwWbC_-_e8!JqudZ+iIL{BcUxi5SeG8L+^r$Pe%v0EX@`jh}N?#K=U_S z&m`gT%hPDGo`p}pK($aNh#%fPx zR7vOdZ)>77YtJ$Pc|QCQE1^octYpAJQVlXh$Th&Nd@Ah^PISDO8}%!hAuvFX_yaX> zt?9%zV&eT_z5>`NI=N^0&~fY6neoOyY7zx!AtOn?PCIQN7S1vNfdLH2sR@P6x?X#$)l;|Px!s9otGi<4#e5&gan`dp zLET$`&rd@H5ps}*N103x7EF6|RnYrkoT{#8WotLYMX++?Jh*)WZY*4*?WrQz)UoCv z$(sdH^3qBPGAN|?8sN?&IAEW6Jz*0i*TT-pio2;!6ORj`<678x&k$h4CJwHJU69z0 zk`n@}xqPDETEImK?AY&Zz{I?@fJ+kCvD@2#pCsq-wSdbK*fD#t0WVA7e1Ehu%#3lm z({ub;t(k$ucl^QjJ^NaEHO`+eRdMcp@q4m-sVqxaroSRN{|jXKSS(YYKz5zFS(XXp z@X5E~`{YxRr6+zM=loxROgoXsBk?av=6_r={|oWc*hdg2*2izf!dOP%Acj!>wp{c7 zHN=cOPwoP6p=ACaLWFQ1F~jow1$q9$@8GvW-^5Qt&zJ8Xjo&psgx@lrfyc!!@IB1` z<9loV|EG~X@1NrTSBWd(bM5v2tV;xXG;aGQue?X+JwpWsBK|C7++CuE#7ZvE?`^f4 z(ul`EZWDRytQwjBe2I0tQGIA!IDsZcp}E1=@(ZD*_^itOuSe5$VBur+u3{}oA*cV@R5n- zvyj2>)%|;Vu(+lNx1p&Ei}M0!iRkLa7EVG4QNU}8gMpfNB&yHFr-eI{9_4mj(FoBr zL2O^sfYqhkLW0czk$si{NW3AU`NjhqI5B+|_@l&muoG!-6*g0VoUgaHxo2;0(gQqH z3!(PUB-+oib-p;)ZfkAhfX$Rw!mmsb_h*>`g@GE+>@T7jVXwAD^q*w~)(|Kw)4zpe z3@;-j1F(#M)+3dYYOA_-@CG&gBnt$9rp9J#VXPVSQcRL zM+>{FYn=wcpWzEn;s6UCMo@%@ok-UReTOr;trd|zpo89RZ>hSU>5Oi34QT`{bReqT z+L}26o#hKn;s6U8ye3D5eIOu(`N^_ToYCR!fULQPo; z5(-$*5MJ7FhX`yLsP;6X?7_3R?P&Oah&@*{Dy8@^ALdk|sa`)GIJWwOjj z&i`5bG;|bgh0ep{#(VKhV?UO_|KSMC|AUpu{Qut1oxgHA_&@7nLQKSPZY8yGmxv1i z#tdDg0qCGx`MAFzs~#J`LWk8lxPceU+iq52_sZ3twWmnZ^ETtjo+4@;04G+O#$fMU zcwPWx=$5e*)0kQoFdk+L*^+l)p6X)$aak<~U?NYLsml~JUTc-Ss+9m7WSBf?wl=rA zz16a7X19et@amk}$x+zerpF_#JruL1Pv6 zJKUC2WefV9DUHbyz=PnK5(lXSEOZ!sF~O{D)%P`2eoVB^ za||G{fMozK1>DCnbG1EUhxr>wFJPIN&DLC|q-~9rw1Pr0Uxu4Dyr)|(KjgCXG!W`)p4XQYg2Eub0(T92`$so3Bt?J)4RK-< zltb*txZ%8P{T|*iTMnrKEDI=5q6N$OZea+!uF{!E2hf7IMl}zW$;{qpK>!Qh;o4Gq z=y`2P0Vrn61GuN$``T)@eIouWWK8xucZ3_#U4fWXnJSw@Q|%Flv! za62R6C(>UmZ;vwBzPH=wm7R(CvrK^4W>>Z+@}bvNP|=4Z0G1gCS23;y2?53N0QMyq zc)Loog^~ooLWV5P?xE&tZ=ZFGouht)AwuOsBnGg|%#QAE0SE>!Gro}^0F(LIOm^t& zo=FbppB<`&0vu#qTEYv<2IW^mtrXyZJ46RJzt{q^S}wpb0C68<@+A0p1ZLCiYQ+Et z-l5qu0<&5#CLxaFB6C#MLF7DLeDZss#ibc<{^|Q+9Zfd9S5E zLKeOBILZG%g`9cuW;_Di0eE%#&GutfL#qH-W*mAZV?ZME zUmg~KE^K>yNjDuPs?UO+r)v`5XTi_cHHr3D3Wb$S4tM(WUgM(^w21Drkby~c_B9TG zu7h*ZRx;1XWr*{$jLg?`i0D^_l|d$#_4dy!jrEV~#PeC;^E4h}`7HSP8jo5NP#X6u zxq<%zWsTJgYB2x@8LF-TY6emOG8`-OuVzr|060eGT+N`C0dOqLy_!KS1>o42cQu1r z3&1e}kwkwrgIWy0!Jl(AgIW;4u`thS2DKu90}q3c?rH|LC_n>UXEg)q0j#bDK@gaW zwFuCEIRl9RENp0-h%IkSoun@wOBzhEP~A4b@y^LhCeCr0z{3PyX`LhptYo(jc;K0wvAc*?+E2Bas zH$KfRjaWYm8)vF;aKO1F(9->8s?3GBME+S8z&&%!;}QMOZ|6Xp^S`ZZ|Kwgi|L{%3 zp9PKc8S?QzJ+NTy-Q&j2p@EANQ}+UJst+lo0-u^L^kQ0SIzvtS7Y3t3KHv9LFdzXl zHz1n?al>O6RVLJIKgR@!0C?M=tra<{U+5$yvXKaoTd9mH`J(p@1~5kBR)PFgjidmu zu;&}2NdaIPfCOgdHXFB#Yk+f90z*0g3m@W}bdDt-d5q2`Jjno+Y`L7t`>Z2?wu3jF z_vA)R{3}2@0L#LBSAcW?mH~)zRD0WoXA5rkpJEGD_n(p)z%l_Xly=~xiCe}cAo|Y& zH|xLAp4=1dv7XYr$i(_t7BG>A`-SSwO<4A2H6)J}JEjfd{Q5R5#mDPj4AmEi^0RC} zUyzN}+QHgm!H)S8i0_xT`@>3cds(d`YaRhfHiW^URbWU22n#kR#&DK9$mtmsU*VBG6X@D3|Wnp z(IJTOv+$wI5x(tC0Dt~Kq@QKLjkz%vu@U)aS%Ef%t2ukNU*WP(C5}B)s{mN|Q`F6W z`5MvwLT055d&qj<(68Usi1V|+=b;2d`&sbd^*JVs)iQulAvY}La5H`PyUt-umT9}@ zn5&jJ&VKu7PfPLX7n}E1N3cJ{h86>NE2Y$FcSn^R$}$( zq>JEVKtxtli*$j(_Gr6O!u|a{auB%Cm{n@_Ws(J02IfmaNETojFq^8qtMO$4mWi3J z=D~r?D?qXU3*4-~y{j=gGsyxh3(nnv(Va;a(6?cA2gVRevH;5lkO^$7R!?AbXi^1O z=+m=I{w#8Y@2ExJl9g%g%h`CJ6&<1N!482?8twW@oT>X3n@t zngGkhOlM|vHPQoE;AZ{pote?mNDE+DaPG{Ejz$Xt^lez3nbF-y8DQA}PpLjJ>ClYP zljH%Gh3O9Mvi}K=no2MsX`o*zmxiTncuu9qs0nRo_g1UU7Z6DmU}1wMCgrw?fa^9@ z1$8$yH_`-H_znXyBljgufMo#Gq{)T{E{`Lyxt*i{7BcV~>2@8pki-Bjc&o)z&-J{W z4SDe7we)&q=CBJOl#Aa*ZohaJa^}S=kTa)0jVwBS1le_Z5qWj$Epq<9fo1YP@JRA5 z&$p`TtMwTalN` zGK)vTpTRT3hh(`Z`Ttw-TMI4xR_GzA0?7LdLRaIrjDJA&!RX^}0sar)75B*U8ay9g zpTGCrpPl0WyIl7FyMWH0EzRfuu`bTZ^KXopM;d@0zIFAz;+@*&j`;tuP|6Hp3hwlV z4C4JPY~TrSqxr^GyAHdUnA`1A)bMP)#Qj+&AY5A}{B}q*CUJ>KKg$BdrP(D0lxTbY z1S0)~0`9}$v)3IMfViZ|bcpkF@#k13MvR|jU@EZs?<7NXpM`9uExKtJ@qLyF{j`e+ zKMUNfzh>G+bf0Bm?zBtY>(6PRQ#P!A>=lNl_&)1*z-53=U{NUljLT3<04ngXrgG6+ zAjFIdo`1o;K}$UVpu&NMp&y|q`;_}VtVzxKbFd*WC~S>~HK~bz4nEvmhWHLoFL23@ zaG<25)KIhk95YCd!Ho560XJrY=+tlwHSe#1ZarMUjn&xDiP{(P@RYA|$(tc#-L)Rp zWDv7wf#dbne{>M>dzJxJd+XsK{wsqFV)ZO^WJ+^9I*4d}wtx@IDtRx6fX=1+=pf?s zEOe;Wnlf6 zXYL6Di}~;utw#qDo!2!1S)4fpxa`}6F+#;R;`1yM(97UgK!{W0i*E*GKCqADBFL=( zod4(`BJ?`-wi>5fT!ysSjLSslS?JKapyD_$H+is_TWkgkYGJ?mm_Ckcp!u7vXEOXc zZoZG#l~x<62lZ+E0=-8E5vymx&;RHk{5g8BrRU%r{?hr9|Gy78|Kb*M?DWr&v8V5q z%>OCKpHp8$z70?Q!}21z=KuZpK6wzooA?=iD{&7VPn7X}{LA=0e!J8HE<%LZhw(_P zhh_ACabGGjVi};{pp|lp8nGc3{wd^~HDW|8Gt}3#pI0er=2R!6KZ9Th*}o+!`9H8y)Xd3F zHo7(B`#-T#)QB4uMy1LqS1{*XoI@SdUF(sRqDIr4*fEWQ5be>u%YRfWLXTjr8tpC_bQQOHkLBC)^Nd2}vu05HL z4ebWKW6q>GKEVzNjk>d@mf=H_XLO%jDQa213ClCwM^}nkj*pHhkmj>1MJ+G!3xz@s zy4AMQs+_wO)L!%9l}^<1H+pR8a^&QTA4j&lxGmo=Ad80m|8VAi4d>od58{#3i;?Z8 zu0l?p{0x4YJc`U5*8c-S;ytq5MBW~M5E*{_&k!?y6*6#G{|^YUZA6GA5i|NWJU_Y@ z&yRdtmUrT(k;^3WzaPIFeg=MP;bASU2GRSi^M6qnSHa<(@ts{dDmotE)K3!8l{)kc3hK4D)=n%vi_xBXh~5V6_@+%aEmL8p-- z8Ww8QA$dy++kDb1wv&@rRUmd)f(x~dzhB;lc2e=2ThMMNY90Tt1Y51&*TH7D6SZz% z2bC^Y<+5c|=fyH&JmlD9L)n2-%fY(q&9SSs`5f@M z^iMU3uTy(V)$mu|3{tIF=#ZArGsjMvRS~4;OlCZZlTO|@r9-~>X^>jQLPuIL#~KGx ztXLMz`s-W(60KP9b1I((30H%Bwou8~c3^V30-D07L7LTII4HqmjJ6u=4p^h#X^VPHP0M7DOD^B^C+Hb z4xfcjzd%dyR8#mY`1$otgG4Kql{w{3gM=%V34OH_$6|Ob$$ROSzeV&riwoQ}X=e*O7-OPvED?i}8Kp!}#6A)9_pIU*hrjYvlWD z@ja~kNB(~Td3fwHJQ96CmN%kC5Rp3zMqZDKK%{}T!Vlv6@Sn>vi{-)>WqF4zFTpZ& zzbwbHOkioeU6$8jDSjf$dt_wY{gelDeF9G&m4=An-uofxrWS z2LcZS9tb=Tcp&gV;DNvcfd>K)JOVs$(>u@oe!%}90T(K$V&H+m1Azwu4+I_vJP>#w z@Ic^!zypB?0uKZpI0X*`{Qnf37koAFK;VJE1Azwu4+I_vJP>#w@Ic^!zypB?0uMYw zJP`2zN63W=su_47@Ic^!zypB?0uKZp2s{vYAn-uofxrWS2Ts8Q0slV*=LKI4JP>#w z@Ic^!zypB?0uKZp2s{vYAn-uofxrWg5Dx_W{}FPbf@%gH2s{vYAn-uofxrWS2LcZS z9tb=Tcp&gV;DJ-{K*0Y`!Fj<~0}liq2s{vYAn-uofxrWS2LcZS9tb=Tcp&h=Bg6y9 G|NkG$!B*A) literal 0 HcmV?d00001 diff --git a/flume-ng-sinks/flume-hive-sink/metastore_db/seg0/c901.dat b/flume-ng-sinks/flume-hive-sink/metastore_db/seg0/c901.dat new file mode 100644 index 0000000000000000000000000000000000000000..92d1897f93ab5006502be6361903478eccd27c23 GIT binary patch literal 8192 zcmeI$yA8rH5P;!xio|q`Kt~A`qM%{|3MxjR0B0LVP6$Y} z=(m)Medol=*X3j_B1SJaG9Pmv(-7-vPUnx;sHHzHCt6wRWkuR?lhMO<{dF`Re;nL8 z;i=?ZKGD7E&GeH)!oEuMXWDHg#{vNa5I_I{1Q0*~0R#|00D*rL*dI1VR{CY}2Rrk# z;_X{wyjg|@7Ap4cGuo6mg#-}+1Q0*~ a0R#|0009ILKmY**eh?VRJnX)V7sDriqb>9R literal 0 HcmV?d00001 diff --git a/flume-ng-sinks/flume-hive-sink/metastore_db/seg0/c910.dat b/flume-ng-sinks/flume-hive-sink/metastore_db/seg0/c910.dat new file mode 100644 index 0000000000000000000000000000000000000000..76bfb9919ad103e51e7b37131650afb09937d8c9 GIT binary patch literal 12288 zcmeI1yGjE=6o$`A5+jNx8ZkGt3L?00pSXw`;+q!};Zyq$YuCcu#{^`Z`)w8)mdWqC(bM~F{fh^xxO1VNk}938}=kD!h!xQL63Ac)}PCgKw~I667_ z0OH^Sh=Wfc=zmVyp4O@lASaM7`M)Gh@^#7&`v8DurNv@qnf>SU zkzKPo0CD#uG&eWnKbPSLGN+$hUu>sW_5Bu$=~uJCxxSmE=M9kW=^PC>9cTe9parym z7SIA(KnrLAEuaOofEM_V1x_yy5cbTQb-(ZDp>9U4p?9qenN(zLGi7iLPH!UM$GzUg z-S2#aS$s;ZJJ~r(Qx7y3umSDP5&EpB2ea3Mn{ZA#z9w1hK!3jF(YJh z=!qF6e$+#7K;1jFrV$j8DeKyqJKDgNc}sjEe;^5t$qo#l&Ph zEQ!gJ@v$r>AroLlOo2>@RWT(p5!S?%$;4O}Q!xf3Xo#tjiJ>W`MkWtqV(MfP92e6d zQ@{x^O)|ywPgQCGEuaOofELgKT0jeE0WF{f{$~Me6B6_Ny2@N&wwNIiKbUXKXXXR* qmU+cIV;(Yhm~G}7bCEg4Oqf7~cjgQ8k$K0wVJ^c~+IwZ@eCZ1<$w7ty literal 0 HcmV?d00001 diff --git a/flume-ng-sinks/flume-hive-sink/metastore_db/seg0/c930.dat b/flume-ng-sinks/flume-hive-sink/metastore_db/seg0/c930.dat new file mode 100644 index 0000000000000000000000000000000000000000..4e5fa8b7a46429ed8362cfdade4421650a22c748 GIT binary patch literal 98304 zcmeIuu}&0G6b9gP0dX}X3PKPyv9Pq+UDim9g)uyUQLHQo2^1tiA%iuAy|zAvzJib7 zTi94y%AMKST;ifmEXg;UIrq%D`|sWP;$1|feAm}?VkPq+G4}ZC)|4{z&QuY4JICDCQj{jzQ`7sY>`uU!0T(I_@<>9{$z945r zfB*pk1PBlyK!5-N0t5&UAV7cs0RjXF5FkK+009C72oNAZfB*pk1PBlyK!5-N0t5&U zAV7cs0RjXF5FkK+009C72oNAZfB*pk1PBlyK!5-N0t5&UAV7cs0RjXF5FkK+009C7 z2oNAZfB*pk1PBlyK!5-N0t5&UAV7cs0RjXF5FkK+009D*PvFUq@4Io5-z&B6_8_k2 zZe{9C=cXpjU~WDq*V4DLd##`Q+1QHL`IBzUMz7sljn!DG-tHV7@9!N|+Xp+x)#mQO z*8bsMEX4J}Xt@4p^mwD+kDDcPyK82#WY)T7mP+PM$4t6aGIzUW25Gru?sduxhJ{l{ zfB*pk1PBlyK!5-N0t5&UAV7cs0RjXF5FkK+009C72oNAZfB*pk1PBlyK!5-N0t5&U zAV7cs0RjXF5FkK+009C72oNAZfB*pk1PBlyK!5-N0t5&UAV7cs0RjXF5FkK+009C7 z2oNAZfB*pk1PBlyK!5-N0t5&UAV7cs0RjXF5FkK+009F3DUh~f6p*d0~PjNfdBvi literal 0 HcmV?d00001 diff --git a/flume-ng-sinks/flume-hive-sink/metastore_db/seg0/c941.dat b/flume-ng-sinks/flume-hive-sink/metastore_db/seg0/c941.dat new file mode 100644 index 0000000000000000000000000000000000000000..b403439df87f360775c1ef1f87a23a5fc117d809 GIT binary patch literal 8192 zcmeI0O)mpc6o${4sdoB3ZB=ROV<#ely^X{VuqY80BGE;oEEr9!5a~u@X~Wh+M8tm) z#Lm*Mz@0N~Pbp_AmwVn%oNw06=s3%_4@0-Dr<@*Ua-rt9>i~+Fdjw zp1B+Vw=JQW-1NoPyC9>VUY&0RZ*~0^6~^Dp4QIM03A`_)Ih~^~HSK5unt&#t31|YE zfF_^`XabsmCZGvu0{=0Av&#d7ee-4Q{rxNyO>eB{TMIpvpb(CjBZDS5or8cMpE@tM z_xT93_!Ly5^Q}?a+%2<^3M|2DYs_rn=}5wUHA{@xopS<&0_45jjh($(b;nz;Z|r#s zoApX zt9emP;}EL@qUvIi)qMIhoh{PMYTWp7g1^g znt&#t31|YEfF_^`XaawfKdO`L;z5J2PNW$hENZ6B`y literal 0 HcmV?d00001 diff --git a/flume-ng-sinks/flume-hive-sink/metastore_db/seg0/c950.dat b/flume-ng-sinks/flume-hive-sink/metastore_db/seg0/c950.dat new file mode 100644 index 0000000000000000000000000000000000000000..56aeaa9f552de3d354a2b5321026bfbdff30289c GIT binary patch literal 12288 zcmeI1OG*Pl5QeMG(?>u-$ZKLuWC(%-3A$8J(1i;ji@0(nW}%1?GzrAV&I_mq&^ven zFW^EEJc7%30wd{m74#rnx6S~HA8J85`4%tt723|Gx)x#asKVC9(m|# zm7S*RPRl>J-4t&gceSB0G01s&zW;D_buF2Cw^{n+e5}le>ko|ab(a6ycwi?)5C8!X z009sH0T2KI5C8!X009sfEdpy>w~N%`d-mu1O_HPbLf=!w`V=LEquiMmZ`vJ~BYuc# zv`bYI$*>2Q5kFTh`6u%#surCizTaD&f=h|(ij9HMN zNxS3NCyrfr>|<3SD61>d!JWx6ExvkX#VPY|*{OKFIbD$H>76qzv$yQ_io7n!27AT4 zX$7BH6trjcv=vP&*ru@~0|5{K0T2KI5C8!X009sH0TB2{1jGYbJUQP`mgAwycT~Pa z1y*KcPF2+1w6v%6c6z^YDR==dAcz-COk{NI;9#6^V{p(!jSeO_m^f(SNaEti z$mqz(=wuif85!C1KW}Nj=A*i~n7>J%w$FJ=+qVfxkGvKUxNCd~$P_t>!SPY&?*BTw z82!hGJ$K}#yM`jsAcA{w_xP`u{J5hJmfyeV`;oszv2g5eurl%@W1i2~47bM|@yajY z7w`-C1^fbj0l$D>z%Sqz@C*0_`~v^i0-wLWg}ic~Y4H1gQ`X$0d*?po(mB%?skA#~ z@Ik`iiGZ>9hEE>+&WEhxXWwR?o*hlN+rYh%N79ogXUF361@S3~#F2Z8>BR6n0VK&H z;VeSyAH7h(Ndq&K1ADk_8D~4=e3{2 zl=bYhm{woVK8G3W+2=8`SE< zVm(L4GH{ksrzDUy!}L`K4Twx&l@KfpCBTD00Iag afB*srAb!iQVM$HECXMBo z$zVlIp8I668Z$oD)RehT4(n4m<~SKgr@u&|gAC~90pqX~o27-MvCBS2y!7R91rz>pYsU0fWE ziB3-bC&oczqH)mw0?%`L+fv1H5+~#LhW>hb?)~+g`yrPVc_1R{UUnpWCVd`^=J|tj zZoD5KzfAw}<5hR2?p|q;z2-&g7I&Y!9=~tMUHQmN>N z#yImVo{QXj!foV+T$h{ixssm~iN3fkrXrb=$<@tTy;@tZ)tAoSudd9mZf@7=TRTe^ zYirdNX@7s@uGe&UBO$Jl)=YS##~K;!i=-m0ZOks7nmv6X-qh~=tBBIk(_`PRvz08(GaA)mZ3kGrTbv(Q=VvUlP!y99+p+%YatA}jdzN93R z@v;FAnQ!#Vv7;L8hs-wybSnJT%C@UPoepnISYxR1r;$Ve0R#|0009ILKmY**5I_I{ z1Q0*~0R#|0009ILKmY**5I_I{1Q0*~0R#|0009ILKmY**5I_I{1Q0*~0R#|0009IL zKmY**5I_I{1Q0*~0R#|0009ILKmY**5I_I{1Q0*~0R#|0009ILKmY**5I_I{1Q0*~ z0R#|0009ILKmY**5I_I{1Q0*~0R#|0009L4O@W>tvgrPE{bS4CvLC;*?0G56ZvL!g z^vPzThyI#JJ}>)g?)g0DGjAEa@wx1C&@%DW=Ubnzkx76=GbC&Ya=WCy5WTWAE I{C)ZJPt6lu82|tP literal 0 HcmV?d00001 diff --git a/flume-ng-sinks/flume-hive-sink/metastore_db/seg0/c9a1.dat b/flume-ng-sinks/flume-hive-sink/metastore_db/seg0/c9a1.dat new file mode 100644 index 0000000000000000000000000000000000000000..bde1b52cb9d2d78079218344510a8850631647a9 GIT binary patch literal 8192 zcmeI0JBSle6o&6Vkz^j3nY`Ax?z+3`W~1vS8{?xAY%FX93$a+(%0{#jT){>Ju@MEE zEkqCm(Zbe3J8f(%EEFp(?8HJ_3qkKWC&|gWYAtdG^5xF|-8(b+C(Vqz5&=xR@ z%yA6;c;?4H2HP+3A9qiBbJ2f7=PJWV@F(~Cf891;ke`0z!TNbwHFhix+MoOvdSfS% zc^k;*46km;Y{wd~2CM;Vz#6aytO0Al8n6be0c*e-_#Yd%b@wvdn*U|P=lfOXpPJp_ zO~19XDUqO9@~;f8yKwX(;J2gE;^A{X+%n!t8r>gQdi4B55N{<{>6 z-Vy|^;P(**kU^ws3nd|9Q6UmlB~lT}nnaYuNHvKmNswuhP*O%N2aHfrm6A$P6_ui< zDkqhpE-FJqRn@DYhzVAHuvJx^R0&N{C2UhQK`O+gs1Vy#HAzL75*1-u)g-AHJ4D5p zQ8h&>!K^0Jl$0?ilN0QeK^ePr&J4{-v0IZ_N;1sL~Eh@(zRc%rg z>}BQe#i9%n?2|zmOR}nhj;>nttB|{{CQFo5+_EMeN~*4>Ntcou_REoT^i}mp)p0;n z9S2qQNi}dtR73unMr*(tum-FFYrqX!?5g(Z^ znU9&bnQO#Ciy0Gxubi(ik8-lkyve-A`8?6RVLl}m>aOP3eq(-MzG6OOK4M;Ao@Vx$ tb3}Y+erCRAK4IQrUSXbL9%pu#HKrrFFUKjUE6P}g>&`5HdT{7Q)}Y;)p%-|$G3So9uG9NUfRLaudTJat9{Oq$Ye zw3=}?Y0RievdLgpjmsvBff|oZfH^g3GENdhH5oQ8=GA1`co?Y(*!WmblVg*@qMDFR z8cS;OY%*9@6S2u+MNNTCfK@dmHaV=RDYFT&uBJi;bI?#zWfMYEO^r<+Ce+l~L^!FY z!KQ#yYMN|{@t(AuGyI@b`Ooo4??gvx8uKVnQ^nnyEiZQ{K#if%zv>5EREb`%=Nry zcw@|rSH1z?fN#Jz;2ZD__y&9fz5(BWZ@@R;8~DE(xO4v+%vJlH2JiRh%!(Zy-S#aP zwwa!Y#qE;8EfWqG0>+*VFCM(-!z|)`&n8~>XJzRjun%&^toLVRwLl&$cwvw6V7N*E z6R~52%7jQb6G$e3q%ujQoQX&#g|sqhWSoggCWEXpS>&AAA(N+yj- zWips@CMTIJI?CiQ?Mz-WdCVwNz)oihk||{q6R1!rnDgBrTZ)XkzZb;&f$k}^$mz?p_*T4q_9wpnqeDVYg# zP?<>_a;7DjDXc2f!C`0Gl9|R4WoCMROHbc`Z@@R;8}JSI27Cj)0pGyCZy;JXMPs5d zVdOjciF`{wCm)md$Srb%JWrk`Pmo=*O(uoG5Arklj(kNvAs>*p$?N0=@+^6hTqZkY zR*0YESMme-ntVz=B=3@&E9{RW8p BTMYmJ literal 0 HcmV?d00001 diff --git a/flume-ng-sinks/flume-hive-sink/metastore_db/seg0/c9d0.dat b/flume-ng-sinks/flume-hive-sink/metastore_db/seg0/c9d0.dat new file mode 100644 index 0000000000000000000000000000000000000000..06e5d765a89a1275711b03dd581b44a6ae0e5ad9 GIT binary patch literal 8192 zcmZQ5V_;xl1Y!_i0+VcD5}AO>BZ~pq|Dga)7$-LXC{qHF6#!8pK+FTQoC6|;lf-I$ z^v&O$5FKE%KuoAQ^%o$v0YnlP3AU4v#ke$&ijRiCXb6mkz-S1JhQMeDjE2By2#kin zXb6mkz-S1JhQJUCfn&lezcZ8q9SrMWGlNF}Fnh%?c{IPVumN>SGG0aqb6x2X*q<4D<|*^$Zw<7zBb-OB9kS6?{`m5`#V6(djsQW``<$prMw!3b38si0y7xY%)IC58de zx8$?^zt5Im=fe(wXs=pgV`^ghf7h)ge_hVi*=Wy!f!`#0xvqaU6La_d;^Fk* zEGvwkI>jO#MFX;dP literal 0 HcmV?d00001 diff --git a/flume-ng-sinks/flume-hive-sink/metastore_db/seg0/c9f0.dat b/flume-ng-sinks/flume-hive-sink/metastore_db/seg0/c9f0.dat new file mode 100644 index 0000000000000000000000000000000000000000..a12ac62e399dfb1194b9392355cdc471ec074ebd GIT binary patch literal 8192 zcmeI0%}&BV5XXn6K|uTnns_2TdeI7E)cO$I> z_yQie>CJ-&owiWWK&(g2Y|?*cXEOiY{iT;Jv;iQ3k2@q!qn-{*uRfT=_@g_%rAJPS zSaJO7wELRHCy9&4{M<-Q4XY?A&Ghr0mm7AsIp1;Pj9(n3-kLuxJgS-UM1Tko0U|&I zhyW2F0z`la5CI}U1pW|#$C-yWXyJx?_xmg)a2DLBgQN87LBbE@D0a?8p8DzdE5;I> z!x_Ms1go$jteVP9R;hEf&ZzE@un^YZR17W;2?;b<3){SJh%z&ZmfqG4Uh8inDv#qamS8cg z(N_wJoW%A`(89Kt5G>Y5g-CY;2#r^t|5=tZOzrl l3ml=|!7(a_9n>@&;NhQP8}$};Q7>T+bqms{IBxDc@1M9+&n*A| literal 0 HcmV?d00001 diff --git a/flume-ng-sinks/flume-hive-sink/metastore_db/seg0/ca01.dat b/flume-ng-sinks/flume-hive-sink/metastore_db/seg0/ca01.dat new file mode 100644 index 0000000000000000000000000000000000000000..baf01f3b7346555834e0627716800e85124ac928 GIT binary patch literal 8192 zcmeI0J5R$v49EXDq&97Nijj?l1;SH2i-go7#DG-s8eo7c9BokORcWZi#=?l$d8~X6 zJ_R!qV&ofulQdC51%(y)q+EPaLZ(!Vb z4z<}Rv_E?AX8}9d3oV)VKtn@`Clb;EvzXCt#gcV-R8>}09&257$$Bk(nm5wI|t)OS;dIWx!#EMP9F-svfL&Wvw&@Hfpg`|#K1W#5}Q8kWSyG{omS~q$G7BQbRNI^U2 zXSJ^}deAE{jme}jwv<~`>$MZ@zd$eg{^VSeu4jAn-F@f2@Gs(>2`~XBzyz286JP>N zfC(@GCh$K9JYt2q<_BssK@IgP^~wiWCV3e1B;N<}9tu?KEApfnLXqS=PQ&sK2W}=m E0n8WY0{{R3 literal 0 HcmV?d00001 diff --git a/flume-ng-sinks/flume-hive-sink/metastore_db/seg0/ca1.dat b/flume-ng-sinks/flume-hive-sink/metastore_db/seg0/ca1.dat new file mode 100644 index 0000000000000000000000000000000000000000..6b5e3a2fcbdd5d1fb4ad70b9a5df479eb06fc66b GIT binary patch literal 98304 zcmeHw378~Db!J4}M|aPCYc!gBMp8qaUDaI5s_vSZ()6Y3?wOHxmv(nol?1{D86>oT z1p*`xAYeHb5QoGmaY)?u8nBle%waHKbL<*qcG;CdoU3Kc;bs398Ihe?U0wZ3Xg}-o z7o~4P)miz!jC>Ut75}?JTqlH(_(!jes*S1E-U62P|^|?J@HDEPhHDEPh zHDEPhHDEPhHDEPhHDEPhHSn)R10R3S`(G!n#oZWw{~R%dzb01QpZUDwgGdb2JtS@x z3C|MH^__;T5Pd1K^6&J~H|zfsQ4gUfQ(|6Rp^v>>9TOsaFWwj{K}cK@5l6(Ka)w^H z_HZWC%nY}irP6S|)hy%B%p(3<%8q6-A|(>DW2b79mHM~{3wg+s5*Ci5#4KmBd5CI~ zn9}faCRZFTlry7ROd*@iLChAhsZl#oYt*L3YU5+m6Z4Z(vyd0w=*f!+r{e2YCPVV_ z!>wE{H(V~|sh4YZD|FNo6BSNXi5V?tGw9axa3-6}4lfs5+2O@pyGUa2D&$+GY+1xa zWPV=VS>$|AL`*niN<@CKIl7!_EkQ(KX}FYK8iAbrh%2JGyqI4c&4^86!+3RJVtTAH zSD&6*m~Yf^U$M9+EiRmKB`sST$rUn7xt`L}A~`lUwlH^YO7$w5^yDOjQ&VzAip$wd zb{Pup(W|70k4;Zb)~4proA?n=N>Vr{l$3I|RLbN_y`^jrgY&bs#=^-)Wom9g^(eX3 zla&(ANhPb9EsmBmOBew?dV~>hYGPrcGCQ|$s#a;tRcjUMTXef8FD;x?O5RerP|Req z7>U_TemK+0j}9-DTP6L~WixFY7XuPk?qP;M{hnk!#m zX}C1nqLGEy(nzy~k&Da6pl-Umh!}8Q;dN(FIF~9ZrR72?lWq4_6~4HcN~1ElaJqIm zdXvC&qo2Vc;ha`-3QPG>+|~MWAYx&vGFhXqZl@<_gO+FiXEk6oU^QShU^QShU^QSh zU^QSh&{G4U--sEU@GEM=r$rrv{pDf|bosrCtA9|Oz>&H*iBG>HPT}|y=t$%My_a7H zqZZ*;(UH)8IwBq;p6*`mMc?}F$Nl_Y-XKQtiDBmMNz|^v|A{At#7%zsuQ3U4*}p>i zR~S6xIgbw$hX=uJg2VUb6ks@tn`DV^9Tv`$U1+h{20I6>>cQGJilN!LMt$n!!bBa+ zO;NxY7|l~$5sSsqLN?c1L|jD2#x-(-aISHY?QFh~&99G~)yR#)`C~qEP^2qoPAg>gLC^{KHrvd6{t3g_8; z39!MCgcFsC*_wi%FP!J_;jozyd}@ATLZSBw=f=L#bB%fT+7}4txqLKPeX0|Ba@jAO z=kcMq>$wUIW%GO<5YF=zJX^>#3)w>NS&b~JwZ?c&U;ChNZsLPsuryaaB%GW30?tm< ztMnb?7vQz--Hx;t%ef^ok*;mur0H8v!U;@*uj3M;o?F!SBcm$b$toWSgN2VY9}NjQP&@6CK6 zT_)iKX1}-aC3KyH6PN+t%9qh?5>8+Wd>dcHs!2G3Iq>a#e3wZ$fqC#9d?#uJ8xsa{qlFcZFukHvRK*t+t%=s7WgIq}_m5uFox z0yE=#6ddzmvyjg(_M92<)y>u}neCk zQ59nFX7N}d20kwyhxezb9g3=UOk9E^F*;{pLOdGn3q(bT^q19<4TOq4gLrK$qTW~N zee^|iX7n?JjZTo9=n%D$uh5amt%QwC3I}b9n0ENHVjS)B>3H}OQ3IF$m15M#z@~+K zgm@I%`{;am7qwvKm!fTo(>QV!9T5-EIq8>@{PYGwrB9RO)Zfr`Q?FF*9DSPFOjk*M zjZn##(K*S}boFGCjwe1%S5MqPSC8LKa$+AO$?-p=^WzwY__RP*kN+cGJ@#WdBHmAO z#C3Gnz~Fp(4gS?f{>;z+#rMTw9Dbhq(BCBefB4=T{;v;3J=TvgqK~b?|Kn=JJV+Nl zAB#!GeahhbKP-}bAf7QbJt&KT+%HmmDAu@o!6pxiv=0YAaIa7W{348jmQ74g)*6@? z+$$dw1AGbeY)?&lhs|P} zPePF*K-P|!!5~m72#VM)w(~`-9*3cAVh10;Mzl?6huG;Cg1Cqt_tEWf3hffR_!7F@ z#!hjNkB24^3eoXIhIWfXd=VJUZj?x4eu4~Op*`YoU+DQ6ER$*m4SMk;7tRTvp@*S? zFqhiGTwO(TBtby`kdKG{+{hd6X-a~H{t;gQo&zOdn&zP-Xs2uWWySuOi%&P)l@bJi zZMt?}mH;&s!LU@HffZ6O(37B=t^v4z5&AbFxm#*mT{(kzsvL#$X!Aw4=pOu#dht{g z-}73jt#m9jzCob<3uWhZQrqZQ0`%p{TttgC=QgP=bSyN!16)3^PlS86jI3l5G&9SEqPvU+{x zTTHXEg{|7wQpZ4DMqYX9D;U4KP(Zixp*f zi?#?n2no{@r)n^y zOuAN-@Q^kF1kueVn7-0Qqz7G5MPx7x{69@+RuVQ z%Pw^4#57%ZgJ%4Ffi*)34QQ^<0%L`HY2{313^56_`b!F>HQ8qgK-UR3J5w1`40lkI zd=@zPNQF|G-m|bE2eUJXGC5HnGacV3c6XUl*7%+=IA<@PokMPs`L>6|o=$kXJX~BV zWrmkpi%11guk0wYl*$6%cD*{;s9mNelJjN8g|^D2a%Qo2E{dNNhDGEendM_UC?O)M zVlu~v`w1biahd01JLw?s30YEjRtgRtETrI!QxHIOlYA^6h{+$o%GhP|^+s)c!A;!R zAur)ez?TIHr>19B&JU~rj(gjMk<2pkkUR?CEyD`f6Z2DJZoEV~fdnr;7Ef2BHak7v zfR9c;o@wMjjqxQw>7E2Pxhsvtr*Xaj3>a9GYxotw9h`&i(`o6{Tzos9D`xV&Q^Iij zQ!t>wPpB*i^U^utf=8Bf<&g|B+IpBjI>29-&Pf;C8qGIzrQVrQ{>KEyi^>`4oZ_Q9 z9uyLARyuXQ0RN*x=yTG!l#lLwSV+R<(mCx)SZwW9c^^DQTX6v+^nFO!48Qc^BT8LwpUyWAQvzYyKOz2I|$3CEe!;LOr!E??+5aOfAA?%Rx<>UrLR-@dULc(`jI_G@Ci zv1cov|9>JT41YMH0222QXHE{Q_G)$fe#OaONUS&(R_!IkpQE2veEx1aGxbx|9;G9x zHl3fklcMO8Kce%~N?PI?lApes&WwMM&WT?`^5YMXw0KdS`A&7t{~$ThN$MdY4)8wm z*K|(g6?9Hyp3aGEB?;kg();jjs$Ee0{|Mbt=svni=zYZPL(2LPQgTWh4|%K~8N~Xe zp8VhS|M#6Muck5=n^k%CYx;A$IRWKRAP`_32?!VTd*wA#kf}gKz&bMUdlZl{KUsBM z>&ZZTz&cWtFK~9OQJo0|5c+U~ySr97ZA#2|yY4JsiL)!gB01c@XC(DvkOaCALc#HT=V;w}_6O zZw|x)taJHb&o>18|LZ_${X`BYGuSo;A^@0R6N9oT;QY__jsU@EuguR)!|mCqQS}A9 zA*G!c@cyqO1EW141Is`}ib2YrKny^iQcMbn+#Lu4SVxL_{;66A`9O_Eb_9X|){#IS zJXP-lHn7V4gu0QP+DxDyoZc#>cnZG&H!5bAHWjc$;K_GIm;td(oaZ0Jy*q#>U!%Qw zXS>+T$74BeOs6_1X{$Kj2SK-ya>3WqNQR5Ja+00UJp0v_CdIuL`2TvTt8j16Bi8 z16Bi816Bk7!WxJ_O|ktPwXt6ipO1ZrSbFSwV&Ji&fM-7XOU3;Els=7KPoGBB@yG+j z_9Gu4Y~)E4)gL)RZTR~t#A1YbLYe|mp)@PDk+^hYa^06qEt zgD7&R)?TY{x#3o66jn|3(#7#;?Ka;)$-X(m9U?n5^Jn3)?j5^)Y@+5xuSWK2($9i} z#CbFd>?d-b=K3r!`i|V_`sDni*Y`cz>(9m`0Njt?tzG{td~n4b*{q8#TQ@`b8`S#XlT!%^_n}fP1AlBo~}-$)zkehagiE#i{)maoNM*Y zI0o^jcu&{gmrRI@`A~FRL8*csE{tJ0hmqwOH+I@H`Q%6Nv zUzq_Vib^2>CQ>?eOpNl;^r!_Q%~Qw4qxf(*hEZ-p4V2V{;?aB{o>AqR)uSYJk$6o1 zU~N`NT`XLeG)vXc*Yw#b_*C_!(=qWl{-r&%@d?MJ^8^j=7@^&Q%E2kAmgV%==>f{8L1Q?af7vMQbojavYPhg2~JE6WqH3&!vcS)U{ zz>)w#UTeE>^RI+=S6w*|N>nL>1WFD9lSe!W%FS z11|b&r7lV{T5c^P#lrm1ba(Os9mtJ%mej>)SZK_-ov9IjB6T4emH>Tu^AJ^TFo`WJ zG{*0$#E55`nPcomf& zp3jHs3!Cv*MdgQ^_~l=M1Rx2fj#W4$i1{kl1qitHgxUFQInu6%W(UI6R z$%(02194SZApT;K6F)_AVre=v`Z;RjUnV(;S1Z)5BnL%)a6G(7JUo1aj)dQ=nEx## zGkRLJLv()RZ>SC5OXoyhOCI~k<#c9b51o^IDYdCV)t;uSC%;5@t^NOQm4PSHr=cTs zMfpQz1$Y--S3Xg-!z5q)gk*|)=m_{fWdwNqf8)_#eJQ~I&nf@^TX9bC#q|FVdi;Nb z_pT@Z|NSb|tI@?0fk5!_+AO&;V=-+4;NtZ9JeJVhpM~?*=LT&Y{r#c!>OPjz)IX0P z;^j=+{4R7a=3HaE6CjBfheokGm;mol?o#$LA=23orol5=a z^sFwj7~i5(oLNA4SF7`5r)zVnj+|C^e5*L31_w(7x*jCW@onOm3+Gh4#}CO2@pa#R z1v|94fK@Sn_2p5Ds_KsIcCLx=+I6dAo^eaLz*>Mk2*&tO%=)fxbO91(-GT>6>^|T| z2j@9ghyFFzr-0-jHhXY2m>Df$?+R2L(5v8X0n+#(SAiwkVuD|;6MS&Ts7>NG>m(n& zYSE`M z^HG&X2MDvej-UILOwLcEyq_B++`kB8 zM?r-@H$Ipp0xH0_uQ%{Wqq^T)wEhb@l)p06ulw>HgWga5s`=nAuKmkuz-qv1z-qv1 zz-r)Ou7NP>B7w=z6Vnb|p`!HvPQ8C6*$d)|^T)oVYzMbc8&&p;*eT-kpjq(A$5lnZ z=zk#pdlX9{w3kr}8v(VdI$-3CvKPEc*$Xx+E5lbv8frq}y7Ig9seHSN{J)ay0`fSu z;^&I<|G8?fqw`nz|8IWqj3+!Xlv5Pf}X8{9M#gAx*JqwBDEY%}71l*+Q z_k-1I@OWIi>RDI>n^V0z_%UZHNKP1En4?5fY&@6HvHE>UL6GvvI_m0sQcV7SwzZrs zWJh`rRH)tT6cJDBkbM>qbf`WxGe5U*V!AP@vXkQjI%=PVhb6i`OZ#8<=d`L&`vxNCZUp%FQ^>dkG2}${6xmtTm zM9`^~`UjB#9i`8ODl%H55&|gM>Zb9tpqO4<9&1WIMh10kJ_{b8!I88M%x6JCVjI)* z$gx1p&Bo-c8|aeMW;2$m@zj}}J88^s(3Ue67WYlN7r53lDpnESsI6u!IK~|M9vrlr z#8G!z=R7UwyKf(X_-1W$%Qg!m?$viPLq$u z0}eFmFu>7Yd7*S#e5k&#@!3^wkxrYB#yp7&(>L#yHl^#oNIF;YC3N%IRbDKetN0?^ zP6T{*m6w?Co9rlK6s+a5tK4cLa9Cp8!PF6WQF*C}z+uV2g9MG5z>CVuOaM+FVB@o^ zyxfG~GLbhd zGzI|B!BO~X6M4gu;9lR0{Hgp$6L7;q<2$3?>+NK(F)Xl`eqtkRGx>fP&zol~`x2g9k>pnI~{2Vd)@Xv#*)A?VfbHLScJp64j ziT1U0JbaG0dU&5Y^A?g2mvm^79t#mx}ZHnp-y;8MjsSn`)XyreU?Et$EQ0D*D$`+8L_wfG%B|fa) zUq}7H**gFKj@J}_>F574OxF=N<6rP~e(+E4uMYmd;`Lvt1h5AGkE>6mz`rkKiI5j-d%*~N<(X`=x3|1w`5(VXn+aIpRm=bQ z#X7B@1@{x<3$=ZqR45lS*;emPuNuaQgf*+V4tft4Ki1ufuqmdk{rty-Tefb5BY@U?feeANpt96T<3)Jg`_!jFrgDd z`a;qiI+!%1vb2;bl?z@)$L@xKV70Y37&^(q(yx?TF3sIds`5rdBUx}fSsLzDTYHn? zkSqb1AxOaLM1soghEB5NfLM?mkBr`7_+&q5kBuUm#ocM5gi!M3!m`PH>aPFph8g4` zxI0etE5m1)K{~=y)Dz2+0UZPkGj|(a$wCJVGxr!$$%5(&dxn|!8dk|dV~$&8n0cRJ zl`IKe4KwdI)RHB_?L^Qp^8u61!IH41VdjG-FM}nfyJ6--COd;Aqr))sVUwBB2iSWQ z{?sI9lrkgD<#OAsTf09Bm`(!K|0-WKSs5dXxm8b8Gdqp2&=`E3MxiQf!)`7HOM-iS z&nQ$)n2knQXsF$16sqFa$4$auf!)prj6&6v)VUZe30~**o`7kob1_(G47q?&s5+4N zcf^q6uO_}4{|2#ZY<_zR?=L5=n|QtYbRY59*bh|uZq;5xZTg9dZ!gk0$?s5`+Nn7I z8&!J+G414DV#Ns+^Pl=2G41r73Uxa%;IxXU4?jR0JN##KJbXR%Wbm6LVNliN8~Cpz zdGOy-JG4u+XVp2o>70Q-AUOlCr=F+(I~|Yyf{vs9KiZh8*cZK)&WSvV&WU_PIQXil5Q>*#8d_ zI`{v3`t8?d{QO^BBPj8LY6J#f(cqunUmg5^rRrbIiv;M&|5>{z;irMt1pX-7PbcqsuoP&2tl+W^Xi!e|DSSbH*;)!b`H2NAH~KF*gI&_)0j5P{XRr)y{7(MItu zV}VEvYHL7wxxAFgWqWrW_w9PC5T1@Pl-Q^Z0xU4@+Xb6|{)8#}&w^t@@Ok$V8%)i= zR$(zSvebLkPgA+7m}YJ}b-tj9>j%H!vDDNBrt}{NA7E=b6eD%e)c@;Sg5i6IO!YsG1Z;swfgt(|^zr@FUL8a* znniABZrONByL)(P3QNa+2O@p8&%i+rBck8cf%(R>e_sLK~+i8$N^oN zj|HWvbykfKMLQ6On2H-XVe0a+u#}*suzHL~_UW2@Tr@>&)Ms_h zOk}^=(~WDU?m2(*~rRqVYhIW0wzmkvV&Mis>u{6gwv zEEb$r(t!JST&(;rsdKVe5=;j?CjNlb`B*GC7E7x)9Q@zjg3Z3M8n7C$8n7DZ)Bpm8 z!Q^9KQ0+^}%77{W;LI@<7ZCk6arNl!#Hgcf;^c6Wf>%d=qB#7!$eIwjn$C~kOy|T; zC?DPMvf1|RAw=(&50m&Ap zzOj&~QZPU7E@p=`X=CAFDmYu8J5?Eba=1kAS^V-=F0X2;hnlPI1B3AoN%`$ zA1oXOCQLH!o$is&pQ{I{U0lu;TI-}76LElp4NzLx-Y1=p@Ud8y64o5tALYaG z^=i21Wc;CY{(>)HZ71W8r1O`2IT+~F5zonZzjXd{-)PUX@?+_IT%lV_`Q>t}Xf`wJ z?%kkk`>DlGr1J?rnpD`qEPg7TPx1xm%X`G)XVUo;A5E;ab0hPgOXt%{LOWkBEVqg# z2BiD#V+aKuOTYAfN9-G`0jmM40jmM40jmM4fqx|$2;Zc<_mh-+kFDZ~?SF&p3!&SH zlZXC|;{4myzJ1>z<{$YaeH#8X9gmEvxPWI+)I<0$=~Fo2$wKj1`ZW4DdLR7@)&7Po z0?}vD`SLZ&F0epXl!q1bzhBu1-cMKW^#5PE@5T4|`Mr-%m&zU|S*1mrh8h%`l49^+~CJ}%ofmkhcHs)Rbp!WT<&_PyvKxYH6phOdh zVXR~{BOzu3U+2Vfkc5xjrpzf5PLrsmB)DG?LW)E;fPLcxMOoT{iooxxVy%b8qp zxKO5Wp-g5mvslPxbFiGD*A=*+r?E-gq_#h8(3UF}el6=+LL0GILSPfBok6h>vR@^& z{fY&|^Kzn&LV724b(3&Po3L1D7-gnvXU$XvDLUpPyiFUlxKgGk#=A(_DxHcub2Qt< z%;G5hOdzYc)V<o<3jTsq-u2w_R#3EjDwbCjOfPu3X*q*Ulca8n7C$8n7C$8n7C$ z8tAHl*cr+ih;5`c`ZdMRU#H>)=7{H`%76oI|1HJ#-$eX8@_2RRJQW@A9o61ON5WUA zcAv5u+)JE4_6CXwh<%&d&|64yNW}$&u;&EW{(W?&d>0*;SCJffA-xwrRm}fGn*Y-g zFaQ60uWY|3!2j=2_5Uu%Il&i21$29L@c)Fz`s3caHTeI4TG-{vBgAChz2eS;)nz6ItfdKW?fF|rLc+gfbq`5i^+Qn9|K{NHf z@IG6Csd&f2Zx>rtTzr=h|CHwIEIFno0OWYody5V{VF`c^R|51?8#QZZLA%;JHfi48 zzXW6N*sMu=ezDX>H1-N%T{pRXlVbL=snw4*Vc9G+ZXak}^X@Sw zB%1}*7xv;V9xL_c7%ViN_Eo9XkCS?b43>nhsnwTAy-fy7gxiT=YPBQvMj0##Yo=CL zq~0%sCB_|0oi*>OQtz3;l7VRh8ucQ8#-!dhqYtnb05mT3mKnLiVyoO5=^X&n{ZaTo zQg5BnuMFR#@J~|jp23pQ`6$3vHtqi0bM7MWDEwOL4Kw<=u=gnZMmm@AFWl`>_^osr zdG~*Cg>=sFWdt9Es+7*ze!=EZsD`9- z&IK_#u_F{^=oG!Xr*V%nc1ULAd*YNu2?Kzumz1=YTc zm~doDdDmmgyZ%|lv~O0estXt%P@Z{J=P&UY#kxO6TsmG8?g$vc$CzD38u+tIG(?byE% z%s;8>8m1h=rf#44$2l#(R z9K@B6uhMyX)e-kV2meoctUv0#TZ8|*_5VuC1?&yiHd94+q6U`dW}@pfc1#^=o?(GO zZC$XJ#tw0OxZkIkNXnFpY_*Gp#nKAB?6xy-AlTaZuEe)=L>ww{plilDJO3bEy9rAS zxzgQ`Z4-0t+$#t9-GHJ3A>YpTWNy?NsAj*=s8#3d6XP)cB+u8j6qXF+vAdhnox;8X6DU8SwznrS`?P6=B?fBn z?`?OUwyUtPu&cOOW8W~v09a@&R9v*@6i@Eg#uXO6dkTH>fGHlx{2pmbg@u(a9Mm=z zRu{0+G`m734{7TP9P;W#PWkL`w%9HWFD{iRyM>(cnWZcOiUhtq+6|#Tfs_z<=EqLe zCMzl~AaqDuT3BMBLaJ)!ic;Guz}C83Jp(K}zI^|+_vvT|78dh|7f}$}tL-W*BwZD; z2ztnJ**8VEHH+L$%F~* z*Y*=XFhAVN<#NO2QXX6Jnc*>t;Q`&AsOuCh?4EmI)i=5(61pX%(d{v9L1A4Lqv>RA zs@70r(jAS*<)OZ3RI}iA{+m4Pg4^vZs;wJiQ}^{Y6dlZKe}_E6N2BYj(%SEoNBIKG zsl967dw07Riz=ApB!u z@8P=%6@D_E8P1cO(ETJKbQgUZdV;c0TufJ&KcREvJ#@ajlFpGA6Wfp9K%a_VP*21M z>3qch(-F`AKe#pd=m7seAok(&!}u3`$>5*fUmg5^#p}Of3FyiHSsSEd?VvVT*Nd+B zPR;-O0&5Rle3xebEU=&b$G7YF|3a&|n9b!)3A^sgtu70u>MorEz!K1H$y42|((PzP7$`97qr=IWAK@=eVCUz$3=}sV9S1>ugx=k~E79P)QXLfsaho<^00ni-;tW*!)uDO0cVDDKo zX%8cX*2riMX$QSF(xNNF%x7p})r3&6zyT{|bx2cw7IaN3W!1z{u*9skQdT!=BLEAJ zX)a);tZvYj02W#+&p3mtCYXY?Jq*UYx8LVsZ3$ojS5J>ebpgRbbEMqN6-=^1_a7H{ z?R=RI)z#*D@Auihu^O-%uo|!$uo|!$`2V1R_$lJm*m#+20bf(LfL9S4kIfMqj|~zV zM-~9sc=XluY4n_G_fQ-8j$-~Q+CMT!`2vwWBq#hmV*BCSC|@9aj%)-drG~47?xjyd zcdGU(b>uj;@<)pQ->urK>3l>GpcX*<1tbUc|6Tr1%)f*GZ@lXEHDEPhHDEPhHDEPhHDEPhHDEPhHDERH&}xAA|NjGQl$VA8 literal 0 HcmV?d00001 diff --git a/flume-ng-sinks/flume-hive-sink/metastore_db/seg0/ca10.dat b/flume-ng-sinks/flume-hive-sink/metastore_db/seg0/ca10.dat new file mode 100644 index 0000000000000000000000000000000000000000..3683d197c21e2bc2ff8f1da52e21e91ba223341b GIT binary patch literal 8192 zcmeH~y$%6E6ot<+))Iw;PNfh^51>|WL8fGn)Fa_X+mEjy^j|f}@4GKc literal 0 HcmV?d00001 diff --git a/flume-ng-sinks/flume-hive-sink/metastore_db/seg0/ca21.dat b/flume-ng-sinks/flume-hive-sink/metastore_db/seg0/ca21.dat new file mode 100644 index 0000000000000000000000000000000000000000..ed3c2cfe8f921402f33f2b136118c683c7e68788 GIT binary patch literal 8192 zcmeI0F%khW6o%gmc0+|5sFZXX;|z+e96^m4okqDQ&}+3yCs3Mk1DzxAl5AFHg#++s zCU2AfPx3Zjmq#liV(F$S8xtGD;QdgO@uO%J`Qvmd&O*8-YWqbj$z}evHDc~{a=tq~ zILiwCn|yc}B#G>|IrS#uKmq{}009sH0T2KI5C8!X009sHf!`vK@77F9xs!k2*QqA6 z=G!g3Up10cN(NifWg#FsDtGyJKGT3NPHpuH*OVSiEg+*=;h1N=Nvuv1QdL$7SbmaI x(l13B#UCz<(LewMKmY_l00ck)1V8`;KmY_l;77Vqi`5yohN$;FJ}M% literal 0 HcmV?d00001 diff --git a/flume-ng-sinks/flume-hive-sink/metastore_db/seg0/ca31.dat b/flume-ng-sinks/flume-hive-sink/metastore_db/seg0/ca31.dat new file mode 100644 index 0000000000000000000000000000000000000000..83b43f8269f35ca880bfd72fe0306b8f1e082efb GIT binary patch literal 8192 zcmeI0KMnyw6vn?<_74>88QQGCJAscoqsL=1zen~G{1IkQPga#?>hJ4wCN zPPZ@~yDJD3ukumlSdxVEK~8U;pfE%NBtQZrKmsH{0wh2JBtQZrKmxx-V7ZzpOyo_b zzE2}1tIXRgjs7Y)uCxpm@Scr;#M$$dsq+;|>KsP1l}OdC2i*&3pcyH;tk;RHqd3=s z3%k`F_(SiwzfVFIG!+f&c>6&nDC9Nv~t(NXa;`2EkCvv~3HvTnvjHS^ctxSLt000k&O0SZun0u-PC1t>rP3jC44A-lh0 zEide^?@jduAi65{`=S44IdW&gI0ofZO>!3lbkWD~y(8(q7x@OP!D)MyNd}UzN8`nG z2?tiM(`rTwHq82b9t>bc0SZun0u-PC1t>rP3Q&Lo6rjK_feC7INfjl1VZPG1JpuID BAqD^d literal 0 HcmV?d00001 diff --git a/flume-ng-sinks/flume-hive-sink/metastore_db/seg0/ca51.dat b/flume-ng-sinks/flume-hive-sink/metastore_db/seg0/ca51.dat new file mode 100644 index 0000000000000000000000000000000000000000..453dcfee86aa85148a1ef98c66c580ad96e16375 GIT binary patch literal 8192 zcmeI0u?@m75Jm5ZltifzOHjf96pR2;Dn_6Nq%?>IIyyRf=3t7f09(MtiK8eO20-s5 zJLms%w){Gu>_kLuR55UK?B?#|{jO&DtBIccb-6HSVS;x6R? literal 0 HcmV?d00001 diff --git a/flume-ng-sinks/flume-hive-sink/metastore_db/seg0/ca61.dat b/flume-ng-sinks/flume-hive-sink/metastore_db/seg0/ca61.dat new file mode 100644 index 0000000000000000000000000000000000000000..aca129793f3813762772775024f493fb8248bf6e GIT binary patch literal 65536 zcmeIuuTH~290%}UH`W#u4nqQh2LM4Z2oN|D$Px^035H}K$UvZG@Hn0T%bQ?%8j^AJ z&$5id3-Gz7*Sp`hch`R6AtJJLf45%M=T*PhwfSN#jb}|;D*rk?Pst?RWyG|7DNBc^ z`Jb)dm2T`mzSQ+;8`qzi4WCmX+l^RngN|o)W;yLhln4+YK!5-N0t5&UAV7cs0RjXF z5FkK+009C72oNAZfB*pk1PBlyK!5-N0t5&UAV7cs0RjXF5FkK+009C72oNAZfB*pk z1PBlyK!5-N0t5&UAV7cs0RjXF5FkK+009C72oNAZfB*pk1PBlyK!5-N0t5&UAV7cs z0RjXF5FkK+009C72oNAZ;J*oMZEwEDefsOH`SX7C(y095(B{iQUZe$2QO=syJ({m- zH_qZ9`>3mHqbAMMbC5pdVt!pVZyZ%gME;ouMIJ>AMprl2w~@!n@a*&?E$bhT$3N@+ z$clOoJplp)2oNAZfB*pk1PBlyK!5-N0t5&UAV7cs0RjXF5FkK+009C72oNAZfB*pk z1PBlyK!5-N0t5&UAV7cs0RjXF5FkK+009C72oNAZfB*pk1PBlyK!5-N0t5&UAV7cs z0RjXF5FkK+009C72oNAZfB*pk1PBlyK!5-N0t5&UAV7cs0RjXF5Lgs=i@h{GjJvwH IGg*EA0!!96JOBUy literal 0 HcmV?d00001 diff --git a/flume-ng-sinks/flume-hive-sink/metastore_db/seg0/ca70.dat b/flume-ng-sinks/flume-hive-sink/metastore_db/seg0/ca70.dat new file mode 100644 index 0000000000000000000000000000000000000000..bdfb446aa13779a444ba5a5f39388ae92439a3e3 GIT binary patch literal 65536 zcmeIuJx)SF5CGs=LQn~%rLC0>v8JRZgy1=}5NnR$B|L#g@eZB=E135e2|l$o=G)Ew zyqVq1+b^CXqU5OBpq@wdd^~me<5=$ZZ955n`?<`_W0T!h%Vmy97C)~(erUbrZ{K$x z&7Ey`wK-Q-&lxDwF517=HGeDIe^rzK0RjXF5FkK+009C72oNAZfB*pk1PBlyK!5-N z0t5&UAV7cs0RjXF5FkK+009C72oNAZfB*pk1PBlyK!5-N0t5&UAV7cs0RjXF5FkK+ z009C72oNAZfB*pk1PBlyK!5-N0t5&UAV7cs0RjXF5FkK+009C72oNAZfB*pk1PBly zK!5-N0tEh(N04W_5NHk2q228;ehz)=#;B3c{69N(~ z`j&FB@0?h9T~5{_V)U{jaWQo<4xt|Bbp9+yBmHqX(aKVk`p5X(fDb znZNgDn4K@ZrId2CB3fnNE&KDXuAk>(emuoR`sQ*Tm1R|37sD{xATI9}i(jJg`sr?e zsLrhGqt(8=*hC^v>TL5^-uc(X%|~Sk5FkK+009C72oNAZfB*pk1PBlyK!5-N0t5&U zAV7cs0RjXF5FkK+009C72oNAZfB*pk1PBlyK!5-N0t5&UAV7cs0RjXF5FkK+009C7 z2oNAZfB*pk1PBlyK!5-N0t5&UAV7cs0RjXF5FkK+009C72oNAZfB*pk1PBlyK!5-N z0t5&UAVA=s34Bau4vS&N9#D<6@KiH%|K^)RRNXEAE&kIRWxmU^Wk9lyzFb6tNOjqS7Ht?Ql& z-enWLtKLjMXyW!&qV1V(D>)VjAbSF|yJx^FP>;oweS+ z)z|BVq@_m&J4wrffW}$*l>g5+GB9VCZJ&y2toOYARAw?SY6qJWi7zfeL;wK<5I_I{ Y1Q0*~0R#|00D+$cMzU~wv-|k*0{%uX!~g&Q literal 0 HcmV?d00001 diff --git a/flume-ng-sinks/flume-hive-sink/metastore_db/seg0/cab1.dat b/flume-ng-sinks/flume-hive-sink/metastore_db/seg0/cab1.dat new file mode 100644 index 0000000000000000000000000000000000000000..278ace9134b769969d8640d58067ae992ee53b13 GIT binary patch literal 8192 zcmeI$p$@_@5P;z;OST2YBjB)LP)HsC0fuKlAb{IRP5ODc}M~&H$%?*;liaLV_j# zCY|oiY&+@ay0Q`xqn90-i@A$wi1jq5^G6u9^vC5`D+|3!k#^ih)K;l1OZ zO5WuX-J9M_KWXCky+qfu+`QyiAbXKq%! zeQT`O3wfbO23sksgMh|a^;G=NH_|g#pKYB()gC?A_7jpCAxRLnL*UV@2f=DGv8ElED%5d0R#|0009ILKmY**5co%d-F|Iksb3a;uoE}S z-o7=|>xDGaBZIBvtb8i`=NsvptIxL1Mb#cX*!B~d7L~c)ZR6zR6GQ|MKmY** Z5I_I{1Q0*~0R#~EL0}{^zx#IS9bTk1F8Tlf literal 0 HcmV?d00001 diff --git a/flume-ng-sinks/flume-hive-sink/metastore_db/seg0/cae1.dat b/flume-ng-sinks/flume-hive-sink/metastore_db/seg0/cae1.dat new file mode 100644 index 0000000000000000000000000000000000000000..8fb96b5b55c3fbd0dd36c7bbd2927ae81fe62cc1 GIT binary patch literal 8192 zcmeI$Ar8VY6oBE^iETmQF*qbB6o){d;07oJkl--jH~}|6aRg3*$H6J!wOyBP3=%B) zo9wN9uiMK$x5`>Xj9zvmE~YNVA=KlX&Y#7or9UpmT3PCqi?qWgqs?{wbu_j=4If?q zRP-*F=-u^Z22m5guM%C)d|Sz}KmY**5I_I{1Q0*~0R#|0;2#AJ$BmJdep&dzPTVYe z`_@pe7t&0R40e*22LX+<@~QBjZ=`RoKHIt#ReSVc+fQU#R2F)-jgyN{5D`EC0R#|0 Z009ILKmY**5J2DufsxGp?#})^djXjZE;RrE literal 0 HcmV?d00001 diff --git a/flume-ng-sinks/flume-hive-sink/metastore_db/seg0/caf1.dat b/flume-ng-sinks/flume-hive-sink/metastore_db/seg0/caf1.dat new file mode 100644 index 0000000000000000000000000000000000000000..86593054652131fe04a4307f30881fc6e91d9a62 GIT binary patch literal 8192 zcmeI$Ar8VY6oBE^iETk~1RN3+iUS}KD2{*z5*!9eLW1N5oPskTxB&@BP5`g%x^!cZ zV9DQPZ|!^CUiP_FmLg*GvLkUZbukX19_MuaEJiK;aXHq?T(4ZD9X1(luIsO(vHfvy z@A#*pcezCOrZ>}%n)rQ{=z8YsN{$5r2q1s}0tg_000IagfB*vjD6l=Oj4bra!Vh-h zX4%`fhI+k_W_o0>k-R(zXq=T#h5vjbJ#+Ed)}yG}qX*l5BGaO>)4Oe)oP2_a00Iag bfB*srAb&fTb@> literal 0 HcmV?d00001 diff --git a/flume-ng-sinks/flume-hive-sink/metastore_db/seg0/cb00.dat b/flume-ng-sinks/flume-hive-sink/metastore_db/seg0/cb00.dat new file mode 100644 index 0000000000000000000000000000000000000000..0d62a64c8581ba65f54de36db98de95a2bd99d17 GIT binary patch literal 8192 zcmeI#u?>Py5P;!>7}RKCqAM5+8yh=^AQ37Hmr!?b4l8?SZ~-efa03eqy>|}+L0{}l zOnwN*arfZh>yVv@M0&U_wwPF~hq7OMmczyYG>oS19uwf(bwS}Ukk>M<5$qU*=er7HJa+LcUFSxT;ZW-+f;kRgBo0tg_000IagfB*srAb`M61Qs&& Jb54i#`~{=<7U%!~ literal 0 HcmV?d00001 diff --git a/flume-ng-sinks/flume-hive-sink/metastore_db/seg0/cb1.dat b/flume-ng-sinks/flume-hive-sink/metastore_db/seg0/cb1.dat new file mode 100644 index 0000000000000000000000000000000000000000..afc543aebf85d6225a074117ea81b231a878ccaf GIT binary patch literal 16384 zcmeI%d7Rec8prYLo~il$W^cBpX_}^Knx+~F9YWFl+#@7qsbtTVWJ@I>oorE(BwMl^ z2Zs)hljS%#AxX9nLPAKAByG}uuIKx`kfZq1>%3miyk32J_q0sz>-purN-hu)r(cpE z4bTj9`~3L*+rOsXW*`6a@=m{bxPH(1`Xv@=^y3e5`pNa%zx9zmct_tmbjZ{X`TBOh z7G9|RO210qZM#phgP)Jj`8%I@2R|+KQtAX!Cy+XU)Cr_cAaw$%6G)vv>I70JkUD|X z38YTo|Lz3deZ4GChU;(J_ItNl()3%~e)S)+ll{LzhOX{ua+SnC{Y}6QWxw9fi@uW% z?mc^Tx_zDe%xU^N=_SY7E7$AS$I70JkUD|X38YRSbpojqNS#3H1X3rEIsrR@=~Arff}LMcv$oHYU+n*_WTUR!zuxw1 zcXEOmk*@4xwqO742d4kDoBzD@|GdNNRv)D$cX$^`OvI8&WJ(s1CD}x_luRnn4ZDb+-^v?5wb4N)VliPlm})JhwojkG1&N;{&R)B$N3vI`MO zdm@%yiA?E0WXWzsw(L&i$R0$lbR_cRPei`#Ni>qZhyv+E6w2O2k?cbh%f3X3bS4^0 z7ot@5BbvzmM45CYnu_;Sx%dRtOnjg!#Me{J#UDUbitk3X5Pu+5CH^3)rTBxXYVqBv zR^kt#YQ*=TT8lrFsukapY9qcE)mHprR6Fs9L+KgfkDwy)M^dr)-c+XeK2(C#A4@e7e;idH{&=cT{0UT%_!FsO@%^b1@h4G@#h*--ia&*F zBK}mWO#Eq-p7;PtPyBRBPkbPyCw>N{Cq9VM6F-yE6F-a66F-~M6F-O26F-;I6CX_J ziJwR5i4URl#D`LP;=?FC@$)G?@!?dR_zNKY>qS3;(zCyi(zCya(zCyq(z72)>DgaG z>DiB>^z8pk>DiB_^z1LC^z1LA^z1LE^z5&o^z5&s^z6q_diGaQdiGaSdiG-}J^OK# zp8YkHp8a@A&;D9U&wc`>XMY`~XFrkBv%j9wv!6ui+226v*-xhQ>~Ex6Ie!yXD1{voQd^AA&{&i{jI z;`}32ne+donmRw7DtG=-s+seTQ5DWVPBnM_398che^D)*f0C+l{@+wf=bxggoqw9r z_x&@JzVH7->HB^LrSJP^DSh8RN9p_ic}m~+Gbw%Fzd-FGeiqeU{EJYAJ`1xceHLD# z^jVlg>9g=MrO(1Els*fuQu-{sM(MNgI;GFTTuPsXHz<7;-lX(dc#G0!VIHN=!rPQS z3-3_+EWAtUvoN31XW>0cpN01+eHK2T^jTOy>9g=5rO(1bN}q*~D18<_ru13(gwkhW z5v9+zr>D4oK$lultKrBnEh(kZN>bPB5}ox&PQr?8gNDXgP(3g1&Yh4qw9;Ri~muz}Jk zY@~Dwn<$;aW=f~9h0-Z(rF070sHQu)(Eak|%oUO)lDQaNfWC>&MxQ~aqxYkCpp(&Y z=xB5}dKP*z+7IoC)}#BN9nc!I94$bjWaBO98gv=D2%V3L5_BOt51oU~Kp#V=p?9G-q2tlZ&=Kf4 z=&9&2=wWC#v@^OpT8mboMQCQS8Qaix=yLQk^nG+LItzUYeFVJ^or+FEuSQ3q!_Yx! zf3z>!15Kcv(DrCGT88GM4U=uS30;LQML$O0L0>_iL!Uq&MDIp#K_{SBpckQo(E;dj z=n?2a=zeHNv@KeRmY~_FOSZv!bOpK?U4Xuc&PJa>r=$0ycc7Eeap-7tIC>U(GTIOA ziPoe0pdHW}v>Yu!qh!;!pli@&=puAJ`WiYDeG+{by$79wPDIC`Bhew~K=cH(H`*QT zitdTlp;c%pnv13=nV8RbQ*dWdJ{Sxy$l_Jo-3L9D-S@AM|-1( zpb2zuv;*22tw4*>Y&0!dw*mbQU5b8!&PQKIXQ5A{)6oaeyU?4_3FwvRNOUMV2t5hy zhxS6dp4A63}?A2qlA zKdQ9-KWbt7e^h1rf7H_U|ESvb|EQJi|51(Y|50n(|D#&l|D!gx|3__Y|Bu?){vXxZ z{vYjP`+uBa`+pqS{vXG-|HqlO|HoOj|Hs+3|HnDD|Hrwu|HpZ@|Ht{Z|HqAN|Bnl7 z|Bnl8|Bs7o|Bs7p|Bp*-|BoBn{vVgx{vS87{XZ_V{XcGM`+r<+`+wZb_W!uT_W!uK z?f-G5?f-EL+yCP#+yCR1w*SY~w*SYiZ2ymIZ2ym2+x{Qd+WsH6vHd@8Yx{rP&i4Pf z&i4OUZ&V(G&(s@Lg3=pRp!7ynPw9>707`FE-6*|L9Z2bo>L5yQR0mUfqv}rSjp`6e zZ&W=fy-^)X>5Zxw7Yx`1quRV&= zdu=~T@3lu$dapf((tGW(l-_HPqx4>TJf-*A6DYmco=EAvwm+r!+LI`~*Pcx2z4jDJ z@3p5=dapf=(tGUyO7FF&Q#yr#P?k>N3`(akh|(#XN$C{MqI3#pQ#yrnD4oK&lultV zrBgVM(kTp~bP7W$ox(6mr*J-{Qy5O^6fU523L_|;!iAJh;UY?>a51G*7)j|AE}?V^ zqbQxipDCTfXiBGWDWy}mjM6DwPU#e`pmYjXQaXh(luqF)N~dr&rBfJ7=@iCMI)!T} zox*rZr*JK$Q) zFm7<+P2)xv-ZE}-VV-fb3vU~@xbTi~s|)WMx4AH%4C2CjM(4u&#xxf`Fs8e(z}Ucr z4~-37SZK^};Ui<@!pFwgg-?u`E-W%;x$vnm+l9}JIWBx|%$2a%m?z;2W4?qX#zqpp zG!{r$YAlrSm9a>|GGnoXuZ<-VmKz&O_{LZ&VTG}Ygl~;y5>^_UO8CxLE@73inS|BG z3JGhB%_XcgR!UfBY$4%$W0i#U#+DL(Fjh<0U~DB}qp?Q9CSz*}n~k*+wiw$;*lKJm zVVkj?)QhoB>YZ^HsZTSum-=+$u2SE?*g@(W8h4ZW4CC%n9~t+M`qGi-HuF`<);@>dIWkP z+8NyqtwEcj`DlY=;|=Jy=wkE(bT0Y=`Xu@gdN+DAIv%|g9gd!f_DB1m-O>Hgj%XXS z0xd+NWTVaKD)dWqA^H|N8+{sm1icrXf?kJSfnJE7gPwx+Lk~q0=w4_Y+5#;>^)c)Tfwg0E~|37g5&q{kd{l6QX7kdI?(KTBzZKe|)RtaRlf-B4t-x%ywFwf%Yg==)RA zT`n-V>t;q#6JJ-;u4lfh#c&`10SG_<0uX=z1Rwwb2tWV=|486?+8SBw%fbgcaaQ*B z){t)|(o9DNd&#RoK&w@q3jcf~Lv!=dPOr?`(Sw}@%w$n!>}DG$g|!d}KmY;|fB*y_ Y009U<00Izz00e#_Fp;INzZ|af7gc>NtpET3 literal 0 HcmV?d00001 diff --git a/flume-ng-sinks/flume-hive-sink/metastore_db/seg0/cb21.dat b/flume-ng-sinks/flume-hive-sink/metastore_db/seg0/cb21.dat new file mode 100644 index 0000000000000000000000000000000000000000..dc181754e444bffa7c9a5b7dbd105dfcdaa4ce9b GIT binary patch literal 65536 zcmeIuyG_JE5CG6w2euIj1t0+(kRnk8;*(IvLF|MaiXf#4dLSjI2##HQeL5Y6h>SNU zkH<569{I(6M5KI`XvW-*c{|PX?Ov|eHFnxRK3?R>DqnrXFds^Jd4B#i7T3S-eU$g* z7{;8_bS_;!mijc}U>+>o$DRGKFwr7FfB*pk1PBlyK!5-N0t5&UAV7cs0RjXF5FkK+ z009C72oNAZfB*pk1PBlyK!5-N0t5&UAV7cs0RjXF5FkK+009C72oNAZfB*pk1PBly zK!5-N0t5&UAV7cs0RjXF5FkK+009C72oNAZfB*pk1PBlyK!5-N0t5&UAV7cs0RjXF z5FkK+0D=D|aCCh59C!JvQ~mO0th#Rw^L%&EcG1K`^y?bS)#hJ)tx}w&x2fJgslBqa z%j>V>X7jtPXUeTD^UtXv6Cgl<009C72oNAZfB*pk1PBlyK!5-N0t5&UAV7cs0RjXF z5FkK+009C72oNAZfB*pk1PBlyK!5-N0t5&UAV7cs0RjXF5FkK+009C72oNAZfB*pk z1PBlyK!5-N0t5&UAV7cs0RjXF5FkK+009C72oNAZfB*pk1PBlyK!5-N0t5&UAV7cs Qfj+tCy$uFExxX3IG5A literal 0 HcmV?d00001 diff --git a/flume-ng-sinks/flume-hive-sink/metastore_db/seg0/cb31.dat b/flume-ng-sinks/flume-hive-sink/metastore_db/seg0/cb31.dat new file mode 100644 index 0000000000000000000000000000000000000000..130efcf474f892b00aae8676aa1ecd7ad2a50390 GIT binary patch literal 8192 zcmeI$Ar8VY6oBE^iERl6kHH}UgMmXJP#ggbBsdUAPQVQ~#ZJKy2%G|5+jZ&2Ai~ zSwp^=NHe`MI7(h!1hmeox59tEk)gTzXlGnz?bU-VzmR2_u@74~DeR1f00IagfB*sr XAb literal 0 HcmV?d00001 diff --git a/flume-ng-sinks/flume-hive-sink/metastore_db/seg0/cb40.dat b/flume-ng-sinks/flume-hive-sink/metastore_db/seg0/cb40.dat new file mode 100644 index 0000000000000000000000000000000000000000..7ba3154f2e826ceb1961e41512cf26d67ce4274f GIT binary patch literal 8192 zcmeI#u?fOZ5P;!JkSJ;)=nkm_2e5F2fTnT{bq7bWwXkr2)Pj{8xPgU*-n*Azh(7ED z;eU{Om%AjFd|hNCB9R_$#TFBb^-%TeHO(L8YLtVgnRZsbJ8;`bkG>A?_*yW29KXEW zxBd=$mP^a)(yPlSz8Q)(^V{Ii&D0S<009ILKmY**5I_I{1Q7TmfnvD3$y#5y`oWGZ zYrlO$)o(OfiC&*Z=U2Hm)4VGS$t9Dy#@uqZvk*W40R#|0009ILKmY**5J2FU0#h0L Jar@KZ_yq5d7>NJ? literal 0 HcmV?d00001 diff --git a/flume-ng-sinks/flume-hive-sink/metastore_db/seg0/cb51.dat b/flume-ng-sinks/flume-hive-sink/metastore_db/seg0/cb51.dat new file mode 100644 index 0000000000000000000000000000000000000000..f8da77f809411c0802f2c12b0d840cdbcd67486b GIT binary patch literal 8192 zcmeI$yA8rH5P;!xiX;*ZBhb+S6%_>|fJDUz)PO_>1tk-(0aGvq3(&v@ba3$_aza3& zMZcw7>^mn`zAh(g5ixodBe5~HF%F>~=XCz`qn7@-oM>gKS1!^Hn~WZ=>#w8n_|x#- zaZg3>vWf0ZZ)Ol3;`UXdJu}}{ax4%)009ILKmY**5I_I{1Q7T~fx~fQWTjsgey|fe z%ig{<)a!*b)0V+r^0E=oI4hqD|M^Dx=Hjxgsjn8ThoToSlew>qZAP0CXO|!%fB*sr bAbBb~pKEMa1Z3N8)1YVjMy}&guMFj9U8Ra-x-`Ub#p+JY=-FuD_1P_NU>a z>vu)(a*5tuZ)OlR@y}JF>zQvWITi>YfB*srAbh(gJ>5;)s^70^{aaQgM|M^Dx=IXPpc~P}T54QbOW<_PJciT8QYhWON00Iag afB*srAb}ab;JF0$UjmQh5L}{c+bwB61%u>G z)9G|(+ihQLA|jQuk7mfLA12^^msy~Jbr>s0@|8LIU+hiSe$SgoUpr^sE63yvC?`yB0 z<+A-v+|G`>y6U1_`_IbV*pLYjAV7cs0RjXF5FkK+009C72oNAZfB*pk1PBlyK!5-N z0t5&UAV7cs0RjXF5FkK+009C72oNAZfB*pk1PBlyK!5-N0t5&UAV7cs0RjXF5FkK+ z009C72oNAZfB*pk1PBlyK!5-N0t5&UAV7cs0RjXF5FkK+009C72oNAZfB*pk1PBly SK!CuX1snm>xIRt{dK+F7}7&zuH&bai;k)q?Tk_~rSw zcX!xlnY36px|n?8nxSZCej6ORnK}XpAb z&9_hR{aT}u==x!FD$AX%=55I&m6^<6dseO>LjVB;5I_I{1Q0*~0R#|00D+$fOl9om I9CnNQ6MVE6;{X5v literal 0 HcmV?d00001 diff --git a/flume-ng-sinks/flume-hive-sink/metastore_db/seg0/cb91.dat b/flume-ng-sinks/flume-hive-sink/metastore_db/seg0/cb91.dat new file mode 100644 index 0000000000000000000000000000000000000000..74bb2b02f1579d78bb651f17c9fa851e42159483 GIT binary patch literal 8192 zcmeI$D-Oay5P;#?ina*`iNv7@C=`c4pg2M`kl<>N2sAg~6g>rJfK$M9AI-KD5-j;Q z>2!By+f6^$k-dl*z3P$JnA#YJP>*vufBMl$e_XD#vehdW8HPWsAu8Pt01;>Xixc~qF literal 0 HcmV?d00001 diff --git a/flume-ng-sinks/flume-hive-sink/metastore_db/seg0/cba1.dat b/flume-ng-sinks/flume-hive-sink/metastore_db/seg0/cba1.dat new file mode 100644 index 0000000000000000000000000000000000000000..d17d52c93a4ead4264b63cd554aa8efc44a83d07 GIT binary patch literal 8192 zcmeI$Ar8VY6oBE^iERl6i^Cy7p*REr#Szdzf}5G-1Sl-0;0`2&Bfu%(wOyBP3=%B) zo9wN9Z{5p2x5`#Tj9zvmE~YNVA=KlX&Y#t2r9Up`T3PFri}b@oMw{#U>*#EMnm&ep zSM)BI7~S<|CQ%drTqC-kdC|zRKmY**5I_I{1Q0*~0R#|0;2#C{hnUcYCGY)YxnP_;^(&t8(Rtem<1S^78y^EUsT4d<^d^ z(T};L>0G*g9_q7*qj|7!A9v<{VWLHV009C72oNAZfB*pk1PBlyK!5-N0t5&UAV7cs z0RjXF5FkK+009C72oNAZfB*pk1PBlyK!5-N0t5&UAV7cs0RjXF5FkK+009C72oNAZ zfB*pk1PBlyK!5-N0t5&UAV7cs0RjXF5FkK+009C72oNAZfB*pk1PBlyK!5-N0t5&U zAV7cs0RsO`;NTY`&1vk)!wkQ zE9-CLZu`5hXZmt)hxzB!kO>eVK!5-N0t5&UAV7cs0RjXF5FkK+009C72oNAZfB*pk z1PBlyK!5-N0t5&UAV7cs0RjXF5FkK+009C72oNAZfB*pk1PBlyK!5-N0t5&UAV7cs z0RjXF5FkK+009C72oNAZfB*pk1PBlyK!5-N0t5&UAV7cs0RjXF5FkK+009C72oNAZ RfWRLJT*PX6%T2y~{Q{uRFaiJo literal 0 HcmV?d00001 diff --git a/flume-ng-sinks/flume-hive-sink/metastore_db/seg0/cbc0.dat b/flume-ng-sinks/flume-hive-sink/metastore_db/seg0/cbc0.dat new file mode 100644 index 0000000000000000000000000000000000000000..46cfc9ecf8e5a5881042f2acc3197b63bb5d1ff7 GIT binary patch literal 8192 zcmeI#F%E)25J1scV$i6CiQa+A!~=K;60!0cYHeZTQS3Z|6&JAb2HwEJLT7i`K+uhy ziOGjB3^NM@uS2#XV)U?E;$rGzGlYJ#runnlTIJ|vuAQ~-j_hfo$54m&d@UM3j$d6L z2Y-iqmQAa5+0EuN-;6~!^V{Iq&D0S<009ILKmY**5I_I{1Q7TmfpUDj%SK-~{9q?8 z>%M(L=(if3RM%(bQkMs3&HGZwQgT^n40}b Jr<`_s^8&_r7t8HrM@E>ui6SK8C(3 z`j-<%cm0@2)Wn}_c-J%EH#9a7fB*y_009U<00Izz00bZafqx`$JnxKb^k(6Kowz7_ zyKBg|6KSR+gM;MtAfPjUeGvw8f=V_YHd!_Db9JGJDy)Ki7a^{G3T9nG><^I=L-2VOOvwvTS zcE}}->(b3jUtdHVPlJW)uxD-;CRzjt5FkK+009C72oNAZfB*pk1PBlyK!5-N0t5&U zAV7cs0RjXF5FkK+009C72oNAZfB*pk1PBlyK!5-N0t5&UAV7cs0RjXF5FkK+009C7 z2oNAZfB*pk1PBlyK!5-N0t5&UAV7cs0RjXF5FkK+009C72oNAZfB*pk1PBlyK!5-N z0t5&UAn@M=&dyI?PW2y_IYKS-Bec=34dHJG*fENMTX%eqs4Xowe^-i&8m^# z6@AME#y5SKX;j3&*BV{TeAkMxKmY**5I_I{1Q0*~0R#|0;2#AJ$E}gIo-F)eCvKL# zoi*gUi8RwIgT3Ud26mzCd8l;_6_ft@d9 literal 0 HcmV?d00001 diff --git a/flume-ng-sinks/flume-hive-sink/metastore_db/seg0/cc0.dat b/flume-ng-sinks/flume-hive-sink/metastore_db/seg0/cc0.dat new file mode 100644 index 0000000000000000000000000000000000000000..226872006d7e7681f9732c7e2268f2312c42abd0 GIT binary patch literal 8192 zcmeI1y-or_5P;_d`8^c1va&H+ZL;?R90n5+qGABigF-C~aPS0bv@<4_))u~qrSS<& zd;v=n-$9){Zjr#!8#8-h*qynL-TN{P0G9xu;Y*AeKSumG8f1Mmhx6}XjHRb88(29X zRMYY%9XF53#gzA?Dt~fYD-QC*&kByY{(*Uo0}Kla-rqBJqo`1d43GgbKnBPF86X2> zfDDiUGC&5%!2dFE|9Gvz1wNDfexCvzSBn!*DeGf}a0JO^?QJCejPnrGp@kPjL$C&` zU2a$o=N{v{@32fKV=>cd=W!{9k?(nqgFgX8++(-0U4vN&6&nqR`7Cl`2-f4Q@8xpL z!uti=#;arGITr-75{yr5+F@qaHW<(Kx^h#??pr+Av{YL3pOymKmp?&Y+~H=qb@S= V>cAb&4@QE!k-y}P` zZ)P|9_KRIalpJ-8nmlguY3lo@wOoI8aaR8IbCsQ^wp-QHU5;s9KB+%0#NhQ0@5hh! zOw}K4_m%Y_3uVz)hv(+bzb+m=s!D(W0RjXF5FkK+009C72oNAZfB*pk1PBlyK!5-N z0t5&UAV7cs0RjXF5FkK+009C72oNAZfB*pk1PBlyK!5-N0t5&UAV7cs0RjXF5FkK+ z009C72oNAZfB*pk1PBlyK!5-N0t5&UAV7cs0RjXF5FkK+009C72oNAZfB*pk1PBly zK!5-N0{=|lW$}0)&-tj*J#5shZht(p@1Hep#<_o1UcbxJdO3+rY-82sZLS3Y0t5&U zAV7cs0RjXF5FkK+009C72oNAZfB*pk1PBlyK!5-N0t5&UAV7cs0RjXF5FkK+009C7 z2oNAZfB*pk1PBlyK!5-N0t5&UAV7cs0RjXF5FkK+009C72oNAZfB*pk1PBlyK!5-N u0t5&UAV7cs0RjXF5FkK+009C72oNAZfB*pk1PBngK7nP-+n?H1`?oKrn-;|Y literal 0 HcmV?d00001 diff --git a/flume-ng-sinks/flume-hive-sink/metastore_db/seg0/cc11.dat b/flume-ng-sinks/flume-hive-sink/metastore_db/seg0/cc11.dat new file mode 100644 index 0000000000000000000000000000000000000000..b9381ede540c69c57607d6fb2fb980afba87fc08 GIT binary patch literal 8192 zcmeI0yA8rH5J1lEH!!$t$%0 literal 0 HcmV?d00001 diff --git a/flume-ng-sinks/flume-hive-sink/metastore_db/seg0/cc20.dat b/flume-ng-sinks/flume-hive-sink/metastore_db/seg0/cc20.dat new file mode 100644 index 0000000000000000000000000000000000000000..1d2d401df636c956e968696f400cc301deb14443 GIT binary patch literal 65536 zcmeIu%}oMP5CGtLgn$Q*-r#^85YHYhfEX!&gc6i9CD4Qo*or++gPwGDmyHA#xf=3K z@_u&az0AHZ))7&1RMD&FK|LQ%ZT&Ep`|o`mrO$rOv+~kZH`Q{RW0cKrs*gXSv;F<< z^Q*bDX|Fct%F{LzW!7f9*ShDW#@$zC2@oJafB*pk1PBlyK!5-N0t5&UAV7cs0RjXF z5FkK+009C72oNAZfB*pk1PBlyK!5-N0t5&UAV7cs0RjXF5FkK+009C72oNAZfB*pk z1PBlyK!5-N0t5&UAV7cs0RjXF5FkK+009C72oNAZfB*pk1PBlyK!5-N0t5&UAV7cs z0RjXF5FkL{$^_nKkCS-KN0t4TOUad9#6348+|$k|b!LXVb`f@qx0 z--M&RD_rau{;WnT{c%}nWu;dtGVq&>HrMsn(b@hod1UUX z;9WK`y6er1gC=fYBif$nzL8^r00IagfB*srAb1u3;AkH=Xuc@;g8F?W;Xg{BHggbXmMSCZLQ@m<7dyG zs=Uhu`VSq5WhEWl}uQs}x`K}gYfdB#sAb#^}GxlK%CxxBS5I_I{1Q0*~ Y0R#|0009ILK;S2Vu`K=m<9Tv@0b@5U-2eap literal 0 HcmV?d00001 diff --git a/flume-ng-sinks/flume-hive-sink/metastore_db/seg0/cc61.dat b/flume-ng-sinks/flume-hive-sink/metastore_db/seg0/cc61.dat new file mode 100644 index 0000000000000000000000000000000000000000..b3cc1e607032c28ccf10b1ec2f3a3db919dec0db GIT binary patch literal 8192 zcmeI$Ar8VY6oBE^iERSG5$2Fgp*RErg~Or{K!QVnt6P`Rdym`^s*yyF?BHxp&sXS{;WqQ{c$1IOVG0jB@GoDP^P0u0cSfVQNW@@ zLi#~IE_XIZUKiPkNTkbLv5kq1&EWgZn&yw{YL$bxxpp?LJ1}1(U8xT5xmqxO9KX7j zdw0VAmPxC9lbgwBt{IAU=C{G2!_*N#009ILKmY**5I_I{1Q7Tmfz$AKm#zL`|AHOc zto!x|zTawe65T(JE_FFwYu=ZIR literal 0 HcmV?d00001 diff --git a/flume-ng-sinks/flume-hive-sink/metastore_db/seg0/cc81.dat b/flume-ng-sinks/flume-hive-sink/metastore_db/seg0/cc81.dat new file mode 100644 index 0000000000000000000000000000000000000000..faee883e230e8756fd1eee7a0cbc18754f0265c3 GIT binary patch literal 8192 zcmeI$Ar8VY6oBE^iERSG5pYOQBqV4w0>u$f2q3|MnB)Z9fGc1)1!sU$z-zlM-54ZT z@;BLA`(C$ZpWBnIh#0->NL)-^j6&a|@DD;Md8O-2vb_1DpQ{CV`~ z`KO|HxkUf2H#3Y5@%w7gotf`zITi>YfB*srAbdiu$Y0IFHylMor&Z?)vf4-4{x%q78P*!d0!S({?vM4Jnm(ixg)hCDuAbn-a5 literal 0 HcmV?d00001 diff --git a/flume-ng-sinks/flume-hive-sink/metastore_db/seg0/cc91.dat b/flume-ng-sinks/flume-hive-sink/metastore_db/seg0/cc91.dat new file mode 100644 index 0000000000000000000000000000000000000000..37e65667d806a8a54d5be4db795e91c9bf33c1bb GIT binary patch literal 65536 zcmeIuu}#E45CFiv4{Re5PyiC7WCSEg6oC*VR6+0%J0XV-NWKz83lu>SJm>TI^?D2m zA+whE*0*P`Jn;|_Dc?gZN^VNtPE&t-FW2iDJKH}#Ugpj+-(5sM9ZGq4e*QIP$G`4< z4ByMqmz>jhE?qqh^=ZVx)Rl*Xb=B=()DbCaTSnogA zy^XJOBUy literal 0 HcmV?d00001 diff --git a/flume-ng-sinks/flume-hive-sink/metastore_db/seg0/cca1.dat b/flume-ng-sinks/flume-hive-sink/metastore_db/seg0/cca1.dat new file mode 100644 index 0000000000000000000000000000000000000000..482bde6b9706dd38acadadb22db10bb52ce95463 GIT binary patch literal 8192 zcmeI$Ar8VY6oBE^iERSG5#|UA6pBM2P;dhj0!VNeBqu;&IR&TS3~&S#yteDojX{DX zf0Mnn?{$0G=T_N>h|$ZA#KqLbID~qf)A_R;jr7OmL@O)3a*=M>WVE@izmC@S=h40J zPet!?iQY|bW*9Z``)bkk%y+dM3j`2A009ILKmY**5I_I{1pZOraNHVM>z9Qe?8MEo zw{H#gW+Bb=$Y3vdbr8@xtDXw~`9=oj>a(3wS+z$Gw*6FQWo4;%+c>%S1Q7uQ5I_I{ Z1Q0*~0R#|0009Jk5E#qC?=Jd}`3v?iE$RRO literal 0 HcmV?d00001 diff --git a/flume-ng-sinks/flume-hive-sink/metastore_db/seg0/cd1.dat b/flume-ng-sinks/flume-hive-sink/metastore_db/seg0/cd1.dat new file mode 100644 index 0000000000000000000000000000000000000000..d919a1b0e320bd23ef9eb70d43b5189acb5d9ec9 GIT binary patch literal 8192 zcmeI1zfQwI4935tkfi@;3lckGKujQ5SxPHVs!#%H23Wc?0}>kx5(_g!cb=gm8#4RL|*JS^htr_`8HUa5uVga3OcH{e2zUYA=4frH#)9JYPlzHwdPKfWqCh%J%sVHk_ACea_xGdeB)w ziqlMaG>Qx*bmO}+CGe7yB-}kf3677z#XL%)s2iYc6FTjIDrHoxvX}N%si5kj3|~s( z=`Msvy5C)_a3a_Md RtIj>O58!+O>G0`x?-%c{mUjRE literal 0 HcmV?d00001 diff --git a/flume-ng-sinks/flume-hive-sink/metastore_db/seg0/cf0.dat b/flume-ng-sinks/flume-hive-sink/metastore_db/seg0/cf0.dat new file mode 100644 index 0000000000000000000000000000000000000000..780d64a60020caae0a1bcced4cd7a860c9606d86 GIT binary patch literal 24576 zcmeHOTZm*w8Lpn`p8L%1PO_WLWj7Im@xdi^J5_ZgFLUV)GSpXSN;G0{Z)mYL8`;}q$w;p}@O@II3yLvG1dk5}!FhYyN{;qe=J#*@l6+K2DjTjg)Fk)cD zz=(kn10x1T42&2UF)(6a#K4Gw{|yFy`uXqu1APR3OJCnVfNqC(cMtr!&cGdaqdftC z`L5Qlj(qwTxSm9t@ZX+mH_;aAOv0^qqkFM22IDNfgbB|tNs^08I$`1xwlAYImZjMC zD~w7x&SYi3ThZdqLmMQ)6sHuEl%+&qG>%$Z_A8Y8PK@EZ4ZV~zRbJxSWcE=R&N!hO zyUtzIw%}V+4xlJZ0S_)K9}M5F3ndlKOl7}8fRwmrUTy+#+5rU46vufGpdm@BaDGyN zj5TL)#sSQAT4=0`N@jq;Of#jh)|LGR8Dz~(+@AG=kZ>3=xP*~WJqTH5QjLqevfmV% zSY6)P-&$`t@Y-n3(U!^rV`7JE0927Njmv6KK-=ol?$YwoPIrH6X?y8P1VDu59YDo$ zm^C;!@K_yyXVER!1H8OWEQL*`Mti{~61ZKDfWJwzJ#Z&g1P<9;9YkEAT!cL*pz-!An(xgweOM8e!ZqblMr0t29%Dg+Y~s zal)!WEuzyf_V!n;fLK99zHLB}ZR)`)S3&}|N1nNXpL3ggRJiixVm2W|vn*Y4Wx+SSed_3j6n z`(A|3J9sjeT9Y&gFCj9Ae6kwe0-D>oy1g6C-0cn;rVtWH7KSEEgGnHPzKHG`61uJ} zZS2MPJcRCWP?ai_CV3cYk(Zd{)hu2>clMy#aN!{CZ(Lp3zp}IyWA8gXsbvWcE}V*r zL=#MOHPSgW0}%JFY(~wKyBx4oCR`H}q)-^lFfrA@>_A+-32EL12aOa3_`MzYx zrl^JnQ@Cf{_`!G4!OH|_3sQ#RX$~H+tcG_6oq|MV<7VW@NA&NI%Y~QFR}os6KwpDw z`cd>eLUX@F&q4bh^eRHLPowW3H1oE-KeJ-{OuvX;fOZ~z0imhK(APmw|J=14J&Vx9 z^Y(8$|FS(hrR~%H5qb&QyU{lg8vnTMIrfgdKlY%#ZhgzO9or&p`=AHVeIDi;+dq5$ znEtv>#=v(rzHF zNS!=)22Hs<_wqVzD*2EvHVBMUMOk%~4q#Q&M37V30OmC#eO1$CBDO zyPk3YOGyhwMUcT-Y3(UAf0)+Bkmm~vGA~QO79=mJlh@vjF5Eb;jlp`x8#>D&Hp$?u zP3k1JhdKCQVjBZ>(Lq&-A&SZ{t?^Ks&u?FgfFe0nR0T(8tt9+3TIeU?*JzYW?A0O4 zf&sW7kF^qUJ9Q2c@faJQ^)@a_SlUQ~!?9K-4k^L$OgsjkFGi#pOB79m+%?75eBk+F z1SzFZGz$(qSw|}Pn&zQYh@sCHBQmZfzzfn>D`P%~ZXL*&V<7sX1kI>YG!HVU>ZHt8 z3)-q{KV^=A=L->%CJIBGSUJ*mcS0X@36quGlgqbp2(V`k5 ztmYgdw8cayNJOxBD1c9JweSr3Cg}e^TLnJ<5P~i3bKkW(e{Rt#`Ps+Oi=hACu-7x# z>HlYJJM}m8WoRF^{U=|um4eBZZ7257YoPysU_b4gx7xn_QQMCH-YWm`b*KMdxAlUt z3$|@NZNClu&Q>Bo|HHHF>ofO!?#W~N|Bhz*-xvW2Ng4EJwdP_kNj@?cW1NMT|1g-0 zX1IFY6qI7Exd>^Lypjb(v>`SW)P*9D&*4^H*OCu4E z*4G7ctQBW0xUvsRbq4xX*Qjiw*?nlQySdWc?-f^@ZjI>gJSY>*ITJJ#=OI@%iUou^ z)SkAlZp0N7eO(}9jAa}NZ$g<2d$ldBBbb!7DIB&W}-QRQ|lqhv#f(EXI5UO}< ztB*wR)Ygn960#^^Blf!BrTD`lI^0@A40PYI zAr@myFYGl=GAvkdj0wZ37J+~}=sD7e-7Q|)Q=xJt7!OT?S^#^xFW&8XO|KGIy%(I_ z6l}dbxQ8vqvc0VZ8Y?Bj+q{@67#F;H-V9h);N~#_`5gr%O#~K6aOl+nc(mjY9aO!l zmy~EOl3>WMg%1d{iLjGl9$$=wy{I>k*lLkhMQGp*D3!`+3VRqSZzQ1eyikXUa@y(W^5H;`omVYPrd2du5Bm3ZPokaX?s2K zxb4~bg}nnO-Ql|ZoV`E(Py5Z|AGCV}#(rq`1&p1yYQOcAeF}QZwGX&m0Pyx>zkYl2 znErq74fOv!bRP2Rum7_Yb`pxTdPdf@E9_{2_2GDr=as+8QiDZc%T!%x;EfNV#Y_h; z{4+q)Sl2Uj8u{aOT)DQ0(Gm)@!rEP_-6!; z70B?Zhg!%WXdNJnuxh7wumFnHLdo#K0;S-joJ^_%@1R+w7deei%6X-qNDeb1T$SSt zPVz}{($EFR_nNBMme>3xFETD=Ft!#1vPLKuRrdM8Yuq zkZRvE1Yq-{e4TTC0aHU%!dV_fm@xtgO5oJw07BS{=4{-ELSG7iBZQ1+d9a9>LOE4s zftBnb0F&l}(3b)PlZIzH2rw^5rm{dHGX!As(~vD+4}cR>L}w!u ziND&D0uzs*FM;y^zP;bMV6WSsMqdQw|3|C*$9L`Z*z2|yFm~2f6IxH$p6EBW$^h&C zwuRR#3-|x&nEwBXI}ZOo|LKvob>#ZUWn1`3icOg&^dP&Wa88(+0^FW`8J!wtlszse zcp*|s`+{wFzhf38iAYG=*w3|)bi3(r3QKVGAPXJ-U<(lT{u@gBLveV&AppIfYcffZ zasU%J6_FlpWgq@v0GO#FH?U!#m#z%&Hw2(xPHf`lQ0f4R0#Z$SP@En9U;qdPV+lPd z&JOQ40MO1;oA5OPcw|71E=LTE7#J}yVqnC;h=CCUBL+qcj2IX(@PEv}*gaPHw?5(Y Q|68s-SpR?Qh2@w319}xYIRF3v literal 0 HcmV?d00001 diff --git a/flume-ng-sinks/flume-hive-sink/metastore_db/seg0/d18e0.dat b/flume-ng-sinks/flume-hive-sink/metastore_db/seg0/d18e0.dat new file mode 100644 index 0000000000000000000000000000000000000000..56231b126cb3b9cbbd317f52bd97b5de56c088a4 GIT binary patch literal 4096 zcmZQzpdSbTT2%tX%nS?yAX`O%n1_Lpfdjg#Z8m literal 0 HcmV?d00001 diff --git a/flume-ng-sinks/flume-hive-sink/metastore_db/seg0/d1900.dat b/flume-ng-sinks/flume-hive-sink/metastore_db/seg0/d1900.dat new file mode 100644 index 0000000000000000000000000000000000000000..898f6a132934d4669c14e79f776f79ff413cd2b8 GIT binary patch literal 4096 zcmeIuu?YYm3`Efr8xac&7qE62N3n2!j9{m=)jvp)BM^AfzjS*wZM>BC*J6|86!!9n hJDuI%88x=Mr9l7z1Q0*~0R#|0009ILKmdV0HW~t>Aut*O iqaiRF0;3@?8UmvsFd71*Aut*OqaiRF0;3^-5dr{ivjWoq literal 0 HcmV?d00001 diff --git a/flume-ng-sinks/flume-hive-sink/metastore_db/seg0/d1930.dat b/flume-ng-sinks/flume-hive-sink/metastore_db/seg0/d1930.dat new file mode 100644 index 0000000000000000000000000000000000000000..898f6a132934d4669c14e79f776f79ff413cd2b8 GIT binary patch literal 4096 zcmeIuu?YYm3`Efr8xac&7qE62N3n2!j9{m=)jvp)BM^AfzjS*wZM>BC*J6|86!!9n hJDuI%88x=Mr9l7z1Q0*~0R#|0009ILKmdVBC*J6|86!!9n hJDuI%88x=Mr9l7z1Q0*~0R#|0009ILKmdVBC*J6|86!!9n hJDuI%88x=Mr9l7z1Q0*~0R#|0009ILKmdVBC*J6|86!!9n hJDuI%88x=Mr9l7z1Q0*~0R#|0009ILKmdVAut*O iqaiRF0;3@?8UmvsFd71*Aut*OqaiRF0;3^-5dr{n+ygiO literal 0 HcmV?d00001 diff --git a/flume-ng-sinks/flume-hive-sink/metastore_db/seg0/d19c1.dat b/flume-ng-sinks/flume-hive-sink/metastore_db/seg0/d19c1.dat new file mode 100644 index 0000000000000000000000000000000000000000..b29acd7c7d269c4e081209de8e5cb34a3311e85a GIT binary patch literal 4096 zcmZQzpdSbTT2%tX+zbo?Fk2ZJ7#TQV95k8{%I8882Xh}(%!$V)J}NdE0;3@?8Umvs gFd71*Aut*OqaiRF0;3@?8UmvsFd71*A%GDA0A&;dNB{r; literal 0 HcmV?d00001 diff --git a/flume-ng-sinks/flume-hive-sink/metastore_db/seg0/d19d0.dat b/flume-ng-sinks/flume-hive-sink/metastore_db/seg0/d19d0.dat new file mode 100644 index 0000000000000000000000000000000000000000..fd69350ab4492229bf299ba5c03a36d8750aadfa GIT binary patch literal 4096 zcmZQzpdSbTT2%tX%nS?yAX`O%n1_LpfdjAut*O iqaiRF0;3@?8UmvsFd71*Aut*OqaiRF0;3^-5dr{n+ygiO literal 0 HcmV?d00001 diff --git a/flume-ng-sinks/flume-hive-sink/metastore_db/seg0/d1a91.dat b/flume-ng-sinks/flume-hive-sink/metastore_db/seg0/d1a91.dat new file mode 100644 index 0000000000000000000000000000000000000000..b29acd7c7d269c4e081209de8e5cb34a3311e85a GIT binary patch literal 4096 zcmZQzpdSbTT2%tX+zbo?Fk2ZJ7#TQV95k8{%I8882Xh}(%!$V)J}NdE0;3@?8Umvs gFd71*Aut*OqaiRF0;3@?8UmvsFd71*A%GDA0A&;dNB{r; literal 0 HcmV?d00001 diff --git a/flume-ng-sinks/flume-hive-sink/metastore_db/seg0/d1aa0.dat b/flume-ng-sinks/flume-hive-sink/metastore_db/seg0/d1aa0.dat new file mode 100644 index 0000000000000000000000000000000000000000..9ae5d87652f235806d34d38d8317614a4de4b769 GIT binary patch literal 4096 zcmZQzpdSbTT2%tX%nS?yAX`O%n1_LpfdjBC*J6|86!!9n hJDuI%88x=Mr9l7z1Q0*~0R#|0009ILKmdVAut*O iqaiRF0;3@?8UmvsFd71*Aut*OqaiRF0;3^-5dr{n+ygiO literal 0 HcmV?d00001 diff --git a/flume-ng-sinks/flume-hive-sink/metastore_db/seg0/d1c31.dat b/flume-ng-sinks/flume-hive-sink/metastore_db/seg0/d1c31.dat new file mode 100644 index 0000000000000000000000000000000000000000..b29acd7c7d269c4e081209de8e5cb34a3311e85a GIT binary patch literal 4096 zcmZQzpdSbTT2%tX+zbo?Fk2ZJ7#TQV95k8{%I8882Xh}(%!$V)J}NdE0;3@?8Umvs gFd71*Aut*OqaiRF0;3@?8UmvsFd71*A%GDA0A&;dNB{r; literal 0 HcmV?d00001 diff --git a/flume-ng-sinks/flume-hive-sink/metastore_db/seg0/d1c40.dat b/flume-ng-sinks/flume-hive-sink/metastore_db/seg0/d1c40.dat new file mode 100644 index 0000000000000000000000000000000000000000..9ae5d87652f235806d34d38d8317614a4de4b769 GIT binary patch literal 4096 zcmZQzpdSbTT2%tX%nS?yAX`O%n1_LpfdjAut*O iqaiRF0;3@?8UmvsFd71*Aut*OqaiRF0;3^-5dr{F)&r{m literal 0 HcmV?d00001 diff --git a/flume-ng-sinks/flume-hive-sink/metastore_db/seg0/d1e10.dat b/flume-ng-sinks/flume-hive-sink/metastore_db/seg0/d1e10.dat new file mode 100644 index 0000000000000000000000000000000000000000..fd69350ab4492229bf299ba5c03a36d8750aadfa GIT binary patch literal 4096 zcmZQzpdSbTT2%tX%nS?yAX`O%n1_Lpfdj + + + + 4.0.0 + + + org.apache.flume + flume-ng-sinks + 1.6.0-SNAPSHOT + + + org.apache.flume.flume-ng-sinks + flume-hive-sink + Flume NG Hive Sink + + + + + org.apache.rat + apache-rat-plugin + + + + + + + hadoop-1.0 + + + !hadoop.profile + + + + + + org.apache.hadoop + hadoop-core + ${hadoop.version} + test + + + + + hadoop-2 + + + hadoop.profile + 2 + + + + + org.apache.hadoop + hadoop-common + ${hadoop.version} + test + true + + + + org.apache.hadoop + hadoop-mapreduce-client-core + test + ${hadoop.version} + + + + + + hbase-98 + + + hadoop.profile + hbase-98 + + + + + org.apache.hadoop + hadoop-common + test + true + + + + org.apache.hadoop + hadoop-mapreduce-client-core + test + ${hadoop.version} + + + + + + + + + org.apache.flume + flume-ng-sdk + + + + org.apache.flume + flume-ng-configuration + + + + org.apache.flume + flume-ng-core + + + + org.slf4j + slf4j-api + + + + junit + junit + test + + + + org.slf4j + slf4j-log4j12 + test + + + + org.apache.hive.hcatalog + hive-hcatalog-streaming + provided + + + + org.apache.hive.hcatalog + hive-hcatalog-core + provided + ${hive.version} + + + + org.apache.hive + hive-cli + test + + + + + xerces + xercesImpl + runtime + 2.9.1 + + + + xalan + serializer + 2.7.1 + runtime + + + xalan + xalan + runtime + 2.7.1 + + + + + + diff --git a/flume-ng-sinks/flume-hive-sink/src/main/java/org/apache/flume/sink/hive/Config.java b/flume-ng-sinks/flume-hive-sink/src/main/java/org/apache/flume/sink/hive/Config.java new file mode 100644 index 0000000000..b2d258227c --- /dev/null +++ b/flume-ng-sinks/flume-hive-sink/src/main/java/org/apache/flume/sink/hive/Config.java @@ -0,0 +1,41 @@ +/** + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.flume.sink.hive; + +public class Config { + public static final String HIVE_METASTORE = "hive.metastore"; + public static final String HIVE_DATABASE = "hive.database"; + public static final String HIVE_TABLE = "hive.table"; + public static final String HIVE_PARTITION = "hive.partition"; + public static final String HIVE_TXNS_PER_BATCH_ASK = "hive.txnsPerBatchAsk"; + public static final String BATCH_SIZE = "batchSize"; + public static final String IDLE_TIMEOUT = "idleTimeout"; + public static final String CALL_TIMEOUT = "callTimeout"; + public static final String HEART_BEAT_INTERVAL = "heartBeatInterval"; + public static final String MAX_OPEN_CONNECTIONS = "maxOpenConnections"; + public static final String USE_LOCAL_TIME_STAMP = "useLocalTimeStamp"; + public static final String TIME_ZONE = "timeZone"; + public static final String ROUND_UNIT = "roundUnit"; + public static final String ROUND = "round"; + public static final String HOUR = "hour"; + public static final String MINUTE = "minute"; + public static final String SECOND = "second"; + public static final String ROUND_VALUE = "roundValue"; + public static final String SERIALIZER = "serializer"; +} diff --git a/flume-ng-sinks/flume-hive-sink/src/main/java/org/apache/flume/sink/hive/HiveDelimitedTextSerializer.java b/flume-ng-sinks/flume-hive-sink/src/main/java/org/apache/flume/sink/hive/HiveDelimitedTextSerializer.java new file mode 100644 index 0000000000..8f64435f45 --- /dev/null +++ b/flume-ng-sinks/flume-hive-sink/src/main/java/org/apache/flume/sink/hive/HiveDelimitedTextSerializer.java @@ -0,0 +1,109 @@ +/** + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.flume.sink.hive; + + +import org.apache.flume.Context; +import org.apache.flume.Event; +import org.apache.hive.hcatalog.streaming.DelimitedInputWriter; +import org.apache.hive.hcatalog.streaming.HiveEndPoint; +import org.apache.hive.hcatalog.streaming.RecordWriter; +import org.apache.hive.hcatalog.streaming.StreamingException; +import org.apache.hive.hcatalog.streaming.TransactionBatch; + +import java.io.IOException; + +/** Forwards the incoming event body to Hive unmodified + * Sets up the delimiter and the field to column mapping + */ +public class HiveDelimitedTextSerializer implements HiveEventSerializer { + public static final String ALIAS = "DELIMITED"; + + public static final String defaultDelimiter = ","; + public static final String SERIALIZER_DELIMITER = "serializer.delimiter"; + public static final String SERIALIZER_FIELDNAMES = "serializer.fieldnames"; + public static final String SERIALIZER_SERDE_SEPARATOR = "serializer.serdeSeparator"; + + private String delimiter; + private String[] fieldToColMapping = null; + private Character serdeSeparator = null; + + @Override + public void write(TransactionBatch txnBatch, Event e) + throws StreamingException, IOException, InterruptedException { + txnBatch.write(e.getBody()); + } + + @Override + public RecordWriter createRecordWriter(HiveEndPoint endPoint) + throws StreamingException, IOException, ClassNotFoundException { + if (serdeSeparator == null) { + return new DelimitedInputWriter(fieldToColMapping, delimiter, endPoint); + } + return new DelimitedInputWriter(fieldToColMapping, delimiter, endPoint, null + , serdeSeparator); + } + + @Override + public void configure(Context context) { + delimiter = parseDelimiterSpec( + context.getString(SERIALIZER_DELIMITER, defaultDelimiter) ); + String fieldNames = context.getString(SERIALIZER_FIELDNAMES); + if (fieldNames == null) { + throw new IllegalArgumentException("serializer.fieldnames is not specified " + + "for serializer " + this.getClass().getName() ); + } + String serdeSeparatorStr = context.getString(SERIALIZER_SERDE_SEPARATOR); + this.serdeSeparator = parseSerdeSeparatorSpec(serdeSeparatorStr); + + // split, but preserve empty fields (-1) + fieldToColMapping = fieldNames.trim().split(",",-1); + } + + // if delimiter is a double quoted like "\t", drop quotes + private static String parseDelimiterSpec(String delimiter) { + if (delimiter == null) { + return null; + } + if (delimiter.charAt(0) == '"' && + delimiter.charAt(delimiter.length()-1) == '"') { + return delimiter.substring(1,delimiter.length()-1); + } + return delimiter; + } + + // if delimiter is a single quoted character like '\t', drop quotes + private static Character parseSerdeSeparatorSpec(String separatorStr) { + if (separatorStr == null) { + return null; + } + if (separatorStr.length() == 1) { + return separatorStr.charAt(0); + } + if (separatorStr.length() == 3 && + separatorStr.charAt(2) == '\'' && + separatorStr.charAt(separatorStr.length()-1) == '\'') { + return separatorStr.charAt(1); + } + + throw new IllegalArgumentException("serializer.serdeSeparator spec is invalid " + + "for " + ALIAS + " serializer " ); + } + +} diff --git a/flume-ng-sinks/flume-hive-sink/src/main/java/org/apache/flume/sink/hive/HiveEventSerializer.java b/flume-ng-sinks/flume-hive-sink/src/main/java/org/apache/flume/sink/hive/HiveEventSerializer.java new file mode 100644 index 0000000000..c233d3d860 --- /dev/null +++ b/flume-ng-sinks/flume-hive-sink/src/main/java/org/apache/flume/sink/hive/HiveEventSerializer.java @@ -0,0 +1,38 @@ +/** + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.flume.sink.hive; + + +import org.apache.flume.Event; +import org.apache.flume.conf.Configurable; +import org.apache.hive.hcatalog.streaming.HiveEndPoint; +import org.apache.hive.hcatalog.streaming.RecordWriter; +import org.apache.hive.hcatalog.streaming.StreamingException; +import org.apache.hive.hcatalog.streaming.TransactionBatch; + +import java.io.IOException; + +public interface HiveEventSerializer extends Configurable { + public void write(TransactionBatch batch, Event e) + throws StreamingException, IOException, InterruptedException; + + RecordWriter createRecordWriter(HiveEndPoint endPoint) + throws StreamingException, IOException, ClassNotFoundException; + +} diff --git a/flume-ng-sinks/flume-hive-sink/src/main/java/org/apache/flume/sink/hive/HiveJsonSerializer.java b/flume-ng-sinks/flume-hive-sink/src/main/java/org/apache/flume/sink/hive/HiveJsonSerializer.java new file mode 100644 index 0000000000..a75073f732 --- /dev/null +++ b/flume-ng-sinks/flume-hive-sink/src/main/java/org/apache/flume/sink/hive/HiveJsonSerializer.java @@ -0,0 +1,55 @@ +/** + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.flume.sink.hive; + +import org.apache.flume.Context; +import org.apache.flume.Event; +import org.apache.hive.hcatalog.streaming.HiveEndPoint; +import org.apache.hive.hcatalog.streaming.RecordWriter; +import org.apache.hive.hcatalog.streaming.StreamingException; +import org.apache.hive.hcatalog.streaming.StrictJsonWriter; +import org.apache.hive.hcatalog.streaming.TransactionBatch; + +import java.io.IOException; + +/** Forwards the incoming event body to Hive unmodified + * Sets up the delimiter and the field to column mapping + */ + +public class HiveJsonSerializer implements HiveEventSerializer { + public static final String ALIAS = "JSON"; + + @Override + public void write(TransactionBatch txnBatch, Event e) + throws StreamingException, IOException, InterruptedException { + txnBatch.write(e.getBody()); + } + + @Override + public RecordWriter createRecordWriter(HiveEndPoint endPoint) + throws StreamingException, IOException, ClassNotFoundException { + return new StrictJsonWriter(endPoint); + } + + @Override + public void configure(Context context) { + return; + } + +} diff --git a/flume-ng-sinks/flume-hive-sink/src/main/java/org/apache/flume/sink/hive/HiveSink.java b/flume-ng-sinks/flume-hive-sink/src/main/java/org/apache/flume/sink/hive/HiveSink.java new file mode 100644 index 0000000000..6fe332a37b --- /dev/null +++ b/flume-ng-sinks/flume-hive-sink/src/main/java/org/apache/flume/sink/hive/HiveSink.java @@ -0,0 +1,524 @@ +/** + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.flume.sink.hive; + +import com.google.common.annotations.VisibleForTesting; +import com.google.common.base.Preconditions; +import com.google.common.collect.Lists; +import com.google.common.collect.Maps; +import com.google.common.util.concurrent.ThreadFactoryBuilder; +import org.apache.flume.Channel; +import org.apache.flume.Context; +import org.apache.flume.Event; +import org.apache.flume.EventDeliveryException; +import org.apache.flume.Transaction; +import org.apache.flume.conf.Configurable; +import org.apache.flume.formatter.output.BucketPath; +import org.apache.flume.instrumentation.SinkCounter; +import org.apache.flume.sink.AbstractSink; +import org.apache.hive.hcatalog.streaming.*; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +import java.util.ArrayList; +import java.util.Arrays; +import java.util.Calendar; +import java.util.List; +import java.util.Map; +import java.util.Map.Entry; +import java.util.TimeZone; +import java.util.Timer; +import java.util.TimerTask; +import java.util.concurrent.ExecutorService; +import java.util.concurrent.Executors; +import java.util.concurrent.TimeUnit; +import java.util.concurrent.atomic.AtomicBoolean; + +public class HiveSink extends AbstractSink implements Configurable { + + private static final Logger LOG = LoggerFactory + .getLogger(HiveSink.class); + + private static final int DEFAULT_MAXOPENCONNECTIONS = 500; + private static final int DEFAULT_TXNSPERBATCH = 100; + private static final int DEFAULT_BATCHSIZE = 15000; + private static final int DEFAULT_CALLTIMEOUT = 10000; + private static final int DEFAULT_IDLETIMEOUT = 0; + private static final int DEFAULT_HEARTBEATINTERVAL = 240; // seconds + + + private Map allWriters; + + private SinkCounter sinkCounter; + private volatile int idleTimeout; + private String metaStoreUri; + private String proxyUser; + private String database; + private String table; + private List partitionVals; + private Integer txnsPerBatchAsk; + private Integer batchSize; + private Integer maxOpenConnections; + private boolean autoCreatePartitions; + private String serializerType; + private HiveEventSerializer serializer; + + /** + * Default timeout for blocking I/O calls in HiveWriter + */ + private Integer callTimeout; + private Integer heartBeatInterval; + + private ExecutorService callTimeoutPool; + + private boolean useLocalTime; + private TimeZone timeZone; + private boolean needRounding; + private int roundUnit; + private Integer roundValue; + + private Timer heartBeatTimer = new Timer(); + private AtomicBoolean timeToSendHeartBeat = new AtomicBoolean(false); + + @VisibleForTesting + Map getAllWriters() { + return allWriters; + } + + // read configuration and setup thresholds + @Override + public void configure(Context context) { + + metaStoreUri = context.getString(Config.HIVE_METASTORE); + if (metaStoreUri == null) { + throw new IllegalArgumentException(Config.HIVE_METASTORE + " config setting is not " + + "specified for sink " + getName()); + } + if (metaStoreUri.equalsIgnoreCase("null")) { // for testing support + metaStoreUri = null; + } + proxyUser = null; // context.getString("hive.proxyUser"); not supported by hive api yet + database = context.getString(Config.HIVE_DATABASE); + if (database == null) { + throw new IllegalArgumentException(Config.HIVE_DATABASE + " config setting is not " + + "specified for sink " + getName()); + } + table = context.getString(Config.HIVE_TABLE); + if (table == null) { + throw new IllegalArgumentException(Config.HIVE_TABLE + " config setting is not " + + "specified for sink " + getName()); + } + + String partitions = context.getString(Config.HIVE_PARTITION); + if (partitions != null) { + partitionVals = Arrays.asList(partitions.split(",")); + } + + + txnsPerBatchAsk = context.getInteger(Config.HIVE_TXNS_PER_BATCH_ASK, DEFAULT_TXNSPERBATCH); + if (txnsPerBatchAsk < 0) { + LOG.warn(getName() + ". hive.txnsPerBatchAsk must be positive number. Defaulting to " + + DEFAULT_TXNSPERBATCH); + txnsPerBatchAsk = DEFAULT_TXNSPERBATCH; + } + batchSize = context.getInteger(Config.BATCH_SIZE, DEFAULT_BATCHSIZE); + if (batchSize < 0) { + LOG.warn(getName() + ". batchSize must be positive number. Defaulting to " + + DEFAULT_BATCHSIZE); + batchSize = DEFAULT_BATCHSIZE; + } + idleTimeout = context.getInteger(Config.IDLE_TIMEOUT, DEFAULT_IDLETIMEOUT); + if (idleTimeout < 0) { + LOG.warn(getName() + ". idleTimeout must be positive number. Defaulting to " + + DEFAULT_IDLETIMEOUT); + idleTimeout = DEFAULT_IDLETIMEOUT; + } + callTimeout = context.getInteger(Config.CALL_TIMEOUT, DEFAULT_CALLTIMEOUT); + if (callTimeout < 0) { + LOG.warn(getName() + ". callTimeout must be positive number. Defaulting to " + + DEFAULT_CALLTIMEOUT); + callTimeout = DEFAULT_CALLTIMEOUT; + } + + heartBeatInterval = context.getInteger(Config.HEART_BEAT_INTERVAL, DEFAULT_HEARTBEATINTERVAL); + if (heartBeatInterval < 0) { + LOG.warn(getName() + ". heartBeatInterval must be positive number. Defaulting to " + + DEFAULT_HEARTBEATINTERVAL); + heartBeatInterval = DEFAULT_HEARTBEATINTERVAL; + } + maxOpenConnections = context.getInteger(Config.MAX_OPEN_CONNECTIONS, DEFAULT_MAXOPENCONNECTIONS); + autoCreatePartitions = context.getBoolean("autoCreatePartitions", true); + + // Timestamp processing + useLocalTime = context.getBoolean(Config.USE_LOCAL_TIME_STAMP, false); + + String tzName = context.getString(Config.TIME_ZONE); + timeZone = (tzName == null) ? null : TimeZone.getTimeZone(tzName); + needRounding = context.getBoolean(Config.ROUND, false); + + String unit = context.getString(Config.ROUND_UNIT, Config.MINUTE); + if (unit.equalsIgnoreCase(Config.HOUR)) { + this.roundUnit = Calendar.HOUR_OF_DAY; + } else if (unit.equalsIgnoreCase(Config.MINUTE)) { + this.roundUnit = Calendar.MINUTE; + } else if (unit.equalsIgnoreCase(Config.SECOND)){ + this.roundUnit = Calendar.SECOND; + } else { + LOG.warn(getName() + ". Rounding unit is not valid, please set one of " + + "minute, hour or second. Rounding will be disabled"); + needRounding = false; + } + this.roundValue = context.getInteger(Config.ROUND_VALUE, 1); + if (roundUnit == Calendar.SECOND || roundUnit == Calendar.MINUTE){ + Preconditions.checkArgument(roundValue > 0 && roundValue <= 60, + "Round value must be > 0 and <= 60"); + } else if (roundUnit == Calendar.HOUR_OF_DAY){ + Preconditions.checkArgument(roundValue > 0 && roundValue <= 24, + "Round value must be > 0 and <= 24"); + } + + // Serializer + serializerType = context.getString(Config.SERIALIZER, ""); + if (serializerType.isEmpty()) { + throw new IllegalArgumentException("serializer config setting is not " + + "specified for sink " + getName()); + } + + serializer = createSerializer(serializerType); + serializer.configure(context); + + Preconditions.checkArgument(batchSize > 0, "batchSize must be greater than 0"); + + if (sinkCounter == null) { + sinkCounter = new SinkCounter(getName()); + } + } + + @VisibleForTesting + protected SinkCounter getCounter() { + return sinkCounter; + } + private HiveEventSerializer createSerializer(String serializerName) { + if(serializerName.compareToIgnoreCase(HiveDelimitedTextSerializer.ALIAS) == 0 || + serializerName.compareTo(HiveDelimitedTextSerializer.class.getName()) == 0) { + return new HiveDelimitedTextSerializer(); + } else if (serializerName.compareToIgnoreCase(HiveJsonSerializer.ALIAS) == 0 || + serializerName.compareTo(HiveJsonSerializer.class.getName()) == 0) { + return new HiveJsonSerializer(); + } + + try { + return (HiveEventSerializer) Class.forName(serializerName).newInstance(); + } catch (Exception e) { + throw new IllegalArgumentException("Unable to instantiate serializer: " + serializerName + + " on sink: " + getName(), e); + } + } + + + /** + * Pull events out of channel, find corresponding HiveWriter and write to it. + * Take at most batchSize events per Transaction.
    + * This method is not thread safe. + */ + public Status process() throws EventDeliveryException { + // writers used in this Txn + + Channel channel = getChannel(); + Transaction transaction = channel.getTransaction(); + transaction.begin(); + boolean success = false; + try { + // 1 Enable Heart Beats + if (timeToSendHeartBeat.compareAndSet(true, false)) { + enableHeartBeatOnAllWriters(); + } + + // 2 Drain Batch + int txnEventCount = drainOneBatch(channel); + transaction.commit(); + success = true; + + // 3 Update Counters + if (txnEventCount < 1) { + return Status.BACKOFF; + } else { + return Status.READY; + } + } catch (InterruptedException err) { + LOG.warn(getName() + ": Thread was interrupted.", err); + return Status.BACKOFF; + } catch (Exception e) { + throw new EventDeliveryException(e); + } finally { + if (!success) { + transaction.rollback(); + } + transaction.close(); + } + } + + // Drains one batch of events from Channel into Hive + private int drainOneBatch(Channel channel) + throws HiveWriter.Failure, InterruptedException { + int txnEventCount = 0; + try { + Map activeWriters = Maps.newHashMap(); + for (; txnEventCount < batchSize; ++txnEventCount) { + // 0) Read event from Channel + Event event = channel.take(); + if (event == null) { + break; + } + + //1) Create end point by substituting place holders + HiveEndPoint endPoint = makeEndPoint(metaStoreUri, database, table, + partitionVals, event.getHeaders(), timeZone, + needRounding, roundUnit, roundValue, useLocalTime); + + //2) Create or reuse Writer + HiveWriter writer = getOrCreateWriter(activeWriters, endPoint); + + //3) Write + LOG.debug("{} : Writing event to {}", getName(), endPoint); + writer.write(event); + + } // for + + //4) Update counters + if (txnEventCount == 0) { + sinkCounter.incrementBatchEmptyCount(); + } else if (txnEventCount == batchSize) { + sinkCounter.incrementBatchCompleteCount(); + } else { + sinkCounter.incrementBatchUnderflowCount(); + } + sinkCounter.addToEventDrainAttemptCount(txnEventCount); + + + // 5) Flush all Writers + for (HiveWriter writer : activeWriters.values()) { + writer.flush(true); + } + + sinkCounter.addToEventDrainSuccessCount(txnEventCount); + return txnEventCount; + } catch (HiveWriter.Failure e) { + LOG.warn(getName() + " : " + e.getMessage(), e); + abortAllWriters(); + closeAllWriters(); + throw e; + } + } + + private void enableHeartBeatOnAllWriters() { + for (HiveWriter writer : allWriters.values()) { + writer.setHearbeatNeeded(); + } + } + + private HiveWriter getOrCreateWriter(Map activeWriters, + HiveEndPoint endPoint) + throws HiveWriter.ConnectException, InterruptedException { + try { + HiveWriter writer = allWriters.get( endPoint ); + if (writer == null) { + LOG.info(getName() + ": Creating Writer to Hive end point : " + endPoint); + writer = new HiveWriter(endPoint, txnsPerBatchAsk, autoCreatePartitions, + callTimeout, callTimeoutPool, proxyUser, serializer, sinkCounter); + + sinkCounter.incrementConnectionCreatedCount(); + if (allWriters.size() > maxOpenConnections){ + int retired = closeIdleWriters(); + if (retired == 0) { + closeEldestWriter(); + } + } + allWriters.put(endPoint, writer); + activeWriters.put(endPoint, writer); + } + else { + if (activeWriters.get(endPoint) == null) { + activeWriters.put(endPoint,writer); + } + } + return writer; + } catch (HiveWriter.ConnectException e) { + sinkCounter.incrementConnectionFailedCount(); + throw e; + } + + } + + private HiveEndPoint makeEndPoint(String metaStoreUri, String database, String table, + List partVals, Map headers, + TimeZone timeZone, boolean needRounding, + int roundUnit, Integer roundValue, + boolean useLocalTime) { + if (partVals == null) { + return new HiveEndPoint(metaStoreUri, database, table, null); + } + + ArrayList realPartVals = Lists.newArrayList(); + for (String partVal : partVals) { + realPartVals.add(BucketPath.escapeString(partVal, headers, timeZone, + needRounding, roundUnit, roundValue, useLocalTime)); + } + return new HiveEndPoint(metaStoreUri, database, table, realPartVals); + } + + /** + * Locate writer that has not been used for longest time and retire it + */ + private void closeEldestWriter() throws InterruptedException { + long oldestTimeStamp = System.currentTimeMillis(); + HiveEndPoint eldest = null; + for (Entry entry : allWriters.entrySet()) { + if (entry.getValue().getLastUsed() < oldestTimeStamp) { + eldest = entry.getKey(); + oldestTimeStamp = entry.getValue().getLastUsed(); + } + } + + try { + sinkCounter.incrementConnectionCreatedCount(); + LOG.info(getName() + ": Closing least used Writer to Hive EndPoint : " + eldest); + allWriters.remove(eldest).close(); + } catch (InterruptedException e) { + LOG.warn(getName() + ": Interrupted when attempting to close writer for end point: " + + eldest, e); + throw e; + } + } + + /** + * Locate all writers past idle timeout and retire them + * @return number of writers retired + */ + private int closeIdleWriters() throws InterruptedException { + int count = 0; + long now = System.currentTimeMillis(); + ArrayList retirees = Lists.newArrayList(); + + //1) Find retirement candidates + for (Entry entry : allWriters.entrySet()) { + if (now - entry.getValue().getLastUsed() > idleTimeout) { + ++count; + retirees.add(entry.getKey()); + } + } + //2) Retire them + for(HiveEndPoint ep : retirees) { + sinkCounter.incrementConnectionClosedCount(); + LOG.info(getName() + ": Closing idle Writer to Hive end point : {}", ep); + allWriters.remove(ep).close(); + } + return count; + } + + /** + * Closes all writers and remove them from cache + * @return number of writers retired + */ + private void closeAllWriters() throws InterruptedException { + //1) Retire writers + for (Entry entry : allWriters.entrySet()) { + entry.getValue().close(); + } + + //2) Clear cache + allWriters.clear(); + } + + /** + * Abort current Txn on all writers + * @return number of writers retired + */ + private void abortAllWriters() throws InterruptedException { + for (Entry entry : allWriters.entrySet()) { + entry.getValue().abort(); + } + } + + @Override + public void stop() { + // do not constrain close() calls with a timeout + for (Entry entry : allWriters.entrySet()) { + try { + HiveWriter w = entry.getValue(); + LOG.info("Closing connection to {}", w); + w.closeConnection(); + } catch (InterruptedException ex) { + Thread.currentThread().interrupt(); + } + } + + // shut down all thread pools + callTimeoutPool.shutdown(); + try { + while (callTimeoutPool.isTerminated() == false) { + callTimeoutPool.awaitTermination( + Math.max(DEFAULT_CALLTIMEOUT, callTimeout), TimeUnit.MILLISECONDS); + } + } catch (InterruptedException ex) { + LOG.warn(getName() + ":Shutdown interrupted on " + callTimeoutPool, ex); + } + + callTimeoutPool = null; + allWriters.clear(); + allWriters = null; + sinkCounter.stop(); + super.stop(); + LOG.info("Hive Sink {} stopped", getName() ); + } + + @Override + public void start() { + String timeoutName = "hive-" + getName() + "-call-runner-%d"; + // call timeout pool needs only 1 thd as sink is effectively single threaded + callTimeoutPool = Executors.newFixedThreadPool(1, + new ThreadFactoryBuilder().setNameFormat(timeoutName).build()); + + this.allWriters = Maps.newHashMap(); + sinkCounter.start(); + super.start(); + setupHeartBeatTimer(); + LOG.info(getName() + ": Hive Sink {} started", getName() ); + } + + private void setupHeartBeatTimer() { + if (heartBeatInterval > 0) { + heartBeatTimer.schedule(new TimerTask() { + @Override + public void run() { + timeToSendHeartBeat.set(true); + setupHeartBeatTimer(); + } + }, heartBeatInterval * 1000); + } + } + + + @Override + public String toString() { + return "{ Sink type:" + getClass().getSimpleName() + ", name:" + getName() + + " }"; + } + +} diff --git a/flume-ng-sinks/flume-hive-sink/src/main/java/org/apache/flume/sink/hive/HiveWriter.java b/flume-ng-sinks/flume-hive-sink/src/main/java/org/apache/flume/sink/hive/HiveWriter.java new file mode 100644 index 0000000000..4a06feb9e8 --- /dev/null +++ b/flume-ng-sinks/flume-hive-sink/src/main/java/org/apache/flume/sink/hive/HiveWriter.java @@ -0,0 +1,426 @@ +/** + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.flume.sink.hive; + +import java.io.IOException; +import java.util.concurrent.Callable; +import java.util.concurrent.ExecutionException; +import java.util.concurrent.ExecutorService; +import java.util.concurrent.Future; +import java.util.concurrent.TimeUnit; +import java.util.concurrent.TimeoutException; + +import org.apache.hive.hcatalog.streaming.*; + +import org.apache.flume.Event; + +import org.apache.flume.instrumentation.SinkCounter; + + +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + + +/** + * Internal API intended for HiveSink use. + */ +class HiveWriter { + + private static final Logger LOG = LoggerFactory + .getLogger(HiveWriter.class); + + private final HiveEndPoint endPoint; + private HiveEventSerializer serializer; + private final StreamingConnection connection; + private final int txnsPerBatch; + private final RecordWriter recordWriter; + private TransactionBatch txnBatch; + + private final ExecutorService callTimeoutPool; + + private final long callTimeout; + + private long lastUsed; // time of last flush on this writer + + private SinkCounter sinkCounter; + private int batchCounter; + private long eventCounter; + private long processSize; + + protected boolean closed; // flag indicating HiveWriter was closed + private boolean autoCreatePartitions; + + private boolean hearbeatNeeded = false; + + HiveWriter(HiveEndPoint endPoint, int txnsPerBatch, + boolean autoCreatePartitions, long callTimeout, + ExecutorService callTimeoutPool, String hiveUser, + HiveEventSerializer serializer, SinkCounter sinkCounter) + throws ConnectException, InterruptedException { + try { + this.autoCreatePartitions = autoCreatePartitions; + this.sinkCounter = sinkCounter; + this.callTimeout = callTimeout; + this.callTimeoutPool = callTimeoutPool; + this.endPoint = endPoint; + this.connection = newConnection(hiveUser); + this.txnsPerBatch = txnsPerBatch; + this.serializer = serializer; + this.recordWriter = serializer.createRecordWriter(endPoint); + this.txnBatch = nextTxnBatch(recordWriter); + this.closed = false; + this.lastUsed = System.currentTimeMillis(); + } catch (InterruptedException e) { + throw e; + } catch (RuntimeException e) { + throw e; + } catch (Exception e) { + throw new ConnectException(endPoint, e); + } + } + + @Override + public String toString() { + return endPoint.toString(); + } + + /** + * Clear the class counters + */ + private void resetCounters() { + eventCounter = 0; + processSize = 0; + batchCounter = 0; + } + + void setHearbeatNeeded() { + hearbeatNeeded = true; + } + + + /** + * Write data, update stats + * @param event + * @throws StreamingException + * @throws InterruptedException + */ + public synchronized void write(final Event event) + throws WriteException, InterruptedException { + if (closed) { + throw new IllegalStateException("Writer closed. Cannot write to : " + endPoint); + } + + // write the event + try { + timedCall(new CallRunner1() { + @Override + public Void call() throws InterruptedException, StreamingException { + try { + serializer.write(txnBatch, event); + return null; + } catch (IOException e) { + throw new StreamingIOFailure(e.getMessage(), e); + } + } + }); + } catch (StreamingException e) { + throw new WriteException(endPoint, txnBatch.getCurrentTxnId(), e); + } catch (TimeoutException e) { + throw new WriteException(endPoint, txnBatch.getCurrentTxnId(), e); + } + + // Update Statistics + processSize += event.getBody().length; + eventCounter++; + } + + /** + * Commits the current Txn. + * If 'rollToNext' is true, will switch to next Txn in batch or to a + * new TxnBatch if current Txn batch is exhausted + */ + public void flush(boolean rollToNext) + throws CommitException, TxnBatchException, TxnFailure, InterruptedException { + //0 Heart beat on TxnBatch + if(hearbeatNeeded) { + hearbeatNeeded = false; + heartBeat(); + } + lastUsed = System.currentTimeMillis(); + + try { + //1 commit txn & close batch if needed + commitTxn(); + if(txnBatch.remainingTransactions() == 0) { + closeTxnBatch(); + txnBatch = null; + if(rollToNext) { + txnBatch = nextTxnBatch(recordWriter); + } + } + + //2 roll to next Txn + if(rollToNext) { + LOG.debug("Switching to next Txn for {}", endPoint); + txnBatch.beginNextTransaction(); // does not block + } + } catch (StreamingException e) { + throw new TxnFailure(txnBatch, e); + } + } + + /** + * Aborts the current Txn and switches to next Txn. + * @throws StreamingException if could not get new Transaction Batch, or switch to next Txn + */ + public void abort() throws InterruptedException { + abortTxn(); + } + + /** Queues up a heartbeat request on the current and remaining txns using the + * heartbeatThdPool and returns immediately + */ + public void heartBeat() throws InterruptedException { + // 1) schedule the heartbeat on one thread in pool + try { + timedCall(new CallRunner1() { + @Override + public Void call() throws StreamingException { + LOG.info("Sending heartbeat on batch " + txnBatch); + txnBatch.heartbeat(); + return null; + } + }); + } catch (InterruptedException e) { + throw e; + } catch (Exception e) { + LOG.warn("Unable to send heartbeat on Txn Batch " + txnBatch, e); + // Suppressing exceptions as we don't care for errors on heartbeats + } + } + + /** + * Close the Transaction Batch and connection + * @throws IOException + * @throws InterruptedException + */ + public void close() throws InterruptedException { + closeTxnBatch(); + closeConnection(); + closed = true; + } + + public void closeConnection() throws InterruptedException { + LOG.info("Closing connection to EndPoint : {}", endPoint); + try { + timedCall(new CallRunner1() { + @Override + public Void call() { + connection.close(); // could block + return null; + } + }); + sinkCounter.incrementConnectionClosedCount(); + } catch (Exception e) { + LOG.warn("Error closing connection to EndPoint : " + endPoint, e); + // Suppressing exceptions as we don't care for errors on connection close + } + } + + private void commitTxn() throws CommitException, InterruptedException { + if (LOG.isInfoEnabled()) { + LOG.info("Committing Txn " + txnBatch.getCurrentTxnId() + " on EndPoint: " + endPoint); + } + try { + timedCall(new CallRunner1() { + @Override + public Void call() throws StreamingException, InterruptedException { + txnBatch.commit(); // could block + return null; + } + }); + } catch (Exception e) { + throw new CommitException(endPoint, txnBatch.getCurrentTxnId(), e); + } + } + + private void abortTxn() throws InterruptedException { + LOG.info("Aborting Txn id {} on End Point {}", txnBatch.getCurrentTxnId(), endPoint); + try { + timedCall(new CallRunner1() { + @Override + public Void call() throws StreamingException, InterruptedException { + txnBatch.abort(); // could block + return null; + } + }); + } catch (InterruptedException e) { + throw e; + } catch (TimeoutException e) { + LOG.warn("Timeout while aborting Txn " + txnBatch.getCurrentTxnId() + " on EndPoint: " + endPoint, e); + } catch (Exception e) { + LOG.warn("Error aborting Txn " + txnBatch.getCurrentTxnId() + " on EndPoint: " + endPoint, e); + // Suppressing exceptions as we don't care for errors on abort + } + } + + private StreamingConnection newConnection(final String proxyUser) + throws InterruptedException, ConnectException { + try { + return timedCall(new CallRunner1() { + @Override + public StreamingConnection call() throws InterruptedException, StreamingException { + return endPoint.newConnection(autoCreatePartitions); // could block + } + }); + } catch (Exception e) { + throw new ConnectException(endPoint, e); + } + } + + private TransactionBatch nextTxnBatch(final RecordWriter recordWriter) + throws InterruptedException, TxnBatchException { + LOG.debug("Fetching new Txn Batch for {}", endPoint); + TransactionBatch batch = null; + try { + batch = timedCall(new CallRunner1() { + @Override + public TransactionBatch call() throws InterruptedException, StreamingException { + return connection.fetchTransactionBatch(txnsPerBatch, recordWriter); // could block + } + }); + LOG.info("Acquired Txn Batch {}. Switching to first txn", batch); + batch.beginNextTransaction(); + } catch (Exception e) { + throw new TxnBatchException(endPoint, e); + } + return batch; + } + + private void closeTxnBatch() throws InterruptedException { + try { + LOG.debug("Closing Txn Batch {}", txnBatch); + timedCall(new CallRunner1() { + @Override + public Void call() throws InterruptedException, StreamingException { + txnBatch.close(); // could block + return null; + } + }); + } catch (InterruptedException e) { + throw e; + } catch (Exception e) { + LOG.warn("Error closing Txn Batch " + txnBatch, e); + // Suppressing exceptions as we don't care for errors on batch close + } + } + + private T timedCall(final CallRunner1 callRunner) + throws TimeoutException, InterruptedException, StreamingException { + Future future = callTimeoutPool.submit(new Callable() { + @Override + public T call() throws StreamingException, InterruptedException { + return callRunner.call(); + } + }); + + try { + if (callTimeout > 0) { + return future.get(callTimeout, TimeUnit.MILLISECONDS); + } else { + return future.get(); + } + } catch (TimeoutException eT) { + future.cancel(true); + sinkCounter.incrementConnectionFailedCount(); + throw eT; + } catch (ExecutionException e1) { + sinkCounter.incrementConnectionFailedCount(); + Throwable cause = e1.getCause(); + if (cause instanceof IOException ) { + throw new StreamingIOFailure("I/O Failure", (IOException) cause); + } else if (cause instanceof StreamingException) { + throw (StreamingException) cause; + } else if (cause instanceof TimeoutException) { + throw new StreamingException("Operation Timed Out.", (TimeoutException) cause); + } else if (cause instanceof RuntimeException) { + throw (RuntimeException) cause; + } else if (cause instanceof InterruptedException) { + throw (InterruptedException) cause; + } + throw new StreamingException(e1.getMessage(), e1); + } + } + + long getLastUsed() { + return lastUsed; + } + + /** + * Simple interface whose call method is called by + * {#callWithTimeout} in a new thread inside a + * {@linkplain java.security.PrivilegedExceptionAction#run()} call. + * @param + */ + private interface CallRunner { + T call() throws Exception; + } + + + private interface CallRunner1 { + T call() throws StreamingException, InterruptedException; + } + + + public static class Failure extends Exception { + public Failure(String msg, Throwable cause) { + super(msg, cause); + } + } + + public static class WriteException extends Failure { + public WriteException(HiveEndPoint endPoint, Long currentTxnId, Throwable cause) { + super("Failed writing to : " + endPoint + ". TxnID : " + currentTxnId, cause); + } + } + + public static class CommitException extends Failure { + public CommitException(HiveEndPoint endPoint, Long txnID, Throwable cause) { + super("Commit of Txn " + txnID + " failed on EndPoint: " + endPoint, cause); + } + } + + public static class ConnectException extends Failure { + public ConnectException(HiveEndPoint ep, Throwable cause) { + super("Failed connecting to EndPoint " + ep, cause); + } + } + + public static class TxnBatchException extends Failure { + public TxnBatchException(HiveEndPoint ep, Throwable cause) { + super("Failed acquiring Transaction Batch from EndPoint: " + ep, cause); + } + } + + private class TxnFailure extends Failure { + public TxnFailure(TransactionBatch txnBatch, Throwable cause) { + super("Failed switching to next Txn in TxnBatch " + txnBatch, cause); + } + } +} diff --git a/flume-ng-sinks/flume-hive-sink/src/test/java/org/apache/flume/sink/hive/TestHiveSink.java b/flume-ng-sinks/flume-hive-sink/src/test/java/org/apache/flume/sink/hive/TestHiveSink.java new file mode 100644 index 0000000000..46724f29be --- /dev/null +++ b/flume-ng-sinks/flume-hive-sink/src/test/java/org/apache/flume/sink/hive/TestHiveSink.java @@ -0,0 +1,424 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, + * software distributed under the License is distributed on an + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + * KIND, either express or implied. See the License for the + * specific language governing permissions and limitations + * under the License. + */ + + +package org.apache.flume.sink.hive; + +import com.google.common.collect.Lists; +import junit.framework.Assert; +import org.apache.flume.Channel; +import org.apache.flume.Context; +import org.apache.flume.Event; +import org.apache.flume.EventDeliveryException; +import org.apache.flume.Transaction; +import org.apache.flume.channel.MemoryChannel; +import org.apache.flume.conf.Configurables; +import org.apache.flume.event.SimpleEvent; +import org.apache.flume.instrumentation.SinkCounter; +import org.apache.hadoop.hive.cli.CliSessionState; +import org.apache.hadoop.hive.conf.HiveConf; +import org.apache.hadoop.hive.metastore.api.MetaException; +import org.apache.hadoop.hive.metastore.txn.TxnDbUtil; +import org.apache.hadoop.hive.ql.CommandNeedRetryException; +import org.apache.hadoop.hive.ql.Driver; +import org.apache.hadoop.hive.ql.metadata.HiveException; +import org.apache.hadoop.hive.ql.session.SessionState; +import org.junit.After; +import org.junit.Before; +import org.junit.Rule; +import org.junit.Test; +import org.junit.rules.TemporaryFolder; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +import java.io.IOException; +import java.util.ArrayList; +import java.util.Calendar; +import java.util.List; +import java.util.UUID; + +public class TestHiveSink { + // 1) partitioned table + final static String dbName = "testing"; + final static String tblName = "alerts"; + + public static final String PART1_NAME = "continent"; + public static final String PART2_NAME = "country"; + public static final String[] partNames = { PART1_NAME, PART2_NAME }; + + private static final String COL1 = "id"; + private static final String COL2 = "msg"; + final String[] colNames = {COL1,COL2}; + private String[] colTypes = { "int", "string" }; + + private static final String PART1_VALUE = "Asia"; + private static final String PART2_VALUE = "India"; + private final ArrayList partitionVals; + + // 2) un-partitioned table + final static String dbName2 = "testing2"; + final static String tblName2 = "alerts2"; + final String[] colNames2 = {COL1,COL2}; + private String[] colTypes2 = { "int", "string" }; + + HiveSink sink = new HiveSink(); + + private final HiveConf conf; + + private final Driver driver; + + final String metaStoreURI; + + @Rule + public TemporaryFolder dbFolder = new TemporaryFolder(); + + + private static final Logger LOG = LoggerFactory.getLogger(HiveSink.class); + + public TestHiveSink() throws Exception { + partitionVals = new ArrayList(2); + partitionVals.add(PART1_VALUE); + partitionVals.add(PART2_VALUE); + + metaStoreURI = "null"; + + conf = new HiveConf(this.getClass()); + TestUtil.setConfValues(conf); + + // 1) prepare hive + TxnDbUtil.cleanDb(); + TxnDbUtil.prepDb(); + + // 2) Setup Hive client + SessionState.start(new CliSessionState(conf)); + driver = new Driver(conf); + + } + + + @Before + public void setUp() throws Exception { + TestUtil.dropDB(conf, dbName); + + sink = new HiveSink(); + sink.setName("HiveSink-" + UUID.randomUUID().toString()); + + String dbLocation = dbFolder.newFolder(dbName).getCanonicalPath() + ".db"; + dbLocation = dbLocation.replaceAll("\\\\","/"); // for windows paths + TestUtil.createDbAndTable(driver, dbName, tblName, partitionVals, colNames, + colTypes, partNames, dbLocation); + } + + @After + public void tearDown() throws MetaException, HiveException { + TestUtil.dropDB(conf, dbName); + } + + + @Test + public void testSingleWriterSimplePartitionedTable() + throws EventDeliveryException, IOException, CommandNeedRetryException { + int totalRecords = 4; + int batchSize = 2; + int batchCount = totalRecords / batchSize; + + Context context = new Context(); + context.put("hive.metastore", metaStoreURI); + context.put("hive.database",dbName); + context.put("hive.table",tblName); + context.put("hive.partition", PART1_VALUE + "," + PART2_VALUE); + context.put("autoCreatePartitions","false"); + context.put("batchSize","" + batchSize); + context.put("serializer", HiveDelimitedTextSerializer.ALIAS); + context.put("serializer.fieldnames", COL1 + ",," + COL2 + ","); + context.put("heartBeatInterval", "0"); + + Channel channel = startSink(sink, context); + + List bodies = Lists.newArrayList(); + + // push the events in two batches + Transaction txn = channel.getTransaction(); + txn.begin(); + for (int j = 1; j <= totalRecords; j++) { + Event event = new SimpleEvent(); + String body = j + ",blah,This is a log message,other stuff"; + event.setBody(body.getBytes()); + bodies.add(body); + channel.put(event); + } + // execute sink to process the events + txn.commit(); + txn.close(); + + + checkRecordCountInTable(0, dbName, tblName); + for (int i = 0; i < batchCount ; i++) { + sink.process(); + } + sink.stop(); + checkRecordCountInTable(totalRecords, dbName, tblName); + } + + @Test + public void testSingleWriterSimpleUnPartitionedTable() + throws Exception { + TestUtil.dropDB(conf, dbName2); + String dbLocation = dbFolder.newFolder(dbName2).getCanonicalPath() + ".db"; + dbLocation = dbLocation.replaceAll("\\\\","/"); // for windows paths + TestUtil.createDbAndTable(driver, dbName2, tblName2, null, colNames2, colTypes2 + , null, dbLocation); + + try { + int totalRecords = 4; + int batchSize = 2; + int batchCount = totalRecords / batchSize; + + Context context = new Context(); + context.put("hive.metastore", metaStoreURI); + context.put("hive.database", dbName2); + context.put("hive.table", tblName2); + context.put("autoCreatePartitions","false"); + context.put("batchSize","" + batchSize); + context.put("serializer", HiveDelimitedTextSerializer.ALIAS); + context.put("serializer.fieldnames", COL1 + ",," + COL2 + ","); + context.put("heartBeatInterval", "0"); + + Channel channel = startSink(sink, context); + + List bodies = Lists.newArrayList(); + + // Push the events in two batches + Transaction txn = channel.getTransaction(); + txn.begin(); + for (int j = 1; j <= totalRecords; j++) { + Event event = new SimpleEvent(); + String body = j + ",blah,This is a log message,other stuff"; + event.setBody(body.getBytes()); + bodies.add(body); + channel.put(event); + } + + txn.commit(); + txn.close(); + + checkRecordCountInTable(0, dbName2, tblName2); + for (int i = 0; i < batchCount ; i++) { + sink.process(); + } + + // check before & after stopping sink + checkRecordCountInTable(totalRecords, dbName2, tblName2); + sink.stop(); + checkRecordCountInTable(totalRecords, dbName2, tblName2); + } finally { + TestUtil.dropDB(conf, dbName2); + } + } + + @Test + public void testSingleWriterUseHeaders() + throws Exception { + String[] colNames = {COL1, COL2}; + String PART1_NAME = "country"; + String PART2_NAME = "hour"; + String[] partNames = {PART1_NAME, PART2_NAME}; + List partitionVals = null; + String PART1_VALUE = "%{" + PART1_NAME + "}"; + String PART2_VALUE = "%y-%m-%d-%k"; + partitionVals = new ArrayList(2); + partitionVals.add(PART1_VALUE); + partitionVals.add(PART2_VALUE); + + String tblName = "hourlydata"; + TestUtil.dropDB(conf, dbName2); + String dbLocation = dbFolder.newFolder(dbName2).getCanonicalPath() + ".db"; + dbLocation = dbLocation.replaceAll("\\\\","/"); // for windows paths + TestUtil.createDbAndTable(driver, dbName2, tblName, partitionVals, colNames, + colTypes, partNames, dbLocation); + + int totalRecords = 4; + int batchSize = 2; + int batchCount = totalRecords / batchSize; + + Context context = new Context(); + context.put("hive.metastore",metaStoreURI); + context.put("hive.database",dbName2); + context.put("hive.table",tblName); + context.put("hive.partition", PART1_VALUE + "," + PART2_VALUE); + context.put("autoCreatePartitions","true"); + context.put("useLocalTimeStamp", "false"); + context.put("batchSize","" + batchSize); + context.put("serializer", HiveDelimitedTextSerializer.ALIAS); + context.put("serializer.fieldnames", COL1 + ",," + COL2 + ","); + context.put("heartBeatInterval", "0"); + + Channel channel = startSink(sink, context); + + Calendar eventDate = Calendar.getInstance(); + List bodies = Lists.newArrayList(); + + // push events in two batches - two per batch. each batch is diff hour + Transaction txn = channel.getTransaction(); + txn.begin(); + for (int j = 1; j <= totalRecords; j++) { + Event event = new SimpleEvent(); + String body = j + ",blah,This is a log message,other stuff"; + event.setBody(body.getBytes()); + eventDate.clear(); + eventDate.set(2014, 03, 03, j%batchCount, 1); // yy mm dd hh mm + event.getHeaders().put( "timestamp", + String.valueOf(eventDate.getTimeInMillis()) ); + event.getHeaders().put( PART1_NAME, "Asia" ); + bodies.add(body); + channel.put(event); + } + // execute sink to process the events + txn.commit(); + txn.close(); + + checkRecordCountInTable(0, dbName2, tblName); + for (int i = 0; i < batchCount ; i++) { + sink.process(); + } + checkRecordCountInTable(totalRecords, dbName2, tblName); + sink.stop(); + + // verify counters + SinkCounter counter = sink.getCounter(); + Assert.assertEquals(2, counter.getConnectionCreatedCount()); + Assert.assertEquals(2, counter.getConnectionClosedCount()); + Assert.assertEquals(2, counter.getBatchCompleteCount()); + Assert.assertEquals(0, counter.getBatchEmptyCount()); + Assert.assertEquals(0, counter.getConnectionFailedCount() ); + Assert.assertEquals(4, counter.getEventDrainAttemptCount()); + Assert.assertEquals(4, counter.getEventDrainSuccessCount() ); + + } + + @Test + public void testHeartBeat() + throws EventDeliveryException, IOException, CommandNeedRetryException { + int batchSize = 2; + int batchCount = 3; + int totalRecords = batchCount*batchSize; + Context context = new Context(); + context.put("hive.metastore", metaStoreURI); + context.put("hive.database", dbName); + context.put("hive.table", tblName); + context.put("hive.partition", PART1_VALUE + "," + PART2_VALUE); + context.put("autoCreatePartitions","true"); + context.put("batchSize","" + batchSize); + context.put("serializer", HiveDelimitedTextSerializer.ALIAS); + context.put("serializer.fieldnames", COL1 + ",," + COL2 + ","); + context.put("hive.txnsPerBatchAsk", "20"); + context.put("heartBeatInterval", "3"); // heartbeat in seconds + + Channel channel = startSink(sink, context); + + List bodies = Lists.newArrayList(); + + // push the events in two batches + for (int i = 0; i < batchCount; i++) { + Transaction txn = channel.getTransaction(); + txn.begin(); + for (int j = 1; j <= batchSize; j++) { + Event event = new SimpleEvent(); + String body = i*j + ",blah,This is a log message,other stuff"; + event.setBody(body.getBytes()); + bodies.add(body); + channel.put(event); + } + // execute sink to process the events + txn.commit(); + txn.close(); + + sink.process(); + sleep(3000); // allow heartbeat to happen + } + + sink.stop(); + checkRecordCountInTable(totalRecords, dbName, tblName); + } + + @Test + public void testJsonSerializer() throws Exception { + int batchSize = 2; + int batchCount = 2; + int totalRecords = batchCount*batchSize; + Context context = new Context(); + context.put("hive.metastore",metaStoreURI); + context.put("hive.database",dbName); + context.put("hive.table",tblName); + context.put("hive.partition", PART1_VALUE + "," + PART2_VALUE); + context.put("autoCreatePartitions","true"); + context.put("batchSize","" + batchSize); + context.put("serializer", HiveJsonSerializer.ALIAS); + context.put("serializer.fieldnames", COL1 + ",," + COL2 + ","); + context.put("heartBeatInterval", "0"); + + Channel channel = startSink(sink, context); + + List bodies = Lists.newArrayList(); + + // push the events in two batches + for (int i = 0; i < batchCount; i++) { + Transaction txn = channel.getTransaction(); + txn.begin(); + for (int j = 1; j <= batchSize; j++) { + Event event = new SimpleEvent(); + String body = "{\"id\" : 1, \"msg\" : \"using json serializer\"}"; + event.setBody(body.getBytes()); + bodies.add(body); + channel.put(event); + } + // execute sink to process the events + txn.commit(); + txn.close(); + + sink.process(); + } + checkRecordCountInTable(totalRecords, dbName, tblName); + sink.stop(); + checkRecordCountInTable(totalRecords, dbName, tblName); + } + + private void sleep(int n) { + try { + Thread.sleep(n); + } catch (InterruptedException e) { + } + } + + private static Channel startSink(HiveSink sink, Context context) { + Configurables.configure(sink, context); + + Channel channel = new MemoryChannel(); + Configurables.configure(channel, context); + sink.setChannel(channel); + sink.start(); + return channel; + } + + private void checkRecordCountInTable(int expectedCount, String db, String tbl) + throws CommandNeedRetryException, IOException { + int count = TestUtil.listRecordsInTable(driver, db, tbl).size(); + Assert.assertEquals(expectedCount, count); + } +} diff --git a/flume-ng-sinks/flume-hive-sink/src/test/java/org/apache/flume/sink/hive/TestHiveWriter.java b/flume-ng-sinks/flume-hive-sink/src/test/java/org/apache/flume/sink/hive/TestHiveWriter.java new file mode 100644 index 0000000000..174f179ff0 --- /dev/null +++ b/flume-ng-sinks/flume-hive-sink/src/test/java/org/apache/flume/sink/hive/TestHiveWriter.java @@ -0,0 +1,321 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, + * software distributed under the License is distributed on an + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + * KIND, either express or implied. See the License for the + * specific language governing permissions and limitations + * under the License. + */ + +package org.apache.flume.sink.hive; + +import com.google.common.util.concurrent.ThreadFactoryBuilder; +import junit.framework.Assert; +import org.apache.flume.Context; +import org.apache.flume.event.SimpleEvent; +import org.apache.flume.instrumentation.SinkCounter; +import org.apache.hadoop.hive.cli.CliSessionState; +import org.apache.hadoop.hive.conf.HiveConf; +import org.apache.hadoop.hive.metastore.txn.TxnDbUtil; +import org.apache.hadoop.hive.ql.CommandNeedRetryException; +import org.apache.hadoop.hive.ql.Driver; +import org.apache.hadoop.hive.ql.session.SessionState; +import org.apache.hive.hcatalog.streaming.HiveEndPoint; +import org.junit.Before; +import org.junit.Rule; +import org.junit.Test; +import org.junit.rules.TemporaryFolder; + +import java.io.IOException; +import java.util.ArrayList; +import java.util.concurrent.ExecutorService; +import java.util.concurrent.Executors; + +public class TestHiveWriter { + final static String dbName = "testing"; + final static String tblName = "alerts"; + + public static final String PART1_NAME = "continent"; + public static final String PART2_NAME = "country"; + public static final String[] partNames = { PART1_NAME, PART2_NAME }; + + private static final String COL1 = "id"; + private static final String COL2 = "msg"; + final String[] colNames = {COL1,COL2}; + private String[] colTypes = { "int", "string" }; + + private static final String PART1_VALUE = "Asia"; + private static final String PART2_VALUE = "India"; + private final ArrayList partVals; + + private final String metaStoreURI; + + private HiveDelimitedTextSerializer serializer; + + private final HiveConf conf; + + private ExecutorService callTimeoutPool; + int timeout = 10000; // msec + + @Rule + public TemporaryFolder dbFolder = new TemporaryFolder(); + + private final Driver driver; + + public TestHiveWriter() throws Exception { + partVals = new ArrayList(2); + partVals.add(PART1_VALUE); + partVals.add(PART2_VALUE); + + metaStoreURI = null; + + int callTimeoutPoolSize = 1; + callTimeoutPool = Executors.newFixedThreadPool(callTimeoutPoolSize, + new ThreadFactoryBuilder().setNameFormat("hiveWriterTest").build()); + + // 1) Start metastore + conf = new HiveConf(this.getClass()); + TestUtil.setConfValues(conf); + if (metaStoreURI != null) { + conf.setVar(HiveConf.ConfVars.METASTOREURIS, metaStoreURI); + } + + // 2) Setup Hive client + SessionState.start(new CliSessionState(conf)); + driver = new Driver(conf); + + } + + @Before + public void setUp() throws Exception { + // 1) prepare hive + TxnDbUtil.cleanDb(); + TxnDbUtil.prepDb(); + + // 1) Setup tables + TestUtil.dropDB(conf, dbName); + String dbLocation = dbFolder.newFolder(dbName).getCanonicalPath() + ".db"; + dbLocation = dbLocation.replaceAll("\\\\","/"); // for windows paths + TestUtil.createDbAndTable(driver, dbName, tblName, partVals, colNames, colTypes + , partNames, dbLocation); + + // 2) Setup serializer + Context ctx = new Context(); + ctx.put("serializer.fieldnames", COL1 + ",," + COL2 + ","); + serializer = new HiveDelimitedTextSerializer(); + serializer.configure(ctx); + } + + @Test + public void testInstantiate() throws Exception { + HiveEndPoint endPoint = new HiveEndPoint(metaStoreURI, dbName, tblName, partVals); + SinkCounter sinkCounter = new SinkCounter(this.getClass().getName()); + HiveWriter writer = new HiveWriter(endPoint, 10, true, timeout + , callTimeoutPool, "flumetest", serializer, sinkCounter); + + writer.close(); + } + + @Test + public void testWriteBasic() throws Exception { + HiveEndPoint endPoint = new HiveEndPoint(metaStoreURI, dbName, tblName, partVals); + SinkCounter sinkCounter = new SinkCounter(this.getClass().getName()); + HiveWriter writer = new HiveWriter(endPoint, 10, true, timeout + , callTimeoutPool, "flumetest", serializer, sinkCounter); + + writeEvents(writer,3); + writer.flush(false); + writer.close(); + checkRecordCountInTable(3); + } + + @Test + public void testWriteMultiFlush() throws Exception { + HiveEndPoint endPoint = new HiveEndPoint(metaStoreURI, dbName, tblName, partVals); + SinkCounter sinkCounter = new SinkCounter(this.getClass().getName()); + + HiveWriter writer = new HiveWriter(endPoint, 10, true, timeout + , callTimeoutPool, "flumetest", serializer, sinkCounter); + + checkRecordCountInTable(0); + SimpleEvent event = new SimpleEvent(); + + String REC1 = "1,xyz,Hello world,abc"; + event.setBody(REC1.getBytes()); + writer.write(event); + checkRecordCountInTable(0); + writer.flush(true); + checkRecordCountInTable(1); + + String REC2 = "2,xyz,Hello world,abc"; + event.setBody(REC2.getBytes()); + writer.write(event); + checkRecordCountInTable(1); + writer.flush(true); + checkRecordCountInTable(2); + + String REC3 = "3,xyz,Hello world,abc"; + event.setBody(REC3.getBytes()); + writer.write(event); + writer.flush(true); + checkRecordCountInTable(3); + writer.close(); + + checkRecordCountInTable(3); + } + + private void checkRecordCountInTable(int expectedCount) + throws CommandNeedRetryException, IOException { + int count = TestUtil.listRecordsInTable(driver, dbName, tblName).size(); + Assert.assertEquals(expectedCount, count); + } + + /** + * Sets up input fields to have same order as table columns, + * Also sets the separator on serde to be same as i/p field separator + * @throws Exception + */ + @Test + public void testInOrderWrite() throws Exception { + HiveEndPoint endPoint = new HiveEndPoint(metaStoreURI, dbName, tblName, partVals); + SinkCounter sinkCounter = new SinkCounter(this.getClass().getName()); + int timeout = 5000; // msec + + HiveDelimitedTextSerializer serializer2 = new HiveDelimitedTextSerializer(); + Context ctx = new Context(); + ctx.put("serializer.fieldnames", COL1 + "," + COL2); + ctx.put("serializer.serdeSeparator", ","); + serializer2.configure(ctx); + + + HiveWriter writer = new HiveWriter(endPoint, 10, true, timeout, callTimeoutPool, + "flumetest", serializer2, sinkCounter); + + SimpleEvent event = new SimpleEvent(); + event.setBody("1,Hello world 1".getBytes()); + writer.write(event); + event.setBody("2,Hello world 2".getBytes()); + writer.write(event); + event.setBody("3,Hello world 3".getBytes()); + writer.write(event); + writer.flush(false); + writer.close(); + } + + @Test + public void testSerdeSeparatorCharParsing() throws Exception { + HiveEndPoint endPoint = new HiveEndPoint(metaStoreURI, dbName, tblName, partVals); + SinkCounter sinkCounter = new SinkCounter(this.getClass().getName()); + int timeout = 10000; // msec + + // 1) single character serdeSeparator + HiveDelimitedTextSerializer serializer1 = new HiveDelimitedTextSerializer(); + Context ctx = new Context(); + ctx.put("serializer.fieldnames", COL1 + "," + COL2); + ctx.put("serializer.serdeSeparator", ","); + serializer1.configure(ctx); + // show not throw + + + // 2) special character as serdeSeparator + HiveDelimitedTextSerializer serializer2 = new HiveDelimitedTextSerializer(); + ctx = new Context(); + ctx.put("serializer.fieldnames", COL1 + "," + COL2); + ctx.put("serializer.serdeSeparator", "'\t'"); + serializer2.configure(ctx); + // show not throw + + + // 2) bad spec as serdeSeparator + HiveDelimitedTextSerializer serializer3 = new HiveDelimitedTextSerializer(); + ctx = new Context(); + ctx.put("serializer.fieldnames", COL1 + "," + COL2); + ctx.put("serializer.serdeSeparator", "ab"); + try { + serializer3.configure(ctx); + Assert.assertTrue("Bad serdeSeparator character was accepted" ,false); + } catch (Exception e){ + // expect an exception + } + + } + + + @Test + public void testSecondWriterBeforeFirstCommits() throws Exception { + // here we open a new writer while the first is still writing (not committed) + HiveEndPoint endPoint1 = new HiveEndPoint(metaStoreURI, dbName, tblName, partVals); + ArrayList partVals2 = new ArrayList(2); + partVals2.add(PART1_VALUE); + partVals2.add("Nepal"); + HiveEndPoint endPoint2 = new HiveEndPoint(metaStoreURI, dbName, tblName, partVals2); + + SinkCounter sinkCounter1 = new SinkCounter(this.getClass().getName()); + SinkCounter sinkCounter2 = new SinkCounter(this.getClass().getName()); + + HiveWriter writer1 = new HiveWriter(endPoint1, 10, true, timeout + , callTimeoutPool, "flumetest", serializer, sinkCounter1); + + writeEvents(writer1, 3); + + HiveWriter writer2 = new HiveWriter(endPoint2, 10, true, timeout + , callTimeoutPool, "flumetest", serializer, sinkCounter2); + writeEvents(writer2, 3); + writer2.flush(false); // commit + + writer1.flush(false); // commit + writer1.close(); + + writer2.close(); + } + + + @Test + public void testSecondWriterAfterFirstCommits() throws Exception { + // here we open a new writer after the first writer has committed one txn + HiveEndPoint endPoint1 = new HiveEndPoint(metaStoreURI, dbName, tblName, partVals); + ArrayList partVals2 = new ArrayList(2); + partVals2.add(PART1_VALUE); + partVals2.add("Nepal"); + HiveEndPoint endPoint2 = new HiveEndPoint(metaStoreURI, dbName, tblName, partVals2); + + SinkCounter sinkCounter1 = new SinkCounter(this.getClass().getName()); + SinkCounter sinkCounter2 = new SinkCounter(this.getClass().getName()); + + HiveWriter writer1 = new HiveWriter(endPoint1, 10, true, timeout + , callTimeoutPool, "flumetest", serializer, sinkCounter1); + + writeEvents(writer1, 3); + + writer1.flush(false); // commit + + + HiveWriter writer2 = new HiveWriter(endPoint2, 10, true, timeout + , callTimeoutPool, "flumetest", serializer, sinkCounter2); + writeEvents(writer2, 3); + writer2.flush(false); // commit + + + writer1.close(); + writer2.close(); + } + + + private void writeEvents(HiveWriter writer, int count) throws InterruptedException, HiveWriter.WriteException { + SimpleEvent event = new SimpleEvent(); + for (int i = 1; i <= count; i++) { + event.setBody((i + ",xyz,Hello world,abc").getBytes()); + writer.write(event); + } + } +} diff --git a/flume-ng-sinks/flume-hive-sink/src/test/java/org/apache/flume/sink/hive/TestUtil.java b/flume-ng-sinks/flume-hive-sink/src/test/java/org/apache/flume/sink/hive/TestUtil.java new file mode 100644 index 0000000000..1fd60bc5be --- /dev/null +++ b/flume-ng-sinks/flume-hive-sink/src/test/java/org/apache/flume/sink/hive/TestUtil.java @@ -0,0 +1,228 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, + * software distributed under the License is distributed on an + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + * KIND, either express or implied. See the License for the + * specific language governing permissions and limitations + * under the License. + */ + + +package org.apache.flume.sink.hive; + +import org.apache.hadoop.fs.FileStatus; +import org.apache.hadoop.fs.Path; +import org.apache.hadoop.fs.RawLocalFileSystem; +import org.apache.hadoop.fs.permission.FsPermission; +import org.apache.hadoop.hive.conf.HiveConf; +import org.apache.hadoop.hive.metastore.HiveMetaStoreClient; +import org.apache.hadoop.hive.metastore.IMetaStoreClient; +import org.apache.hadoop.hive.metastore.api.MetaException; +import org.apache.hadoop.hive.ql.CommandNeedRetryException; +import org.apache.hadoop.hive.ql.Driver; +import org.apache.hadoop.hive.ql.metadata.HiveException; +import org.apache.hadoop.hive.shims.ShimLoader; +import org.apache.hadoop.util.Shell; +import org.apache.hive.hcatalog.streaming.QueryFailedException; +import org.apache.thrift.TException; + +import java.io.File; +import java.io.FileNotFoundException; +import java.io.IOException; +import java.net.URI; +import java.net.URISyntaxException; +import java.util.ArrayList; +import java.util.List; + +public class TestUtil { + + private final static String txnMgr = "org.apache.hadoop.hive.ql.lockmgr.DbTxnManager"; + + /** + * Set up the configuration so it will use the DbTxnManager, concurrency will be set to true, + * and the JDBC configs will be set for putting the transaction and lock info in the embedded + * metastore. + * @param conf HiveConf to add these values to. + */ + public static void setConfValues(HiveConf conf) { + conf.setVar(HiveConf.ConfVars.HIVE_TXN_MANAGER, txnMgr); + conf.setBoolVar(HiveConf.ConfVars.HIVE_SUPPORT_CONCURRENCY, true); + conf.set("fs.raw.impl", RawFileSystem.class.getName()); + } + + public static void createDbAndTable(Driver driver, String databaseName, + String tableName, List partVals, + String[] colNames, String[] colTypes, + String[] partNames, String dbLocation) + throws Exception { + String dbUri = "raw://" + dbLocation; + String tableLoc = dbUri + Path.SEPARATOR + tableName; + + runDDL(driver, "create database IF NOT EXISTS " + databaseName + " location '" + dbUri + "'"); + runDDL(driver, "use " + databaseName); + String crtTbl = "create table " + tableName + + " ( " + getTableColumnsStr(colNames,colTypes) + " )" + + getPartitionStmtStr(partNames) + + " clustered by ( " + colNames[0] + " )" + + " into 10 buckets " + + " stored as orc " + + " location '" + tableLoc + "'"; + runDDL(driver, crtTbl); + System.out.println("crtTbl = " + crtTbl); + if (partNames!=null && partNames.length!=0) { + String addPart = "alter table " + tableName + " add partition ( " + + getTablePartsStr2(partNames, partVals) + " )"; + runDDL(driver, addPart); + } + } + + private static String getPartitionStmtStr(String[] partNames) { + if ( partNames == null || partNames.length == 0) { + return ""; + } + return " partitioned by (" + getTablePartsStr(partNames) + " )"; + } + + // delete db and all tables in it + public static void dropDB(HiveConf conf, String databaseName) throws HiveException, MetaException { + IMetaStoreClient client = new HiveMetaStoreClient(conf); + try { + for (String table : client.listTableNamesByFilter(databaseName, "", (short)-1)) { + client.dropTable(databaseName, table, true, true); + } + client.dropDatabase(databaseName); + } catch (TException e) { + client.close(); + } + } + + private static String getTableColumnsStr(String[] colNames, String[] colTypes) { + StringBuffer sb = new StringBuffer(); + for (int i=0; i < colNames.length; ++i) { + sb.append(colNames[i] + " " + colTypes[i]); + if (i partVals) { + StringBuffer sb = new StringBuffer(); + for (int i=0; i < partVals.size(); ++i) { + sb.append(partNames[i] + " = '" + partVals.get(i) + "'"); + if (i < partVals.size()-1) { + sb.append(","); + } + } + return sb.toString(); + } + + public static ArrayList listRecordsInTable(Driver driver, String dbName, String tblName) + throws CommandNeedRetryException, IOException { + driver.run("select * from " + dbName + "." + tblName); + ArrayList res = new ArrayList(); + driver.getResults(res); + return res; + } + + public static ArrayList listRecordsInPartition(Driver driver, String dbName, + String tblName, String continent, String country) + throws CommandNeedRetryException, IOException { + driver.run("select * from " + dbName + "." + tblName + " where continent='" + + continent + "' and country='" + country + "'"); + ArrayList res = new ArrayList(); + driver.getResults(res); + return res; + } + + + public static class RawFileSystem extends RawLocalFileSystem { + private static final URI NAME; + static { + try { + NAME = new URI("raw:///"); + } catch (URISyntaxException se) { + throw new IllegalArgumentException("bad uri", se); + } + } + + @Override + public URI getUri() { + return NAME; + } + + static String execCommand(File f, String... cmd) throws IOException { + String[] args = new String[cmd.length + 1]; + System.arraycopy(cmd, 0, args, 0, cmd.length); + args[cmd.length] = f.getCanonicalPath(); + String output = Shell.execCommand(args); + return output; + } + + @Override + public FileStatus getFileStatus(Path path) throws IOException { + File file = pathToFile(path); + if (!file.exists()) { + throw new FileNotFoundException("Can't find " + path); + } + // get close enough + short mod = 0; + if (file.canRead()) { + mod |= 0444; + } + if (file.canWrite()) { + mod |= 0200; + } + if (file.canExecute()) { + mod |= 0111; + } + ShimLoader.getHadoopShims(); + return new FileStatus(file.length(), file.isDirectory(), 1, 1024, + file.lastModified(), file.lastModified(), + FsPermission.createImmutable(mod), "owen", "users", path); + } + } + private static boolean runDDL(Driver driver, String sql) throws QueryFailedException { + int retryCount = 1; // # of times to retry if first attempt fails + for (int attempt=0; attempt <= retryCount; ++attempt) { + try { + driver.run(sql); + return true; + } catch (CommandNeedRetryException e) { + if (attempt == retryCount) { + throw new QueryFailedException(sql, e); + } + continue; + } + } // for + return false; + } + +} diff --git a/flume-ng-sinks/flume-hive-sink/src/test/resources/log4j.properties b/flume-ng-sinks/flume-hive-sink/src/test/resources/log4j.properties new file mode 100644 index 0000000000..252b5ea193 --- /dev/null +++ b/flume-ng-sinks/flume-hive-sink/src/test/resources/log4j.properties @@ -0,0 +1,26 @@ +# Licensed to the Apache Software Foundation (ASF) under one +# or more contributor license agreements. See the NOTICE file +# distributed with this work for additional information +# regarding copyright ownership. The ASF licenses this file +# to you under the Apache License, Version 2.0 (the +# "License"); you may not use this file except in compliance +# with the License. You may obtain a copy of the License at +# +# http://www.apache.org/licenses/LICENSE-2.0 +# +# Unless required by applicable law or agreed to in writing, +# software distributed under the License is distributed on an +# "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY +# KIND, either express or implied. See the License for the +# specific language governing permissions and limitations +# under the License. + +log4j.rootLogger = INFO, out + +log4j.appender.out = org.apache.log4j.ConsoleAppender +log4j.appender.out.layout = org.apache.log4j.PatternLayout +log4j.appender.out.layout.ConversionPattern = %d (%t) [%p - %l] %m%n + +log4j.logger.org.apache.flume = DEBUG +log4j.logger.org.apache.hadoop = WARN +log4j.logger.org.mortbay = WARN diff --git a/flume-ng-sinks/pom.xml b/flume-ng-sinks/pom.xml index 4bac01916b..de12891e4a 100644 --- a/flume-ng-sinks/pom.xml +++ b/flume-ng-sinks/pom.xml @@ -72,6 +72,7 @@ limitations under the License. --> flume-dataset-sink + flume-hive-sink @@ -87,6 +88,7 @@ limitations under the License. --> flume-dataset-sink + flume-hive-sink diff --git a/flume-ng-sources/flume-scribe-source/src/main/java/org/apache/flume/source/scribe/Scribe.java b/flume-ng-sources/flume-scribe-source/src/main/java/org/apache/flume/source/scribe/Scribe.java index eba8d2edea..3b5b3c7d10 100644 --- a/flume-ng-sources/flume-scribe-source/src/main/java/org/apache/flume/source/scribe/Scribe.java +++ b/flume-ng-sources/flume-scribe-source/src/main/java/org/apache/flume/source/scribe/Scribe.java @@ -168,15 +168,20 @@ public Log() { super("Log"); } - protected Log_args getEmptyArgsInstance() { + public Log_args getEmptyArgsInstance() { return new Log_args(); } - protected Log_result getResult(I iface, Log_args args) throws org.apache.thrift.TException { + public Log_result getResult(I iface, Log_args args) throws org.apache + .thrift.TException { Log_result result = new Log_result(); result.success = iface.Log(args.messages); return result; } + + public boolean isOneway() { + return false; + } } } diff --git a/pom.xml b/pom.xml index 1350fa43ee..ea7ffe35d1 100644 --- a/pom.xml +++ b/pom.xml @@ -48,11 +48,12 @@ limitations under the License. 1.7.4 0.90.1 - 2.4.0 0.7.0 0.17.1 - 0.10.0 + 0.13.1 + 2.7.1 + 2.9.1 @@ -81,7 +82,7 @@ limitations under the License. - 1.0.1 + 1.2.1 0.92.1 hadoop-core 0.7.0 @@ -133,7 +134,7 @@ limitations under the License. ${hadoop2.version} 0.94.2 hadoop-common - 0.8.0 + 0.9.0 @@ -147,6 +148,11 @@ limitations under the License. hadoop-hdfs ${hadoop.version} + + org.apache.hadoop + hadoop-mapreduce-client-core + ${hadoop.version} + org.apache.hadoop hadoop-minicluster @@ -211,7 +217,7 @@ limitations under the License. ${hadoop2.version} 0.98.2-hadoop2 hadoop-common - 0.8.0 + 0.9.0 @@ -736,6 +742,8 @@ limitations under the License. **/.classpath **/.project **/target/** + **/derby.log + **/metastore_db/ @@ -971,6 +979,7 @@ limitations under the License. org.apache.hadoop hadoop-common ${hadoop.version} + true @@ -1026,6 +1035,15 @@ limitations under the License. joda-time 2.1 + + + + org.apache.zookeeper + zookeeper + ${zookeeper.version} + org.apache.hadoop @@ -1121,6 +1139,12 @@ limitations under the License. 1.6.0-SNAPSHOT + + org.apache.flume.flume-ng-sinks + flume-hive-sink + 1.6.0-SNAPSHOT + + org.apache.flume.flume-ng-sinks flume-irc-sink @@ -1331,6 +1355,18 @@ limitations under the License. 1.1.0 + + org.apache.hive.hcatalog + hive-hcatalog-streaming + ${hive.version} + + + + org.apache.hive + hive-cli + ${hive.version} + + org.apache.curator From 6f6f69b8b3435385b578af4bfb7790f2984bfb35 Mon Sep 17 00:00:00 2001 From: Hari Shreedharan Date: Wed, 11 Feb 2015 18:03:52 -0800 Subject: [PATCH 201/341] FLUME-2592. Fix help message in File Channel Integrity Tool. (Ashish Paliwal via Hari) --- .../java/org/apache/flume/tools/FileChannelIntegrityTool.java | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/flume-tools/src/main/java/org/apache/flume/tools/FileChannelIntegrityTool.java b/flume-tools/src/main/java/org/apache/flume/tools/FileChannelIntegrityTool.java index d0753a65a4..1030442ab2 100644 --- a/flume-tools/src/main/java/org/apache/flume/tools/FileChannelIntegrityTool.java +++ b/flume-tools/src/main/java/org/apache/flume/tools/FileChannelIntegrityTool.java @@ -118,12 +118,12 @@ private boolean parseCommandLineOpts(String[] args) throws ParseException { CommandLineParser parser = new GnuParser(); CommandLine commandLine = parser.parse(options, args); if(commandLine.hasOption("help")) { - new HelpFormatter().printHelp("java -jar fcintegritytool ", + new HelpFormatter().printHelp("bin/flume-ng tool fcintegritytool ", options, true); return false; } if(!commandLine.hasOption("dataDirs")) { - new HelpFormatter().printHelp("java -jar fcintegritytool ", "", + new HelpFormatter().printHelp("bin/flume-ng tool fcintegritytool ", "", options, "dataDirs is required.", true); return false; } else { From a103a677145a43aa6fa78dfeeb34018879e24a94 Mon Sep 17 00:00:00 2001 From: Hari Shreedharan Date: Wed, 18 Feb 2015 21:10:56 -0800 Subject: [PATCH 202/341] FLUME-2574. SSL support for Thrift RPC. (Johny Rufus via Hari) --- .../org/apache/flume/source/ThriftSource.java | 274 ++++++++++++------ .../org/apache/flume/sink/TestThriftSink.java | 139 ++++++++- .../apache/flume/source/TestThriftSource.java | 37 +++ .../src/test/resources/keystorefile.jks | Bin 0 -> 1294 bytes .../src/test/resources/truststorefile.jks | Bin 0 -> 887 bytes .../org/apache/flume/api/ThriftRpcClient.java | 113 +++++++- .../apache/flume/api/ThriftTestingSource.java | 63 +++- 7 files changed, 527 insertions(+), 99 deletions(-) create mode 100644 flume-ng-core/src/test/resources/keystorefile.jks create mode 100644 flume-ng-core/src/test/resources/truststorefile.jks diff --git a/flume-ng-core/src/main/java/org/apache/flume/source/ThriftSource.java b/flume-ng-core/src/main/java/org/apache/flume/source/ThriftSource.java index 551fe133c7..06bb604a2f 100644 --- a/flume-ng-core/src/main/java/org/apache/flume/source/ThriftSource.java +++ b/flume-ng-core/src/main/java/org/apache/flume/source/ThriftSource.java @@ -35,18 +35,30 @@ import org.apache.thrift.TException; import org.apache.thrift.protocol.TCompactProtocol; import org.apache.thrift.protocol.TBinaryProtocol; +import org.apache.thrift.protocol.TProtocolFactory; import org.apache.thrift.server.TNonblockingServer; import org.apache.thrift.server.TServer; +import org.apache.thrift.server.TThreadPoolServer; import org.apache.thrift.transport.TFastFramedTransport; import org.apache.thrift.transport.TNonblockingServerSocket; import org.apache.thrift.transport.TNonblockingServerTransport; import org.apache.thrift.transport.TServerSocket; import org.apache.thrift.transport.TServerTransport; +import org.apache.thrift.transport.TSSLTransportFactory; import org.slf4j.Logger; import org.slf4j.LoggerFactory; +import javax.net.ssl.KeyManagerFactory; +import javax.net.ssl.SSLServerSocket; +import java.io.FileInputStream; import java.lang.reflect.Method; +import java.net.InetAddress; import java.net.InetSocketAddress; +import java.net.ServerSocket; +import java.security.KeyStore; +import java.util.ArrayList; +import java.util.Arrays; +import java.util.LinkedList; import java.util.List; import java.util.concurrent.ExecutorService; import java.util.concurrent.Executors; @@ -77,15 +89,27 @@ public class ThriftSource extends AbstractSource implements Configurable, public static final String CONFIG_PROTOCOL = "protocol"; public static final String BINARY_PROTOCOL = "binary"; public static final String COMPACT_PROTOCOL = "compact"; - + + private static final String SSL_KEY = "ssl"; + private static final String KEYSTORE_KEY = "keystore"; + private static final String KEYSTORE_PASSWORD_KEY = "keystore-password"; + private static final String KEYSTORE_TYPE_KEY = "keystore-type"; + private static final String KEYMANAGER_TYPE = "keymanager-type"; + private static final String EXCLUDE_PROTOCOLS = "exclude-protocols"; + private Integer port; private String bindAddress; private int maxThreads = 0; private SourceCounter sourceCounter; private TServer server; - private TServerTransport serverTransport; private ExecutorService servingExecutor; private String protocol; + private String keystore; + private String keystorePassword; + private String keystoreType; + private String keyManagerType; + private final List excludeProtocols = new LinkedList(); + private boolean enableSsl = false; @Override public void configure(Context context) { @@ -99,6 +123,7 @@ public void configure(Context context) { try { maxThreads = context.getInteger(CONFIG_THREADS, 0); + maxThreads = (maxThreads <= 0) ? Integer.MAX_VALUE : maxThreads; } catch (NumberFormatException e) { logger.warn("Thrift source\'s \"threads\" property must specify an " + "integer value: " + context.getString(CONFIG_THREADS)); @@ -107,111 +132,58 @@ public void configure(Context context) { if (sourceCounter == null) { sourceCounter = new SourceCounter(getName()); } - + protocol = context.getString(CONFIG_PROTOCOL); if (protocol == null) { // default is to use the compact protocol. protocol = COMPACT_PROTOCOL; - } + } Preconditions.checkArgument( (protocol.equalsIgnoreCase(BINARY_PROTOCOL) || - protocol.equalsIgnoreCase(COMPACT_PROTOCOL)), + protocol.equalsIgnoreCase(COMPACT_PROTOCOL)), "binary or compact are the only valid Thrift protocol types to " + - "choose from."); - } - - @Override - public void start() { - logger.info("Starting thrift source"); - - maxThreads = (maxThreads <= 0) ? Integer.MAX_VALUE : maxThreads; - Class serverClass = null; - Class argsClass = null; - TServer.AbstractServerArgs args = null; - /* - * Use reflection to determine if TThreadedSelectServer is available. If - * it is not available, use TThreadPoolServer - */ - try { - serverClass = Class.forName("org.apache.thrift" + - ".server.TThreadedSelectorServer"); + "choose from."); - argsClass = Class.forName("org.apache.thrift" + - ".server.TThreadedSelectorServer$Args"); - - // Looks like TThreadedSelectorServer is available, so continue.. - ExecutorService sourceService; - ThreadFactory threadFactory = new ThreadFactoryBuilder().setNameFormat( - "Flume Thrift IPC Thread %d").build(); - if (maxThreads == 0) { - sourceService = Executors.newCachedThreadPool(threadFactory); + enableSsl = context.getBoolean(SSL_KEY, false); + if (enableSsl) { + keystore = context.getString(KEYSTORE_KEY); + keystorePassword = context.getString(KEYSTORE_PASSWORD_KEY); + keystoreType = context.getString(KEYSTORE_TYPE_KEY, "JKS"); + keyManagerType = context.getString(KEYMANAGER_TYPE, KeyManagerFactory.getDefaultAlgorithm()); + String excludeProtocolsStr = context.getString(EXCLUDE_PROTOCOLS); + if (excludeProtocolsStr == null) { + excludeProtocols.add("SSLv3"); } else { - sourceService = Executors.newFixedThreadPool(maxThreads, threadFactory); + excludeProtocols.addAll(Arrays.asList(excludeProtocolsStr.split(" "))); + if (!excludeProtocols.contains("SSLv3")) { + excludeProtocols.add("SSLv3"); + } } - serverTransport = new TNonblockingServerSocket( - new InetSocketAddress(bindAddress, port)); - args = (TNonblockingServer.AbstractNonblockingServerArgs) argsClass - .getConstructor(TNonblockingServerTransport.class) - .newInstance(serverTransport); - Method m = argsClass.getDeclaredMethod("executorService", - ExecutorService.class); - m.invoke(args, sourceService); - } catch (ClassNotFoundException e) { - logger.info("TThreadedSelectorServer not found, " + - "using TThreadPoolServer"); + Preconditions.checkNotNull(keystore, + KEYSTORE_KEY + " must be specified when SSL is enabled"); + Preconditions.checkNotNull(keystorePassword, + KEYSTORE_PASSWORD_KEY + " must be specified when SSL is enabled"); try { - // Looks like TThreadedSelectorServer is not available, - // so create a TThreadPoolServer instead. - - serverTransport = new TServerSocket(new InetSocketAddress - (bindAddress, port)); - - serverClass = Class.forName("org.apache.thrift" + - ".server.TThreadPoolServer"); - argsClass = Class.forName("org.apache.thrift.server" + - ".TThreadPoolServer$Args"); - args = (TServer.AbstractServerArgs) argsClass - .getConstructor(TServerTransport.class) - .newInstance(serverTransport); - Method m = argsClass.getDeclaredMethod("maxWorkerThreads",int.class); - m.invoke(args, maxThreads); - } catch (ClassNotFoundException e1) { - throw new FlumeException("Cannot find TThreadSelectorServer or " + - "TThreadPoolServer. Please install a compatible version of thrift " + - "in the classpath", e1); - } catch (Throwable throwable) { - throw new FlumeException("Cannot start Thrift source.", throwable); + KeyStore ks = KeyStore.getInstance(keystoreType); + ks.load(new FileInputStream(keystore), keystorePassword.toCharArray()); + } catch (Exception ex) { + throw new FlumeException( + "Thrift source configured with invalid keystore: " + keystore, ex); } - } catch (Throwable throwable) { - throw new FlumeException("Cannot start Thrift source.", throwable); } + } - try { - if (protocol.equals(BINARY_PROTOCOL)) { - logger.info("Using TBinaryProtocol"); - args.protocolFactory(new TBinaryProtocol.Factory()); - } else { - logger.info("Using TCompactProtocol"); - args.protocolFactory(new TCompactProtocol.Factory()); - } - args.inputTransportFactory(new TFastFramedTransport.Factory()); - args.outputTransportFactory(new TFastFramedTransport.Factory()); - args.processor(new ThriftSourceProtocol - .Processor(new ThriftSourceHandler())); - /* - * Both THsHaServer and TThreadedSelectorServer allows us to pass in - * the executor service to use - unfortunately the "executorService" - * method does not exist in the parent abstract Args class, - * so use reflection to pass the executor in. - * - */ + @Override + public void start() { + logger.info("Starting thrift source"); - server = (TServer) serverClass.getConstructor(argsClass).newInstance - (args); - } catch (Throwable ex) { - throw new FlumeException("Cannot start Thrift Source.", ex); - } + // create the server + server = getTThreadedSelectorServer(); + // if in ssl mode or if SelectorServer is unavailable + if (server == null) { + server = getTThreadPoolServer(); + } servingExecutor = Executors.newSingleThreadExecutor(new ThreadFactoryBuilder().setNameFormat("Flume Thrift Source I/O Boss") @@ -245,6 +217,126 @@ public void run() { super.start(); } + private TServerTransport getSSLServerTransport() { + try { + TServerTransport transport; + TSSLTransportFactory.TSSLTransportParameters params = + new TSSLTransportFactory.TSSLTransportParameters(); + params.setKeyStore(keystore, keystorePassword, keyManagerType, keystoreType); + transport = TSSLTransportFactory.getServerSocket( + port, 120000, InetAddress.getByName(bindAddress), params); + + ServerSocket serverSock = ((TServerSocket) transport).getServerSocket(); + if (serverSock instanceof SSLServerSocket) { + SSLServerSocket sslServerSock = (SSLServerSocket) serverSock; + List enabledProtocols = new ArrayList(); + for (String protocol : sslServerSock.getEnabledProtocols()) { + if (!excludeProtocols.contains(protocol)) { + enabledProtocols.add(protocol); + } + } + sslServerSock.setEnabledProtocols(enabledProtocols.toArray(new String[0])); + } + return transport; + } catch (Throwable throwable) { + throw new FlumeException("Cannot start Thrift source.", throwable); + } + } + + private TServerTransport getTServerTransport() { + try { + return new TServerSocket(new InetSocketAddress + (bindAddress, port)); + } catch (Throwable throwable) { + throw new FlumeException("Cannot start Thrift source.", throwable); + } + } + + private TProtocolFactory getProtocolFactory() { + if (protocol.equals(BINARY_PROTOCOL)) { + logger.info("Using TBinaryProtocol"); + return new TBinaryProtocol.Factory(); + } else { + logger.info("Using TCompactProtocol"); + return new TCompactProtocol.Factory(); + } + } + + private TServer getTThreadedSelectorServer() { + if(enableSsl) { + return null; + } + Class serverClass; + Class argsClass; + TServer.AbstractServerArgs args; + try { + serverClass = Class.forName("org.apache.thrift" + + ".server.TThreadedSelectorServer"); + argsClass = Class.forName("org.apache.thrift" + + ".server.TThreadedSelectorServer$Args"); + + TServerTransport serverTransport = new TNonblockingServerSocket( + new InetSocketAddress(bindAddress, port)); + ExecutorService sourceService; + ThreadFactory threadFactory = new ThreadFactoryBuilder().setNameFormat( + "Flume Thrift IPC Thread %d").build(); + if (maxThreads == 0) { + sourceService = Executors.newCachedThreadPool(threadFactory); + } else { + sourceService = Executors.newFixedThreadPool(maxThreads, threadFactory); + } + args = (TNonblockingServer.AbstractNonblockingServerArgs) argsClass + .getConstructor(TNonblockingServerTransport.class) + .newInstance(serverTransport); + Method m = argsClass.getDeclaredMethod("executorService", + ExecutorService.class); + m.invoke(args, sourceService); + + populateServerParams(args); + + /* + * Both THsHaServer and TThreadedSelectorServer allows us to pass in + * the executor service to use - unfortunately the "executorService" + * method does not exist in the parent abstract Args class, + * so use reflection to pass the executor in. + * + */ + server = (TServer) serverClass.getConstructor(argsClass).newInstance(args); + } catch(ClassNotFoundException e) { + return null; + } catch (Throwable ex) { + throw new FlumeException("Cannot start Thrift Source.", ex); + } + return server; + } + + private TServer getTThreadPoolServer() { + TServerTransport serverTransport; + if (enableSsl) { + serverTransport = getSSLServerTransport(); + } else { + serverTransport = getTServerTransport(); + } + TThreadPoolServer.Args serverArgs = new TThreadPoolServer.Args(serverTransport); + serverArgs.maxWorkerThreads(maxThreads); + populateServerParams(serverArgs); + return new TThreadPoolServer(serverArgs); + } + + private void populateServerParams(TServer.AbstractServerArgs args) { + //populate the ProtocolFactory + args.protocolFactory(getProtocolFactory()); + + //populate the transportFactory + args.inputTransportFactory(new TFastFramedTransport.Factory()); + args.outputTransportFactory(new TFastFramedTransport.Factory()); + + // populate the Processor + args.processor(new ThriftSourceProtocol + .Processor(new ThriftSourceHandler())); + } + + @Override public void stop() { if(server != null && server.isServing()) { server.stop(); diff --git a/flume-ng-core/src/test/java/org/apache/flume/sink/TestThriftSink.java b/flume-ng-core/src/test/java/org/apache/flume/sink/TestThriftSink.java index fccaede5cf..1beec76e85 100644 --- a/flume-ng-core/src/test/java/org/apache/flume/sink/TestThriftSink.java +++ b/flume-ng-core/src/test/java/org/apache/flume/sink/TestThriftSink.java @@ -30,12 +30,16 @@ import org.apache.flume.channel.MemoryChannel; import org.apache.flume.conf.Configurables; import org.apache.flume.event.EventBuilder; -import org.apache.flume.source.ThriftSource; +import org.apache.flume.lifecycle.LifecycleController; +import org.apache.flume.lifecycle.LifecycleState; + import org.junit.After; import org.junit.Assert; import org.junit.Before; import org.junit.Test; +import javax.net.ssl.KeyManagerFactory; +import javax.net.ssl.TrustManagerFactory; import java.nio.charset.Charset; import java.util.Random; import java.util.concurrent.atomic.AtomicLong; @@ -195,4 +199,137 @@ public void testFailedConnect() throws Exception { Assert.assertEquals(Sink.Status.BACKOFF, sink.process()); sink.stop(); } + + @Test + public void testSslProcess() throws Exception { + Event event = EventBuilder.withBody("test event 1", Charsets.UTF_8); + src = new ThriftTestingSource(ThriftTestingSource.HandlerType.OK.name(), port, + ThriftRpcClient.COMPACT_PROTOCOL, "src/test/resources/keystorefile.jks", + "password", KeyManagerFactory.getDefaultAlgorithm(), "JKS"); + Context context = new Context(); + context.put("hostname", hostname); + context.put("port", String.valueOf(port)); + context.put("ssl", String.valueOf(true)); + context.put("batch-size", String.valueOf(2)); + context.put("connect-timeout", String.valueOf(2000L)); + context.put("request-timeout", String.valueOf(3000L)); + context.put("truststore", "src/test/resources/truststorefile.jks"); + context.put("truststore-password", "password"); + context.put("trustmanager-type", TrustManagerFactory.getDefaultAlgorithm()); + + Configurables.configure(sink, context); + channel.start(); + sink.start(); + Transaction transaction = channel.getTransaction(); + transaction.begin(); + for (int i = 0; i < 11; i++) { + channel.put(event); + } + transaction.commit(); + transaction.close(); + for (int i = 0; i < 6; i++) { + Sink.Status status = sink.process(); + Assert.assertEquals(Sink.Status.READY, status); + } + Assert.assertEquals(Sink.Status.BACKOFF, sink.process()); + + sink.stop(); + Assert.assertEquals(11, src.flumeEvents.size()); + Assert.assertEquals(6, src.batchCount); + Assert.assertEquals(0, src.individualCount); + } + + @Test + public void testSslSinkWithNonSslServer() throws Exception { + Event event = EventBuilder.withBody("test event 1", Charsets.UTF_8); + src = new ThriftTestingSource(ThriftTestingSource.HandlerType.OK.name(), + port, ThriftRpcClient.COMPACT_PROTOCOL); + + Context context = new Context(); + context.put("hostname", hostname); + context.put("port", String.valueOf(port)); + context.put("ssl", String.valueOf(true)); + context.put("batch-size", String.valueOf(2)); + context.put("connect-timeout", String.valueOf(2000L)); + context.put("request-timeout", String.valueOf(3000L)); + context.put("truststore", "src/test/resources/truststorefile.jks"); + context.put("truststore-password", "password"); + context.put("trustmanager-type", TrustManagerFactory.getDefaultAlgorithm()); + + Configurables.configure(sink, context); + channel.start(); + sink.start(); + Assert.assertTrue(LifecycleController.waitForOneOf(sink, + LifecycleState.START_OR_ERROR, 5000)); + Transaction transaction = channel.getTransaction(); + transaction.begin(); + for (int i = 0; i < 11; i++) { + channel.put(event); + } + transaction.commit(); + transaction.close(); + + boolean failed = false; + try { + for (int i = 0; i < 6; i++) { + Sink.Status status = sink.process(); + failed = true; + } + } catch (EventDeliveryException ex) { + // This is correct + } + + sink.stop(); + Assert.assertTrue(LifecycleController.waitForOneOf(sink, + LifecycleState.STOP_OR_ERROR, 5000)); + if (failed) { + Assert.fail("SSL-enabled sink successfully connected to a non-SSL-enabled server, that's wrong."); + } + } + + @Test + public void testSslSinkWithNonTrustedCert() throws Exception { + Event event = EventBuilder.withBody("test event 1", Charsets.UTF_8); + src = new ThriftTestingSource(ThriftTestingSource.HandlerType.OK.name(), port, + ThriftRpcClient.COMPACT_PROTOCOL, "src/test/resources/keystorefile.jks", + "password", KeyManagerFactory.getDefaultAlgorithm(), "JKS"); + + Context context = new Context(); + context.put("hostname", hostname); + context.put("port", String.valueOf(port)); + context.put("ssl", String.valueOf(true)); + context.put("batch-size", String.valueOf(2)); + context.put("connect-timeout", String.valueOf(2000L)); + context.put("request-timeout", String.valueOf(3000L)); + + Configurables.configure(sink, context); + channel.start(); + sink.start(); + Assert.assertTrue(LifecycleController.waitForOneOf(sink, + LifecycleState.START_OR_ERROR, 5000)); + Transaction transaction = channel.getTransaction(); + transaction.begin(); + for (int i = 0; i < 11; i++) { + channel.put(event); + } + transaction.commit(); + transaction.close(); + + boolean failed = false; + try { + for (int i = 0; i < 6; i++) { + Sink.Status status = sink.process(); + failed = true; + } + } catch (EventDeliveryException ex) { + // This is correct + } + + sink.stop(); + Assert.assertTrue(LifecycleController.waitForOneOf(sink, + LifecycleState.STOP_OR_ERROR, 5000)); + if (failed) { + Assert.fail("SSL-enabled sink successfully connected to a server with an untrusted certificate when it should have failed"); + } + } } diff --git a/flume-ng-core/src/test/java/org/apache/flume/source/TestThriftSource.java b/flume-ng-core/src/test/java/org/apache/flume/source/TestThriftSource.java index 357965fa9d..8b9fa2365e 100644 --- a/flume-ng-core/src/test/java/org/apache/flume/source/TestThriftSource.java +++ b/flume-ng-core/src/test/java/org/apache/flume/source/TestThriftSource.java @@ -35,11 +35,14 @@ import org.apache.flume.channel.ReplicatingChannelSelector; import org.apache.flume.conf.Configurables; import org.apache.flume.event.EventBuilder; + import org.junit.After; import org.junit.Assert; import org.junit.Before; import org.junit.Test; +import javax.net.ssl.TrustManagerFactory; +import javax.net.ssl.KeyManagerFactory; import java.util.ArrayList; import java.util.Collections; import java.util.List; @@ -87,6 +90,40 @@ private void configureSource() { source.setChannelProcessor(new ChannelProcessor(rcs)); } + @Test + public void testAppendSSL() throws Exception { + Properties sslprops = (Properties)props.clone(); + sslprops.put("ssl", "true"); + sslprops.put("truststore", "src/test/resources/truststorefile.jks"); + sslprops.put("truststore-password", "password"); + sslprops.put("trustmanager-type", TrustManagerFactory.getDefaultAlgorithm()); + client = RpcClientFactory.getThriftInstance(sslprops); + + Context context = new Context(); + channel.configure(context); + configureSource(); + context.put(ThriftSource.CONFIG_BIND, "0.0.0.0"); + context.put(ThriftSource.CONFIG_PORT, String.valueOf(port)); + context.put("ssl", "true"); + context.put("keystore", "src/test/resources/keystorefile.jks"); + context.put("keystore-password", "password"); + context.put("keymanager-type", KeyManagerFactory.getDefaultAlgorithm()); + Configurables.configure(source, context); + source.start(); + for(int i = 0; i < 30; i++) { + client.append(EventBuilder.withBody(String.valueOf(i).getBytes())); + } + Transaction transaction = channel.getTransaction(); + transaction.begin(); + + for (int i = 0; i < 30; i++) { + Event event = channel.take(); + Assert.assertNotNull(event); + Assert.assertEquals(String.valueOf(i), new String(event.getBody())); + } + transaction.commit(); + transaction.close(); + } @Test public void testAppend() throws Exception { diff --git a/flume-ng-core/src/test/resources/keystorefile.jks b/flume-ng-core/src/test/resources/keystorefile.jks new file mode 100644 index 0000000000000000000000000000000000000000..20ac6a81642c02a24aaa4dd385b170c481266bea GIT binary patch literal 1294 zcmezO_TO6u1_mY|W&~r_#GK5;Vj#~eevL0DknT5VV(d2HW8>0hV`O2}ViIIzWMyD! zVl4lju+DF~x0b%o{e`P!u5Od~m15y|KVi$Ax21L=1#BzTre?kNT{>%zQ`KdSQ(v+w z3x9P*ehZg>7x2LF?89|SzG*h9J4?5vZ~3U?q^97=5YFCR%@-E`#B?Haj*gzimZ)cn zc3u9vSDg%cH1m*3>l2nUS$fa?zNPG2d+DM{md?StUorbm{L2 z$s5Ht)hbkcU9;G7LYl@gnIn$t9qeD!eqykxn#Pd*WM$fhv>A5}IPkj7Hd`Ab$31U# zW%!nw(*aMtCusb-qswl{%k34sNB`&h5SG89!lIu;%y!CsWXeCX$b$82V zxh)pIwruy8+9i)J?`k((#8N!1_v!VQ&!X~mT^DOe@2#Eba#11i)ebo((+9VLY^5$5 z9=)^rj4_L1W4oj6p7)h+U*sQIRGIkZOLSw_iZwYKR@p#eDMHWGz>wk z>GJ|+CPpSE7H8Q$6$5TIcC9v#a~3SjtPBP@h62FYV-96u6J`(1%g)O$&x46@V2H3| zh;U(uaASxt!$l0_#CZ)(42=vejEsS#X%vuaYG4fK(%Cglj5~m-R|ggxj6jY&Q)45; z-}>N2*-|0Rp!;5P^xiyq{!MUukj;A+hQo694ch|MGF@XGlmdT8+V)2L+JE%xE`x8U zXNTroZ?tZZHOc>!masj0X28-V^XI={sTNQ=zp_2;W&bW&Q47RfIql8MN+;$muIK)xXqxo#cTyFj ziuIW#ANQ>YoY1+V`ToPGr}Nn+b*IHOREMgjF#1^kJMXE*+}O$ha>oyr))gzSGauOg z)%dUME=LxhOC9XY6(=&;vri{Tz;2Qt#Hm*(g)@OtTwWm)HIl5TUK;A$Wm~LhHSj1RFo~i%&@NOz^l*deqcP2(c zCpr1l?xCb}W_<<&9VQXy<*)pHoMn01cGoN4#Khyk#lC_yOd_ni*fx4QuU>v!rs}!I hCGA$m4;K#0thB26s3+@t{_KbOwRi2$&1I6=3;=Qh23`OF literal 0 HcmV?d00001 diff --git a/flume-ng-core/src/test/resources/truststorefile.jks b/flume-ng-core/src/test/resources/truststorefile.jks new file mode 100644 index 0000000000000000000000000000000000000000..a98c4907e6be573cbdb0e4c876cfdbc3bf29173a GIT binary patch literal 887 zcmezO_TO6u1_mY|W(3o$i8+~x#Xz1{{KjHE2G$5YQv*u|24+ixCT3HECZ^8|n3))v zm{^=;`&10L+1Rz(JkD9LFtaika^#sB8yWuA2RF)= z3TX!2_nM>k=E?JKg4=^^-n%dyma}iz7O0l#8tb4G_&d_JH{#d+qhEI!d^66fYKrV-7ByE+kQ;?hjsY#V=I=4^0WMI{&xAO++ilu z5aB4XiALW_Kd;kHysq{BlM=J!+>0KJ$C*SKrY8jSiz;)U*)(Zq)1ucc+#e!jzJi?g z{o#VvYqM?do!+xL#%xFU&dMq4cmJ|_)$}vmhufCDDLpVUyl>Z)NdISr>;jDqTRg=I zm0$R1hzV~$&uP?iV%b9*v8wKnnqG|ui`U6%Z(de9F>i4__b)}$q>sOosu)$Q&n)@4 zZ$;pQ&K1q~A4WZ$&o-$$Ev}(DR5gXs$NJxSPc7!gRtAteez3HzSb3fK!1k}ke`R+$ zviMx;U}vs4k;$HYBFTJ{PNKKtnUcF4OHN(z+VI*V<2C~3(E6c|s#v<}e{m+MYQ+cC2W?H;6F%mk-$)|P? zC7mDN-acm2$knVNk`{jVwj6}3o! literal 0 HcmV?d00001 diff --git a/flume-ng-sdk/src/main/java/org/apache/flume/api/ThriftRpcClient.java b/flume-ng-sdk/src/main/java/org/apache/flume/api/ThriftRpcClient.java index 6382a0eb24..4f75a2b464 100644 --- a/flume-ng-sdk/src/main/java/org/apache/flume/api/ThriftRpcClient.java +++ b/flume-ng-sdk/src/main/java/org/apache/flume/api/ThriftRpcClient.java @@ -31,7 +31,15 @@ import org.slf4j.Logger; import org.slf4j.LoggerFactory; +import javax.net.ssl.KeyManagerFactory; +import javax.net.ssl.TrustManagerFactory; +import javax.net.ssl.SSLContext; +import javax.net.ssl.SSLSocket; +import javax.net.ssl.SSLSocketFactory; + +import java.io.FileInputStream; import java.nio.ByteBuffer; +import java.security.KeyStore; import java.util.ArrayList; import java.util.HashSet; import java.util.Iterator; @@ -41,6 +49,7 @@ import java.util.Queue; import java.util.Random; import java.util.Set; +import java.util.Arrays; import java.util.concurrent.Callable; import java.util.concurrent.ExecutionException; import java.util.concurrent.ExecutorService; @@ -78,6 +87,15 @@ public class ThriftRpcClient extends AbstractRpcClient { private final Random random = new Random(); private String protocol; + private boolean enableSsl; + private String truststore; + private String truststorePassword; + private String truststoreType; + private String trustManagerType; + private static final String TRUSTMANAGER_TYPE = "trustmanager-type"; + private final List excludeProtocols = new LinkedList(); + + public ThriftRpcClient() { stateLock = new ReentrantLock(true); connState = State.INIT; @@ -311,6 +329,29 @@ protected void configure(Properties properties) throws FlumeException { connectionPoolSize = RpcClientConfigurationConstants .DEFAULT_CONNECTION_POOL_SIZE; } + + enableSsl = Boolean.parseBoolean(properties.getProperty( + RpcClientConfigurationConstants.CONFIG_SSL)); + if(enableSsl) { + truststore = properties.getProperty( + RpcClientConfigurationConstants.CONFIG_TRUSTSTORE); + truststorePassword = properties.getProperty( + RpcClientConfigurationConstants.CONFIG_TRUSTSTORE_PASSWORD); + truststoreType = properties.getProperty( + RpcClientConfigurationConstants.CONFIG_TRUSTSTORE_TYPE, "JKS"); + trustManagerType = properties.getProperty( + TRUSTMANAGER_TYPE, TrustManagerFactory.getDefaultAlgorithm()); + String excludeProtocolsStr = properties.getProperty( + RpcClientConfigurationConstants.CONFIG_EXCLUDE_PROTOCOLS); + if (excludeProtocolsStr == null) { + excludeProtocols.add("SSLv3"); + } else { + excludeProtocols.addAll(Arrays.asList(excludeProtocolsStr.split(" "))); + if (!excludeProtocols.contains("SSLv3")) { + excludeProtocols.add("SSLv3"); + } + } + } connectionManager = new ConnectionPoolManager(connectionPoolSize); connState = State.READY; } catch (Throwable ex) { @@ -341,8 +382,27 @@ private class ClientWrapper { private final int hashCode; public ClientWrapper() throws Exception{ - transport = new TFastFramedTransport(new TSocket(hostname, port)); - transport.open(); + TSocket tsocket; + if(enableSsl) { + // JDK6's factory doesn't appear to pass the protocol onto the Socket properly so we have + // to do some magic to make sure that happens. Not an issue in JDK7 + // Lifted from thrift-0.9.1 to make the SSLContext + SSLContext sslContext = createSSLContext(truststore, truststorePassword, trustManagerType, truststoreType); + + // Create the factory from it + SSLSocketFactory sslSockFactory = sslContext.getSocketFactory(); + + // Create the TSocket from that + tsocket = createSSLSocket(sslSockFactory, hostname, port, 120000, excludeProtocols); + } else { + tsocket = new TSocket(hostname, port); + } + + transport = new TFastFramedTransport(tsocket); + // The transport is already open for SSL as part of TSSLTransportFactory.getClientSocket + if(!transport.isOpen()) { + transport.open(); + } if (protocol.equals(BINARY_PROTOCOL)) { LOGGER.info("Using TBinaryProtocol"); client = new ThriftSourceProtocol.Client(new TBinaryProtocol @@ -456,4 +516,53 @@ public void closeAll() { } } } + + /** + * Lifted from ACCUMULO-3318 - Lifted from TSSLTransportFactory in Thrift-0.9.1. The method to create a client socket with an SSLContextFactory object is not visibile to us. Have to use + * SslConnectionParams instead of TSSLTransportParameters because no getters exist on TSSLTransportParameters. + * + */ + private static SSLContext createSSLContext(String truststore, + String truststorePassword, String trustManagerType, + String truststoreType) throws FlumeException { + SSLContext ctx; + try { + ctx = SSLContext.getInstance("TLS"); + TrustManagerFactory tmf = null; + KeyManagerFactory kmf = null; + tmf = TrustManagerFactory.getInstance(TrustManagerFactory.getDefaultAlgorithm()); + KeyStore ts = null; + if (truststore != null && truststoreType != null) { + ts = KeyStore.getInstance(truststoreType); + ts.load(new FileInputStream(truststore), truststorePassword.toCharArray()); + tmf.init(ts); + } + + tmf.init(ts); + ctx.init(null, tmf.getTrustManagers(), null); + + } catch (Exception e) { + throw new FlumeException("Error creating the transport", e); + } + return ctx; + } + + private static TSocket createSSLSocket(SSLSocketFactory factory, String host, + int port, int timeout, List excludeProtocols) throws FlumeException { + try { + SSLSocket socket = (SSLSocket) factory.createSocket(host, port); + socket.setSoTimeout(timeout); + + List enabledProtocols = new ArrayList(); + for (String protocol : socket.getEnabledProtocols()) { + if (!excludeProtocols.contains(protocol)) { + enabledProtocols.add(protocol); + } + } + socket.setEnabledProtocols(enabledProtocols.toArray(new String[0])); + return new TSocket(socket); + } catch (Exception e) { + throw new FlumeException("Could not connect to " + host + " on port " + port, e); + } + } } diff --git a/flume-ng-sdk/src/test/java/org/apache/flume/api/ThriftTestingSource.java b/flume-ng-sdk/src/test/java/org/apache/flume/api/ThriftTestingSource.java index 63d2fc39e0..70d2c1b5d4 100644 --- a/flume-ng-sdk/src/test/java/org/apache/flume/api/ThriftTestingSource.java +++ b/flume-ng-sdk/src/test/java/org/apache/flume/api/ThriftTestingSource.java @@ -26,15 +26,19 @@ import org.apache.flume.thrift.ThriftSourceProtocol; import org.apache.thrift.TException; import org.apache.thrift.protocol.TBinaryProtocol; -import org.apache.thrift.protocol.TBinaryProtocol.Factory; import org.apache.thrift.protocol.TCompactProtocol; -import org.apache.thrift.protocol.TProtocol; import org.apache.thrift.protocol.TProtocolFactory; import org.apache.thrift.server.THsHaServer; import org.apache.thrift.server.TServer; +import org.apache.thrift.transport.TSSLTransportFactory; import org.apache.thrift.transport.TNonblockingServerSocket; +import org.apache.thrift.transport.TServerSocket; import org.apache.thrift.transport.TNonblockingServerTransport; +import org.apache.thrift.transport.TFastFramedTransport; +import org.apache.thrift.transport.TServerTransport; +import java.lang.reflect.Method; +import java.net.InetAddress; import java.net.InetSocketAddress; import java.util.List; import java.util.Queue; @@ -184,9 +188,7 @@ public Status appendBatch(List events) throws } } - public ThriftTestingSource(String handlerName, int port, String protocol) throws Exception { - TNonblockingServerTransport serverTransport = new TNonblockingServerSocket(new - InetSocketAddress("0.0.0.0", port)); + private ThriftSourceProtocol.Iface getHandler(String handlerName) { ThriftSourceProtocol.Iface handler = null; if (handlerName.equals(HandlerType.OK.name())) { handler = new ThriftOKHandler(); @@ -201,6 +203,14 @@ public ThriftTestingSource(String handlerName, int port, String protocol) throws } else if (handlerName.equals(HandlerType.ALTERNATE.name())) { handler = new ThriftAlternateHandler(); } + return handler; + } + + public ThriftTestingSource(String handlerName, int port, String protocol) throws Exception { + TNonblockingServerTransport serverTransport = new TNonblockingServerSocket(new + InetSocketAddress("0.0.0.0", port)); + ThriftSourceProtocol.Iface handler = getHandler(handlerName); + TProtocolFactory transportProtocolFactory = null; if (protocol != null && protocol == ThriftRpcClient.BINARY_PROTOCOL) { transportProtocolFactory = new TBinaryProtocol.Factory(); @@ -219,6 +229,49 @@ public void run() { }); } + public ThriftTestingSource(String handlerName, int port, + String protocol, String keystore, + String keystorePassword, String keyManagerType, + String keystoreType) throws Exception { + TSSLTransportFactory.TSSLTransportParameters params = + new TSSLTransportFactory.TSSLTransportParameters(); + params.setKeyStore(keystore, keystorePassword, keyManagerType, keystoreType); + + TServerSocket serverTransport = TSSLTransportFactory.getServerSocket( + port, 10000, InetAddress.getByName("0.0.0.0"), params); + + ThriftSourceProtocol.Iface handler = getHandler(handlerName); + + Class serverClass = Class.forName("org.apache.thrift" + + ".server.TThreadPoolServer"); + Class argsClass = Class.forName("org.apache.thrift.server" + + ".TThreadPoolServer$Args"); + TServer.AbstractServerArgs args = (TServer.AbstractServerArgs) argsClass + .getConstructor(TServerTransport.class) + .newInstance(serverTransport); + Method m = argsClass.getDeclaredMethod("maxWorkerThreads", int.class); + m.invoke(args, Integer.MAX_VALUE); + TProtocolFactory transportProtocolFactory = null; + if (protocol != null && protocol == ThriftRpcClient.BINARY_PROTOCOL) { + transportProtocolFactory = new TBinaryProtocol.Factory(); + } else { + transportProtocolFactory = new TCompactProtocol.Factory(); + } + args.protocolFactory(transportProtocolFactory); + args.inputTransportFactory(new TFastFramedTransport.Factory()); + args.outputTransportFactory(new TFastFramedTransport.Factory()); + args.processor(new ThriftSourceProtocol + .Processor(handler)); + server = (TServer) serverClass.getConstructor(argsClass).newInstance + (args); + Executors.newSingleThreadExecutor().submit(new Runnable() { + @Override + public void run() { + server.serve(); + } + }); + } + public enum HandlerType { OK, FAIL, From 4a91456a72276e19e1a63d6a4df214bccef2bac8 Mon Sep 17 00:00:00 2001 From: Hari Shreedharan Date: Thu, 19 Feb 2015 14:20:28 -0800 Subject: [PATCH 203/341] FLUME-2214. File Channel Integrity Toll documentation. (Ashish Paliwal via Hari) --- flume-ng-doc/sphinx/FlumeUserGuide.rst | 26 ++++++++++++++++++++++++++ 1 file changed, 26 insertions(+) diff --git a/flume-ng-doc/sphinx/FlumeUserGuide.rst b/flume-ng-doc/sphinx/FlumeUserGuide.rst index 912342c9b8..47e899dd40 100644 --- a/flume-ng-doc/sphinx/FlumeUserGuide.rst +++ b/flume-ng-doc/sphinx/FlumeUserGuide.rst @@ -3666,6 +3666,32 @@ metrics as long values. } +Tools +===== + +File Channel Integrity Tool +--------------------------- + +File Channel Integrity tool verifies the integrity of individual Events in the File channel +and removes corrupted Events. + +The tools can be run as follows:: + + $bin/flume-ng tool --conf ./conf FCINTEGRITYTOOL -l ./datadir + +where datadir the comma separated list of data directory to ve verified. + +Following are the options available + +======================= ==================================================================== +Option Name Description +======================= ==================================================================== +h/help Displays help +**l/dataDirs** Comma-separated list of data directories which the tool must verify +======================= ==================================================================== + + + Topology Design Considerations ============================== From 407874b9041b2073e3ccc8e0a18353f865d900ce Mon Sep 17 00:00:00 2001 From: Jarek Jarcec Cecho Date: Wed, 25 Feb 2015 09:51:59 -0800 Subject: [PATCH 204/341] FLUME-2633: Update Kite dependency to 1.0.0 (Tom White via Jarek Jarcec Cecho) --- flume-ng-sinks/flume-dataset-sink/pom.xml | 2 +- .../org/apache/flume/sink/kite/DatasetSink.java | 13 ++++++------- .../flume/sink/kite/policy/FailurePolicy.java | 4 ++-- .../apache/flume/sink/kite/policy/SavePolicy.java | 5 ++++- .../org/apache/flume/sink/kite/TestDatasetSink.java | 8 ++++---- pom.xml | 4 ++-- 6 files changed, 19 insertions(+), 17 deletions(-) diff --git a/flume-ng-sinks/flume-dataset-sink/pom.xml b/flume-ng-sinks/flume-dataset-sink/pom.xml index e929d60544..ad3f603537 100644 --- a/flume-ng-sinks/flume-dataset-sink/pom.xml +++ b/flume-ng-sinks/flume-dataset-sink/pom.xml @@ -93,7 +93,7 @@ limitations under the License. org.kitesdk - kite-data-hcatalog + kite-data-hive diff --git a/flume-ng-sinks/flume-dataset-sink/src/main/java/org/apache/flume/sink/kite/DatasetSink.java b/flume-ng-sinks/flume-dataset-sink/src/main/java/org/apache/flume/sink/kite/DatasetSink.java index 3e665324b7..fd9f99113a 100644 --- a/flume-ng-sinks/flume-dataset-sink/src/main/java/org/apache/flume/sink/kite/DatasetSink.java +++ b/flume-ng-sinks/flume-dataset-sink/src/main/java/org/apache/flume/sink/kite/DatasetSink.java @@ -46,8 +46,9 @@ import org.kitesdk.data.DatasetIOException; import org.kitesdk.data.DatasetNotFoundException; import org.kitesdk.data.DatasetWriter; -import org.kitesdk.data.DatasetWriterException; import org.kitesdk.data.Datasets; +import org.kitesdk.data.Flushable; +import org.kitesdk.data.Syncable; import org.kitesdk.data.View; import org.kitesdk.data.spi.Registration; import org.kitesdk.data.URIBuilder; @@ -305,10 +306,10 @@ public Status process() throws EventDeliveryException { if (commitOnBatch) { // Flush/sync before commiting. A failure here will result in rolling back // the transaction - if (syncOnBatch) { - writer.sync(); - } else { - writer.flush(); + if (syncOnBatch && writer instanceof Syncable) { + ((Syncable) writer).sync(); + } else if (writer instanceof Flushable) { + ((Flushable) writer).flush(); } boolean committed = commitTransaction(); Preconditions.checkState(committed, @@ -484,8 +485,6 @@ void closeWriter() throws EventDeliveryException { throw new EventDeliveryException("Check HDFS permissions/health. IO" + " error trying to close the writer for dataset " + datasetUri, ex); - } catch (DatasetWriterException ex) { - throw new EventDeliveryException("Failure moving temp file.", ex); } catch (RuntimeException ex) { throw new EventDeliveryException("Error trying to close the writer for" + " dataset " + datasetUri, ex); diff --git a/flume-ng-sinks/flume-dataset-sink/src/main/java/org/apache/flume/sink/kite/policy/FailurePolicy.java b/flume-ng-sinks/flume-dataset-sink/src/main/java/org/apache/flume/sink/kite/policy/FailurePolicy.java index 47b6a256cd..f6f875a776 100644 --- a/flume-ng-sinks/flume-dataset-sink/src/main/java/org/apache/flume/sink/kite/policy/FailurePolicy.java +++ b/flume-ng-sinks/flume-dataset-sink/src/main/java/org/apache/flume/sink/kite/policy/FailurePolicy.java @@ -22,7 +22,7 @@ import org.apache.flume.Event; import org.apache.flume.EventDeliveryException; import org.apache.flume.sink.kite.DatasetSink; -import org.kitesdk.data.DatasetWriter; +import org.kitesdk.data.Syncable; /** * A policy for dealing with non-recoverable event delivery failures. @@ -68,7 +68,7 @@ public void handle(Event event, Throwable cause) * This allows the policy implementation to sync any data that it may not * have fully handled. * - * See {@link DatasetWriter#sync()}. + * See {@link Syncable#sync()}. * * @throws EventDeliveryException The policy failed while syncing data. * When this is thrown, the Flume transaction diff --git a/flume-ng-sinks/flume-dataset-sink/src/main/java/org/apache/flume/sink/kite/policy/SavePolicy.java b/flume-ng-sinks/flume-dataset-sink/src/main/java/org/apache/flume/sink/kite/policy/SavePolicy.java index ed47898548..bd537eca3c 100644 --- a/flume-ng-sinks/flume-dataset-sink/src/main/java/org/apache/flume/sink/kite/policy/SavePolicy.java +++ b/flume-ng-sinks/flume-dataset-sink/src/main/java/org/apache/flume/sink/kite/policy/SavePolicy.java @@ -30,6 +30,7 @@ import org.kitesdk.data.DatasetWriter; import org.kitesdk.data.Datasets; import org.kitesdk.data.Formats; +import org.kitesdk.data.Syncable; import org.kitesdk.data.View; import static org.apache.flume.sink.kite.DatasetSinkConstants.*; @@ -87,7 +88,9 @@ public void sync() throws EventDeliveryException { // dataset close(); } else { - writer.sync(); + if (writer instanceof Syncable) { + ((Syncable) writer).sync(); + } } } } diff --git a/flume-ng-sinks/flume-dataset-sink/src/test/java/org/apache/flume/sink/kite/TestDatasetSink.java b/flume-ng-sinks/flume-dataset-sink/src/test/java/org/apache/flume/sink/kite/TestDatasetSink.java index 58aa467616..621920d0b1 100644 --- a/flume-ng-sinks/flume-dataset-sink/src/test/java/org/apache/flume/sink/kite/TestDatasetSink.java +++ b/flume-ng-sinks/flume-dataset-sink/src/test/java/org/apache/flume/sink/kite/TestDatasetSink.java @@ -78,12 +78,12 @@ public class TestDatasetSink { - public static final String FILE_REPO_URI = "repo:file:target/test-repo"; + public static final String FILE_REPO_URI = "repo:file:target/test_repo"; public static final String DATASET_NAME = "test"; public static final String FILE_DATASET_URI = - "dataset:file:target/test-repo/" + DATASET_NAME; + "dataset:file:target/test_repo/" + DATASET_NAME; public static final String ERROR_DATASET_URI = - "dataset:file:target/test-repo/failed-events"; + "dataset:file:target/test_repo/failed_events"; public static final File SCHEMA_FILE = new File("target/record-schema.avsc"); public static final Schema RECORD_SCHEMA = new Schema.Parser().parse( "{\"type\":\"record\",\"name\":\"rec\",\"fields\":[" + @@ -254,7 +254,7 @@ public Object call() throws EventDeliveryException { @Test public void testPartitionedData() throws EventDeliveryException { - URI partitionedUri = URI.create("dataset:file:target/test-repo/partitioned"); + URI partitionedUri = URI.create("dataset:file:target/test_repo/partitioned"); try { Datasets.create(partitionedUri, new DatasetDescriptor.Builder(DESCRIPTOR) .partitionStrategy(new PartitionStrategy.Builder() diff --git a/pom.xml b/pom.xml index ea7ffe35d1..3e405585d9 100644 --- a/pom.xml +++ b/pom.xml @@ -50,7 +50,7 @@ limitations under the License. 0.90.1 2.4.0 0.7.0 - 0.17.1 + 1.0.0 0.13.1 2.7.1 2.9.1 @@ -1328,7 +1328,7 @@ limitations under the License. org.kitesdk - kite-data-hcatalog + kite-data-hive ${kite.version} From 3d03053615694ca638e5ddf314081826b8a5f1ac Mon Sep 17 00:00:00 2001 From: Hari Shreedharan Date: Wed, 25 Feb 2015 21:54:34 -0800 Subject: [PATCH 205/341] FLUME-2570. Add option to not pad date fields. (Peter Leckie via Hari) --- .../flume/formatter/output/BucketPath.java | 10 ++++++ .../formatter/output/TestBucketPath.java | 31 +++++++++++++++++++ flume-ng-doc/sphinx/FlumeUserGuide.rst | 2 ++ 3 files changed, 43 insertions(+) diff --git a/flume-ng-core/src/main/java/org/apache/flume/formatter/output/BucketPath.java b/flume-ng-core/src/main/java/org/apache/flume/formatter/output/BucketPath.java index afe095c78f..e565192739 100644 --- a/flume-ng-core/src/main/java/org/apache/flume/formatter/output/BucketPath.java +++ b/flume-ng-core/src/main/java/org/apache/flume/formatter/output/BucketPath.java @@ -74,6 +74,8 @@ public static String expandShorthand(char c) { return "datetime"; case 'd': return "day_of_month_xx"; // two digit + case 'e': + return "day_of_month_x"; // 1 or 2 digit case 'D': return "date_short"; // "MM/dd/yy"; case 'H': @@ -88,6 +90,8 @@ public static String expandShorthand(char c) { return "hour_12"; // 1 or 2 digits case 'm': return "month_xx"; + case 'n': + return "month_x"; // 1 or 2 digits case 'M': return "minute_xx"; case 'p': @@ -261,6 +265,9 @@ protected static String replaceShorthand(char c, Map headers, case 'd': formatString = "dd"; break; + case 'e': + formatString = "d"; + break; case 'D': formatString = "MM/dd/yy"; break; @@ -285,6 +292,9 @@ protected static String replaceShorthand(char c, Map headers, case 'M': formatString = "mm"; break; + case 'n': + formatString = "M"; + break; case 'p': formatString = "a"; break; diff --git a/flume-ng-core/src/test/java/org/apache/flume/formatter/output/TestBucketPath.java b/flume-ng-core/src/test/java/org/apache/flume/formatter/output/TestBucketPath.java index 8d1da12dc9..21b972b21f 100644 --- a/flume-ng-core/src/test/java/org/apache/flume/formatter/output/TestBucketPath.java +++ b/flume-ng-core/src/test/java/org/apache/flume/formatter/output/TestBucketPath.java @@ -121,6 +121,37 @@ public void testNoRounding(){ Assert.assertEquals(expectedString, escapedString); } + + @Test + public void testNoPadding(){ + Calendar calender; + Map calender_timestamp; + calender = Calendar.getInstance(); + + //Check single digit dates + calender.set(2014, (5-1), 3, 13, 46, 33); + calender_timestamp = new HashMap(); + calender_timestamp.put("timestamp", String.valueOf(calender.getTimeInMillis())); + SimpleDateFormat format = new SimpleDateFormat("M-d"); + + String test = "%n-%e"; // eg 5-3 + String escapedString = BucketPath.escapeString( + test, calender_timestamp, false, Calendar.HOUR_OF_DAY, 12); + Date d = new Date(calender.getTimeInMillis()); + String expectedString = format.format(d); + + //Check two digit dates + calender.set(2014, (11-1), 13, 13, 46, 33); + calender_timestamp.put("timestamp", String.valueOf(calender.getTimeInMillis())); + escapedString += " " + BucketPath.escapeString( + test, calender_timestamp, false, Calendar.HOUR_OF_DAY, 12); + System.out.println("Escaped String: " + escapedString); + d = new Date(calender.getTimeInMillis()); + expectedString += " " + format.format(d); + System.out.println("Expected String: "+ expectedString); + Assert.assertEquals(expectedString, escapedString); + } + @Test public void testDateFormatTimeZone(){ TimeZone utcTimeZone = TimeZone.getTimeZone("UTC"); diff --git a/flume-ng-doc/sphinx/FlumeUserGuide.rst b/flume-ng-doc/sphinx/FlumeUserGuide.rst index 47e899dd40..4122cfe970 100644 --- a/flume-ng-doc/sphinx/FlumeUserGuide.rst +++ b/flume-ng-doc/sphinx/FlumeUserGuide.rst @@ -1628,12 +1628,14 @@ Alias Description %B locale's long month name (January, February, ...) %c locale's date and time (Thu Mar 3 23:05:25 2005) %d day of month (01) +%e day of month without padding (1) %D date; same as %m/%d/%y %H hour (00..23) %I hour (01..12) %j day of year (001..366) %k hour ( 0..23) %m month (01..12) +%n month without padding (1..12) %M minute (00..59) %p locale's equivalent of am or pm %s seconds since 1970-01-01 00:00:00 UTC From 542b1695033d330eb00ae81713fdc838b88332b6 Mon Sep 17 00:00:00 2001 From: Hari Shreedharan Date: Thu, 5 Mar 2015 23:19:13 -0800 Subject: [PATCH 206/341] FLUME-2631. End to End authentication in Flume (Johny Rufus via Hari) --- flume-ng-auth/pom.xml | 88 +++++++ .../flume/api/SecureRpcClientFactory.java | 40 +++ .../flume/api/SecureThriftRpcClient.java | 113 +++++++++ .../flume/auth/FlumeAuthenticationUtil.java | 99 ++++++++ .../apache/flume/auth/FlumeAuthenticator.java | 45 ++++ .../flume/auth/KerberosAuthenticator.java | 233 ++++++++++++++++++ .../apache/flume/auth/PrivilegedExecutor.java | 52 ++++ .../apache/flume/auth/SecurityException.java | 40 +++ .../flume/auth/SimpleAuthenticator.java | 88 +++++++ .../org/apache/flume/auth/UGIExecutor.java | 80 ++++++ .../flume/auth/TestFlumeAuthenticator.java | 53 ++-- flume-ng-core/pom.xml | 5 + .../org/apache/flume/sink/ThriftSink.java | 14 +- .../org/apache/flume/source/ThriftSource.java | 67 ++++- flume-ng-dist/pom.xml | 4 + flume-ng-dist/src/main/assembly/bin.xml | 1 + flume-ng-dist/src/main/assembly/src.xml | 1 + .../api/RpcClientConfigurationConstants.java | 2 + .../org/apache/flume/api/ThriftRpcClient.java | 30 ++- flume-ng-sinks/flume-dataset-sink/pom.xml | 7 - .../apache/flume/sink/kite/DatasetSink.java | 39 +-- .../apache/flume/sink/kite/KerberosUtil.java | 187 -------------- .../apache/flume/sink/hdfs/BucketWriter.java | 37 +-- .../apache/flume/sink/hdfs/HDFSEventSink.java | 229 ++--------------- .../flume/sink/hdfs/TestBucketWriter.java | 28 ++- .../flume/sink/hdfs/TestHDFSEventSink.java | 2 +- .../apache/flume/sink/hbase/HBaseSink.java | 34 +-- .../sink/hbase/HBaseSinkSecurityManager.java | 134 ---------- pom.xml | 7 + 29 files changed, 1087 insertions(+), 672 deletions(-) create mode 100644 flume-ng-auth/pom.xml create mode 100644 flume-ng-auth/src/main/java/org/apache/flume/api/SecureRpcClientFactory.java create mode 100644 flume-ng-auth/src/main/java/org/apache/flume/api/SecureThriftRpcClient.java create mode 100644 flume-ng-auth/src/main/java/org/apache/flume/auth/FlumeAuthenticationUtil.java create mode 100644 flume-ng-auth/src/main/java/org/apache/flume/auth/FlumeAuthenticator.java create mode 100644 flume-ng-auth/src/main/java/org/apache/flume/auth/KerberosAuthenticator.java create mode 100644 flume-ng-auth/src/main/java/org/apache/flume/auth/PrivilegedExecutor.java create mode 100644 flume-ng-auth/src/main/java/org/apache/flume/auth/SecurityException.java create mode 100644 flume-ng-auth/src/main/java/org/apache/flume/auth/SimpleAuthenticator.java create mode 100644 flume-ng-auth/src/main/java/org/apache/flume/auth/UGIExecutor.java rename flume-ng-sinks/flume-dataset-sink/src/test/java/org/apache/flume/sink/kite/TestKerberosUtil.java => flume-ng-auth/src/test/java/org/apache/flume/auth/TestFlumeAuthenticator.java (67%) delete mode 100644 flume-ng-sinks/flume-dataset-sink/src/main/java/org/apache/flume/sink/kite/KerberosUtil.java delete mode 100644 flume-ng-sinks/flume-ng-hbase-sink/src/main/java/org/apache/flume/sink/hbase/HBaseSinkSecurityManager.java diff --git a/flume-ng-auth/pom.xml b/flume-ng-auth/pom.xml new file mode 100644 index 0000000000..292731dcfb --- /dev/null +++ b/flume-ng-auth/pom.xml @@ -0,0 +1,88 @@ + + + 4.0.0 + + + flume-parent + org.apache.flume + 1.6.0-SNAPSHOT + + + flume-ng-auth + Flume Auth + Flume Authentication + + + + + org.apache.rat + apache-rat-plugin + + + org.apache.felix + maven-bundle-plugin + 2.3.7 + true + true + + + + + + + + junit + junit + test + + + + org.slf4j + slf4j-api + + + + org.slf4j + slf4j-log4j12 + + + + org.apache.hadoop + ${hadoop.common.artifact.id} + + + + org.apache.flume + flume-ng-sdk + + + + org.apache.hadoop + hadoop-minikdc + ${hadoop2.version} + test + + + + com.google.guava + guava + + + + diff --git a/flume-ng-auth/src/main/java/org/apache/flume/api/SecureRpcClientFactory.java b/flume-ng-auth/src/main/java/org/apache/flume/api/SecureRpcClientFactory.java new file mode 100644 index 0000000000..c976458204 --- /dev/null +++ b/flume-ng-auth/src/main/java/org/apache/flume/api/SecureRpcClientFactory.java @@ -0,0 +1,40 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, + * software distributed under the License is distributed on an + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + * KIND, either express or implied. See the License for the + * specific language governing permissions and limitations + * under the License. + */ +package org.apache.flume.api; + +import java.util.Properties; + +/** + * Factory class to construct Flume {@link RPCClient} implementations. + */ +public class SecureRpcClientFactory { + + /** + * Return a secure {@linkplain org.apache.flume.api.RpcClient} that uses Thrift for communicating with + * the next hop. + * @param props + * @return - An {@linkplain org.apache.flume.api.RpcClient} which uses thrift configured with the + * given parameters. + */ + public static RpcClient getThriftInstance(Properties props) { + ThriftRpcClient client = new SecureThriftRpcClient(); + client.configure(props); + return client; + } +} diff --git a/flume-ng-auth/src/main/java/org/apache/flume/api/SecureThriftRpcClient.java b/flume-ng-auth/src/main/java/org/apache/flume/api/SecureThriftRpcClient.java new file mode 100644 index 0000000000..7316e1b54a --- /dev/null +++ b/flume-ng-auth/src/main/java/org/apache/flume/api/SecureThriftRpcClient.java @@ -0,0 +1,113 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, + * software distributed under the License is distributed on an + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + * KIND, either express or implied. See the License for the + * specific language governing permissions and limitations + * under the License. + */ +package org.apache.flume.api; + +import org.apache.flume.FlumeException; +import org.apache.flume.auth.FlumeAuthenticationUtil; +import org.apache.flume.auth.FlumeAuthenticator; +import org.apache.flume.auth.PrivilegedExecutor; +import org.apache.thrift.transport.*; + +import javax.security.auth.callback.CallbackHandler; +import javax.security.sasl.Sasl; +import java.io.IOException; +import java.security.PrivilegedExceptionAction; +import java.util.HashMap; +import java.util.Map; +import java.util.Properties; + +public class SecureThriftRpcClient extends ThriftRpcClient { + + private static final String CLIENT_PRINCIPAL = "client-principal"; + private static final String CLIENT_KEYTAB = "client-keytab"; + private static final String SERVER_PRINCIPAL = "server-principal"; + + private String serverPrincipal; + private FlumeAuthenticator privilegedExecutor; + + @Override + protected void configure(Properties properties) throws FlumeException { + super.configure(properties); + serverPrincipal = properties.getProperty(SERVER_PRINCIPAL); + if (serverPrincipal == null || serverPrincipal.isEmpty()) { + throw new IllegalArgumentException("Flume in secure mode, but Flume config doesn't " + + "specify a server principal to use for Kerberos auth."); + } + String clientPrincipal = properties.getProperty(CLIENT_PRINCIPAL); + String keytab = properties.getProperty(CLIENT_KEYTAB); + this.privilegedExecutor = FlumeAuthenticationUtil.getAuthenticator(clientPrincipal, keytab); + if(!privilegedExecutor.isAuthenticated()) { + throw new FlumeException("Authentication failed in Kerberos mode for " + + "principal " + clientPrincipal + " keytab " + keytab); + } + } + + @Override + protected TTransport getTransport(TSocket tsocket) throws Exception { + Map saslProperties = new HashMap(); + saslProperties.put(Sasl.QOP, "auth"); + String[] names; + try { + names = FlumeAuthenticationUtil.splitKerberosName(serverPrincipal); + } catch (IOException e) { + throw new FlumeException( + "Error while trying to resolve Principal name - " + serverPrincipal, e); + } + return new UgiSaslClientTransport( + "GSSAPI", null, names[0], names[1], saslProperties, null, tsocket, privilegedExecutor); + } + + /** + * This transport wraps the Sasl transports to set up the right UGI context for open(). + */ + public static class UgiSaslClientTransport extends TSaslClientTransport { + PrivilegedExecutor privilegedExecutor; + public UgiSaslClientTransport(String mechanism, String authorizationId, + String protocol, String serverName, Map props, + CallbackHandler cbh, TTransport transport, PrivilegedExecutor privilegedExecutor) throws IOException { + super(mechanism, authorizationId, protocol, serverName, props, cbh, + transport); + this.privilegedExecutor = privilegedExecutor; + } + + // open the SASL transport with using the current UserGroupInformation + // This is needed to get the current login context stored + @Override + public void open() throws FlumeException { + try { + this.privilegedExecutor.execute( + new PrivilegedExceptionAction() { + public Void run() throws FlumeException { + try { + UgiSaslClientTransport.super.open(); + } catch (TTransportException e) { + throw new FlumeException("Failed to open SASL transport", e); + } + return null; + } + }); + } catch (InterruptedException e) { + throw new FlumeException( + "Interrupted while opening underlying transport", e); + } catch (Exception e) { + throw new FlumeException("Failed to open SASL transport", e); + } + } + } +} diff --git a/flume-ng-auth/src/main/java/org/apache/flume/auth/FlumeAuthenticationUtil.java b/flume-ng-auth/src/main/java/org/apache/flume/auth/FlumeAuthenticationUtil.java new file mode 100644 index 0000000000..02afc0d1e9 --- /dev/null +++ b/flume-ng-auth/src/main/java/org/apache/flume/auth/FlumeAuthenticationUtil.java @@ -0,0 +1,99 @@ +/** + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package org.apache.flume.auth; + +import com.google.common.base.Preconditions; +import org.apache.hadoop.security.SaslRpcServer; +import org.apache.hadoop.security.SecurityUtil; + +import javax.security.auth.callback.CallbackHandler; +import java.io.IOException; + +/** + * FlumeAuthentication utility class that provides methods to get an + * Authenticator. If proper credentials are provided KerberosAuthenticator is + * returned which can be used to execute as the authenticated principal , + * or else a SimpleAuthenticator which executes without any authentication + */ +public class FlumeAuthenticationUtil { + + private FlumeAuthenticationUtil() {} + + private static KerberosAuthenticator kerbAuthenticator; + + /** + * If principal and keytab are null, this method returns a SimpleAuthenticator + * which executes without authentication. If valid credentials are + * provided KerberosAuthenitcator is returned which can be used to execute as + * the authenticated principal. Invalid credentials result in + * IllegalArgumentException and Failure to authenticate results in SecurityException + * + * @param principal + * @param keytab + * @return FlumeAuthenticator + * + * @throws org.apache.flume.auth.SecurityException + */ + public synchronized static FlumeAuthenticator getAuthenticator( + String principal, String keytab) throws SecurityException { + + if(principal == null && keytab == null) { + return SimpleAuthenticator.getSimpleAuthenticator(); + } + + Preconditions.checkArgument(principal != null, + "Principal can not be null when keytab is provided"); + Preconditions.checkArgument(keytab != null, + "Keytab can not be null when Principal is provided"); + + if(kerbAuthenticator == null) { + kerbAuthenticator = new KerberosAuthenticator(); + } + kerbAuthenticator.authenticate(principal, keytab); + + return kerbAuthenticator; + } + + /** + * Returns the standard SaslGssCallbackHandler from the hadoop common module + * + * @return CallbackHandler + */ + public static CallbackHandler getSaslGssCallbackHandler() { + return new SaslRpcServer.SaslGssCallbackHandler(); + } + + /** + * Resolves the principal using Hadoop common's SecurityUtil and splits + * the kerberos principal into three parts user name, host and kerberos realm + * + * @param principal + * @return String[] of username, hostname and kerberos realm + * @throws IOException + */ + public static String[] splitKerberosName(String principal) throws IOException { + String resolvedPrinc = SecurityUtil.getServerPrincipal(principal, ""); + return SaslRpcServer.splitKerberosName(resolvedPrinc); + } +} + + + + + + diff --git a/flume-ng-auth/src/main/java/org/apache/flume/auth/FlumeAuthenticator.java b/flume-ng-auth/src/main/java/org/apache/flume/auth/FlumeAuthenticator.java new file mode 100644 index 0000000000..dbe241d722 --- /dev/null +++ b/flume-ng-auth/src/main/java/org/apache/flume/auth/FlumeAuthenticator.java @@ -0,0 +1,45 @@ +/** + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package org.apache.flume.auth; + +/** + * FlumeAuthenticator extends on a PrivilegedExecutor providing capabilities to + * proxy as a different user + */ +public interface FlumeAuthenticator extends PrivilegedExecutor { + /** + * Returns the current instance if proxyUsername is null or + * returns the proxied Executor if proxyUserName is valid + * @param proxyUserName + * @return PrivilegedExecutor + */ + public PrivilegedExecutor proxyAs(String proxyUserName); + + /** + * Returns true, if the underlying Authenticator was obtained by + * successful kerberos authentication + * @return boolean + */ + public boolean isAuthenticated(); + + /** + * For Authenticators backed by credentials, this method refreshes the + * credentials periodically + */ + public void startCredentialRefresher(); +} diff --git a/flume-ng-auth/src/main/java/org/apache/flume/auth/KerberosAuthenticator.java b/flume-ng-auth/src/main/java/org/apache/flume/auth/KerberosAuthenticator.java new file mode 100644 index 0000000000..324404601a --- /dev/null +++ b/flume-ng-auth/src/main/java/org/apache/flume/auth/KerberosAuthenticator.java @@ -0,0 +1,233 @@ +/** + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package org.apache.flume.auth; + +import com.google.common.annotations.VisibleForTesting; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +import java.io.File; +import java.io.IOException; +import java.security.PrivilegedAction; +import java.security.PrivilegedExceptionAction; +import java.util.HashMap; +import java.util.Map; +import java.util.concurrent.Executors; +import java.util.concurrent.ScheduledExecutorService; +import java.util.concurrent.TimeUnit; + +import org.apache.hadoop.conf.Configuration; +import org.apache.hadoop.security.SecurityUtil; +import org.apache.hadoop.security.UserGroupInformation; +import org.apache.hadoop.security.UserGroupInformation.AuthenticationMethod; + +import com.google.common.base.Preconditions; +import static org.apache.hadoop.fs.CommonConfigurationKeys.HADOOP_SECURITY_AUTHENTICATION; + +/** + * A kerberos authenticator, which authenticates using the supplied principal + * and keytab and executes with authenticated privileges + */ +class KerberosAuthenticator implements FlumeAuthenticator { + + private static final Logger LOG = LoggerFactory + .getLogger(KerberosAuthenticator.class); + + private volatile UserGroupInformation ugi; + private volatile PrivilegedExecutor privilegedExecutor; + private Map proxyCache = new HashMap(); + + + @Override + public T execute(PrivilegedAction action) { + return privilegedExecutor.execute(action); + } + + @Override + public T execute(PrivilegedExceptionAction action) throws Exception { + return privilegedExecutor.execute(action); + } + + @Override + public synchronized PrivilegedExecutor proxyAs(String proxyUserName) { + if(proxyUserName == null || proxyUserName.isEmpty()) { + return this; + } + if(proxyCache.get(proxyUserName) == null) { + UserGroupInformation proxyUgi; + proxyUgi = UserGroupInformation.createProxyUser(proxyUserName, ugi); + printUGI(proxyUgi); + proxyCache.put(proxyUserName, new UGIExecutor(proxyUgi)); + } + return proxyCache.get(proxyUserName); + } + + @Override + public boolean isAuthenticated() { + return true; + } + + /** + * When valid principal and keytab are provided and if authentication has + * not yet been done for this object, this method authenticates the + * credentials and populates the ugi. In case of null or invalid credentials + * IllegalArgumentException is thrown. In case of failure to authenticate, + * SecurityException is thrown. If authentication has already happened on + * this KerberosAuthenticator object, then this method checks to see if the current + * credentials passed are same as the validated credentials. If not, it throws + * an exception as this authenticator can represent only one Principal. + * + * @param principal + * @param keytab + */ + public synchronized void authenticate(String principal, String keytab) { + // sanity checking + + Preconditions.checkArgument(principal != null && !principal.isEmpty(), + "Invalid Kerberos principal: " + String.valueOf(principal)); + Preconditions.checkArgument(keytab != null && !keytab.isEmpty(), + "Invalid Kerberos keytab: " + String.valueOf(keytab)); + File keytabFile = new File(keytab); + Preconditions.checkArgument(keytabFile.isFile() && keytabFile.canRead(), + "Keytab is not a readable file: " + String.valueOf(keytab)); + + + // resolve the requested principal + String resolvedPrincipal; + try { + // resolves _HOST pattern using standard Hadoop search/replace + // via DNS lookup when 2nd argument is empty + resolvedPrincipal = SecurityUtil.getServerPrincipal(principal, ""); + } catch (IOException e) { + throw new IllegalArgumentException("Host lookup error resolving kerberos principal (" + + principal + "). Exception follows.", e); + } + Preconditions.checkNotNull(resolvedPrincipal, + "Resolved Principal must not be null"); + + + // be cruel and unusual when user tries to login as multiple principals + // this isn't really valid with a reconfigure but this should be rare + // enough to warrant a restart of the agent JVM + // TODO: find a way to interrogate the entire current config state, + // since we don't have to be unnecessarily protective if they switch all + // HDFS sinks to use a different principal all at once. + + Preconditions.checkState(ugi == null || ugi.getUserName().equals(resolvedPrincipal), + "Cannot use multiple kerberos principals in the same agent. " + + " Must restart agent to use new principal or keytab. " + + "Previous = %s, New = %s", ugi, resolvedPrincipal); + + + // enable the kerberos mode of UGI, before doing anything else + if(!UserGroupInformation.isSecurityEnabled()) { + Configuration conf = new Configuration(false); + conf.set(HADOOP_SECURITY_AUTHENTICATION, "kerberos"); + UserGroupInformation.setConfiguration(conf); + } + + // We are interested in currently logged in user with kerberos creds + UserGroupInformation curUser = null; + try { + curUser = UserGroupInformation.getLoginUser(); + if(curUser != null && !curUser.hasKerberosCredentials()) { + curUser = null; + } + } catch (IOException e) { + LOG.warn("User unexpectedly had no active login. Continuing with " + + "authentication", e); + } + + /* + * if ugi is not null, + * if ugi matches currently logged in kerberos user, we are good + * else we are logged out, so relogin our ugi + * else if ugi is null, login and populate state + */ + try { + if (ugi != null) { + if (curUser != null && curUser.getUserName().equals(ugi.getUserName())) { + LOG.debug("Using existing principal login: {}", ugi); + } else { + LOG.info("Attempting kerberos Re-login as principal ({}) " + , new Object[] { ugi.getUserName() } ); + ugi.reloginFromKeytab(); + } + } else { + LOG.info("Attempting kerberos login as principal ({}) from keytab " + + "file ({})", new Object[] { resolvedPrincipal, keytab } ); + UserGroupInformation.loginUserFromKeytab(resolvedPrincipal, keytab); + this.ugi = UserGroupInformation.getLoginUser(); + this.privilegedExecutor = new UGIExecutor(this.ugi); + } + } catch (IOException e) { + throw new SecurityException("Authentication error while attempting to " + + "login as kerberos principal (" + resolvedPrincipal + ") using " + + "keytab (" + keytab + "). Exception follows.", e); + } + + printUGI(this.ugi); + } + + private void printUGI(UserGroupInformation ugi) { + if (ugi != null) { + // dump login information + AuthenticationMethod authMethod = ugi.getAuthenticationMethod(); + LOG.info("\n{} \nUser: {} \nAuth method: {} \nKeytab: {} \n", + new Object[]{ authMethod.equals(AuthenticationMethod.PROXY) ? + "Proxy as: " : "Logged as: ", ugi.getUserName(), authMethod, + ugi.isFromKeytab() } + ); + } + } + + /** + * startCredentialRefresher should be used only for long running + * methods like Thrift source. For all privileged methods that use a UGI, the + * credentials are checked automatically and refreshed before the + * privileged method is executed in the UGIExecutor + */ + @Override + public void startCredentialRefresher() { + int CHECK_TGT_INTERVAL = 120; // seconds + ScheduledExecutorService scheduler = Executors.newScheduledThreadPool(1); + scheduler.scheduleWithFixedDelay(new Runnable() { + @Override + public void run() { + try { + ugi.checkTGTAndReloginFromKeytab(); + } catch (IOException e) { + LOG.warn("Error occured during checkTGTAndReloginFromKeytab() for user " + + ugi.getUserName(), e); + } + } + }, CHECK_TGT_INTERVAL, CHECK_TGT_INTERVAL, TimeUnit.SECONDS); + } + + @VisibleForTesting + String getUserName() { + if(ugi != null) { + return ugi.getUserName(); + } else { + return null; + } + } +} + + + diff --git a/flume-ng-auth/src/main/java/org/apache/flume/auth/PrivilegedExecutor.java b/flume-ng-auth/src/main/java/org/apache/flume/auth/PrivilegedExecutor.java new file mode 100644 index 0000000000..0aa321a70c --- /dev/null +++ b/flume-ng-auth/src/main/java/org/apache/flume/auth/PrivilegedExecutor.java @@ -0,0 +1,52 @@ +/** + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package org.apache.flume.auth; + +import java.security.PrivilegedAction; +import java.security.PrivilegedExceptionAction; + + +/** + * PrivilegedExecutor provides the ability to execute a PrivilegedAction + * or a PrivilegedExceptionAction. Implementors of this class, can chose to execute + * in normal mode or secure authenticated mode + */ +public interface PrivilegedExecutor { + /** + * This method is used to execute a privileged action, the implementor can + * chose to execute the action using the appropriate privileges + * + * @param action A PrivilegedExceptionAction to perform as the desired user + * @param The return type of the action + * @return T the T value returned by action.run() + * @throws Exception + */ + public T execute(PrivilegedExceptionAction action) throws Exception; + + /** + * This method is used to execute a privileged action, the implementor can + * chose to execute the action using the appropriate privileges + * + * @param action A PrivilegedAction to perform as the desired user + * @param The return type of the action + * @return T the T value returned by action.run() + */ + public T execute(PrivilegedAction action); +} + + diff --git a/flume-ng-auth/src/main/java/org/apache/flume/auth/SecurityException.java b/flume-ng-auth/src/main/java/org/apache/flume/auth/SecurityException.java new file mode 100644 index 0000000000..5760481c17 --- /dev/null +++ b/flume-ng-auth/src/main/java/org/apache/flume/auth/SecurityException.java @@ -0,0 +1,40 @@ +/** + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package org.apache.flume.auth; + +/** + * SecurityException thrown in the Flume security module + */ +public class SecurityException extends RuntimeException { + public SecurityException(String message) { + super(message); + } + + public SecurityException(String message, Throwable cause) { + super(message, cause); + } + + public SecurityException(Throwable cause) { + super(cause); + } +} + + + + + diff --git a/flume-ng-auth/src/main/java/org/apache/flume/auth/SimpleAuthenticator.java b/flume-ng-auth/src/main/java/org/apache/flume/auth/SimpleAuthenticator.java new file mode 100644 index 0000000000..f7b5beac06 --- /dev/null +++ b/flume-ng-auth/src/main/java/org/apache/flume/auth/SimpleAuthenticator.java @@ -0,0 +1,88 @@ +/** + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package org.apache.flume.auth; + +import org.apache.hadoop.security.UserGroupInformation; + +import java.io.IOException; +import java.security.PrivilegedAction; +import java.security.PrivilegedExceptionAction; +import java.util.HashMap; +import java.util.Map; + +/** + * A no-op authenticator, which does not authenticate and executes + * without any authenticated privileges + */ +class SimpleAuthenticator implements FlumeAuthenticator { + private SimpleAuthenticator() {} + + private static class SimpleAuthenticatorHolder { + public static SimpleAuthenticator authenticator = new SimpleAuthenticator(); + } + + public static SimpleAuthenticator getSimpleAuthenticator() { + return SimpleAuthenticatorHolder.authenticator; + } + + private Map proxyCache = + new HashMap(); + + + @Override + public T execute(PrivilegedExceptionAction action) + throws Exception { + return action.run(); + } + + @Override + public T execute(PrivilegedAction action) { + return action.run(); + } + + @Override + public synchronized PrivilegedExecutor proxyAs(String proxyUserName) { + if(proxyUserName == null || proxyUserName.isEmpty()) { + return this; + } + if(proxyCache.get(proxyUserName) == null) { + UserGroupInformation proxyUgi; + try { + proxyUgi = UserGroupInformation.createProxyUser(proxyUserName, + UserGroupInformation.getCurrentUser()); + } catch (IOException e) { + throw new SecurityException("Unable to create proxy User", e); + } + proxyCache.put(proxyUserName, new UGIExecutor(proxyUgi)); + } + return proxyCache.get(proxyUserName); + } + + @Override + public boolean isAuthenticated() { + return false; + } + + @Override + public void startCredentialRefresher() { + // no-op + } + +} + + diff --git a/flume-ng-auth/src/main/java/org/apache/flume/auth/UGIExecutor.java b/flume-ng-auth/src/main/java/org/apache/flume/auth/UGIExecutor.java new file mode 100644 index 0000000000..a5aeef2eda --- /dev/null +++ b/flume-ng-auth/src/main/java/org/apache/flume/auth/UGIExecutor.java @@ -0,0 +1,80 @@ +/** + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package org.apache.flume.auth; + +import com.google.common.annotations.VisibleForTesting; +import org.apache.hadoop.security.UserGroupInformation; +import org.apache.hadoop.security.UserGroupInformation.AuthenticationMethod; + +import java.io.IOException; +import java.security.PrivilegedAction; +import java.security.PrivilegedExceptionAction; + +class UGIExecutor implements PrivilegedExecutor { + private UserGroupInformation ugi; + + UGIExecutor(UserGroupInformation ugi) { + this.ugi = ugi; + } + + @Override + public T execute(PrivilegedAction action) { + ensureValidAuth(); + return ugi.doAs(action); + } + + @Override + public T execute(PrivilegedExceptionAction action) throws Exception { + ensureValidAuth(); + try { + return ugi.doAs(action); + } catch (IOException ex) { + throw new SecurityException("Privileged action failed", ex); + } catch (InterruptedException ex) { + Thread.interrupted(); + throw new SecurityException(ex); + } + } + + private void ensureValidAuth() { + reloginUGI(ugi); + if(ugi.getAuthenticationMethod().equals(AuthenticationMethod.PROXY)) { + reloginUGI(ugi.getRealUser()); + } + } + + private void reloginUGI(UserGroupInformation ugi) { + try { + if(ugi.hasKerberosCredentials()) { + ugi.checkTGTAndReloginFromKeytab(); + } + } catch (IOException e) { + throw new SecurityException("Error trying to relogin from keytab for user " + + ugi.getUserName(), e); + } + } + + @VisibleForTesting + String getUserName() { + if(ugi != null) { + return ugi.getUserName(); + } else { + return null; + } + } +} diff --git a/flume-ng-sinks/flume-dataset-sink/src/test/java/org/apache/flume/sink/kite/TestKerberosUtil.java b/flume-ng-auth/src/test/java/org/apache/flume/auth/TestFlumeAuthenticator.java similarity index 67% rename from flume-ng-sinks/flume-dataset-sink/src/test/java/org/apache/flume/sink/kite/TestKerberosUtil.java rename to flume-ng-auth/src/test/java/org/apache/flume/auth/TestFlumeAuthenticator.java index f53ef73597..45ba2b0a78 100644 --- a/flume-ng-sinks/flume-dataset-sink/src/test/java/org/apache/flume/sink/kite/TestKerberosUtil.java +++ b/flume-ng-auth/src/test/java/org/apache/flume/auth/TestFlumeAuthenticator.java @@ -15,21 +15,19 @@ * See the License for the specific language governing permissions and * limitations under the License. */ -package org.apache.flume.sink.kite; +package org.apache.flume.auth; import java.io.File; import java.io.IOException; -import java.net.URL; import java.util.Properties; -import org.apache.hadoop.conf.Configuration; + import org.apache.hadoop.minikdc.MiniKdc; -import org.apache.hadoop.security.UserGroupInformation; import org.junit.AfterClass; import org.junit.BeforeClass; import org.junit.Test; import static org.junit.Assert.*; -public class TestKerberosUtil { +public class TestFlumeAuthenticator { private static MiniKdc kdc; private static File workDir; @@ -41,12 +39,8 @@ public class TestKerberosUtil { @BeforeClass public static void startMiniKdc() throws Exception { - URL resource = Thread.currentThread() - .getContextClassLoader().getResource("enable-kerberos.xml"); - Configuration.addDefaultResource("enable-kerberos.xml"); - workDir = new File(System.getProperty("test.dir", "target"), - TestKerberosUtil.class.getSimpleName()); + TestFlumeAuthenticator.class.getSimpleName()); flumeKeytab = new File(workDir, "flume.keytab"); aliceKeytab = new File(workDir, "alice.keytab"); conf = MiniKdc.createConf(); @@ -72,9 +66,10 @@ public static void stopMiniKdc() { public void testNullLogin() throws IOException { String principal = null; String keytab = null; - UserGroupInformation expResult = UserGroupInformation.getCurrentUser(); - UserGroupInformation result = KerberosUtil.login(principal, keytab); - assertEquals(expResult, result); + + FlumeAuthenticator authenticator = FlumeAuthenticationUtil.getAuthenticator( + principal, keytab); + assertFalse(authenticator.isAuthenticated()); } @Test @@ -83,21 +78,29 @@ public void testFlumeLogin() throws IOException { String keytab = flumeKeytab.getAbsolutePath(); String expResult = principal; - String result = KerberosUtil.login(principal, keytab).getUserName(); + FlumeAuthenticator authenticator = FlumeAuthenticationUtil.getAuthenticator( + principal, keytab); + assertTrue(authenticator.isAuthenticated()); + + String result = ((KerberosAuthenticator)authenticator).getUserName(); assertEquals("Initial login failed", expResult, result); - result = KerberosUtil.login(principal, keytab).getUserName(); + authenticator = FlumeAuthenticationUtil.getAuthenticator( + principal, keytab); + result = ((KerberosAuthenticator)authenticator).getUserName(); assertEquals("Re-login failed", expResult, result); principal = alicePrincipal; keytab = aliceKeytab.getAbsolutePath(); try { - result = KerberosUtil.login(principal, keytab).getUserName(); + authenticator = FlumeAuthenticationUtil.getAuthenticator( + principal, keytab); + result = ((KerberosAuthenticator)authenticator).getUserName(); fail("Login should have failed with a new principal: " + result); - } catch (KerberosUtil.SecurityException ex) { + } catch (Exception ex) { assertTrue("Login with a new principal failed, but for an unexpected " + "reason: " + ex.getMessage(), - ex.getMessage().contains("Cannot use multiple Kerberos principals: ")); + ex.getMessage().contains("Cannot use multiple kerberos principals")); } } @@ -105,16 +108,20 @@ public void testFlumeLogin() throws IOException { public void testProxyAs() throws IOException { String username = "alice"; - UserGroupInformation login = UserGroupInformation.getCurrentUser(); String expResult = username; - String result = KerberosUtil.proxyAs(username, login).getUserName(); + FlumeAuthenticator authenticator = FlumeAuthenticationUtil.getAuthenticator( + null, null); + String result = ((UGIExecutor)(authenticator.proxyAs(username))).getUserName(); assertEquals("Proxy as didn't generate the expected username", expResult, result); - login = KerberosUtil.login(flumePrincipal, flumeKeytab.getAbsolutePath()); + authenticator = FlumeAuthenticationUtil.getAuthenticator( + flumePrincipal, flumeKeytab.getAbsolutePath()); + + String login = ((KerberosAuthenticator)authenticator).getUserName(); assertEquals("Login succeeded, but the principal doesn't match", - flumePrincipal, login.getUserName()); + flumePrincipal, login); - result = KerberosUtil.proxyAs(username, login).getUserName(); + result = ((UGIExecutor)(authenticator.proxyAs(username))).getUserName(); assertEquals("Proxy as didn't generate the expected username", expResult, result); } diff --git a/flume-ng-core/pom.xml b/flume-ng-core/pom.xml index 89924142cf..fe34c0344d 100644 --- a/flume-ng-core/pom.xml +++ b/flume-ng-core/pom.xml @@ -263,6 +263,11 @@ limitations under the License. flume-ng-configuration + + org.apache.flume + flume-ng-auth + + org.slf4j slf4j-api diff --git a/flume-ng-core/src/main/java/org/apache/flume/sink/ThriftSink.java b/flume-ng-core/src/main/java/org/apache/flume/sink/ThriftSink.java index baa60d0cf1..32021d37df 100644 --- a/flume-ng-core/src/main/java/org/apache/flume/sink/ThriftSink.java +++ b/flume-ng-core/src/main/java/org/apache/flume/sink/ThriftSink.java @@ -23,6 +23,8 @@ import org.apache.flume.api.RpcClient; import org.apache.flume.api.RpcClientConfigurationConstants; import org.apache.flume.api.RpcClientFactory; +import org.apache.flume.api.SecureRpcClientFactory; + /** *

    * A {@link org.apache.flume.Sink} implementation that can send events to an RPC server (such as @@ -102,12 +104,18 @@ public class ThriftSink extends AbstractRpcSink { @Override protected RpcClient initializeRpcClient(Properties props) { - props.setProperty(RpcClientConfigurationConstants.CONFIG_CLIENT_TYPE, - RpcClientFactory.ClientType.THRIFT.name()); // Only one thread is enough, since only one sink thread processes // transactions at any given time. Each sink owns its own Rpc client. props.setProperty(RpcClientConfigurationConstants .CONFIG_CONNECTION_POOL_SIZE, String.valueOf(1)); - return RpcClientFactory.getInstance(props); + boolean enableKerberos = Boolean.parseBoolean(props.getProperty( + RpcClientConfigurationConstants.KERBEROS_KEY, "false")); + if(enableKerberos) { + return SecureRpcClientFactory.getThriftInstance(props); + } else { + props.setProperty(RpcClientConfigurationConstants.CONFIG_CLIENT_TYPE, + RpcClientFactory.ClientType.THRIFT.name()); + return RpcClientFactory.getInstance(props); + } } } diff --git a/flume-ng-core/src/main/java/org/apache/flume/source/ThriftSource.java b/flume-ng-core/src/main/java/org/apache/flume/source/ThriftSource.java index 06bb604a2f..1d8bb333b4 100644 --- a/flume-ng-core/src/main/java/org/apache/flume/source/ThriftSource.java +++ b/flume-ng-core/src/main/java/org/apache/flume/source/ThriftSource.java @@ -26,6 +26,8 @@ import org.apache.flume.Event; import org.apache.flume.EventDrivenSource; import org.apache.flume.FlumeException; +import org.apache.flume.auth.FlumeAuthenticationUtil; +import org.apache.flume.auth.FlumeAuthenticator; import org.apache.flume.conf.Configurable; import org.apache.flume.event.EventBuilder; import org.apache.flume.instrumentation.SourceCounter; @@ -45,12 +47,16 @@ import org.apache.thrift.transport.TServerSocket; import org.apache.thrift.transport.TServerTransport; import org.apache.thrift.transport.TSSLTransportFactory; +import org.apache.thrift.transport.TTransportFactory; +import org.apache.thrift.transport.TSaslServerTransport; import org.slf4j.Logger; import org.slf4j.LoggerFactory; import javax.net.ssl.KeyManagerFactory; import javax.net.ssl.SSLServerSocket; +import javax.security.sasl.Sasl; import java.io.FileInputStream; +import java.io.IOException; import java.lang.reflect.Method; import java.net.InetAddress; import java.net.InetSocketAddress; @@ -60,10 +66,13 @@ import java.util.Arrays; import java.util.LinkedList; import java.util.List; +import java.util.Map; +import java.util.HashMap; import java.util.concurrent.ExecutorService; import java.util.concurrent.Executors; import java.util.concurrent.ThreadFactory; import java.util.concurrent.TimeUnit; +import java.security.PrivilegedAction; public class ThriftSource extends AbstractSource implements Configurable, EventDrivenSource { @@ -97,6 +106,10 @@ public class ThriftSource extends AbstractSource implements Configurable, private static final String KEYMANAGER_TYPE = "keymanager-type"; private static final String EXCLUDE_PROTOCOLS = "exclude-protocols"; + private static final String KERBEROS_KEY = "kerberos"; + private static final String AGENT_PRINCIPAL = "agent-principal"; + private static final String AGENT_KEYTAB = "agent-keytab"; + private Integer port; private String bindAddress; private int maxThreads = 0; @@ -110,6 +123,9 @@ public class ThriftSource extends AbstractSource implements Configurable, private String keyManagerType; private final List excludeProtocols = new LinkedList(); private boolean enableSsl = false; + private boolean enableKerberos = false; + private String principal; + private FlumeAuthenticator flumeAuth; @Override public void configure(Context context) { @@ -171,6 +187,18 @@ public void configure(Context context) { "Thrift source configured with invalid keystore: " + keystore, ex); } } + + principal = context.getString(AGENT_PRINCIPAL); + String keytab = context.getString(AGENT_KEYTAB); + enableKerberos = context.getBoolean(KERBEROS_KEY, false); + this.flumeAuth = FlumeAuthenticationUtil.getAuthenticator(principal, keytab); + if(enableKerberos) { + if(!flumeAuth.isAuthenticated()) { + throw new FlumeException("Authentication failed in Kerberos mode for " + + "principal " + principal + " keytab " + keytab); + } + flumeAuth.startCredentialRefresher(); + } } @Override @@ -195,7 +223,15 @@ public void start() { servingExecutor.submit(new Runnable() { @Override public void run() { - server.serve(); + flumeAuth.execute( + new PrivilegedAction() { + @Override + public Object run() { + server.serve(); + return null; + } + } + ); } }); @@ -263,7 +299,7 @@ private TProtocolFactory getProtocolFactory() { } private TServer getTThreadedSelectorServer() { - if(enableSsl) { + if(enableSsl || enableKerberos) { return null; } Class serverClass; @@ -277,6 +313,7 @@ private TServer getTThreadedSelectorServer() { TServerTransport serverTransport = new TNonblockingServerSocket( new InetSocketAddress(bindAddress, port)); + ExecutorService sourceService; ThreadFactory threadFactory = new ThreadFactoryBuilder().setNameFormat( "Flume Thrift IPC Thread %d").build(); @@ -328,14 +365,35 @@ private void populateServerParams(TServer.AbstractServerArgs args) { args.protocolFactory(getProtocolFactory()); //populate the transportFactory - args.inputTransportFactory(new TFastFramedTransport.Factory()); - args.outputTransportFactory(new TFastFramedTransport.Factory()); + if(enableKerberos) { + args.transportFactory(getSASLTransportFactory()); + } else { + args.transportFactory(new TFastFramedTransport.Factory()); + } // populate the Processor args.processor(new ThriftSourceProtocol .Processor(new ThriftSourceHandler())); } + private TTransportFactory getSASLTransportFactory() { + String[] names; + try { + names = FlumeAuthenticationUtil.splitKerberosName(principal); + } catch (IOException e) { + throw new FlumeException( + "Error while trying to resolve Principal name - " + principal, e); + } + Map saslProperties = new HashMap(); + saslProperties.put(Sasl.QOP, "auth"); + TSaslServerTransport.Factory saslTransportFactory = + new TSaslServerTransport.Factory(); + saslTransportFactory.addServerDefinition( + "GSSAPI", names[0], names[1], saslProperties, + FlumeAuthenticationUtil.getSaslGssCallbackHandler()); + return saslTransportFactory; + } + @Override public void stop() { if(server != null && server.isServing()) { @@ -402,5 +460,4 @@ public Status appendBatch(List events) throws TException { return Status.OK; } } - } diff --git a/flume-ng-dist/pom.xml b/flume-ng-dist/pom.xml index a083fe2e0e..9f7c4f63ec 100644 --- a/flume-ng-dist/pom.xml +++ b/flume-ng-dist/pom.xml @@ -202,6 +202,10 @@ org.apache.flume flume-tools + + org.apache.flume + flume-ng-auth + diff --git a/flume-ng-dist/src/main/assembly/bin.xml b/flume-ng-dist/src/main/assembly/bin.xml index 5aa7cc6574..a61180d9cd 100644 --- a/flume-ng-dist/src/main/assembly/bin.xml +++ b/flume-ng-dist/src/main/assembly/bin.xml @@ -68,6 +68,7 @@ flume-ng-clients/** flume-ng-embedded-agent/** flume-tools/** + flume-ng-auth/** **/target/** **/.classpath **/.project diff --git a/flume-ng-dist/src/main/assembly/src.xml b/flume-ng-dist/src/main/assembly/src.xml index b1e79a2279..e5f4156bf8 100644 --- a/flume-ng-dist/src/main/assembly/src.xml +++ b/flume-ng-dist/src/main/assembly/src.xml @@ -49,6 +49,7 @@ org.apache.flume:flume-ng-clients org.apache.flume:flume-ng-embedded-agent org.apache.flume:flume-tools + org.apache.flume:flume-ng-auth diff --git a/flume-ng-sdk/src/main/java/org/apache/flume/api/RpcClientConfigurationConstants.java b/flume-ng-sdk/src/main/java/org/apache/flume/api/RpcClientConfigurationConstants.java index 33a2330b01..343e07b1d5 100644 --- a/flume-ng-sdk/src/main/java/org/apache/flume/api/RpcClientConfigurationConstants.java +++ b/flume-ng-sdk/src/main/java/org/apache/flume/api/RpcClientConfigurationConstants.java @@ -145,6 +145,8 @@ public final class RpcClientConfigurationConstants { public static final String CONFIG_TRUSTSTORE_TYPE = "truststore-type"; public static final String CONFIG_EXCLUDE_PROTOCOLS = "exclude-protocols"; + public static final String KERBEROS_KEY = "kerberos"; + /** * Configuration constants for the NettyAvroRpcClient * NioClientSocketChannelFactory diff --git a/flume-ng-sdk/src/main/java/org/apache/flume/api/ThriftRpcClient.java b/flume-ng-sdk/src/main/java/org/apache/flume/api/ThriftRpcClient.java index 4f75a2b464..5c4cc4192a 100644 --- a/flume-ng-sdk/src/main/java/org/apache/flume/api/ThriftRpcClient.java +++ b/flume-ng-sdk/src/main/java/org/apache/flume/api/ThriftRpcClient.java @@ -28,6 +28,7 @@ import org.apache.thrift.protocol.TCompactProtocol; import org.apache.thrift.transport.TFastFramedTransport; import org.apache.thrift.transport.TSocket; +import org.apache.thrift.transport.TTransport; import org.slf4j.Logger; import org.slf4j.LoggerFactory; @@ -73,7 +74,7 @@ public class ThriftRpcClient extends AbstractRpcClient { public static final String CONFIG_PROTOCOL = "protocol"; public static final String BINARY_PROTOCOL = "binary"; public static final String COMPACT_PROTOCOL = "compact"; - + private int batchSize; private long requestTimeout; private final Lock stateLock; @@ -83,7 +84,6 @@ public class ThriftRpcClient extends AbstractRpcClient { private ConnectionPoolManager connectionManager; private final ExecutorService callTimeoutPool; private final AtomicLong threadCounter; - private int connectionPoolSize; private final Random random = new Random(); private String protocol; @@ -95,7 +95,6 @@ public class ThriftRpcClient extends AbstractRpcClient { private static final String TRUSTMANAGER_TYPE = "trustmanager-type"; private final List excludeProtocols = new LinkedList(); - public ThriftRpcClient() { stateLock = new ReentrantLock(true); connState = State.INIT; @@ -319,7 +318,7 @@ protected void configure(Properties properties) throws FlumeException { requestTimeout = RpcClientConfigurationConstants.DEFAULT_REQUEST_TIMEOUT_MILLIS; } - connectionPoolSize = Integer.parseInt(properties.getProperty( + int connectionPoolSize = Integer.parseInt(properties.getProperty( RpcClientConfigurationConstants.CONFIG_CONNECTION_POOL_SIZE, String.valueOf(RpcClientConfigurationConstants .DEFAULT_CONNECTION_POOL_SIZE))); @@ -352,6 +351,7 @@ protected void configure(Properties properties) throws FlumeException { } } } + connectionManager = new ConnectionPoolManager(connectionPoolSize); connState = State.READY; } catch (Throwable ex) { @@ -372,33 +372,41 @@ private static enum State { INIT, READY, DEAD } + protected TTransport getTransport(TSocket tsocket) throws Exception { + return new TFastFramedTransport(tsocket); + } + /** * Wrapper around a client and transport, so we can clean up when this * client gets closed. */ private class ClientWrapper { public final ThriftSourceProtocol.Client client; - public final TFastFramedTransport transport; + public final TTransport transport; private final int hashCode; public ClientWrapper() throws Exception{ TSocket tsocket; if(enableSsl) { - // JDK6's factory doesn't appear to pass the protocol onto the Socket properly so we have - // to do some magic to make sure that happens. Not an issue in JDK7 - // Lifted from thrift-0.9.1 to make the SSLContext - SSLContext sslContext = createSSLContext(truststore, truststorePassword, trustManagerType, truststoreType); + // JDK6's factory doesn't appear to pass the protocol onto the Socket + // properly so we have to do some magic to make sure that happens. + // Not an issue in JDK7 Lifted from thrift-0.9.1 to make the SSLContext + SSLContext sslContext = createSSLContext(truststore, truststorePassword, + trustManagerType, truststoreType); // Create the factory from it SSLSocketFactory sslSockFactory = sslContext.getSocketFactory(); // Create the TSocket from that - tsocket = createSSLSocket(sslSockFactory, hostname, port, 120000, excludeProtocols); + tsocket = createSSLSocket( + sslSockFactory, hostname, port, 120000, excludeProtocols); } else { tsocket = new TSocket(hostname, port); } - transport = new TFastFramedTransport(tsocket); + + transport = getTransport(tsocket); + // The transport is already open for SSL as part of TSSLTransportFactory.getClientSocket if(!transport.isOpen()) { transport.open(); diff --git a/flume-ng-sinks/flume-dataset-sink/pom.xml b/flume-ng-sinks/flume-dataset-sink/pom.xml index ad3f603537..92f7021655 100644 --- a/flume-ng-sinks/flume-dataset-sink/pom.xml +++ b/flume-ng-sinks/flume-dataset-sink/pom.xml @@ -149,13 +149,6 @@ limitations under the License. test - - org.apache.hadoop - hadoop-minikdc - ${hadoop2.version} - test - - org.slf4j slf4j-log4j12 diff --git a/flume-ng-sinks/flume-dataset-sink/src/main/java/org/apache/flume/sink/kite/DatasetSink.java b/flume-ng-sinks/flume-dataset-sink/src/main/java/org/apache/flume/sink/kite/DatasetSink.java index fd9f99113a..a9f42b8992 100644 --- a/flume-ng-sinks/flume-dataset-sink/src/main/java/org/apache/flume/sink/kite/DatasetSink.java +++ b/flume-ng-sinks/flume-dataset-sink/src/main/java/org/apache/flume/sink/kite/DatasetSink.java @@ -17,6 +17,8 @@ */ package org.apache.flume.sink.kite; +import org.apache.flume.auth.FlumeAuthenticationUtil; +import org.apache.flume.auth.PrivilegedExecutor; import org.apache.flume.sink.kite.parser.EntityParserFactory; import org.apache.flume.sink.kite.parser.EntityParser; import org.apache.flume.sink.kite.policy.FailurePolicy; @@ -25,8 +27,9 @@ import com.google.common.base.Preconditions; import com.google.common.base.Throwables; import com.google.common.collect.Lists; + import java.net.URI; -import java.security.PrivilegedExceptionAction; +import java.security.PrivilegedAction; import java.util.List; import java.util.concurrent.TimeUnit; import org.apache.avro.Schema; @@ -40,7 +43,6 @@ import org.apache.flume.conf.Configurable; import org.apache.flume.instrumentation.SinkCounter; import org.apache.flume.sink.AbstractSink; -import org.apache.hadoop.security.UserGroupInformation; import org.kitesdk.data.Dataset; import org.kitesdk.data.DatasetDescriptor; import org.kitesdk.data.DatasetIOException; @@ -72,7 +74,7 @@ public class DatasetSink extends AbstractSink implements Configurable { private static final Logger LOG = LoggerFactory.getLogger(DatasetSink.class); private Context context = null; - private UserGroupInformation login = null; + private PrivilegedExecutor privilegedExecutor; private String datasetName = null; private URI datasetUri = null; @@ -159,15 +161,12 @@ protected List allowedFormats() { public void configure(Context context) { this.context = context; - // initialize login credentials - this.login = KerberosUtil.login( - context.getString(AUTH_PRINCIPAL), - context.getString(AUTH_KEYTAB)); - String effectiveUser - = context.getString(AUTH_PROXY_USER); - if (effectiveUser != null) { - this.login = KerberosUtil.proxyAs(effectiveUser, login); - } + String principal = context.getString(AUTH_PRINCIPAL); + String keytab = context.getString(AUTH_KEYTAB); + String effectiveUser = context.getString(AUTH_PROXY_USER); + + this.privilegedExecutor = FlumeAuthenticationUtil.getAuthenticator( + principal, keytab).proxyAs(effectiveUser); // Get the dataset URI and name from the context String datasetURI = context.getString(CONFIG_KITE_DATASET_URI); @@ -395,13 +394,15 @@ void createWriter() throws EventDeliveryException { // reset the commited flag whenver a new writer is created committedBatch = false; try { - View view = KerberosUtil.runPrivileged(login, - new PrivilegedExceptionAction>() { - @Override - public Dataset run() { - return Datasets.load(datasetUri); - } - }); + View view; + + view = privilegedExecutor.execute( + new PrivilegedAction>() { + @Override + public Dataset run() { + return Datasets.load(datasetUri); + } + }); DatasetDescriptor descriptor = view.getDataset().getDescriptor(); Format format = descriptor.getFormat(); diff --git a/flume-ng-sinks/flume-dataset-sink/src/main/java/org/apache/flume/sink/kite/KerberosUtil.java b/flume-ng-sinks/flume-dataset-sink/src/main/java/org/apache/flume/sink/kite/KerberosUtil.java deleted file mode 100644 index c0dbffbd64..0000000000 --- a/flume-ng-sinks/flume-dataset-sink/src/main/java/org/apache/flume/sink/kite/KerberosUtil.java +++ /dev/null @@ -1,187 +0,0 @@ -/** - * Licensed to the Apache Software Foundation (ASF) under one - * or more contributor license agreements. See the NOTICE file - * distributed with this work for additional information - * regarding copyright ownership. The ASF licenses this file - * to you under the Apache License, Version 2.0 (the - * "License"); you may not use this file except in compliance - * with the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. - */ - -package org.apache.flume.sink.kite; - -import com.google.common.base.Preconditions; -import com.google.common.base.Throwables; -import java.io.File; -import java.io.IOException; -import java.security.PrivilegedExceptionAction; -import org.apache.hadoop.security.SecurityUtil; -import org.apache.hadoop.security.UserGroupInformation; -import org.kitesdk.data.DatasetException; -import org.kitesdk.data.DatasetIOException; -import org.slf4j.Logger; -import org.slf4j.LoggerFactory; - -public class KerberosUtil { - - private static final Logger LOG = LoggerFactory.getLogger(KerberosUtil.class); - - public static class SecurityException extends RuntimeException { - private SecurityException(String message) { - super(message); - } - - private SecurityException(String message, Throwable cause) { - super(message, cause); - } - - private SecurityException(Throwable cause) { - super(cause); - } - } - - public static UserGroupInformation proxyAs(String username, - UserGroupInformation login) { - Preconditions.checkArgument(username != null && !username.isEmpty(), - "Invalid username: " + String.valueOf(username)); - Preconditions.checkArgument(login != null, - "Cannot proxy without an authenticated user"); - - // hadoop impersonation works with or without kerberos security - return UserGroupInformation.createProxyUser(username, login); - } - - /** - * Static synchronized method for static Kerberos login.
    - * Static synchronized due to a thundering herd problem when multiple Sinks - * attempt to log in using the same principal at the same time with the - * intention of impersonating different users (or even the same user). - * If this is not controlled, MIT Kerberos v5 believes it is seeing a replay - * attach and it returns: - *
    Request is a replay (34) - PROCESS_TGS
    - * In addition, since the underlying Hadoop APIs we are using for - * impersonation are static, we define this method as static as well. - * - * @param principal - * Fully-qualified principal to use for authentication. - * @param keytab - * Location of keytab file containing credentials for principal. - * @return Logged-in user - * @throws SecurityException - * if login fails. - * @throws IllegalArgumentException - * if the principal or the keytab is not usable - */ - public static synchronized UserGroupInformation login(String principal, - String keytab) { - // If the principal or keytab isn't set, get the current (Linux) user - if (principal == null || keytab == null) { - try { - return UserGroupInformation.getCurrentUser(); - } catch (IOException ex) { - LOG.error("Can't get current user: {}", ex.getMessage()); - throw new RuntimeException(ex); - } - } - - // resolve the requested principal, if it is present - String finalPrincipal = null; - if (principal != null && !principal.isEmpty()) { - try { - // resolves _HOST pattern using standard Hadoop search/replace - // via DNS lookup when 2nd argument is empty - finalPrincipal = SecurityUtil.getServerPrincipal(principal, ""); - } catch (IOException e) { - throw new SecurityException( - "Failed to resolve Kerberos principal", e); - } - } - - // check if there is a user already logged in - UserGroupInformation currentUser = null; - try { - currentUser = UserGroupInformation.getLoginUser(); - } catch (IOException e) { - // not a big deal but this shouldn't typically happen because it will - // generally fall back to the UNIX user - LOG.debug("Unable to get login user before Kerberos auth attempt", e); - } - - // if the current user is valid (matches the given principal and has a TGT) - // then use it - if (currentUser != null && currentUser.hasKerberosCredentials()) { - if (finalPrincipal == null || - finalPrincipal.equals(currentUser.getUserName())) { - LOG.debug("Using existing login for {}: {}", - finalPrincipal, currentUser); - return currentUser; - } else { - // be cruel and unusual when user tries to login as multiple principals - // this isn't really valid with a reconfigure but this should be rare - // enough to warrant a restart of the agent JVM - // TODO: find a way to interrogate the entire current config state, - // since we don't have to be unnecessarily protective if they switch all - // HDFS sinks to use a different principal all at once. - throw new SecurityException( - "Cannot use multiple Kerberos principals: " + finalPrincipal + - " would replace " + currentUser.getUserName()); - } - } - - // prepare for a new login - Preconditions.checkArgument(principal != null && !principal.isEmpty(), - "Invalid Kerberos principal: " + String.valueOf(principal)); - Preconditions.checkNotNull(finalPrincipal, - "Resolved principal must not be null"); - Preconditions.checkArgument(keytab != null && !keytab.isEmpty(), - "Invalid Kerberos keytab: " + String.valueOf(keytab)); - File keytabFile = new File(keytab); - Preconditions.checkArgument(keytabFile.isFile() && keytabFile.canRead(), - "Keytab is not a readable file: " + String.valueOf(keytab)); - - try { - // attempt static kerberos login - LOG.debug("Logging in as {} with {}", finalPrincipal, keytab); - UserGroupInformation.loginUserFromKeytab(principal, keytab); - return UserGroupInformation.getLoginUser(); - } catch (IOException e) { - throw new SecurityException("Kerberos login failed", e); - } - } - - /** - * Allow methods to act with the privileges of a login. - * - * If the login is null, the current privileges will be used. - * - * @param The return type of the action - * @param login UserGroupInformation credentials to use for action - * @param action A PrivilegedExceptionAction to perform as another user - * @return the T value returned by action.run() - */ - public static T runPrivileged(UserGroupInformation login, - PrivilegedExceptionAction action) { - try { - if (login == null) { - return action.run(); - } else { - return login.doAs(action); - } - } catch (IOException ex) { - throw new DatasetIOException("Privileged action failed", ex); - } catch (InterruptedException ex) { - Thread.interrupted(); - throw new DatasetException(ex); - } catch (Exception ex) { - throw Throwables.propagate(ex); - } - } -} diff --git a/flume-ng-sinks/flume-hdfs-sink/src/main/java/org/apache/flume/sink/hdfs/BucketWriter.java b/flume-ng-sinks/flume-hdfs-sink/src/main/java/org/apache/flume/sink/hdfs/BucketWriter.java index 62f4eee875..6b97de6e12 100644 --- a/flume-ng-sinks/flume-hdfs-sink/src/main/java/org/apache/flume/sink/hdfs/BucketWriter.java +++ b/flume-ng-sinks/flume-hdfs-sink/src/main/java/org/apache/flume/sink/hdfs/BucketWriter.java @@ -38,6 +38,7 @@ import org.apache.flume.Context; import org.apache.flume.Event; import org.apache.flume.SystemClock; +import org.apache.flume.auth.PrivilegedExecutor; import org.apache.flume.instrumentation.SinkCounter; import org.apache.flume.sink.hdfs.HDFSEventSink.WriterCallback; import org.apache.hadoop.conf.Configuration; @@ -45,7 +46,6 @@ import org.apache.hadoop.fs.Path; import org.apache.hadoop.io.SequenceFile.CompressionType; import org.apache.hadoop.io.compress.CompressionCodec; -import org.apache.hadoop.security.UserGroupInformation; import org.slf4j.Logger; import org.slf4j.LoggerFactory; @@ -75,7 +75,7 @@ class BucketWriter { private final CompressionCodec codeC; private final CompressionType compType; private final ScheduledExecutorService timedRollerPool; - private final UserGroupInformation user; + private final PrivilegedExecutor proxyUser; private final AtomicLong fileExtensionCounter; @@ -120,7 +120,7 @@ class BucketWriter { Context context, String filePath, String fileName, String inUsePrefix, String inUseSuffix, String fileSuffix, CompressionCodec codeC, CompressionType compType, HDFSWriter writer, - ScheduledExecutorService timedRollerPool, UserGroupInformation user, + ScheduledExecutorService timedRollerPool, PrivilegedExecutor proxyUser, SinkCounter sinkCounter, int idleTimeout, WriterCallback onCloseCallback, String onCloseCallbackPath, long callTimeout, ExecutorService callTimeoutPool, long retryInterval, @@ -138,7 +138,7 @@ class BucketWriter { this.compType = compType; this.writer = writer; this.timedRollerPool = timedRollerPool; - this.user = user; + this.proxyUser = proxyUser; this.sinkCounter = sinkCounter; this.idleTimeout = idleTimeout; this.onCloseCallback = onCloseCallback; @@ -165,33 +165,6 @@ void setMockStream(HDFSWriter dataWriter) { this.writer = dataWriter; } - /** - * Allow methods to act as another user (typically used for HDFS Kerberos) - * @param - * @param action - * @return - * @throws IOException - * @throws InterruptedException - */ - private T runPrivileged(final PrivilegedExceptionAction action) - throws IOException, InterruptedException { - - if (user != null) { - return user.doAs(action); - } else { - try { - return action.run(); - } catch (IOException ex) { - throw ex; - } catch (InterruptedException ex) { - throw ex; - } catch (RuntimeException ex) { - throw ex; - } catch (Exception ex) { - throw new RuntimeException("Unexpected exception.", ex); - } - } - } /** * Clear the class counters @@ -700,7 +673,7 @@ private T callWithTimeout(final CallRunner callRunner) Future future = callTimeoutPool.submit(new Callable() { @Override public T call() throws Exception { - return runPrivileged(new PrivilegedExceptionAction() { + return proxyUser.execute(new PrivilegedExceptionAction() { @Override public T run() throws Exception { return callRunner.call(); diff --git a/flume-ng-sinks/flume-hdfs-sink/src/main/java/org/apache/flume/sink/hdfs/HDFSEventSink.java b/flume-ng-sinks/flume-hdfs-sink/src/main/java/org/apache/flume/sink/hdfs/HDFSEventSink.java index 33f73a9cf0..9a48841c4c 100644 --- a/flume-ng-sinks/flume-hdfs-sink/src/main/java/org/apache/flume/sink/hdfs/HDFSEventSink.java +++ b/flume-ng-sinks/flume-hdfs-sink/src/main/java/org/apache/flume/sink/hdfs/HDFSEventSink.java @@ -18,7 +18,6 @@ package org.apache.flume.sink.hdfs; -import java.io.File; import java.io.IOException; import java.util.ArrayList; import java.util.Calendar; @@ -31,7 +30,6 @@ import java.util.concurrent.Executors; import java.util.concurrent.ScheduledExecutorService; import java.util.concurrent.TimeUnit; -import java.util.concurrent.atomic.AtomicReference; import com.google.common.annotations.VisibleForTesting; import org.apache.flume.Channel; @@ -41,6 +39,9 @@ import org.apache.flume.EventDeliveryException; import org.apache.flume.SystemClock; import org.apache.flume.Transaction; +import org.apache.flume.auth.FlumeAuthenticationUtil; +import org.apache.flume.auth.FlumeAuthenticator; +import org.apache.flume.auth.PrivilegedExecutor; import org.apache.flume.conf.Configurable; import org.apache.flume.formatter.output.BucketPath; import org.apache.flume.instrumentation.SinkCounter; @@ -50,9 +51,6 @@ import org.apache.hadoop.io.SequenceFile.CompressionType; import org.apache.hadoop.io.compress.CompressionCodec; import org.apache.hadoop.io.compress.CompressionCodecFactory; -import org.apache.hadoop.security.SecurityUtil; -import org.apache.hadoop.security.UserGroupInformation; -import org.apache.hadoop.security.UserGroupInformation.AuthenticationMethod; import org.slf4j.Logger; import org.slf4j.LoggerFactory; @@ -100,12 +98,6 @@ public interface WriterCallback { private static final int defaultThreadPoolSize = 10; private static final int defaultRollTimerPoolSize = 1; - /** - * Singleton credential manager that manages static credentials for the - * entire JVM - */ - private static final AtomicReference staticLogin - = new AtomicReference(); private final HDFSWriterFactory writerFactory; private WriterLinkedHashMap sfWriters; @@ -129,11 +121,6 @@ public interface WriterCallback { private ExecutorService callTimeoutPool; private ScheduledExecutorService timedRollerPool; - private String kerbConfPrincipal; - private String kerbKeytab; - private String proxyUserName; - private UserGroupInformation proxyTicket; - private boolean needRounding = false; private int roundUnit = Calendar.SECOND; private int roundValue = 1; @@ -150,6 +137,7 @@ public interface WriterCallback { private final Object sfWritersLock = new Object(); private long retryInterval; private int tryCount; + private PrivilegedExecutor privExecutor; /* @@ -225,9 +213,9 @@ public void configure(Context context) { defaultThreadPoolSize); rollTimerPoolSize = context.getInteger("hdfs.rollTimerPoolSize", defaultRollTimerPoolSize); - kerbConfPrincipal = context.getString("hdfs.kerberosPrincipal", ""); - kerbKeytab = context.getString("hdfs.kerberosKeytab", ""); - proxyUserName = context.getString("hdfs.proxyUser", ""); + String kerbConfPrincipal = context.getString("hdfs.kerberosPrincipal"); + String kerbKeytab = context.getString("hdfs.kerberosKeytab"); + String proxyUser = context.getString("hdfs.proxyUser"); tryCount = context.getInteger("hdfs.closeTries", defaultTryCount); if(tryCount <= 0) { LOG.warn("Retry count value : " + tryCount + " is not " + @@ -269,9 +257,13 @@ public void configure(Context context) { + " when fileType is: " + fileType); } - if (!authenticate()) { - LOG.error("Failed to authenticate!"); - } + // get the appropriate executor + this.privExecutor = FlumeAuthenticationUtil.getAuthenticator( + kerbConfPrincipal, kerbKeytab).proxyAs(proxyUser); + + + + needRounding = context.getBoolean("hdfs.round", false); if(needRounding) { @@ -482,7 +474,7 @@ private BucketWriter initializeBucketWriter(String realPath, rollSize, rollCount, batchSize, context, realPath, realName, inUsePrefix, inUseSuffix, suffix, codeC, compType, hdfsWriter, timedRollerPool, - proxyTicket, sinkCounter, idleTimeout, closeCallback, + privExecutor, sinkCounter, idleTimeout, closeCallback, lookupPath, callTimeout, callTimeoutPool, retryInterval, tryCount); if(mockFs != null) { @@ -551,197 +543,6 @@ public void start() { super.start(); } - private boolean authenticate() { - - // logic for kerberos login - boolean useSecurity = UserGroupInformation.isSecurityEnabled(); - - LOG.info("Hadoop Security enabled: " + useSecurity); - - if (useSecurity) { - - // sanity checking - if (kerbConfPrincipal.isEmpty()) { - LOG.error("Hadoop running in secure mode, but Flume config doesn't " - + "specify a principal to use for Kerberos auth."); - return false; - } - if (kerbKeytab.isEmpty()) { - LOG.error("Hadoop running in secure mode, but Flume config doesn't " - + "specify a keytab to use for Kerberos auth."); - return false; - } else { - //If keytab is specified, user should want it take effect. - //HDFSEventSink will halt when keytab file is non-exist or unreadable - File kfile = new File(kerbKeytab); - if (!(kfile.isFile() && kfile.canRead())) { - throw new IllegalArgumentException("The keyTab file: " - + kerbKeytab + " is nonexistent or can't read. " - + "Please specify a readable keytab file for Kerberos auth."); - } - } - - String principal; - try { - // resolves _HOST pattern using standard Hadoop search/replace - // via DNS lookup when 2nd argument is empty - principal = SecurityUtil.getServerPrincipal(kerbConfPrincipal, ""); - } catch (IOException e) { - LOG.error("Host lookup error resolving kerberos principal (" - + kerbConfPrincipal + "). Exception follows.", e); - return false; - } - - Preconditions.checkNotNull(principal, "Principal must not be null"); - KerberosUser prevUser = staticLogin.get(); - KerberosUser newUser = new KerberosUser(principal, kerbKeytab); - - // be cruel and unusual when user tries to login as multiple principals - // this isn't really valid with a reconfigure but this should be rare - // enough to warrant a restart of the agent JVM - // TODO: find a way to interrogate the entire current config state, - // since we don't have to be unnecessarily protective if they switch all - // HDFS sinks to use a different principal all at once. - Preconditions.checkState(prevUser == null || prevUser.equals(newUser), - "Cannot use multiple kerberos principals in the same agent. " + - " Must restart agent to use new principal or keytab. " + - "Previous = %s, New = %s", prevUser, newUser); - - // attempt to use cached credential if the user is the same - // this is polite and should avoid flooding the KDC with auth requests - UserGroupInformation curUser = null; - if (prevUser != null && prevUser.equals(newUser)) { - try { - curUser = UserGroupInformation.getLoginUser(); - } catch (IOException e) { - LOG.warn("User unexpectedly had no active login. Continuing with " + - "authentication", e); - } - } - - if (curUser == null || !curUser.getUserName().equals(principal)) { - try { - // static login - kerberosLogin(this, principal, kerbKeytab); - } catch (IOException e) { - LOG.error("Authentication or file read error while attempting to " - + "login as kerberos principal (" + principal + ") using " - + "keytab (" + kerbKeytab + "). Exception follows.", e); - return false; - } - } else { - LOG.debug("{}: Using existing principal login: {}", this, curUser); - } - - // we supposedly got through this unscathed... so store the static user - staticLogin.set(newUser); - } - - // hadoop impersonation works with or without kerberos security - proxyTicket = null; - if (!proxyUserName.isEmpty()) { - try { - proxyTicket = UserGroupInformation.createProxyUser( - proxyUserName, UserGroupInformation.getLoginUser()); - } catch (IOException e) { - LOG.error("Unable to login as proxy user. Exception follows.", e); - return false; - } - } - - UserGroupInformation ugi = null; - if (proxyTicket != null) { - ugi = proxyTicket; - } else if (useSecurity) { - try { - ugi = UserGroupInformation.getLoginUser(); - } catch (IOException e) { - LOG.error("Unexpected error: Unable to get authenticated user after " + - "apparent successful login! Exception follows.", e); - return false; - } - } - - if (ugi != null) { - // dump login information - AuthenticationMethod authMethod = ugi.getAuthenticationMethod(); - LOG.info("Auth method: {}", authMethod); - LOG.info(" User name: {}", ugi.getUserName()); - LOG.info(" Using keytab: {}", ugi.isFromKeytab()); - if (authMethod == AuthenticationMethod.PROXY) { - UserGroupInformation superUser; - try { - superUser = UserGroupInformation.getLoginUser(); - LOG.info(" Superuser auth: {}", superUser.getAuthenticationMethod()); - LOG.info(" Superuser name: {}", superUser.getUserName()); - LOG.info(" Superuser using keytab: {}", superUser.isFromKeytab()); - } catch (IOException e) { - LOG.error("Unexpected error: unknown superuser impersonating proxy.", - e); - return false; - } - } - - LOG.info("Logged in as user {}", ugi.getUserName()); - - return true; - } - - return true; - } - - /** - * Static synchronized method for static Kerberos login.
    - * Static synchronized due to a thundering herd problem when multiple Sinks - * attempt to log in using the same principal at the same time with the - * intention of impersonating different users (or even the same user). - * If this is not controlled, MIT Kerberos v5 believes it is seeing a replay - * attach and it returns: - *
    Request is a replay (34) - PROCESS_TGS
    - * In addition, since the underlying Hadoop APIs we are using for - * impersonation are static, we define this method as static as well. - * - * @param principal - * Fully-qualified principal to use for authentication. - * @param keytab - * Location of keytab file containing credentials for principal. - * @return Logged-in user - * @throws IOException - * if login fails. - */ - private static synchronized UserGroupInformation kerberosLogin( - HDFSEventSink sink, String principal, String keytab) throws IOException { - - // if we are the 2nd user thru the lock, the login should already be - // available statically if login was successful - UserGroupInformation curUser = null; - try { - curUser = UserGroupInformation.getLoginUser(); - } catch (IOException e) { - // not a big deal but this shouldn't typically happen because it will - // generally fall back to the UNIX user - LOG.debug("Unable to get login user before Kerberos auth attempt.", e); - } - - // we already have logged in successfully - if (curUser != null && curUser.getUserName().equals(principal)) { - LOG.debug("{}: Using existing principal ({}): {}", - new Object[]{sink, principal, curUser}); - - // no principal found - } else { - - LOG.info("{}: Attempting kerberos login as principal ({}) from keytab " + - "file ({})", new Object[]{sink, principal, keytab}); - - // attempt static kerberos login - UserGroupInformation.loginUserFromKeytab(principal, keytab); - curUser = UserGroupInformation.getLoginUser(); - } - - return curUser; - } - @Override public String toString() { return "{ Sink type:" + getClass().getSimpleName() + ", name:" + getName() + diff --git a/flume-ng-sinks/flume-hdfs-sink/src/test/java/org/apache/flume/sink/hdfs/TestBucketWriter.java b/flume-ng-sinks/flume-hdfs-sink/src/test/java/org/apache/flume/sink/hdfs/TestBucketWriter.java index 7c74b162e7..2581f73992 100644 --- a/flume-ng-sinks/flume-hdfs-sink/src/test/java/org/apache/flume/sink/hdfs/TestBucketWriter.java +++ b/flume-ng-sinks/flume-hdfs-sink/src/test/java/org/apache/flume/sink/hdfs/TestBucketWriter.java @@ -29,6 +29,8 @@ import org.apache.flume.Clock; import org.apache.flume.Context; import org.apache.flume.Event; +import org.apache.flume.auth.FlumeAuthenticationUtil; +import org.apache.flume.auth.PrivilegedExecutor; import org.apache.flume.event.EventBuilder; import org.apache.flume.instrumentation.SinkCounter; import org.apache.hadoop.conf.Configuration; @@ -53,10 +55,12 @@ public class TestBucketWriter { private Context ctx = new Context(); private static ScheduledExecutorService timedRollerPool; + private static PrivilegedExecutor proxy; @BeforeClass public static void setup() { timedRollerPool = Executors.newSingleThreadScheduledExecutor(); + proxy = FlumeAuthenticationUtil.getAuthenticator(null, null).proxyAs(null); } @AfterClass @@ -72,7 +76,7 @@ public void testEventCountingRoller() throws IOException, InterruptedException { MockHDFSWriter hdfsWriter = new MockHDFSWriter(); BucketWriter bucketWriter = new BucketWriter(0, 0, maxEvents, 0, ctx, "/tmp", "file", "", ".tmp", null, null, SequenceFile.CompressionType.NONE, - hdfsWriter, timedRollerPool, null, + hdfsWriter, timedRollerPool, proxy, new SinkCounter("test-bucket-writer-" + System.currentTimeMillis()), 0, null, null, 30000, Executors.newSingleThreadExecutor(), 0, 0); @@ -97,7 +101,7 @@ public void testSizeRoller() throws IOException, InterruptedException { BucketWriter bucketWriter = new BucketWriter(0, maxBytes, 0, 0, ctx, "/tmp", "file", "", ".tmp", null, null, SequenceFile.CompressionType.NONE, hdfsWriter,timedRollerPool, - null, new SinkCounter("test-bucket-writer-" + + proxy, new SinkCounter("test-bucket-writer-" + System.currentTimeMillis()),0, null, null, 30000, Executors.newSingleThreadExecutor(), 0, 0); @@ -124,7 +128,7 @@ public void testIntervalRoller() throws IOException, InterruptedException { MockHDFSWriter hdfsWriter = new MockHDFSWriter(); BucketWriter bucketWriter = new BucketWriter(ROLL_INTERVAL, 0, 0, 0, ctx, "/tmp", "file", "", ".tmp", null, null, SequenceFile.CompressionType.NONE, - hdfsWriter, timedRollerPool, null, + hdfsWriter, timedRollerPool, proxy, new SinkCounter("test-bucket-writer-" + System.currentTimeMillis()), 0, new HDFSEventSink.WriterCallback() { @Override @@ -147,7 +151,7 @@ public void run(String filePath) { bucketWriter = new BucketWriter(ROLL_INTERVAL, 0, 0, 0, ctx, "/tmp", "file", "", ".tmp", null, null, SequenceFile.CompressionType.NONE, - hdfsWriter, timedRollerPool, null, + hdfsWriter, timedRollerPool, proxy, new SinkCounter("test-bucket-writer-" + System.currentTimeMillis()), 0, null, null, 30000, Executors.newSingleThreadExecutor(), 0, 0); @@ -230,7 +234,7 @@ public void append(Event e) throws IOException { BucketWriter bucketWriter = new BucketWriter(ROLL_INTERVAL, 0, 0, 0, ctx, path, name, "", ".tmp", null, null, SequenceFile.CompressionType.NONE, hdfsWriter, - timedRollerPool, null, new SinkCounter("test-bucket-writer-" + timedRollerPool, proxy, new SinkCounter("test-bucket-writer-" + System.currentTimeMillis()), 0, null, null, 30000, Executors.newSingleThreadExecutor(), 0, 0); @@ -255,7 +259,7 @@ public void testFileSuffixNotGiven() throws IOException, InterruptedException { BucketWriter bucketWriter = new BucketWriter(ROLL_INTERVAL, 0, 0, 0, ctx, "/tmp", "file", "", ".tmp", suffix, null, SequenceFile.CompressionType.NONE, hdfsWriter, - timedRollerPool, null, new SinkCounter("test-bucket-writer-" + timedRollerPool, proxy, new SinkCounter("test-bucket-writer-" + System.currentTimeMillis()), 0, null, null, 30000, Executors.newSingleThreadExecutor(), 0, 0); @@ -283,7 +287,7 @@ public void testFileSuffixGiven() throws IOException, InterruptedException { BucketWriter bucketWriter = new BucketWriter(ROLL_INTERVAL, 0, 0, 0, ctx, "/tmp", "file", "", ".tmp", suffix, null, SequenceFile.CompressionType.NONE, hdfsWriter, - timedRollerPool, null, new SinkCounter( + timedRollerPool, proxy, new SinkCounter( "test-bucket-writer-" + System.currentTimeMillis()), 0, null, null, 30000, Executors.newSingleThreadExecutor(), 0, 0); @@ -316,7 +320,7 @@ public void testFileSuffixCompressed() 0, ctx, "/tmp", "file", "", ".tmp", suffix, HDFSEventSink.getCodec("gzip"), SequenceFile.CompressionType.BLOCK, hdfsWriter, - timedRollerPool, null, new SinkCounter("test-bucket-writer-" + timedRollerPool, proxy, new SinkCounter("test-bucket-writer-" + System.currentTimeMillis()), 0, null, null, 30000, Executors.newSingleThreadExecutor(), 0, 0 ); @@ -348,7 +352,7 @@ public void testInUsePrefix() throws IOException, InterruptedException { BucketWriter bucketWriter = new BucketWriter(ROLL_INTERVAL, 0, 0, 0, ctx, "/tmp", "file", PREFIX, ".tmp", null, null, SequenceFile.CompressionType.NONE, hdfsWriter, - timedRollerPool, null, new SinkCounter( + timedRollerPool, proxy, new SinkCounter( "test-bucket-writer-" + System.currentTimeMillis()), 0, null, null, 30000, Executors.newSingleThreadExecutor(), 0, 0); @@ -368,7 +372,7 @@ public void testInUseSuffix() throws IOException, InterruptedException { BucketWriter bucketWriter = new BucketWriter(ROLL_INTERVAL, 0, 0, 0, ctx, "/tmp", "file", "", SUFFIX, null, null, SequenceFile.CompressionType.NONE, hdfsWriter, - timedRollerPool, null, new SinkCounter( + timedRollerPool, proxy, new SinkCounter( "test-bucket-writer-" + System.currentTimeMillis()), 0, null, null, 30000, Executors.newSingleThreadExecutor(), 0, 0); @@ -388,7 +392,7 @@ public void testCallbackOnClose() throws IOException, InterruptedException { BucketWriter bucketWriter = new BucketWriter(ROLL_INTERVAL, 0, 0, 0, ctx, "/tmp", "file", "", SUFFIX, null, null, SequenceFile.CompressionType.NONE, - hdfsWriter, timedRollerPool, null, + hdfsWriter, timedRollerPool, proxy, new SinkCounter( "test-bucket-writer-" + System.currentTimeMillis()), 0, new HDFSEventSink.WriterCallback() { @@ -442,7 +446,7 @@ public void SequenceFileRenameRetryCoreTest(int numberOfRetriesRequired, boolean BucketWriter bucketWriter = new BucketWriter(0, 0, 1, 1, ctx, hdfsPath, hdfsPath, "singleBucket", ".tmp", null, null, null, new MockDataStream(mockFs), - timedRollerPool, null, + timedRollerPool, proxy, new SinkCounter( "test-bucket-writer-" + System.currentTimeMillis()), 0, null, null, 30000, Executors.newSingleThreadExecutor(), 1, diff --git a/flume-ng-sinks/flume-hdfs-sink/src/test/java/org/apache/flume/sink/hdfs/TestHDFSEventSink.java b/flume-ng-sinks/flume-hdfs-sink/src/test/java/org/apache/flume/sink/hdfs/TestHDFSEventSink.java index 1b7a364716..23862eb44e 100644 --- a/flume-ng-sinks/flume-hdfs-sink/src/test/java/org/apache/flume/sink/hdfs/TestHDFSEventSink.java +++ b/flume-ng-sinks/flume-hdfs-sink/src/test/java/org/apache/flume/sink/hdfs/TestHDFSEventSink.java @@ -276,7 +276,7 @@ public void testKerbFileAccess() throws InterruptedException, Assert.fail("no exception thrown"); } catch (IllegalArgumentException expected) { Assert.assertTrue(expected.getMessage().contains( - "is nonexistent or can't read.")); + "Keytab is not a readable file")); } finally { //turn security off conf.set(CommonConfigurationKeys.HADOOP_SECURITY_AUTHENTICATION, diff --git a/flume-ng-sinks/flume-ng-hbase-sink/src/main/java/org/apache/flume/sink/hbase/HBaseSink.java b/flume-ng-sinks/flume-ng-hbase-sink/src/main/java/org/apache/flume/sink/hbase/HBaseSink.java index 5de0bd56ef..e659ada928 100644 --- a/flume-ng-sinks/flume-ng-hbase-sink/src/main/java/org/apache/flume/sink/hbase/HBaseSink.java +++ b/flume-ng-sinks/flume-ng-hbase-sink/src/main/java/org/apache/flume/sink/hbase/HBaseSink.java @@ -36,6 +36,8 @@ import org.apache.flume.FlumeException; import org.apache.flume.Transaction; import org.apache.flume.annotations.InterfaceAudience; +import org.apache.flume.auth.FlumeAuthenticationUtil; +import org.apache.flume.auth.PrivilegedExecutor; import org.apache.flume.conf.Configurable; import org.apache.flume.instrumentation.SinkCounter; import org.apache.flume.sink.AbstractSink; @@ -54,7 +56,6 @@ import com.google.common.base.Preconditions; import com.google.common.base.Throwables; import java.security.PrivilegedExceptionAction; -import org.apache.hadoop.hbase.security.User; /** @@ -103,11 +104,11 @@ public class HBaseSink extends AbstractSink implements Configurable { private Context serializerContext; private String kerberosPrincipal; private String kerberosKeytab; - private User hbaseUser; private boolean enableWal = true; private boolean batchIncrements = false; private Method refGetFamilyMap = null; private SinkCounter sinkCounter; + private PrivilegedExecutor privilegedExecutor; // Internal hooks used for unit testing. private DebugIncrementsCallback debugIncrCallback = null; @@ -132,17 +133,14 @@ public void start(){ Preconditions.checkArgument(table == null, "Please call stop " + "before calling start on an old instance."); try { - if (HBaseSinkSecurityManager.isSecurityEnabled(config)) { - hbaseUser = HBaseSinkSecurityManager.login(config, null, - kerberosPrincipal, kerberosKeytab); - } + privilegedExecutor = FlumeAuthenticationUtil.getAuthenticator(kerberosPrincipal, kerberosKeytab); } catch (Exception ex) { sinkCounter.incrementConnectionFailedCount(); throw new FlumeException("Failed to login to HBase using " + "provided credentials.", ex); } try { - table = runPrivileged(new PrivilegedExceptionAction() { + table = privilegedExecutor.execute(new PrivilegedExceptionAction() { @Override public HTable run() throws Exception { HTable table = new HTable(config, tableName); @@ -160,7 +158,7 @@ public HTable run() throws Exception { " from HBase", e); } try { - if (!runPrivileged(new PrivilegedExceptionAction() { + if (!privilegedExecutor.execute(new PrivilegedExceptionAction() { @Override public Boolean run() throws IOException { return table.getTableDescriptor().hasFamily(columnFamily); @@ -233,8 +231,8 @@ public void configure(Context context){ logger.error("Could not instantiate event serializer." , e); Throwables.propagate(e); } - kerberosKeytab = context.getString(HBaseSinkConfigurationConstants.CONFIG_KEYTAB, ""); - kerberosPrincipal = context.getString(HBaseSinkConfigurationConstants.CONFIG_PRINCIPAL, ""); + kerberosKeytab = context.getString(HBaseSinkConfigurationConstants.CONFIG_KEYTAB); + kerberosPrincipal = context.getString(HBaseSinkConfigurationConstants.CONFIG_PRINCIPAL); enableWal = context.getBoolean(HBaseSinkConfigurationConstants .CONFIG_ENABLE_WAL, HBaseSinkConfigurationConstants.DEFAULT_ENABLE_WAL); @@ -371,7 +369,7 @@ public Status process() throws EventDeliveryException { private void putEventsAndCommit(final List actions, final List incs, Transaction txn) throws Exception { - runPrivileged(new PrivilegedExceptionAction() { + privilegedExecutor.execute(new PrivilegedExceptionAction() { @Override public Void run() throws Exception { for (Row r : actions) { @@ -388,7 +386,7 @@ public Void run() throws Exception { } }); - runPrivileged(new PrivilegedExceptionAction() { + privilegedExecutor.execute(new PrivilegedExceptionAction() { @Override public Void run() throws Exception { @@ -416,18 +414,6 @@ public Void run() throws Exception { sinkCounter.addToEventDrainSuccessCount(actions.size()); } - private T runPrivileged(final PrivilegedExceptionAction action) - throws Exception { - if(hbaseUser != null) { - if (logger.isDebugEnabled()) { - logger.debug("Calling runAs as hbase user: " + hbaseUser.getName()); - } - return hbaseUser.runAs(action); - } else { - return action.run(); - } - } - /** * The method getFamilyMap() is no longer available in Hbase 0.96. * We must use reflection to determine which version we may use. diff --git a/flume-ng-sinks/flume-ng-hbase-sink/src/main/java/org/apache/flume/sink/hbase/HBaseSinkSecurityManager.java b/flume-ng-sinks/flume-ng-hbase-sink/src/main/java/org/apache/flume/sink/hbase/HBaseSinkSecurityManager.java deleted file mode 100644 index 762fce98ee..0000000000 --- a/flume-ng-sinks/flume-ng-hbase-sink/src/main/java/org/apache/flume/sink/hbase/HBaseSinkSecurityManager.java +++ /dev/null @@ -1,134 +0,0 @@ -/* - * Licensed to the Apache Software Foundation (ASF) under one - * or more contributor license agreements. See the NOTICE file - * distributed with this work for additional information - * regarding copyright ownership. The ASF licenses this file - * to you under the Apache License, Version 2.0 (the - * "License"); you may not use this file except in compliance - * with the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. - */ -package org.apache.flume.sink.hbase; - -import com.google.common.base.Preconditions; -import java.io.File; -import java.io.IOException; -import java.net.InetAddress; -import org.apache.flume.FlumeException; -import org.apache.flume.sink.hdfs.KerberosUser; -import org.apache.hadoop.conf.Configuration; -import org.apache.hadoop.hbase.security.User; -import org.apache.hadoop.security.SecurityUtil; -import org.apache.hadoop.security.UserGroupInformation; -import org.slf4j.Logger; -import org.slf4j.LoggerFactory; - -/** - * Class to handle logging into HBase with the credentials passed in. - */ -public class HBaseSinkSecurityManager { - - /* - * volatile for safe publication. Since this is updated only by - * a single thread (configuration) and read later by the sink threads, - * this can just be volatile, no need of Atomic reference. - */ - private volatile static KerberosUser loggedInUser; - private static final String FLUME_KEYTAB_KEY = "flume.keytab.key"; - private static final String FLUME_PRINCIPAL_KEY = "flume.principal.key"; - private static final Logger LOG = - LoggerFactory.getLogger(HBaseSinkSecurityManager.class); - - /** - * Checks if security is enabled for the HBase cluster. - * - * @return - true if security is enabled on the HBase cluster and - * the underlying HDFS cluster. - */ - public static boolean isSecurityEnabled(Configuration conf) { - return User.isSecurityEnabled() && User.isHBaseSecurityEnabled(conf); - } - - /** - * Login the user using the configuration, and the hostname specified to use - * for logging in. - * - * @param conf - Configuration to use for logging the user in. - * @param hostname - The hostname to use for logging the user in. If no - * hostname is specified (null or empty string), the canonical hostname for - * the address returned by {@linkplain InetAddress#getLocalHost()} will be - * used. - * @return The logged in HBase {@linkplain User}. - * @throws IOException if login failed, or hostname lookup failed. - */ - public static synchronized User login(Configuration conf, String hostname, - String kerberosPrincipal, String kerberosKeytab) throws IOException { - if (kerberosPrincipal.isEmpty()) { - String msg = "Login failed, since kerberos principal was not specified."; - LOG.error(msg); - throw new IllegalArgumentException(msg); - } - if (kerberosKeytab.isEmpty()) { - String msg = "Login failed, since kerberos keytab was not specified."; - LOG.error(msg); - throw new IllegalArgumentException(msg); - } else { - //If keytab is specified, user should want it take effect. - //HDFSEventSink will halt when keytab file is non-exist or unreadable - File kfile = new File(kerberosKeytab); - if (!(kfile.isFile() && kfile.canRead())) { - throw new IllegalArgumentException("The keyTab file: " - + kerberosKeytab + " is nonexistent or can't read. " - + "Please specify a readable keytab file for Kerberos auth."); - } - } - String principal = kerberosPrincipal; - try { - // resolves _HOST pattern using standard Hadoop search/replace - // via DNS lookup when 2nd argument is empty - principal = SecurityUtil.getServerPrincipal(kerberosPrincipal,""); - } catch (IOException e) { - LOG.error("Host lookup error resolving kerberos principal (" - + kerberosPrincipal + "). Exception follows.", e); - throw e; - } - Preconditions.checkNotNull(principal, "Principal must not be null"); - KerberosUser newUser = new KerberosUser(principal, kerberosKeytab); - //The HDFS Sink does not allow login credentials to change. - //To be uniform, we will do the same thing here. - User hbaseUser = null; - boolean loggedIn = false; - if (loggedInUser != null) { - Preconditions.checkArgument(newUser.equals(loggedInUser), - "Cannot switch kerberos credentials during a reconfiguration. " - + "Please restart the agent to set the new credentials."); - try { - hbaseUser = User.create(UserGroupInformation.getLoginUser()); - loggedIn = true; - } catch (IOException ex) { - LOG.warn("Previous login does not exist, " - + "will authenticate against KDC"); - } - } - if (!loggedIn) { - if (hostname == null || hostname.isEmpty()) { - hostname = InetAddress.getLocalHost().getCanonicalHostName(); - } - conf.set(FLUME_KEYTAB_KEY, kerberosKeytab); - conf.set(FLUME_PRINCIPAL_KEY, principal); - User.login(conf, FLUME_KEYTAB_KEY, FLUME_PRINCIPAL_KEY, hostname); - hbaseUser = User.create(UserGroupInformation.getLoginUser()); - loggedInUser = newUser; - //TODO: Set the loggedInUser to the current user. - LOG.info("Logged into HBase as user: " + hbaseUser.getName()); - } - return hbaseUser; - } -} diff --git a/pom.xml b/pom.xml index 3e405585d9..aad8be6ad8 100644 --- a/pom.xml +++ b/pom.xml @@ -70,6 +70,7 @@ limitations under the License. flume-ng-sdk flume-ng-tests flume-tools + flume-ng-auth @@ -1225,6 +1226,12 @@ limitations under the License. test
    + + org.apache.flume + flume-ng-auth + 1.6.0-SNAPSHOT + + org.apache.flume.flume-ng-clients flume-ng-log4jappender From 49150fe6d4fa11d07a19e3ed106ccb0453a18c84 Mon Sep 17 00:00:00 2001 From: Hari Shreedharan Date: Tue, 10 Mar 2015 13:17:15 -0700 Subject: [PATCH 207/341] FLUME-2639. SecureThriftRpcClient client privileged calls throw IllegalAccessError (Johny Rufus via Hari) --- .../apache/flume/api/SecureThriftRpcClient.java | 16 +++++++++++----- 1 file changed, 11 insertions(+), 5 deletions(-) diff --git a/flume-ng-auth/src/main/java/org/apache/flume/api/SecureThriftRpcClient.java b/flume-ng-auth/src/main/java/org/apache/flume/api/SecureThriftRpcClient.java index 7316e1b54a..f31582c984 100644 --- a/flume-ng-auth/src/main/java/org/apache/flume/api/SecureThriftRpcClient.java +++ b/flume-ng-auth/src/main/java/org/apache/flume/api/SecureThriftRpcClient.java @@ -94,11 +94,9 @@ public void open() throws FlumeException { this.privilegedExecutor.execute( new PrivilegedExceptionAction() { public Void run() throws FlumeException { - try { - UgiSaslClientTransport.super.open(); - } catch (TTransportException e) { - throw new FlumeException("Failed to open SASL transport", e); - } + // this is a workaround to using UgiSaslClientTransport.super.open() + // which results in IllegalAccessError + callSuperClassOpen(); return null; } }); @@ -109,5 +107,13 @@ public Void run() throws FlumeException { throw new FlumeException("Failed to open SASL transport", e); } } + + private void callSuperClassOpen() throws FlumeException { + try { + super.open(); + } catch (TTransportException e) { + throw new FlumeException("Failed to open SASL transport", e); + } + } } } From 39bc4da1ed286fa3690876dda741cadb35386d93 Mon Sep 17 00:00:00 2001 From: Hari Shreedharan Date: Tue, 10 Mar 2015 21:53:31 -0700 Subject: [PATCH 208/341] FLUME-2642. Limit the number of calls to UGI.checkTGTAndRelogin method. (Johny Rufus via Hari) --- .../java/org/apache/flume/auth/UGIExecutor.java | 15 +++++++++++++++ 1 file changed, 15 insertions(+) diff --git a/flume-ng-auth/src/main/java/org/apache/flume/auth/UGIExecutor.java b/flume-ng-auth/src/main/java/org/apache/flume/auth/UGIExecutor.java index a5aeef2eda..cd62b91721 100644 --- a/flume-ng-auth/src/main/java/org/apache/flume/auth/UGIExecutor.java +++ b/flume-ng-auth/src/main/java/org/apache/flume/auth/UGIExecutor.java @@ -27,6 +27,8 @@ class UGIExecutor implements PrivilegedExecutor { private UserGroupInformation ugi; + private static final long MIN_TIME_BEFORE_RELOGIN = 5 * 60 * 1000L; + private volatile long lastReloginAttempt = 0; UGIExecutor(UserGroupInformation ugi) { this.ugi = ugi; @@ -58,9 +60,22 @@ private void ensureValidAuth() { } } + /* + * lastReloginAttempt is introduced to avoid making the synchronized call + * ugi.checkTGTAndReloginFromKeytab() often, Hence this method is + * intentionally not synchronized, so that multiple threads can execute without + * the need to lock, which may result in an edge case where multiple threads + * simultaneously reading the lastReloginAttempt, and finding it > 5 minutes, can + * result in all of them attempting the checkTGT method, which is fine + */ private void reloginUGI(UserGroupInformation ugi) { try { if(ugi.hasKerberosCredentials()) { + long now = System.currentTimeMillis(); + if(now - lastReloginAttempt < MIN_TIME_BEFORE_RELOGIN) { + return; + } + lastReloginAttempt = now; ugi.checkTGTAndReloginFromKeytab(); } } catch (IOException e) { From e60c22a318f7c4fe3868a3ad762e3998f61b49d7 Mon Sep 17 00:00:00 2001 From: Hari Shreedharan Date: Thu, 12 Mar 2015 18:02:47 -0700 Subject: [PATCH 209/341] FLUME-2626. Remove trustmanager-type from Thrift RPC client and Thrift Source. (Johny Rufus via Hari) --- .../java/org/apache/flume/source/ThriftSource.java | 14 ++++++++++---- .../java/org/apache/flume/api/ThriftRpcClient.java | 12 +++--------- 2 files changed, 13 insertions(+), 13 deletions(-) diff --git a/flume-ng-core/src/main/java/org/apache/flume/source/ThriftSource.java b/flume-ng-core/src/main/java/org/apache/flume/source/ThriftSource.java index 1d8bb333b4..7df5ddbaad 100644 --- a/flume-ng-core/src/main/java/org/apache/flume/source/ThriftSource.java +++ b/flume-ng-core/src/main/java/org/apache/flume/source/ThriftSource.java @@ -62,6 +62,7 @@ import java.net.InetSocketAddress; import java.net.ServerSocket; import java.security.KeyStore; +import java.security.Security; import java.util.ArrayList; import java.util.Arrays; import java.util.LinkedList; @@ -103,7 +104,6 @@ public class ThriftSource extends AbstractSource implements Configurable, private static final String KEYSTORE_KEY = "keystore"; private static final String KEYSTORE_PASSWORD_KEY = "keystore-password"; private static final String KEYSTORE_TYPE_KEY = "keystore-type"; - private static final String KEYMANAGER_TYPE = "keymanager-type"; private static final String EXCLUDE_PROTOCOLS = "exclude-protocols"; private static final String KERBEROS_KEY = "kerberos"; @@ -120,7 +120,6 @@ public class ThriftSource extends AbstractSource implements Configurable, private String keystore; private String keystorePassword; private String keystoreType; - private String keyManagerType; private final List excludeProtocols = new LinkedList(); private boolean enableSsl = false; private boolean enableKerberos = false; @@ -165,7 +164,6 @@ public void configure(Context context) { keystore = context.getString(KEYSTORE_KEY); keystorePassword = context.getString(KEYSTORE_PASSWORD_KEY); keystoreType = context.getString(KEYSTORE_TYPE_KEY, "JKS"); - keyManagerType = context.getString(KEYMANAGER_TYPE, KeyManagerFactory.getDefaultAlgorithm()); String excludeProtocolsStr = context.getString(EXCLUDE_PROTOCOLS); if (excludeProtocolsStr == null) { excludeProtocols.add("SSLv3"); @@ -253,12 +251,20 @@ public Object run() { super.start(); } + private String getkeyManagerAlgorithm() { + String algorithm = Security.getProperty( + "ssl.KeyManagerFactory.algorithm"); + return (algorithm != null) ? + algorithm : KeyManagerFactory.getDefaultAlgorithm(); + } + private TServerTransport getSSLServerTransport() { try { TServerTransport transport; TSSLTransportFactory.TSSLTransportParameters params = new TSSLTransportFactory.TSSLTransportParameters(); - params.setKeyStore(keystore, keystorePassword, keyManagerType, keystoreType); + + params.setKeyStore(keystore, keystorePassword, getkeyManagerAlgorithm(), keystoreType); transport = TSSLTransportFactory.getServerSocket( port, 120000, InetAddress.getByName(bindAddress), params); diff --git a/flume-ng-sdk/src/main/java/org/apache/flume/api/ThriftRpcClient.java b/flume-ng-sdk/src/main/java/org/apache/flume/api/ThriftRpcClient.java index 5c4cc4192a..857948f182 100644 --- a/flume-ng-sdk/src/main/java/org/apache/flume/api/ThriftRpcClient.java +++ b/flume-ng-sdk/src/main/java/org/apache/flume/api/ThriftRpcClient.java @@ -91,8 +91,6 @@ public class ThriftRpcClient extends AbstractRpcClient { private String truststore; private String truststorePassword; private String truststoreType; - private String trustManagerType; - private static final String TRUSTMANAGER_TYPE = "trustmanager-type"; private final List excludeProtocols = new LinkedList(); public ThriftRpcClient() { @@ -338,8 +336,6 @@ protected void configure(Properties properties) throws FlumeException { RpcClientConfigurationConstants.CONFIG_TRUSTSTORE_PASSWORD); truststoreType = properties.getProperty( RpcClientConfigurationConstants.CONFIG_TRUSTSTORE_TYPE, "JKS"); - trustManagerType = properties.getProperty( - TRUSTMANAGER_TYPE, TrustManagerFactory.getDefaultAlgorithm()); String excludeProtocolsStr = properties.getProperty( RpcClientConfigurationConstants.CONFIG_EXCLUDE_PROTOCOLS); if (excludeProtocolsStr == null) { @@ -392,7 +388,7 @@ public ClientWrapper() throws Exception{ // properly so we have to do some magic to make sure that happens. // Not an issue in JDK7 Lifted from thrift-0.9.1 to make the SSLContext SSLContext sslContext = createSSLContext(truststore, truststorePassword, - trustManagerType, truststoreType); + truststoreType); // Create the factory from it SSLSocketFactory sslSockFactory = sslContext.getSocketFactory(); @@ -531,13 +527,11 @@ public void closeAll() { * */ private static SSLContext createSSLContext(String truststore, - String truststorePassword, String trustManagerType, - String truststoreType) throws FlumeException { + String truststorePassword, String truststoreType) throws FlumeException { SSLContext ctx; try { ctx = SSLContext.getInstance("TLS"); - TrustManagerFactory tmf = null; - KeyManagerFactory kmf = null; + TrustManagerFactory tmf; tmf = TrustManagerFactory.getInstance(TrustManagerFactory.getDefaultAlgorithm()); KeyStore ts = null; if (truststore != null && truststoreType != null) { From e98bcd4529e462e17680053e9356a6280d807164 Mon Sep 17 00:00:00 2001 From: Jarek Jarcec Cecho Date: Sat, 14 Mar 2015 13:31:03 -0700 Subject: [PATCH 210/341] FLUME-2630: Update documentation for Thrift SRc/Sink SSL support (Johny Rufus via Jarek Jarcec Cecho) --- flume-ng-doc/sphinx/FlumeUserGuide.rst | 37 +++++++++++++++++++++++--- 1 file changed, 34 insertions(+), 3 deletions(-) diff --git a/flume-ng-doc/sphinx/FlumeUserGuide.rst b/flume-ng-doc/sphinx/FlumeUserGuide.rst index 4122cfe970..576cba1c6c 100644 --- a/flume-ng-doc/sphinx/FlumeUserGuide.rst +++ b/flume-ng-doc/sphinx/FlumeUserGuide.rst @@ -742,6 +742,9 @@ Thrift Source Listens on Thrift port and receives events from external Thrift client streams. When paired with the built-in ThriftSink on another (previous hop) Flume agent, it can create tiered collection topologies. +Thrift source can be configured to start in secure mode by enabling kerberos authentication. +agent-principal and agent-keytab are the properties used by the +Thrift source to authenticate to the kerberos KDC. Required properties are in **bold**. ================== =========== =================================================== @@ -756,6 +759,14 @@ selector.type selector.* interceptors -- Space separated list of interceptors interceptors.* +ssl false Set this to true to enable SSL encryption. You must also specify a "keystore" and a "keystore-password". +keystore -- This is the path to a Java keystore file. Required for SSL. +keystore-password -- The password for the Java keystore. Required for SSL. +keystore-type JKS The type of the Java keystore. This can be "JKS" or "PKCS12". +exclude-protocols SSLv3 Space-separated list of SSL/TLS protocols to exclude. SSLv3 will always be excluded in addition to the protocols specified. +kerberos false Set to true to enable kerberos authentication. In kerberos mode, agent-principal and agent-keytab are required for successful authentication. The Thrift source in secure mode, will accept connections only from Thrift clients that have kerberos enabled and are successfully authenticated to the kerberos KDC. +agent-principal -- The kerberos principal used by the Thrift Source to authenticate to the kerberos KDC. +agent-keytab —- The keytab location used by the Thrift Source in combination with the agent-principal to authenticate to the kerberos KDC. ================== =========== =================================================== Example for agent named a1: @@ -1938,6 +1949,12 @@ This sink forms one half of Flume's tiered collection support. Flume events sent to this sink are turned into Thrift events and sent to the configured hostname / port pair. The events are taken from the configured Channel in batches of the configured batch size. + +Thrift sink can be configured to start in secure mode by enabling kerberos authentication. +To communicate with a Thrift source started in secure mode, the Thrift sink should also +operate in secure mode. client-principal and client-keytab are the properties used by the +Thrift sink to authenticate to the kerberos KDC. The server-principal represents the +principal of the Thrift source this sink is configured to connect to in secure mode. Required properties are in **bold**. ========================== ======= ============================================== @@ -1951,6 +1968,15 @@ batch-size 100 number of event to batch together for send connect-timeout 20000 Amount of time (ms) to allow for the first (handshake) request. request-timeout 20000 Amount of time (ms) to allow for requests after the first. connection-reset-interval none Amount of time (s) before the connection to the next hop is reset. This will force the Thrift Sink to reconnect to the next hop. This will allow the sink to connect to hosts behind a hardware load-balancer when news hosts are added without having to restart the agent. +ssl false Set to true to enable SSL for this ThriftSink. When configuring SSL, you can optionally set a "truststore", "truststore-password" and "truststore-type" +truststore -- The path to a custom Java truststore file. Flume uses the certificate authority information in this file to determine whether the remote Thrift Source's SSL authentication credentials should be trusted. If not specified, the default Java JSSE certificate authority files (typically "jssecacerts" or "cacerts" in the Oracle JRE) will be used. +truststore-password -- The password for the specified truststore. +truststore-type JKS The type of the Java truststore. This can be "JKS" or other supported Java truststore type. +exclude-protocols SSLv3 Space-separated list of SSL/TLS protocols to exclude +kerberos false Set to true to enable kerberos authentication. In kerberos mode, client-principal, client-keytab and server-principal are required for successful authentication and communication to a kerberos enabled Thrift Source. +client-principal —- The kerberos principal used by the Thrift Sink to authenticate to the kerberos KDC. +client-keytab —- The keytab location used by the Thrift Sink in combination with the client-principal to authenticate to the kerberos KDC. +server-principal -- The kerberos principal of the Thrift Source to which the Thrift Sink is configured to connect to. ========================== ======= ============================================== Example for agent named a1: @@ -3497,9 +3523,14 @@ Sample log4j.properties file configured using backoff: Security ======== -The HDFS sink supports Kerberos authentication if the underlying HDFS is -running in secure mode. Please refer to the HDFS Sink section for -configuring the HDFS sink Kerberos-related options. +The HDFS sink, HBase sink, Thrift source, Thrift sink and Kite Dataset sink all support +Kerberos authentication. Please refer to the corresponding sections for +configuring the Kerberos-related options. + +Flume agent will authenticate to the kerberos KDC as a single principal, which will be +used by different components that require kerberos authentication. The principal and +keytab configured for Thrift source, Thrift sink, HDFS sink, HBase sink and DataSet sink +should be the same, otherwise the component will fail to start. Monitoring ========== From c9289c083c8dc66a103119f2ade1879e03acaed6 Mon Sep 17 00:00:00 2001 From: Roshan Naik Date: Wed, 1 Apr 2015 13:26:40 -0700 Subject: [PATCH 211/341] FLUME-2624 : Improve Hive Sink performance (Deepesh Khandelwal via Roshan Naik) --- .../flume-hive-sink/metastore_db/db.lck | Bin 38 -> 0 bytes .../flume-hive-sink/metastore_db/dbex.lck | Bin 4 -> 0 bytes .../flume-hive-sink/metastore_db/log/log.ctrl | Bin 48 -> 0 bytes .../flume-hive-sink/metastore_db/log/log3.dat | Bin 1048591 -> 0 bytes .../flume-hive-sink/metastore_db/log/log4.dat | Bin 1048576 -> 0 bytes .../metastore_db/log/logmirror.ctrl | Bin 48 -> 0 bytes .../flume-hive-sink/metastore_db/seg0/c10.dat | Bin 8192 -> 0 bytes .../metastore_db/seg0/c101.dat | Bin 16384 -> 0 bytes .../metastore_db/seg0/c111.dat | Bin 20480 -> 0 bytes .../metastore_db/seg0/c121.dat | Bin 16384 -> 0 bytes .../metastore_db/seg0/c130.dat | Bin 12288 -> 0 bytes .../metastore_db/seg0/c141.dat | Bin 8192 -> 0 bytes .../metastore_db/seg0/c150.dat | Bin 28672 -> 0 bytes .../metastore_db/seg0/c161.dat | Bin 20480 -> 0 bytes .../metastore_db/seg0/c171.dat | Bin 20480 -> 0 bytes .../metastore_db/seg0/c180.dat | Bin 69632 -> 0 bytes .../metastore_db/seg0/c191.dat | Bin 16384 -> 0 bytes .../metastore_db/seg0/c1a1.dat | Bin 8192 -> 0 bytes .../metastore_db/seg0/c1b1.dat | Bin 16384 -> 0 bytes .../metastore_db/seg0/c1c0.dat | Bin 8192 -> 0 bytes .../metastore_db/seg0/c1d1.dat | Bin 8192 -> 0 bytes .../metastore_db/seg0/c1e0.dat | Bin 8192 -> 0 bytes .../metastore_db/seg0/c1e90.dat | Bin 8192 -> 0 bytes .../metastore_db/seg0/c1ea1.dat | Bin 8192 -> 0 bytes .../metastore_db/seg0/c1eb0.dat | Bin 8192 -> 0 bytes .../metastore_db/seg0/c1ec1.dat | Bin 8192 -> 0 bytes .../metastore_db/seg0/c1ed0.dat | Bin 8192 -> 0 bytes .../metastore_db/seg0/c1ee0.dat | Bin 8192 -> 0 bytes .../metastore_db/seg0/c1ef0.dat | Bin 8192 -> 0 bytes .../metastore_db/seg0/c1f01.dat | Bin 8192 -> 0 bytes .../metastore_db/seg0/c1f1.dat | Bin 8192 -> 0 bytes .../metastore_db/seg0/c1f11.dat | Bin 8192 -> 0 bytes .../metastore_db/seg0/c1f20.dat | Bin 8192 -> 0 bytes .../metastore_db/seg0/c1f30.dat | Bin 8192 -> 0 bytes .../metastore_db/seg0/c1f41.dat | Bin 8192 -> 0 bytes .../metastore_db/seg0/c1f50.dat | Bin 8192 -> 0 bytes .../flume-hive-sink/metastore_db/seg0/c20.dat | Bin 102400 -> 0 bytes .../metastore_db/seg0/c200.dat | Bin 16384 -> 0 bytes .../metastore_db/seg0/c211.dat | Bin 8192 -> 0 bytes .../metastore_db/seg0/c221.dat | Bin 8192 -> 0 bytes .../metastore_db/seg0/c230.dat | Bin 385024 -> 0 bytes .../metastore_db/seg0/c241.dat | Bin 8192 -> 0 bytes .../metastore_db/seg0/c251.dat | Bin 20480 -> 0 bytes .../metastore_db/seg0/c260.dat | Bin 8192 -> 0 bytes .../metastore_db/seg0/c271.dat | Bin 8192 -> 0 bytes .../metastore_db/seg0/c281.dat | Bin 8192 -> 0 bytes .../metastore_db/seg0/c290.dat | Bin 8192 -> 0 bytes .../metastore_db/seg0/c2a1.dat | Bin 8192 -> 0 bytes .../metastore_db/seg0/c2b1.dat | Bin 8192 -> 0 bytes .../metastore_db/seg0/c2c1.dat | Bin 8192 -> 0 bytes .../metastore_db/seg0/c2d0.dat | Bin 8192 -> 0 bytes .../metastore_db/seg0/c2e1.dat | Bin 8192 -> 0 bytes .../metastore_db/seg0/c2f0.dat | Bin 8192 -> 0 bytes .../metastore_db/seg0/c300.dat | Bin 8192 -> 0 bytes .../flume-hive-sink/metastore_db/seg0/c31.dat | Bin 28672 -> 0 bytes .../metastore_db/seg0/c311.dat | Bin 8192 -> 0 bytes .../metastore_db/seg0/c321.dat | Bin 8192 -> 0 bytes .../metastore_db/seg0/c331.dat | Bin 8192 -> 0 bytes .../metastore_db/seg0/c340.dat | Bin 8192 -> 0 bytes .../metastore_db/seg0/c351.dat | Bin 8192 -> 0 bytes .../metastore_db/seg0/c361.dat | Bin 8192 -> 0 bytes .../metastore_db/seg0/c371.dat | Bin 8192 -> 0 bytes .../metastore_db/seg0/c380.dat | Bin 8192 -> 0 bytes .../metastore_db/seg0/c391.dat | Bin 8192 -> 0 bytes .../metastore_db/seg0/c3a1.dat | Bin 8192 -> 0 bytes .../metastore_db/seg0/c3b1.dat | Bin 8192 -> 0 bytes .../metastore_db/seg0/c3c0.dat | Bin 8192 -> 0 bytes .../metastore_db/seg0/c3d1.dat | Bin 8192 -> 0 bytes .../metastore_db/seg0/c3e1.dat | Bin 8192 -> 0 bytes .../metastore_db/seg0/c3f1.dat | Bin 8192 -> 0 bytes .../metastore_db/seg0/c400.dat | Bin 8192 -> 0 bytes .../flume-hive-sink/metastore_db/seg0/c41.dat | Bin 69632 -> 0 bytes .../metastore_db/seg0/c411.dat | Bin 8192 -> 0 bytes .../metastore_db/seg0/c421.dat | Bin 8192 -> 0 bytes .../metastore_db/seg0/c430.dat | Bin 8192 -> 0 bytes .../metastore_db/seg0/c441.dat | Bin 8192 -> 0 bytes .../metastore_db/seg0/c451.dat | Bin 8192 -> 0 bytes .../metastore_db/seg0/c461.dat | Bin 8192 -> 0 bytes .../flume-hive-sink/metastore_db/seg0/c51.dat | Bin 32768 -> 0 bytes .../metastore_db/seg0/c540.dat | Bin 131072 -> 0 bytes .../metastore_db/seg0/c551.dat | Bin 8192 -> 0 bytes .../metastore_db/seg0/c560.dat | Bin 65536 -> 0 bytes .../metastore_db/seg0/c571.dat | Bin 8192 -> 0 bytes .../metastore_db/seg0/c581.dat | Bin 8192 -> 0 bytes .../metastore_db/seg0/c591.dat | Bin 8192 -> 0 bytes .../metastore_db/seg0/c5a0.dat | Bin 8192 -> 0 bytes .../metastore_db/seg0/c5b1.dat | Bin 8192 -> 0 bytes .../metastore_db/seg0/c5c0.dat | Bin 12288 -> 0 bytes .../metastore_db/seg0/c5d1.dat | Bin 8192 -> 0 bytes .../metastore_db/seg0/c5e0.dat | Bin 8192 -> 0 bytes .../metastore_db/seg0/c5f1.dat | Bin 8192 -> 0 bytes .../flume-hive-sink/metastore_db/seg0/c60.dat | Bin 16384 -> 0 bytes .../metastore_db/seg0/c600.dat | Bin 98304 -> 0 bytes .../metastore_db/seg0/c611.dat | Bin 8192 -> 0 bytes .../metastore_db/seg0/c620.dat | Bin 98304 -> 0 bytes .../metastore_db/seg0/c631.dat | Bin 8192 -> 0 bytes .../metastore_db/seg0/c640.dat | Bin 98304 -> 0 bytes .../metastore_db/seg0/c651.dat | Bin 8192 -> 0 bytes .../metastore_db/seg0/c660.dat | Bin 12288 -> 0 bytes .../metastore_db/seg0/c671.dat | Bin 8192 -> 0 bytes .../metastore_db/seg0/c680.dat | Bin 98304 -> 0 bytes .../metastore_db/seg0/c691.dat | Bin 8192 -> 0 bytes .../metastore_db/seg0/c6a0.dat | Bin 8192 -> 0 bytes .../metastore_db/seg0/c6b1.dat | Bin 8192 -> 0 bytes .../metastore_db/seg0/c6c0.dat | Bin 8192 -> 0 bytes .../metastore_db/seg0/c6d1.dat | Bin 8192 -> 0 bytes .../metastore_db/seg0/c6e0.dat | Bin 65536 -> 0 bytes .../metastore_db/seg0/c6f1.dat | Bin 8192 -> 0 bytes .../metastore_db/seg0/c700.dat | Bin 98304 -> 0 bytes .../flume-hive-sink/metastore_db/seg0/c71.dat | Bin 16384 -> 0 bytes .../metastore_db/seg0/c711.dat | Bin 8192 -> 0 bytes .../metastore_db/seg0/c720.dat | Bin 65536 -> 0 bytes .../metastore_db/seg0/c731.dat | Bin 8192 -> 0 bytes .../metastore_db/seg0/c740.dat | Bin 8192 -> 0 bytes .../metastore_db/seg0/c751.dat | Bin 8192 -> 0 bytes .../metastore_db/seg0/c760.dat | Bin 98304 -> 0 bytes .../metastore_db/seg0/c771.dat | Bin 8192 -> 0 bytes .../metastore_db/seg0/c780.dat | Bin 8192 -> 0 bytes .../metastore_db/seg0/c791.dat | Bin 8192 -> 0 bytes .../metastore_db/seg0/c7a0.dat | Bin 98304 -> 0 bytes .../metastore_db/seg0/c7b1.dat | Bin 8192 -> 0 bytes .../metastore_db/seg0/c7c1.dat | Bin 8192 -> 0 bytes .../metastore_db/seg0/c7d1.dat | Bin 8192 -> 0 bytes .../metastore_db/seg0/c7e1.dat | Bin 8192 -> 0 bytes .../metastore_db/seg0/c7f1.dat | Bin 8192 -> 0 bytes .../metastore_db/seg0/c801.dat | Bin 8192 -> 0 bytes .../flume-hive-sink/metastore_db/seg0/c81.dat | Bin 8192 -> 0 bytes .../metastore_db/seg0/c811.dat | Bin 8192 -> 0 bytes .../metastore_db/seg0/c821.dat | Bin 8192 -> 0 bytes .../metastore_db/seg0/c831.dat | Bin 8192 -> 0 bytes .../metastore_db/seg0/c841.dat | Bin 8192 -> 0 bytes .../metastore_db/seg0/c851.dat | Bin 8192 -> 0 bytes .../metastore_db/seg0/c861.dat | Bin 8192 -> 0 bytes .../metastore_db/seg0/c871.dat | Bin 8192 -> 0 bytes .../metastore_db/seg0/c881.dat | Bin 8192 -> 0 bytes .../metastore_db/seg0/c891.dat | Bin 8192 -> 0 bytes .../metastore_db/seg0/c8a1.dat | Bin 8192 -> 0 bytes .../metastore_db/seg0/c8b1.dat | Bin 8192 -> 0 bytes .../metastore_db/seg0/c8c1.dat | Bin 8192 -> 0 bytes .../metastore_db/seg0/c8d1.dat | Bin 8192 -> 0 bytes .../metastore_db/seg0/c8e1.dat | Bin 8192 -> 0 bytes .../metastore_db/seg0/c8f0.dat | Bin 8192 -> 0 bytes .../flume-hive-sink/metastore_db/seg0/c90.dat | Bin 86016 -> 0 bytes .../metastore_db/seg0/c901.dat | Bin 8192 -> 0 bytes .../metastore_db/seg0/c910.dat | Bin 12288 -> 0 bytes .../metastore_db/seg0/c921.dat | Bin 8192 -> 0 bytes .../metastore_db/seg0/c930.dat | Bin 98304 -> 0 bytes .../metastore_db/seg0/c941.dat | Bin 8192 -> 0 bytes .../metastore_db/seg0/c950.dat | Bin 12288 -> 0 bytes .../metastore_db/seg0/c961.dat | Bin 8192 -> 0 bytes .../metastore_db/seg0/c971.dat | Bin 8192 -> 0 bytes .../metastore_db/seg0/c981.dat | Bin 8192 -> 0 bytes .../metastore_db/seg0/c991.dat | Bin 65536 -> 0 bytes .../metastore_db/seg0/c9a1.dat | Bin 8192 -> 0 bytes .../metastore_db/seg0/c9b1.dat | Bin 8192 -> 0 bytes .../metastore_db/seg0/c9c1.dat | Bin 8192 -> 0 bytes .../metastore_db/seg0/c9d0.dat | Bin 8192 -> 0 bytes .../metastore_db/seg0/c9e1.dat | Bin 8192 -> 0 bytes .../metastore_db/seg0/c9f0.dat | Bin 8192 -> 0 bytes .../metastore_db/seg0/ca01.dat | Bin 8192 -> 0 bytes .../flume-hive-sink/metastore_db/seg0/ca1.dat | Bin 98304 -> 0 bytes .../metastore_db/seg0/ca10.dat | Bin 8192 -> 0 bytes .../metastore_db/seg0/ca21.dat | Bin 8192 -> 0 bytes .../metastore_db/seg0/ca31.dat | Bin 8192 -> 0 bytes .../metastore_db/seg0/ca40.dat | Bin 8192 -> 0 bytes .../metastore_db/seg0/ca51.dat | Bin 8192 -> 0 bytes .../metastore_db/seg0/ca61.dat | Bin 65536 -> 0 bytes .../metastore_db/seg0/ca70.dat | Bin 65536 -> 0 bytes .../metastore_db/seg0/ca81.dat | Bin 8192 -> 0 bytes .../metastore_db/seg0/ca90.dat | Bin 65536 -> 0 bytes .../metastore_db/seg0/caa1.dat | Bin 8192 -> 0 bytes .../metastore_db/seg0/cab1.dat | Bin 8192 -> 0 bytes .../metastore_db/seg0/cac1.dat | Bin 8192 -> 0 bytes .../metastore_db/seg0/cad1.dat | Bin 8192 -> 0 bytes .../metastore_db/seg0/cae1.dat | Bin 8192 -> 0 bytes .../metastore_db/seg0/caf1.dat | Bin 8192 -> 0 bytes .../metastore_db/seg0/cb00.dat | Bin 8192 -> 0 bytes .../flume-hive-sink/metastore_db/seg0/cb1.dat | Bin 16384 -> 0 bytes .../metastore_db/seg0/cb11.dat | Bin 8192 -> 0 bytes .../metastore_db/seg0/cb21.dat | Bin 65536 -> 0 bytes .../metastore_db/seg0/cb31.dat | Bin 8192 -> 0 bytes .../metastore_db/seg0/cb40.dat | Bin 8192 -> 0 bytes .../metastore_db/seg0/cb51.dat | Bin 8192 -> 0 bytes .../metastore_db/seg0/cb61.dat | Bin 8192 -> 0 bytes .../metastore_db/seg0/cb71.dat | Bin 65536 -> 0 bytes .../metastore_db/seg0/cb80.dat | Bin 8192 -> 0 bytes .../metastore_db/seg0/cb91.dat | Bin 8192 -> 0 bytes .../metastore_db/seg0/cba1.dat | Bin 8192 -> 0 bytes .../metastore_db/seg0/cbb1.dat | Bin 65536 -> 0 bytes .../metastore_db/seg0/cbc0.dat | Bin 8192 -> 0 bytes .../metastore_db/seg0/cbd1.dat | Bin 8192 -> 0 bytes .../metastore_db/seg0/cbe1.dat | Bin 65536 -> 0 bytes .../metastore_db/seg0/cbf1.dat | Bin 8192 -> 0 bytes .../flume-hive-sink/metastore_db/seg0/cc0.dat | Bin 8192 -> 0 bytes .../metastore_db/seg0/cc00.dat | Bin 65536 -> 0 bytes .../metastore_db/seg0/cc11.dat | Bin 8192 -> 0 bytes .../metastore_db/seg0/cc20.dat | Bin 65536 -> 0 bytes .../metastore_db/seg0/cc31.dat | Bin 8192 -> 0 bytes .../metastore_db/seg0/cc41.dat | Bin 8192 -> 0 bytes .../metastore_db/seg0/cc51.dat | Bin 8192 -> 0 bytes .../metastore_db/seg0/cc61.dat | Bin 8192 -> 0 bytes .../metastore_db/seg0/cc70.dat | Bin 8192 -> 0 bytes .../metastore_db/seg0/cc81.dat | Bin 8192 -> 0 bytes .../metastore_db/seg0/cc91.dat | Bin 65536 -> 0 bytes .../metastore_db/seg0/cca1.dat | Bin 8192 -> 0 bytes .../flume-hive-sink/metastore_db/seg0/cd1.dat | Bin 8192 -> 0 bytes .../flume-hive-sink/metastore_db/seg0/ce1.dat | Bin 8192 -> 0 bytes .../flume-hive-sink/metastore_db/seg0/cf0.dat | Bin 24576 -> 0 bytes .../metastore_db/seg0/d18e0.dat | Bin 4096 -> 0 bytes .../metastore_db/seg0/d18f1.dat | Bin 4096 -> 0 bytes .../metastore_db/seg0/d1900.dat | Bin 4096 -> 0 bytes .../metastore_db/seg0/d1911.dat | Bin 4096 -> 0 bytes .../metastore_db/seg0/d1920.dat | Bin 4096 -> 0 bytes .../metastore_db/seg0/d1930.dat | Bin 4096 -> 0 bytes .../metastore_db/seg0/d1940.dat | Bin 4096 -> 0 bytes .../metastore_db/seg0/d1951.dat | Bin 4096 -> 0 bytes .../metastore_db/seg0/d1961.dat | Bin 4096 -> 0 bytes .../metastore_db/seg0/d1970.dat | Bin 4096 -> 0 bytes .../metastore_db/seg0/d1980.dat | Bin 4096 -> 0 bytes .../metastore_db/seg0/d1991.dat | Bin 4096 -> 0 bytes .../metastore_db/seg0/d19a0.dat | Bin 4096 -> 0 bytes .../metastore_db/seg0/d19b0.dat | Bin 4096 -> 0 bytes .../metastore_db/seg0/d19c1.dat | Bin 4096 -> 0 bytes .../metastore_db/seg0/d19d0.dat | Bin 4096 -> 0 bytes .../metastore_db/seg0/d19e1.dat | Bin 4096 -> 0 bytes .../metastore_db/seg0/d19f0.dat | Bin 4096 -> 0 bytes .../metastore_db/seg0/d1a00.dat | Bin 4096 -> 0 bytes .../metastore_db/seg0/d1a10.dat | Bin 4096 -> 0 bytes .../metastore_db/seg0/d1a21.dat | Bin 4096 -> 0 bytes .../metastore_db/seg0/d1a31.dat | Bin 4096 -> 0 bytes .../metastore_db/seg0/d1a40.dat | Bin 4096 -> 0 bytes .../metastore_db/seg0/d1a50.dat | Bin 4096 -> 0 bytes .../metastore_db/seg0/d1a61.dat | Bin 4096 -> 0 bytes .../metastore_db/seg0/d1a70.dat | Bin 4096 -> 0 bytes .../metastore_db/seg0/d1a80.dat | Bin 4096 -> 0 bytes .../metastore_db/seg0/d1a91.dat | Bin 4096 -> 0 bytes .../metastore_db/seg0/d1aa0.dat | Bin 4096 -> 0 bytes .../metastore_db/seg0/d1ab1.dat | Bin 4096 -> 0 bytes .../metastore_db/seg0/d1ac0.dat | Bin 4096 -> 0 bytes .../metastore_db/seg0/d1ad0.dat | Bin 4096 -> 0 bytes .../metastore_db/seg0/d1ae0.dat | Bin 4096 -> 0 bytes .../metastore_db/seg0/d1af1.dat | Bin 4096 -> 0 bytes .../metastore_db/seg0/d1b01.dat | Bin 4096 -> 0 bytes .../metastore_db/seg0/d1b10.dat | Bin 4096 -> 0 bytes .../metastore_db/seg0/d1b20.dat | Bin 4096 -> 0 bytes .../metastore_db/seg0/d1b31.dat | Bin 4096 -> 0 bytes .../metastore_db/seg0/d1b40.dat | Bin 4096 -> 0 bytes .../metastore_db/seg0/d1b50.dat | Bin 4096 -> 0 bytes .../metastore_db/seg0/d1b61.dat | Bin 4096 -> 0 bytes .../metastore_db/seg0/d1b70.dat | Bin 4096 -> 0 bytes .../metastore_db/seg0/d1b81.dat | Bin 4096 -> 0 bytes .../metastore_db/seg0/d1b90.dat | Bin 4096 -> 0 bytes .../metastore_db/seg0/d1ba0.dat | Bin 4096 -> 0 bytes .../metastore_db/seg0/d1bb0.dat | Bin 4096 -> 0 bytes .../metastore_db/seg0/d1bc1.dat | Bin 4096 -> 0 bytes .../metastore_db/seg0/d1bd1.dat | Bin 4096 -> 0 bytes .../metastore_db/seg0/d1be0.dat | Bin 4096 -> 0 bytes .../metastore_db/seg0/d1bf0.dat | Bin 4096 -> 0 bytes .../metastore_db/seg0/d1c01.dat | Bin 4096 -> 0 bytes .../metastore_db/seg0/d1c10.dat | Bin 4096 -> 0 bytes .../metastore_db/seg0/d1c20.dat | Bin 4096 -> 0 bytes .../metastore_db/seg0/d1c31.dat | Bin 4096 -> 0 bytes .../metastore_db/seg0/d1c40.dat | Bin 4096 -> 0 bytes .../metastore_db/seg0/d1c51.dat | Bin 4096 -> 0 bytes .../metastore_db/seg0/d1c60.dat | Bin 4096 -> 0 bytes .../metastore_db/seg0/d1c70.dat | Bin 4096 -> 0 bytes .../metastore_db/seg0/d1c80.dat | Bin 4096 -> 0 bytes .../metastore_db/seg0/d1c91.dat | Bin 4096 -> 0 bytes .../metastore_db/seg0/d1ca1.dat | Bin 4096 -> 0 bytes .../metastore_db/seg0/d1cb0.dat | Bin 4096 -> 0 bytes .../metastore_db/seg0/d1cc0.dat | Bin 4096 -> 0 bytes .../metastore_db/seg0/d1cd1.dat | Bin 4096 -> 0 bytes .../metastore_db/seg0/d1ce0.dat | Bin 4096 -> 0 bytes .../metastore_db/seg0/d1cf0.dat | Bin 4096 -> 0 bytes .../metastore_db/seg0/d1d01.dat | Bin 4096 -> 0 bytes .../metastore_db/seg0/d1d10.dat | Bin 4096 -> 0 bytes .../metastore_db/seg0/d1d21.dat | Bin 4096 -> 0 bytes .../metastore_db/seg0/d1d30.dat | Bin 4096 -> 0 bytes .../metastore_db/seg0/d1d40.dat | Bin 4096 -> 0 bytes .../metastore_db/seg0/d1d50.dat | Bin 4096 -> 0 bytes .../metastore_db/seg0/d1d61.dat | Bin 4096 -> 0 bytes .../metastore_db/seg0/d1d71.dat | Bin 4096 -> 0 bytes .../metastore_db/seg0/d1d80.dat | Bin 4096 -> 0 bytes .../metastore_db/seg0/d1d90.dat | Bin 4096 -> 0 bytes .../metastore_db/seg0/d1da1.dat | Bin 4096 -> 0 bytes .../metastore_db/seg0/d1db0.dat | Bin 4096 -> 0 bytes .../metastore_db/seg0/d1dc0.dat | Bin 4096 -> 0 bytes .../metastore_db/seg0/d1dd1.dat | Bin 4096 -> 0 bytes .../metastore_db/seg0/d1de0.dat | Bin 4096 -> 0 bytes .../metastore_db/seg0/d1df1.dat | Bin 4096 -> 0 bytes .../metastore_db/seg0/d1e00.dat | Bin 4096 -> 0 bytes .../metastore_db/seg0/d1e10.dat | Bin 4096 -> 0 bytes .../metastore_db/seg0/d1e20.dat | Bin 4096 -> 0 bytes .../metastore_db/seg0/d1e31.dat | Bin 4096 -> 0 bytes .../metastore_db/seg0/d1e41.dat | Bin 4096 -> 0 bytes .../metastore_db/seg0/d1e50.dat | Bin 4096 -> 0 bytes .../metastore_db/seg0/d1e60.dat | Bin 4096 -> 0 bytes .../metastore_db/seg0/d1e71.dat | Bin 4096 -> 0 bytes .../metastore_db/seg0/d1e80.dat | Bin 4096 -> 0 bytes .../metastore_db/service.properties | 22 -------- .../hive/HiveDelimitedTextSerializer.java | 8 +++ .../flume/sink/hive/HiveEventSerializer.java | 4 ++ .../flume/sink/hive/HiveJsonSerializer.java | 7 +++ .../apache/flume/sink/hive/HiveWriter.java | 49 ++++++++++++++---- pom.xml | 2 +- 305 files changed, 58 insertions(+), 34 deletions(-) delete mode 100644 flume-ng-sinks/flume-hive-sink/metastore_db/db.lck delete mode 100644 flume-ng-sinks/flume-hive-sink/metastore_db/dbex.lck delete mode 100644 flume-ng-sinks/flume-hive-sink/metastore_db/log/log.ctrl delete mode 100644 flume-ng-sinks/flume-hive-sink/metastore_db/log/log3.dat delete mode 100644 flume-ng-sinks/flume-hive-sink/metastore_db/log/log4.dat delete mode 100644 flume-ng-sinks/flume-hive-sink/metastore_db/log/logmirror.ctrl delete mode 100644 flume-ng-sinks/flume-hive-sink/metastore_db/seg0/c10.dat delete mode 100644 flume-ng-sinks/flume-hive-sink/metastore_db/seg0/c101.dat delete mode 100644 flume-ng-sinks/flume-hive-sink/metastore_db/seg0/c111.dat delete mode 100644 flume-ng-sinks/flume-hive-sink/metastore_db/seg0/c121.dat delete mode 100644 flume-ng-sinks/flume-hive-sink/metastore_db/seg0/c130.dat delete mode 100644 flume-ng-sinks/flume-hive-sink/metastore_db/seg0/c141.dat delete mode 100644 flume-ng-sinks/flume-hive-sink/metastore_db/seg0/c150.dat delete mode 100644 flume-ng-sinks/flume-hive-sink/metastore_db/seg0/c161.dat delete mode 100644 flume-ng-sinks/flume-hive-sink/metastore_db/seg0/c171.dat delete mode 100644 flume-ng-sinks/flume-hive-sink/metastore_db/seg0/c180.dat delete mode 100644 flume-ng-sinks/flume-hive-sink/metastore_db/seg0/c191.dat delete mode 100644 flume-ng-sinks/flume-hive-sink/metastore_db/seg0/c1a1.dat delete mode 100644 flume-ng-sinks/flume-hive-sink/metastore_db/seg0/c1b1.dat delete mode 100644 flume-ng-sinks/flume-hive-sink/metastore_db/seg0/c1c0.dat delete mode 100644 flume-ng-sinks/flume-hive-sink/metastore_db/seg0/c1d1.dat delete mode 100644 flume-ng-sinks/flume-hive-sink/metastore_db/seg0/c1e0.dat delete mode 100644 flume-ng-sinks/flume-hive-sink/metastore_db/seg0/c1e90.dat delete mode 100644 flume-ng-sinks/flume-hive-sink/metastore_db/seg0/c1ea1.dat delete mode 100644 flume-ng-sinks/flume-hive-sink/metastore_db/seg0/c1eb0.dat delete mode 100644 flume-ng-sinks/flume-hive-sink/metastore_db/seg0/c1ec1.dat delete mode 100644 flume-ng-sinks/flume-hive-sink/metastore_db/seg0/c1ed0.dat delete mode 100644 flume-ng-sinks/flume-hive-sink/metastore_db/seg0/c1ee0.dat delete mode 100644 flume-ng-sinks/flume-hive-sink/metastore_db/seg0/c1ef0.dat delete mode 100644 flume-ng-sinks/flume-hive-sink/metastore_db/seg0/c1f01.dat delete mode 100644 flume-ng-sinks/flume-hive-sink/metastore_db/seg0/c1f1.dat delete mode 100644 flume-ng-sinks/flume-hive-sink/metastore_db/seg0/c1f11.dat delete mode 100644 flume-ng-sinks/flume-hive-sink/metastore_db/seg0/c1f20.dat delete mode 100644 flume-ng-sinks/flume-hive-sink/metastore_db/seg0/c1f30.dat delete mode 100644 flume-ng-sinks/flume-hive-sink/metastore_db/seg0/c1f41.dat delete mode 100644 flume-ng-sinks/flume-hive-sink/metastore_db/seg0/c1f50.dat delete mode 100644 flume-ng-sinks/flume-hive-sink/metastore_db/seg0/c20.dat delete mode 100644 flume-ng-sinks/flume-hive-sink/metastore_db/seg0/c200.dat delete mode 100644 flume-ng-sinks/flume-hive-sink/metastore_db/seg0/c211.dat delete mode 100644 flume-ng-sinks/flume-hive-sink/metastore_db/seg0/c221.dat delete mode 100644 flume-ng-sinks/flume-hive-sink/metastore_db/seg0/c230.dat delete mode 100644 flume-ng-sinks/flume-hive-sink/metastore_db/seg0/c241.dat delete mode 100644 flume-ng-sinks/flume-hive-sink/metastore_db/seg0/c251.dat delete mode 100644 flume-ng-sinks/flume-hive-sink/metastore_db/seg0/c260.dat delete mode 100644 flume-ng-sinks/flume-hive-sink/metastore_db/seg0/c271.dat delete mode 100644 flume-ng-sinks/flume-hive-sink/metastore_db/seg0/c281.dat delete mode 100644 flume-ng-sinks/flume-hive-sink/metastore_db/seg0/c290.dat delete mode 100644 flume-ng-sinks/flume-hive-sink/metastore_db/seg0/c2a1.dat delete mode 100644 flume-ng-sinks/flume-hive-sink/metastore_db/seg0/c2b1.dat delete mode 100644 flume-ng-sinks/flume-hive-sink/metastore_db/seg0/c2c1.dat delete mode 100644 flume-ng-sinks/flume-hive-sink/metastore_db/seg0/c2d0.dat delete mode 100644 flume-ng-sinks/flume-hive-sink/metastore_db/seg0/c2e1.dat delete mode 100644 flume-ng-sinks/flume-hive-sink/metastore_db/seg0/c2f0.dat delete mode 100644 flume-ng-sinks/flume-hive-sink/metastore_db/seg0/c300.dat delete mode 100644 flume-ng-sinks/flume-hive-sink/metastore_db/seg0/c31.dat delete mode 100644 flume-ng-sinks/flume-hive-sink/metastore_db/seg0/c311.dat delete mode 100644 flume-ng-sinks/flume-hive-sink/metastore_db/seg0/c321.dat delete mode 100644 flume-ng-sinks/flume-hive-sink/metastore_db/seg0/c331.dat delete mode 100644 flume-ng-sinks/flume-hive-sink/metastore_db/seg0/c340.dat delete mode 100644 flume-ng-sinks/flume-hive-sink/metastore_db/seg0/c351.dat delete mode 100644 flume-ng-sinks/flume-hive-sink/metastore_db/seg0/c361.dat delete mode 100644 flume-ng-sinks/flume-hive-sink/metastore_db/seg0/c371.dat delete mode 100644 flume-ng-sinks/flume-hive-sink/metastore_db/seg0/c380.dat delete mode 100644 flume-ng-sinks/flume-hive-sink/metastore_db/seg0/c391.dat delete mode 100644 flume-ng-sinks/flume-hive-sink/metastore_db/seg0/c3a1.dat delete mode 100644 flume-ng-sinks/flume-hive-sink/metastore_db/seg0/c3b1.dat delete mode 100644 flume-ng-sinks/flume-hive-sink/metastore_db/seg0/c3c0.dat delete mode 100644 flume-ng-sinks/flume-hive-sink/metastore_db/seg0/c3d1.dat delete mode 100644 flume-ng-sinks/flume-hive-sink/metastore_db/seg0/c3e1.dat delete mode 100644 flume-ng-sinks/flume-hive-sink/metastore_db/seg0/c3f1.dat delete mode 100644 flume-ng-sinks/flume-hive-sink/metastore_db/seg0/c400.dat delete mode 100644 flume-ng-sinks/flume-hive-sink/metastore_db/seg0/c41.dat delete mode 100644 flume-ng-sinks/flume-hive-sink/metastore_db/seg0/c411.dat delete mode 100644 flume-ng-sinks/flume-hive-sink/metastore_db/seg0/c421.dat delete mode 100644 flume-ng-sinks/flume-hive-sink/metastore_db/seg0/c430.dat delete mode 100644 flume-ng-sinks/flume-hive-sink/metastore_db/seg0/c441.dat delete mode 100644 flume-ng-sinks/flume-hive-sink/metastore_db/seg0/c451.dat delete mode 100644 flume-ng-sinks/flume-hive-sink/metastore_db/seg0/c461.dat delete mode 100644 flume-ng-sinks/flume-hive-sink/metastore_db/seg0/c51.dat delete mode 100644 flume-ng-sinks/flume-hive-sink/metastore_db/seg0/c540.dat delete mode 100644 flume-ng-sinks/flume-hive-sink/metastore_db/seg0/c551.dat delete mode 100644 flume-ng-sinks/flume-hive-sink/metastore_db/seg0/c560.dat delete mode 100644 flume-ng-sinks/flume-hive-sink/metastore_db/seg0/c571.dat delete mode 100644 flume-ng-sinks/flume-hive-sink/metastore_db/seg0/c581.dat delete mode 100644 flume-ng-sinks/flume-hive-sink/metastore_db/seg0/c591.dat delete mode 100644 flume-ng-sinks/flume-hive-sink/metastore_db/seg0/c5a0.dat delete mode 100644 flume-ng-sinks/flume-hive-sink/metastore_db/seg0/c5b1.dat delete mode 100644 flume-ng-sinks/flume-hive-sink/metastore_db/seg0/c5c0.dat delete mode 100644 flume-ng-sinks/flume-hive-sink/metastore_db/seg0/c5d1.dat delete mode 100644 flume-ng-sinks/flume-hive-sink/metastore_db/seg0/c5e0.dat delete mode 100644 flume-ng-sinks/flume-hive-sink/metastore_db/seg0/c5f1.dat delete mode 100644 flume-ng-sinks/flume-hive-sink/metastore_db/seg0/c60.dat delete mode 100644 flume-ng-sinks/flume-hive-sink/metastore_db/seg0/c600.dat delete mode 100644 flume-ng-sinks/flume-hive-sink/metastore_db/seg0/c611.dat delete mode 100644 flume-ng-sinks/flume-hive-sink/metastore_db/seg0/c620.dat delete mode 100644 flume-ng-sinks/flume-hive-sink/metastore_db/seg0/c631.dat delete mode 100644 flume-ng-sinks/flume-hive-sink/metastore_db/seg0/c640.dat delete mode 100644 flume-ng-sinks/flume-hive-sink/metastore_db/seg0/c651.dat delete mode 100644 flume-ng-sinks/flume-hive-sink/metastore_db/seg0/c660.dat delete mode 100644 flume-ng-sinks/flume-hive-sink/metastore_db/seg0/c671.dat delete mode 100644 flume-ng-sinks/flume-hive-sink/metastore_db/seg0/c680.dat delete mode 100644 flume-ng-sinks/flume-hive-sink/metastore_db/seg0/c691.dat delete mode 100644 flume-ng-sinks/flume-hive-sink/metastore_db/seg0/c6a0.dat delete mode 100644 flume-ng-sinks/flume-hive-sink/metastore_db/seg0/c6b1.dat delete mode 100644 flume-ng-sinks/flume-hive-sink/metastore_db/seg0/c6c0.dat delete mode 100644 flume-ng-sinks/flume-hive-sink/metastore_db/seg0/c6d1.dat delete mode 100644 flume-ng-sinks/flume-hive-sink/metastore_db/seg0/c6e0.dat delete mode 100644 flume-ng-sinks/flume-hive-sink/metastore_db/seg0/c6f1.dat delete mode 100644 flume-ng-sinks/flume-hive-sink/metastore_db/seg0/c700.dat delete mode 100644 flume-ng-sinks/flume-hive-sink/metastore_db/seg0/c71.dat delete mode 100644 flume-ng-sinks/flume-hive-sink/metastore_db/seg0/c711.dat delete mode 100644 flume-ng-sinks/flume-hive-sink/metastore_db/seg0/c720.dat delete mode 100644 flume-ng-sinks/flume-hive-sink/metastore_db/seg0/c731.dat delete mode 100644 flume-ng-sinks/flume-hive-sink/metastore_db/seg0/c740.dat delete mode 100644 flume-ng-sinks/flume-hive-sink/metastore_db/seg0/c751.dat delete mode 100644 flume-ng-sinks/flume-hive-sink/metastore_db/seg0/c760.dat delete mode 100644 flume-ng-sinks/flume-hive-sink/metastore_db/seg0/c771.dat delete mode 100644 flume-ng-sinks/flume-hive-sink/metastore_db/seg0/c780.dat delete mode 100644 flume-ng-sinks/flume-hive-sink/metastore_db/seg0/c791.dat delete mode 100644 flume-ng-sinks/flume-hive-sink/metastore_db/seg0/c7a0.dat delete mode 100644 flume-ng-sinks/flume-hive-sink/metastore_db/seg0/c7b1.dat delete mode 100644 flume-ng-sinks/flume-hive-sink/metastore_db/seg0/c7c1.dat delete mode 100644 flume-ng-sinks/flume-hive-sink/metastore_db/seg0/c7d1.dat delete mode 100644 flume-ng-sinks/flume-hive-sink/metastore_db/seg0/c7e1.dat delete mode 100644 flume-ng-sinks/flume-hive-sink/metastore_db/seg0/c7f1.dat delete mode 100644 flume-ng-sinks/flume-hive-sink/metastore_db/seg0/c801.dat delete mode 100644 flume-ng-sinks/flume-hive-sink/metastore_db/seg0/c81.dat delete mode 100644 flume-ng-sinks/flume-hive-sink/metastore_db/seg0/c811.dat delete mode 100644 flume-ng-sinks/flume-hive-sink/metastore_db/seg0/c821.dat delete mode 100644 flume-ng-sinks/flume-hive-sink/metastore_db/seg0/c831.dat delete mode 100644 flume-ng-sinks/flume-hive-sink/metastore_db/seg0/c841.dat delete mode 100644 flume-ng-sinks/flume-hive-sink/metastore_db/seg0/c851.dat delete mode 100644 flume-ng-sinks/flume-hive-sink/metastore_db/seg0/c861.dat delete mode 100644 flume-ng-sinks/flume-hive-sink/metastore_db/seg0/c871.dat delete mode 100644 flume-ng-sinks/flume-hive-sink/metastore_db/seg0/c881.dat delete mode 100644 flume-ng-sinks/flume-hive-sink/metastore_db/seg0/c891.dat delete mode 100644 flume-ng-sinks/flume-hive-sink/metastore_db/seg0/c8a1.dat delete mode 100644 flume-ng-sinks/flume-hive-sink/metastore_db/seg0/c8b1.dat delete mode 100644 flume-ng-sinks/flume-hive-sink/metastore_db/seg0/c8c1.dat delete mode 100644 flume-ng-sinks/flume-hive-sink/metastore_db/seg0/c8d1.dat delete mode 100644 flume-ng-sinks/flume-hive-sink/metastore_db/seg0/c8e1.dat delete mode 100644 flume-ng-sinks/flume-hive-sink/metastore_db/seg0/c8f0.dat delete mode 100644 flume-ng-sinks/flume-hive-sink/metastore_db/seg0/c90.dat delete mode 100644 flume-ng-sinks/flume-hive-sink/metastore_db/seg0/c901.dat delete mode 100644 flume-ng-sinks/flume-hive-sink/metastore_db/seg0/c910.dat delete mode 100644 flume-ng-sinks/flume-hive-sink/metastore_db/seg0/c921.dat delete mode 100644 flume-ng-sinks/flume-hive-sink/metastore_db/seg0/c930.dat delete mode 100644 flume-ng-sinks/flume-hive-sink/metastore_db/seg0/c941.dat delete mode 100644 flume-ng-sinks/flume-hive-sink/metastore_db/seg0/c950.dat delete mode 100644 flume-ng-sinks/flume-hive-sink/metastore_db/seg0/c961.dat delete mode 100644 flume-ng-sinks/flume-hive-sink/metastore_db/seg0/c971.dat delete mode 100644 flume-ng-sinks/flume-hive-sink/metastore_db/seg0/c981.dat delete mode 100644 flume-ng-sinks/flume-hive-sink/metastore_db/seg0/c991.dat delete mode 100644 flume-ng-sinks/flume-hive-sink/metastore_db/seg0/c9a1.dat delete mode 100644 flume-ng-sinks/flume-hive-sink/metastore_db/seg0/c9b1.dat delete mode 100644 flume-ng-sinks/flume-hive-sink/metastore_db/seg0/c9c1.dat delete mode 100644 flume-ng-sinks/flume-hive-sink/metastore_db/seg0/c9d0.dat delete mode 100644 flume-ng-sinks/flume-hive-sink/metastore_db/seg0/c9e1.dat delete mode 100644 flume-ng-sinks/flume-hive-sink/metastore_db/seg0/c9f0.dat delete mode 100644 flume-ng-sinks/flume-hive-sink/metastore_db/seg0/ca01.dat delete mode 100644 flume-ng-sinks/flume-hive-sink/metastore_db/seg0/ca1.dat delete mode 100644 flume-ng-sinks/flume-hive-sink/metastore_db/seg0/ca10.dat delete mode 100644 flume-ng-sinks/flume-hive-sink/metastore_db/seg0/ca21.dat delete mode 100644 flume-ng-sinks/flume-hive-sink/metastore_db/seg0/ca31.dat delete mode 100644 flume-ng-sinks/flume-hive-sink/metastore_db/seg0/ca40.dat delete mode 100644 flume-ng-sinks/flume-hive-sink/metastore_db/seg0/ca51.dat delete mode 100644 flume-ng-sinks/flume-hive-sink/metastore_db/seg0/ca61.dat delete mode 100644 flume-ng-sinks/flume-hive-sink/metastore_db/seg0/ca70.dat delete mode 100644 flume-ng-sinks/flume-hive-sink/metastore_db/seg0/ca81.dat delete mode 100644 flume-ng-sinks/flume-hive-sink/metastore_db/seg0/ca90.dat delete mode 100644 flume-ng-sinks/flume-hive-sink/metastore_db/seg0/caa1.dat delete mode 100644 flume-ng-sinks/flume-hive-sink/metastore_db/seg0/cab1.dat delete mode 100644 flume-ng-sinks/flume-hive-sink/metastore_db/seg0/cac1.dat delete mode 100644 flume-ng-sinks/flume-hive-sink/metastore_db/seg0/cad1.dat delete mode 100644 flume-ng-sinks/flume-hive-sink/metastore_db/seg0/cae1.dat delete mode 100644 flume-ng-sinks/flume-hive-sink/metastore_db/seg0/caf1.dat delete mode 100644 flume-ng-sinks/flume-hive-sink/metastore_db/seg0/cb00.dat delete mode 100644 flume-ng-sinks/flume-hive-sink/metastore_db/seg0/cb1.dat delete mode 100644 flume-ng-sinks/flume-hive-sink/metastore_db/seg0/cb11.dat delete mode 100644 flume-ng-sinks/flume-hive-sink/metastore_db/seg0/cb21.dat delete mode 100644 flume-ng-sinks/flume-hive-sink/metastore_db/seg0/cb31.dat delete mode 100644 flume-ng-sinks/flume-hive-sink/metastore_db/seg0/cb40.dat delete mode 100644 flume-ng-sinks/flume-hive-sink/metastore_db/seg0/cb51.dat delete mode 100644 flume-ng-sinks/flume-hive-sink/metastore_db/seg0/cb61.dat delete mode 100644 flume-ng-sinks/flume-hive-sink/metastore_db/seg0/cb71.dat delete mode 100644 flume-ng-sinks/flume-hive-sink/metastore_db/seg0/cb80.dat delete mode 100644 flume-ng-sinks/flume-hive-sink/metastore_db/seg0/cb91.dat delete mode 100644 flume-ng-sinks/flume-hive-sink/metastore_db/seg0/cba1.dat delete mode 100644 flume-ng-sinks/flume-hive-sink/metastore_db/seg0/cbb1.dat delete mode 100644 flume-ng-sinks/flume-hive-sink/metastore_db/seg0/cbc0.dat delete mode 100644 flume-ng-sinks/flume-hive-sink/metastore_db/seg0/cbd1.dat delete mode 100644 flume-ng-sinks/flume-hive-sink/metastore_db/seg0/cbe1.dat delete mode 100644 flume-ng-sinks/flume-hive-sink/metastore_db/seg0/cbf1.dat delete mode 100644 flume-ng-sinks/flume-hive-sink/metastore_db/seg0/cc0.dat delete mode 100644 flume-ng-sinks/flume-hive-sink/metastore_db/seg0/cc00.dat delete mode 100644 flume-ng-sinks/flume-hive-sink/metastore_db/seg0/cc11.dat delete mode 100644 flume-ng-sinks/flume-hive-sink/metastore_db/seg0/cc20.dat delete mode 100644 flume-ng-sinks/flume-hive-sink/metastore_db/seg0/cc31.dat delete mode 100644 flume-ng-sinks/flume-hive-sink/metastore_db/seg0/cc41.dat delete mode 100644 flume-ng-sinks/flume-hive-sink/metastore_db/seg0/cc51.dat delete mode 100644 flume-ng-sinks/flume-hive-sink/metastore_db/seg0/cc61.dat delete mode 100644 flume-ng-sinks/flume-hive-sink/metastore_db/seg0/cc70.dat delete mode 100644 flume-ng-sinks/flume-hive-sink/metastore_db/seg0/cc81.dat delete mode 100644 flume-ng-sinks/flume-hive-sink/metastore_db/seg0/cc91.dat delete mode 100644 flume-ng-sinks/flume-hive-sink/metastore_db/seg0/cca1.dat delete mode 100644 flume-ng-sinks/flume-hive-sink/metastore_db/seg0/cd1.dat delete mode 100644 flume-ng-sinks/flume-hive-sink/metastore_db/seg0/ce1.dat delete mode 100644 flume-ng-sinks/flume-hive-sink/metastore_db/seg0/cf0.dat delete mode 100644 flume-ng-sinks/flume-hive-sink/metastore_db/seg0/d18e0.dat delete mode 100644 flume-ng-sinks/flume-hive-sink/metastore_db/seg0/d18f1.dat delete mode 100644 flume-ng-sinks/flume-hive-sink/metastore_db/seg0/d1900.dat delete mode 100644 flume-ng-sinks/flume-hive-sink/metastore_db/seg0/d1911.dat delete mode 100644 flume-ng-sinks/flume-hive-sink/metastore_db/seg0/d1920.dat delete mode 100644 flume-ng-sinks/flume-hive-sink/metastore_db/seg0/d1930.dat delete mode 100644 flume-ng-sinks/flume-hive-sink/metastore_db/seg0/d1940.dat delete mode 100644 flume-ng-sinks/flume-hive-sink/metastore_db/seg0/d1951.dat delete mode 100644 flume-ng-sinks/flume-hive-sink/metastore_db/seg0/d1961.dat delete mode 100644 flume-ng-sinks/flume-hive-sink/metastore_db/seg0/d1970.dat delete mode 100644 flume-ng-sinks/flume-hive-sink/metastore_db/seg0/d1980.dat delete mode 100644 flume-ng-sinks/flume-hive-sink/metastore_db/seg0/d1991.dat delete mode 100644 flume-ng-sinks/flume-hive-sink/metastore_db/seg0/d19a0.dat delete mode 100644 flume-ng-sinks/flume-hive-sink/metastore_db/seg0/d19b0.dat delete mode 100644 flume-ng-sinks/flume-hive-sink/metastore_db/seg0/d19c1.dat delete mode 100644 flume-ng-sinks/flume-hive-sink/metastore_db/seg0/d19d0.dat delete mode 100644 flume-ng-sinks/flume-hive-sink/metastore_db/seg0/d19e1.dat delete mode 100644 flume-ng-sinks/flume-hive-sink/metastore_db/seg0/d19f0.dat delete mode 100644 flume-ng-sinks/flume-hive-sink/metastore_db/seg0/d1a00.dat delete mode 100644 flume-ng-sinks/flume-hive-sink/metastore_db/seg0/d1a10.dat delete mode 100644 flume-ng-sinks/flume-hive-sink/metastore_db/seg0/d1a21.dat delete mode 100644 flume-ng-sinks/flume-hive-sink/metastore_db/seg0/d1a31.dat delete mode 100644 flume-ng-sinks/flume-hive-sink/metastore_db/seg0/d1a40.dat delete mode 100644 flume-ng-sinks/flume-hive-sink/metastore_db/seg0/d1a50.dat delete mode 100644 flume-ng-sinks/flume-hive-sink/metastore_db/seg0/d1a61.dat delete mode 100644 flume-ng-sinks/flume-hive-sink/metastore_db/seg0/d1a70.dat delete mode 100644 flume-ng-sinks/flume-hive-sink/metastore_db/seg0/d1a80.dat delete mode 100644 flume-ng-sinks/flume-hive-sink/metastore_db/seg0/d1a91.dat delete mode 100644 flume-ng-sinks/flume-hive-sink/metastore_db/seg0/d1aa0.dat delete mode 100644 flume-ng-sinks/flume-hive-sink/metastore_db/seg0/d1ab1.dat delete mode 100644 flume-ng-sinks/flume-hive-sink/metastore_db/seg0/d1ac0.dat delete mode 100644 flume-ng-sinks/flume-hive-sink/metastore_db/seg0/d1ad0.dat delete mode 100644 flume-ng-sinks/flume-hive-sink/metastore_db/seg0/d1ae0.dat delete mode 100644 flume-ng-sinks/flume-hive-sink/metastore_db/seg0/d1af1.dat delete mode 100644 flume-ng-sinks/flume-hive-sink/metastore_db/seg0/d1b01.dat delete mode 100644 flume-ng-sinks/flume-hive-sink/metastore_db/seg0/d1b10.dat delete mode 100644 flume-ng-sinks/flume-hive-sink/metastore_db/seg0/d1b20.dat delete mode 100644 flume-ng-sinks/flume-hive-sink/metastore_db/seg0/d1b31.dat delete mode 100644 flume-ng-sinks/flume-hive-sink/metastore_db/seg0/d1b40.dat delete mode 100644 flume-ng-sinks/flume-hive-sink/metastore_db/seg0/d1b50.dat delete mode 100644 flume-ng-sinks/flume-hive-sink/metastore_db/seg0/d1b61.dat delete mode 100644 flume-ng-sinks/flume-hive-sink/metastore_db/seg0/d1b70.dat delete mode 100644 flume-ng-sinks/flume-hive-sink/metastore_db/seg0/d1b81.dat delete mode 100644 flume-ng-sinks/flume-hive-sink/metastore_db/seg0/d1b90.dat delete mode 100644 flume-ng-sinks/flume-hive-sink/metastore_db/seg0/d1ba0.dat delete mode 100644 flume-ng-sinks/flume-hive-sink/metastore_db/seg0/d1bb0.dat delete mode 100644 flume-ng-sinks/flume-hive-sink/metastore_db/seg0/d1bc1.dat delete mode 100644 flume-ng-sinks/flume-hive-sink/metastore_db/seg0/d1bd1.dat delete mode 100644 flume-ng-sinks/flume-hive-sink/metastore_db/seg0/d1be0.dat delete mode 100644 flume-ng-sinks/flume-hive-sink/metastore_db/seg0/d1bf0.dat delete mode 100644 flume-ng-sinks/flume-hive-sink/metastore_db/seg0/d1c01.dat delete mode 100644 flume-ng-sinks/flume-hive-sink/metastore_db/seg0/d1c10.dat delete mode 100644 flume-ng-sinks/flume-hive-sink/metastore_db/seg0/d1c20.dat delete mode 100644 flume-ng-sinks/flume-hive-sink/metastore_db/seg0/d1c31.dat delete mode 100644 flume-ng-sinks/flume-hive-sink/metastore_db/seg0/d1c40.dat delete mode 100644 flume-ng-sinks/flume-hive-sink/metastore_db/seg0/d1c51.dat delete mode 100644 flume-ng-sinks/flume-hive-sink/metastore_db/seg0/d1c60.dat delete mode 100644 flume-ng-sinks/flume-hive-sink/metastore_db/seg0/d1c70.dat delete mode 100644 flume-ng-sinks/flume-hive-sink/metastore_db/seg0/d1c80.dat delete mode 100644 flume-ng-sinks/flume-hive-sink/metastore_db/seg0/d1c91.dat delete mode 100644 flume-ng-sinks/flume-hive-sink/metastore_db/seg0/d1ca1.dat delete mode 100644 flume-ng-sinks/flume-hive-sink/metastore_db/seg0/d1cb0.dat delete mode 100644 flume-ng-sinks/flume-hive-sink/metastore_db/seg0/d1cc0.dat delete mode 100644 flume-ng-sinks/flume-hive-sink/metastore_db/seg0/d1cd1.dat delete mode 100644 flume-ng-sinks/flume-hive-sink/metastore_db/seg0/d1ce0.dat delete mode 100644 flume-ng-sinks/flume-hive-sink/metastore_db/seg0/d1cf0.dat delete mode 100644 flume-ng-sinks/flume-hive-sink/metastore_db/seg0/d1d01.dat delete mode 100644 flume-ng-sinks/flume-hive-sink/metastore_db/seg0/d1d10.dat delete mode 100644 flume-ng-sinks/flume-hive-sink/metastore_db/seg0/d1d21.dat delete mode 100644 flume-ng-sinks/flume-hive-sink/metastore_db/seg0/d1d30.dat delete mode 100644 flume-ng-sinks/flume-hive-sink/metastore_db/seg0/d1d40.dat delete mode 100644 flume-ng-sinks/flume-hive-sink/metastore_db/seg0/d1d50.dat delete mode 100644 flume-ng-sinks/flume-hive-sink/metastore_db/seg0/d1d61.dat delete mode 100644 flume-ng-sinks/flume-hive-sink/metastore_db/seg0/d1d71.dat delete mode 100644 flume-ng-sinks/flume-hive-sink/metastore_db/seg0/d1d80.dat delete mode 100644 flume-ng-sinks/flume-hive-sink/metastore_db/seg0/d1d90.dat delete mode 100644 flume-ng-sinks/flume-hive-sink/metastore_db/seg0/d1da1.dat delete mode 100644 flume-ng-sinks/flume-hive-sink/metastore_db/seg0/d1db0.dat delete mode 100644 flume-ng-sinks/flume-hive-sink/metastore_db/seg0/d1dc0.dat delete mode 100644 flume-ng-sinks/flume-hive-sink/metastore_db/seg0/d1dd1.dat delete mode 100644 flume-ng-sinks/flume-hive-sink/metastore_db/seg0/d1de0.dat delete mode 100644 flume-ng-sinks/flume-hive-sink/metastore_db/seg0/d1df1.dat delete mode 100644 flume-ng-sinks/flume-hive-sink/metastore_db/seg0/d1e00.dat delete mode 100644 flume-ng-sinks/flume-hive-sink/metastore_db/seg0/d1e10.dat delete mode 100644 flume-ng-sinks/flume-hive-sink/metastore_db/seg0/d1e20.dat delete mode 100644 flume-ng-sinks/flume-hive-sink/metastore_db/seg0/d1e31.dat delete mode 100644 flume-ng-sinks/flume-hive-sink/metastore_db/seg0/d1e41.dat delete mode 100644 flume-ng-sinks/flume-hive-sink/metastore_db/seg0/d1e50.dat delete mode 100644 flume-ng-sinks/flume-hive-sink/metastore_db/seg0/d1e60.dat delete mode 100644 flume-ng-sinks/flume-hive-sink/metastore_db/seg0/d1e71.dat delete mode 100644 flume-ng-sinks/flume-hive-sink/metastore_db/seg0/d1e80.dat delete mode 100644 flume-ng-sinks/flume-hive-sink/metastore_db/service.properties diff --git a/flume-ng-sinks/flume-hive-sink/metastore_db/db.lck b/flume-ng-sinks/flume-hive-sink/metastore_db/db.lck deleted file mode 100644 index 45e0595987566b3f1fe9e6fb22f32ccb9ba1bcb5..0000000000000000000000000000000000000000 GIT binary patch literal 0 HcmV?d00001 literal 38 pcmZQjNwhFDOExe_)ip3QNzyepGDy=+OEm^^K_JO2)zH|;006BL2@L=M diff --git a/flume-ng-sinks/flume-hive-sink/metastore_db/dbex.lck b/flume-ng-sinks/flume-hive-sink/metastore_db/dbex.lck deleted file mode 100644 index 720d64f4baafc33efdf971f02084aca5f25b34a5..0000000000000000000000000000000000000000 GIT binary patch literal 0 HcmV?d00001 literal 4 LcmZQzU|<9Q00jU7 diff --git a/flume-ng-sinks/flume-hive-sink/metastore_db/log/log.ctrl b/flume-ng-sinks/flume-hive-sink/metastore_db/log/log.ctrl deleted file mode 100644 index bc4ad4cd28195e8506bd37318bc51ca4ac21bba7..0000000000000000000000000000000000000000 GIT binary patch literal 0 HcmV?d00001 literal 48 mcmZQzU}#`qVBiE|W(KaW8bF#0h&dR9*t~!=3i#1)qy+#^*#rv! diff --git a/flume-ng-sinks/flume-hive-sink/metastore_db/log/log3.dat b/flume-ng-sinks/flume-hive-sink/metastore_db/log/log3.dat deleted file mode 100644 index 1e8b5d6825dfea49843854c5e72f498422372f97..0000000000000000000000000000000000000000 GIT binary patch literal 0 HcmV?d00001 literal 1048591 zcmdqKcYIvM)jmE~UAAn?vRrUoxnjV!swGqJ$`-b{QpI@j%Ec>3mFDs1!SeeswRza@yA7Y?Zm4{sZzY@%flx{|4O-n4|{t5dH1Q8$>(Gm zXz&wD18*$>j(Plq8wDWH;9sIadj<_g0~Q*1;{bCGG-y{J8V!^xAUUN@HgAfM0rcV& zXsBCYl-y-O5691>eRc@FMQGCuIRZ(jfdY9OAj`BA-Ad`9k`FHtfR}_+n}V`1Gzd{l zntPinD2##X`LU#hBYyUo`;^M6+V&~+Rqgdt>SuIS60owe zeR^kCRZT;vYn$Fj}q zx9#d$v3}>eS-ZAwT;I{Qv#WX6#*I5P4((VJz=VJ4zh!%8*Y>W?nOipQ+O*j~LOu_s zf?X=>JG!c>XF$JbbwRfcQgZge;JJTy3|^&jyi@(>Re}arwaR}!Cu_nnc;p}smZrVN z(9c^P((l4w;;`WI{wh(1pVdsSe{QPo4lm|Z^B>5Z3Vf}PZtI9 zQOa}G0bxzHV#xfaP*KL2#W3U${|1#9E{43p7&1GDv<)e9KmR7s=iF~oDRW=uSN@CE z{4&w{UpM{XDab+qiFNtE2Ng!^&J3|mqIJOlz%W{ODn{#8I-*gN(Hd7N|0BT?x@bzU z{ue*o-!=8ZlO_p_6|D;{urw$*odDlIW%w8IW@%7xmuS$HK?8}u1+N2!(Yi}V>sC@? z7{rw!S{E?>+t5%qI$9S_385LSTR}4-S{JSWBuA_(b;Rm{1aB=oRY-N}Xx&Oe6q6x$ zWL|;tVrsDWJ^Uh-hF6IJ!iHNq^h*z{G6ps;L8z#n>QkBq76!%JV zQ(5+6{^$CfeB>jpTNKRyPR038UNQ3G{BP#`&xK+5{~~n&7?Y{Y{^FUTlAQg!ku>J` zdsU)=V}LNa*UbLx#=wYaXnzLy7~|p_9Qs$NlDz(LM4u6)-=^XlihD7}#j3D*euBeH zlfZvt>L855P*u=4clO+7`qv=5^i@>B3qMj28PPtKjxc)>2hR2zRiZD2@s;10`)>KRH7)7%ir+_|bOM>lJ0QwBqovGoP*$0;BH_1v*>MZ zht(5~#9wR?-xm?rXAp5cV4-XOL4aWmuwK^yTR{>Z&{{k3!d1%u%JDkDH9-GMEDeTV zK`u+bb^PD)W@*rWpJ;GW1`VVJ=pW4glXMLL4SK2p2IR_+1kg}7x&|0f6+*KHAnk}6 zV8980q*I-gI%2bd1VaxvQ%J4XzOa>qf*Rm3Qv*zJY5+1T#)4~^{@1LU@y_3Hk>pi) zk?i+uvZ@s|L5{x)I&m0JQT{4Z6I4#Gt*Wb@frV4_dLgFDRO!rx3l_y^w#;4FoLIc9 zX<1VYO#Yp^fa0|p5lc@FMGWeKiasjAgru6hKDn9L_Nq2m3_`7Vu@ZnuN`D>l3u=S0 z2Xp=FRU(Jufhc`_j*imVjR)%kjHRZ1 zhR}b`q0jnYKzt+0NYYQJ_(oG7WJkYH<e^=iiS{kHK5Orn;f6p{}PXp!%xfnJqI-TY!xG-+%;}2%|xbX^du9dHzF&(L=pw z6mOj0q#6BAn4E}rXf)`KG)8kOJ^x{RIumc%Ga8`!QKLoiB`tF~tu?dXgA5r-omY860#%JC^p8uHP^H{I>1OpW>5guZ@!7XWg<`47y z#{mV~J)S)u*e-RP;{8B{d``d%wj2C(8m|RIJ^xXB8pm7qy!xrWDZ>=+D{Q0!8FDCC zF`sQn&1Y-?LY5({f9NwhpXr(1>_4dzxeP={EB{GTF7`eUl`x#?HX{ona(c+&#FfGH z#yD06_mdS?2JxH%%E{MbHoC;p}mD0501&N04b|hjrTOo`G_n3>E75aMVu1qr5TCQ4w~pNqtO}~XToR? ziqU2UMk6AT&ch2vJ1F5G(r9II)Qw|^w^qfs^u}lmrHhkc+ZZk{%5dP2^RK|+C{CM* z^pPVH4Y$~TMhu5G`!l^UTn3R!sGnvg#vAU|KQ!Kq##u1lLt?yHf$@k)q%)1A!-Ms| zi6=DPpg3ya81Y(Fd~0uvXNGubOT0xqEx!TiyhwB1dBZ^7HbMDMnqanO2HAYIEb`Zd3>8vdclrT za9eLIhWd(gbk!m;Si$}cMlmD_j$Gx4LN{CLKPQ_DXcRry8-ux`lu#wRQ8e-ehZpU2 zMB}lr*CS%DV*`5;(VCtk|LGvoUW4P?l^RPtOwaATv6q<(D;6%9*R&+8pbLALo+``- z=Yd@tUb)CbS2`lmZY|uD(Y-57&*yt%w+tecP(OQmj?Qry(s)NU&W7n-P&n=Lq5!2az@#8b@1YJn=eJ zd}nWL#+Aw9#FAyriTF}uG+<}Wq9h-B8)aF87ZnNiKawTea{ooKTBGt`?2Xk@){Z0CR1bwd0H)dF$)8>E~TP8AW)a&y&CerH3o__~EJsxj*MbS{*(B08l zJ-x@(Se6Z6EhpnL`3ExmcbW_*PVoGDjSTl{8S2~Gx@x;{1fYlhRWEu*gM4OtONMd7 z(_~mv;rVwP8Sd6HOmC=bsO+rmdC$E!b~q(PhOec`kcfHy14f1ij0|<%U6ozkJ+F*= zBg1E18OD!Ilc9CA=kGHz>=PMk>M<7e2W%`D*sZY`AI$%xPqvLR-3a)k=|)ViZ0qdk z>OOEj`8ijH3Bmd&DMR}Z&%ejm;T~;=+RCoBhR)ia8%n(yi_=nMxI1kuItF?EokoT` zwG1^~9n&j2YYv=0FojuTF|j&LhR*(;f1i=zJ|n~Qs#^5#4&2;4Cq;%}{hu6*t^(8C zgFl4Lz5VwPdovd2xiU;TJk1W>xt@Q!vBT}!4q0`rdLsi@NF0lScp+9#dJ;0QC9^TL zB}2y{z|TcUlQhNG!BTZ(2cvo;Ix}Kav6sb)z?}CjUBwh6fEMZJHs6Ja7%i zOFZh3Qb5w9Hl=#hB>;j)9TMz+X^#Ra5aOm4&3zz`8EXAOlQL4)J!e;LTP37yeIc$=xyRm6R=AJin^0 zvbu6cHU6uaKAk5PJ+F614QZ7BlcRE0MzrvG7U#Q{wm zs#5vtE`Oa$Or_gk`mZz7KO4d(ykNle?Bu%`6xsi^ZMxz(OHNs5T^IfE=WVgryc#~O^LR3X{7UG4Gn!3$4na3b>qrXul zCQ+vv}5%TB3Wl@D`h(%nM-HHSR9d+H`1M^<7af1Ek85}{$%jv!pgQIwi?ti!be zN?q(HB+Wvpgi$IxKd_WysDsU>@%(4FYvEObd}{?lBXwG)DtcqFtY@zxZ&9yug2O+K z0CAGvtP;ay%$j>6L9WVy?3%RXoJ#KhG3TsL%{da+E1~~f;8;gn?!RRDXYdy*$5!I} zi_WoIGUQmP5-Pd=WtFf+A8ok;Bte9enNzYEZC;Tn7E(AKcEc zG^oP)7p=k83>u6EEL@@LUck^5w(2SY8uU~pRQ)Uk8tO(@3DqSbG^+&CW-iE3B~-Kj z#Sz<@I%4%k%xa#0rI=gv(UvPnh*bhxDjAQr8fGo{SRZd?{RC~RgZQV(s8&S2TpzoQ zg&elCm4AG`4oU5G4YfGR(%o|e9Il4xPSW{%?l_}~bv9DvVmoQ%kZHv0fjvi9asNoHTRO^3z7e**mgt>#_Dz3>Ax zcbm~S?d2I*N+PZ1x>`q?y_iye@lR0+Ee-a9PC>DO5zS896lqGRmfdQu+v_l6&-9qa zCF7$8Tc4vv$%e2gEny?=*Rq7J5Bgum zw|NM5ufIHb2zGgIMmx)io)pAdLe~f9e~chJ1p94PPWtFI6qU*;JGW!887L}gxS=l$ z$)LU^HK-4UMKB8+4zrS&IRtxF!^UrNxbb4vB+z0|;Zk>dh9pM!B+M=uvh87Z$nFqq z1N&bTaeI`AuUaAwXQ)&!?tJJjyxAgtEFzwoLBuIW#2LZ)r&ATeIqrz^)QQtoLZ7)W zQ`2*FCL{S$D3n7B5^(}v&~L^v(eG61r;*n*s6;hqULVzvQxOZ0h;X@!4mB3ZH@9>o8o?DKL6Gd8cxmc1-m_8LILbcbINvWM)&ZLoW7h1h+V zs>n^Mj+n;_)-S~y(MA2kgY&O?=!&1QNtB^lwnzvv5l3iR>In8fShyJEMHq!Cq)+vE z5lloqBG~^lpcJ)kteAP28k;Jr7mT?j>Jeu-+$d^DqD+lV5jF6#K-9+_WQsZ!Af7Gi zu@DzlJ>u66XdHY-l&eI4f*z{MZBd7+!8^CNZQilIYxB;;#{jSlEuz)Tx0(w8ahUrW-#;E0e! zqi8WJL5E*6(UxC$Hu#qNAwUsb?~STTV= zarPy-=L-SxkE3}0J9MRjZQxv!(UnIeU715{`NvT~{cEC7g-yb7WvzLNe;oCu!@m_u z9G2=IV&S6;Y|x^C;FCGpIef&zN6!qA88L|5L$o`>SorAR{Lo0=mEJc$Y;gKXJKd~e1H?2|1mgJED?z#5zW8U5t1ozL4?QRbg(tK zia9gFh!!1tWN`gY9OT{EAulk-hzI*$TCw6Q1zA^N)r_U1)yR%5n|H3?j0k#Ee8>8> zX&qa3ZQi;4w4>&3?p)s%#tU6JFsP8QSS^0JBiUXMuL`17F`|{WG;s(oR*O6SuY=WC zg4mH%LQ^gm%O6b$C!_>HB9$?>$7bXVNxD}EbgZ{r!{~53eNtCH^$zVJ% zK&H_})W*#Yf2OQypsyQQWJfR&wee~Pv<-af$E$=sZ8}ViPZd=gW{{xEFgO0xAwf|? z5@l+9il{`kM4fq%gG^B?0is*4fYhQ&cAYuP0gZ#t$iY~Go9sH+7PY7BS^}z8cAfbZ zhw46?YMC0GB0Z69>3`@TGrNugh*KMgm1YI!zs!K{1)u6MB7K<}ljJj!0~+v|0Dj1> zvyKV*1lctVUwu$(Rm1`$BGd}f&Jby%WY=*X36IPx39_pyag*!Fe8Cs@nB~lWom^`( z7GEPHJ$csOLNU6Kw<@4`NWv+;wBnowmVuiZY~+Gg7WAO0ZwrKA?Z@lI1+nj|IX&4onm2x z$IcmSgX-iO2DIqVBZE6kAU^Is587CI&wK?xOk}=YSYs+E;E1g4#LwK>;T*v=)avcwlOo4RnJe#~8$c{Q$X_TqD zWFXawb5Ti)$vmHXL5R$G?m@QB!^4)r;SxxRL{>%agF^3IPW0EMYlfh)hJ0bzac_)w55_8nv*Rm$3QdzYadb2dP9qs#T^2rSM_ZX7dT|f6#pLpN36F zq!xCf&%9v&+lWLyl~^JRA8Uy`oW=wI1p4C*j@P=_b;d5Ztd z|8^)5TuA5hkc2a)+6F|nY`wto|AJPiGd0!!#MTRH96p>XndiB~ho*zs7HkcXVQalu z-v2ybfc?+7Dcgg2zSx@QVLs6D=5jmlC>@@+l^v`gq8 z(IY9oyQI-Z*4ZOW`^Cr}x~ih<5MGW4EIHZ1c}p1T9?D4LkOgfUySDG#5ymmCBFT`1 z*k?(y{$VitFgv+r{w(>cErU*A6{K%h95gA$bLj+ta@*f2M~qk_>MM7qt`NTfQXJ zZq?K{70t!Z#R4sli%7SniUiZq0xd!R=T_dyQp^@kRjxM(9~O9na!mH1JyGQL(~&__mRTu;K~Jg8tX73WsqZ zJRU-nY?Nsz`W7jp{}(!62^OkK-M-?f@8FF9L+~QdZ$kFl#1`D^;6(^ADIU*g9R1c* zlghX(p9TPq@*Nq*MeMkIsj#|3o1ML<(Vp(_6nG`+hFKou|0MVr<#8q!=J(BDnB~Fx z{|;@KN&u!BMs!{tjQ<@t7Nd$ONDR=tES*>6h|YIJ=)7W_=zLcOoyUO~%)4T?=zN#f zxfHA>7@hm%qo-wxGJq01Ksh}PWuno$5A=3X#3xn+_0L^ebvjRk>MMf&-(6aD$}aY0_8`5ea1RuY2mmMiD1_uhpxhZubOQ4~P4r-diB^vR04Ku3 z(I#3wQ&>H$y-4lHM60)_f%jyh)xr6(iB7Bl z6ixJSgo#dM{G*8;&0wO5V2>DiVz;n*RGY{LtL0{*^~<-K@cDoyMt~Lcf09@$%zW#Y zZ|#exVtqG_*Ge<(`XRiA4T1nm;?tU-|M4h23#-XUv#qHP>CjN{Frzl#Ngna6RmgR z{ScCaM`5M`nwK?rt?Ykr@V*|Qck6TjaPXeWp!dNbhQVt+URXV)^^SqnSd$+IAU{e= zP-lC<6GuF;HOPNYX-|ydlZZiVeIn#QWyoP1pn2KK{8E&8DngmITv6tEt&AbS|5@|O zO0#W}uzFr~At^7$Dlv=W22()*mT2pAklAdWSR^-x3{pioe3ISHt{pfrpj*`}eTgaS z)VAv#Y#&;tbxH;bBlF0~D2gDOK4(59WXB z@!bsL8WTPKC9J;7+bJ}w(IuYm;Zt+Mp6@Zd#5p^-{$T?CE}c^=(I2|!5DXWgWGp&X ziFn_Q5U+#vKgD}JgLn^u5X{{1HDUKU50Yx}#+2YA9(el-H_KyTe`H`qOtq=R_O??F zpSE@T`jhc}0@F@z+di#(%SL=@z>aCBZJxHaep}5c+bcIuZvpW$-qqE) zt_^1fV@RTT%a%4Rfpk=Wkp)V0b~s9K?t~kH|G%!M9%s)a z&K^MENPFb$nIX)75i;Kd3ljU_VD60;c6vu7BOwxz@$8xmLUgy+Q^!;0R@ZUX+)@`( zybHP>{RQ5vxz)7=R5<8A%`oV-fQ7xfg82Va9t0lT3XyZYLVPwd4_`r%v{nvpgP|wz zqQ??Hke2H|O8e6&Y5!tL>%B@6Q*QqXTXdGR-OPWK_U9R-{VZT1ZFd|noDo0QXSpjO zNqClfy(%n8%IjtP5wj~9{wgXsB;G0d|Lq#yo44)Z+kS?>18Udi4V$-|0zbnybu8L( zTF1IgZJVcV+|q%4U!^F|;doJ|A2>3_jZ6uTMjE3Eb2AF!X{NL@sfh2cW&NAfj=l@+ z45e~aX!$PmfaF9cs@y5Ec1?)Rp`>xlP&Q#GOU%BF#%r69RQ6Zaj}s=qgt8>2Ve~ex zF^Q+Q(8x%}d@c9CIOac(81uE*{|=*n>0`iDXMX!{)A44F`8uBe=9s^mVa#KIg?-o6 ziCXV6YU;|l29Kejc{4`MbzLF+77fp5U1yA%>n;}^-c8j(g8I69MZC8&h{rmV;sxj5 z-=<4y@oL%i<6Jp9Z;-Oae^f;b2cRc!*}isa+t#*@bzM`}wRLXUvUTdZ^(S{t-L`S+ z`YltpZ10%5aC^tnE`0k6D3FPG$NHm0FT-dchBrJ5SB|rANk1FKQO=Vf&*?OR>?Ma$494CFp7FcgX_GUOX zu>VWXe=oyuNYiISkpJJ)^}^E_4%7=V$e{1XIu=^q#I3p2nvUe!t=KfaGAj zmpT}S011P!;Z|F>d@>FzfdQxlyrj_>4u~aI*}(djAxv9aA3+F9*K(ZFw-7m4Vfqy( z-N^MXt@6hVR*8WPgxlC8EdHpSR6)4PvL52W#$6#iofJf(?6UDXK+-OMOtni~SU)YK zj4zRORT)0wT%U;3Zuypa6}+tqXIEmlMJ#vy=AHc2TqMRg2wgS>_rDl|oYUbUp%FMu zZqWB*kkreOa&saSZg$w-{!a}*0mO3)$+(QI>Y{#TRHnuTP_ zam$KBCOzhD=J^M@)BDlxv^l8%-`DBIhxW(!lwLMJFV=cruNZv@8!^o`WBsGu>HQRU z@+JWZhTk&4HhckDpCGBj!_!AqaZ){mbZZQ&)U<4p$lT3ackMh@-*mun_nL$M*QxIR zoILIaf+?j|4hahePAN1$Qz)aZ=>rY6WvG1 zvGj6vM5Ipz=)w3Km*kXP!=#Qkl-`u{`ilCAIyq~RExCR5zswoLT- zP&;HRpduV{+u49*eETrPA@wl;IOMh<|7+a}$T&>GqtcSFgtt7(g^NcCB&=-<_Wul> zZFbHbk(zTgK0Q-sSSze;pXv-$g-s=kf@eGTe>wR-%3$djgoVP}uM`#^X-mhT@c5oq z+G5cNV?7^ zsjjmWNbuTS!Tzt&j;zD-tO2g$X4j}gRx#Yq->0g!x~8s{kA(thl9zIP*X_0==tEqg zBTSW{05mSxKbiw3HAjX-#YIkL{0j?2eU2+6oJE0nawE_=GJlR9nUjP4k3Z{4eGbgi zV}W>bQ2+c{2hejMY{bZ%%=(Wb^XJr&nFl0{%*k)r+7*y>SRhuW7l`%=GVR=UMFom@ zm=wSER+B#ENQd;5aOuk6-0WDKnNH3vjEGAZqrj4ngk%&>xzHx2&w7Q#%5jX!AtvZV zog=b*(?-{7h85$Kw2BeE6ghR4rX#CH(mcY@JvHXgrH)m>x!f4ei#lT2WF>i!jA(f3 z2Ah=DFeK%MBA83iiOTg12J%$;KS%oQ)RAWQf&l1$(5=l=g7)Aib=j;he->|6d*f7| zf2Fs6n!#JeWlk#>K|j?EICgFFbZ4g~*|nK4L2J%d%|3@W0jZosWD{Lh`2BzxT_Y)>^ONrAS6ea$aK?y1MPvibSXZ7Dy7m(Ql%MChL zBhdp3i&wjch>Db}F_MstkJE$kH)IyuWXjM03(0T@G%^uo>*Dla{cDhE$b8WE=wrMH zCWblv0x`_rA`El-ZI)p?oof1&pG^0%40HM`Vwitu!<3SVzL-TO;{`W4{Ub5VKe%26zwfWl zOUa}mQhjqe@3F(kz-lC8E!J`;w&IOVH;n%Gl?_cQm)=nsn+{m-EP83w{WFF-`K zc+@SI;?0^WpZQcqe^iFi=LuG*b=D}c%BZlBS_kLRqkoiBa*VF%bsv>HBr}{P>%XCf z&p?Z?O zl=`2xD9%^s{eRP;97IcRy#N7qr^$KQh(*Q#u4B!F2{^(YbvbGTUq$ zj7ar%S<<3Pqb{T%x}3YnrlN1c%F2=?Lm{~T!<-b$5S64Cb8g`OF)k+bEtv=R^I{;7 zsTXsu)c>HkMMFm}CC=*$-ps18)VK;n3U}UNwr-m7wbF3w=ADa)sbSs0KqCYU$ zcq_7C^nI)Pp%6(-3~@k&37xFPwCCO7u&=-XjT(Y4xMnp~u7+4%2JRRcibm7`UHZSq zNsD(!qZ=m;&=-LOl>aREzr(1bAA3RWx)V^Rh9DSbtzwcsX^VKDmGzH^w-@5EuPoZx zePuHV0`bIgKFjqF9cOH`<9zl#QDLlhoJE)@)62aWWXR+==MN8|x4>~IYQ!4!{8@mc zz?-EivV(S|b>M1t|dQ4uqvN8#V4u>fC)DUqUfRGBO7)IgzdmKt_uu{VW zmC#=&HbPBs2}MrQ^FMK@JY!QSR}<1aiL!E-M4ivoI_R`x11bw$7L377+DQ+FHuyqO z@oZe1sd{b)AqfK- z%|T)giOR@aAm<-M_I)eNg2jhmi!k6_5Y->fG^esd`+@|x0n29bIFg=o~uM&oXWZvu!O%jOa2SnMTtof zN?drJt;EN`R-e9i6xYU<5*JGUbLiv@N=PAaq4Yli(*V}7t48j0mlm{^jlb~FXd zT8Jd3;02>zG)TO`kYO^CF$$!xqVwlMh zhPmh(%P^XedUx}-r(DBa^o)$|q1rH|_|#k+eOdBf^efTm&~VA$*V6<`p;MHJt<_!!}SbcSK6g9NbW#T~-^=y1uOulo%tOa44($qz(($$v4&pJC$Y zR7qqiT>LHDSbE9-&b5Vq2NlbUF6RCxr$Rh~xU%HGq)fz(hfDsFVj005s<-L1&u zCI6~oy;}1BtU|;0c(lt}@?UbVBOQIbpjf^wb&M*uM6!N_bnsLtVK4bF;rSO9G)7Ol zc}S^T73Vk_2|B}?Ecq{;5uyi6{(hQ8Q(kf8lK&k|R0oQ8u;ee!yyX8`+LHg$o575u zACDOQOTS@_zS;vZ)%W-PsRD1-=wHhEhoe6`!{|#OysT8znjHqh{#vb?q*`_${FH$Z z3Ulu83IpLg{KW}`mu(OwW=AM-*(J6TIuM@M|MZ!z5|`~0CFW*OLIUAsKM*D6hJmm! zaFUU5k}!fLf6{@F9%2Q;%Y*g5Awv%d$>6Sylt4&iD-d4J^S^X2gUmoUAecsgw0obI zbN$EBof~18%Wtv_Q$z{XSH^Ur5V1%1@|VOg^R!_y2ExlZ|0&eGFc6k{dU#9otwGcS z=>)2AXn%aY8dLSN$!y4Pyvp%l+S5)8y_m04ro8_W(=o;sY%AZym;z z8;s4@DwnGPmfVIhgtF!S(!mZo?cEnCwvUm!)RMc;ftCA9dmYA<8;ld0ak(nBm^Br3% zV(7%B9MH>D`cZu8cYbxeqe2`i6a^`LjOt@8G+h`>y0BBbON2c8x` zC=O)J`PI8Z^kB_drdc%Q^@&<@-mxp`$h+XkeKM{&@7R*I=DddeZ%)Pb2$#O*2+O6_ z7onv(`QB4c$D8HS*R+BPr{dZSE`1DO;bGTs{X>^t8&1W3dMZvzn~G=ZlQzy=T&w3I zNy%Ki=6fRe+9<(4wgk^%^ij_aTv~`XOYmzC62VW(Ah^uMYmXAaPtvWDQZqG-Zy(sX z$b}^=b?x0D!!Ah6xtJRq>Tyov=;bUmBBHeBuKm4@T+YY=WVbaZ3+S&6u|ahwu{iMf zk?{C1#^i-RX^!+)YXSYW6CE;i_mE5(YTvO^elA-^jCEJ`TnXJ#R`;9O)-!11dootxy zDG{by!|p)2r{ha0mM~$9C%rqcr%AY{eI1-|-_tGJ&&)8HI02#Zo{NP0nPL3^UGZ5; zo3y~GAMyb}0%C_fuZkpRrrLo6KrSEHBc$7;pFisEvG~v$p}m7e+_N)?dzm4$cZP_2 zwx0WR&%sy|>Abc>^({%}VXT*|WET9WBC>Nu3}U9|t;g~AUYv~D%lm&BQjK&>S$%@l z+!)o@N?ERSU=1bgly!ZDLy2>klw>_?uM44(pflS^%DSHM-v|<G43aQ}nz^ZX2> zF9Gs~dQt0qI%_dwX+=Q^=#0T(?;KM*lDZnt-*w@re;tGWNkZK92JU}S;PWE{zTvtw zfwyid{t_tI0(1Y90)M^-fjR#<4xbMN9$VBObjIXiW6}{Azefsu<1`WY^AQ5yxX2Q? z=(k{?e)HvTyyOae<1P{SybJ=%*|Qt37J<*>{3<}JOihQ*x?y&dv&lTB;y*cib`#b= z+AxOCK#XL#qwem7kD-6((C+l_Id~zS+~nKng?nrg?G~Uv}v3v&EX?Z1h8{ zp3a_8Udk#J=uL=!NpU$DU^Tcn8R{Tb5b6g0|KRMI ze!9Fll7V&he{}C0Vt{@2?3QyKlQO_&Ga|u&-tvr%uJ4^&ca)q6DOX43IKyE1ulU2Q zSnekWRG&SIKzH3=;^3{^|4jNrtL^?HQ{6xM?AfiKbI5G7ZQvejffRZ0W}R#FT(r)C z-1;4xiavXml_fWW9;WhbJpUEyqK|hSf(WTI`ckK*HV8swdAdvdCYTELtN>SXmVvnK zLWg?=OfY(=N(}S>Qm%$t;|T5;N5*cV2I${((050pV>Vjoi$DUp-(KjTUn%JNHnBQ2 z6!k*%>8=zNlk`bj#Jhd1Lw_&m^9E7TZk(l^akD_X86qC{Ka)el6GtGmZvc_JN~2>KrQ1*i_2ZBug7JRz#;E zq@Qp{aQ;^ZQGFcj@Szw64)qvq?7&@sj`LGi>>&uLh&p%R{EKB5s&mA_DlwK|BhUeta!OT|Y&OC<-)d{JTtAo?jAv)Drf){+}&Y2E64=9Zb=1slo?+OupdMNdL5~;IR zBM0n%T5852U^q9XrW!1ogSb73id)|KPlqtGY)H7LNhjsoU0naOd^<0q8oFzZRSl^< zV63`JzI7ztn3iR6bQjM5=<@Bt4Asy)z`+7{-6Dcss24|EZ4A|AbWO61wp*5Ob6Xw- zHrHyrn2UMr;Et#Hi?c0%H`l*Z;=%|e?jC0=@iDN~?dSd1xAA5varYci;^GWSNV2#) z*#Es)bpe^f*f&^|jKrcOEEtmIr|v3nMM4s?+Q0jLv4tVSgh?`bMN)>Hs*nsVr;TKF z748n|AA?Lo4g-B1-8E>~ri+=elPa^w#7|B`YK9E2z8G4eO7L6+S8 z4y@ew1^a(Vdk+Y{(WV^VXp^+JVGN;cx$oQRsQnt$E`zV?LZexgTXOe5z;Xxse@VGZ zgRj7K?{YSbIe{!cx&Poaxd&mE4>0x~Y{@;~z{-99UWYOLWMJ^+g6`Av3}c2KTkc;t z=(Kk^$`O66zF7^i5O@2P3lTg0Wc*wdB_I z=7EbR4-QO|dr;;&+%RUix9t5O`X4EAaL6$1^%*}IX31T4VD0_jw;aZ_clj`t(2UE~ zFl$~K#t_Pu`=KHSoqjU(AU||Kuw5@TriSnfr^W znVS=s0O)_s#gAsA`y%|L{(f<^I=#2TLPeSS>V-~v zN5jqTt3NVH#JxH~+((fAlj2?lZ1wi@4|7sm;y$uT#JwhixJLsPdOvcdh~>j`8$tBZ+M`+7kFn~>c0FVtbb_cYa--+6!|}t+w9B#`pHqVUAZ5fAaY-q zLGCsq_oH(~?(1{`^a0F_v*o6+JXRIuCG#Q*pp5rK?{|$2Mr1}GfT13J%x0+f0wZo& zHWe4wltjH>&_8w1Dwq9a&@T64%t{`KG1A_n2p_BZZqs_6Z`{9{iz zD#W2eNpSRQj4H7fo~{bu4^8b&j>iieN?cNel-xsP#+F2Vfak-JOEPY9+}V`2$?^DoU`Cg|F2bcB$NqQd(q>kb;E-m}=PXylePe{+Ph4UN zZuaH>@Z{npuHaAb{0jxYC4=BH7oXtyACAW@dT*{2!x%-{=c(r%ggI@kY3S|IDa-hv=!T$d(ihStt{Q{3436D?C+3+X5 zf~LQcL?G@bHpjtq_mE5(&P;~LT3jL%QH~KjS?wS*aEt_q4jh2eV8Z9gVEtM}!H z25U2X%Tx*<26>xLu>Wt6Ujv^J0MRSTfR7~8CtrX_oD#Q0c-E7@wmeI1BKldgK4wa` zveQ%S|I)KEediQ=h!}vUxc{LD;}xEx|MKLogLjw!4-(1O1aK#_%JmgU+82pPdVdP} zKb)lATZDRL=*Y_iK=gX*TG1=&>zJMj^8dXVCPTo9`JW=+t3Cqz#b}$78!iGg;|0B* z4*I|M(y{QUfE@2M|9LsqS*`r%bMU1U<16bsx~i&YR8GMY!rQ0RSGCtqsh`n>UwO4p z@9e6oX+YO>kbk*QUL(}IAObzyP>-Xjxe{kab!|Em9YTczSaaqlT2?mCot2o|JgaFX zx#{PUgD}Wku&4=~W> z2CYj~!;H?(>W22Lbs1@Nc_vu@8eL*2k#C~D`T#6I@ErVbd-oaoKmGFF2)}&hEXxaY zw^LoR^ojRC!Sc&zZU7bfrTH=<@-P>RKpEs`z5y5x@;(`4PdBdxAOL#L=2k%>)0Y12 zC;<|mdzR}T%AEGaLPF(P%>U4H`}{AdJhSue?+X|s>Z{xvmD?NM~3^kn1gqPD#<&DwDmb{fN3~!^XKL} zuvBQ9f1KRT+^GEHvR4RQFetPrzNBSt%iM*{TC!E(Moyy9b3y%Q%twvljZ>k~9?@v3 zXhb|)qmLXZsnPH_VDu4efNG77$2+X{{AdT38g2LI%6`ew%AcFPMtxKfG{OMQH{-)? zesU3Qp1;81L~R-yr$C$QM4Kt14e=~(o)6Z4#tPJCL>w^AbPTkC|EDJDjM^%*I2Z&9$P-`Qi3%hh+7=%VM-L&mtsPpAY6 z8|DTdU%o)%9aZgh)%Eq+5GP@&A70$<5MPA-y0q^zVWNbJFEam`2(b}=`2f#tJ{eyk zqf|xSSXGEE4qUQL$3L(?j-r#1kAjkj*y=^(|K!NyYgRpVB)*zmHyJAV)$G~ZDwpLD z$qMOj#2wNVLVAc91|glDr9#qTsc&54kfupulfy%I& zxoB5OcHbHUQ0CS9bzTKThT`d4vmAO=5cUm!w@TELAPSD%TG+O_y85c>`WZb=7Zy(! z0gI(fCtk4rx2|`n$I(2vkTLjRQlFz^@PB6gfG7D$3H4HN{?|m+#h9bt@z<-wp`y|H z?9CrgCuPa0F9rMGhB`~Ao2hj!sq-S7NA|5zG7 z5WQx5`M*V+2PEt`<$OTsaUt}OCgkocSAZ!T`epV%>2VKnXW3)2mG9{)!p?GVket9= z3{Agxn^iN8uA*gb)i!^766g?wF;5> zu?U%8YqY2oFn6l@(H++b6&b+S){4weWRO`x{c9Hkh6DJ7>H_v&tg!|lEtp$*v3$Bn zm*+K|?Uhy44L$cDuzZ@Cx*$9OG_B>+YeE0paAS!+tFaQH{(3q97KD{jq7%<5(O(bN z{|1jG`lvWy!oC!vN&+vqc? zN==Q`Q0Ycd314TfS1ZJ`RQhgk{wt}{=r~~XVFG~a^hGm($9F3o9vq~B{>9QLI!5^y zXRi{Th)tD}OTyJ)5m=k7zUr`|I>$CvL7f{!ohng>c(yu0|0k)=m^fhcqv`SJWHML1@&K2omd<& z`VQbgoptyDCi~ti4i7Bcya_lOl}`=}c*1J98pcX#ep3s-Z{Xjro0fqfc}gPx{;>{U z)=l&MYo)`7M1HMK}4)ax%qg(0^fhlfvrNqI|5Tc zuk@-^wU(o{t&Sn2YQII(#X^@Q3uym<`9C?G;}Bq-|By=PLm()c9?ITXdQ&t}3Q8ch z{=t2Y6pK+9Ugj^C#W`%fyf?NUgt_b3`i&s}8*gLLbSq)>={kVw$s_sWjUyomlgtyk zN6;?l-Zc*u5L@GE@!##5Xh|E7c zL1ca^L(bzQhfDqNY{1Z^o|2r`*IS5gvbnn&Evd`3OLbOvVOd>Y*Q3L-!G;MvXW`No z9Y&UcLl8>HDLDURI5GUpX*>ks=Nir;DbM+hYxj`+la?t`d<=VCWtP? zvvv7{gGXJ)#tB0-UJN9%?EdjV4lH$9>|ZJmpuw{H((FeD%X8{7KfbgjF{cSV-NvT4 z8Kb4bk1|59A7AS5q+Z80;ykzi718Tp(TjMNUO(pkmpew|;(!TjS?a~4PaYpSu+(b_ zYZ5)d(RDX9NmgUTt~>Nv+5(lDOdph+wLx_OWX3&F9s_sg}*t)VMTT3H{yAv zJ|3c>=g>q^hj^AcZwC8cX717NCd2{b*?uPx;ms=@9#p5rzf7uNM1;$-_dAIQ`Wtz< zwb25ptcdXD-yBZVWqKTX)MUK}%RgFjzd z?&u`X*U2i&PIAr|2x9f|e)@z%oHrh{5hSoGqk166yACM(qc{5}sl?%gg&ZgKMvi>0 z{Ybb2FL?R?1pPlfyt+Sni@!vgROpW`$=n~U%(_#B7A9Geo#^vF4?Fs>KYE3~NE&p| zXOY$?E2kIYmR)~VZWGu2(NoFawkz`-RR7sA0987Fah>^=TLMVM^0Q8d9ygay_n%dX z!$=Sn%d=Y8>KSd*vC&u4W0GSN=>lMpvIOd%x$~d>(Tli^IEmB|sIiThsXv;%^#h(( zfAr_b|5kir>%#Z* z=)LY4Du%rs+^V*+xp!SrrWev@>k>SEG2Y?9>VAR$RRjuDWO@L<+8cFq(PuQInUzRB zIR9dzV=K~DfrcSvyTdF(OHqmZZD89(()s)A#F2pBKtf;J?++ zq0X-84V=gmNgV|~CNc{5tY%ej3Vcdo75Hym0vG!YT z{-<-@u3>Cz%k2K;C`gi=-LJW`TZ-jhHUfyn@>3B7{x3VMV%Ze1 zBJ(p*GGqM{w)1kYA&*5HPrO&C1Ttg&qh)?BgUl>-Fc$9w@&7qRjrYOG_8DKvd6fv*PUD%7!*PxnKx z;K+XJ=3?D#)RFM?t7eB4BW1Jywd79V*K~Nw%5v<3kh+x@entOJD#AcnNSIvOikB5A zfAu2=mO1wf|2ZkVkaM5Yl2mncV*99}=Y=JM$R(goNf@T!A-@jtpD`%|<&D_LO2jA$ z0%c3~9+JJajTDqXto`et{xecAP`<{ZPM`gNj;5%~(%MNK#l_wo?pTn4@($WTuPY#g zwzf>*ciI2qEPqb79%2~-<+~dI#6bC6!~}l#5^DmRK>2L{2lBy+Y-9qzyAM1RsjyG>)jWh6>btJe=|m8em+9x-}n}l0e>Nt2Y&eEWkN+{ z{teeZl=;OBGPeR2F7=xW0Ym40QG!m9y4qhOose0|U!z?rt5bXCG+}(^;$?G}pv@_3 z?RDToUR=qLU(o+I{1|x_HsZkv{yp?zM4qE0@(_IXDiOVMOCaI37L& zkKYFU|D;Y|@mESe-96QmrM8_>L(5Zc*zplbo{DX=!6Ebh=q| zsui7rJt~<(?*->Sl0$V^95A|%4)WUX$Ok&T=juk&2(Wd|b zs#RiL1|EN$?Z8r{J^pG*#n`)DoxMufyA4!oS+&UY;kv;kr~t_==>H})THc5+jQ8&n zjgAnFh-YgQ^na6%H!ThreOM2mwnpy<{a*u1jrRJ-%g!J)I=;6Wxm&kvCRiH1|9QuL zoJuPiahlV=Lo}Kp8WGRZ=>4Z0Ji1+F95DJe#6TnIEWZDH2bLOL@2`?a38S;PDtnF4 zS;QD6cWyht%IYkBu*~5^ZB{lmK%3h|n+DN_c$PLF1pV*i_*BIKqj&F8wE5sshX=K} z!CxUa{zID;z1Ak&xNUQ_`EZ=WiQ24coDOYn6>X-AHpH{E`H=fR$?(Xrsg46iFFXQm zB)xvXiOidgx`Z!vkj5H@dYO#5EZ2E5W7pTN^vOsfkfnIxbRCU#0?XkbJ$vH6~Sb3URKlhqn|pUyc24ffE=@WBS-#t5T_jU*<7XmG{YgzL$SAV z38cS@4K)OoKpEPO_1U)_(U&DlvJ<=hDOmsM>ETsC^-EYH>!T;oXN_6YWpiVT5V!35 zaj~tBZu*`E{_qBK(XO(*__!9JN|)+r`rh`fkT1A3boS7DB zS9jeEY{l>YP%Kvzq>R``>wg@q|4pP|)AwTDYB8DA(e%Zw7XO*`x#%TN@)M0dIn>c; zF>Ev3zg{Iqh(_0EZ~lNfDN6zIiM#&ca`R3~;M`~;k^B=k|FNSn3HMX!U~vrZr^2(8 z=FDB*qzi-IwSKvBAs5m4f4Tk(XPaU}W1+t%xy0Vn8=Z3zpABWv`G14+pJo(W(a9(x zjI)yzNOGGaNfo@{hyJ@LEDr#m?+GR6Ob z{wL-&SL2-GmF^Um=D=TA|Ffa-VnlP`FIfN11yS=11OB8=s9#bcY@|8x7o7jl&4F)Z zXbuG8@cbvm`Gyt;86}Uza1kH?`nJuDT-3LgT>3Ks2xMmdqs-rkkonVrHWe-NSH8S9 zpdvDV%KjG{8sChT`P1V?=5LA#Gs*lZ`kzS)X_=3>Yx3!?%%9#UGJi8l<`*m~xnHH` ztNt}|k5CC@{+-DDQU;l&4Ebwt{{1D@1?;^DBWAEGZDb|9T39VO%kcScaU?=%Wm|&jSFuFTRu^#IwqfzXtn129IS(Z5%K< z1_M+_)75wj&bLLJOflylf3l*5ssI=DSlJ_u z!cF4jKZl9T-;R*^pQwMs$kp#OQaSnkN#7AFBJ)3a{)aNZoIz$ekn~T+f4bDm5_59A z75-DwS;T>)r*zDzoL*a1ha&<#?lxf~XXaw}HsDPV$U2bp&-Wc>j616v5#am>=)j0O zjS_c=XT_a=O>*!Uck1GR8AUW+^d0H?{%ffNyBBe1rT?UK7t!^7GJA#4^@T#xecLE3 zB!vF!F^3IRIk7PgRqhv6;-U)iY*m8$e~rDV%Jev3^pOF8YE{^pz>_NQ)YsIpS>-7>f?aX zOEG|IjkxJZje_-W(%X*rUzC0#_BLP4UL&4gUz}L7tT}wm+F z>LeO*dC-^T&D-m6g7qV9eJsqf{ z5A$hlc0ej}5p8lNrwvYPBaYelkBK&OL>uB++T`&3Z*qFm-DboAGcJQRIYIrK)Mhmg z3h50!X!Gp9p^fxJx29ICmWE#57aXCeOh@CfQ05U)=2%gNc$PAG zFFANr=Eyi;Dr{vWuI2IkN75;&OoER)(VY~;wdb=}#>BN13zy7mS`tL+?U0O|C>>%> zKJ$N)8#U@|#B&GyM@6IAq7m^djqj#0iq<#rVkV-nBWj+&#eO>klI`X6FCYNIiMauZ}B(EX*|fW zu{Uz$PY}Xvf>DmU9OCS`-Nx!u&p!0rP<^I%eWv3nG1=CdXp500*@-@dF^6RZ^tqIc zS*<(tNoakts!oKsW!J)>|6!sDpKowF_~+VoHIXAMdA~MuMn|&Hc1p9o=l}^P5gF;^Xq$~ zxuI@pUhL+-R`^^WO5pT@M3O$i{)ZWNt|-^=40j-jIm11@x~m>dfu5h!!!fv0bTlhU zug`8r&p7OQlK-es$BOdN>_c(3vpkR@xuW#?2K_&yBmHcYf1?a9tbJo|tewdfo=Lu+;B{Rvp=!|#3N z6TFd)S)F-({|+i#oxPG_btdAJF#b`TSM-V!ZfJMjgiuM75PDS;DkVF8@$=DmLFAJ4 zfMF7RmA7Z>eVfi4uL2+DdlfnP`2rb0e<=;R%kyFyI+lbUgFldVQ2!)pcPoz{7UYk? zw;7Q*e;Y2~ABV)TN`NS|N|?;7`@X$?!TRS_g-SvT&GQ+&hTtWm(Qg*$aWq~{9SyNh zKaM{Q{dxxbi~}L)(CUG*j@4uXp>meJ!OD4#pZKwyq1C%GyP|nQp#%q-EDJU|W z{jmNGSMBOkfU4mijsFbZ2q1L(m#5MFjOX#gKHj(#y5t7w`v?86r1WPgZ!A9Sn;$-z zhvE}nhN1r%Asa5}KESIy+0wFyq5n-GB!)rT=4`z4vWB5Q`oDS@UQZnc8K?g53BW6< z0Aig1{cP*#KFmS8e!{>^02#~y)u6(`{9g2|8L%8MoHgImvu3M1YXWZ2KRFhh32NG& zzO)$uMCJiJ|3csTUX;wwSX6RvC6&Lw@zqqBe=RcqAcM@Zlq~H77*5b1s4fuUso%f) zr^q8guv|Gs2jcd+hFbJtx_c}hu#{{@g&!XJ-35BqQnGZX!-@59S0gs>{3kpBq6u>x zCL$4bXQ&>*jB2 z&SnjteIm*AsTX-lvK$!HzlJZ<_7XmPuz@lnk(h@MW@SYd1*C4J?SXeYmTN`Y?n@YF zlQ59vCk`x&!TWtY!$%j(kiD3m} zwmFxTm7dvK+el%h?Xo8wDcA`9E>|16fP;=^wULz+wvLjv2aR;7Gi~pq9rQjc%%S~_ za~1wJi08jJrqY@Vya$F3}wNkQxw_|p8s&|HfN;A9x~B3Mxt#< zLz_Fo2muPDv z(SrVWau}P3cs$rhwC+f>zol_oIN0L}Poi~2p$%jG82?cv|PT5BX)QySW` z0UnPulUpJZ?aVZ^<^4V0#zeF=k!Xznu5K#;&s{FQMWX#K4Q*wK#|PsOZFLmd@QO6F zRee1^@rYI?OOJmzn$KrxklV6IwDkDbn&;`8`GM9FiN^Tn zdP!TZ$EQM&+tMhsk)>&9?KvKwy+E`jk!ZpCFFCtAJde*fAll+IG*AT#pXG%kq{nod5KeA4ho07@mKpx4fCbTPA`SS)C?|ys5o~ zi;Me8JiW}~Tai&j*VV)bu zy2M~z5`k3_^#7CN^%y1*59vU{i=(g-X}liCJi5qWT@;1&UujrRU@Bc`ur7?kV*iUl zF)88KF`F(hSQn&XNzFOdOT&6ndHRU}{qpfc!?m*A74f2CafaDUg-3s0gIc#)|r3dS94Eec}a<)^a`x%^E5AVWwuj`^P& zyp=%%>3fe0>YumNheiYRuu0CGf8#qL1L(m218AsQV3gcNP+WwRAshF5@TY&g#UZ10 zU<7W^nu|~a@q(n|2LqCW@mA_!z`7b>x{&&X!r}y^K!{=%;WaU@KzT77zWE-0kxMfx zP%Y?Xwcy_}{KG|JVw>@t|Fq37GuVc$drEbOF#RQMql5Ec{{*iRKdV4pf{ME1gZ|eq zwTHIilY>AegSr#)MBQJes>`_p&iHyJV>zv^<~#&9>=}JG=HbWu7@LM&4Q<^WmDRy> zN2)8kr_^+I;Q{%)Iy+``)^s++Fhcr?EHP4U(RkTBBwd5E45LgfB3Cf}zfde{wt-@I z3=e<7&uWGj`_CT!(q?!}4C?=1DE0-&V=Z~mj>pE(rp<-={D0|x^0XN;TPzHN^B>08 z?4T~?R-8Z613)eJ;(B3hBcCBIzBZBZpT72X24AZ%LQD$kpSQKIv25&)pX~YSX?z}f z8-c~!Cb9p=$@RAOwkz<-I7FEY-Zp8wc-z~l-o|Yh_{5~E9B(72a(u<71cw{nb!a&^UKCZiB_22H~Zzq6%L4k%EY^4rc$4#(GB^tJFZs=UiVw z;Dcw0!0+f%hL0>;HpBCGuWzWk!?Bw4M0%Rek zn|yW}o0Zc&w6L}5HfGPpoZbshi23&lkATm+X?#}Ic|IG27^2O+lP zpW|yif1Ba6?cdSDdWaLxb2X>Y#ODsB z|8rg^eQtHN=Q9+-=Qd@}D8ux9nNrKpbtC;RzK}z=_TuFykQMl*A7a4&oD2x2EaUIo2#~;1d^S&qaJ9e2mC6@F5TJiK#zI#@t zzegBJMw)he8l$yiJpXJ!AtRk_7&UZsRkrQt3;Ns}#Qi^mfR&Ld2d1%EH`>EN0G%n! zA%L!I*kFM8jDX;-djEY;-~fpuR0jQzq`R#j<@u)>KBr}`j_?V-Sj^|zfIK9CSH6{| z&q*Ua|8zhgfS;Z{9|Umd<2=Q||48@kkJ~$ta@S^vrPwiK8w;>c*~xd^<0_cGg$mT z3a5bE7t^?H9_IO6_}s{b%^Jm0|-pybFyb>CUOt~{l5qcsU~WI^>1>5Z5!#}E*>H9(a>p+H#Bs|se{F__Lo?PnrU4TF+?8=@QYuS{qSHZVE7(Qab=_}KCor31^ zPC#L(c4}UoU3D`m+xGMDhqSc#9!QXh%%3j_^!y!wfId46n}*uXj=Juk{ge6Yantm9QJ6?xtiLji(dp=E zUj`@?NdMu}zsVy|=+iVlXP^ttel`@k*eI0E_YOcI7MS6ZFwQ;>nMl#x(4NNV%skU> zN6~yq_KY$V%}a!h6wM9n|GEzT8Fbq(1r&CLgza}KtV z68iih?w-6d_uuVImFNx^*-RE$u69`QkZ5-!?qm1A?g5}gHO|YaU`xv&5{Vg`N_^Kr zWczGg9ObJ=yfzg-r8nI(+qq2xEhmQNS$|k8Z{{4mEe@*Y^4LAAz|Ry!^5YQ+M;IDx zH+JjH9gVor>iB<~SOUJ^oLLI@!w;gMcQfHWWk2t3j#kKgw@bMCz}yL;tb*u4LL z{aLT1opa9joH=vmOue|W%KfHTZh>bx5|ZdL4RnTU5z=xS<9lFK(rt3$d$O@y+OfJe zV16bBBqru4sL>hT_%~>&``sJFfb$*qhHMNNimQ~WDv@Hx6TJC9EtYodXv8r@_o&!$ zp=U=Dl8ED1;5~usf3^pn8OI<^T?N$Y#Lvyfj>deEwq8LD^Yq=}f<-IiOOxB}Ua#E} z3ezRPnd!~{2j)BA-Y6T3aVf!#*_ba0PATXd4?B!Dvs2(H40|BcWlm znZf!GtDVlbDUK1Kx)%gSJMUu{jl&6Tw8CtShC^Xy9kS6LbZ?U00Yl$h*s=PU_v}>Q>PbHb%K+#6;{Eu#9 zJfQF>CcE`RhOy4KF`6-BI&2?hB=TlxA^%@n9>OCEpLKCVN9Lhh`?rRchhy*w<2mt_ zwRM$<85J?ek2!Zn_D323vj_^8bZlvBKWF)#ZSDA_pmF)qrY8K5zpiy#MT8UmG)l>t z?R!pc-*U#T4gQ}BJ9fr*wgyE8b?jWRrDOZ<_ES1`ZJ51#+oq1zmR;=&cW>IXQ_+x7 zu~)|D(tYcWw)P$EZL_v++P!&;E(v_Zh$zs8>ow~us=WE7kT|CdF3tbMR?Fy&b6?_Z zRqEIpn*W))`DemdhZj_s3N37KsMHgmwD|K}{vn>vo=CSus&_KtT!`JXifR-~k)qbz z<1Dd^YCpk(a-xA^q1Kjcqnf9d#ErSPfj{HMD{10J3^JhlT<`qXE2^L71qyqy@~0sB zyEIrA&KezMx4H3+4@gbjE^Ll4HTQwdi>bmiiREZw$U-gMJ`?jDuhR z>sHsdR##M1d3P9NwXlr^s~0$+ygY^Dx4R1?LR=h z&7G|}5p+PBb4`0o&9rF&=DNwPwbhs&*4qJFs6siiS)D^TK;lFz>?D>MWE1=2@C`O?6d$ z%d~*?w8_&N+Ht0tuXS~8?UmK_&_CKw*m{_~^Pd6c-3^gSg}G8OZmpkISJARAcv5tF z%+W&7kf|_U#Qpz_qXFZy>LW80;~AQ9?fSO*sO!v!WqWkMUzEjL)r(oS+z=pc%K+w6s>W2aH4fqL(lgms?Vn!uY(Y5w{7YVC5#o zx}gp2ucamMj1blXKt#44>#ZrQ&z~4^8B@UeB*nVEt-iLt{TM5l@aCaOtiPAS`hp1& zmx94sU;A3$Hm$C;eSPOQ_kuNhBNB?GmwNl31Ao47d}JC5R79p}%q^|8mDTOR3R{S8 zlnPUiv3LF{V0_Uy)#Ft}Dm3Hz^%WJO0|D>|znMstXrm;EV)@%~{EW&stEV$Ha6U42c(x`1(rok?TZdAYa#6;$8>?07j5 zrC{e173;d%+VSH<`(;vea%bEY%apT~?NR`4|r5RUGYeNI~rkpC*rNE)WA@x$;}*oLhNK|ArXa%B zR1qmsj4RsO+A6T&@ECchq9IeU@~Yh_jIYI(0G2=n&eN>dwY9h6pU*nP7(;}uhxxx# zSRc-d)G5q$!nkE!TSawnQC*1Bbp;KXGC6?Lt>*bZjy+=-u@M}GLP5AneQGFn~z z7PxRc_IjGpY6r@J(V9LeU(#qGFte@~M@6>8;v2 zFi+a);VgrDT7jA&YQScDOka8-H)KKzyYaZEhj(KwTG+I3 z`LY;AEYEI^X{mraT?0JLi+`^>?fA=faNJPtB!=Q=>{u<&gIgsD6POtN zv4smM6wod zXnQ*6zu4Ba-|^bG_p!-TKa2>AzUSn9d^Dw0O9>h;!o2aX7RE*2O^r*Og!?4nVN=kO zXgSr;ge1CH2)*-9nvgcz62Cw$PFwH9FUZDb+)pygf6Rb38|pNJ9R@_xtPv&lq}c5P z?0(mF!w#1{+1M==Q7N@kvcz}79mZyH4X<* zm$GA%!G+nFPTY;#VYS_$WqP|2)!uTx1d8}uGvmMQZy#bef1(=+;BV(TaegL{};yvwBec@92v->p{9XUO8t~DG2S{a{|_4L)>dN1}J5eI5HqinHj5(1&CeX^SBouf9jKFoU@P zqP1fEH1w?Ah$C&!3jWklI*L}S|5w*_f0?%zss}`(qu3NxP%>dmpXfsyDc;=O2PbTO z!K(2%mR48e@x1ER;nSR&Z0bA1o;ZL(2%=_Ns|8~Tpxx7`n-cYN(*PcifUE)Dt^S@bGrSZ{dgl^)!QGAe(DY;B1XR2-U4*YxZeoRSKAMY zh2Bv4Dm3UUUv2mLf3Gyu4bNBGU-o%3UnOsvAxar|anhw1-f)-oT~U(x>Kn=VDz_Q1 zz$9U%+9gXpJj8Sk8wm%%jUnnig9~?X{Mw^=0zH8J)3D-TGRSSO0 zu`24q+y1cLTmJ|G%eUQ|p&NK|`@_xu*1+<7+SSdPirD{uwm;Cioo|1r!4xvKd3k3h zL!z8AI49IlGaJ15e{J+i`0DZ1PC`9ozS^nQjXx8%I=G;^aEplEi9qg0O`k*pP~u#T z^$@+fi_H%U5arUWn$zl8#Wae=AkB+u9n&najA?iANeflTXS{q8Z>ZIA zv&Emg3oagu+9CVR!GqlGH-aX^<^t~a3(+uR zM-Zu$C`z5_&3`NBj&O6qb+`#dUxSQEpJmzjfuR&PLP4oB@ABPX6aGRs*bl#ToO13k z8LeJA9gOe~a`{<43h+fHoTecoj(^w-y&2vMZDRaqX8k66Aq3w%XJj|Eu##?kXPT?yzNtxWik??!b-OWOcoW)@_QShnM0jx*7GbqaYa3zVcjf`qC@Ox z*6cHbj=P@7o~(P}4Kr^2mDu_9WIHoEWr6pNl*zdhe{`mQZT92%Fw)m;F>qnvcsq@+ zNCLiXHp(yoe_LHP{t~l@AOXjZJ@HnND901V+pzy>9$2d>?=GiO${%LFN-sAx_?>9* zP8tn{p)5?X-8=vEj!M8^!e8Na>Xl}~czX>vvTb;!p>B8rzTL}z-eJq=`x3Z@B;eaI z|E&V$JIOUR14M-Rc@g|Ja}ltT1e`4+?SwIfrA!##F3jFDnW16lMFzN-`-rmgE^^$p zI_%b0R^bU+)!s&jkR)S}Gk}xDCpRr`n$2U?IhZ;o%ncgqj!*kg+wm7Z%wb|zJOm0q zI}0LR)x#XZ#All3!A^PZ3NWP1-FU;^JBRvmH{&mqJBe}`)z+}z&4a--U5-Hz0AnaY z-0OotrQ^h2B(lQ&jFXtcdP1uC8674vt0xUv^OpI&T~Uiar>eHYyY<6~Fu%9AL}z~) z`2mh7`UmSJG09{;nNUv08|v;lY{9t)fBO|KIZ3D|JmbXVMR}h9PAL^rlE#a)k1T1N z;M>-SC(5}`)4G^YUJ)&)9-5Fu`!(V2dW(=&+Y#S~rT79?8k*X~}i947eoW@EWDkV>ha5+=qw3-cc^DC0golg8WLi2c0o zcWDF|Z>48E5|U^y-avYmxBjaMX}q2Bi$OxVHYa{@HpcT@F55X}ViBJ^yTDR}mV3e7 zhf5BL2R^qi8_Ol*DWztn&z-%~l18)bXv9dxeMZc7if1+w(rgFthS|=3$RecKcE#ZU zYU+D~6W^bW*^)1e&`y^h=5&A-%qkPlI*sWZ*TO{8{l>jmOgG1IFV4nv=`fX2LnTeT z?wl`L5^27jjjLe3?}_$oB>(@`NX>G%V2R1rXwA55$$X$^|2r32uy!yVB1&ukaT+vWnRZ~&z?NUqI|AN0U zt-R;QmY$X9wa(`=fmQ25hw)TYLCJ(MouUtIq&lNL_=FptTW#@=ga4WC*PMiU+%nG9 zeJz_R&v@+Z0U-$K8PEN?1!M`p-RKdZ-8A`UhcM##H)hc9sto$C|7RKWdFcP<0fWk* z3(vz(PC0Rahj1k4|LI)#=pN+S}}xw(Z6oEZ(XokuhzaH#U&(g-g=y>LURYMb&yE}Em6s$js|p+ydn8QZ~p6D*=7-q zOrfYKzFxrSkq5Kz|CP=smrI$8Mecw3DtbxrO82u)q8}>&P5-mH>1S2}8l{yKU)0~? z&#rcN9Nk`b@<&p9KzFs7@kd6mkd@0?*4=C*Bq;{-V3UQB6kqfi3(h@w;9DPDYm-nH zt|LJ?q(knm2N6V>)CLE8yNe{bsfknt3_B-ya zavuSD*jtOU@N4u8nd#&8uz$^>&6z!HT|-@MYX#5cRMxDUJS}Z26&=O<_SIT?@`<4OFZSlYbevm) z`R8jmsXj`Y*(upIvp#eNrKt3<7hecP8L!_>j@Rsd5MM67+vuO13-L$i)wQp)pECNV zi@yyn>|y_rrhlS)lF_ST(tr4YpZmfiRHQqaQ%p@S;urJ?I%^b&`^yF5KSChxZ!{sQ zX|#7PSpAAD#ppcmVR*bHB%F zG{rk42{^#~mrnIwa*fHX?g7q!lhwcI3t(kTqgCl7V4|4>C}E{ohW4IaumUTP){_j# z_z>Qx#e+*s=1wzmf;RGjK44ksh}h# z1}8J<<-sWY{}ksFi={|53yOUXVi^kFS6-@G48GCdFOV|-BhcPg*VS`E?G$%*#JwF~ z<7)5Q)h_Y2Os>8o{$-#dV_|;j;uPkq@Zp}MVVTCfE@-QIRGUoMPK z#wWD9)SLeekgvuihxcgS_bA@$+Saw;IqaREwju9Tpdw%4eOXBg@HMz6^ghk|KFvFm zhyRi{-Oc3Pp2GXo<09@|n)h9rH|Cfds_HvGhD+WnlXyRo!uvE_)yX)F%DhwYu5Mdb zQCU@gtcTp{`CkL~<=*^nVDGg!4Z^j0@V;5|uB)u6X>a$&=jo3R6E~M;Oy1u9&w%&o z(TIDm=6$c?T~UV&sH(x+Y!S|TweXfc>vHaYX9RpdIRZ)obNS!JXet7#Ipu`=fSgr8 z0`q|G*E0)JR1N`7_3O+e;iy-vwD>c9J}Z9Ean$3NF;jI=r_Y)3*H^=;%aHXgz6>7- z+L`3g#M6p5F?stRwa)jT!~e9~;v@#L!#~k+TXa(O2{4sXMI}y>jw{z%W?&+{yKxP2 z#qU!l66rP3aw?+QB6D&-$ z-)G!)PGXSQZ{5eVUx2BUDw_7Y>P|}}?RQq=sj%M<#D1rG_9G$f#|#m5ebvt_LfY?~ z_@!7HB^|cNUz&~muu0gv$Xr_vil((3nWx(#f2hquL(6^EZIy}nxZ}2FW4RNsvmmZ5=NVy-FO;o_CvARX`anUNSkqwJqWMvViD42d*YX28J~2`PW-ZLZ06k= ztqqnRHmeNI!BPfsNS<=F?f)iO|4u-t=9y0`&XJam@)L;ptAk1RqzPFq& zfx@aM9PEEI^q|EaaNA`b9zNEdjm47Slu|F#$FB9pzglBy%?^^ zao1;KxHOPTsh<)?JD{cxbN`F&apyKBV7%wWcnQyVBqY&3A$-sxr18#=++?HV0X+YO`>wiIy`k;%A-D=qkYd>dXW4d&JN~xZbCbqlIJO2|n z-g%AKGUNV;Hb95k?AeZlB-)ENjDDSW{wHXy7sRi^;E!}zal9%U+v!^NpU$97SA1;K z5lf54bN$&?UDE3waKes5_y~bmE7yv0S=GBxv)2OWHmaAc8)Xkk$*VuUN2Xz|V8- zM`a_}?%9!z0mERGQc)#Q{O`somQ)753mTCFxjzvruJf!&LJ~ity7C;%UVjHG3oWe;Jr)!Y}atO)nwd zn2ib3Vk)JIWr*vkbKEf8e2b3-K+kTw17Z zhOerrU*B5Q*7-aDI%6&#_>cME=#w%$Io|j;kn^8-C>{rTkh56Jslrt!we8-~r1axR z$}t~_@eNzfEu8hgQ#epnp3i#02j*P3KnI3n4;4ocq-l!d>h9{=CbVbmrB$(MGWV~AnMi)DWV=7 z8Bvo0n88`3L{+v`x8T`@olhm$kEw7!5B3xx-f%1DzwBE2QaAlXZPe0oEvCM`s-mva z+tC;54GU*2QA<`zo!^KIQRlX4DRRDy?ZKP|fI110gG-XC>O1%Dpe<5Nyu()Kw!2cq z{8v}~bFaxo+VN)l;z zdihUKWseu=$J#^MijO5NconO9{l#v9Bl zUDk;Jsj9oapJKcxA`zG0qN+~MTu4eXqoi#BAM%n(*1LKBgOjWuBu}zR1HAiGffEdm zzQucg)tJM@y}#MyS3#U)&Qi+UM>Rl6Oz(c$lEuX8qDD-ZxIdG`s?AHRNN6NhN4)wE z21fhh*E){Mqmfu$n|)&CUD=NTS;mQ#nC!^amKsc&p5%#Wwe12X+myY@JT#@$$~4&# zZ~uERBHG)C858&CVzPG6WF(}?et|c9`5tfon~grRIEytptj=dP z<6I2pc=dA>^UP?>T@J04tqoXk^m4!qb~)TT*s2WPTm4@yZUa^mf>+3VYB;n*}jKe4qwlvva{OdKp@60hx@Spt$IogWC?%_;|}}LtWYQ_ zJM1&Fxv6Le=;1K;asJmqq|gidh#zFk41E{gsMwzPP%o}%=kR|1`+j3A>_>>kS+&Ty z93RHQ{->fq$HB~pYA~a4sQ)Us@hA-@Y>SDk2 zR`@r0(+p9{z-tiR5^+Dp{m+cJA0|g!jur{?2f|8KBv^q_A-Zf>pi)>K2{psEhGP}JLJM_N_FIQSJ_ zbg$-a5FX^B`%D=NSJ3spU}WY8P|rYOOvvhMHLg7)ou^ zN`0=yV$Un+H;zOsqfRTFkf{pp#oJBl@_~&b&ueo`nZR$%9Y-sS%T&=+XJpBUS1TQ0tcxV&Kz|&CX zZS?feptCIEq1%#5L*4K!;vrf81aD>$-o{8ix+Z~FXy7L5??bKaBl9^T1Z0=sJf|{yy~C$M9il@ORN5FO3E==KJCx zlwq{aQ_&h4bQZ0@$oOxThPvU=`ip1zr5UZsJ0x0v@d!#XTIVI#*fbEq&|iE)SVdK| zt{|gAe8nC%-H9#{wj5up6J2zEuz94|WaBhDd6B_xvm9O7>A1}WDmJvWRAUyU#@nTu zHa5g?prBNp=u&g$AOA=fp~=M9BeMPpcVYzKcIwKWPC_|RJkm2SS!5RNU+)B_9C6)8 zrTs$|+dpMhQ5M_(Lbm@r7>56^a&jP;LKowWI(Ss_UkK(nWgj|?X7_VW;siDwwEoZO z)}L7&XqcM!1;YO?RvmK@`QrFt$5B@c;K|U3OR}irU>%Mof&YfAzT>N4^_@VZM!zKc ze??h7EcFuXEKaDz*u`CRc^?~0DU~v(I=}QsOL5K)?`y;!C-)b$EVj+8kCsylO*kHZ z`vu;pkuMLk2$@NIk6WwNHKv&Ayt7z=%;b{hEVNnzM9Ze3_(!G8K z>!0&bUJr&Y#7|C-s;`X0htWBG<$7>o_gavqbE0#?q+gNqpD4o|ol!rt<6K|)t3WIW zf%w&4CPW(0S=gS>O`itDuht2~qBIaYG{moR{(~7nk;(vEc5(kF8NgS6Y-rH)9td^5 zaL%T;@nLH4PtgGTL|MXrF(r`<;A=ep&OW_ZWdJVAuxrKZEd60#qyM#&l1f9}@C@K< z=lG?W0l457k^y|}UX-L$6(`r&i6DY6eC{I5XaGU&It@;08Xhwbm1b%&8Xlu{~WZhwDV?mrSq%nUAW#1>xnmlTD};EZUw z&J0M%;CnINfcTAit2St@197;f+Q5nn^{>yyTG&1XYt4=?k2l7bp<&QwsLc$nKO&lD zLG(Qf5smf~Cufz>W@B6KnDy4QxmUKbiJkF9w)9r7>7GC#PV!92U=}1T-b|Ns{ zH+NcuG~L1Y4T!p=!^wdgvN0VtHou#3|o8fNTv$0wdmQpHZy4$yYYYC&pE@{LT9QQY3u`@l3k&qUX&frP8|HJ4E zE{osnIOU{6XK-^i7Q;}HZFKcwF<9ZT4MrF;P<)d4PtfntV9&TakNp1nwDG|z9JXc5xGE|1@WTPaD0=6Opt_R?+PltoMD zH!b!1(4w&ZfD-nW^MxPw&3I+slElls5W+GwCopM4OP?`_0#y2kUg zH~*37v+PR@|di)9Gay;7=W8 zu<+fVTY6SP_g}gPWKRVK3kRYKN+yiy5Pj5L1?0U4Z`kn}+5ay6LmYFYTinNGFoG$a z$Fr&Ktfq8OS>}|^Ge<2&c|*^&oLkDHBS6trTG7mGZYtWL75!ce{4wZ+qVJ#vmpBd? z(-OPzM#c8Thk8ZuO%C_2eeYId7DKN(|Kq{e--W2Yc$eXgYWUtafR0l|CE>Fe-}{|d zs6@?TK!eU^F`n(2R2u4r&tg1#f?t}m801YeL@5KWL3sDV8%UqsfRdcWC`q2hkX7er zFBeuSfPfVkaTbH-&gH7J(jPpDkywMBYp(iBx>e`zk2dO};1QH_u0H?1mH0qisAqjZ z!|!|ZKRN<3w0z&aTi=Kq!NXPV{Wl6H^rPs!WSGTG>s zx-BhiXI0oEf7Zk!8c8>!OC+qa)J9DOv%t~(EPunn|~&Z zb(}+msn(5IRr3GgaEm|prd{jep{DBcY3xl~ggBRG(-rNWRaB!`iWIeec#|cTQSC4$ zm3*~UXB*W#wVo8U{%-N--n4`0B~{S8-x-fw6Xv)N+5iH#TpzWa9^hQrm>Rod46kc!Yb?&K!jRNbW0~ zQQ255_Xv5 z&qM!t_|e##Htf9eC-E1fkNwdIaA6u4q;W(mx& z)9!X?#ux15@gj@c@us$3{8Mb4W<VI?83Otzlo82ZL$49D^W$B?7{Wf4ZLbA(>g{K7?`JEVeK7?+@t+ky%A) z%$oUyBH$xmOLo5;Mqd?&_0`cAOg-FQl7%B+ufvYaS;9DmzOoV1x9;y~TTIt%iL;pg~e<7&TH#mm= zm3z=hsLOibdk3}eWx`nJmPf#aeS(ake`MJiwGfE!bcT&6EP&YG&riD%&Iw zf{<4B%Y3Usxh>*noXB_QNGscxT`TKDZ&1voBmQMO1Td_3O%Chqr4Vd>iT&SxH>R(- zz2n-AkK#+C8~Y{i|6xbmJxw=8|0JX5M3wG-|B%_fh+nb)BgF0@5PyaHAAE=rLUR^< z`i+xph`(AW5C^4!*s3A^O71@zFvvkbj~qrnxQ>Y`QdX|NyL0k04%;{_zHr5?lbcp7 z!^GXJdGlz2Iry)HT_VNF_MTsP>mOR=J?Ie^@`R1+Qf4KN*r@9k+bgQq*HlzalcoIh z7b6IFau$VtJxgq^xjzhrTHSdv1p|fVse{LvgadmhRZ&$@RohTiVJb8jVxi=(zif%z zi(y$Fb*4Iev6%SYuV0pm9TZZrFJS%8uh{%MNPfnmb1*+zig#peezBu_y2{qdy84RRPNy1UPO{@=5%BAr znyov|n9|m=t7XgX)=llZcTU;4YwM2oDLdNMZQePhrLAS#uJ#>MX1DKHw`bXsljrQ- z(z>f->y{YRoa6l;fgMkaJc;@)XxZ7?(E&B|iw=PO?M@%32v-_3o!GP#f9S&R_>Gzs z;E$16D1$NFthsR(P%llY3r12$_0^R%l@<7w))dG8t=#_wi?-t%w&YgTlF1L(738ud zZvjhK;J04>JA^w>l*l$EG7sM`rFn{r@gD*}&ZJYaxY>)`|H247Ou3mF)ZknF{d{}^ z;fqVe&4%&bp9o{kqDc{V0luxohYBE==o}EZK=k?4XH{I zh{#r&@{3&mWsDw{+zb|p>>pSzPT7my#q#7Aj3?!+Q)WJ{Oli#lyfn#T&p_Aa&d2l6 z=sK7Kn6EqROd?%~e@O+ZlR2K&FLC{oS`SIqS~|~{u>Z>sV398Yi_^66hrh>%(Rse~ zD!8!fVyO{=-pi_cxm5Iv`TpGQp6Ddhd;n%JPt3e$>gDwCdZq>O<+J8CEu#R{H?dMF zkRCyy`z)cXvn8BiQ!6s)nieUKsl@e7^%a#~SPJc$$efI&g}(e-i#cno-91U>(a<%W z^ohErYSlGSB*jVB^gGZ0wGr7h<)et|-BHTyntpda(6MG?AvOEEJB^wxs3CdRN4I|i zA4bjo?pxr(nk`S$C`Y3#jQqPli5<#Sqa>(;NQB5sBSj_pvAOskzms5$Ci=aX{{}>b zPK4cTw}L{)q9TR#YQOjNFZX-3QoUaqWpt^2_ynlmd;MQIMz!pausE;EAj`+6H2U9j z{7WO1CmTtE((gU}N2PjpWS5BY^dIH-eO*d)r9@UcCA#aYufb7VTnyFuPR3Nv{{N!$ zJES_lqRlP4S~s-s6sviQ8prJZ@|WWV9yGL~#qs6Ko0cv_XE@L)S{YxO)E^FaJNd?w z1~aCOW=P!dG($jZ#~aoC$Bp8Hqx|TyUPTvt?aYhhIJ#^Tp)dMEz5enHC6xW;@)&Rx zBXx92igFpfiz(P~?9+lN*v#67vZmE#zC!((bn1r30KDEkS)TEWF7xD1 z*kx8#NF7>T<|}glJEX2h)(1`aqDdGx;ft?e{?G3+2d&|^Yj2!CP6|lfy~_BZWtkEAeNF6r z!?DjnX_)Rc@BD+Zdr2I2AKnjVv?zX}(+`e_ANcC+^mqF1+R?IQXGi;%U5QOCJ9o`) z+qAr6b31L@4JF7?Eb`}WK0a8aL{ik=`CL38mxzv$y`MA#pKf6!{YpUo2LQ zH_-oiw~(q#1_-wnQekI zDsYcs$mz;KUh$k(?J!}ztzJkx+F6>qqH-;b+SpU`m@bme^!G2Oa%wlHOqT6 zdo3E)!I_~nQpImQYSI+43v3Cg;(<>{cJ0_RW8Rjwjuzj^edc5= zR{N_r|EZ(@Jz(y8ZV6h5vNQp)yjbn8-u$1YVX&ISB4*H+-u3Oi7WO#G#QNi@mMl8a z=^wP}KArZi-BH}=UEe;_;`*@372R~utoyK8lXx=Tg8uFME#eBGAB`ebvzF%eHqQUW zby|h+QPKiACybSiAOoDSCi;8`U~NS?;553dcluc@o-kSTcE%<_C9whZon;m=yQ~=~ zqGnMjjv5KncP_Mw66(xxPGW@aUB@LuRn^*G$SDzi_RbR)3qti-^mfK2K_#&X^?xm5 zLOlUR@=Tu)IsYT~U!&8YRquhE<3@^4^u|3#zP}DPYj3m$`9JG@Iq0So@pLcY>4I#f zJmwLRphk#p_ldmTb#jDmF(kjc*Sk91?ap<0fzW%Xt@f|_$4zN-T!q@pJp}c;LHqYQ z*UjNM?sSd%U2py)=v>#SjA*3FasAE}32#d0Ivgy>rb$@c;TxbGdlT4Tj`OTG!Aui} zch>t|G*ZR?=w;Fry3nUJq>BG>yjfIru6{QW z9?-=Y|JY*@?*q2<#H#=EdV>W}4ngl9cK>%+h}AW@RqUX`JJ*Hn+gdjH!OCY&#$vU9 z_OXyL()pva5(Fz1g$an|#cKcb&i@3R>y+f~NfiI5xBjcWEdusir^}O1dOOpDK&r6y zfeaFk#T$zM(_8-uhTYp-SLO`5I<70Blw>4fG$ENc{}}|OHms51h)sSUW8lqSf>ghk z#3x}R?d@Ncrge(wQ+4I?EUy7hdC+BXi)6H;je)&4Sdd&S@ULAai(*#2&!V?eo@4o4IIUrOAng==~OsEA;-sYWW5G$=hdcpdhHU3pt zeIC}J(yA@N`rcX#0I}A9zm>FjaxOEA7CaBEL(1gh=iW|Pl1)klpeN@ki})zy#8||% ziKKS#@%{%tIV#$2%9-etnKtQc;)L9d)b9N`CfDL8Os>a7|Gqc>sTxF4b2{`g`St!6 zOgV*zA*X7%Jh5fCGu%XvGemxR?fD?Nh`rzcr^%_f(&W_J8J=YCBp>{s)+8?6o6ZO8 zAqy?8kT3HLJ;Abx88V_fY;Wti(_sKJQ~_ml=`t;Z-T%8H%wI zwe`Q;E?SWK_Y03hhRC^>aRkS9t|0K=O3Fn*_nNJ}baiIp1`o+3H$> z3C_sCUxs#;^HD7_6r237RsSmI^O;(?yJZt`o{vUYh>k$H2J$z9i^%yXsDFT0IbRlX zt}x}OoG(RAwKmd6U$TmB2Nqw>0NY6UpkURixK45ps@?v}O%}&8Nfh&cMi4D*4mU~E zvD^QW%;b#Gm~jE0AxGtW+x_0Y3wwK&i?B9G;Za%RU**J|QR!-qMoLUs%7@pdujXi^ z#wuC~oigaFMroZQ+f8B>BmB<41TjKs#OBZP+_H_ji+tLmd07OO(gla;POGqGQ~Qox zJN;&*x?{2-3ti{Y$X_frZ)b*E>=U)dzbZ_PC4aLS=tZ|^bUEa3(e1hP%TLkhZe#f= zas*tQi}p-z#0L|2Ol48&=xyMF<)?we(=0y?k}}`W8VwvC!Prq%+8LP1B@Vt=%ER=k z#sqHb?Ikf0m&f|2xO5vtL|oov5f}5tZNS(zjkvs1L|n?&22b2MB5p*uc6l=X9XP@l z*MYFg&X_MlyS(p+xDjF6y&~e$ZIa-y8wStsiZV!z2CDrIC7fa!6HyZmg}7r>cQh=@ zI%09$f8wtpmU2^ub5i5NT~Z*1PZmm~@Eq$zy5Y;Bf^Lp5Wm&Os{xyDi>)jJQ6$AdqdJ@AFj1p0! zxIkUPS)!SF5l$r)M*16u@ruj*7g&155vT~NFw!qgaq0h!;_*m?MlOu>HHydZ7wBCe z`M*)T(Sgs?qGf3;TJW+Do)%qgL&cY*v;NBF1Iw!&)P*Tv_@hMnZ^e*~|PXWClB3y5o{{&#@t-!Fq;LFgv znEua3G%iH%V&;F;dqNt$^#QABafg7KK)nl<@A;>_G|&a3~DzP+cY`Diq)Bmxc-` z9jXEbTiTuzsDk24Q}AV~!1zZMCWNSft-xA^NoiDwff=fj^6kt6}Nuz}LU&-4dY6_Jo+zU~i z`(GEyf5FrvIEMdqVgFA%O<}u(h)}(|$oxM_LT?2ojsJD|l;}MrMDH$7iQW}y^p>dH zY@TH=jhHe21AH{ zcRkx@PuDmSZlbRD_$BEY#61J0`7->w>(eMj*Qf|FQCH?abd9QT6P1>Vs47*{&M{0> zIz6R$XC^9@{ck`(KO*-|O;pPKk6u)zyod^iw4Ty$`z7f`$pP)hMmP1NmizdRdSbGV7RG5*n*HL9TzcN!CQ z?}kz|QB8=6x^w-5CaOzgqA?JUDA}F;KSk9k6S)wzQ8%=11;(OJ0ZNm?qq;1G>P+3< zx*@Icrwx*+y!=0qwMjR-ZfJHXRMPmmzw67Q*<7Fq@vI)h{F3x6>arQ7`7#<`kJ%_i zv(<%|tw%cw&}= z2~*~EUjmh(gz+fNm!VA0UZPAxh%!B^M49PI8O?zIGyjv3X3v!(YP!>oPMM?rp?&3% zGQFL?NwalK0!TOoZ&;$|^%n6y5RWgGUHE;R;wbk??`q$PodasZn$)C}aca-!EoxR- z%3N?S5%d9yvfidAt13gBUPFx`PGmW#oPWMt`QP|3#;Ls;!G)E(Bu(W?^4e>?fLg-J zEg&e#9J-)#(o~l>w3Y2$bf_sMlWX|ETPeWMw69@7)Cj zs7G^}dS$PSs_8B3pD42l`R89zH6x?kYB(GRsG_4|{`A8j{vIr)&|G98hVPB>zrxa- zj71FJ8|$C`*fI+J9p{lxzxOCU48!++OaQm10USeF0+#jff$cn#rVKw4&qLOe7Q0b~ zuNdCan)=B8AC&j3&cQ&G#qeS&(VKH8+XnH7VTReb8z#;j+V2Jo4RdB$5}1A+NYk?vSaG{t()4~ckG5Vy84F7O|561eb$zW%AE~-m`O#&8C#|awQA39qmFAd@q|)5Wyzy_FiWO|`6WH8Q0w=3On|lMk2D7rGzHmh50#Z-(g2gZq zhhnAa;|7^XefBMp`fN+CPr4jf^qpY%q*HC;|#Dt|-F)1k~Xal?HetX@5lP3gUHnIjbacw)%gEDj%7@ zqsJAE_&ZAN-g4%at!Kl}@KBk>JNLA1*xa&Z%BHQYSg>{+;yDg)Vv_mqff`dUjY*#% z9_|$7r3DgELIGWI5#R0ik)<7iGPTp3%A6v@ccBN6lbob$@dl9oC-~&7N*YI3WfQ8h zRIhs|x)vQ9m9k3$y_bw04HL>3o`z8&Q=25BR(2>z>b(Dh5XL&+5mM*6OTc?V01vnz z1@N>HUCy=v58(U<0iWLqV9tLs1fA~#9$i?f0GC>T$72!A06YpsjDBgLm;at00vO}e zpkLA!p8N9pYi+;-IsZq%7li{J=#Bp_@&S)2VqhGOsBc>szlU0QAoE{Z_@Xet=>PnV z6dSPcT31*9O&jo_d;z>K4PcqR88l8TyiawcN6{ssBaMNPf3%w=r5hdO<$wEBH+mG` zgmj~W_KF7kk~NTsI*8+6TJx+l;3j}En&BXh|5^QKscw|u{BE=d3>pan0!=_FY5#+~ z_OB_QM+5c~)iNZi1|i$A-**YP!EFBow?7TI7{q{N@HzpvpO!-a!CtPiPEwG;*ZJkQ zs`8<>8~k;YWbN-ywq0CA|3+9T?Ud*SB4?*E2HV>U4E4%+M+f$+#2oAe>DaQ1zv>s6 z)!_|YhIsj(X2`%>@2$=b#=@h)&u0fR$|MId`jA~FEVZY^=Te4*(&8fIB%LiF9x=(4 zR>z@QBN2Q(-szj9l-}7wmW4Za?ZB8pE5^d)ClPOGk;RjWHF*2@hhU4k1<7LxXmlsC z5Y2`zG}$Q4d^Tn3LNTvD@2lL2W$lj!`ILgu0y)iKTBj`9(YmaCM_W5MP>9s~@Hu9^ ztA+pLy;DMEIN}qy&nQ*z1Yy+s@PktCSA?5!_yZ#13Kd3<9LtQuf9aRsDl?2BW{l;b zBwgW(WHYkoKn=!52`k;|DcU#Vlq568kOieZ=R&Vb|F|Ew)n*UhY?E=g z$}(FgeAV`K@|5a`{nX^E_IvnzGpLU8GLW+wUKw5;<@F-sO6AoqY_g>@uP)!ze|)e#x1tyXTuKd_2ATg9s*rKpY)%WU*@z7TLKovE4l_q;9JcJG>_ z4kxKfTj>42!CC)r9)3>MyBKtc?hzrM2-Ef8#Q1tXp6#&RV>Uz}AOf>4an9n7H z05F>4A6okAG?tEOprbkeLwCAbSvm%VCv-NmG`hntPg|2qh^wXWu>XQ`PMX;rF{j^tNF@7;VwhC!wjjPReqz&X z{j{0#axftvG5MHz7N|ptdrkaM4L59VoQIJ_DtFiKT8^(Cpy2#n$S zm*2RHUKZ)yNIu-@t9lY&6;DCkOv#B{1jty z*q_>^vZkuKwr1M22DUr0muz9&QcJ{k(|}W?Do`4)P9*>=YlcjvX2vo8`8A_9S!d*7 zQZwTi|5-EFhu6&b?jquP)u{KvJe`i!#(VQW*Q;E0FTjR0>hbLVSu@us*Nnb5Dmwmh zQ@cWn_Qz@!sbe*Bd$($*=HLb})V)z$;)5N>|G{E?7&TJkZA%`3BMU*>LXFsrK~3|K zk<`M3N|Tw|i0v~g!~R8!nUIrouEsK9n@O(J^apQ~QwDFC6O=9CDLS%cljn5d`(2AK zb*%NyI>d0+!GdM71w@W`$VN1r*vDk0H1t{7p$O&@a+3DaNG8(%S<`1F*R+hRChjyw zR?ZPnIgz=4z7ZcRAl^Fh7I2}r-k8Q)#bqXP{KE!uBZq=WHzH2HnxU)?W*zroBHbLckz?+{i6e}!H1#5q_v{N z&1sa7@$961qQuQkJCx`JCGxx(*ddt9RlOS7n{(k9N?_ct#4D60VcBlI!8SWFBvxk3zcbBJLkFEI=gQC-cS<8M9+4u)YS zpCyL5RT+jm)AWnAKOS#rG}+t#a4XjgRqYn)JF9vEoL`BoOd;|h{#YR1nha5VYx27y z{gE`)Qmds;8GthMtw)q^@d2yaw}#q_6nXsBH*OTfWFSbUu>RQ;9!W+*TR_Jt7n;WU zCI0AazwWt__%Mt$U-seJH1Ow8#cl(+{n$O%V>Bp%_%if zlQ3&UO}b=KBi@`z&0QwBnsnLeHp*QVm~?40JQx)r9$X3eeB;*|tbYX!O`Qb_Wq>$g zFqozp)W%E(h0SIY7?RuslBPAtf`cP)tv$nHv)%Og!O0$<4;pkx0)`yKnzf%dxu|K4 zjI1gBe(hgC&d78$O=OY@q`ICW;wVQzYA1YEk4Ov{(oG=MdE=j>9RH{t_95e+I&b`Q zG�{kgC>s;~&-dl;G+f_=@dpIl?w(OpGQclNw^F@YC+R00A*$6?HQp zJo5J7{mFuN5BT#6GSSXlpU{nf+=)PPl857#mJ6IkbC zO=>iZwM4|t`kkUG$oH$xBtUNhGbS|}xc=>D0_w`WjQmb&rs10)XU#m7re>r`H~d~i zJf&)8G0Zc2KqLkn>6-L$-u%~7s$S?sA@y<`_kXiqo=UD4Nqdhw&C-p6n+AX%O*K4! zKvKP6Zy9fa(%leBsI*tY=W(C0IJLn1?sz9LlHN7g8E;o`5I!l)2%pC>{sp#v!sJrs zj8AFfBxfrSb*4|W$f?epfs(WhcLv%Z@Bny;J%mG?1Ij$b@ofV1X@8!gM2$MCgHeYuPjTf-oTtEeBzd0V%Cl1DDUPTA zQ;Da;lz7Qh;$157)Qp=C+Dgpe_@7EVlSTcVwh(_ z3^U_jhGCQ-r+4*Kh76;2PZ%bKc~%*wC%)A$h98-yIH6Gt^Q=El(KVtN_Jn4BC8o|( zoN%5%d^Q=P1h*6J5a}7Ar`ihOIHA`ir5AWdN=>z`iq zLNXHC0#as0;Q<_z~zKn$eY9XbadO?u~!6yU>h2r!uFDt+9}k zRHl34zwv>1(>>i)O*Q40giZGxnyI{v(>;e~V!Ed!?R3wf>R`G@gf*U&BCJQ0`ZVQt z2^BUg5H<_Kx}+C2Ef6Lxw$a=F{9;IpZAAQInx*Hb-^WkR(Fbq-E-G?L}2; z#`${>?q4lAY?~!s+Z=+n>B|Xzih2#xfZH_9Vnet-o8D-cJ{tv>t-yfWDS-^ha=EJ-(*uF>NSZgaf;|FvY@r2n7e_5UBFalaZM zpmeYQ|A5W2FHa<;JxtM_?l47%=7HzB=yv!4JLSGdz{5Y{u+S+oW*#Cj`odN}Xk9(u z`mwuHYM+-$caDKVCA^<_fhG81EjZoj11-2nNIl$Nx5$Y*59y%lB^Nry25#NCf*XQ$tg^Eb zEf}Co-18<`ISoKS-8G(ZbxyfN%O&s+Zw?2WrV z@B|zbTj+E(a2Mnj_epD0a6_TdZ2uhK2Dpn#v0S2Ymm0Y7 zV=mj}eD41W1k>ISoKS+xoKgd~77S1(?)hH+7vL`LhK)NKcQ*rfQI2u{AO$xBCzRka zr<;LW3kE0?_sQP+M}WJeyTb=_&`-J>xQla(`{aWuxFI;91eZD84cuBVKv}rG`M&^n zmmZjo*0_5ZxJz=2d%=tp+z^~lg3Fv925v1FpiJBgy!B9tOOA2> zEd@6O>qm~2IXw;BS}+p^Bc3d*vB((*OVOuy)%NaX;O?4R+zY+=4{b2EO9)OV!MOFy zz^w%{(J*m)^M8T8yWt)kb&c9Wr?-K-G`F}HwOa-wZU{~&!DUWw1Gg5;%Y6*oi#}zM z)85_t;3;I<-hB+*-Exb2G4p>r4nlB32`+Q`7`U}yfHH9}KFuO0?jA_()LohjoiYP= z_Z;K?sU?{9hG2b(XBjS&4f-7|nE9q*?=H*ojLG42m7!Nd*0`Z*6}cp^jKCt5H-nYfo7Z;=ysAFLUuRkelA00VdL z+~QvLB}*`IL$J4;XD(K1!QMhf2k>LewCr7roVd%dIM7Gid!T{4Pi}E9-)spcZU{~& z!DY@s!`@o3w_+inPb_lc9x&8#`fJ=n4cz^6i~E!XmSEzB;Di!f z<_tA(Yrz0z;y&dni=4OzVnJqr#y!lyJs`KZR~1--i5r3wN^qGo%)qS$bCt;OlU1`V za^fC@Yt#m6+`|pr19Ob~aZ509LvTU~F2g0}LHyQ&0m{PtfkjT-gJamBpmE0x+=Ft9 zd-ZNhFmXe0LJ2OzW#$2HEf}Co+^Zk8$ccMMIc7OE?s5Y+xaM+Pyk@c`n7AQ0p# zoKS-8WAIuqK$*BtyTKwS?$~H-8PT{$8@Pw(7Wdk2mSEzB;Di!fh9{o{?NSQ{C=>VE zlPz-ME+6AKF^zkSfjgFC+)r78i5r6TLn`g#j#@B4nYd5Sx5$Zm#8}5E*SNzHSTc+?h(1gov5`06E_6w$Cj1hnPY*U zXu$ww;!fagR4}kIXUdH!Q)#4Z(UFmAwH<3kE0?ck?8RoVZ6%z(w^M z_XGp?sNCXi{# zi@WupC78G&SU-cj%$Z`~)`9`b!u@TFoVX_-Js+=ePc?9l&n@n@8J1w;hTwz}T;@zQ zaBIOFGw{v%Hq1k`oey#@OMjGReV1blmN_CKKH}!pzDKa>66a-N_%hw*)i$1g=7i-( z+`Nj?SPfPGSYB^E=A1D3V*i6$SpO(_@p+>z&^}ODd6UIaCsK&7it%2QUuf<}pO6Eq zQYuXYXze`zZ_m0;tZ)(&Se<1~g<(}CI7(C`?GW-GS>&`>$ZGb)9N?xqdB{*7=gq!<+1GQ=Ikk#xdxy8K!=bsJ1j2csG zoWvB3yT-t+1gjlUhP^j@(IO}Akk#y|xy9Y#?f(w!T~X^KrfS@^25u!d+SRsqhj;%= z;3pxg*%dj)?d|^$a97qji3*Lo&cLk%tB0By_TD%m#oi&S*_FA)z43NSF#V)znv*;^y@6W^j&`^G5NfT)J>9^q1V@><81~-$HH)0MLsqlv za*KOQwI!IirycJk>NM`-4ctnwditq>d&>@soVY_)v!~@4_v@Bm;;zTDu%~I>$G_3A_3Y_6MxHwUJ>JC~b<+j8c}JZRtq%Mf$ai@69|bWy zWJUY<+#=s`X^PEf@Sykcg4{gltwcu$z}_Zu@BWtndB~dfjNBsM>Fs|Hva1u^Sx#bx zAfIJ;ixRDl@fpZ>UTmpOKMz^eJ|V}*-?s$Q&u6-`ox}-(e71pHiH;7kZNBSxivf{` ztZUEAE%IGovIG;k>oz%wnS#8@K(0hb2PYx_yTyRWLsquk93ig)0K}Hv-ubViXX-cP^ke3iP=KbR|2==_dpF+1~l@AiRVuanH&v^s~MBufSnu zyYm2jmO!6pKv%M(!;_$=_WxL&O}ENDJGao!N$vle+>-!(wm?70fUacYcgyPM*z2Ec zrXdU6O}T}>r=Qix=<9Rb`GDRe(B~V_mF#G_4SmllivyvDtaZ=HG4$sx!GwOIdorNU z5$Goy(3R}y2pjsjcK$=?An;HF69xJL1GHA z+}uJx&%6I4Xr}Yrg@8U+pf5C_E7{ReHuUqn`%ePsA&cJgatr+{{kfPRudUu-~EvZG^?pnqg>AoP%>@A|Ly1^N;Lx{@6oYeT;vb^IT)`h9Y4p!GU^o0U_xdC0tj!sB|ZqI)ZddM>PqTE8?+m-@-vAY7$7YXzg26QDmI?;x{ z*Sr5E2#z5u;fr$%ec$jD=u6y{fWBCuuQZ@5+0jWh^nEQB2Ummq1*G{beN^M zmLcIT5$LBF(3R}yWE=X$LsFoJtcNenE%b|Tv;-6SGItfAFBRyk4CqRBbV?F*o`1KS zX~>fJvfM)7zt|E?=*!*JfWAziuQs47*-_4Uai}4y;`=%Ok5zFDwjaP>gBQP-Rq_2_ z)>2UBZMDvw$5rtPmfbA`EXnaY2A%t5{|kQIDy!mo{JWH7sb0F|0nY#6xHRKoeram_TkW0>=&J~KqJFiO;^yK)L7q&>gI=9d-OO1bP+ytPn7U&5B zx{{s8b#k^8@%77+*FW=k_&VLG?=`uFetGKp=c#Tppsx|=%?5NOI}f-_=$9`@iH;#l z<)`Kt`gbhB>~&6aTLAr3f!<<3SF-cE+0d`>?!WXhth}(*^3!q){fd1l(AT=_0R1$9 zzRrNIWan|g+;EsH?DgLx=q*AP%-7}?`jz(lH?4lU+Y0Dw1$wIiUCGYtVMD+2LQ8o< z4_PxmJ;%`hVF@PmgxiMq=>omYfUaccaoOCk`c>ZZZvux2SvF7P7W!3ROo86)wgY-X zptl>)mFzqwFedbOEe^DL$jW(hZlNFAYzd~-Tio@4-Yn498_<>PyxunSL*D)Gfz?A6 z&s%Z}{pxWk(AT+V0D6l+Kf{2oWass9=o<~_N_Jj9fi8oQYi|b!-hK5!@?eBIB6+{|>&D$z&K3Bh^Y_hp zEk>&0kH|pa9_Ichj-B#C z;2u8HgnK2N(+2m3q1Dy zT90V*>m~o!c;THmK8<8m4H$26x|RlbZRi{U@fG>FTEqZ8vSN!?SLg0xoP4eU)c%NT6TrY-4ys`CFEXMtwPHsOlH z^-V@5J@_s6iO~LOgpL=XPm9q06sp3@z=(=3Jqr+DRM?RJKtLjqeybzW`-e!swOphR zPScp!0HN}&O(K1;pFcoXe3x_yb)l6%6rcbZcu1}gB!iQYumULML8FFn4KiMD-81&* z_%Led*8di`!_vT&{NYv^{|p-D2dydURLdCLx^CqU*Gxy)Ew*A-O8#(Lqu?GEg8Q~s z1Gml}&RyBF-o|~~WrDjr4Q`29w>^Y1j9KM={(u|Cy_i*;A9Q9&EkEA<&j4lQ54R7s z1Ttp9LdU-6k{vKA?Do@4a^;kRFk>Y7LuqM%w;=d268!kFwTMA_O8#)o?DFjMhdX5c zSDRc#(^Bsy#NkefF$+Y*H`EcaYwE4p|vs3biJNrTygJpS$OW#>%xU{44ho>8V&5qesa@iK8qrB?@Utl}T%(j^R!3pZRW~SgqhkVzc zEaIaeF7zS?N}Bn@-SYgCLF1TeD39;vdHfJ~yx;K$kC9XP1Kr)oAMRddN>uLdv+0d1 zM13|2E{P4ecgy_0U_i)lhMIcyz=cg>lJcyy`NI*J|3?`we<;$= zlPHQ5r^z3Vd`pl`N=7F6!;#;M&?#wzO8#(95z4S@nd0XULnA7_^ejMp3C|zyaYg!+ z5b5`{i1ey7jj0yqK}T`VL6KhN=MTl{@`oZTf8g>a8AuJ?^F2XQm5hWHKq*-NWQfv( z9+f}TR7CdUFGdaB+f(4yrhzN@!@V^Ex7H6@m3bxLGX^VrcK)y#VYkGLSy!jz5BGZc zUu_8Pdy)V7xOM(8;)ib%zlr<1g1bHqZi!j<{zY)t`}qUznD_FBlKi0OMZcDd{NcW} zmO#d=kf&fkAo@JokbR!y5BJ?+lB@h-SU16Yi_@(%z*`vn`2U!D5BR8x?tT2;1VW2Q zQBiCOMMP0IjU?8)3!x}dgI&`j*y#2GVpjx2>;+UnQ4qTV0v5!EVsC(A@9nj{zP|c@ zo|!Xy=kDE2c76T+KL6#jY?7T*o-;FN&YYP$O|{BTEA z076GtpuIVfcvojDD}D$}6 zX?A?{=>@9=R=>FCXp_uyv<`RMlh7JKh6ZoCQi}{Lry|@L~19 zh0K3a-2q9|y%J@i?n3&1y4eBjfvQF4wU+^BlGEO5(XISu?D{`CDf(r8@)r9D{+_`# zy^C@K+Oe^2ZH}Sd)oE`v)}VH1tN>v*)QS#0W@~(Ww9nVpW+XYyC^^SFH^#|LpYNugwytP_n;MWOSE+KCzsAp) z&s^vC`E}^??UVQUNBlmIhRiy<3-OOMZ6mC^5;m*}VbA-*o=12Ot#qWpHc=DLaL!!E znpHk{or?G=QZU|S;=cjP0ic-EL#;Ab;Nrtn7+>dfUKjEnaQVeF}d z4QoQ!Yre48fDu|H#rRSj<4bXj;-41Z0kL$a15)^>#cM795SQ7tq2+E=o?`|&uhsCYAo2tYjzY?wGm6>m$1^KlGy-htm4~$?*wubfC5bIvb zc16DJLagJ)vRJ?7vljc@{j}I;SPJ{x{fT8Cec8`-%l|_^ZrSIawql$sXRs1J4Iih>CCP(*739w?k_O z`hQog16%y=;2)T@7=~q#pM{S=8eatbZt_j$l3Fo(0#L9KTn&Gw)T7W7?MGyMTbh65VO5bMs|&y0YsPNzj;*SgWEtnb>}Xom`B3ISbH{NfXE$F?Pb$f(Az@Fv z@#}qN{(l4m(>5eNsF$2X+1BYL?U_U*C=sIl;}X%H)s@x74$JHem(F=Gnlyi66ONu6Umr|CQp3yveS(pGcMcUoe*GiW5~=?5?_^^u&Dr zA|22|02X9Zp;_I}{D0i;t=MZ0G^X1FSDX9D5%6y_sbS*Snv=$!Su=U;xKs9S95?ag zn#rsH!XiVd%=@pknYP|yGwp=>u$T(5)oAX2-6n1|4~-@Y{>d86{eRg-b7=jz^3BS5 zz^{K)T@J>eupH^~4;&wmGr^X#eb7kpE>DwK{pf+kHjCEzHjCy}{mdrLsZOSH_&$R@glWn7lS<1-Imj4T6)BXYJ)sf9Uz`9#9PsU6>_9%=0ehVLs#Q4oeLTF z5ay2un9m1u&vd;u=q1Y5q+xln+Cx7CXgZ)Jl(2~HQN=Yqypvs2Rpx0PjuXjaSwsm@ zzZ)v;FrT-hgonBQp&f&X5>Ds&66$0OL<#-HtCEaSg1DxKP5#$+O_P}_aJskH&WaNF zi|>=HFoA5O*F9|FKUM9m*t_>N=Vx#4boX7;ws_VhYivCo5tGJNuX`jf4g!6=H7h^P zqLb4-!2%>^s}Wm2a;#039+` z>La~0)JI;lc_>}uJUTf&5_Bc8rR$?@Y+|MB{wR{nX+CQ5zsjsFUxBW>m}_b}Iol@4 z*(yQK1Y1taHLyuf7O`zI0KWs?qc_<_TTQU#?CET4+oY)|kKD{?1M}TVrI^>c&ku zW%9%`OgkhG(RX>JEmG+l=hMmQmY{D!J6t~7CRX~UYKO~L+C^Kj9hN~~bx%`3P7f-% zl_=O#z~cUM-#aPda^oA?qGRXh86pzMa_tH;|JA-J560}{kZ~_$k=yC0Nljz;s}7vv z9k+)H12rpguq)1q>(C&vUn&QyAS5{06+V7C*p9%Tj)PtCueekb`%SSNY&`a(gO!&L z0$Ax_EAWO*R~{H=1pq4c!&T{efY`&?&2KS1nu`4hiwvdttej&rRk2^3X&{h~+k}MD z+*fk{OT4eB*e^!@wtF#h{$m$av0wUf9yjaX$_M5+XbB-XG73G8{V%>sITN5uT+a4E zBej60%CJ<~8igJ=``?r_75l|mG;5N_x&9e9Ne+EhNL4Y(DwF@xs7BbTrj{d7+^R!t zIVxN@BR<|E_Uo_X5kez{p$gA`nxc|wUKN=C=6Jw#YKm%J^`Wg+P605bFdG#YA^D)yMbh#)?}e5vrb2($U3b-{2O2WaW;dC(>QhA=lG!O=5PZ; ztr-L^%sQQuBpDK>km_*|Ux1mVnv z#1~75VV}VMA641sCL$HXK55qfrzHV_TBE2ZP5d`afnZW*u;YPo$9%ugEsKTTmCj)EyLKWXMa(;Q-^BFwh)Wnq4j{*MiOS_;fhwLk$@ z-1H=vyMq}pKefAV7yaJca$A3di(RI2ivE$Aw97~<_E5Y}!NFAWGNof5)YFJ2N56yIJb&6fpX zZ6ivt30|B6!rF^bfK_*C5(s%<1_*0e|MX{Pl0<~e-vQl%}9bHjKNo`^eP)-i z&s=BnpMLTFj>{I+s^xY`>U!O}ID7WFd8sk4Gx?txs?Sl83Y6x{YOv3C5zI4EV1AbI zFEP(djrrMQM5&pouAOEuKYMwoc$1i)T@f$OdM1Jt?fcnJ;w4$nR3r)J=h~qZJL$|6 zn4jx|0>nH!HRk7ri>TQOb9FPApF1y9yh+T@-5W1Y%+;weKle_&Br#KwWSE)%Cg#~G zFh9@y7ctLCf>}cN=h^=0KISOQ9z>m?+xAv1tp$XZP?i*i>dX+;nYxK@g(e=KdKxP6 z%+D{2Q_%okY6Cnk=YK#e8h~1zh0=Uk4gCcN{AhqVDGcxe$3GfiZW06R4rcJoFI0$9 zbCm(|AZl+txNHs2EQtU$RH8g-1k4L&{F_S;os)-eQncU~7R4nH3?=M?(tO!q)(e=q zsla?IU~W)gGz0$6@m%H|FPi-S4Ne0_G|mBcHJ3Sea<)sz-f#pZ;of+|5-$!DJvA}Y zrG3nHk?2o3lAPT40S}O>g&qui|Wgq7EWjHCh#EVV&Wf-Wd!C;`QDv! z9T>3Osjw83*VdFpit5tF!ta=QLEf!M#uv*;Gk?Y8zvF;h$2-fZ zQW>BaeoP+_b_4>@@aV|O!IguD4ON<*Cz1rwD^CW@M?^DXJQ(sBK~yP-NM|GZJwQi9 zS4L5$jg9Ci(4euux?7+us>pS{Go7mTEPJ5iotZwOwoYpxLIn-c^>Z#bP>P^=byh$U z(Oenh37u;N%_u=bIt$IK&j#p-W@Z#+0^QM|LHiFjI5*2XT^0_2=Jbrw&|#^%IXj5v zwc`Skh-PMtr@THTXhsMc(phL;WB$wU5=1jAiZW`;8H%dbb_hOEH?IZi->t#lig4}P z&Xf>w3Y3$pDM!zV6gD*J)7Awaa5MSr`r_F&C3+H>MrE?K{0`e3sc5_#aWA`-nb*6HBAx?{}Hzj%$?&Lhzt2tcvIjMDOHQh>Pw3wg@|6737xJ5Px7r7-ggFgM-fQR z_HZ_OA1XV@@eci8ZO?FxGAU02sJC|t$Q%KT^LTKQx)BTk)R^>*Z@7~)M@m@h?Lhx` z7(l&D4YG~-Sx!dOw-*L9nE?6|u$HcTTtBX*BXC62Sk-UXp~Hs_tr%K4D(yDc!pX*g zmRgeFerKD2G)!XLI&Xp8a0T28GQq96Te#nu7~ozFz`Ia`c(WsJqlujVB%x6O9N%N! zNlsNq+6bDQq%^6EG?pSXs&BSUT_c<);}bRTPO$wm96uHpF}5Z1Bc0;(M?p>r(hbt7 z1LX_Ccc%ud!8SJEJ6>+f1H$7ov2|z9Afyd}@ZHS;?rdWlSSHJEW75Un&6REfabU-2?{aYC8RZnNz>Sx!^SLq$HVZ zY&QN+OJj6CE5^g!pLJ0PbHT&lCpZ@*ofVzGUmVbq(fMqaX-6`?=zXg2M!S8#I#8B~ z-uKRTs&-=8evWs3`iQo6wj!e7WN^BO5=ifx_|JzlJI0gKpQk)PIzk|k&VuwotALt> zbX62()RYxP6{Mr_2}mCl1__{HsHG z{#93Jdxl9vDk>}bGu8HYgC-_bC{qUV56%2X>&T#~-diZwI%6PTs0Q-1x-LU9a<_t} z4^94CbFYRY-<4%xFa2VdGn$z7KV7x}h0gWXO8NzwtW}ys%8Lq%BSnSH22Ixq=Tq?s z`+igqux}LH$9cWD+(}3u+)3KjQuu-%&Dby3NDEpsGaxcCc-sJXhE1=qObgrEGJ<>@MOh9r%hWLQfef`^WQ}?sH=+`7c7Se7Sh>R%>Hj5%QaD!v9T~}gN)C13CNg_A^tM&YUyT}bzhx67Gy?< zWk_^XW!0bxBy?jHQC&x8Ko;c*uFtLt$Rw_7Vncyzjo=z8xJYN=`t11t9qZ`YD9Qxp z({!uE^-G{Eab51sajH}VgOvN6^l>re4qT%u0jWYKK~IMe3KBq{GyeCRHvwH6unis6g_RO2N`OxLQ-;=8m!Vxj%Ia(B32Ub%O?WNBbdLP#3IJ3J4$ghk}tmw6vgSo z&SLLmr)qnag&w#G9z$}PdbDKznR+l=3Fcp!_(vJUWpeah50i;}GDmXsu%rn(tX!rW z1t5P(krXF({c2?ZPd?PRnisL{C3r?EJZX&^!rZd!*IfVen=5nfmr%ZKR~b&f?vJ95 z8o!vTxGxw2NW}8>*Z@DKvX*-%I#s)qAtII&m9q6^<@HEq6*oD^F_}0QROBp<`s-l- zXL|H1-g>zwxx-Np`CqMXWKy0W`Uc~_WuXyB_rB$wBKL#?(JASh-*6{q8FIg2{>QgJ zW3X3=I^ZrUQ66pYo3(+m6Y#ea^JI=$?&M)%$7undmRVAio|XKEX&K-44#;49-{9Q= zM-i0@ydx91*&f-GwZQ%Mk^uW^3|o1s47u~ga$@*z9}JXbQ(EESX!(6alM}Qik=k-( z+OaF4nN0iD!kIAujQL%s0QV@2x0tIOs-Ysl$0`T9Y+Ai4-LB2*rxw&eFn)JkKnpgm zx9O|O1*7&=X=!1&TaD|x;P}tR^_tSawzl-{@4G=0zjx0I_HIdQeLoO|7~nsUB3S-@ zj1?^F0RN%V_8jpkW*fX=`tP~_hr#lLNrGjg&V8cJgDSwkFKJrqd(8ji2`MKZ+@0UQ zI=Lr4$i?XVAJO^26gvNa^$(v5e{vpPc*P>&B0B%T`44q|D7DT%G>Fa*DV^sf)%l0Z zY-vj8r)J*73bAzl;a<`Cp;S6!{u9?Z`zGq#>-MpaCFqR#kJ9;(Bsxn(^`r6sk2npW zo{yal+jui%H472d4Al*a3rfncd#0qxPKJnTC_}2D6-VfZ_X04a7>THUH1i)ViZRFy zG2WT?0?}hua-_r{q_bj>AI#;}QS_9J-Xf=JTcW|Zut>S5v@FL=2)SEb_g`%P zJ`g&}e3l7>`9_mXfwBdNNvb_W-YU$8CWk6b`j+=Csf!dfJ+Y({xn498NBNT(|FlV& z2^sIL5S(z7E7ETv>04WCVL8g5O#G*{7=gjyD!S|4#M*~vVr?Iz3`;-n8n7T81X8sqjZ#ys9yekQb+l-$^So+q67at+v>pDQC{5Yru|`hiV>&t^TQBDr}Jo% z4s6sh`Ts{non)DipTqh8^<7t;CDO!E{?Y@|=-eMoq4O_x{(tM=K<#92vtXof5uJZA z`Tylfbgm|ell6`CV`^L^Bj&(VEVN-P%42!ZnJa{)1ga#M`g9lk73juj}Rv!MLu z2IvUo<|xW^A{k%o2}j}$D8KC$C`%|Wb2(Z$+yk&2ojw%o0|u0Mo@$l|5iq}96A(l& zH^+F#*INRnLBNpC2J=#Yj$m$yqD)sCj7->m`#n&WU^Xy?svaN5gl%m4U^+V;0VYXm zX9m=xIDz$VlmGXd+buEPb@#f!suftIvta$(x2~EZv2M4C{AD6-P!%!(9kIlqHycXZ*wQ$j5YRjQ6s>D432COr*0g z{eDJ(j+o{}QAU{`MQu#K2l2D2UiDx%wG9H8YX26d!9(MiE`q#jyam&LO#a))G%v;r zTh|MwqXiS`EKL77G=yn>6lIj{QB+})g~9&>@e|W)-1wmS9To4+X@C4C+P5#$M^oEBE`?g_OAZpAMQE>70GLa;R{(D8h*7SO}$9Q+@D}snO zPjEe&bQYrjJ{_PVqB~edX^El=5p#a1qyK&%D9bu}(}QPK^@+@4L;8r2SwtQAYqys| zf|XhP^QeF%qPZi+OKdj?n&Siw=`1vVn*6t4KMSHLqgoM)+Gze<6_7zRZ+RHMRb2v2 zUB+nQOScyX(fqZ0KoZd`i18ximjul*f`)V!n!mXIOhW zZKlYSo5%2aV)|(G@ETA0&V(cik%smc=YPIZ1hX)Pki>gaz|;#E()nPr@uZJUM=*<` zD6<2}_+r;#4w2W3A8D6$ne@HJ!%~yl5Q}8%a21yqBHf3%M{^sKv$_l89fdjL-#7r5 zN#B!E#5H=bn9*ZPpeU2RcQZ?;{4Dg)S-Ld!Xh~y`ah=w~JutwWN#A82wp(Uq*y&9Q}ZY&d6Niv!&4U}bjdYoBN z)tRA*&WhF*)RiMC)%({lZzc&1YT+j|vRekn zKen*$+(^@toe&uHMw+x+n87G5tYvWgXA67NLz(#gPL`7{o`d|S)y46J-NkisAUlWg zKa&H?lP3qVb1?snXC!q(W78k^jS?|BIgp)${U0i!u_8%wz^KFL-{^5xFrmSld0K!8 zUo2n>ZLp=~^V^)SIx}Em0i!d=KkB?9h0d+Ev$;@v=Y@@ZKN2pYb1UY*n1orGTIW_( zqVr0nGlr-nI=4dp)3=b)`P}w1w-;%mbE~^V=as2+#`=GpOEz*;nD!2<*d;;d??mTS zNp#j>NOo(p|96$s0BWwKUx|(R`l?2RA=j#IP+wDoPFCFXN*%(Gp+o$IlPnvP+}ol%t8kz{qS2~##`)E9Ynp*a~CT=nU>+` zC&}INlWi6UkkC(dXPH1MatLUEy3Lz`vV0QdAnfc{dEWx3Sm{w*Sd0f>iHaXddatFhrqLu;&Ix=P?m0KmFnV=q+YV!@hC)(vnoaJZFh;)d$pH*_}HC~ z0%(d6kJD}mxX|OQPSSggI?Vr2ozyv3 z7MHEgAeZ;|IhJFDrE_kL==?+yosUIXG}qh^}J7dUFc<)jgyv`W$I;69_&Q^N_ zl%&_WD~dAeAtV%4UPtGAv$r}dP?mt+Wt_4HMIh&Ua{7Rf^93NuyT3r2xV~;Wz?p06je}k3M9LIen1AHyyuOVofSxdj87j5 zx8#pV*uOp(()^@OcKewDX+*R*h6Mm`qaZp#5RuM8)ZXm>@Dbe|MVVbm#utZ#qwz)q zX#cN(3?ka-O_4|Gu(CNNeMH>9KB8*m=%H27Q7SXTXp3T`t!-`QKYm-gI|eW1Z4o%* z1P~gMPZVW(*l_3^0B7qb17!(klQ&srLCAqjP9F|3I)J0L@+mY7A}B@B zbm$Y1#LBrRhHmP8AZShzG^Df8bf^!|5zUe)%5+bHro*g2S)$qOO^|KmKr`WQpplH| zH3?`;{@<^gB{5`MyiJ1UL_tG33r$Be{`pO9X%uDhlA!4rtbd|;pQF@1EDJQ_GDYKO z+pZ3x>G*8GZmgT7F|1~I?+Kc*f`)V!nodrDj&*Zy6lK)Y4=AcS&XM>;OY0P@f1>%o zo9I+UC<1}u#Prc1Fhom>r*^Le0}7Fb)@g1)DT28-hIJ-yvw%5Cz>v-cvpzsaFw3GS zlTR|f=mo^Db^3RpEWvE?COK6FEDOIjDSa^7uN^*YVXuhtjL5hhnKb89o>9h)A}LPrY}-8`k{P$z9u^wRGxaqJPg>Cl zGHwQs*mc{fHglD6y9`CLZM!m>0`zUo{PzhJ*^LMWnlcC37V}@Lp%tKp&hwses#G=w zbC740vXR2_0vzB|-ef4n*#B0DCg*ys!ghNHxHIE+HFxsvO?8oR8}`3i-^ipqLA2d% z0h!$3F^4s75FMGm`3-j~((e-Ve~k3sCTcZRf)dJQRO1sB&?V^qIG5b7tte9Aw9UrR zvK{znRkR<^YN;AAx?*%iMjMH0AcwL9d6%04$T^=J>)q`?$aHrm$g|O81M|r)!SSye zZLSB)1TsOQ$@b>?4?X4;z#1F93;c(>E>N13*VJJ%SaH)w^so+Y5luLsgI%{r{=;gs z%)3qW9v7ULQ9ho&E6!v_NiC>>VBDc?Knu3Bt2wV#T@gL!h)k^QW8{znj5~}8aA!Mv zhjs`&!Y3VlhpSPRQ#jqxyd>S1JB$ zocTDV3N};tgPHYkc=nEF{__-1Xj4j01P{Kp!&_=&$ARF-+IT9lHpD(Vvi@o4wMpzl zP_WOASBO$;X&*HfT%PT6z-a6CmQ!t7;x`sIB&$OtS7f;iAlkaSryw!&Uth`$*X^Z9 zdL@vsZlKssJwsBibX-0S&<@WeSTrmz7T(FM|E=ZY_W(nC#dB4*g?E}B=MD=i*jQ6k zFU!I^nf&M4L<>tZ-O0{>I{BoZb>i3S@xcO8PdhRH$$EM+1(%&WSiMH)VcHz>&6mPO zDs$%&aA9SxOWkXBCVuvsb*k5RNi)AYpKD7~I?wC!=L12VcfMV8UYAPer!6koi$U#- zymi!82|9lvIzN|0XPHXkc4^%~pK}^O&CKsiJog!2PQOk$;>OwnJbqt-^Z1(BSdXdX zP(=9gEx$K{pEZ@-<+Ok##>4l;kX7(LbWspVnDG)1lg^5Vcj5kLx)7})Tvau6M%N(ZV?rP#+ts{r+ z6TEwXtsj-dNOF%FNzyVS^T1?J)-sapdVc^`HOB3pEE8BGlre4B&4IEUw>Nng%KjB- za-q_stf;UC725Q93LLlRK?`!0aXSzBKdUmB3BHGsnM&JX+(s-8Gp2S%t zX7oamfr+C6dQvyz|6fo86MNkQJze%Y91sz`F>qEdNetOBV9n0zlRenOWgd%UH>4y3 z{sgjzy1Psmk?i`EWbpq9WDj?9Iawyzb*ad@6Ms<2NAg`}4M=uvN;32OqmS*;JePSO zl3kOMY+Oj%W4pS{RFZ6NN-}f)l`n1iE-v#lB)d8#nc4sEldag z_I-$K&Gs($Zj6&bMN;1a3L9!QfT^?#fY3HXT zGwa{JlIz>K(P^4&S}L;LdH#P;+DmO+{6$Hdnv!g2i0tJyZuC4&c3w&{v;OBRxuLZi zWpx4Dxhct3hNQjH%8j0*$<9eh_OB4xt2u7;Y)y7{Dze^Y{m)nOwU%!5EKPP+h)iZ2 zy=x$i!~OFqhWp;|e}loqU5j|7)gAcBIkR*}F1TAwwD+Qr&aby{qi3QNEOcf_xHQq; zCjRqvek0qBo}tOkNJX~CwjpV6X1USRHQDJY$%cf;-g4dOX`1Y`5EGIQ`Jb=k z+m5S_mHBvg_K`JTf?)Or;vF*@I^m8)X-Hu#7iRA4gk zS5yu@)QfG3Z2|3Zug-DqY%Q$9y;Ye&YuT$DrPx~5r)VvE9fbmHEiWf&EzF%z6z2b` zQeIZAg_Dc-a$Rp8zU4=24Mb(eNMWeV{wJSmV>?&n>OpmDDylC-vNpAKRW24(x1^xj zJ1<1F8P#&L*6`+3RA&E!ui^WxU3DD*WZjgC%H+R%st?e2IL!dnjj5>2{LiP_g6_jv z2B>aGMfF<<*N5mqoMeFN`V>_AnDq}|)<@_>oMV9Mx>QtV{oAMd82yM-3{YL0ipu0a zeX38;l{muy)itT8%=(W{^(lH2Cm5ibn@A<&L?kC*fg>}Z2K&^@Df(1|`+wP|URQleIN*aG zdMw`PQ;}Cu#!+Q{M2{VhkriJI$9po~cy-`^B*?`#t^Z1R@yq63dtZ72t-u$BN~~Fc z`A>WxZ~qj(-D6rQgBTW&F{EI;Ab2AQf?ZG!2nv}0r>lO$`9UK<78_sTb*lbQIdW6l zw0~0?aC0=sDOPEzvH~;zd4p9(>%a(Zd}A+5b;KK56n2Fctc*7jD+AV51x)<=x^ui@ z1x8e}HQu|m$Z^~}Y`=L8zsO4o%fXoWSVGN8c=5?08GEGO)E)PxF?hR?PJuHnq2sG_`!KL&=V!!&|DU zF=~7_dXwupYw&%%v9Yl9tr8RezNH%5EPT@wP9{lXD>+pf+gpi^jguGPFA3tO?DwKK z;X5|Anilx6O$G*tpQFdBLWj?DcEG;pE>2Et@IX8Ti$8V7TfJK6^MxO&2m>l8)3-;U zGUj8|e+8fG+|$MP*f2LH`|s+Kt1wYaZxJ;!Ad2e(_s6jP%lnA3a-G5%y1~$38>5I)T+-Dz(VH`|mwg1BHW`{*7nls&P5KRrvIE9nF70Iu#?TlBD;*s` zL4&RIuov}I$;22jh4R@(2E9K#V1IW0T1C5q#5+olzQb7suC%7+_xeimUet7hApVx zr((y+#*QQ;(Ro_mea-yG@3rqo=Q)nr+f(O6=Vf9?EIIO&AwKk~t%vcrD?05CPOHPB zBNO}WRMT&%%CTCV{mlBO-~Lwfd<(T*6Z_Cl%EWvL>f{oa5+p|3@63?V9*$ufviF%7 zZHh4(2`!`T_dtMRCS^w2F#Fq9*>{J--b1~Yf#%Ls@(PY5i|8r=`0srmP1O5TIg5+&A;B%7l zg*V^zg3)C@n96{EfSLci9iRW~iJ>$LCHS(U%L6$7VKDu6;(*UY6ngFfmjtXx*HW@Tj0mms_Ui~C8iyQk~?18jnBX@xK4ha<2s6>Z*fs!U2#p*&u=>pK}uwIpvnL1 z8swQ&H69+?Rb3vD-O#MG8MDE5Rz@Q4Gw_BX59ISN{usi|WRtvA5;-30c&jpLj%hAS z`-)Hr0eTSgKfWR(pzN(49@JLn2m)xe2AbYc`gt%HP6M9blRS#G~lM5=X!6)OLLw}-ZVp!t{KfjNp{V561ygs>3}!dU0A6W z3szvnc`nU8!O3Zr2d)?9`4&$3Bg(563dm1xp0kg>Bt~@P->D;YtyCLk@QcjS@y#0i z<~Y<#>LQB$UxV?hT0F?%j2O?aT=J4G;5Ois06w+>IP@)FdAETG2;9MrciaD|Ynhe@ z9~`_Dtf(wf8p(tFJm+S2x?2u4TLEyJv8U8GoL+gx_y+vaGFCaVq5?m(sI40xarZ$P z8l}yN4QC8#IOX`sW8#0x89OODsm?Fbb?l_kr;MF2rQz_glgIR*GXA8obv2V4hE6%@ zq)Cc~j9QyAaW3u0O{{O2*iheZ+(}bTK1G*=e$)|OpuViO0Di9NXU`mEaOwX2%>Fmy z=UO5Z!=oQ+a}2uwINklzVXWN?DonW+3HAD!^>58TieY!IH(X*BgnGlXQ`$WvuST&H zDX{uS0&R(2ZJvkC@CH^*re4j!62H-Z7Wi|Vem9BV$b%ke&;3pQ`&~6o^JwB(d$^ub zF+MM6SsAALoA&>%8m7O;H(fxJg-^b$VY;F{q|+C?o9GLkMnQ>=ooLL zEGdn2;HB}><5X#!CJ2fojdUR6e>T$h5*z6RutS9od|g;+>=Zn?C%$4MJ*-7fM?+7q zKGXJnDz7ib9~8)cxKTLV>v%AEOIA6IKQa>ykgcWiWZe;O;2XsLPXoN3XaE@%2fbm9 ziVm|L&O>WQ<>G?{qzVW94lb<1O-ZUyVywZtpbTTIO=<_-EDjxhjD;UN;H?<`!)pnH zk^hU2>+nW+UG~V`+vCGRFqr!vh+uOP1kwlxoBbb~Rg47$O~qIT)BiDMHKhSJb&PfJ z{_)aivfGh2hv+0t_TWa8WQ}c3tg!z*11t)H{RBp3xVWgq1xVETiFCDsBG7lo$ z%FG-uGlo>|IIZ9zQ{reF@RteTcuGDl(8AA7KR5rMy3X=Mk@itsnn~V|qcqmjaE=Js zFb@Ae=AXGqlVXQ!(L;`li)IWl4OMWGw^UB1M-a4BJA(9TLMy6ypDX-V1^6=+vL$*O z+AM3j)`{NMCWD%0!UlJd@dP+MPP!?wZ)6_NlT(vp@kYBl)a3tlyW^4B+j>{Z^RdW; zT$%m%waFzcI}>uKiT|`PWR-67&4xNnBke|QsKvyKU#q6uTkqJ2Mlzg*D10Iweay|InWx zkACxgTx_)ji;*> zmHl9~MK#>4|M^s_@SHpzmseEwlkzszF(D0C;~)o|=b)(Uqa19it3p(3@DzKDLozIH zKgVuUne`uE!zXZ}M{JCyvd{Fess0SfdJ>0c;0z5V%RWBCq8d>gqIwEv7R4Ghm3?ZF zO?5(u>S>&3jAMsXzpvQM?Ps79Ln zr*Fe`IGQ+CtEue6iES#A|MjV!#Tl%z8ck)N!fI2U9@6kRoJAX})>PH<_z$-{+f++K zRL|pdR2-73tYM#zYEzl{pReHyIE6IE(HJ(g&mpy_Ms*FzdJ%_d<4kQuWgn+)QwYsrEc=vDo9dzv)k`>R7-tSED*L!$n~L#Y&;z`TW0zw`X)62RWt+|0rFs^>#guj69P7^6d2-o9GXqQbX{HhjZ-#*I~Ks!C~xTqLro%=({?>rHR1 z8ylsmMhO*T1DookkgT`7r`;H%LufcssLn~Cx-CTYw)d198=`UQns%0UnO}Ke2Hbhem5o>VIu}$@9h-x$L35*@0sqC8qEvmzJ3Q@i9t#o4t zYpR1q)|8;E!$*dwKJZq!F;3@T!@)u|If2To|M-^Q;w^V$gEZA3p_-IHW#V6->O=1_ zH#SgH4HT-02~>=Kf_~#8?@>24KvNA6stG}=Bh3CMU)IOoBW|ohQ&k8R10>6aM^uMo zed0ar#`s{MD?lnkQ?i#srm^O10P&URaB@LAlX!9A*wID``wtQsXU=#fMiph5~BLjyU&dsq^S-PDh5b4)g2+K zue@b$>_AO*pinVDvZ>w;QGM;*>&6byR0jwZ10;*;s16~jZ@i^$Y=2F)zfdtivZ)4y zsJ``#EplTenyN&osuQTzgs6V@7P_%wO;s#Z43KPDzl5lM@fNtTB285!R1ApHq~_@S^xHKb7K)r6%i^1NH*0= zA*$c8f*;#QQ|%*E43I3UV_Sr%{^QMaV|#0=y@iSal1;U5i0TjTRyVenrrJxW7$DhH z<3m*c^=@%veKb`cp<;k!Q!NZp{psE8#`e@yc*H`@^cf)8R2xH7e|a~#F(&BIQ}z%l z21pjwah(#WavbkQH^vkKsCo+(102ef@7>t$nre5UVt`~*og1Rc@~(Gdy);!X zp<;k!Q#}}>%J!~vV=MvM(1){<>o`=b489N<$m|I*FKpy7wtbCn^xgEcgh_NIIP5h$?nY;QZdIvH~q{B9z zJKE$yksj&g#w2$YjSZ>LNf@L|k|$xveQG{mq6)h!)T!qFX9fj+@~VqBOU^?>@@f`1 z{#|u)iOWu2Ri7FVhU`jCPV5XA?N%|`8OCTNq|v71O)bs-AKz#nM;Gu(A$)(l6J3yr z(Qs}Xl2`F7Ike$gG-0@`zl`CiPK{Qn#>{_x!)@=)7Q=)>f8tLwRgMh_gv|Cx3^ZBrnj7!c3Wr-;dpH~ zD+4pwTZ{^7i)zb?Ynwi3T4QA2MI)(wJ%E#EpiTAG$r*9TIT{j}v{nO6&xIKN ze;A zvMt`*;Eigo=kuRT<9?8O`K#VM|M7uZ{sMxgmcJUz{!ddHa8oaTHJImLKH#Mm~*VDUxo5VrK`$(x=^qhl})X6!^5dOnn;pllN1? z|Azp7?%=r!VYFHszSG&WmC`;^+_wy4PLpTjtAgpd))Z#KI`V)FjD_MAIz|B!1&3)l zm~8CRHY6a)a+dG}Bt*1oyTy2Ivh+Y(*HBI-ojA`&NtiHy*)Mr;WdT-;k66iUGYYB zo;W978siFGHTIMzeP;1Qj{h82wj_=#3x(BYVWp~03GUq%=QR&czbHS9?mZ>5TFxO1vNr7B3b4o7f6fdK+f>!Y5U`hR zs_7|@M`#UuvGlmUV(CqZmS%j4su*{;6*TD`XvYqm^(MZw0@!iPf3mfFoTRl#2tIBe z$}j}~SnWVt#yG+c!SQ1Uyp=Edhj*Zj-@!uAb_Asi_-XQ2_^=R+KTr^Sk^})=8cZ?X zN7Kd}U4VK*KSW zJfU((MJ4XB<`ZuGZP?I?p_QZZU|_d0vlxQTmTHdCs!lZL-+k;rLF`4#at~;St(y4R zIn-^ZPRUFqKHWeMcFOjS1w+c5f;XDQ#53Y1IvIbV+yg0>e(flZyxB0Aru&DLpaR21 zi9h2?Fa({3Fz{sWUO!WCul7Re)st3P^OhXnq~ikondAE;ijhv8`ivak!Zxbz(&Nv| zM*hYIHLAkoI&414_FWy zy_?t#N5y7hwX?<8_JI5G}gf!X(c*AI?Wd#Unw9lh=@mw{0KgNmPm5I@C6*t@4-^wXdpK8$w`7bpJ z`Wtz5lAVY(JO$H7mu#y z6kA!o9vD0Bpr`MS_QiBk)xu}3M%>H*>;Y=w{@z%&dgp~4{OsYJi^a~ZC5|UFo z)2#n%MMmJkyxx7}3e+N}PgVvNH!#Ur;^s5C|AA51$B9uGgG$)pOq~BQ7(c4Sua29) zKkAQa{DlQ%mUZTX;KI21vm{aHSd=59w^4?1^Jgkw+o6mYSp)@*R>bUu=U*e@1Hwqc_;nk#;YE?yE9nkX8ykyBvZ4awRY$17T2x;e_^nCP)DbVt=jS2 zj@bzkGm49k0}qNBgEtIuj>&(1CPyvd%r}b4!q)SQxdZm`f~t?g?=lGS=oe`$crItGg+|Ep5V?>M99dhK!lo;tY%W}ne> z-thrx9AI-}=fS!Qh#t}Kd1inmp*6ss$N4|KfUXt}u;1Y}Xnm6g2H1re4zTIrF$3%x zRd4kdT zX8rTaxaZGmg^oxd?ePpxp2{oY7YqryO^b@>PZkxwOrhfWb8QuGCX3t8`)w*dY!#P_ zieD#DQKEqJH=_)rfUi{)utBTBnwX3?ULE)!!-{`6dp6z9|8<{1DL3zX$2xpi2&S9- z-#1ARNE9&L^#5;E6aWNGMFG>z{x?$^mQNi8O#cJcVW0R$Ent&(iYVX$j(==a-z3)9 zp&&wAxWKIcey!TV20IEU)KS2mfhd5&a`F78ISFz!0?@#@kYr4|k-6Fbs+c>~2nvcl zj3H{0F~RYU*RerqabsCsaa}XdbLKj22Ms=~qKfx@|Ly!472_sCD0iWWf4+81Xb^3W zTXjtq`hkwKO7KZ~VEiMB85 z2C;Mn->O!sdJMkR-?Nrrgo_4l+G+%9P&R zf=j(+(7LoYY(f|KZK4YtjzXyHOAi*KX)nb-aGH0&T)&D4=6*e}r{|>zj(w24FA)BV z1N<33e;I|@w&v)HJk{Mgeol`+o@yU7MESrfM9^xIly(qJed@GMmmL_;X+HLdJi&|G zyHlruytrNG3)1q;Np;F4L?uq5j?3-~h-4(aRctCE>4lVuNP22kKAoW^B$3$fGPD0z z*AMOYRrDT&0Ajy;{-*tKfNI=+sN&`iytM3h`MCiwwBM6FdZmZhZ`|LspRZFcA=>u4 z{D*)@+OK_V8tk`7>^IHWkA$=zLqxRiEA|Kw(tck@m!N-;4$s;z$;5uxuZ;F*J!=o3 zNi9d_$)2^p;?aOMwA@on$f;9huw8k4CYDRkCYP`rM#R$QW|;iHZaTEt*0J+pvpdCR z=Np@mkVMS;z-BX!3usB3eG^^kIGspW?L?PmVzX9GOW3S`jIA5NPhN@*-uTYn;7vUw zP?-^I|FqcCoOr0ds%}=#ps(7xJXa_bKNE8nr_TRLJ~=e;K~^R zLYnK_=)KtfLOP@f?#;wpQmOv(MaxtcBloHKe2FTodeq*`odW96VrxAdth*Zt(8ebI zjW%Y~$t5nkjm^9?AdE)q7{e92ct{Ba;nyxOMk67OHXUzj`DB2QM*A+h4B;8+uwJ$- z6Qf~y3T-T5`68zY!zJAkKWlVAB^vG--mR_-SHZj0GiYi_RLUhjrv`6nakHKY2&C~k z#c5?gwOL$6}*zPK`{^dKj&M{+E!7tBa1RtQu4i9a$Nxh*s*B2QEp+R0#3MRI1d7H`Ka1 zH-Lu*e9pt+x+)sMM$c0-F<>gSatTyPl=gS^`2nf4;D8Nl2m#L|yLx z*Yjb9%Iv%Uiav;A(@2Ma_`yu9sDsaFzY(JckHj_`W5Tv6Oh^&&u!Q=^WO?o|=)}qL zcS$D8d=wC7o;ykO{!UGnY3ApPT#v)_f%s6Drk2*!G!!?~7s_-t`7A{q`r?5_2nlkd z5pbXkBj>%4oEI>C<2({_j>%XKhb|?jP-Ku3@Lba`B5oz zG@1!^@@fJs+=P7iu#tyUjO5kAtSU|{DTk9xnDE+*LMD6}H#*hf+XL{CzN|J*E6Ab; zRbmbi5z->AHT!>jNE>hkR5ePWMO3FR1}%?b;w@qjI9M(3Iy3(Ja$douQ5@nRr#4eL zYA{uJ4o;B6{m=9OzMNNa%@o}k;WP``s)ooGNfsOdbFJjNPB-?D~!ldMlZ-gDm*smPe)JefC&p?&5bz{yV2 zyr*g2B_*W=g{5XwT58^x317q8%>RAfTlRImb2RUBH1Fcl`r`VcnkK_Z>V0aE_q<&~ zdVh!ojMFr4?3_fU*Ay2=>JSGt%bSBKbr!wnof+c&QK<{JqU?Q!=3UsgZ*kv(ro&+B z&80G%_nRT!9~ZmcRLy&;<{hamC~jydZC3B|gS_V#hIoI15R~o`?c`j=yS%=jsJNl7 zd3&Fgz}t-feqDaLkLz)Q0p1sA-Zgbtcquk*urSiNS5ZA-@Ks+|USCwt1&q$>W>3o| z^G*KG=lxk97XzT;eY)mdT33X+Y;JL+vzn|}yl*q>e?ITecXw55g5IYJ?}CQ1k_NM2 zl)8Oa25oZNZ6SMqv776$R>1pw&AX&HQeIeQY?7Myj0E0)hIoJ3-9^$s@t&f27nRjU zN*c^V8mW1Y3i7`Fs1Wb3y15tw6z?+)@4j_4WzF{;hMJbWZ+|$%`|EtyV{3v<&Jx~* zb&;AHtZWq&*X~uCTJIBryzl4~;{DC8u6MTPeYWOZP}@*eU(me0Pfp;?{hu78{dV&0 zHm-M}=6#{!T~<_B6N!|VIaTWRJuiW`8UKCW-*tAq>6-U+&ATvy!K}D>-Zu$vi7glG z2E7?uexDdy4o5*a_yvLe4-Q9(e~3OJHt@$cd+HQ1#ybTJ<6tkzn|o9>YO7m4BY z5YPR=r+QB~Rr%zP?MqK&5>8~aeF@=!y?tq6B%lZ%&%PaVVx6A^(0UCty-2_YI#vT+ z$o!`ROY0pwTeu}fFN03TCqjn~_)sryRrv>xoLGeMe=r#9BPXbhGjAIttphuD7VQn~ z7)bn(ddJS9kz%1A)Q%k>XllpKqNxd`0XOxIokdIIrMYjHylI9k%D}5D-qK<@`8_DU2&i&|v%KdKf5R$!EauW0T`|MRUj1@T*L6n-U=&ocAK3IAT34fs2C&;9`a z>8-%q;uG54iTO{wHac)t<nBy0D0$Gfxj|7G~P zB@aoz#CvTDB9S~S{O36<f)IOI;mIWS~kGGymyN!gwS_w6Dy1ik!aMIi$l@dqGqt zc7?^WtBaR_JGkqq`_bygRx zQ<_M;+vGoeiP!Drdej>dhbEO+l9|LCMVgWLPzdAo2<158fW$+SNkovI5yoqS67T61 zl6b?OuE&rB5{D#}7|BfH+ysf&ha}#JOMaNqgv3JBaECZ+7iDDNxY?(>kZNp z2PK!78OBS35|`G5B;Jbfoyj|39GFC6NpV?b5*a>P7?-{lk~puY>oGG4i35^K%&;D( z1ts2F5t2B+hwD{ni4{pDX6OMpowqRF`$$OQZJ4<9*An|Dm6)Li;AFy*xQzSX=-GWQ zcRQvt{j|h>NhB7R_06mvITf`f-WZa22PQc&EislfE&lyq6g*LsIJ zRcf0#PEI;3lXhT%;=l)+Jvu(Xo_7;Y*$q>A7hjKdcGJw$vsl^_REM$+yzc|@qnI9F z&)#LOCLEQC1~`-rx+~VwyC38Hm%aPz#NI85jmLmLp4iY2xPS3w!=B*60^8sXBRw`3 zTsY7DDM^}Xtd!yNA54_}q#kYi0XD=J>~IWfTL0fA%9dmOBaQ%R7QCu<`s@>YSO}Km z`8S2&=OhSrL0O<&eiq6wqWoDU%7CD$MA`DY5=sMZ>O|S{x8kM8AyG!&DH3HX=>J(` zKPT4M6z@^Vh|E%N`)t+?N_%{>20sMOW?iw$s*7v}0nP)H zj{FWEs0+6SpRn_ak8A)A(`2~B0Br;obg?h;uGSeiLv*_Ih5)oczP9@z4=GzdY@#;% zR}CK8zv75Im0-%`>Sy@i|C_6y*)Edh=UnIrQvq&4VFL~esWa<8Nyj8`p|RQfVDQJy z{&&@4uo#f#PV=6XjV2gMpA{dI24kHDLuD!649&=&=U=en{Nyy?FHxM)aU325?2Mmr zeI%a!3mH740G?|fl*B+D(pLHHpZQlxtD-cv)8R%xX7X?1V~78{Xo?j>uDUr6lzxUW z$pWuY9@>T>8#5V;sG>1sM{pxF?tft4|0%KWi??2l_3yzbYaK^2;EVgO!(S|4x0?As zdh1`4_`0!L((2<;hTi&Ddh0G^i8o==TjSM%|Iz*b!w({_{=h=86SpNG)O!*>(MGVQ zr6Bk%2?FufYs~!rH+pL#Xv$l!q5o$SHKhSJwYOe#VZ1anQ@m4n>owf}$r}4DvBpZF z52|#{24VFpy){^gx28Kvb{vTat0WwDR%jQjI9;cF0Z?nL+>0 z_NqmWG~Fjn|Nk}q*l~Z#V*ek?{+|uQ@c(M(9}rBTBk)EYJZa)zRh4*R!*x&h9>BIH zbT+l9`hf2K=~aT(Q!{5WjRO7%R5CZ&{Stk|anzX;HBR)AHW^ehwzUl%lIeLQR79%u zsY?UO@H~>)-VJiXl;?OiAlk=hlpffiGPwk0Zy0;(=YTA37~3Is1uT0%EsJf5S7hb0 zW;J1N{OxVLVX>#V{>g5wtAwG%zxb%Mx_$#2#um3$N~SljN*j%Y5?!1OEMRXKD=n?X zY2I~aw?fihDj4a3fphJ4z)6?>Z{-kGzrr`#LvzuCW%dhvV`V8qmZ z0~}{oURTm!{6p$J7)(l#?_lA5O^A10iAwTB-2H?zmKsaT{M{XW_o^!?!sG9H7f?-6 z4bl|ZPUo$KsbOsL|Gv`o#VR`%aSzap8|ur8YwG>2ODPz0SRhle#xsLL8aLpfhCwI+ zYYft?aVku4ePQ!Et|kay!`qzy;49sT<*q|C??W{2nv%xy!g8}IFtw#;32#!P7T4iX zFx8@c-pB8*6R>!O!PC4;%WAQ~s9BS6{2*Jy`sfhrF<6I)q6An+HS6NWqOwS1(`us{ znV%(Fsm682|NB~x?V*xn5w~0z*Vji%OH9;SP@AF}bA+jw`q`~RI-iiISENvlgEi}- znzBfJd=pJ-Q=bOjU$_CvU09=S=?J zugFt6scd}2JxFt|Eo!Kb6q!>^QdZ>d!qhP4{ttR;pYga3Ds3Ba_Z7yGhDd3tZ|W2+ zq@6I9T714B#CSZ;Z{W%ktkGYyF0U<*)Rg*-FZme@oz;U4mZhIJ&;R;LPsr8l+TeYt z@a|huhs9pm)|T9vGuFQz8d;Wp{)>>_6WgggeZ<{gb1ti^t*fasoKsqQ`=CKyh=mwW z!b3F!v_S?4>(ahuH8m!$nY!tn6jb_!TSB}iw^WNK5%)mNxo>%Wd8E*HASr6GM}pG7 zh8R!D)@$9c^ud~SQE}f0+Fr982pPp#wfN$w5bsm7^j-n*9-?^{Hk8&EmN!2z&KBMs z@d(&S~Co$)01%I9$h!>4XG&Zj0zf&C_Hf-uj4u(-9EQ z5nJlvGK@VLIbkCnFT+SXutCA}LM66mDncbBL|M=LA48PilY}VT2?A@a-z3WZu0j-^ zHMY^i8hACnMpaa+s%WgQt4G`PTTrUb%C`S>!l0Os*UW1Ruq&g) zw4v0!bzTDVYeSlE_6E4#5t{iCnt546ZE;~^^YH{H6t?C+g_yt31BDLP%n#3uIUAnE z{N)iL<{x-5{oEjIUay%KmzLq&9}|(L_6J;qx0##$f4;VnDO7Y$!B=tgy9ExpCY`|Iy0-YH#eyFtHmL` zKgSa%Cu`m(XU6;L1m3ekyuT=Oy^)&tNX@$_Qc_mf(7eYwIDxmh|IoL|mzbj*t$81< zc-J>zH(*U!^MPDLkoRk5{^Rrh3Wgr1d5_b)Yl^s^(&(Mq8Qz@0+suD_-d|$|QLA~^ zYTo6Ajgk7I=Jif^!sm4}|M7W$gQr=J)Vz<>yvvH~@l=Q(7^Uj-^v-PGG68!1m{47Q zi>F!mE!5ZZw))cAddw!wNukO6B_=@Ja6m%C`jHUp@37006%N)qeo2>gxv&*a^@e%= z!MDiw*v`qmiwZqnTcof7&zqDsAJ1I^Dyzr8aY{(-A25?;-v#f+%y?g%z}r0k;OqS( zwrGykyvHiubq(c3n1DAQc3l?a{U)D(;B3O@{V#0Bq_2eDV>07?X#(%_L%e^&wo7gc z0q>(U@4j_~*iF{Fy%VNxZ*C6p{&_q7_%V1_Dc-g9jU^ZvnxCxDl2%>56^?&?!6A7k zYQ0Ys-nBIdbN$I%s-80|fw#&3`1bx4&&e>3f!-rD@4DK)b&-}TMCckxd z5hE+!!!zUU32*6^Zx0Re{&y$6`5N7FSo*xJl!sw$)_;AI{N7Q|4Z&K^4KeGitBW)= zZ4$8FA6k>GuzsgSNbCP}&=VH0))SU=S#u4RY{i+pW1fHawf+MKJDh|PsK}GFK}yPs zu_eB-*|2Ah@HMz$z0K0&j_p_O=^&y1AGM+Hubt`M~i@0SvJ zyEup#+gFrH4pY2q%Ib>|Jv6O0c@GWpes7NuZ#T!qm4u4-=*)OC?5575_j?zGcxSbA zy-LlyGBe(WuS}WW`w+ajWcGWCDf7l`5w54Sm?gT|Jm#z$r?i+dPt;T9^j03sOqK3L z6xdiN3jCU=fUL4^yuezc)F)EUUA@Eo;9#w?Ze;)GeD{weYm~iF7L$yP>jlXl@x{Na zR#*ZgkS0&0-jpLM{*gk(O?ducFt}I?!H@Slwrx_TL|8S{8t=7huffyle7Q%tRlh|n$4(FL^`Q1%=~W09iP=%foT(QG*kblS>k8*0@vqTGv6+DUtDNYg zZB%8Y$KM>4L#v=^r#M^;U-^fNQ4PkzROOxkRxS}=<=rS-GjW3FjXL+3xhlB?q@?uL zl0M}6r>;_3l?%jOV>4jY2Z$OA#51z;^#U;oNpwEmQ1QcI0YX~qx9DSzvoq-$o#!mre(8gFW}C7=nC>H!|I_V-rrR-gB~16AnC?nrIueqIp%)bW=(2#KG~K_W%W*R>>FS*5 z@=Q!;9nhui29#;ojjAxU1$92&A)pTJcC~jiE~242QOCPE6T2m-lS^DmkQnV_6aV-| z+bK2^M#JuW)!t?rqmhtA)A6R3s{>lnXun5SIL@x5!{MhZGBH}hp;_8+Em9c{5|SQ% z`pL+EN;KSDoW-FHhwYZPWMa4^D&-QNQ-ik{?~|tj0@>nrj?IGc9v0)xGR7kziO$9w z#{0zVfAL-0f1)cLCy#X1PIP4^#?w`7oqLLbunT%>N2f(%H+4-nr5@7UKD{IWf>yf* z5876#lakTgZq3AM3A*GGl-=AuHTz%u=C(_0HZ1mtSZubj7zt@HdSO`Xv%Ldavbp^c zecW;KNr%DU@k}g+O*%Y8FMir-PV)w%8pnt#T>tT1G!1sGH&5;e#D1Z9nHVgnN-iPU z`-ML58IZ*mwrlJv*y~ZT*Hy+|B(%Ec=f?#IX|MlASK&fk(xH2<%EVr}FT~GL)ZMdH z6W!B4%WOKt)!>aP{QQT2O0?T`-h8J@o%oCHIX@G-B~dAt`0VcaMQuPJjh7dj1LHj= z#+ze|M?xC!T&?LBX8)VtJ^zfZcAV~{!+~b2GcjJgON>Nt$>5`Pir|CH<0qR^6X~E| z_71?H-LCg;^E>Emnb<8sn_R+Df(Rd#^rgvv`5iPrb~S9aTx@o=u^9g1y+)rv%$J=)R##LNv&#ttpmTk#$Z<1}vU6yRgy{u%*1siw3R9UuVW5B%- zYJgDE14$?ekeDQt6bL1dga^bB0)d1Yl0Xu8K&WZ-@-Rv1wW5J4ei`Fm|6@D^nl})Gu%NSLjMwD;i+~#D#xG;M zc32>r%NUeRHzcV8p8@zP#}@+fHJ<;%G6v*g6Nq`4AO7)+_#(kRBMOsi%)}rHf(q~_ zx;uwY9~t{0lNgCLxTHrGVWOTsdVQwT7O7jnsm_~o=wy6DTO>O09R|GK?i!$&THfJ% zq4+xQ#3Ggc(f?^`sjk9#nXF5CCe~MBouEn`%o9j-O2$Pw{8i@7J6*ZcC;JATxF?W+ z5m{ivdhC5Q1_!aC_l=FirTf3}2hZtzUGaWCnL3_fgYLh>bpNghHhB&Wrh7MHkd6B0 zWKTXX*|`y`)#fT)JfplCc`nB92bI;RmMX=pH?Q~9GOMNdas*v|!>rv0o7JpYzcgmO zRp!a(Gs+i_hr#sPb!t3nO`yk%uclx_-0n?8wuUC`T5^qX6PERsYbjkjAO`YQQ=$N> zqB+a(g_e5jS=S`2geNf~VR2u(43G25;(phR!sEPfhZc=&Me%RX@x)`Yr*2IjD^F-0 zqHz2Be;x>;l^VsUw{P{d@)_mJaC*z!wg;n*IG8wajM{FDdZ*G8&u5hPjzj`v7#yZX z8sS~@SeJXR`;78CU-$GkBh7r1y~{p=z4~Af;2C9L*PfFx=bg78o=fK+XO1}>LeZ%2 z!v010(b?>{>W3RYh2J=1?7Pz-g=1_oVvLn4;oW0FgDL?uZ({JOhe!xvXJ(avPfz}* z5??uf>g`?Rzrccmebn;BN51h_d~+@MKrG0MuwV#i;fnW$f(C{0Kop9`g3hXh_YTdV zhPm-o!h5F$)T|QV9k?g5N_g*TP_jzM%WN?|azL2(iKwKm66z_4RRSyQ$fw?nveo#r ze(LRVe219#CI6wln0418^G)H#Y$2QN(<*&JN~fgt>o-op=7v3|7n1=8sDApXx6B<5 z{j#+G-6jWN9xqV2|HFn!Di#{tyuHrD`Z2D-U-UA4D8ruD=xPTnG84ZfE7;-{1AnTm|>; zPA`^MLt|-hF~%6QXxFUGK+}z8VP+MH*#S;_e{KVF$;jFACZ&yTF}7w@;q90i~rq{1n-h zA>Wxoe<;s?NbGS6{bm}kG^nGUh`rC5*z1a5Q(D4DI<933{ozHPA324-krjoGJxrll zQFPhVE_U<~h%<%$Xn+TV#byC+h7R6HzUSaNSuX~IHdKb;k4}YrCiP@yQs)?jQ2Yq% zf59MTjzv7N^^+$2kpvtyNwgnQI1HyE1~G;wXFOwb|5CV zL!CP3Pt8~0n*;GL63JTf`?g{A7_UBG57!I=H8h$hzCDO3cSh)c(*|q)O}Jb@G9v+4R3UJ{E6M~ z^8D_qpi2LUf2lszRY;L*cebBAeZuygoA z*xj*f_vWpe>Nc#a!j!8}{;8iZKYrKoJ3Bfy9N)gPeQOmuO+MfE0X(!|0={U4oPQo@ ztXQEEs;aMSOjWABh-XPBs!tW@leCbmb(%e2jSZZDAvc9lM~yF1I7Eq0uS8MOK}7B{ z;hZMin*;7G!d*vhHt`Tqr@0~pQHbd``RZl}VuMy@HmK=0`3mPB!hZ8PaH>_K*1v{t zVNm9)_aTKrIW(fx8$p|=@)Lsu@=$dWaxBZUyhI`XHYmSPB|>i{Ss+>0nKT%nO{^8P zLu1e;&UVly9)WbVYx31c;+umuagCrI7J>F$&=Rej{~b7tG1`Z(U=MovW>wfN#G8ox z5#1{csp@JrN#bPv-<_I1v3O^XuRe(1gKE##6Srllysf6g|Uve@W47FsZtd> zu?stdovdc}B0u?Zzz&GoIi@LF(Uje4^hfix*M!FY&iS#X1};qLk9JIo$c;-9sgx*^ z2+dn9Z9Xif&GYs+ZQhex80yk5y~LjCw0Yh)rOn4gw7Hp{31HS#VDXxN#BSsvv{l*#9|y z7)Q?`*pe!Zli}15`mPE;i|T?x zT6i9lEZZwMnB=SE<_%R?iptx(bvOTMJ`^??Ule$+C$JpunpvaT^cjeOs>bPMaF>Wf zs-;d!5T_TExKfIk-UBI>swT_nWGCx9d;vs(t$#zST9`8F(b1}JhEcF)%dSl{v|!in zofy)LVI-!!DI)Vq7g8Y&n^h~D7=u);Z_v<8_3QrE`w`uy=a4NhClnkf3%y8lu0mm2K zBcil*TJ?f{_!DiKo!?J6UaGCG!XQ3++s;jO?c3YeAKy`TeEWuN+qTyozj<#*-Hwy$ zHgBujwsU>mvYqQ!b?n@LQC`0RPuy5!$A6=Ut&tuj$Al}@C?`N_wHpb-abwX5z)3eY z#JjOb&VPc2y>j1y+*oAuUjs85=wsbj^oqE#A=8a472w9=V%LpYb>POj3^!J(KB{t8 z$Ii{|Cv85pefQ>VTkAG%+qtD3-4jDk0|Zj&LJ=(c|45aafbk~RZR6D_w|P31o1{M} zQ<|grb{B9F$JT>sj$-bAu{kEjH%AHUeJq#DHIdCxkOtV80j7lU z�c5(Hx0UU_x`0T;^H_QAu@%M0KJJ%~2dCNkm%_nD{|{Yfuxq65orqZr`(ee%NSB zb<^R$o7JZO>)L%n`$-);ckePOucS&%RP|kfyll^I2eUP~7CDwalc*?t)brblu0{oe z7H;n*_kWmR#SMW8bt-P?FwX+?R=B0Zb2gGC8->hPz;aiA5tBr$fG|yvS+Z?6&i@&g zjS+DvWluM|{?(}Ca;-@dRS*(Y_H_HJKx))cxfWD0QCY_IPex^9MpUY)V1j$f~LFYU}jFx{0vX+YiGnRyTg@IRP9RXLP9RH z>)%Zgo~{B|7~I{ie>Z7QSHa-=PUhwL-R1o6z$Wc$N{R7x_fy2zO_{zvPgJx1W%4n@ z4OKVTIrj-DA4|PR%wPS~m6!OxffyArH5PXU{fA5iwnyTP>4!EiZC-#^KL{P}q)eI+y7ib_OofsUFvR>`eL8-koLBP}N5d zC4ni{K;%S(4kbC$sUGinlHTa59nwjAa{rf|bZUI)^gK#bOx0sK`$}g^I{7{2{4->3 z!#KGXfMY@j?*lU3bZTbktO66Fw&(9X>!>=Ic}zv)7dZab-c z*Y0^6PFlTrOGg!!drNR3!K9|fDO0Ag_fou+3*}Qi73{%5Ak;}yfEu4pHAu@)sDwzl zUH{Z!Uv#%CxC?Wia^`=w%=Gw{DSunAPuHFLEV!rBxlhH2fF3PVa2CMEv`huhKQVz$ z&ukgaPuYrU|T!@rot8m!GA5$c7S-{ChI32+rP{#E9A4uGDW^%d^W@gRl@5 zRIu(sqAX)k#CQ>7+uw+J1C17lslX~&Acjh{lgKzwnE&o!-~wT)P-f_$&}8D6K6d?Q zc8p{COmQ5O&_m6=qrXJgI*#dcj5sE(9n+Ia^oOlK8eecspR)v1npYI*)-6bA#V2nOZC8=7oUS~jYpL?+cqzs2HAgO1fnzFpxT zs*0TCJmxDR7r0CT(#v+98BD3odC{=kK$*HHpkCV*k}di^i+hQ3AQ0n46eq%pJ(~XRw)8NyFSTrBxE6RsZ6!_rY65;Nb^D>VLXU zalShD|I<3fgrEeQ)x2wSJN~I_m063nZrI$On?;O~2xIzFnq-je|Hh{)x*CBE5P^;| z$o7BJq6x5{hWi*z1Y;3nz7xV|2?=2`i-5ts-8R*W7_kDvK0sy}?g!iY&ls@Q?E`}{ z`#{Dr0Yz|<4?_mq^=~6)hwF!dnIWGK9-R(#fK!5abFf|iFz%>!rF0VDwniH=49;L` z+7OYNWI<-gd~w(Y&M1m|!rrMp6IH+vwIDNOcR;;OQ^(FHhI|W@99lPICRZ@hLqmQc zqO@&V@if*M>}ad=dwLu52vxI4Af6hc26Z0Uwzs++7jmguTsw5Or=G4I-XHHq;DB7I z`e(%%p^??HpW(5e-BtGG5CP+d+Vvk}{6)g9mjjw~ly%ue(+voZvISn1-TyZ7uVUa} z!C$O%7yL~pu+~WdR>l242F|94z^O7ORDD`Jut^6FkJ{GZ>K*zo44o?cJOE!Xx$4IO z{WgS7J$@TA6|H&-lnk9snV}<#1XceQQN}u|oQfOq&VT~%{I3=Xs*m+lG`sCv z=cR#8H6TF|n6pSweVZrvCRcE0aVITP{SMgKGMgh>M!NK{{-R>DZkgq9&yoR&D&UCf z(!=cj*JgH);^m-D%IGmK2COaATeinyT?v* zmSRGdV2BS@1AP#LLI)gC4|X=WM{+*r`%Qhz@-Wv8~6#nIpyw%y^z8HJTW2w zW!^u+uK%7GRPn{Sik}o#@$S6;9?ttw-)GMIzjl7synh7apGKS*V?+(+zd_Be-lh@9 z-FDeR--wzSV#JmRBRGn~T{Uf>Veo8G9Vq*rgSS(5PBaK}A|2HfA5Psdr8g5IijaAK z4bT6CV%~7YRA53Bh@n#LBxl}VW5<7kf%Ej@c69m|CoV2aXxu zkL+`jOJBq{r*&&ri(|HF$1r!957WflBlMWHO>kCG?S#0%lBb zH1iqh1#$&+sltu$)m4^_ERqODa{rrcxR>!IRBhv!ZGe8EpeLJy zew01`5~81*BF!)il@9uz2Sh(=XBPUiu!+Iv0WoIu$w1G*bkL8o<9~?$?DQgeunp)J z33{@*=wAwSTHntrAkDD+audM&plW{xhuz-xoKd6-V&0BcdP6`;9nE@sl50 zGB7quKtYMpqNC6Eq;fnDq*kbs&?h*oZ*u{la-**WRrySUmhuk zkoje*O+Vk!PnEgo^>|2aWnd+{KpoTIiQxDVh^SCy8BodWKpnHyV_pvCDv;<0>_F*4 z9rF#3lu#GowPOfarTXzrW1*wGzIv*`xRB4q9~2hK3K$A#^0`(dljfx~qUloVLae*~#7zz-1=(<-8V9w6wDV>K;! z5J#u*F$&M#gnTiFb?8@`MSbg_(@f569c}Pwmjceeg*& z%xk@cr}7@5TSg2WrQI7Xs<>uvpO#5aSqrc_?qJ64s3ojuj* zs9OqYlqepq_u9YsCRb{ODh+$PwwTOLZ*Q39p|1yX1xTWXvIgw`grTh`ZhU4?rz)=( zrcgETBe})tg$>twBJPH^K!n#jyn z6-ZlA%V?{-LZpQLw5?YIDF~>E?*+_u)SW*@Uba`F#>l8kCNjvVJ84|Ngwacny8ZG< zGbq0_7m8P|Kt0Ls|C#ES3G1xRYWEq!7Ph|~ zbrZX6#BJ@cF=!SAiccEP zb%p9tHxTMYjOP=Hn8v$ZWs(!W)Lj_U%#(lTk``A(K`>avr*(12 zl%5{**`5VdgLB zbcpQ+X_P4LoNCvt-gl6`4?VvsPq4y)_GJAfiH5WI!de$&MUkqb;3wQWclVrPW?AE-6+XF6M8ngQJ~c1GC#z^&8MBE7&g zTrV&MLo8OCvW8gpS&C^DQC|)G)*%zyc{Y^CxS?Fy*$_*!C{TPd<7HQ<9%2KbUT}Lp zk>H*=!e!P&>~P4BD#~Uqb4m3Odk}iQ=rMDM&C($`tOizeO^*zgv%c*z>ml~sARIBoE;4DKrjf&P z{odm$hsw^lq?u=zx?DPjkwYV!dG=(NR1dLTNbBN|*}FXEvw<#}S)4B+UAn;E5cjIv--|{UMgZDVRehgwHASgfrhAVpCQzw)%7~CJ*VwYjc`Co}Sp3`87o7K;~PI?RQ-0LLuWt7T*qNqh0Hq0X_hl>Y84CB?^T@gOR(vP>nvG*+{> z{;RPDGQ0|0Hx`-6C_&MDcox>-Ac+c!=CfVW)MIdKm0jA1uU4&|yFBLoVCEtO0@Zv- z#%c_)-TbOcTDZg2^S@t0X_@0n)lVyQ9evB`E^{H1XcqdZ27Sx5E@=U~{dBu=G1%R> zNU8AHJ6p93{FEWB+VYw!N1y8td>SwCdb&{*N9|a27xe0*M!aO$P7WBV0Wi z?+m+pfg6OwA}__OIyem=SoTW_5U_K-`d5Pugm~qbmoFqV=04>iAlSi>9~FZ0@?FwG zhTu$qm6a4{pQMN&S*{RZ4ZaXp^TPaxyrh|D;o)yxffUGN1$z1Z=5BC#{Sx67;xz7de`1-j^&pLR)g!FLdP?D;pdAfpflS$bsj zneWg4nZ5!&-gaP7Sn$y$&ge7$?=DomrV@xapi35va+wR(MnM&qELi1{u5h8o>ygrJ z!G$if9_B9yhItfxhnOOSra1-Q0)PLb94b5Gl4f4;p{uJP4X#6Dnt9;W?w#Yc;(TpT)6aRtR2VOf2{%thS)#hxi~@Ixw#>AwZj_cAe6 zf(V!nQKIhZcy))^`QO!nx@A~Dj!<{VQc-txoVsnI?wU^2eGN3s$kznwmLsu5sk>9u zT@$B{>tA%{b@A#J+w=d|1?pBHlSimq%<-QMd|iyX#a#cRx@)yM7KP?RXQQR)@Wneo zL+h^P!6%$xO{mHOJ@Vtfd1h=QsU_c+MczA=C<8%3qAY^^ku35)`FdTgAkobsmxx^a zu@jM%??9^Bcl0!VBY{9zGCWIW-#lFg6eLO=nU;KI7J0vL`BHl_`hJO%=`!4KlwV97fI@dvsMxCB<)Ezw8o?BnPZiw?E% zACiwOP0&LHiQW<9Hvfm@qe>DCfr3P31o=5xU`7`wdTH`r5#%>zk&h`#aE-4ZQ4vA@ zY!><0!UUaJkSNb4U)m#!d|ZK^8sWbYHL$EWnHuK;(jX0`QFJO2wkQU?2q}`qwton$+Vu63RB;B2xP3YEAKm_=7`HFK z8w7Ow4H0gyHf~@3qCmZYZZFhzW|8iZ_-_D|#KN$mPZmQ>q3)AKiC&Qm3$hq$(H|iZ zeKV55o_`G?jZEo&S(GS`WU%W$A;YMA-7||4JtG2HfCx+ z`cSWYtrh=Gq(DSMOU(L*qislBnWu9CIz%S%s%}}ty^}g4}M*zG@>R)9@Q(3}1Bt2x$1t5r#{`TJ=MLce9RSPC>-*$WmaHZU2y5O!rFxk-oY( zOS%{zIm)lL^WRXqnD3AB;(*ng!~r+QIAHZ>#Q{HwaDep6)jtx5KcNHk`pXbA^u=_t z)lX+J3^jvaOeZ_6dlo~L8UA8A*S2Edq3cG&hT4a3Y37}LoPv-v;NFx(7+ znVn3$blBrr8b+95uqa_4YBq^Z7!J?NVyH0#VNt@az8D5+%)=YOz-azSOk*B?7zh~6 zw?#Cj#M|Mgh}PS5H19(+*ObH;RfpU8e@I?i9A8u&{>LooBayn17zz>*Mb!}nS>&S% zb^4Ahs*Y&NA|H)%N=uI@s*bSp-w^wlRD4l&gq{C{f2;$pSMjFTSWc;*Biw z@yYn2YE4xZd0ir+sFKiFvkc@6joV^EV-4e)|SFLY#Qp>dQQ|3dQ4 zLgOet|HQF1B=0OVjyeJ442_?~gvL={00Bee_V~~^>c^t8`VV__i127L9+hEko5dx=(s3_=ive~=On|i&f`9?`i--Ulg^3t`3#v~qyDw1~|Oa#3yL&z#?^=p6Yu`>ho#8C|IoRYQm zJ+bwkOk26E2Pp0RoR!hU6ZoZipz#&1$&i2slW3MW&u;nSDz=qpROK{VZv?r_00e534e%a_$JUIIMyE} znEw#mEtX1P?MY?aEtc=yEk=(}fWTesSbH@B@K4sbUoL<2=}Q7 zxaWZua5vcb|5J=peYaRQtitO!JwX+B3mZ5u{4o*!sH)uFSzXOm$+}YP;uZM)fvyxI zAZhmv-x41hP}~J92-;m0J4l=#6IwNQ5GX|gaKkH}ByIu*l9*aNLx7y@WM#v{I~}(E zc`CS!piI-W<`>6ZDRwdDPUzdbd+_2lxKgax&c5lG;9A0qS7co&*1`D?WAv$*M(@D> z-y#6hSBe$4;U~55kyrSxdZ*Dl{v?h5bVQ?%1})9{SiJJIPEPybmA+W;(~aI;bx(v| z=?exji90vi^N)t{F)&s~ze=XBZVj#?k~NJl=-s&2)r(G%P~!ky>#b3}Q;y#1Ds1RS zz5#)cb>_Dr*F-t;8b^VOWeJ%p@@A23ROC?f7q;wasl z8M!y1VFOs>)fm_OPUUL6lqvDH`E=K<`ixE>w-a-5t(*?}&Hv@H7o=hCr=?Y}{4=1~ z^Hvdf_(A*rr#H`?s5bwf*C=vD{zUdxJO}?D-jP2FebGr?C%o-BZiU;Po%kJ&(|$j3 zlFP1-=zNy%Bh+Dv>*W?ZajLtMZg(TFTHLu{vlC(Qj04^AFJw06sz-$EhnArvZEYMDm;X0 z>y93!VHCrc8MnOiHwIRB)y;nan*82|J=}=0!`ya?pgPbs?_2tKpc>$r&W5+lYIh6Q zzk_sJ_!!vd;Jr2a>1+RmGp2cAulA;#ecEEzKTW=2)kE!a_Ng!OjcbcF(S#aBQH(Us zmjCvUywO!dRzd^~e?koQ{cYK{1 zY};{=!ovH7h#G7$XyM~+=ZFVh2ujQDy0n}WmX^*vR~K)^*;4++Y_Nv+TVK2ezkx`K zwMKCLmxbgDaYlUX8u2zb)q)RZT#Rpy5!-qGheo^y^+71q zgH{ZG(x?84p^748^qf7x-27=~*g1}kLY6B6{Cb`lw7`oH}*9`mDM?g9C_s05WJ z6US_S7o>E|OEHew(bsW|zLrWQUijUIkm5LI2iJe;m{+u8{NrAuL5tSiu~tC6!bKxJ zt?FrxJoHRCjmX9X1DOQkj&BIWS27{W;=_*HMf)ca&C(>=pAqe!1ar`Ws5xlLn}afU zQh*@Y$@ZtSKFLHvU%-T^0Iec@RwR5w9J1FXX;TM9If~5Wf zjh%z?CL`T~dsK-t2h9)jLL&STj^7_>I!bJuNCMxk z@y4BkkBPacL@xRnid!{8BtAZ_cH3X=!ynWM zKUw&y5JAe z*n{C**nZ}%pu<*`ys#B z{Sdx6y?Zb7KOIx59TS-f_ns)AN`q9`+pJ6X#JY6Qh}o%d@Am{^X(mL8t-Zez?R_Jf zr4dNcclW+3+WYF*+7FjRzRa~F>SeB*jf|6R|L?2w({{*6a`7U8KQC2hS*nU1+(^B-#C?j? z|Bk<*INXQ`8`>xaC>QrBQvW;tMuaQ=>nF2)KkmO_{ZyF0jxa*O-)8$89t4aMeC$V%Mk7~(dH z?XvS`0K*t?`aeB(;x6f-l)i{-skawboLk&yZ1fZpHx!#2z$$UCV2Il&1}N9xXI$;E z6L%?wSbgi#Qq|M(cS&w>pE<-+Ox#dxu6wG)U0xw>qnP8qlTXe()?+8`Zm1jeRj*4` zxr4hj$GC0%6ONZqZ0?w?#9b~SZljpRtb-dja=G1uxXUoJ?`Gn=!ol4wx41to^IvD& zfnswvOeHRM32__6TyAi2Ka_>LdoQKR4DMbI?y}tCK5M?Gm~jHdZCY_9ZZ-&U8^r+S z;y&vcV z#l#K8Q7?8kiUG>Sea`hBJ8@U^#Z=VzyRU=0Jh!+%Q|l=vZYZ`3x_upg8^r+S;{HsB z$4=b6`r)=~gS(%DyCTQ9pYaqEHx##N#r~-cqZpuE+~*GV*onIm?>5r+nJrcQ9o)Tg zi~HP5JjKKf#rDl3?t4{?Vu12+|K4LK?%voLt2F)|;NY&zE$;J{dWwk~irci}N;SZV z6QkH}U`l;*-j_Uf;_idRO?{WbQZ>-Q-8;9q&oA&46E_r_OB^cIKnJ%`%oSHBpPb+7 zu@iUSL0J1R{vPDu?vrEOcX^758;aYs;z~8h!EF=+l!yBxkDa*t4aOd+!9Cc)t@j9W zH!r?mucw%}q1asdU8x2;xQ$|fa&cd9tH(~<{qfqCeg^jt2Y0{R`unqWo?_yL;x?_g zQVnr%8^r+S;{NOwkDa&&;BgRrCEQXq)WO|9$GBhe6caZTo4e>L)ldhwQOsvlocR7+ zjmJ*h1FP_yz43RIgL^=3aewYxo?_yL;x?_gQdK#)jbecEa6j#_6ZfEMto|9?)ede* z&E>rK^M`wii5rUBwBkxt?cg?w0m{Yw`HMYv;vPH<2VM;BVGiy=xyAj3?w(@ehT=A@ zxKa&sa2v${<>LOrp&mPN4;hXFe+Kt(2lwC{Qq^Qyx2U z4;`VDey#9QHNwF?B)7P~c&4YAxS`m*&)9znu~7_A9_~9mcH*w8!Mz2>-!%?yeMmHS z`TI+gJ;lTg#cf(~C2lkc>u;kNpj_Nv+T*bkcXh2&RR(vhgS#rX{{GUxJjKKf#cf)# ze=xx)1}GQzm&beT#64`JQq>0cNC$UyZgGG48c#8CLvfo{?4QgtiUG>Q{i4TC+`~uV z3Os{*l!JR%ZgGF*I8QNgL$Nu&=byVXiUG>S{go>{cH$l}8aD$N+@l@b!*h%K!akm2 z;)dd=SHK#@0OjJo@Gy^^xNFAX)b8y$@7WY?Y zdy0u0irche|7CJUF+jPvzj}tpPTZr$<7PI4d%S~tWR7va>nSE~C~nh=E7f=>e;dUB z<>LNYlgCcnqwDazkHKB%;2xD*++VxFQ%u}Y+@=**syYX^Q4CNX?tgmh#61RO?r4L1 zf`fZ>ZgGEoqoyT4eRz_zZ&e^T^FiLF?jbQ1alwfXvP36F>;X7t8s_ zV6!i&5(W5E5x$n?7bwT8V{+hCYNcza`3^B6T;(2HuiD12IV)AY<5jIVNmOLrEbJe6 z?DSa7ZuZz5;HFZ+eTkfZbym5@HONi2m8!wPtrZiLi~EvWvT(=jW{=A)?o0Xno4;!| zexkfNuTo8PaBIa$Mu6k*OYQjwy9=L;+sz)IW85!f;jWt`@64-IlN{VyakALQeHqXH zgl02NVs^9Za*O-2D?G)_8WS2-Tb+sTMhCZ6tY3iU`1`Uyd+fv=vzt93x46Ht+EYy2 z^-Zd6g2CP7;MR)u#itJLZ^-=*PC|*<&92Wa?r+NTFHXN~n5^3B4erSfZml@k&5x6B z%JVM{?wH-|h8*L*H_P7>r>M3DgL{gDTPseM`M597^Vk_DF}v9lbBp`((>=wElSxxm z+eCwVs)JiAPIk}0Ezf^A{*Kwro|IeMS2SkfZk(ptCK=q*9Nb!Qa5`Sbohx>E?DTic zZgyjiasR_pOn)~`S8a_3_jCuhR-EkV`}@kV9y@Wz>}EIR7Wb7r{}v`-;+{N1wKW;s zGaTGnak4xE_g_7B;*QzPo}63U-#W%qOx#muspC1On*2JT;a?DTicZuayX;cfr` z)ZbUl^b`~KjC5MHO($?X2$pt`Yt_j<_yd_G6JC`$|4qj2XV1tn^6dHV%yhGAn<2=X z9pqYdvagT)JM#RmGxo&nXwS?o^6$v~A5M(VO1G%CnS#8!12F?-sx za*O=Cy#L70;k06=cJ`Z7;3kvFI3tG0AnpwD-pYu(8~;3T&={c1k{AI2M@$82sl=N9@k{`g1e zE$Ib-jwKwO7Z*6tweI9#{K0{KP3HbzGH!pnCAZM8&EEf;n_dX$EdqU^16}J*a{0-D zey!YpZosHV%o|jwb*JY3Y zt$gP}dY(W(#DT7LUX0LmGj-mfM3;Kfe5mPbOX1jYqZlV9+)-31? z(}x230)c+016}J*4$pvY*MGycxR@R9g}H_P!|hqn7p0d1`a*%e)Pb&bCr9|ue<=4K zI{_23>Afho(0}C5e;MnCq?ZBuB7wflfv$BYYkcTG+Tt0{fQi}nJ|xG`U-A?a`r`C* zKtDvFFL$78-N{-X`t@@EmE-l8t?$LTg?{~2Sz6 zYWI6dZlT|hJ^w#6y%Nxu2=tW>bgerHT<&Ok!?!%+889&$;D_cG`j7j1iWx9V)2jge zP=UV6fv$BYNBhu!e55CV&|~(%m*yC{-2W_*O6beds{wtfKws@Z*SeEqeCRj&>mP(3 zvkks1x6p6=lIJr*U!Fb;(3c7H!yM>ZcXDh7bb0@Y(@kS`!k6b3`b`V7psz?D4(Q7T z`r!_Atvfl+hkn!NJqh%B%x3tC+(N(EU;n1pSEi2u^c4dA2nV{>ogD8&zqvIFddz!G^?26yQ^?&Y)4~*XxzeVnU0nHnFpF5Ac;`I;`wJZJ;d;Vo$uq&R2$!L<~ z_zb{TIleGG{R!89xGO#|W>@?tYdrI|gDGxTe02^Sdjeq5HH4O({|-wZ1;?%y#~$T4 zR_o5=#yGiU=6q{r{(}p!2c00l|Csk-Ie<>30{zzP{C9ZzXh1(qpdal(*Shn7%Y&Yo z|ImI#kxzObo?Ga*W#_*m(#HV$;R5{_2fEfxTrTw6GV>qWvKTFDyzV~ieMD}d|1>-Q ztw|pX=tl_jV;$&PcOG!L(0`hl|MKE?$k*f;dUpOhGJPDNuMz0SIncH4yb>S!&oc90 zUfd@6k-3Hbv+VqLRJsk&j}+)_4s@+MuQUUC=Kg11+*992Zl>%`No*^4Gr!{kZgcKtEQX zuXmtp-FZEHum9Jzp7DepvuA!>ZlT{%Qf!^Uj*Shmq-8j(i9PCLT^q8IVwYi0U=OvzELT^uR1oX86 zeWL?i>&~mlfc~T>fzV?%&)ahg{jL?BVnSb+-UR6F0)3MMUF*){{<-7zyE4x|<;Crv zugfj;Uu2(uTAw~1(ANp{;~nT)cOFY|2l_9TddAc1FCa}51{Pcgl|A-x&U*9-K` z4s@+MkGtp&^!>R1%G)l=i`zxtkXz{c&+!x!dPn*MK;IzHPjH}X-Fe(bCv=&N?7tTh zI2q}iIT@jiC?og3=v-l?K966jEpIM-2j55_)2X{lAcfPZAraH5CeT69-F1R>2rrSs zo;dD%iMmg(z`IW`1!f8mxOemX6X6btfqVDYUASKZr~2k;U%Lt4T)00MxYZGGXM>h- zpBA{)`pzq!_a3X^)&gGvE>F|XDe)$}OHRa%aD{wnBhM}C->lX*5>k@jzK83-#9bYO z`<^8Z?&MBzs%y62|2=$jaNn~-aF2+XtA9oGu{k`5$BBa{yUlZ&4bGcV>rhe2|!$v$UoZ@z%Y4Jac*ZTp-E6 zk10_F?`n3k9*ZwH``*I>c61YcJz-x>)0$rzcRk@Hr)zjR^iD@GmPTGrc*)6G*Aw1* z50o)1N5q8Xy;%QV1YqW-*019y^~r{fSF|@Wp*o1CbqQV)myzdm~+iDhzLG)pr|mm4b{sT|wqO z9y9wlm<=9KXtF6ny3ZqJOqbf29>n*+a}|Lfow)Q*S|bCRA}rjOJuwXEKuiTL7z@M% z%nsc9?fI9ning-z_p_a!8flg2xqq3b=WLK}?6O<#|Dwmd9n9l1-6FAb|4%{6pdT9( zJNKjh3xYmz24TLqa*auE^ zT1e?pzVJD|_l#haPqFx5**8iR<%h#n zva6%~gAaQmnU&y!1K&4!v1yXu*!6FNh+#JXD>oX^sj6GI5K%$+Ve zZb$inT19Jqx7bnsTJ0@8%3q6M>=r%BeEB{5hj)eu(Qo1l|$>d@hVcLV4&n0=y*{<%6~VgY8c;oIN}w5ZNK4Ji4gq zPZg>t95$1T4-Uo7u=()O9y3Rd0^7eq>5lRb+vlHK^a#tCF0(N`2r(TD&jrCxdZ1=N zJEQ!=4|y6H(1955<#QS?nH{)~aQ?@1WH7Mv4+4qq{1&db<&h?jl%xEnF1zIsj(?$^ z?O?8lTarwBR*#G!k6Z~-27OCR>^y??{~&hEC_nd>8fG&m=pRA-t7B)rj-AL+{?S4K zH9r{T3lq9Q3ll{Vqx_?G|8IUKG8yF`<^CT7c43548RZ{6U6d{iMtQubkj>H)FGlQS zlz;SAFw!pzz z(Gw$A+#DHdDNbgmkN?CI$x%M$U2YJGyhJaN^EQ=oHAI=)(pZGGcvOBGe3Ckyb z?1ZIR$-Cg`j(-EP)197p0#cZJj*1A&CeR`Xo_Jrpa#Ya!OQL#zvDf=K!clR!ZFsKVo*deGRT zx;rsj6b5P{{1LOcxV{w~v#(bl)SEH>u)xE|aP46)W*(}#JK^f%L%6~WH(dX4ktd1a zN=f<-a{r2F*d**uxc=eMfc+$yt5@p>XIt}e={_bydOifc_L?w8-*bLMxOR^Y*Vn^v z-DFzjk8FR_DpuJrt!!B*%038{eSrEAs5ps6*+@{~^6Pu7baeW7PnVQbDPi|V=0Aqr zu`yxy$KN|)r}lwUoipV}_u`uqc7Npf$6m57BJ37}7Gd|Kt^d{qoqxDCa8hWXyNdiV zyc=2s+}4HgxCy9%A(*WU0gCSo#OJTF`gI(CdKH$tiavl0W&?Pqk239$coof zn>=Rr-x0R|f=ZJ~>pt~WkkT<5VjT0-Ev{qqZblC^>KI6I9P`wZkiynIA;K|DphfFG zW#@k<1UaxYDhHN$Igq0%1qj?fH4EGmV&ML1odeh8z;Bm6kN)Dq{nG^kcS{6Z$$@|R zA!yi+TY?-|(*uUEaNeVOI61I5Oniy($Ais*mrklas2up`#U4J!S-HLBeW>@cG#?+L z1!lN8@Xr@}k{D`~R6tAp>*7rgB)gLX|IG2vILfA_T796PHNQt(4!m@Fkd`i;jvUy- z$x7kNYA>CXl>`66_V<;I)XJ81qHI>EY!)&GQ0WwykpnNCZu7^b(=&76Uw#I)>?K=b z!tO6mIbo;v!9%Ls{E64&o6}4F@-d{am+XuPJIR4h4-*4-203t)HgHmCppyeHZDtM( zEDCerrL(ef;L~S_5j$gyc=}te5jqFnHsF$8z7bDz{f|cMjxa)U;L|UQ5xavNSYcyx z5Mnb(Ud6#rItMbcoE-R!-TyRV7?>2J1G56}F$kD-4kWXa1D`q3!@wLk8YH^^g3`@_ z&v5;ht-CwMG0$NAdl7i_o=(wk$$s6A;T8DiwC*#15yzYs;TZN8KVD|+=rOM{29ti1G5Ial8(kS1YapXSjmp53 z-hK%$=_^(!(W2-2hY3prMbX`^)GYLx*+Kt&FPFJ+En;&l`l*Lr^gKV?Bc+$p_C==^ zsy6h}C<~H|HOBPk?fSp&JNhlD%{FpN5O&qN<3O`!kY$kP!}e##sRFo3dibnG&Mh^) zM)w}oqg2L}B1qt86#tFylq9|XdFKDnPvyWL*ZZ{v1B6w2|6gtWZ+bu3JMaB7z>kjb z*FAw++xySfy??mw{liM-nO-FMXcq-av%b*V6Xnnub;9#5D9v_Wnmh+5ipl*)+&UBaBrqeb;q~&J)4#RE#{K zO=6lL{Z!%kqsPMtrb7LOlfg`@;j2M|VAkV1w&ew1Yyi{u3CQxDmfpPd)hB;X+`g-j*e`A)(8fqC{^`s?^V?Q z+Iv^34rpXH7;fL+Qla{Vi8zQ|GRolg>YqL4v%x&keTkvFA_&jxZbw#$-oFj-*x9|a z$^fzTZ>=DucdyK}RT}kg=>I{ZRvZZ>>gXTJ6Ys$R%)fmBQW(A0MmR)l`Wx1NwL`9@ zLv+3UMP7?CoaH9U_4%Po`p_m`p+i;~u>Z#WUk*6es!hWFC4F&#zViU(`k_*FkdHjm z@r9uLdu_mH2kdhbq#2gq-!0esJLS4WcS@v4z4!O+fg~7E+>AyUY2R!!R`n0Z>pnPM z+I#oT_N}`%cWm9=c2fJU-Sak_w0iTF4%)@O=G1$Czt59;lPk4C_0Mn$nHeY2(ErHu znCsEd^(3K7`5~DMr2nw&V_o!d1L;3@c-roEMO3J<8Bm#yT&OpA z%*z3F8A+m`V*W2pO-B>zq4sL(erjAMR6U4TOQrYybC4&3Py-PaYFq|XGCNTJ$^LIb zjrs|7IY_+ZZ}pt(kzNHoi!m?iMXS6y%}CqyWURp#Li3+?{-;B_n9NJ8oh1NSU1gcb z&?HRn1{x91#XSLTx?-CT&hM!=6EOqjPfVr3_&72p0;Dhg%Z~r1FP1}dTwm0{Gy3Ac znE%4QNcPVA;tYfRUpEJQwlAKi`{Hoj7lYy%9n*{it_cu9F|8u%qd-)L6weFoHTPrG z=rBe`c2+#oE((;^e9hLs#x2jmu0Uvo8XZPW;1nlnUi*rNXdfD8G2}-@&1<&))p~UC zd=PrL{~JVSfr`>2eg1X3{$=#+aP8^mJ1OY%bcs_uzi#VatDC^uYtOvYkEeSV0gox|1cmEVE%GAHlTPu)Kr?ZjVzYUu74OdhHYS8*^ zTX%2XiXm!N^RCV96V`9rvvv2*Q)VsNx?yvBkUI2O%=|*ZEYsnEH^csq&ZiSskX?(B zxO7G)59!5gZ~FP4d22@IP$k*vO)vlJO5rPMT>UnT;87{Gh8xW-EmBL$g{U@vx zE=(^~ayt?#g{6{N$!F)}upQs6(r%<7a1%udqr-nu*x#q7k$ znrB}!(@)ie*{P4RD|`BaNvGqz^^7Y*ua*TOD)62fzkxDV%id=FALNKa=7_l@aR*yj z*?W7wM@p!zSO=;iX@6DgLJd~-^ielwW$*3JyD~J^K!&%Lq?@ZRBs0V%Fy6l3W!6~d zfg~z#zWt_4s#o@g+jme_s6H7!$ynKYXM)GPA3hmF64xhE>AiE5OR87)0zG3@AJ-?H zReF@}RC@2&^)Hid?uO<->VeVQ^-s-Q$W%Hm6TMWt?}qV@;45I)pK0GHHB=3E(U+>? zAPKP6V2n_TWT1Dk{}+Z)QG+X~LJiOGcSa1p`)!xG;H;<^)FUE(MJ!$*}Kd*MD&tR-3>q=w#UUL;GPuQ8`TLQigp$yZ-yElVOAU zj~P}zL7J-t1ZLP)d?94s&#wO#*$s&??uJBdXZ0WLqCknC|H-cZ0-;_tA2`LyWB-#~ z|1F_>H@{dtw+F4+f!3t|4nj|M{g zUj0|lfQIfu%9kqZ1F!zmiK5jeiee-RU1c!`kbVlj;E4|_-N-6@P8900Z;%Drjprw?Cn zR-Y2zg%tJ3n74c2o3nQM5ubnN>eC$&t4~~GfMY%yFQD!S)=ocq385V8R!WJ-Ktvk7L5e13*c)-j!*Bi2B&(|2x$-t(tW;0!lul;%E+#AKU(ar`jPP zCkhoQ#AhFK{f`Ro%v30=h#&vRX~)vHC}HkBZC}GTryW0j3{vRW{Sl7Einqq|jyPez zb{f}aRwNRPyn@6)@fk_04EZM`vdCAe1nnzK4T$jZC#wT`I%YWtV;u9znE@poLm;a_ z&5z@lPre5#I%a>SVM`4tFg^F0wZ4~HtFZi24+x2+mS+rYRZv&bOTfPEKu_k+L6Nf8Dqyi(RQTNm&A^HBkZV#FoL>C zRNYHE3KdkfCK4>eiV|`{0zZ>@m@qt_#W3F-D=1F&QKmu=YOy5pqur_fuzRdIW|&HKS9 zl-7hl3b6ZV0;pq6K!-I=cORQh%N+-5ejF2$BgKUKVoXRb6cc_OVS?#)sU+K<6Utxf zrdtaW#wQXzps66yBdgn`l9y+Z*BOfoVR2S!OC^63(6iarLT*gBCT;)!HEjlEQWvIR zaBa>T2zhL_U&q*-XZODzjIde4HE*qedXP5j2|?%Knuq-_?WB<50O6X)@sDnOkZlbW zv3BR#`QL-ITZ4%R*L_y4n$WAd1&RI+c@+epGvwR-zmR;HO0a$^O!bQhjr^Aa@eGZ9AdCqOq5@n0 zhvc2PsNh2}BqZ2GBzXX7M;tH{>BA=j5pw<|-p z%hDxdQPFyF#^XA&(m)XtSw$TGnFSu#`H8lC1=RdFov#Sz-?iHwk8xYk6XLcfBHU(n zf>TBBi>fEI+p3^yiW!S~q;QbTt{j;9nLNUW3fn*o`KFpdsRst72tIK_@$ms4W2Xud zW1LX@)qs+*LuAuH&5z@R;-7(vvGYWX6N=gY89RT}PA~%aU)GFfhL$R}>mPqq9df*2 zMC2jIV0uT!I(#AUN~Q}=!_1w9Nk#H(a4$X^j8(SX9lNkOunFXP0gIewewQlY{D)0# zxY^WMOOfTGi1S~X`o~OD&AcF0a<4NlNZt-EwY}n&NAWG37oqNGQD`phP5S zI1@n4j}r-{_kfD-c{0;IW_FM&{VND)`O^`RATI1y2pXkn-h}iw6ihcC5-i0RjO}Lg z-_r~i%rldTbJD+7Z37bc*F=^6HSP&lBN`@lY^-lwKXrV4L*u&fQzq7L96xD8$CUB) z{A^f1ZNsDuQ?X&uEB$?ue4I$P>;Fc2vno_;@qCTm1)s~uba4C8!X<61*DPH$uWixN zd97=xZK@VqUk~vcCXQb>Y3j!Djq5kGQ(Jp|dt<}I_4W0EHfplA*#2*{tpLq6Y22_k zN^IGvZJE-zaq9ZU^^>}6%OGP*na%%3dX?&xc#rmC8_Il0C*fUoj-OdrjZl*6owB4iMH#~`xNoh0 z`YZBM(q@(ZN*<$N^2GX%^^NN~Z>KUGv(RTd8XK6lP17k37=cJz71G7vgc21Z5Yx%`IWxV~0meg9fB8lDCz% zsPsi$Hwcdo!l32ND_1XCy=d7|jqGZmq9$w6e|y?!QBBLKuxOuHbgEcHKF6Y-eLX%} zlxzk~v1`#$;6W4hoa@ojqJilP84orzbu>1% zZ|tgYDG(~Y;R#=kiAD|e+W;gP>mnFgf$?7)f_*LZIu?m8QDg8^b^aJtXf}kZ&;t1I z7qaBTe^VHRm`A-*y}bUPAQz1){3_EpKk1AwTRXk!isL_nx_KM z;;CN049xG)<91&5p0u=BFMIw`qe$RdMRj9(yA*9`vF!(AVrneRqu}-ULNin@c5x)i z;qlvedwngvyg;Ro&O_~4v1-MVhNk+7_0uLcG)FQNoj=6l5hwHA5Cb7G( zUH>q?#>6LaTKWdn)?e(t;b81G(jB||+VwvpeJu>0N(4dJ(Qb?6PS0)@6G_-}^vlheOYZG$NND3$(2*GIyGDb$IA2*k#I zS9vxv%C6(J@1 z5OFs4Y)oa`AN5~Q87KZgEiG^UWTvQ*%DDd}kiyFNxroZx!r}fu;XEff$E)V8AddeO z^t=|-lj`(`f3LI)E-z zUH}Q^K~=6c27x+1byXFoUf=L-;+JI&3U4PFb0)FZQv!8ZRH zU7T%uB^Z(F!G-!0onK0;kV$hjl5cwQ~hK%s|Xiz~jXv)aO z50m7P_(B5>Vg3ts(4aaz)Z3e+v-voBR}I3cLK+m#1+Nn&(mF$a?kS=%V_MFDG2azq z&JbhB=Nj{d$46rdn?a+i7LdBe3?1yz(wGT2f!oIzgA=$1YfLZ|yv8?X==q)^8Z)-# zG#GQ07;~B!Lq5ltp|<`tAxdM4nn6=SHh$PL$KwkvGxTqs4$RgPi|kFxhw>)n0jeJh zq$RDZTjz0^?SES?v13xF4B%DkJiRQJPEY?>4q~HB{IMRuyVPk}j8Qrz(=u39WzWAD z53EH3pGF$^%X~;@k@)f7fP(&R&VF=}3?yduvwm6w)-TshJ7BqVwMb^M0wmRkcy_W} z`c!(q6k{lt_IEv85LR7Y*CYxekV!`MqaKPCSVbL!$ANS}H1L_Fu7)g2a>}M$@%6BA zo@^FN|0a!|ZGlYF$+$6W4QM&IyrKt}8)bk6kuz=#I|D=<>t2Z&H-=&VZxMLR)$ROA z-JgEs6Okfg-7wC7Io7=zF>Y8mFAJPk1y0mhH@usxt(w{@&K~!OXp`}Ic!OYmH3su= zoc|0kmvVGh_wBr*QKSgw;U^2`zeiw}m8jvDgNCW}@2UgbS7A=~r}P%pHin^%m8dNy zpw`t-ncCRCq0?~Xx*9t4!ey&gqkitzd=(@(>uMuXo+hTBaV>k1ey*S@jC6Y?{gBT| zKO<&&d`v&Zl$|0QKMb5T_(Jy>vDu@gHGk&IRmPHbrb-{*b!%|tF0GMOwJSsiaZ%4> zo*vpVzGVk&xs-}v%MP)He2y(OcK_RWo3@lRlg71$GePL6Y4UW?mcOJ=RBhu(i`n*x zUAF}*BD7^m^QzTt3tO?2+tS)>+UVPmMSWsd&G$W>w5zUVH|+YR*tJ{iBA;v5a~>b< zDs2W$Q0TIaq#CKU>tA8SJ)Op@SjJNY7TZtgx?Nmshh3{y!=zTT{P`_WBnH)T|33_G z8Z@D07YzD_7_>_aBA;VWE$Ux~j|O#X1`VjqhXx&uFEmu`GafA)>X|fNi&ID1IV!!m z>jt475rbBnwdpHGjhHiXk*A90)VJ(}IhTn!JH;IGIp&N!+vB4-WzC>TxaLSk7-`qP z!}j^_^hv610%?&EPU^Zjk`WGD)w)t=gnf|UWQ0+6{oiP!H4QCTH%?zJ*6a~$$mduy z%0B-P_M-01pz($XS-cx{nrnHY89nWCz8=~xAFnj47hPKvP1XR2vv@b^Nl$n^W^E^v z#CnyBcY*`Yc{%|FzbCPFwoSF^^UhdA+;%WHQe1bY;Pv=I1dd+o3Fo5dPt$lCsa~SN zqUceP3v*KrYEhIbsm|K-J5MDSMZd`xK3l?j(DRmE>uLn8QWXAa$rmkyP-`Kvc(?O0E z%Xa{Uk|j}(@%w)>`v1*oy!X8?;UZBl|4+SdbyA;LG#2B(lb^U?_5Cz|)(rBgDq8vH-@Mo&6xun%apmo8Qd?O#0F9pK+lCF^(7R24RKR6cUr!7YX|&?sN5 z9hdfWu)04ZeZ6WMN)N-@>$@IotnMk@v34BlKgXAAF=|cMG?dN{y<(9(=+QR7@1IKJ ziA_B&0LfB~WOC!AiHM|*14hzSB3Y_?%*tIPtnN=q-zBpcRQGpvJ@p<;bx#yTAU2M- z&%c`9$JqKodaVr5uyO6dc)M5H+95UP2+l7I(OYv(1x9RyWhE`R<1i1 zM6B=s9#bpVo$1ud=4ivyc^|ES!Biu)a@}_!h4uYEB5Gv|hx;Fd^AE{2QT2Tt=6^wD zuBxFy4Ouhs3DG8)CzK23f5c#(FwvEw=Op7#`(UU@5zG^g5X}FK$2@`KA2I(^W8NOQ z{x)Hst4(8m>pRE)Jp=Qv1oJ=RFys7dV0j)Z5cT3`Z++dzTwf}fUyHyj1y%h7(3}O# zuc;1jUxh`j%hTUbZF(mi1=TlnH|S_=Z^w?q`c69;3aX{7sFt>_F$M1rAdqULpsN3- zr-~)W#FkH^1i3l^A{6qUmJ)<~P6<-~xW~tWsz)8Z>nGo04^k)w5}w%hCrA=c)9k%;4LV z!+PcrVe{M-hb>x(WlouE{{(udOHxmhz5meYW9pgI@;UU?A16SB)blw>J>+vzPt&0u zPNtro&7jc>j3CvW@CbZ|O--kGw49ng#{eBj6*E+NN7rpCR4E*coY#VCpj-0|V5bUe z%Cnvtn$p6wgxW zQ@d^o&J4qpV4dnVQ6k1nw)5YxU7A|H0Ap?vW4<89kk2*dMGqj2sb~g`jvtU}V_15k zU8ZpS_hap^X{?3m!vvVH?d`fTy;M1jiCEgX1*)h{%$mae&rl`XXmZPEVb%>|)@Q{m z@;PQrxx?e5S-qM;qh~iD)n={5cbfHzN6R*PA-z?#4WSBjp{-pv%XFb7%jPa_Jpo#uJQEIrYS9F!KQ1(rnAH*@;Nq5z0l*MO_j}{=|MJr7p zId{}^57@Nre`V96rGZU95?x}`G`s#ChTqhdvtiSUacizh@9w%y*b-#;v7N9CC#zOBK@C+|lkEA2(4=WCCV|({+=uYimtfedKbp z4tF!8P?K0R-LC(J7EN!t02W;@7F{3~klH_nr<~^EYnvjSy?LAG9^W#_jdKHM8b;nrO|emUCdubz;pq zVh#BmYi4o(Cu~!;&46am=$VJLMuyi}e*4p!zo&8PxrQn*yq?l^Ys~N(Ec)IG0aQX@ z$UU>|{6FkEvs*q7V{Q~*akDIXPaP8MFJu*3mqQQLV*WUO7CstR@^x{oD zy?D-_?JNcs{%Swn4_IBhB(BRcR&*y|vZa%IUk?dShy^p;x~U^*DF zayVA7JfHD6SWW-8oFhgr-|zspHAN_1to00 z`Cx3$OH;aIbIYZk^tD*Wm`NIMAIT#5rAONUbZ@6I;!k3D0?7p$NqYwZs&V~+Pr+U* zNcI{#=T>{tx$t{x`fjOSu$*ys*RyX|dpBc%D2PC8oO`W@f)Vzk^aeRw4;wcejJJ(+ z$J_H#o^(doCq$rk;s8QNm~`-Y4WLzeF3yL@9`!r8EDg*%97J5scr#{cU>?T5U_sI> zXPn!-W+PNljVuk!y8=?UH1Jl$(tw4tU*NnY`6g;PV;<&zp|)yT^v|0=<_Jb9g1NOs zFuxUpxpk~7MKApR_p)z4FH!_^EAt;Qza5Xcb+=%CTVuwHaPlF9AHosCT5m@2U=PbK&=j z^f#p&pa}VrZ(Yu$L+J~p9b$@vd>d@Mo+HG@X4r+`e;9fR-4UGx7RbMFBlRgtZa*PT2tzyL!|Gvv(V zprYM_2uMx}DE4F+g(0hm0n9nBu30eWoG^l}Sy$bxyX)#Eci*PxzPCYr`}}=hg?p>I z`;Of*`~Lsdo*t&_ob#Pir>ah!3ioeLS@x3i3iSeMR1{`npI?DY`^w2vnt9)~kU=?2 z&O+9#Mn@L=$(@nMpwA55Jv}CyZc3J8-k0pjTB|*a{vXr}JId)S<76UrCo&7`A6D5i zm~lH-L*>wjPGphlL~5FF7_7PTSHOjx&OeegU<2p( z8qPm7oMfHOkv(l`B?Q}JSn203<|BFjg_!>lg89g07MC8UkW1T!`stdB#(X66pTzum z63nYn7H#Lqn^1<%{kh-i+!VP{2esIbbfa{sR=g@Jar(Qn1Ls)CnX%YC{JR%ID2Sa1 zgw19D9|&W|6RE!x4fSz0RCGL->W+tWq|rVxx89*+$1^O7GBRO^VzS)P_@s;ur!0HX znqo(Dl9#t70 zMH!jMMNz4W&g#rF@n29kw?uB%YiG#n+?={9+>5_BZtME9kY;6d=9%~>(5Sxta%l9V z*64Dr5$SA=F#cJUN{vQDQAVZ+jYh}d7gWxCu73s&z{=T;vp_MK^2XZcZK-R-&FhO} zOO`K)MVHFKY9B;UjIN;hCpw~7K{M*FhC0t`b*|Rxkj_$PzFGfQW}rI7QIwI14HT6X zq^AP&KXhbJojuqoqRvVnrE_cQ>L8^9b>vbha{x|1IaCqm7QDOo?4Jx?fw| z+bK(B?uhKsTgK3@?MYo3)vq1BaLL@6OH5Dw0+3n#+G39XL7zisn_YhuG}@~*x=L$A zI!mL)PdRkdXiOAkM%o(b)X?Hz*=0SZ+-~B2K$99$`vEJV= ztMz>@NV%;}n)WGAE9FL!R?2ND*T4O`^O$nGF>2gS>gtoI`&`sJBJ_lS-#ACr;N@K7pr_{OLHxbpSR#P zov*pO=YF6hXPtCg#`EvKDFl;lkJH8E0xObk%MR4~oNrL3m2_M7j3bkgem+;qWD|f! zTT?f`;ZBV>=D_~T-Tse}zDESkA-)D(*6sf|ot)oIUqUcbbj!r;x`jMh6`jFrT4IZr z&s;t;9#2Pg#on4y9@W=j-vFmOln|V3 zkanx14s*(u!E}F%VCh<}nxPWTmq==B%dzRQrSt1~SP5JVCTBgRJj#v#Ii-Azd31SZ z1XD`n(J@s_sfDF8r6dY#N_q4sM@KqZH?H={ehWHoNyplOj@r?V=KhzU>Cn+KXa{*> z3qYhp>Xv>q$G@PZ7rHH7=Y)>t{x?p~J`b6m9sOU{^ep2o%J{thbvyAvF?xEog8t9x z*%wKsXBy6mLo}Q(leQtjyD?HigQqgHuzGWK<6NWdx<#G zWP9T>va@wbQ~I%l(mOnlOF`M**r4JPNVvi8KC*4;Ik%bWEP+@yxDAp_#EEF<%NC@Q@n z5_me!XGWe7u|ugE29hV_K$4Ob85NMb6|t;-)8XELVf!$aah3`7u)X^4PFW7y ze~(~;qdB8;fke_+UEYFdu=9%)IBY+lk#Nw2eUC%_&+--y+mACAmvw<*8?iX7n3A5U zTU((Z0Zab1b3u~*Z&r(8R>!^qASgoTYhwV(-!MR32H|AIi%A&;s z2?KPDDM>QHu?tWL-53gC87vEW@WqyHxZG zZ2V|1x(_8lwl9@f{Qvgbm{^!*PO*_06;6O|-;Hk9o71dvmQdrZ@i?3qw9)BgqaHf@;4 zoHfbzhLJT*N|07M)MI{_WKV~ZasRt($+96Hm*zk7loo3lY7@e_FyQP&i`3t2lw{4Z;sL)2qpV2aHYB?{ls~&B*t7pDfAl2qn|; zzg69b=Xt!im1MVvl6^lx+RR*eA1}#n3nN?8ErINa94~sSl6Gq-Su}xc7HGL&+XWAe)ov@jeIYc1tK3&i}Z%rWC&0%C~hb55B^*r8A zK(d<>$aD<9*2I6o_%cs;@-mH{+4ykiNTcU7YcGa4hCmr1A<$a1e}7h#b2%tQ@p+$2 zzz4e|J-=Mb{D*2toL{ok*dQ#k=a*~$$1k}_mMqAgk59h(G(kJP9#<5B5QseJ|t zFy_im5-REGc>5YHDqDt149+9Do($ph@!?-jXU*W-uTLO}=6FoRVY@-3Jt@gM2_*H| zp4>75lC{Z5FhNk(3V_YX@*-=Llr>37DiTNzhdPK_39KzCNm~NROypBr6-jGS5;Oh> zz>e^|NQ)wAaY#nvkBCQ)e>*|SEa63(@ol$keE2jhuf64jK!18m1KvYXIrxw-mZC3! z`w0_$?(`N`;cQvP@s<-7p%i^MJH%T~*n|S~mfR%XGFFSaT8qk+-onX6R)Hs{S^UUT zipabeDGb$H2~^qLWzrh%AGtq-YMqxrmDA0WnOIQW7e-Z+K$VNi=O72Fd&8*KCs5_( zc@d6rpt>iF>b3-`e6$@7aiF?8jOv2~s%~gL9N|E9R|wVmUI|p)(Sq(&RCk6^%}k&w zKqKP#23dE6QSC^e>VbB2yP~>1jOvjDs-9>{w<)UI!l*t?pz4Lz#K8>+Z;hwYVOodj z{{ufnx(chI=>N0vVUv+Wx2X=(|L5|7FQORl$-y}dy9e$#-G{|CMMr0^%a_#ycifIr zY*V=*ZK~rH6kwYwkZnpa;7|6@>@d-MJARKcLYCnqJc5)hzF36!R=n{lt0hYIG$GEi$ZlAy#e!|Iryi%!>s4a{PxEy+kd7D=rLr&d0K-WNi@)T?+y2% z3A@E9yxC>0_yDkgAW*5tJS9M!iWi6kB5}q6_Mh(duS&G5gTPEpJ zxd>1eUjmuLv`phN^MEtjAq+g7i5XXDBe z^xxv@PuUBjFX1~juEs3ALc7VEo#eJu zobz6gj@=Rg6;L@TOIz-wB1ay*ZC*Vtxrw}}Eq9%<90^Huz5;sEe20*hTM~T*bx*o= zBKk@?mP>jwq%vSuDh4Dbc9L+slYZmiq5-!@7HY?vDaJ0;(!SYCE2MtRt3o zoL`SKBawaDj@KJIl8{7aD7+_|`H!k_+Hq<0RUxuS*DRv1rejCzrbA`D>|*rHJ>i16 zM@5&!x7(ej-4e=b1!wzcM-Q6s6rP)qn~1RueNj5*i-S`@y`&&*wCz_o!f3Pw_1GdF z*{_XugE1NjEu(FJ(;=kMmPKFFCqZDe*S>+#coBy(T28t~J7uK`p!3QM3$e z@@Gfh&|AF_*}ajGGGjWdG7@=j#~X$`^>+?`u1P-;IZm%huNINx(s9QWSEU0W1fdl@ zjrpHoFypdxLF7c;E}-a%O3~En$yT&oDSF!b;2+F$7Yo#h6229msMtb$$QLJg-)Nru zi?)+5)jTM)AKtL==|@`gTyG5vy#7j$TwOr7x6?NO9ot*)B<)SlTu;ABTd233=R$+d z=DDZuk1Gvz!{@oD|HUuOc`kX=3`v^i8G}%gO|y4=)4Ui&(D#gkG%GnsffX3#NPe7c z<>VHF>p8`Ng+qS@jeMbi{>>{E)6}D3G)DfN7>RY6+?0cV42a0T;G2E;MF`^3b#caz zth&hIj9A<_uWYO;;I-nD3VdlRfQH*QqX-xC%h=ePgNyn9tzFCFV)VgrPXogN?b2e% zFBUg>XL)sinG1#U+dEoXPhPftV=MlVUB7I}%$fKltEqWosW%N}Xq5aDTDQ+@?O3yQ zt^cQ-_ASvZ%|VgD?OT?2v~SwhdUX5NwTExpxURjqack>>ZR^%;ku+qKXUh0odT!X< z(z>~|WyXee+tzoelE6od=mpBlYg%h-TRXp}LP&$_=D%yBV|4a&=S1GrSBIndzp4AV z6d0@S1r;W{7B)Ck>MpbXtwJa6#D0QLM3cK-5b7;QKNn*6w0bp)rAV#TnX??V*sJx3 z;8}(;M2A`@rQ53+wRFF6=3U^=expy4exq2aex}KP_mSf?uO{9-)r&}o{=A%J^)P*= z@&7(@nEp?EGYB+E`hv5@Ksx(^KJk44CjhYeSw}ePQ}mvx_=*kso~)??odK-d+EiVI zQL)n}OUQz$b^amMh|JL<$tC5?(M%QM0jr=s3-N|dXJh?87^plCe>)~JOT?b-8uU-1fp)_4%>Ix5GRA5kryTxJCm;C9lDhQy?Ee_En$l1=JjObIsb3l{ zJD>r4I7BDuSI<8UC0S$r<7;d>h)|{H->g~n<6r|;U__X7*^z23VTiMq(10(p^eVz~ z{Eqvvgt!84GBf@!nTZCZW#;1P2{PV6(~GRjmfdDkCGHWeZSC~l+hWvkPhMUeJW;Y3 zBD`hkb-c_N65MeL-~|`^+A>7j1ddxvaPuU77c;z)u6m3m7bjqj3%>BBndJR8N@G1O z;)sw5K1iRkP3Y1A|mHh2yhZ-7(dq!*HXBXd7bP9&@GhM_Mq`G2MD9>B|CBIhLKfBIWO3wW!b7I~ri>D*p$I47Z)f5S zlo#hG*zBn28+!YAn~1)Vj?I{44IZ_w%!WEmA?-#q4I0r64kFquD{`8)8`8d~rDL~5 zL@*Qoun~mv6NFtrCyyS3)knVPL^i4Pe>5zGQGab|E z3g#owmEE9aa=VfH&Jw;f6zy-9u>WUo9r#;zDVm}P673@{q53-BaB8{ zQjc@_kyo_QZZSq9A&qtx-mvziCjL_>X|xs5xAaviYee*|bc}|R*7m~>mEp2FF!(H-uX$(Xu_dR)Fj@N*OH2Nz$D_GZB+t^&& z+WE6Ll#Y6!xZD~4@pPC>@s9{@q?8dO28s(aB$QMb(~yD}>%^#@O4%mDWwE7*X)P+G=5{BW{@SK0HQe%<-)yup9oix1R=$qIgp^B-RU z!LFr$uw+-Th019v;G^fe)?cf?(qZa6BO&3ma@9?8j~~#Wj#u>or6!HzK-j^>}&(p zduzPm__EX+t{9g!VNs+hz?{4qy?oG+sbOr!zku;GRUSA0m3r8o2XC)#ZE3A6ZTwos z4Eo4a8{_)Y1jNt63T*WJP^_z}TAN!c%yl23#vmNa&eva*!1}pq9&Xf;tkrEg)fF`@ zm5t4TcO*AQ*1$N{rvD4f@%$8z$seL#Cs9|mAvIrK7c`gAH6dW!)!^waNbltoKdy z5c*1k#1!i~EN+$hi(8?rySuD!H2Z%8*83-T5l&8F>(z>NZEa~)McdaoUpAf3@{k)J zNbr!C#(PL1OV-O3Yg|E4SyLVKO(7n_t&JAzoAMG^zdTkQHv#Gv#kv9y25+nlSciDX zfH>AB{}UMGJ0rcwDusHLWL;U^+FVy!9ynhJ>bNFz(+dfxUn%i$sg^XzX2m+Si@oFp zRF*+*=K8nWHN84q9od0FS|#g>meQ))ma?zy^%y|gthXkxer=dKrvui_igj~Ub#ry8 znJb1mA7>FZ>vt1azdppnQ?sQ()+^Sv%@w7k<(+R%LFYWyWqpe|{}WW>8-qPWQIfT~ zU%#TZwgQhS{o3v+&O>hD{KvKRn}gJeDj1|qp)PHzY;Gz4+D?_@xn+=BK2Bi$RzD9< z{*kCVBYCE>wy*8^ddH#uEP?fRaRg?QVy&)0OXsl<7V16o z5?H^Bqf{#t>lF%hC1SG17Q|ul7(i%PHb|q^_|L~2((f^DvCMJnF$t`{hZ9)b)(zIj zXx5EQEu|G^suXGtMv^vblm7}_@B27#b*y52tYTf;SY1`&$NgcfvuO~E^=)SU8?gQX z&R1H^&V&svr_Q5o3(kXPEt z+tjc+y)`3^84UztE@Usc--h|W#hy#<&obL3^P5<5XSOS3dXw@jP)lvolJU>m&HWEb zDK5F+jH!xR=!a4(l~Sptwk)OUl+A88`@i}LhI5<)-X^%$poIG_#>kh-2YdX3J_DMC z0(_B)o_XET4c6hzYfyL+>W-<}f`gcZ%3!Td>4hbHyteTNUTg$6VY1zVD)lQo1QmkQeQGQ%%8EnIio}~FX!anXPzp4pSgC1 zy9Q`y7@WiziXj2o-DSqV!E^?15fY5!G5(1LeC@kKF@D$XaMuYM@L;+F%JYz8j`9m# zN_P}@asNBrVQ{=V>_9<^HvOM!QxrX+C%&RPG-UM@S!uXKhImhwPR&#(4H<-?tKn`l z|4~pH(WxACkbeQ>LFv_T4wbO#_--fv?`;C}Y14GqQ7)z_=BWY4fj7fkTlwxUwUztD zTbV&6tbET~Meg~?(x4{=ie0b2&&jc+!M&MUgJDTDV0Q*n+&d0sI5G{B+4r6JD|}AA zlAJxgcX?cCs2iSrzxOP^G};BN1M?)!zTeCFAKS&S_!^rIBEtNxW;Imq7uiWhgvoZ1 zbic?>e37tUw& z5B253FjDR!%H>$Milc5O45sM17z6>(p9$h+9|Y4-yO0q*C-St2am9jbXHTncBDH!_ zkQHyey#K%`hd)=US4Q8)of71a<^8wwW&fEPe?A%D8*C>b6y4!G@F?K0!x=7{tL!@P zt@@r3Fii5mk2K`r@&buH@P~%(UgSIY7P;K-$ouz7t8vwKd0k_tQGK#2%$Z%F946)< z>%sa2S&f+Wyo_(B<0Exhad*FO(jqv0q8PLO7pT;P8z=VR+e7h@x)|(e;1%=SU;)=G zh<1PjWoS7MnfO1D(~LVRUQ%*iN>@(s7lmk_dl=_Gq8uUuh1zg+#(tE5&+SiLkn%bA zk*F9Lq@3mWgi;Ti_$QFH2J^oc@GUa)FQhKZd5Q|>4-{~W67q=Y{{tawGrh=*_%@1< z)P-R05j^hjD10uuR5=}cON!B<$Rj5HA09uj9jCFF_+Jief(^Po@_O)8g4Z(-pbsiw zZzeY(PEgVw?We0!N#nkhW%YOlLgY1edDuj@C!>VDyCNjfS(=dhpXmdNkX_}f=sWt> zbZqK*C!Ma6e(()$n8|+!##>&G z=dMIv*T%cm7>|S`+KD%u`Z2Tq5g6~7=yzdn(zS@_chfPR8E4YnM@B5#=N>oyA6V`L z9#xS~v&V7uH`B3PJe~q-X8YXZ7dWm%pF64^Pv42Wq0M%iF&hbKwzKhu*&ct%A*9b8 zi^pK%hBAD=Rz%-T$830JHhs?Td{pKji}72s6T=V@>?9+eCWYyVqZ~|ZuIsp-FHLs@ z*7MUbT{27s)KE&(UiXAq{|vnD==$4XzBje`Za3y5A&GvXpg!q2P`Te@0G{J12{(Ldj>A8SD&K*qAtvpJtz<8yQ{^eA+YAtb(273w zJqO5Q02v**N5o{fig5TIrD$sPWcZV}9;==%2Y*h&hQ^Oo{gfr1#{T~~_*EWKP}Ao1 zZ+`YC{DlQ{6#F#qKV%eJoMh73t{HLtlhfv6dCcPS|FdcHUW|Xf201%`Mm)UovO0WN z8tmoyPijz-L<2o--g_L%a6BxL(`IPU*|d4@#c`$4bi=33drkhUgz2OKl_=FWR$7nh~c6X|3mk?C;mng7t%Ef%f# zE9b~~)>DjLEh1j;%s7c@O^e&Kn6>zXJ)brCk79i8Dl!3Z>$$XTr}-arkqnT^)T-h@Ko{FcW@H4YjH9 zM?|oImCIUI!>ku_uma43gB66L1C-~UbKva3uC_l$b_7B34%Ov;GdKlQ%nneVH|xL3 zyBVOYsJ{av>|4|xLGc|KCDcO^PQc&v2>X1q1Ct}{Ks*6fJ@W!1>|=Rp_%%j`)bueU z?B!{+IeUa{ZES2pTFWeDCvBx#N9}zt%ysnSBc{H?=X=%AKJdQH%KK7btVUSa$R1%| zxX!V-uz!oFDP09lMme*in6)O|2ebWM8AtS_BGDym1= z7rzDm?5~UC`)dwA=wDuZ!Wy5%>G-2q{=H@FU99ox#Sg)SaM9$0UK+ zj^JRytPBF6 zL1#h0{wu(dA%iIm%ZCR6`(N}+GYBB>kRV_`<6k<}*!UWo4v6+ho!~_c>KrExEg$6CPc#Kp z;|*xv$=A@v$&@j(=;v`$G5e-+e8`OtDLuWL_!*Rtv0>hp!2JBkOb-vOlguBN%u{-f z8=13bu$i0ue}Mdg$l;zm>jUHuE9RArjo7kbHrOTgENOX$au)MfiW87u$oU_5vdDC^t*YN%z zf%nDO8u)p%ZsY z9B9WYP!TqAos*ydlZi>pGi<2Y!+4w|-zZSHAHctl7$3@>4LOzNWxvKY70G%3t96_Hw zPj-SagwS%5lm(_xB zk&yOdjEK5^|Dp}1`+*1IyS?46Kr;PJ-r*zPgq_V-vvjzi9<&!Z@K)3i^5hL$vbKPUiMM4tk9KqX*6U@~V{h?6zzavNR!*tB0 zD>Yce$PTj@QSB_@OG9DRqxRnZ*nvZf-5Wu*%fJ&pc6T}!i-S`@y=)(QXTBqhKDN64 zUKs5?ZM1uh(MU+6ouyE|!}UKu_Lk#9bM!}u-$;jbvmd2nG%QlV$KsYSayl_wu!KQ4 zjSA9t&Hn$OTe}aVureIh&F)FZa7iE)P(LY5d)#*~bZkK59an!JjQ74a-hIY+BqY%; zjnu?{3Mm_COZ3N>rjZW!75_LL<6*JI461VJUNzL*;;+5unP8d}pm&!!n$YL&$D=Wn z>9G29Z#t$+2B?7Q*{eVAn*BdPYmL?458HiD+wOj2I}*}%JMl*Ic=sy@Ds9&qMczv8 zIL4aePtviSs%6f~V#`)8nyE~e_w}YDmPUo=y~`XtG~EMu7Ku6s53jo~9n&SjQ$Xcx zuS0-gw>)}XL;VA=-1oKR9x#?8AuV^FLi&E4LrBZDML)nCoOCCM=m+UouCM4#uZu2S z7MndYx@1}X%;+*@K5$7orb0+CrVN18smWE3dSJjZsa+BtYAEzm);6SYIDgv0KjX& zZpl0~8hZt!vb?$!>x|~uQu36K(s2Rgz<+<-!_l)0iPytxL`?1^sSq_85=bg+*?#)U z>=ro}Y3|(r&n(>dBw4tvx{MkB#>*@mFQj=2XP~)+xe6b0&sj=IxD?knB+h%mo$@t? zpP2qH!0_}`b-MsCtW+49%Bo9o@>p*kO>WzCFzk-Q@NHca0fxOeu+4r37><<~Qpy*m zgW*0G!v|*nQ-I+aoa{bYVK`c0sBLU(E-mkLEjx{bao@Pagz_7}~@Y6{Nc6b4&&zC9;OC^Tp*1C$8@|xg0W3r=?>0r1Z z4#PtU7+xHw4%eeDRx1ptWTVr;z?`vVho4m>VAwZGox%r(9g8aFoJORa=P-?!hGk2U-_bxfp(aZ~}&xioM7(lmdoj3PWiH?ggne z?c%`gaIwbF8=n}le$M!h8SU}$8Evjtvw)lb;DGXSq*261bI8Y{d1E^9MT$-njD7^6 zQ|lkKYip_CgLeTHMotm2VgkXU`6+62O>LMrqbqs0;|)bWdIS6gETPtkA3-8q3#aQM zmMFxBd~t!a3vW341@gahz*uf~^N2<)I`Jls68-Ql!W-y*!Szp${S)Nak3Z@czt`); zzqm+SXo6fPh6bIj6aT`*zosQ4YSMxwjnfa}Ub==N?G?rAVHgaNCKa#puVLX_ZbU`4@Zc5>H=CY zH9#*>QPEg|(Ax}y$-_fc9lCq?<@rvPlwoE3yvW)%~D4W!JMtRrxP} z=kVtZXip)Oiw#4SDpPV=soyyJUp!uda7$#Bh{@CHP->M@Dzyr*l-j6l_8TYviwg-d#Lfxm%V@IY zVnR9=OS$z5%GT=r*2({QQS=?R@P0v=4260}rGo%BBmlc)-CL>m+dEKa9FxYxk4gP> z0qF1d`fAJhGXuWhteFM$82sBmfD6aqNlC`wcE#w^UMRyTVv^i#wjVac7mM)zH%Ael zqW|+XQ1_%(_x^$_9hL^4KA<(2oJ0d1MSS``%5Zp|ETaf$&{-7m=~r>3p>B8-@w+L0 z=?#b?$U7v8_}$Sc$r_s+Utp; z8(sr;;|H`A8T;_VxhM7aMOIy89?Wk$zG}DzAE*nj5uZ@t_a^@@6%dRehTu`O^3rb) zD-}=XrO_Fp(WTb(O-^ioG2&Tn|B%b?u{jG4pShw~j=t&Kk%~U}-}B@oS~gkm;sGN} z1$7&ns!J_15?Pdhz8zX!Uh}ybfI0TTe~W*XiE+kG%htz4cVtTfaxQQ&8Aj<5kEX?Ebs> zUX4FhTN)J5wxZ^bS3HFeOM^chqcxb8L<8Mh|LF{rVK+EU_SVp#v)=ko?Emf3P&d4{ z{?kwW((J9tJEXV%Gtd99x1JVXV>KXxxBhvoW;Ipz*6)!K=X~srk{%FJ!j|D{c|hng ze3H$2n~i7?nchO)tR-(b-y$L#vt{2<-B?~;TUTllc_HyCR(ItAA-UfC%})ps8q>;v z_m=`69(w@p&Rt$8Vukn}C%y|au_fLSnMV6pJ7OtEyY631`=9C`5ju*p*!~x={b#~3 z{JB0sF7V)*YJ2^e-_9bv!1pe_ul!LzuUE1S(aLeilijc&LvHcL8n@!w|t6Zpd} zTt)k$!l3T>NNFc~I(E3t#qc-N{sRnW_V=*uT>9j*(jRbD53WqE=yXU-%MRDM82+A{ zP#0(Q@i?DhyLeJ!sAy>_Ei11zYrY4*mT^%W29y5|>~MBZFY>&?@VvxOUfxvIjN=ep z);4c;G5mw`fA&!UhI4Rn(sK&Ka}q;Jrw`KchwI}oT$O;~T%3SlZ-=^gN@1w2YiTL3 zHFh}gy12x}@XrAW7|zS^aF$qN*efxlbb(YlcDO=g(DR6Y-T(}YN=xITQZ6f@GykVK z|EmTR9tb@j7bvM)Hn6t!N;;!^ia>*rhamLG_W4Y0Ed_)JLLWrnrtWP;Otm(>n5wU= zYx35X_}t{brp8-B&!|3k@?WQeQXKqn17k^RKJvNAf0ZYhk1!sANxv}uU+xF`sT~OY zVz#cV@(_q$VEu0ngjlyic*nf;z64OHHP5QpPHLf(m!thDomuCNG zmEfe0yhA2^Uw#3;Y+hCIH8u-G@P+>wq6Jq@Dj zT>cAxCQB}#lkH}kt!Tg`a8YCvbb?;uBoW!9CV?$wZKaq5c6!FESo9Hvyw@-~W7(XA z3u23x&s>g01*El>r4cftIQNkdUIX5M_^*CG#8&);6TzdqVgELM7c;$XnaRYaG5V?# zj16GM)DPvImhplR;j6O1dr*PBFUk`riy+I45kokVU)#gKq3F)(1MT z=T+yGT73{C>QeIFflny+zfBH*P6$tkeuTUM`Nu@`BQ+sRjla3CjYffIhwSXdSHaF+ z!eGvqQ2yjV;bzN6aZJ=dKJ%~|Fw%!oK!xn%GhZ?P%{W4~9eF~lt^P4g20x(Im<&Fa zQ9>OQAv@nQ@dm`NE_VoNt##2~2qD)g@fgluq+>1j;yGnA)MoO#tRO$w)JrrCc;nVw zG15UqqdkTPf+?dNjt7FJW3)s>1=LQ;q7^95tp5e3TT}lyO!uKS-Q&h|BqY%pnvnCq zz~(gF`sl~lUO_sXJo-2t)8VL(xg}EB4O*sPH$qvh;CQPXIJDd2xHQT?-S%iYc8h~k zK)s|OZ8YySM;Mb2YwMqY(SE9p_JlDS2}yJo-mrFtIsc%nO`~;0KM`UC>2UJslXQ%R z{1}WDck(DJ%y5v9?Br2ShKYX_BpU7s979)z!^+BI=@>2vqyp+E8u8Y~%QWZz1CML3 ze-g&~8I6F(`lK-)2}yJ+-Y{O~bjRv6-iGKeg(xOngNXhz9pkAgwhkR(92|g@BUY$q z%#Lz|@?TyFU-&0+Xm`ux{-?nEX|*S@138vOEI8=)csf>#!%{${q#$jvEabl}VRX0S z>+t~R$j>Pa7JJHAjD#dogF#N#28WOq+Zg?o5GAC;VDPJSEQW*}+o*rkDCg@9Mi?qL zn3Ki$$2Hhf*aV~=P8Y!@pmYr8LMfn1R&FpSyUmeBd!0~^D~cl@X?s0w>_tM_>nyzC zVcEAhgtXVD=&x~^HR;eif1QrKR9iTD;gY#Cm-xq>b319C!FWJ8jS5muxdVxg_B0X& zF>am`T=-1KZb={&P(QnQ=G^88r193(<5sK4FSPOY8sm|W#^VGC#>@GcLrCLoj{Zi7 zQKZA+nBSyhJikdSL2Q`?26-XD2bufFiikuv(A?u4G_>1ZO!WK)`cyi0i$haDwWJ`b z57l#j>FP`$cge@VDvM%)T;M84c9Q zM^VDsX}UFHIZor*?ckx+p5f)*vRlI?q)(?~wRk)Q)J#g!X3OvCNTb<0>T&-=ak-_OSzzSoWGzwB76WsaTqfV~o#tsWhQYfW0^ z6&`JEHArC9nB{}y2`nxypQjlc<_{+@e>DuN|x&xCh=%l6PxcU0rjl|8UoE-t02T*YGy`p98DBevo=_B6zRSysPSvVGW+) zlx)X3jl!+CHt$ywaKBNjp4kcB>Y1Ib&23d><#pyBnQ+`!y1WZY5_rD}ro7399}0JA zQ(a|gdFOY<5%;&^c$@og0-L;rQ$CzPg134W7p{uLPLURqbxytlK;HMq@%|tI_uDwV z)1r8{NZc*W)lKjdgFD$NP!| z-rvPZ7w$0t?-LYmWPfW*%gw2RaGP8m$J@+*1AD)VRiAdnTRrTnrM$ekth~|KJKQGB zmsmd7b4~*8_pp-Erg*nW+|5lDFNovK{HNQaygx`i#11x553y^mt81w& ztuT)>2)D@%E^l1D6VLm51J(1sz*{}5yyLV0`KqlQBNZTZ}l|7 z=CWp-oT})&52l}RmB6xhFSGwYsLLPpR*y6VZ}mvi=F+wr_{rDv=H52)?u|Ek@IvOl zm>R5(PYtS^QBI*7|Fesi4=StXFo~Rzeo$E|;|t@0z8orL&PZkUa|*A~hEp>#o?7`% zq$xPH(v;4qpH^m{K!VJEPVW&85bl-g8#zt*r&dnONMq==6=j=)rF)+N{zB%7n9Tl6 z2wIR<=3+6%cYlOs7Psq8t>pCnri@o^&9ROy$pTTDUrqcECTwu#j5$I!dUfls4&^OnQ2F* z_B-9-&r>N+M^@+yn{g^-MP{ho)9TeImLiQ>=KPcLCH87Xk<)`yDW|8~s~NRkR%-QM z?x@96DR-1XVK%nAMOm(_jn?X#YHT3t^mu+6ook4DVsU4YGIA$x8rKS|}{4j?+;lTI^jxgXnhdpwB;`s=UX-=lea4LnzG($8@ z*%7ozcdQz^S~*Fda{QgLkKm~{NWQfi}OKk!(GJx`_VnurdN+?R`q=~xU( zrqam3n6kBc1D*Vj7ezO-n`gepYW0pz2LVo{0J~(}8=t7NK?g%UjyaX_V-AN<7J&Y9 z@T;}tJ?GLB?!sTJF?P@{aN!tRmt>69QNkd`zl;*nyc%qRh`LjK?syleM4 z=L|;w=WCF2Go@rdUHd9NEDZ+F&>9?+M1$cd3sVeUgEEZZ4`P%+4LXYw245Xl8tR5e z34>quOCw5jslKgmWkpyykRo6zO?Qmj@*HfPH&sky4MytJ;Y(~XAAmkpHroa5KI58$Z? zgt=Bh9ctEp>I4G~EI0bDh8cc@^cd z)>g3x&V<4E^LWw4AP9g>hagPac<1HBh8s0y~q6ZK~Ck$lZ0C?NK_ayM}T`YOuFrizm`mv&Uw)|IVmKH;PjJ4(y8)RoGP zVNJ)hWz&w|*3rJTrmn8Grmmv0qO78}tgfo8x~zKI*48as+dI}wZD}gTgezD7vzW(w z)^1t5xwW-rZR6&~j$$;LEZ&9(Jk($s-q2!{S^rd8EJlnlW)M#9(aJ?)5d2wpCSqVQ z%4bkRl1|m+n|vc+z|F>(BP#Q7KO?@y9l3cK79vvj)Rc9a`XQehTeu{36{#tFxE3d= zk}@I_-EKx9|0CPYtoRDm?Pl~KyWLy>N^ySf?;pm8)ow-~3NEbmB}r<%5@lhp(PsbG z60r_R9G)G^^m6gt(5zhH1=h-7cw|gReMqm-j=oc)T@r>C`JXu`uP$ME7r%**GOj|f z?4afScSO5930f`%0qq#G{<)k!I$$@@`mMZOYy`;Yi+oeiMSS8*`^5&8bM_TD}WCG&xCi1Tl7sPtK~Oaa2{#L{%1y zkuOQtI3`f`M(fuxV_?E;`I3goCURvGFFRWc(se%edI)3BzC5JP$3AP-xnQ18Jkapr zBz#yNI`(65VVxhFq|SBJKhEs`IhG!p|DRB6LQVnrjL0ew8A=s$7gPCfiCO-W0|QxT zz~km>z{iFF9*6$VchZa>lgFj+HNA%q3-CC`KLoro31CWuLyUU@W#|wqWkBy$A|%#)fh({Si-O*qrIgVxu4ATj;;KsdPVS2jW_w7X6upL8~l!#po5elVuIz^ zfTP*LDjU zMKXcnYDRLB>M?l&^B;7j#w4z!ks_y|v~To%e_|d7oe^0*<9r|g4)y(sv%rRpwK2r^ zC$6=8Kkp|HAP&0d(&hM|@2|ic{xIsBvSGuz^U~JMEv;IHw(G>dIdWJ%M@|wk zIV~#^ldNViL3#`@NW0EfFeg>oVUpDUDm(H*`#G+pu|k zBiaM1yA1ru)9}66;roaUYl@g;tKT_a;w4Y5?xbJaeDglB`F5hNHx4XNE0agr z#iCr4rUH54wyhTCJeY|u^s~}r+Uirx{$JTGM)zM`jqQztLJ$jzcR8=J83@= zS!j8w>a1DoBsL5_bQsXdLm8mr;EslY?i!|=^B+oq1~Bkj5uI~XP$}v8LoBQWS7er^_DsnveecsHH5->SCP-)r@2U8g^ zbvhWcSEe0n9df$a08=$Jcnn&g2c!=W{kCFiwxzwbV{2?(DWz^@mE{#xl{J|1fit-?*rEid|I4KZ`i7pWRYgg>I+dVpNT#|9%BI`U z^Y+_}lZ8{J3XTK|;3_-SCePUk^09rjQ%)0ovn|a_iUl?=HgD*_#1*PbKeOWxFvK#E z5YzD{=N~zouK?#m$LI%Cjuc~Do4SmlB``=vTFY{$|IaJ4nH7mKmZK^6AUR2AD=g)1 z`=^%U5TMBTYKka|(-PC7<_+6Awr<{@pd(`}@;q4cb^JevIvzGw@613J&(+ap3k?|{ zk&V`{BHK2X)X-;DB*rFaNOF?)#v9hII2_~*epZGAKNYQ3@FRAEO5FCH$>-pM!HiPq+#93?%dfgUgC(z~H^mYm~N1QW0^W2E26S=EP_lJOfg<-!aL4OxD1 zBmM%GWUG~^%-2dBAEHDh@?Z3QwVL(c_)UzO|JQr*VJT6$P%E)6i4uBTsWkT=tP_aK zL4fc(SXEd~n!WaIg$hh4@$2Zih4slS1mn+Mj?o zG^#T3Uk5kN%IS5ECy^HdoL`BoOig4UzM>&^#6#4+RrOmf{k$aAQl+Fj@27&V|{=>io-tp-ZFex#;WG}ADhC3 zN#I_p8ST|@FZ2^Nx%6(<;1-c*C2FkZ>jhn^uoE@1uTkT6Nip5!B>6K)*<4?SzkglI zhRF2jT4T=tDRhspp_)A12$5%XPA)o#rUO}{Yt0@_^efy@-^%>F+mr#GxW zce36=P$VW>)}JV?4-EF$S9hEZGnd_r4|9qOv-25U8TiiDZ?vMiCmp_YFCW$vG1*qX zbH2n&zIq@$sHe?W&J=yVov7;_4iC=0EMoE@fkk4l zRg;j8p#aDvsUbJ(|I&i_yTZwpBN*~UU;{@u)U1Ch@+&mCoG@02!RT{RRx#jD-WuMa zjDP*kBYzM0FPP$MH^tIUxtW2dr;&%g;gFYt+|P;EU=N8?HK|M^v5*LG)->|4Nsff5 zZNI6~0Hz9I{%{GpVytQ8VP`vxAF&yCRvZ*wUUc0DRDWecSX=*THz_J1fhqc(mz z-sB}v8ZVIJItZ)hB~EfN>ipCJLUk5S6|wP3ovDfHc(TW{FMkaNQ$Ut0Z^t3>?? zT8V3uD4_?7`pdKu*NRpcf=waY#LYtyH~al-FulXHPX*0nq34G6CjX(>u+n@seGoMJ zY-l-YBx`OsqpyQuH;B8TWy?+;BJ8>046c8(cCQUF%#4_2n2f)Y{FmR|kC4@_-5G5E ztljISVG3DFzO1?743q!7-k%$G_ayQ{Xy#X9;@oh?CmQ1Q@euXg@bC;R{kA04QVbmM zt;5Hn4EyHW^v04*2Hg>FCt+^m54?);9*j53oAs~T{Ea33@b?US%NI-O_Hp=54ez!v zczZOwTa&#|_P9SUfCfrnHjmv{dig6bRws-s+Hm3-vIJ!F}gS^1&4ikqc%nAE)fX5=dJljka~ zNyt^q=K2SH^r4VSpEKC1^zLH;Q+!lCe;YomN}qF>uJp%}RQd#zMWxSa*7ki&_D=g? zl_BWkWdgjXDD;Br9Rvar>GnM5Mu+fy5QgkDlih*GfrF$GyeZejKT>Wk+)PwLf^5%Y z#Gq_TZVBVTuiLGGC35AwieGQ=n{8Ij-Gw~B33h8xDMn;ksUyw)H_1irY4EwY3wcT@ zl9QBEVkmW_IsYFhRfXY1UI#Hj431OEY^~^gx5iwye-0mkVslKAXQ(r^1SG=}?hCp0 zZjHHfZ8lOfpN%`WRGN{TnypZp%{B8Mi9)8d$_DD{n(nMpO)5E%l-At8+Awoijq^;D z@5bkoirGkkLBgD^V9qPE$#Zvu+=n^Eh1q%PsSNzc)9}5(;mcmghc(5Sd;nHwn<|Kx zJhi{gWBlV+u1sn9d^=Isn+z84xB103c}@hdj?=Z`^%ev3gGK~cd7$BI(1EA4I>fqv zdyFz|?S^fe*KKcU+}fz-gWV2r2gD@wF|?aD{Q{hS3QR9|K%`=NlX+Ue@js|J52pVX z4`0gpC{kBEd}&!+yDHs^Z`9oa?tcmx(9=U11insUJsVj#%w~|&00zFV4~_Tr@YL19 z7KhCyu<`BS&Z~hU3*y!2NJq9h&V~2dT=Gh6E}b~CI3H|ajD^3q$q^BIVni=J3auBt zx;wVisVmF68rIkCfPey6h`0SJ=`ILj(oV7n^WUJ7a>{_K(?JfHC8;6bXOri?4{~mi z)4Q>G#?UCCYS@q;s;si;D~J3OTly$52mw!=I))pXI2?}L$Vr1)Y~mj^kbGuybGL>E zk&QRH;Dfsh~r1viBS^6&&xuoEI7g|+z4j=J7x|V+7613)X0?n6j z|6kD582R~(JJWnNSim`#oNJTk>;(CEL|HOanIihdjq3Jde%#0ocCz!=djytzJFEv3#Pfgv@dMzzAC{Z_4~lpIX`LJV_M;{Jzw*UcUXE>`UxW%fVp zm4@L|MZK&X*HNbb+w14J3O$LuFx#rc#2nXA%>U2<_r^oiIj*CAt9|RmB-K&{Ah0KH zcvda+ML(60lPr~x?W7VoE|P(63P+p$4==_ep)DxI+5hkn{wPkL^dzg+GS<;%|HDg3 z;9jK}nf(th`Kg57-E<9}EV@~#gsh;K^MYU9R6^W1S5i!$IZ6JsHbK0Mzkgj~f$(%< zVTFnR6vRj1WO=;Snoc$XH(3W@T!;`2P3*+NiaT}PC{FZspHq?OW_3_1bRs!BD4keX z@f)9f9aqdmz7*mDE7ATrbjpoX z`RvNY9W$D<@nN~Z%4@-ez2R#~Twn#t!k1P)r`39mE+A)&<(Mu@`%ZU8^T{O#GGkmd z+L6;6=9_O8Qrsnw$E;*~Q-pKmAXdiq}3W6b)Os+9;}#dNKBQ86e#@Or??gIK-> z9r%p#W^v>H<&4qa`Y@)vYD`VN-UxA;oKsH6-?mS|kM;b~!XGDe%?uN>OKE3aRmZ+y zyMUZwrczaAf_Ut|Y;w6d;%Y2kswG0qFqc>z3u*}c5?zIE$TwQxYOen+Ln`(g3);zAj<|1d9DOq!u zGRAQx|0RtfH%E*zdCoRbXf47FgflpsBd`I;?4*IOq~=bh>Da9ZKnTX@W9I(~PzkIu zVTg$Jk~hFFygKrn~NpK9{h5ZfGbzeEVR@(n}YtjrNx<&a;YrOTP=Y%v6*-+^cy zh@?1!_#H>&9*E>xwN}_(t5(iv5bOD8gH!+c+(O|es%!8V!l1i?y~1aQoy9Hzt#&Fw*A)j*t$72^{{{^)pJPEtUAa|#n5wJB9EY5n4*XR4Xr~#19Quvf zVfs7Ot$~XmD>f!{3yoa=^1FrnMzC8nO~fY9@rHku1CHHX? zl@(ky{S90gT)mMbxYAS2=0O_h8~#+YuZ$Thf|${sYW|&5O+=FkQ_Z&$rkc%s{tuOS zBSeX2#Q%PTDCAUg=$6C&;40C~`5%>dD~S?%cxg8IkGDjto@(Zs2z)4?C1aXfmj9ca zYBGMdrkX8dv@H}HR+`VI55jMsm~|wLGzARwRI|nG|5L;a|BApr*;KQI>)$lYTOo#N z!TjGhjNJVoDt>z-TaZ<|E#KFMc}E(Cot%1EQ_U7L|9!`wYWDLa^1^JZ5)-GIt)&{` zJMj?pRI_!lmi|$aYN-Mccya4Clwtqykw4YUO*Ylcai*HtASMId6k1vTG}cG)NN5X6 z`Ke{B%zN=i@w+p3?ZAg+tTv38(u+Px0#{Ep+f4lbi9gls-(A<>$)bBfFfL>Vzk0#1 z!dbOWX{vd{^pdounr){4R}k6%ck3=<^1;F*M0cxmav{Q9p@}`!T+>_Ejp9UK_c;}b z?p6l{wKO?<-Bp%p|!dE2~eP5&R5*XL9u3S3KQa++7qKNf(r+QPm?rfJ+ig` zmsaaj--d&w4J!g0S|i&Hb2+lXy74314Kst0O|w3JK?3U`(qa{X#S&PKofenZ;d8Z=M%wl<_9W;Zev{GCD5zA5-wYT1$W&M4l&^lGrb;4+UewV-2Xv8 z|9wb3uDjB%$0*4oSALG|E4H7nRC{lOpOhiAzCs4&~|Ir<)$ zPtb4l;p>YM2uFCbh6=N-c3=@|A6?J&KkD{}c-?exu%7cD!u~vo`*Ccibm%W(f9?kd zdC7uoK78NIsxkoiLzm z;QWuy_&*^ou;EL~1@it0sN&HdY`g^@mJ4hw1{b)%aFN6XR-i0Ab|csS&~>298l8u( zz`RWcU(=nL=9H2H$wP0v-;vWB)?Z+z)$W9NN-=vlGk2V=VgA7;&t)g&!<^#6>^xsw z27ctJ`EJ7cSD?PkL;J9%IJ14g>MRdUyyQubfmt`{`M)x&%tQNpJ5kqTvnDy~Kd6-m z)NRnU;^DYTe$WH~D-Se$A+2G5q zs_SJ-g=5&osk$NE2KBOKvrR6O&euei>g*7b&P%OcL}mHt7bG?e9&;G58H6$j8Ul%R z#IWVRZ3a0Fc0(8%-w?u+&RY+5*le=x;AXjif^&2=VWKuxLTIbr|D(OU#OBh8m&Ad_CCJ!#u%4)|TEk}m=q=)O&_|7q!0*yL4WD8{vvRaA~fc;v0&ov7zO zn*Sc~XBx9VZJVrWjCMMmeBv>Vc1$M^HC0n1dRxOTq<^Dy@`-x?iv}3A4KP(2z*J$U z5-?N{y4A!_>hC~*#Ae)Cz1D#ZD&{1e|JAsddaXt_Ob&>{oisOab#@}}q?;WajbOB- zOvEO!28M_-*Mvb6gDks=ouuO*4QId2rAU+|cqz&00y;T<@+b!cwW-0i;qrOiqvNeC zlmBk_OR!dQ{A8#9lgaU=<@!knLqvI^PF^u=>OrEd)5%Xeps172rAU+~=tOer#EM1@ zoy`0nf+Ls&88B%%{iTkNw-2;}Be5G_it*=(*YRNmN86`^3xlKFB*BrM5^rzRKy&@s z_DC7?RRl4gJtcmbvu#9!oL%t6p;@LbuIE3TD_gd+{ZonDFeQF&EAih{qNdNjhg>C2 z(dWO0=OO68U6G0C}1ntKUIsR{QO3e7znq!`#<6q5&mFBbQgYelW zW*tc*Vdyn^| zd|0)+V+gpgcDqZ%B%NdK;Q0@N>h8}mM|%=^VYXF?vLe+SQ$yTwvWD0_9-^Kq?zm1% zACRP4DuEPUyuOJMXLnoYEpw+?|&K|;Lb!-k(*@~h&&57 z`T}@Um*GCGJOMWZt1Fn@3t5z44nb5|<35e~4|+_18#@$Lz_LK(Te#5^b}8=va0Jud z5Uh^(xreiqV1Tl5pFY7Mr=MV@M99rX3q&^ycUITpKK)ilFmXe$+C}E>{ZfLtc4gW7 z^iLde;?6-PO^(G2M0X2!cGu!QL+Af=MH4p!t0f6{AwdZSC>!?~Z2zvGHMeFF7Y&Af%zG6_q1^5buI2)XC~l=;FuI#Bzjtrj1mk`4(|O9 zIsLm^FCp?3Zu?4-{I11)=0T2N+8csnQgD&zW#LwW0m{aG=5~jixVvMB?WSiY(jh1AUI^`aD%?dD z?w(zY`+JUH;)dXu6kH^VEZj;k<4wy?&h71x6E_|Qi%aEEY=P)!;qKM7xX-=N5lq|= ztS+Q268$XPN-!gJ8}}OyIdS*yk0+xk-2E-wg5wi@Fx~g(Dro#0|m8ZXHvCncubA<%Nw7 zIdS*LR9)^#T_A>7xchZ6?hhTo#0|k_GpxP)R|z&*FKzFO207%!Jpf50nfzQJhFZA$ zcP;LVu5<(wHw2rktDRF;f=y=2;C|mBC+>m6kh4+ka+rmCK-c2F_((@EaYJxS3N8}E zto}p^1}NK4F22AaC+F6ys_pg}d0oJ-BOe zU%J^5OxzHxo!@>?G8C{4;zVxU@6=qE!;!9822w7!Nd*0F)6r6jI?ko!2o6BzP!XCC+^{+ zgczo9kFszN>ss8GU+)MeZU~M^!9`+}g$RdDcqwi+{3#T_Z3SV z!Nd*0F)6r6jJ9wq!2o6BzTygpoVbg};Jl>5J;uU4qHA$q+07A5+z=d-g57KWm0*Cf zabG#dAt&yVu|gCp++!`=#a)d1B}XuELvTzAb}#Q&f&t3LeU;~s6Zgn*LX;@n<1E}I zU5opwQyjs>4Z$%f*u79d2?i(!_g;sbxJQi_Vx+=7-oibyYjJ<8+!0LN5FC?&-OB=$ zV1Tl5e`}LNPTZp>;86k!_XG>~s4m9+AC6$+hTxbK>^>$;2?i(|_tg^|a^fB{QHaqB z_e2Z#=&r?m^{tLz;)dXu6zo0(ObG@k2lpoqIdPAjq#r@NKuof5kLg<6*Q{~`6E_4W zd&rg&3{W=iYp!$1iF@2+A>^~$7l_Fg?y+5q``djS!Nd*0>Y?%OHC{?EK-svzz1Sfq zZag+m$V-bBh$$BCab1l2O-C?sL$JCn$-N&)2?i(|_qF*BIdM;zibZeb-%~Bz_cwGs?a4(|O9IdMWi?&}V61QRy|t3!V7?ieMQ zb7H=kZ^L~4y6qt6>hj${v|u*^+b#9&^#Iv^r4xV^F%U;3A>vgBaAS^6!%^afcP ztD*8gmUk{)z3y))$&CYp;}@TgM?tW<-t2!di}%P0sTV_UaQXkP9@*i|y6yNnL=OLN)Fi zW;=rE4U^4I9e0n76iiUI7vFH6L(ZBGSWq(xTRqK?riNRduBQ0^plX)?8+|2ZT3G0{b1EW zB37wzA7tT{g8iq_8{A(d;0{^MuIgIcw{rhm;Gnd3^}!-mrEni?;g*6kdb@sd>r)On z?H#h3UEQ^~Z!2{K6L-xaB37+%A7bH_f;0NKxNqCwkP~;vYIaQ*9}B1!weiao;}9At&yT)$H1?#eMs3M=){M9V%kA3iqKFZYel}(Y4io-~J1SoVY_) zv+KGR_Z=%7!Nh&gVIo$ia35yjmVz_-xw!B6wnI+bA*2}$LW-ag`kQMDix)%9eXFGyv^XZX#5j#X9ueXp((HR5dkn{e-pfe@%kTvb;U5or~bN(e5 zeGZMx5V7eR`3wuW6rC~1MSk}g3CKfMwGZuLao4i~XQHS)tPpv^8IVVAwAxfSh4k}VJfP&;G zK^TT%NRp$7B1RAt11d-o6mtM`&SK6vhg~=7npSsRcisKox9++3R(1CsXO{jw&-ceR zPt!fsbX1t>`^jmyor|0TiEBNn#{(+m9;xBsKji}VSAju4J%7f-OD8`&90;s*=* zEs5)Y8By!oMQMhfy#6;(4*~Qdfj-27Ze(XLsANIEwQZ8sBbK-arWg9He*7zJUIyu* zfId*554E5h*%_@8peOhL&2v(&at}%`^xKmA|H1l1Kp!N~Pqd&L+4$SB`fcv|CmR^7 zH>X;tGuk9T_v62z z)gxBChol$!LI3=JSWSoOk$^r#ppUem8`&9cUFZkTca*2qBNn}frWv|B|E0s6s7C?% zP=P+mf^KAIv~!`~QJ4fhV%_`1^g_SmK}Rs557T(Et3FYn+mCiNvNPHzK>wG+fmV-L z`W}{E=y$Gk1QYsjjVF8PVFKNLw1<(M(ZPj&XY%|Xi_oc7zlWz6`d!KM{}H@0Ru32G z_MNdtb_Q_SjpJR*9OY^Chz0NwX@>6Ke-T#Gk-T_7j}Ykg%>zbuMn@O=-CdKQN34O5 zOfU4iuXhB~VMgikfId>7kGHC+k)6>g0lFLipw%Om!AGSR`aLB{&`0YDfIdo~Pq3gH z*%_T(==WUfa3J)EmGIH&g?_Jp{y%h>F?u4Pj~3_?E$BveMi&?Qz0;GRM=XYqNi%de z{!ND&t4{*-F#`Q03%Ze=(ba{1U$Z3W5$oY&(+mB+%N)UkK2A>p^sxeck_Fw!&ghl^ z{Vj(B9VTK)d|Y~=-_QH6!Z-w>kJpm{eVjm_Y(Y1&GZ?GmR6|z9?`Qm}iStKykl70&~cRIB3m`}<#-^HHQ&LsyuJrhF9X1C0N0RlIq`s`vw49Q8EV1dka_ zSQQ_i2E)z=RGJ1e=LeG8zX^H@3_D&7JH;}rk)6qfaZ=07`GLgt4<^7Gbc%)X32A^% zp#uHE3}{-pij4;8`+uF6QC!yf2cuNk4|yqdvbcAKa$-3P0_`GK3Sj_ThNW{OyIJu z{z&5bXJ*to`IPiRe>8dhbE=*J=u-sx3=6uEomta`{^*HGtz*Pe`P4K+f65WeUgu<8 z0_al(dWi+y$j;H3+6U- zcm0>pBNoi3r5F0+Zv30jr|VKcpC-^tE$BveW*ry$<7*w|2|Z%Xe0rLp`}d!PMNbtnWg9(!c-FD>p)qSUfLDFZ8G6{(CDTI#bU9 z^b&zS$AWHTXL9}As-{mR?tjXRT0fteX6VWLpGx&yK%Xhl=UUK>>`V^DE$C0TcFK)b zk61!4O)vDPcRGUUFthYLKra>O^DO8_b|zQRZRoiF%9@W5dc-RFtn@-ZG|>@E=w*67 zpwANM^DXE`b|#n637ypyc|WaSXWr7O-NMx)jwau2Q$cY>d+_P z!s%4Y6w|5R$OlHxWC+%l<`SvbfVEnx`-Zcf`-VA2Cj)`|4EH|~Zp#R`&&;*qZUv>f zxcUvh#fJs=nTrH&ZVI@^Aur+HA#igIT<-VoY~Z#BzACspE!>G*gzIvvR6rT zOu&Yzs`2nTiCB4$OZe409U+YpIj^QvfBnmoy3~{U>z8a%QIJ2^QjG19?*(}~3_#4R zakr{nGXz~V{2Ftof#r_^?+oKx5z~d`n(_i^(?|G>;g{iZ9Opqoa7{(hBplQyubget?^zOw7n5A zIF6n2@LuD<5L;sTF@ijpR~qHV!li4cya9UGu%sKHpZ`dlyKRJXKmUv6++G`U#s0@W z2UN?sUts&sHl$|?=VlWCwO%-0?AtSFL#n50L#jD#2q(*AAa(JD%>uV)1l$+U{|`le z^Wemc3HYYs?~Y%94;$|50=G{JxNLU_m**cCr|M(sB1@qI2B#TpoKIDCEVs@MJtG!= zRpr&tTvP5zZhINhEAi~i7uo)Y5b3e?{M(gUsXCU&x)9+y(8TuG7k}e$qQ{bx+%IL{ z(*(7I-169$aszT$x(RJ{l-d0{I9?~xW4DJMyB!`|C$-037J4k2L>A0TTatuzG{Q!e zBWy<~3>Ty`Bd=RTOV4T)+@HB?dr*nCn_hq-#q(a~{Exjrp9s%;xxM9i>OxSeJEt7d z_^>?h|Bin)(W^YygHA+Q4fqxGf5s071c19#sc8V#bO7@S9~mLwZZ6ZCi<>R<1aH@v?;_D~z#6`v3obbbEdtE=-3~)pJx2l~)m^My*_7Y2_OQErBtr zl*=1S9m9^PV7ZvBmnfGv4%+1C#A0e2SAsjS@5cUQdm$Lbfi8f4L7(HB48C(1uo^@% z2t9+u(uBX+!)8!r1{eetp+#avh`v4l%{302rM4ZKC;C0h!Rwu=O&`3)nr}X4bIH!N zxs>DO{_Xj1{@W%ux97L3FPFNGb1nru0?yz(vbFrNJA5Z7|L`j}|1Ju_zjeAJo!~pC zzA*^WQIbGr^cLOUddML^E1KMM1fP5~*uW9q`q?4BRLIT6RK3-a7-3ad#gIRF3%s|d zIQ;j4Kc93d_aWI&y7b)4hd0)P_x4o|c^=4*M3y-HysxTn4FF-B|LXSqxBux#D75X@ z%^1Kh!g3`@!LZ#F&EFaCFuucP?A`?BmTNyx^LMs72>bSYR3Tzwd6N|@J-qf^r~NZJP4Lf;Hdnn|c81&jayZZ;*2|cTB|K=sm>0g29p63o3Qx1&^-Bhc*0skMVyFj*3zYj)oyG z)P3)20a|1xmV052PG&;TGn^3EA)m)Fe-38=80W)<&BQtR_Wbu)|EWY#gc9#Jx0Uz- zfYb*WFR_MOO1!T{i9sopkRIdxS)#-sRfZ`P>qC8iF4qEcxuB`c4!)V&^XotqS#W0x zZ1Mgr4jVcm*_er3eO$O35Hoj-#3XDk#Qyrfhk=Idj4Wo#iyZdt`5$oqFAXy&!Z087 zvJ5ltJd$^Q>h0t3VHxIw$zqscDGbvad7;qc(2J?i; zlK~TqpPOhfj<`UxEQf|!Z2BWQ#qNgA*9u)#K%qT6!5<#u2p$9Uv-K$m@|MY1taC>@-97>;Etl03q}_dI6x%7U&Bs=tg#CQy2P21&;Exdc={V zIq8M|(H)LpLZ7P_0{R?*zR-ehWM^{N!7>x8fAm*}1EEKpIhvbZ=pXy%UqY+T(~AIo zu0UU8K{v89Iq$cif9#)s@i-8A#G#{k>4pA@fBq$eK3^{e^mzh(u?5}8&Sb-5LI1>$ ze}~W`P94opGju=x6+%BnF9G!V0)2@E-N?>NICb=?fBq$e9&zgEl=MRXbhqQF^z{XL zDWIPs(3e`!jqFU0J}j$$>c{^>=n=PSE=Vu*&n6{7U#OP>`T~Kz%z|!YXL68XLH}%r z!$HH>BTgMHOfU4$+5d6C7dp%$y&TXN3iRa`bR#>nl?(lI|NKj6_2xM#PaQ2vGj#v_ zO9*|jUIFNf1o{dKx{-|mqhs~o)OD1n)gw+Fp&w0WZ~7bm{7VRZiCziliv{{h3%Ze= znVSIp#U$tvr;e7S7y57g^DiOvrFs>hFA?agEa*mdW@{JvZ_iJH9&zeuX_}$`#}UjH zW|>|M=t~9qY74rNo!Q2P{zaj~fesUK>IiFi>2#Pc9&`i~`f`0Lpf3~Xr&`dB?98?a z&>8;@D>|V^oH|;bUg*E`&%cBYvqG-{^yLD5jRoDv&TQvG|DAvS#p6Ke5vPt;q!;?{ z{q@ff`bvEopsx_t~Ss`Ntt!}X3}LSL;<2lQ0}{d5buk)3%|0(5`=7h1h-$CTG@ zdMlS{npaq8&Q^jiH-Et0IhMy~_( zQw91u3%Ze=*~x|eC$9gvElk9zqcv%U?(hE&t$vz56VTTP^fN8!Ms{Xr7y6%@B|(ok zb#z*Kq5s+6{~bbKtJeeiX##z{1>MNb?2-W8-~SytOvI_9wdsZarN93>gnqg{3((gJ z^s_AJMs_AcG}fT^OMm}&2tDG|(dp@h{#CY9+-zaa&}Res=>q+13%Ze=+0BLimB0Tx zgdTC~=!`T&_xFE?R$r&j0rWEj`Z*SKBRjLZKyQdo#G(Ge^?ybpN2Z8X$zQj0B-1O-)aSyq>%_F@TBbF!aXF~t&|lARI1qcV?ah6> z*X#2D{Y-&=o(0{=Mv&2g{te>4PU9VqI(M`_z0kk8+!0LZXX*0+ zeZ4?G--2#r#}VeXn~`tcaX8TG5$BH1N-y+pM>&EC{cODf(9aU+8!YHXHgVa|zrEDq zKdLy8pEzmbw(2eYPbr5BpV0)c+91>MMwb97^k zZ2y)R|BpwVJKB_H=*jW_&H55R-z3m4v7j5-@%k?GzbD53<5B01Hm4W*-;?A27wSs^ zeX~Hn)Ping#~UO-PmKS^qs|>&m|o~VCCC3S(w71Hg#!ID3%ZdV=V07&n4c2k|M94E zM;E0R`ahE6|6BA{K)*c+_h8#p#9qb8`Iu5`6`rUo6nCu%H{+@r2d%pA+N%@u>SdFG(--e08qa{g>qU|K<8BK;J6R zud<*U+3|$c^j{L=|M94&Q(m54=>JZR|6ieZ0Q%(ueTN0z$c`thrvE!J{vVHeK;;!_ zhMpY%zf$i6^eY7VP7At`9Zy(I|4(B4KOS|5=auP&{-5Od|2Dk~(61EeyDaENc06G< z{XdEE|9I4D`nI$}ud0&c|J(I$K;I_NcU#bn?6|v{UR5Q=|KogMWU41sZci`tSaSUT zD!m8Lw+r+=7IY&!4qR3XQ#FJHDH>4lz|9RJ^?_W}A&fxgdz zZe+*XxX?2b>&33u)gLZ1!8I)gHMR6S{8_U!)OM*P{dWS zW4oHFu4<^7surGoiv_)R8!i(h8=_RjpMU)mF7r z?NtYLl!=^CO6!Orl%>~(1n|>^bDm-6mBrmr3yFd=rW~eD?LZy#t}VF;RX!s zkH$V@?AOJkEj}Bi@a;G2*X?VX!p>5pdKn!<1AG*+*ogR<)hTX6g;L)#ZZc2 z3NIIop;W^ZEE}m9N;yn1N0U+NVM-O|ffs}+Rha^>2vf3|0xt4s^Q#FkkXAQ)m85>r8`q zN>8TLQAdX$^ zm{iE5BhhMyNkvR*iB>#J8pxzpXw}1{K}^a)D<38eW>PL%{V?fxCbdQ{5GI|#q&Dak z!lWTgYKvYXOd86hcIY+2q!XFc9=%AIG>l0d(5r+=!V{q{Od7|e?&#IRr140~tb$%HOq#%? zs_6B?q=`(*MlTp9oy4SS=oQ1HNldDaUNTIY%%mFVHN&JSOsa`qG)$Vxq*~}z!=#g$ zR2#i)m^6(^bm^6z?s2!(-No7nz?N|^d&1Mp6$HFja4wFzj7KKT3nS|Q0I82(yB-D;2VbXji zp>`|{lTKk0YR9rLX#tZ^JC=t@3z>x4u_8=b#3a;?m0{9iCZTq$3X_&F3AJN&n6#8h zs2!(0#0;CZTqm5hkr>5^Be~FzHk# zp>~`ZCaqx-YRCF8=`<#xcAOO^tz{Bw$Jt@h=}bcHI44XxgGs0z=Y~n^kc9m{`n)jd zOeUdroF68wXA)}1hA`RJu_a8}$RyN`i^HS~n1tGKNtm>WNvIu{hDn>5gxYagm~AU zNf$8*wd3+IX$zB3JFW(y`2uuUny;K1DCk3-uzsSTE5_^)kI& zuh1*?D!p2ts@Le#^jdwoK0~k5XX^F(EPb{cI_v(H68htHiEB~9Ua2p(7b>J%ho2>ZO zXi01QZ?fXK1|lUG=KXK75;n8{o2>X1<$seE-xB|utoWAr-(C4zD^KxE$K%sFcqI+?xZ5vBiep;a>(+wapMmFqb;_>FWV~uRy=vf^KBTfy*c^ z(A#WsI1qZo3Eh2Zf-Z}tZ8IU5i>2eES906>`@hG_N^T}sa`T{p^3T3mE4gj`{jcMd zzmkjKk#SH`aJ^StqV1WGO_vy-=n_7-y#j7r^h$2qr^RyP)EraIP_V3Jw0xRYa*e9+ z(01JeQ42*BmP@?4*9gAk*IMhFGPE=m=@%!_BfjiGB|X0Kg~55i$)!J#K%b$mvE_T@ zkd#Q@1;0@KHvzr35{(E;%v`yJb?2(#oM1098wVv_H;)!4m%UvQ%sv0c z_@QoJM5Nh(#bLnG1DKgt?c?K&ydmgFN_YIMc&j&BAR2#{tU0{jx*OCM4px`L*fAebn)eVB)S?T@}YY+|@1IMsS>G z%`JN$b)G{`+!1HZ_oo^6za7EEon1o}XL-15Sh$VgIBSE2yJLTcoVX(nnqQY*+#Tphr6bQ+X#+#O~C!5Lr&Zgr_8TUFYZo@9Kpn0y_PCAS9>{vb_OKIE^sqU|hw9SV zKmRaM&X;Gght+B{eh&Ir=9S%2ogWZL6HOnh@kvcTjXRPJ2A1fL^Dp& zUE+>*8f>D@m*1EMyHYBRL&=Hz-!fHA|7OgF*y$4bgAtrnMM$Nc?(#?yaBNlZ9#7&P zF25-axG7X{ckSZ{ra#oGtBR|8^{=jl+X&7ADBF*_u5`$0aI7DDmvN3%*;PWdZ%#Ar zuN}d}UAvwtuHoUXXW=%2(PuiiyLENQi5ts!sUBx}OL}p4OWgmC$7j`7#Wg+L^)1{+ za27z>_U`t(B-{~a%5P0C?(XjXZ`!+V165qh!`;BbZ3GjPjl28qB-{}P%5O_A?!3m1 zVB)UVP!-qqa5uDY8^KvMT-CZ9r66igXzWH<2Xk!aW`zD zitBl}n^?Gw;H=s%?jC121S#=X||HUCE?uf@$-j!b5y=FUtiMz=Ws<@$t z`v?oS5uC+&9=lrU=X+h}kP~;rGb`^-FYewg9l^xiw7Dua&j@IU9jjqSZ3Ji4cX9Wg z=8zM2!~-kuNi*(u9Kpohtc5CW?Ag18h1&?uYT)AT)5sww?m8ICn&y9`YG{p5?oBW5 zK3g2Y#C=5SCqfv(SsY(me$wYh%ZGi@V@nM=)_8 znWKu2@Nnl?xQ*bfCJDHIbjXQ2;@i~Hy@hn%=0Zd`vj&A2~t1QT~o8&!Oy zhr5l1+X&9$tl4T$jycjHC+>*r)*neP?qjZW1QT~|TUFfB!`;@xZ3JgEPr&`QLr&Zg zcdb8~UfjoybOaN3>vpQRm4~~Xh1&?uYT@EO_9BOzxFarFe=NPY`&V%U6L*{TsyN5P z-QL1&1ZN%T;_g4#At&yLTh<>>Gww$n!NiS~n&Mm!cLxi%5uC*}8q2@?|KgAncf=Lz zPox+3ajP7`#NF;FRovRceUydU2+nHd;y&&khn%>Z=OE@q@{uaX8lOCwUfct^If9A1 zeMeQ?#>3sw!fga+0hDd;0Sg^+;*Pjn{i!tL{=FlZxI1)G#b!aHAd$_w;xQ*bfHZJbMD;#p-j<{3(ne^f=s_h6S?oM4*aR(1~ zR|~fhoYmIFT{OZWC+?hfc=9I6N2+#~pFEpp+|M|IiMw+*ReY3(yPJjE2+jg1+fN2& zIpoBh+a9Zu9`5!Q?&s2rd*C`pFmZS3u8KQ)xVu}pjo>VRGPq?Obl@W(XJmg^Vq~8a z48+$5equeTNo_)+y62%6&cg>YaQ1{ugZ%aX$thMRdP^RE{eQCA$?}?tsT!%C)KndZ zKVrc;F2k^7AaDoy`#&Z}z#WACe<<>s2Ry5-_@)~C^!NsR*l_QJ(p*fLo&xT1$V-*p z5xY(|a5d68*W)ChJq}wq4~gI1rCmGK#C}Np?uNJyK;gJ_fO|0Sf1{bFN8lce{y)H- z`6DRR7i%Z}8y^<#!Se-oNebMvkr%iJZxP%jW+%f8*fLlc7^br0VLM$`-Xo}g_0ffp zMu{8>Kgawd|C=m`4(bOW+&V)BBs$(GFTzGS`%Z-@wUp&&cXcv zZF`o?>{C!$(z|P3ey^Eb`}WA2-L>bevc6sO_?lJPZ&uG)1tw#U9(~IS3d(*xV`-n+ z`MnCrqCAc@tA2_d_KgaT>fPyGWr}8K4xr^=y%rF2RIk`Yv3~f98y8g7MJpGMUA%gE zj1`El=a5(pi4zvhTe`B$N>+-xufiMp4{`PX97H%|=K1VYm5suJ{uM)uj1|*gT%QP} zMH_k?N`WoIw1}2rXtC8Ys4bvWZ})!xCw#DFn1(lW8G3HuE(@@!h!vHD)LF1sA&LoD?oP0C}O-iBm0ncD{R8@W)(*Ykg@NWh9{1m{Y6&}`5fX_Fr@OqR-u&g{xRkd2- z*x|Io`9MaAUGavwhWY#d=BqMz0b60%q}bGGXKxnz1ZC_W3#4|i%uQ@;t zldZ30{ocvSX0D;Xm!XBK9X^ec*OPcJBav%2WFc^e^Zpm!Z|}in8%H+P*Z|eHudN7( zEx5z|`JYEjkM9QlMpufYR=%8ljvQ{R++Sg{49ED#0;wUVYt;lf;$9=FAuH>}{0R3N z(aCZzZ!Gn%8;@ZG!*Z_?qriprVo3`3>W#c8s}aiu)RJHPV0T?&bT()4%tdbQazc!ZgYq50wqeeSYS&Jdn zA4}Og_#hSa7>LZMLB>*}E_TRSF}kC~%|^Q2s$R_yboKCSEc|+Ssu)B`W2u?Bl^;uu zKFQI8jZ7ndqe&}O&l*d)dcY!daBD0z+OPj!F|wT@C)sLaZ&~pSN^+Kk^sS>m3xN22 zYaWhxvu~{z*|*Ly1~uL69JqA7)P3vBq`q|w=fA9!t0J6x%p%LVmFZhQ+jae1z_FZr z%vNw=8?rWqb4%Yk#=rk_Z7{*@U;=Z$722$ z*2?|>tghXB(@`$)v3&jkfv-yeTx#W5)_-;z>w>=Z|8j;Soyb@}{^?eK`qptHovd{G#{Lwd1*Q-|k6JDGX1YfX_N=~j zoFD)3;L_*G#%$B7Z(sLh;F8#e`-X#oO-Men7+sOmXqAUg^hD$Q_^&62ed~3&p-Y}% zj6L1S_jq`_(Zw?3`#YjEcy?`qE{aVoU(Ck322tbZA}c*?U4(~?$NtB_!@Tab`+&(` zgW9SZ<8KESdf2%sJdC{~CHeZF8+5O=QgyF2o$i%8n8-ljPG}-<&y9dPp}>Y~y4RM| zhva~R1$V+Ufx96ETtHCB}(@? z@qk#%lgjQjVxu{vqAL3v>ZE&}__agMy3qrBHq3tS-KxITz19f()L8iS@KiB~lDgLo zxs~r;PvZE8PUD$@?MxGYKV&Oa-|AjnJzx>2Yj>}cl*5TlO~9!kZjEzv7jmn6oirdI z_q*4gW*5w0T$&O!b-gg`UN3}8*H7KOZb<50C*2BRY(q9hIQOKNE$8;S*MAQEn(d=i zDJOCM!!~3~3g?#Yb#h}-YfCWZ?QX`r9g@bpm!o^dIN0u9rzUo$U2|RFlW!5=m!<$NwQ@4+KON)JpnDB==>8uJdfDxu$(@4ur${Z6scW`B=&f&G zj+ntZSq(&1e>LT7hj<1&5X5eOCEe?k+nlU)`=q*pR`)vP z2Zs%Po@~sJvc7$HqJc|d3+~i>hnP)BFJv(zbPBYlu2WBTva)-<-j7i^qYfj7h;2Dn zIm8-r@)AcK(pgP?8d+H{E{$-nslT(_%j;Xe-PPtA2(;?O$<@Jy^uW#L(+v8;y_q3&g zdv^-lQj4eg_%Ic;I@{`U+@a;tAG-WdS#`_?{Hm}f9Ut0^-5_<}dTUbOI)nFr)46v?IQNW>mUDZ3Yv#|Z9{?Py4Vl6D58IG^DV$sS z))~u0t$o3Ox4Rkec1RlV-iy8!gJ8RFU7y&u&UiwA?~4Nd8w+@bnYz~N_m7D#@RF(m ze18hyQY%YX|Jk?h5Bk>N6v_X=h_@>cqRB1s;~)E#-?vuB(Mr=6)VHr0=0zoDyh}ov zpq2RXPmg#8JP^cgXC-~>OvXPw?I?$Ce}q4zY=I$V;Po|vZ>DeMc+%=yXZrCkPa=Ju zY|M$5`t~iv1}=#$xHGZ-Yhhp$($^oJBd0xeo$0Utcyidc-k=v?>N=b44W_Qi_f0xe z*O@;#qBM+`*A$WK6jRqy&i`38_D6VF=^)F)yzX^hkIij?!>Ssk^T35wvas7k1Z;QY^3*+B}?semUv$`bUeob&6 zOo3Y}@+^P+cQEK)Yha9Gda7ZnW;|@+=NwM=nlN=On=jV#q_TU3)Kq77Ar)1bqb93+ zExXAfXWb}hsFXQ6yIVD^8G^1AevO4+4^NfU-RsM_mG53>bNoZ6@yx(>rn!H#td(kL zb+4`-un5$(yVuzdIh@$k1e_Y;GIK|FA-B5M+5Y|i2aWgAp#7>+#Nc?t$nNzG!_;)I zZ@{G+rtV%}PU>Fg%mG$9_rVC~p0m+%Zm$jb?GZn12OO&nnZx-H+mH`ZIJdMRb6yv# zd=RuDbyKw=b(}VYi+ffZGMDpT!u=ou?p&<@26Zv}Cs3;)&z$|J3wJKZKZN^n3b<@{ z(7nxFFK|CLb&;iTG&Ig;JAA5YWVvO|9TMVu`T>Vg{@R0%VS-La5@xy{@efv=&{oGZR?3jyq8g{RmCLle$wI#BUb=YYq7}>5 z^l#Urce}nt^4A&Nxf;f_HC^%FLjE_4|DBTByi;<2-MrLD%sbzo|A*#1(m(syN;OVx zUT6jTKzZA|^Ivl8L-Uf8EEUi-|BPrW)ikv& zp#p3PO>J8)oaJz$Ey+nvuiK|hJzJ8Twv;8zh1Uk;XqlR@i99ZwEz4lsFC8%t+*4-S z5_VL3%Ft0YO4(8EnG|+ZbHEUlU{T#9VFQe?QRN83*aCB6BMj@M%!nx)XKk=WNwB47 zPg#(w%*^5_(igiSNv!@=vGYT*^UD!CClEU)5G$r##QhJRm;0tItj1PlFz)G&GVOv; z>;jm!v88Od^J>qO(6l0K@hM5dimJ2T495Cwl(0piFf8Pv6*7i3mbHYP910VIE`Bdb z*gzv}R5`+yhQgM@pg%iA6)=QIF z-=4&Js44NDVTmWPhT|onUKO&IQeDpY2TS#hZLhP?{e$OgPd z;caZpZAPLbl)%vPzdOY2Q%(ebGwen#lTF~R@b`a(aC80fSqqHM0+1nGv!20b)PlQ$ z@sChz!X+D7bZMLbm&6v_71uk&ggXpbjIN1r{rOJ_x3xb6YJnk83b?$R(2Rd9xGVkf zkH>~^$wszdG)aKl#D%-k-~a6q6Yg+iF}fzeUFpw%Lbz@GQAP`lGE%^e6E53S8}74? zfe4pu%n+l#{Xj0`3?#O!yDIJw6YdCPF}nJ2rIxJfj;yRDUq{rERfzuu6T|u=N#6DQ ze_VnOtCp-<2`;Q9Kc}cA9QnY#SNZE-KeLv=F=J{w9!KY$p+bCbbXWvU{qiGe2O;H^(Ox6o|vsUsYv(yuNbaXG>DHzmXuJRbg~laN&eZdrm9;aRMV8Q0$5PjWH~GdHDFQSeuSEVnoxVz z6gk9%dJM9dyCxyVSXE52#-IOtR^5lJV_{Wu4T@uuPIl?F0}~Xw!m9O*FV!lq8j!c-qVW0&614%dLsWh}5wD`!%uD)WkNXCK5Vcx%@-wpQQmpvbbDp0Z< zsqv?C{1eppDobrSOz&gkG!u@uUql2O(f-2WHaRD)_(L@}R%_-~lC z8f{mBhFuJD#vzCJtVm+hQ~JHa2+ttvG92Pdh1lGg)`J~YOVy%u@gnSvNB7jfaQVEF zAWS%P(X4qTL01qkC*!ul9Os`bwf2Gec)!uAhDOWSnt+&JthR2eL&G+!1F|@6Kid~H zP3s=Dvzo%(sYl5HaSRehB@7ZctR$&4P3wLM$k{Z-AZZsKa2pH~CQ749GLpz3;mlqE zncp;(nOJ0N%$=-3LgVN`f>gRQ*9Dx)&@$1jPW0~+XsSAfEmKo;vl;XFc)i4K_DrsS z2R@d~vJY6)w;wDTjFRXyV(K&f_=jm9ssaFd?81%(pirGXFp+cO-4^* z8$H)Q19zx81H{dc#i=~v#%KN0&RT^PzsSF=rIl)DmuL-DEocH*t_>iRTm zvg~}DQ+@j(xyIfkwh*6vwgY25h&v&RgSY@BsP1QT{VPz;ARg#n({iM0ZzIO~jx1jg zFJms`;p6?$=40>%eCpee@HKczY~ekphfQ3CcsnCYs=oUir~R)=KOBPUZ8jM=b<;eE z*DuL-p0elc2;`t~GQ>l1_l3DDsydzqoSQ?6z42o_{BSXepZA2aeBy)3LmfAJrm5@t*T~ zI#&4{Y6VQ|+mC4uY+`xOc}r|!GxF;L{*Jf%a{T>Y-r$YqtAw0z{)?ik0tqTfj&wrj zXWEd=$S;u7)HPBt@}o&eE>O=u-ezG&egTVLHOcws*u+(4q=M?p@#BAFsw1y*);L6TQ(0RU?H^ZOC<~SPJD)NyoauU;pxeYEaFIDCP}r`$jXq@&k-S~jZax2C-n*O?!RIr zsjZOH#y^4!LQbuYN)d880(oJW3u=0xS{;Rl#(Tk^xtMa26JN|Fm@eq!VFI_=*iL7D z_I_k%K~^k(itAX8xV0j#V<2uG+PPHXt`l(`6U9+OihIU3!I#8y+T=@@Y4(R6zV34e<2T=jVxBM4*mtR^;I)>IE{y{d*09upFpum=6^zb7KjV{{q7BT$_|~f zoU%oBLrqF?iA|ft9G+;69eq70z>ZFd`8VAo_UIa6k4^7b_Q==>F6xSt))nK!vd1Ql ze`t@M#vYXAy;z2A@Pm`>2k~bxqvFtgJA*c+uvu0m))RqR_Vl zqXUzRO$?cz9^_mq%zNP%zL-Ol-7NkOn5lhjGyLD^QIMis#X&az0eKX3El}TM&EW}* zSBxCbAS3Y|crNt)zkr3W@{#tlzEBnSViEK(k66ZC8Msp(gas2wm`aZE&Ae z&?CQ~VD_vE>%!yy(B&f5e+)#0dyc|pZt9Cp#d|TIl;1@&kr(Y3@+Op@l+8ux|ALUY z;y1PA1odxxSk8LUjo?CO?H66h7rlf$EaZMt$T7cVA|qcK;vb>3TuO_z?Ddgk5X@T| z3+8?in78z`xK#fuxg1)%C11EmBe+HOf3-R$1!h?k-y-WD$is5lhjOZ_&yXvy3h~Ty z+_0zG&g|7|Htwe>znp5R8si5R4=NmASX4M}Fr1=h;qzcgK}X?@V%zc;sKsLQgx#Ss zt1KzngH6Lvv)pQ^qCKdmJ{@VKxG!$!&|QKB#g*i3HeVu}$tP}p@fasB70T9Us^U%* zQG~~{S6rdmswNZ~T{w39(D6e@jWoz!1Q$vYjV_k{-_mGl(OziujA*o1G$Ng?(cc}E z)M!;9@&qet)O-rwu-YYUoV?VinqH@hJ5$7Qcm{pNHNy5YXoLb9?v>9AB1N>hFOCFOJ)9JX>)2J@|gX}$ZEW87(P)pmpbJiD*=vUj+TqQ z^VHF%k?UPh+6Q+w%`Uf*!-J-3gC*Gt>&uR?;m4MOb)J7WU7qT0SeMQ0(>t%AtY1Z0 zOILpx*Z+cg7#ofGNheCDUHD?Qet1K{%k~CYF&V@SP+97F)eYZr@o=E3-dr#T(C}h{ z%xU>InURUF&YuECXlODxjxQ>ZM_S~mg34Gb6X77+Qd(?W{cb}g93VP>NxC_Sv+9;CZM4!tg{vjfBq2oS1 zS$YQOGuh}<< z8LQe`W^D2Cze7F6>K~jh|_!-vd9O^m6)aZ&1yeczHCki4E9k-w9prDsE)u+nw zHt2Y2Wvtym`#Or3ZU5NePA_ZCJm`157xPIqzl!UhVKtxOR`Ytu2%}xq4VgHWKPF;a zeAOsxT|xJecN`nAMyqI8 zck$%g^Vc2z{HK4S%|weGnEx1a3`n6xYfpCc76fo3SjfqcwZ|4Wz*PVf5L zAui@!9RCw@VFcz~82zI{u270fO_0D-yFz{J_$%ivLf&47UBZ z!={ChN14)*?;YY`-eZSlufsXus(~(!G>T*yv_x#mi)1AUuhE4nO9PWjHaDjeO^u(~~$%?y6 zWrj^eVGW!1-Rq#h>GxPWeUt2_hK@H?#@Y?EucKJ|nuZQ{4x8FAPjJT#^GOeKjYeK} zGXqUGvtGJd$soPnHD$=e9%Nud4|2`>Gk%2gO^7}E&mYuWx2 z^YJM#OGN5gj(^w>9WT9JT%DoE$m|aBwK2w}Dsq=K?}A~K9x`g&c+Aji7Cs9!lp+n# zwSN55lf(w-)S_KzfSzPMLXWpg8X(eH4bXl+{^`-N0Xl=c%_S7bYO3iZe4+u`@5etw zZPw}GGF>}X>ERXE2G?p)8;Pzx3I3EJy6k_zu^M$*Q?w1bJR-Vm6J1DW>*B}%LS5Ds zG7ri4qI1i<`#RbGV3o|7++Jp$!-RSFsEX@?=m>QgUN~-i@sPm?=@tzx^vdXv$P>M; zyEI9!(~5RLug68N9ikWMEWNIK-l1a|omq%HnYLaMbieLLC+|M^%6dIgp3{V&`^bvx z#h^R%8aEy)4fevIPXm}0bibbC|IoczP-}~>fkC4eMI`I(|R|sPgQE0puqkc-nNcr55?}(y0rx$I9I**Dv+eIDH zS?b)t@o#7bmd{y*$P=^Gkrv^GE1bMk=WL#R$Yox%2%{^mjUur+MH3e6WTl>+Uyi>FE-!M1XX9$`Dg-{@0MIOWe znXzn$rg$H%Q~%A=FnC38P`8*(MhG{v{pWz?cr!v6;|^FxdMe$*_0O=fGMu}KbDVBe zALFYRjA<(bmch>hpd#m4!kzJ-kUPV<53)j-t$pZLCLFG>_%bQa*NFRHLYWM-HPfr5 z$%jU(EAA{lcXE~y)mvym&XK3_aQ%J1IFXq$;Ek)7GdZM_`sGtkG5H+lTZL2U)eZ8My_ z^rX!UX>=7N&wHMf-(x1$(F%TbHgdagmf6T{k2>5L#C?{nPFr%vYy_=NB`RNq_gWBz zH5<9zU;pxYYWmg@dX)^5;9ILIW909kg9^2Ydxo>CuBdpxot=uvj zxt;spII=%JVq|{+^Pga3uQ)bT=Ny0NOMFm_jO-8e0~e0$Pe?Jc7dQu|37iw8t#M`} zKBKdR(GbJP4dS+iKs2!ju16l$-XSLZ`5Kzosxi|D)`(?i2fJXsZGNKWfe%1KHHSp0 ziTGEjW-cVD-P^w0iVwmUE(ddj%Lx%&4i?&6$WMKCV=MEtcb-#-Jmz8E$Y%11I(OD` zWH5MiGA_n7_ql7{uZz(>@tL?0rezh6U}Wk=5JfQ~@rF8gp6ZBV3w@=2K&FXkGafK) zMujZLcoDgivnYM1-~M|_&p-=3fq61%E54XdkldBwAj!v4@H={$YzP68Wd=!ENy$tE zoy&dj6$aQ1^j*SPTIjp{_TNLoc+n>Pl=Sszp`WU_hg96!9tx|4zU%jnj$?rIfSx1s zBj`A%GS&`t6l>q@xBs4wY@yrJ4o-MLn#8-g|DCmbxT)orq->$@=KP0E*YJo6d^h4> zK?U~WML%uq{Ck+*YIN@Y5M07KT>v|LD01-&oWEYhUW zxo4@AD9@4i- z@NbaPw;7kJ(AGw8%VpufF%yQ4ML1X@5+6VgWl76(FV}xUd2D&kEINRe=WW&|v^)o- zgWR`pjM6t$T&Jok3)@Hs z6`ASoTVPKSzA9fkVyMdcBE(JgtD?$Hq6+CORqku=&{3813XvzcE|fK4GTu<-zOhbT zs`3cN!romB{gpmYaaFKi8ma`*ypKePD081b|MUES%A8en2b6hNl(|EcA)T!Z$3J0t zQknA$k;gnf8Cgxa$h7wUR!&|j^QgX6?sLVo_ST9k(?HdOGAROJ??WWTiCXtx=7^+P zXBXWLwcZl7ZWpykXQ_3+zyHZI2i4k8h&*QT5Lt~{Gw_Kry8myE465~*#yXkl6j6ot zS6nTx3Jo7Ma9A;WMhRBF4mp%1Iz7PgPiS)LbWYLr(CH=7>3Y$Lbe2vJ>~ZL*)5b#N zF=sW9)z;|&SAXjCxW)j%nb$UYItcp!uET7(7ah~^F zsPvFO{t0W;c}4Jc{f4M?tEfaeOQnYfJEmYUZ7M{bMz%__#{JMzCoffcQr{wjEUa)6ed!uk92J4Qy)JUr6K$xDr%);G#o1;k%(thh#qzoLx774Ekofl|cd9y!yIL~S+} z!NK)wqRq{s4e2ax9=Xq>uq=eF6KiVi}VcSM5P(KHWY)c`<`bK%Y3rwhzpm5xY>oq>2m35R7i>vPgenO3m*$q9}|3)|7PrCgdBO23i=*(;c8o)zmc-H5Tp%Vw2fkEX$&RprIlu>x2sGs8e zFQ{#b0gY|?Av715f|_D{s4_}tYO*J5wiaa1^IswR83<)eWFBY1ZMfuR{430xkErnR zePzs^ulgDsy?XS?gFF5Dw&AdRZ-9B61kQQ@r;s+H@}sMXAlJu zNR@dS^M6Z2I#>&RnzUDNu+u7I>js+KEeAV<_^-vC4t5mtI0tZ8fnkqB8hOJSzBgq& zl{BzyD>liD%rAGKhNU2U}V$ zxT~K}uku@bkc-4Kp7Hm;jE=zk495RKJj26$%(2s}2^YcqOfk&E7|fVx%+Ij>qY1|t z%zIN}eg^&@N;8<7oPRQAtq$gw!HIFBF;SQ?{|~ri?gX{Eq0f&~UChrm5X|FJV3twH zv;O+`I8_E}27bTcDKfJM6r&LD6j`5=9z6>B<@GK%@QYE%$l+nw_hYcLMj_Ac1sK+4 zPY{Pxn~Uy5v+`aH84-NBQ$``AvqmA$`sZIgI*vjvDMTK##)zz@T{#(_@ZsmMTf&mT zMshkHH)dRnuUJc)UnNTiSCX9o$p-HY_gvJeR*FB>HJz54Vl?Uz@6?4F&}eS$>V z$>%3I+!^=XtPjb`0NTkzrk$*io|S1QiNb0ppZ~-`!MN{1JvTg-G`BL=4t111=>>oP zzh`Z>lO1UX=MbqhOfSp@5?1nYrjoywvYmY4B4lFy9T!o5U%1_>zh0Zuv-9NHP}*uI zUw9K-*yfB+QGb0ne-}981x~8C?~5&LX{OD&=HXXbi8N^^U&Q*C!8|?!^NZO36tp?D zP6M^-e_-3+UCb{s{z=ReQ(&Hkyr`xxvi)b(nW)N;!kN#rI5SloV@rTtTX%R}Z~p8& zv^k~awmDeG892s0D*Y*#P>{@iUh?;Uc*5B3Tv&8B+MN%i-ML%Z9nx9t&P$^mkZgA@ zEkvFpNyZm_=OnzL&`YN~d8yEIIG*F}96`|cV8s3;j7}$e?wI+D!L1* zyf3QUC906lQsw354m_%ISt0V6oeIcmRAHocn&{3nUg~-#2WPGu3*c(ElSJ?l% z-u41FbejPumNxINxJKOOKc;x>gptLC<4jbZ3pErY>b!EUBZ}%=T*RUC=c3L-q7LaS zbzXVcp`$vN7b1_DW+1CkM-sAG0K8f|_DNt7bm zyz0;YLwCER=s{@nnP~H%XhS+nn^(F19U7e4Tv3QTZS2AkZC>@qzo9lSVT9^EObXiE z{atms+PvoXe;z_=b7dj&MwxIYS-Ri z{~uNXI@{$%*c6~Y6^$McjYwx{^u{5Fjv8H6h&-)rjU+Vm2KWC3GGYw5UCVe#d*(%` z>m)NID(lm?4~Bi^PmghjQ6~_QadW)UXudhpVa|}-xyS<4-iz4;$D3=MtlW({T@O*k zJp{*)%HYV#7sjRdBo06RWn3f1klPa)vvhOG1ww9^rKdQqUr?!#8%0u_=<}8z|L{aI zam<(tfjg z&ME+91MFMzW+4oL!TN8Hcepd;_Aoc*`o{()9j^6#nUp6Qz5SdclM#_4c}a_@0?=q( z#m(LC(sH9@=}}PF%t$) z7+k5HL`xxuvLx{K&XbPjoKaq-A3>v`r7$0!7;nEXHSgK%fHNsg>XNRkh4rF-*w0T-0beo9(kfX;DK9&-9^qyGpkPDhSialT-6#2d5($>_sZRA+UTTsL{rOKQ<+fO?yCLbGK++be0QRAO|6xdS zyNY$gk2P?-!lGe*vG9lM17xr;l;JNyxXQNhhc^c~VPT{80etdhS@^@(kc$?cl4xOZ z(+@HJ3BnO7m*fXlUGo_}m_f?vqsGGJqzEn_A^sV1;WxO&o7Bs4_K!{m7Z&Ep(Y59y z>d#7ivZ*znxwS?xe{`KK&0y~MQ+$v}^D%S(3pGDE3Nz0C1ei0gC+6B$Uiztv`Qy5R zd3p-WGW+_t7xJ)zPFH21=JBj^^jL{uV!krgG{mL7d*LRe{QPo95131i#1KE&^!o?! zqZHOic+|&x97!AyUr~f;ss2q284&$_7?qI+JxON`h(Gqne;z^(h<7kgJCgCm=5{XL zknxGX|JBRO{(43o*)}vn)iEc=P4qXh)8-();vGpQ_S8Bj#y>gD(T)A~4*jSqK8_gC zkvwWTk_uUokzjJSMl7HB@o!J78R)hrGmo=KDC=dP`tffsFW1XXD3IdJm#e{W|6re9 z{qoDdbb_7sgTP^}mwh_J(UP6^3%XQBn&`AkEAA^5x3!1D>a;)g+dof3cG|6YDWQqV z!PaM1#?~Hp%hsO_a&%;;-Gz2=u3M0*{2Al_tnbrV-6BRGcD9R|dRYq}!A<4tm`LK2oe9=#N?MBT1EGRBBC$v%xF^Vk1EOYUzGV|18gpG1;z{Nq~k zy5=#i%aQEkNHX+)31rtF5#yd+l6@3O_E8eq4b5WQi$t;yqsYE!kVJN4(-?Pdlk9^e zGO0>mVEh|)ug@X3I(Xz258;Cu#NWT*_}4YSO-*9lrc2)MC-L^lu1_Mnxp9me5=r)6 zB-zJFWVbYmac3sU-i;#roxlDc7VfPLW85rAvUeiMh9yb6twD^t@<{e}B-za<4oPHp)QNE`GNrv9Nj5!+ z?9SS;!q+^q*CNT-{<~gs7sR~kk-Zv8#`}+5vb$@>c<~QGz7j?Dhs-3hduqgZqZ7$q zjwHkWhXiT&R*&&s9+JHjNwzeJ?7nI--n~n*7bD4TOd`8KJH|_HN%le{*{4Zl4^)kr zdyzr*d=iLb7Kg$vFOZr9B*v@lpbkJrhOtC(Qp7$R5dz@#YMY9f~9?NFsYQ zBgXqFNcMCjS#taLSS-eSAV~I964_Ds0i%mQCAWW%t61Ta__cNxJ{&sI93gc6`u_~x z<)<#e`>1eOX3sAFoE_-1)a0z0^#nfovSycm?uJ~PW6enP3mJ3$c@#2m%r!H`P)R1^ ze_k%4W}2ZAhvrkN#>@b|8a{%{{FvP*@%0Z$rdElWP#sA4oM5~pyHE0dl9ZFPVrIh# zNS34``Lbpb$+URP?8F1f;*=x{aoFn6-XAQB=P(I zQ2iOPn7O$WBzyuhULu8aaCpp@{`@B-DN!+VO1*j(J^~u%7jOA0)Suom1Mik79DJA; zbI}*T{i{BK4)hil;Y^dq@s_Vx|LH9=BfRCSGmwGaGAo6*v=>qPMAR(fEu36*tr;`Z zEPmLBpGo0Uy_-bUtwziQ>Ou8D1l3<^B~f*+9y4)RP~9IzH86=P55-!BpSwM(yCbN+Zk|Nd3pMC2kLs={ zsxe7ay-|to^r-HPqVmuGh1TeUdUS_Jbw?D{BT2IQqADHqs18O^{WXcI0JZ6WM|B{P zN_wSlIwn!|Lxs8>RJF43VUv+W>eM&>{C}2dfphRnxA2sP-2#8JDGU z4?Fw$>N`&ZM6k)uTdUxw6(j%F_kTY()UDzvxsqA&Vs0t2?-~TW(G1c$FanQ6crWIv zhd0#uu0L|JFym_1{kh4|@C{TO1y17pb0OX58#9Zy{lNGiEAE0|MA{LDFj9Ei z4?D!$79@Jx8DNJ}{ox_U0F?by;UDoEy{#mUuhlAHfZ(!0mKADzj%tQ|&$#)zXy^$z z1dF5wc$-&tE?@XcAXD)Mm>>Q4pH~{(DDxVI8{R!X*e>@#=6{Q&rqPLFI*O>f9KE?H zaAgq}ynoAruc?Y2$;hP=9wCXuU_XB45Yk{z7yb#llCDG*{;4wdvhT_BOa^yr0+V60 z$-p6ntbxHRMtk`Nn~r1{>~H?~Cp6gW7}a@KY++~7gOxE@BAA*~%j_%?oBiz%j_S18 zwjwMx=r6=(kNP$vA&JiNVE#SFA*9U?751j_JL~0n_cRxU7WDkB^85G2PGiCz1*-p0DM$jPSU3C$H9U^#OMAsIZ2@p+sE`CxGbCIwcADxNS`D!mow+sJM_6 zhc6xg6N(Y1`^ArchlTV>o0z@=IpEw^c%lkQW|!vmDF0v*oSS!w1xMUZ6B*(RzxQoUk6Q3xi ze{=rpwyB@z#PsF(bqGExE-Gc0VJ}NFT7=jup zt3-nM#D8u|((AV^WBN+uf=^soaVg>x!L34T!GSV7pUB4eEzy2o92wJBAqRZoDo;+Y zl9Ga6Gs{0?1fOs(w7Xe^cpXKyIxR`i?;6MSHsnG_vCRmYRf4C8<@M@SZl{#c$I|W; z;fwh^NzCsXd2t(vIlS8b-5Bl^QfeSRDy~=LeL@uE&2h3brzI)& zRh^i=4mk$lqvFD>OQsSoWTGTr+KWlj{!%-JYpsm5{r{D;ge$EmsRVBrGTtvq+Sj#W zxEoBV6Yx=SY31GvMqyHs<2NS>{08xLhHX)i5ARkritxq~*;&c$-?vyfxB)pZmbjtf zdL`XMM*&h)S*wzC`3?&Ww;=}{BP|#BQIgcJtS^$JeUAl-dyxZmISog6C`U@E%IG9n zf6ew5C{Rj=clOC@Zman5->|v;0qY=l0tkA;!#pr%HMdn;C8_hHteim3VIH%y<@oC# zp+0}ZTFZT&K8JZYk#eMWulh}rK7WsUD_rQ^5BE$bg^15(Pf8N>6BcytLJs&`8jizK zj#Sy~r;}v;Bg0z;LX|z7Q^XYD4PmpZ`RhMn3H==NR;Vz9J^ZuR6jKjxG_BPxhdx|Z zSeO|3#3F*paE~Y)#?Jo9BQNHN;V`8isZ0zg;tYpJNYCEJHz8 z6+Mnn(C-=2LE!OmKNLhlhJwz-8wyn~a|js;=a+d{+_Lu*{XdLu=Mypa;qt*2NKlHg@ zxZGeiRKsYG{TD{_;nbvFw$W-{>F7?Q?JUAmw)7vxXixe^BO#6E1^2RRz6wGH_ZCI2 zS!CDzt1w?=)-1TtdW?Q3rri~?TrfQXZ@6zQfB);EU=6oBRs%BdD__hdHMiDCux8C& zlvs1$M<%jb>?lUr$0B;`tyw%0$JgvitaMd)DJ>fk@M>DO5{nQqm7Akt%24#f^PKa_ zekBO06qV#EG8C=txBs5Ia*)U7UbqyJ$cg%DXM1OKOrZp=HkSv zd7%fhUUgw*JQl3Lh->vU_fl1*Y7V%bmlIkz@+UmC zKX&k7e2LF2U6N1|PAOY6yzKum_Z@Ij70LhaZ5~))S(c2jiwdFwn}A&qd^4~p z3QHD6aanekph(7y0-~NdojHLy=YW_nfjR4$&O7hC^E^Fs*3;kj+gKVF6|dDYi?QA(mZ`;|ct2=)>Y_)Sib*ZPAB+%>RKmVnxluwdB=KXYoils>LX(gR4rRdYjyfw1M2%oklQ=dluEbBoKQ%fVp#8#j0Qw7s={UOzey)Tm#m$di3)ad{ZSRLv!5pS5Y z^S{JAN+*{t@=XtS^r*+!t9VCG21?<82HJzH=W6y{_ORIVz==osY`VJUXfHmS{`(uM zNTozks>_3xRGv9;wReEipq{sfQU_$xP7S4`nRe|3_H3q0Q#4Z<+`9Vxe<_OcRpl#!$gID3I2MI-Hc0c5k0E=_EtOTi8m+Vwt5cM3jv7yQIV zx=+q7PA2u4w&%k@7#P3Xa0@^b?)EAVr(F0gYTrx{V80o`vtB#m4gI>Y|I+{q5)B|@ zVz*7knCNif*ZJ`2&79`3fK*|(zkmy?aCwR}Hv-CzG)p&~43&{~Am-Jk#XrJ-(k zgw>-A@;MD%!6w^{yg5LpXtF){{1a^ITuk1Em|X ze~8S3oqj|z3vXC@yGjcUvqwKuT|W`B8ABZ^GCge*&=gWFC1`*MyWA4SM93vETp#7( zu_cvqdn~JrYG^_dtrfyIEJE7s>FB5O{QF!d`e`OMbvSvR7Ytr7J; z$U;QB{o)}JrtF4|Yma1Nw`4?x)K19~*Wdds$fM(2sT`+r;wYhqhJN@&UH1N2Fs)Sk zylCD_akc`ESm4=4*69kQ5UM`aNmQ5NtU?@FsHn`-3Qa0s|G2EnaV8*+22@n$semTc z`XpKFaU2>>MN?GffoLYx>q%4_a4Z~7hf`GM!EgpuX<-u86*%V)$NecP^Q1qMDwafb zB@Uv*d6bIEJc`n!Iy{N$DxAWH!}t`Hc@CdRbxRV})i{w92a_r)^Gs5c%3uGATmBjx z{)sa{6_t7Xr$JTLBT3e^ICVL8ho&;mT{fxw^{=?B>u@1c>~>9MUdd!qoscB!dfYP+ zyG>J>H%*vS4HjtGOv*_sr=_3;<9eWWjeT1M^TyA>6lbp|F99>g6myy%Zs8iFL*Jjo=&2=71yZX zHWfu>UZi4BZSN#e-GjSZjC!|TM{l! zQdH)BNhZ}5NwV(5y-~P6N>Q0NN10T<|BuVM3s*(r#z;kFUKVLk?a)3+*4^IIZtN;e zb(Nep^8)^9QjJNXy2pFUja{j!t`w^06R3_%qPo|6(v4lAsjd*J=Mt#={V#F7HhNp# z*al6tL8zWhp!#Q$toytzZfw1#S}#=3*i<`m{%5ztP2Ll3>~c+YxllcwKs6VEHW zH+Gq(x=g5^N}&2f64e9VV{YtHO?9bIJ()oDP!iRH-lJ~p5>0i9P;E`1^4GuOHhc&d zzT@6^)i%s4-;G-Cw{w!LhrP{i>>^Edk;r<&mep@|64hq!VK;W6rn*q5n5i&j`RhM% zgpYU+xv>j0)dfQJSOV3vNwOaG9&}^OSz|OmU#K3nsrqLnQ9b58;Kn!)1J!v#^+*C$ zO%m1PSVW4QtEtWvs?7;h{`^0#*Aw0*H}*$O^+%z4IDyKKf8$hJy!+f3w@(7$IYRYN z0u|#QyY+7MHoCF1HPzWd^`K3)Q&AGtlis~%Ww#!lB%rwbJWB$LX|f5mP1taqmy zJ55uaCR7ZN45|UWlVss!VmHRrJgjl5P%%IfPwZj@MK;5K+x~7$BKc z&nHp6=H1}N)@iDBLd5{dpxPxTiRyLldN+2QraDfj7$BKc!;`4~i;8Dh5a<)iFs_fA+3$V@GSM zqlJnAl1X(}64krj1~+z;raDTfu1TQ!cM{cKy!CGENKJL5P+e_P4J=Kf`m1-j8#_W% z9U)WVk7b*rwCe^7)RDbg>bz_HVs>6he0g_4eXcE=my-VEK zp_=MYp<;k!Qhk?1^}ctp8(XQVRtnX6n`+RYB&vUS7rC(&nrekmF+eh84QfuJ`lolH z8(XfamJ1aFB$Mi*B&rX*3*6W;O|?v@7$BKcFD6m_%RAqVE!9*@g^B?ZsXF3~Jiy>Q zP%(A6GBI^IKnnC1wmQefM8Su#$B~D*%tZlgeR@2rOy^-f#lK0!PDajt{G$n(hj}LY z88S!{cwe5e-}ezxHbqiV$e(InOgwq8U@bf>aOLvQsQ67GxxwqvGBc(rh5@Qy)$9EW&i?poG_+sra8d_k~MR9-r z`p+Y4@4^S_!rx&ubfDD|=~{Nj##t=Ap$v8yJ@h|yEzy))2!_JrxbFWaJB(PDzp}%q zx}p+(uBz21+N>Q$!~Fc89tgO@Xxwl)pSj8zt~-Bq8f>+5VV0mC?|FEmP1ZP8m2!vC zL*7jBqS#?H6J9RG))ZPjBcDdG6e&Kf<}gbvecJZkadJTkeA;oD`ZV$<@5k__T7Rri_*z7V+n=C!0u-OIXQ2Q;!&km1(J(tT4OR-$ms38pZ`~P8FE>qXa zJce@w_$!|IYW%&|ngCPhaK!2)fm?9Cz=2xefhh%6XC!b?g22y{1a8Ga1DqeA4riqhSW{7% zk-&krz}@`(PrMGF?C)ZmmJ)bCN`d7W3EVY7;44WQKGo0l_SXXUPbIKCv#XDGu?3Fw z*FWMKK8@oCW@>>mQwkiGk%l`b2=t%-i3@yY2NxSdl}YzY7Fba`q&6dg18jk#`X!n4 z+3j8I_EG}(O(`(5Zn0B>!1I#?KG)avW@v#a?kumUtR4zJh(&x)u zZyzmipHv!FXVyxYZZHKNktFbiQrDZV1x`;8SYBCPH5BK-q&w(haLC;;L7<=iifi~H zjwzU?1x`yPa7bpYbO&1?dR(GOU+V38Q?lT!#BQk|KnDN7J|OOn7>d%7M2A2{zxDFtR$ho!c_ zSf3<;uk~;}h9nR;F{QxFI)0x7foCQOe7(EtP0#`-q!gH0$L}oyJK__O;%FBFnff~{ zan?3h3iK63D*jeM&!70DQ4bftV{0>?Gztm*(vFZN9HT^u_D1{bA6hE6HeZ85x?u>V zBHEj!qrLPZQzP1|6{Wrnlv;?tFh0K1sl>N3?r+~wy>zC`FTGVv6(vU$IQVn}p^Ix|S z@Tx1FhVHtKD|P2i4+%~$r4nQ;zGIJ9Ej@9glXC_i#_)IM&(ZADgD&jJT0G{SRbuDm ziFRh%4tCxX>)%0MPw&h>>>u|o#g9f_Z%@WQY%52lXe$eZ(WNNks4`!m+EMqk51YUW zyzwgH|JeAp@y`5lT?`F6t)Z0N|JZjeJ`4@UjSvlvPN4xu8JJ>R1In=V9<4HygpO9H zUR!#I>;jZm%ut%RT=L^)I1E?+d3-3N_Y>P|5@!gr=wpDkZg^5o)Q00#HL%@(SVFucJ z??VH*{DpGoP%gdNOpd$RFqo#>#vllQl}8Xi1|U%BIPlbUFXe{pa@G@aaF1v=kzPG% z$eOn-eUA6zf7Phew*BX#=*Q|VPb__Y5z%R}wAulW8~hFX_6g-ct?qb#{-fb+aN0Oi zJx>y;IB?d>{vB`%shE<)nKO(#VPDI*T)z7_wiOw>4YV%SKDTC-v7$90iH;LO|M};* z)t--j$*mXo9c!OoW@0s-o%kDf?h}p->-9T87rN?I%Fz|!7yaVzoFPNsdVl}NDmgK= zjWey<9gZJIy2I6>Nwq_V4Qpzy@Xt%Fm@{nPu-X=!Ys%N0q0KE7RW+y+`@q5ik-&*R zrTFuoxRmKt?lAl~${m)zl%(Sd+2JXI85D?_=s*7&7qd^Li&VMexi#sFNpV;qeLscF z#+!ot{V#DrGb+@B(tEhI=?lX2gRXIWbn3K*J?o-Vrp4-_`t;9yku^=^eIl(gE^ps* zcPM@w;SNn-o_$auX9eUhEn(7>BpLe+c1NJVNOwf~GNdI0#}r0H!gPGXa+5YDNtrpw z-3~wQ=5D8@45@0wHQoO6xfOGU4s04yg#~85nwuJ{8j%u%l;HkN%#wBhJq3s=lebF} zvj0FeaT)2BYavzDHBF6k{Cnb43$ac@%ytULDI?qT zKO_m6+t%F;LPoi}X(5$0bDNrj1AkMGpJte(qKo$6KoPzefB#p!ht6x`j#6SqWiCef z0R}OQ2`FNJyiqIr`1>E@V&+HO-SK10-93FVIAfFHOR#rKX~bD5#mp2@k0cpxL4nJ8 zP1GHkzNmyqNlD_gno?xmJ7X6};w)-a;w-9BLzyvNpu{80gS?oWD~3H{u}mV&1VB1g zL>#Er5G2T)@t`G;LFRvAPa?>?LW0aE{UDQsBwCF(>Sx9`79n>fz7YKiaVhDVo#LX1h|%_LOfn64Gqz@P^s;^Vh%QvH6S9uX%MJexL6|zs|&LiH%#EE~gdK z#WyAr&m6(zxBqyfTj-4u(~Wh!F`1Yy6{bRJsH91&o5}o7kSuU$zE5LM!+ckX`JVR8 zM?w<4p`p(7_djW<+&<*t2stN*bl92raDkqIsR*r{GXq>lFsG_!?!fA%X5RyDU^(+FS9O|{Kc%{Aap9c5y@ zKk{FOfhv$(e*^RRhMlM_8mvyCfn*M6ZABT@#cGvjoJ~&QdB#~^ zCzOV|;d#ab{rG3K$}^I8NS^UP#{aCb)rmDW3`A(T2QCu9$ErNzY%(HDYTQeo0Mgky z0c19Q;08v#$l`Ro$;RLRpx8LGkzt&N!;J8O-U&=2q2Iqv!gQvy?rz)ImgB z&ttARgXWnKSdf{fU4`o5zvhQz{f5XGl>l9XpZ`48X~AEZ+-vISG_X}8f6y^IL1IRE z#6{piG4t`JsD}c3`gz1=ips*z6X-j#VHW;>k@NfIQs!hWclOXguAe%*jE@m`b*urn zxJPy0Pp<|vOwIgQ5#aCt&^62e<)!F1xb2qwkvse*l0gk4AP8bFO3EQVd0hM4^wdp+Q%J?k_@wcVmUTJ6einNM*1ekABX{^r zRvGosgahz58DJZqvj{oBM!n5)WGn{Q%>@}WYzz(Q>BG*8SGU<2REasjR?e+x!a_i+ zw*i8`F~H8T`+o&`*csjnPD8DM>3%Q{1Yt{h36bN26(jSoYEmg_}><5OrLQ9z5||BqKu05oVV3TW}yfBe$0 ze0UTvcL3xw!Z=<<0puML1uS*sU8<$@ny3{*uYunT?=ezK5t)O){1`-eEd6sH|>msc9MNr&L2?gF>gx zp5ykZYZyOq^xjiTf&I6H1_UO_ig|wgvn~h?TH0f-h7w4_wMPQd6qr$DuvnyUJcfsJgB|59P>GxwkWj5A;FdHS*oFuzAJ zZyGwcqPm82xbTht$~XrL^YI$_0)PG=XTD)~mruYr;Qf&1J*2v_ytcw$c1YQ<6z^rA zB43~PNl8|@qDJ2U0p9m2-gBy}=hoD;w0=A-jj}L-H`o8!cjLIP?4xhX0Pi~#?_pq# z>kM0eFonEX+Xn7~qDj23!lujnPzt>7)4a=vHaFGGZT;as@@9Bz^5*^z8~4@NefWUp z{ea>yQV@igHLofjTtVcLLc&>y|=PJ;>o_HexMI;i>` zm1VUbsEj5gk=XB$E*2r}_e%6TC2n6{ZPffe(|eeze8@YFwuVJdJ`n=1N%+*jrNOUDx`|0{TBJ_pSqdSeqY3%gZ-`* z`#tB|kA$=zBSh5oqJ1o^wBM`I@6kUM>*K| zt>${^^;Hu~AvID`#9T{OTUxV;eHFu04(G3+Ak6iGZ!Qv&NHPRV?z0GKuGgbKI*!Ud zB17+a2MhU3T-$nnoZ8ca48@aQa>e3THLZPEP-rs-^5;m@op63 zz2qB@gd|#{kuK-@AES}D@!p937n3y7wK&oLW@5a=>jSm*@_+pbb%L2AfVlrX4v40^ z2v56c(_!gnN+zaD1*nkfnM*$_23ZnmyKiGJ!*(}`?Oyh6M?%_eHQwkRE9O~*Y@av`{mPWzjKmQQ#t{3CUq=sDLL93gZiRn_{ zDWq~{t6S;kKjK~XyVxtR+|6RSSA5Hnkd`|UZ&+?6^B?g_rsdv>{(@&BNO!Ok{UsC2 zrMhcTn-5%4j;Rn5A5#fwF5ZB3XpIGnZSN9qnnWXbDrQ5_^NM35|W4k6bYICv90)a3_o=v-FzqJWMV}f5JyK(o<430b|purX%n^y zF(E|+VF~dx;SEfO9c5vn2`}}g#{=T&nV2vorb4P{26Ttn{-5^yA@&;VcdOX%HL;%} zr2QnIJG{Szm96ld7--Z33v;lmBNO}K!d?s zW!^rLj`tjIpG+)gqba0TN{%$SBg!nzX|f+bWtPEGu)9`S+zEFv`?qNBqk|Lz7+Z&5#%vCYzDD$$T`0)XFs3k^cI3+}HgS`x8ud zyO``xzR5^Plf8jA>h;LeERbBkh+#)MvflU|>lcp{WVU{hUVK^r=6H2ml0i&su3t2l zR}X7y8QOYw!&<*MYNe$oHwdW9Sb9-UO#kIFmUI}i2X&&2%=L?-UbFZ|k^DQ=!`V@4 zxeDtSTQaHajMgs*o@bNXCWx zHONYcaHi~0u|Wb7Dduf?bkV9|M#|nL8fWLbM=ZyOk+NTP0MIcRdSZCWew9D}IZ>tT zp+RdY`&It<=a+`M;VJu7?}+tJR4IG%rWx4j`09bTB&?3%^FK^jotT)g;%Wp8eT?P* zVQ4U7%AV$4{<|+ns|YSfQ~E2Bd|s0)|MA0%n7||U?7!lFdM^NtO>|EfwUPT0fAvm3 zm%LEGJx&Vw&wpsIMCURZYt~qMLYaA3vi_esmo=qGktbb`R#RSz?02bizg&+Nfy$}Q z527j3x$i9X)oS|Yw2I&kw8Wd#c=ek8m2&Kh*phEhOSqa1J(wE6OAKos)&xJ;dInin zhonJQuaX9;tE2+H>}XD4c0iTT4^GkUAsxor528AiA7qXi?P9f^|Ks6C4}1GN4Q1qy z!;SXO4y~1pydK3;q^PyV9{=d|&ZAg0j6tpaGxd7DT90Y9*4$;8jSp|%RV~lgUFTF| z5()H8b6P5@TZXo}#*u;Z4Qaes5u2pAq<5Skipz8jnwW%lj z`BC_et36y!*iY(gzf2I|0e@gutYu7E$Mat~Cap^xlf>(t9mL&HDe53 z_g8S?7<^KSF<7FA<8x4kQN&4V;r}eK1QT}V|8W#?Jp5mvfj$6n)cU3^wg$)Z{5NWF zataN4qb!=j@mo-aE$w6#ML>hrqKMUD zW^%nMeDMF~dR1CBRq)~hBTNM!Q`OQq7h_VZk9>m*ja{z|{)fxW7Nf3L-D^Z}lEDb= zVbWo%;{#Nc;(VZR|8Ky(1%Kg2YHr6U!FOEIhJ6u9=B5_u40RY_I(hei4JC0G0UFOq z2M5x)UQB7+l2O7|I?2bU>9&bzilq~yXn*|E?UoVA`5rFHRL6cGB9V(S(;JBhk%lDi zj++0;W1$**{z-{FUwrk+^Nil5w`z|W+Mg9|`0A5S1s8g{Q&V_4@zp2ahBEZkrz&5) zo{cWKM^ei5eZNZ>ZBD5d4Ngm;f%xiEmZ1!N^=ZmiuO}xu zLAT<_BJXd*!rix*6u^p?`oh_x< zC|IbrYOB=$Z{Mp;Xp%`h^_Ku1Jpi{!%`0*mR3nPIMcIiWGwA=>-V%``P4_gv|DWox z|5Mg{%3}YoXaCQJVfg`JX-}Fd}C`%yhmR%XSss)qyFNA1 zUCRJ_)|1BYq&I3$TXe}RAT$Qpv;G4v9AM8#F+8z(!lY++6ivtH6=1+%FAv z!()JRUJgnxMhrmSAu+%?KcOTW)!B(Pwg-sN7XG-MusTb%h1OXGMp056dKa=G-!ghtp)|ri%;2;m3N#(~5bG##RRj(aZ*;Iv1y49`i(k*q0 zog$vl6dgTn+{F5Z$GmfQfC_pr{GCi_<78Kuzjg&R`J30TrTpL zqqb=0m0o=ir0*Dy68_B=f5r!YiNTbLe}fat(eXig{IQ#C^kns*9xKC5CpsXEnRvs_ z=lSO!XjM0$wSMNU5G|S!e7CJT?=% zCBP}9UP_P{?ZT%l-C?wRC-ydscDESqZQp1lB++piBMDmY<2y zFhhgUf&)8rgUbnPaFCF4PvAx8T9DY_zVud$;gDuqor&R6Kq{nuP9xr8yo>WJfi#{Q zdk4n5M~wH5Z#)u`=xDrA#TV~x5z=^nj}l7CwCFVHsky(tr;RwI zxn2B>g@so8%3C9D9L;S_CRR(pQb?uD=5~p{|4X-RS}ZH}XISiBvDlw|i;<8PlfmGU zr4~wB?EP58amq-C!61@}#V}c99}PAE=KZ?C2t%fdmoWZ`k9#!O*WOyELAeV|71w5B zFdIrCRWhfFm-_qv;`Yjpy$gG76nnkv+lz#>*Kv5GguD*FR!ya4aY3&^_B^ zVlUknW=x#2ciog=49#z)d&d15;WTPUmsKa(?Hdo5wbJ@I_$w}n~Cv)E-_`A_G<+p!3UWK z8~%tyI_Ty8`6u!5;9EY;tKuMZ&~=&EEdfm-)l!0}KHAOYZ(6F;X1TGy!e*PqW`Ff< zMnV#;)`aVeEkfGtgIJ;C>_EC^Csvq=&3G2il>nYeliCfK%LpyKP63q}e2;vtgV?$dt}lB-!+7 zr%35Mf!30Ch5-fB3qcD20IzPhWiVqhQ#wOhhL&TF)%w$OT1Q!txuU|-vjQ{Gy?1jO zDrlZ6XSXZ`BptSF4xMNrwIU-4oh#0;xO3;)X&#PERlzS3I$JZT?6l481M%lA=ss6= zvw%zn*xS7Yl37ARXMt8Uy;}15bH23zZ}LAJ{4pbz9L6Jd4g=L%gmUV|$@@0EbnnW~ zjU0wMk0N$GtFY0~)lJ!??;H|NPQuFX1_itIi8} zGlxOmAvuhzHlrkS80RGBFql|^p|5&RSShCqR$#;k81j*V-*#)xN`LUoQoPMIXC_4` zEo(FWXRSG3ZLa_4-pOB=&-n9R_&{B_E%<~2SKI!Nj%5$b&eegyE-u~M@;`MfkEGlo zRa;(~@wziBFk$Z)|CG2HEuC#{$(z!`)}3ADjw2wVy+UsdNjup;E6s%xsld3ywIOJuJwos17^R4uGFavE_<( z&&aD$EJgZiePW5FSG$8pcByy>YRQpZ>3cO_t;e)l*Kz$bK2~#Q+TBB-Fq{8%&X9bK zzIJYNQ_HYc_e-mS>A?sw6HyG$Do!!X!567Id62}}4p)3IX$Kk87((&Beb7S}m3vgkA zg^mOlj(_M^D)alK>|5I$7+3IiebK>a?@n1Kd&BOKfT zw;|v=uDPbV^ye02&>pp>TIr?%A{jXR%$Jvn{5P64o zkqp`aX8Qga^?HPIL-G#boigum>-!cOKKr#EcQ-buM@W%(*cW3A{A_yKCZH*#T4vti zw#k+-PN4H+f5Qa&0qTx*kH2M=Q4LKv0DqGS^lkq62f9ue=kAF*(Pt840=>0Bp-g*E zM0)D5gX7ih#SBbfPN3)33>$*b@>jMrDjlWm-M-8+5Fh?}3+J~q45NL}_72l+FCE6T zV<*~3wQHF|-|nyfYCCcYeY%H3q17S9m_k39No{8oY!Zkug}!5dOHpn*yuia1cIprs zC@NRjrB_d;qTY)QI>#vV@;lx!1~I)SqRS4yjKGh^Aco~QJq%xvVi05V z1fx556iqG&Mu)6?3^uH>Ru8Ycllxx?@q!SDcOGj(9)78`e;oB}6n$5J(tk-hqNZtzbIl4Kz^y{m%&z9IzhUF8N~ zm!qI_&A1Oo;==%Z7vn#MHdm(rd^gI%rgt47fUl-ab3Obmu<1okeqJ1LkFrwu%-PEt z4;wys$+889Hm+zHd}!mc!E+ZcY;IY$eDD#A2G1L|wCb>B<%@>a)K)BPI{2_d7nN5m zuf?stc+Fchc-r8DRxVnwqO!7jXcdm<8j7oEYAS~et*sh1cty+d6$=*48`M0f6mzhA z`KQ`yIDh&4Wi2hu^Bb2nE-FPBlEWvK0iFsB)=Jzx%2Hx7tiDrEr$L<4|75vnS!3v)vQ>-HH#R((mE^ zA6ELUDJs1hWdZq~MFRO&w$g6vp>VcKqx8zjcigzOvKTS>rqg~gj`j{gdutflR}8f7 z3*heD^S9IKm`t?X|4OuXq(Hk4Wr6nIUV`=x)$MssVbDYuIQfNfyl%>`j@hKj%2*}t zVe0?ORC#yrWgXUEhrgrYtz2~QqQ!@yong!Qq~%96%|E1Z(V&Hkn;Ky!b%?=md=eA) zKeN&`CN>#h(g*YOaq{8Y1G8CQA)0n37iqiq{$O$9!x#al5+~niyU;_(Nh*sO8*2k{ zRwa$2tFjJNS?J#&3c5z^e3Y`c#`V5+#~oopxmc8jQ9enV#LdkWLDE7uUISrlp?8GT z`NpkAoja$3+L_byU*F=xsPl~`zG}uiyil&fw`e{pWubIXZ*u)bx~% z=sP++F*Eaf4hNhZCu>1-DMFU)1&dbbup4X~5AUIPgWvsr{#*0w2!7)SIt~2^r>8T} z=!t+M?9k33QgINa?)URQic10ebHJrUrAh)+rR;?yC+Qfx3F2YXCQ7N=UYIiz!~myP zf>O}tkmd8JXx{P_%P{O|#aNj9qzgRI$>K@HCJd6pk9#?TY{h(*fCjmWg=qG`Op}e$ zEMQZD;KZ=E(v0LJ-4}0Y_P~WEIp*T7GYm!4OnRi#%Rn)>tkl`X8L)WSyg`ji8k^>~ z44U8Aym;}FLGu?J+A?VA!a)lb4_dseY0$)FO;cNzHMfWif%zcgKlUG00eRGRN`=#~ zi*8jF2F$@0%sn;C2R)NKpCeiTbD#~g)al}sEN@x1pmE`XBO6yNSiES^+{McdX+-ma z^(w%RJbk_wSbT3cVGVRDO!ZsmOS6)vukL3i--7o{z9mkmx^924K&?D9*d)*MKz@+a zid*RPio`1w)+zuEUugdo&fs+FXWXJCD_4w-8>L>E2LE5+%==%*e}fwrwk%t*Tz82! zrB0<&o(|-RD_0npeU)KtWiqk)!*eaOO-|G5Fq^Xa!~XsU?Ic`~U&g~*R3q(-!&{72 z3S&!OB(cwc`LCdn7qF3sGKe>G5{pmR+}~u7HyaFsX5K5YnTxZ_0Xbh}AwF}nAOGpj zw$!vk&%{=(=mTxYL9Ds?E)z^anaQP0O=9D=;-%zMy5%iiU-a%U6tPUN~*R zAuXlIGPS9wuEbPtSWP(_F}X9?q6EwT<+B9_h8`rUq9k6eN)Q{8sZ_yZ{{9c$M;|d6 zr;1MeD&T4_a6R@1Q}evlAm6pCL??rtuDOQhWu*?bu{14Sgo!IuSMAIk&cP688sPRo zyvg}*7U#3TdCUM=^6Tpiux)BHhL*sf7>SmTTmHYG!emzB3@{p+ZXP5j>7E+P<97e2 zmIJ#g;UmXtITdkgwQ? zhEH@f&80L9Sd}l6KTco|pu3&Q$IRiKCxD{+rE4&!yMno3r`o^Rn<^%v7G> z{s+doxa{`v5=6F@{3D}N1s&=DhRUcwb#$WW53o|TqWIZXv zU%--VMTsq)MTspTN^BWwD)AL4oktGX@d11oN^D{Nk4ik5LJ1BEP+|-6Uus}_(rLjg zat)L~l0;8udto{o_$@OXfJ#H8tHzaq!p5(E&4%?Kuqnn|Hef@cikO5wHSyML3j+z87-s7VG0fA-F!TiaRf3NJcth>2 ze*E_|56M^4>pWK>F9Nur5?Pr-T4k=)7o3z7bQifS36%RJcyWf(8KALMCt z+uF{~(`56XfpOW0$v}`i$@*tkcs~&dZ2=@t9&H-yP5jaMaHZrr6hW}-&<8a-~8m4bUx%4Wj|{QcKbHbf?) z>r+h@I<|4-WObgdk5gbcCmS6^vjG{S>r*#doY>3*P9;u(K@N?CoW4m?Hcv7C!CL(w z@E8S3L>;(yM(r1T0+`M&*RAS{ z52N;PMg%bkudnfCXSZ%lcedAo{3F92l%x+q&CJh^7$W4E@~Q?P6Kxqfeozr z!f=y3QUnk?b&`mEs?!PcO21Y2L}Sp?NWs$E>D7AM@hM?M|7#COrDSOeFKo1=mqGf# z-cEx$&$z_tZPX;BqauJzk^1D{TI3v)LdbPP@X04>-Csof7jFoWvxHnt7>78$;d9bf zG0va71>TE{e*@%`m5v;Syn--!zSUX_zG; zYT9p*GJs!itxAxJF{Y6(ooO+qHxH>8yIy5Y@TC_l@<|}yt&gM6mFVsCu{Fn%CLNb{ zRWQCi7%iSzSjbOAd?wB3S!LFyeW7beG-o+rx>|!r-{X1TdkCNw6+ZvNkm%!(km!}ujF8A#4=U&8kFNU^A4W*@ z3fDgw5`CE>B$Bz|E3b>XUj~t0L`8a4@kq~@8*Vs+b3?>{iF3mZ(WJTIs}-Wemmx~L zI>l5%%?)4r@apNd60aU6N_?F{2^lP2y;zj^+G&xwVXhxH_d?tpxW@SO4%0pzG?RtQ z4PW*1ADRt)RluegL36-{b)<}B%ne`bV_{eV;x=gbspf{Sas8XM`*nz6UNg_XP=cJ{ zyPkEaZJ5{C|5>}=D#N6l8@}e}Kfev;hHYKNun3w3m6$v?eC-m+=5_kSQwEST<%1SxuM|%-1xYQ&;fJ3 zU|&d((=97yb2&`?f0-)IWXgs+@%IeohJTX$N2x})2FA>b$f@jooQMI*_zH9o4JFLE z;h*IBZ{fsd9&jpgB8G2*MnX>CGz)K-=?$*`vQ`TMkI_zvsLP3j&J8!l_1=iNVI=k3 z@Xn;U;TtQZ_6tI4{|$`)LG5ct_)E>Ij{%jK2#gm3&s)M_87)n2Jp z6<5o2gwN3tLg6GwxG~8QzRCEHO}KrC5^o~^5m!Pv!UsOt$c|tr@g~=Q(ceppQYayg z@J;UjV%-!4jxgVEB)!l`0uL7drW_$#h~Ws|{K#U%780;2Mhgkput6vz8IJI+oh)KH z!Vch{$`QUb6-8;7q7cKph5T1wm=2h;<1p{}7d~qn<}I%O(lEu!Fex43TkQWdOmW}{ zi(G}gDA%aOWJmbcHv+LZ5u&uMx4Vk;-YKf38c1R4w?~Nd-l}cQhDlOxX6l@3Gt&a$ z;rIXE$~zT-N3g}H-r5D{s$RP|=y6vJw7%joYJ0z0s(nNh%05r<-v!`N0o}^9V4Zli zcmHU?`9dnV-;wyYG>$tPg8a6_<8rr9YlEtpX%RrYaz zYLV04=zy_R z?woCl``uDYFmXe0gA!cgv@wPiEf}Co-0wD6B&Tl!dB#r_jKSuiLVn{6*$}BAB=#SRWl<;uIRV zwP1j!Q< z&f(TO?G4-o+ZOkGP7-biZs38=h%?$7xV2z_GI77RyG73S-KK-gx9gn_2JXl<#=X%J zOnXCcgE~O6#OYw*)`9`b!u_p9PTXxV#J17?q{zVCX4~Ta+Y(DKaYJx}5?taG8Mw7z zj{AoH{@bka9@AHyy zw=0%sTk4%+12-ll+tQ!>{jZi_;)dXcLfyWL4cuBVK$*DT?_-hE-tCb|Q_qsrTS?A# z+ZOlxnEx3Ln7AQWKg3dkM}y-2TMPCxfD3>h;m!MhvdD?M1429Xd{@0wV%WR=w#EIA zNtR&ZhG6|*S&36(;MRf}ZyN37A7@zP#9f5hLI>S1I~%w=Y+Ky_EU*Lx?i=4Piy7OTds8{cFH*j~_#<>4w2_|j`Zcu_t zobCp0Ef}CodwsrDF$v?0moGOYarvuj zbD*U#dUpu?xXcn5)dGJ#QE;b_&$h^yfV^WHfMdtGz-fb^!^dkw_y6ktRxL^aJ}EtW zMXW5v&fQe062g~NsQ+wlsjvnr6!=r%J~)myOa;h?re==zKZjVHn1Ks8*^_CgE##~Q z$t`$=~Qqvse9(xg8@{xfLO z&L08O`Q?%$?!$-Cq(5l@7iRGWrD)Q-qbyq3C&!6egMzNuRdvPwNnLTW+QMk{#TwNY z$x8a-Coc=|L1DnL|2qJlMepNWdT=lHNdx$&-2`|=3gFTgKiy4$SE#-?Np}t1wzq9x z90+i*)TcKG0$b1k*cY??-ICTeN0t7~2N4N-3b8!@R9fMXn4SZ~b_Q1x&YyArC!M1% z$`&8-TYN7bK<)R=oZy?1$mVV&oIg9jV#DShuqj6FD4E!R`x*1UaSU`EoqWduF-j{j z%V&>UMaxjMOWJ1nT%P|CU5h~6EzvB}vOeDxa@Zv*LR!}66O5MSt|H7`mhD1|7%l7b zW#GatF+{a2PH^>$gKa6^Fz@HG{{dwVlDf*#yFc^Y{2a+=v#gJd`(KG{NFp*B!9V{% zgbqs~bbuE6MO&1i#fJsi^b%M3mm;M6f+dmUZ2A{tMEbB0>0h*n^bsjK6FVSO{^BH& zJ|c)8peue$SXRul;)h%mAOo?(7tafl5s64x0hIEoQA6rHdZ*!4JjHF*(3d3wcT@_v z5-eGf6AgDH;C@zc zM^oSypY`Quf;$?-4_GtyeO5tEd@w^uoO0na!}#H=MoS=l7A*AZ344hjzVhQ=jhAkq zGjg>OKeR23<3%3VcS~;k>u=Vg6wM_$emJNs^Z4Ow+5cOrRiHO*|KP0+%|Rm za8OeG@D2BW(j`Shn)EkE8ckZq4-?++w%Bgc-&_tZbe44~nzY0Z-^l)7l%ZPm;5}4V z?4Q^bZz)&XT&%u0Coz8bwpf7Ig#iDy$^?E606RBce&@Y5@Nf4O;CrS3EK7t z9QSwCf_v`(cP~Vs#N8$VH|I`mTF3p}TEV?{2=4E$Ov3%<*|(pVfcr_oUEd0Bzx~$- zxVKZ!_V+{kx9vTOO}%y8-}~$T^&z;wx7PnvJDESqZL)EHzglol4#)kyzy3ctz}-8q zP}#dM0e2m67yb(FFT~!HLva7lDaqb{%-c|JaNIxm@!zxncOP{$K|dT# zVB4EhlXk7+{^16}JuM9PGfB9|Zy4E-fScnVanA_H{UgVJ20}9e+@%HWmA%_1;I0Sm z_O0XoagN}g5rX?i%zqO4<;!2~^A8*MkJkzAnXTaF_)pw31Keei4hnaN1l&`AyF=@^ zf9fH)XNKVZ3Hu)sa6dL`>i`?~Pn`eL-m}7S|FlkU&kAt&ZNs>@Urtd1?tOr}=vQ$2 z^S@bPxXt;W_Jdc>=ySS_`@jDDZ+1BD|N8U4*#YkD+cFaEm(wu;_e|jK*gEe2o-FpB z9fJG6HzwKpyAE5oCg6Ti?A_D~?ytn&O#$v53c2vpFQ+&G_blKpZXNf}F~Qvwg8OI0 zKMC#RtmzlUY}`NpL2%Cv$Nlq7f_rX&d&hR26z)z5xSN2x)34zEL2%Cv!Tn3mB-|y} zo&2JW`xj4eF9^r|OS9l!5a90Do)fcvIVB0W7XWuj>$rcpLvSw$!~IGU?zMk^d`$xG z?*#Y4P}~uxm*8F);O^gnu|mI`&Iz~|19#_EaYrzz)B)kb)VL$gDM`5dt}fi)!X0sL z6Wohi!TqY>UL4@wsfY_d{c^e_;9d#bU0TQO_7&WVLvXvJlW@PY_r|kr-0pnAy(}EJ zd#d1G7T_Mxv8%$}H39cgz}@v%aK9zEmxbZ}F$wn_eeSZe72G$f{rAdn+*zFebKF@O z;NH2I3qSpGx+UOV3*6mW$DMVy;9ePmJL}FQ+<$+eZMJRitTzPr;jQ5QNpK$?;NGPZ z7k>KXbWgzj2jK4BI_~Vf1oz<~xU&yV!hOJieG=@QeWu_(DjawAU4r|l0QasXT=?mi z(<1@*X~5m1b=)~wg8Qft+&SBuxK*O}wcD=2bC$-uA(FGF;9ivix1Oj)auy2iRRQjS zArrMfCQa0CDNmTFMRM*J;H$!bzheN`iQad5Y#MI^&t?9P0m|wWz;)6jlFRikHt^L! zqBq#>IM&&gNfVvXjO6mdgp+KgueK~&qLz{((-mZQw$$FF{m09xsZOLJpU=YlrwygScbQ&>|&-^!Y z7pp^BR{l_RzR%_621BQQI34KAUV++ z$;bMqLbfgunNIXZ^8NVlgcL#_$o z^2sSW(-1AaV40|Vvho7!Q4a;F6219WqL=Q545Wq%xc`AQbaEmRRseJ?fd7vNoyuuF z^zPbY@S{;f1^*SesV5F{n1`2@Nb-`+^6Um9^NRffu2ASSS zo4qZ8%=CslfdYYWA~YhilRDEIX~X$X954MqcO*F!-V2<&=<>7_r*=~lKbM7iG8t<@gJQ? z(%nJgMA~ha1pM-yM*i0Z-j4HM0#9|1g8*+g7iHMC&kcZAsEld<#Ej`CJc`M`8QV>6 zNnGNNw7Xk?hn##JX@~kxu(&g`#7>z;+9Cc|z|T+NtOa=cJ_7vw0C=TJnf6ainQmGJ z;7FsIz(*xoy!|2petwACZhuA+@T&_p9Bo^?J>y?m{K8g%GybQ=FARWJshnwl^Z+aO zxCwJW{>=nlmk7KA_kXdeTo?kp1LEHVi?4rk@;x^24u=Tvi^G9;fd5y(FAjiLtE6fF z#H8t_!vP$LR1^4KNx=Uqz%LE~jxCyrz|ZNi{ns||qJ9GW(s1BKod3|`mj=LxsH|!K zoP@0DCd^*>Hxqb$BJd)v|I^}^h5_G{1pNE&x8ETF_+JG0<>A0PW()Ak1K>kd+O&U8 zLfUlGDgejdOyE-zfp?rDz%LI0-f?*n@LP*9N?Y!*;{^hILo2`^5a1gE;KNkjw0~mW zbkk}8M=sR_zE2|X;3(aoRsI zak>c!1OCkfJ~IjU;{yE3FyQ}40{+?NhI|`1c9&?clIqZUafh7-3Gk}};I%4q+CL{D zbGqqx0LR}<;Ik5ecUmRDLk?b#bh<1Fc)Lp~-cA7im;k>v+~S?M|AQ94HUK_crB3_j zB&1F^A=SaZnZTP8ftO(ZqdMocAr>#0kp%pZdp54LftMUDz^@MnUb0?*UmpM;p>n7F z6LY7VP6cq}Q%&Ftl7M6XhXQ_m2=LBDNx(}U`^jD;jdUI&z;6r(-kIy)wD^qy@ZD7M zw0~mqbkpeoj)bZSd~qW1&esX>8^eI(`R{}QXw18{C)pP7{D}a+IUIPGVgY`00DPp% zp7u}7o^Cn|z>!fkfv-#i-i7-==*Mpk0p11k{{-NlUcBN58+aG&e^Enzs>=nWb9Q+~ zfa}WzIKdyK(x?4%64Ixe&H-@z%>;f_BJi#~1bE0j1Cg#Hl7Rm+|Htob;9X}6@Y};J z-gTV-zdf+{?ka!UKQVv0>3jf3PSpgyHVHWMKeYJmAr|k7`G11NOUjPs^udV7yLA`f zcZLJ+Hd27!836ZG0=0i)0(H|x0FI=p3H%RChB*@zxo{~+*9K?*f^@Os;3BP3VYi1h4Y5i@rZ zft6k6PWWO@Qc|crqgGMQ`P=z({$7~#$EWQKnG4`Ol$Shd`tPm_?nO4MfT;lA!Cg zFN(5dZ3<~wy^b(imQJBQ-{pf1;9@wTUWk8G%X&z)tduF#UXKc>hk_Jp8+`~wz8guA zLhbd1AbThgnWRv+>m))qrw}>-0Fj^BZkPz&9HdZtxhlg_l#2`tyhZQ=0g1#e+xhvw z&0*3x|6^x*JVj?3BGNhkXJ>j`d4ct)he)avYJrtPWxPuUK9V26gf;YdA`(^rbnM;J zs3E;U?}&Xq`WZhOHPqV^xT#JmmVvo9@;?f$KB<_^W=u%-<4$;6JNvQekVL1|`%VEK zayoIO_p1hQokBgaY)qd7;NJ`Ir&8EioK_#?e-!Yig3YpR{XOVooi;h~d>+1Tixet$ zp_l@hLJfI91Onldk9S}M?;w)f8w=;O3AQmD2HFa|U=Q>dkW{##cbJ!-(oK9NA%lH^7TwT$r(``A;e zkI|OBm54gzTtYU>Zn|IDk}1?p_alYcE@cXJ)5fF}YMHBw=SZ413c1{LVXXDh_ zHTW?4V%f8ROxKa>WH|PGbm}tS|Lc?C=sMb@avc%NbuevB2I4yUa{fct5po7xr0+gP z56n3VluqsYf2SKX;PyRA;C_>$2XZ?W;P$;-YT+Bz16d1wuoF)$+AVk58!fsZ-i+P& zS0|)jmZfa5TmFr|r+<1^WP3mU)#%t*JNSo3^l{o7jnzhn#>(x4M!tXh6D>X2SjkC_ zSuu~HfJQ>j#(E3hFw^#1EOMHuv;%%|@$)%Ohty`m)((F;BFs#`y4OFo89yoebu$t< z4FL2z`17B*8H@b0Ec!SdQkxOzVFxH_nsJBcEj!SR4vs^t42?X^mY6^N$mF7v_l;-n_)Ws6c_swbnIYAjo;h0 z<&z{G`yH4hYzHN5Vz{uaxq+~)xzG`)QtJ3qpyN-0j^dyCy#}%LPv3;N(|(x$jf2km zsplW@&3XFpC@p08r~Zh4lso+;g@4)`Wl@R!cNYbJ2}X$>RV7x%E75qCcxwTkERNS2 zaQpT2J#++1vh`VCl*GE9vRzf&c1f%QW5ro-rS*iZ*r)&3VxM0^?6VW*KY@MRD?#l% z_|-#yz=vUaeahSv_L14qP7PWKlxe{%j2^P1?;(4^Lk1qGAo!+cN9=UQ>}aQJ zEH>=dWTV_;F;3|Th*>Ghw+;jY&eBjvJN?}vrgQ8M{?^71fjgkRRg`dxeNWU2o+ywL zhpV=lu>Tlv2TZrv5H8s$H&hHa6tGcnNo>F!aJEHExI3Xps!7&>2T>I59M+wi>J-h$ zfOn00EUt$jXOHN+U*N;2$DO(U!Fp_$q8xS64`@qr*G#=lE!zeoDwvM-7R#~Y}MiLnqcD^(qgFnvmV2XtW zWZgGsJ2=oXbSA!VntXhJQV;Dc*y71$EZ)TbLBI+4BT;Y~!JTyHnI7UK|ZgR2l5^}LJte<3ahaVhw#MgsP#^;U_Tt1a}6C_2+W zjJX>QV>aaEkd#n4-66RT^QfO^3(+9~ z5!$mlGJZfV%k+{}7&K|UZ9U$e%h{4iFFkSK?Ew$8IdukqSWi^A6MEu6iT}~+*b_TK z=~jE<2#tJDy#M<>@etJ$w^uz;_ro2__z&rSh6Z3kHg&>knW;cNJIyrz8jYQCc)wxx zkoyu@9aM`9r7j0;HJPf*Dgvg-bxC5Q%?!>liSs!pszniX1{_M7ZD#Obv#7c)G6OlX z{}r}J)#V}lNR}hrXt3=6k)lgYBXzTl6!ds%MRGC8V444F7V3I~^r4n_we*~w3aZE{ z=l&P%+o{!L6Kc5W`vEArQTeeZu^Jez2nL2W92k^`plYP>sqVI@*oa%Bk~pCJBa=#v zCW}I4+IEpL2ndFYaK|nNshDcYiL41FRwE_5KM5z`4TDsiX%U|##OffeN)A-N zau#2Ti=HC2u`*{jOD z((|vWM368e?ot)HUXWr036)&`u!DrFyqWPJVWv1$k{LllrR0AB(05IX^I4{E4h~0- z2MN8xg9P>(*uJVT0DzX9&t_Hc-{{*7H~Jd3Rd=JTQ_DzFb+1*FP$v$OJ9@k0j$T8KzM+>`X%y1?Sgr9-VGVc` zZ18eO*y;ZYDYs(*qtA=5*HcBQLHq?YX1d~m|T<5lh{B% zMCQK&iEgwfidg6+)*T|}U+ALh3d6Ajh)!$h0HuJ(TmmxZ^(yMfdP8UEB_rE*f%>QYO%9xT+U#>8BorS zTG?Sc1#%EKDdGw64cqB>Z=kp)RzC^!1OAa=vjQTzrcy}UKAUCuU~5|)G^t4pZk^R( zQ=sc${#!VSdmVOTz^z3>w=nmrG+=i(-0LuT{s9dP8mP#xRrfjqDL=>k{Q& zw^!~pi2UGdlOjJ#C!0E9WyL=#@|*5owY@WL|BWjjc2hPhhH8m;WIkoZmFRCuj0mvjdG6>aj{V(3D z=*0p#-E1QTksnQB3_`V7|BACvkzc^#S4}d!qe+}cmz4xnOh0@#tLS7CYPcMU;f7=V z*8u5)`08M^N8~qCMApgY93+xzY4%=Ww7$y zy~I}|CYW-Rli=vDoJ0hpzlyI&!}5}H8nMQrVf&>(74Y@aqn2E~9JFK=Jdur&avJfZ z#pd($aEpC*%QcCWpV|TZtzk-1hr5-dC{s@DLQ+n@q3suKb}o~h~$w&_BkCNhCN0x{-Hg*6m{e? zx*260RR%fCZf7BhOJ;bnz<9i&_bBfFVnup1no|LHJAQ8u&TLb*uZkWnwx5ytD0+^9H*1_nD9MN^WJ@s#XE|?cOLT4>KNG3 zOlRXL$vww6Oxw49+?a-O^<(PxrL-Eww!9qdn=1#-sj8Vfu)3)k!pqAW%NwgJDx1p7 z18J1RL@hMg{Sr&sWGC0T&^tg5)vt8C1C%b6)r~DREklQ-r3*RJrIPy(7WYydsy&YS z25AqLlPdQ5p)7m77t-XtJ&Y<=cda|`|EJTz;9yj-$MK(4oa%fssbY`wUsiF{(N*l` zfC*nL;1|ODKWI=+8Gr4z{JPeAxQOeBmI~(4 zDVpUhl!YauV^D@IZnSD~8}P(yw)cqCQ`GSu$!05PnN!tN(=w#>dTQsioi@5*Omtc_ z7M)s$Mo}336hbIyClTi7|Fp2%(eDdlk2?+CWiAS|bz+ZWnDOaIBh?-K&;ofTR@FWv z?;zG;Icesx&Q@6}l!Kpd#>C5&>sN7L7+jZ*-P zP(kB${d^`;M4QnV|BS}K`e_q;0@_?I+B_lJkj~I%H2q(^5mB3uq9|i^N6}`qpZ|($ zljm)gTNI$p=8UzW!%}tgiLDKGO_`{uP1_h+f_JHC^O$HuIzyW=Gb}o4^Kleqf_WM1 zMix59SoQBVqP~V8A>5r3%9)2p=c%n6I;^^72t0Hv4m;b)>mY0=;1kt82J2sOqq<8m zUYzO82?hgij$&TX)Y4eqGNd9M=432GWSu|%Rkh$w!ay^a@`;--7OTY@IO_cPe>8Jc z&S&uC_c)dKR+dxg($G=KuR!96krq*dF)wZh5#8zq| zO18p!ET4Y~8emqLuvY$|+|jJlv{b-aEH$>pVoj4&BfeD1;X_$UYHTe1e}DqvMz%A@ z8z&b(R5;$aOjOL}q=<~0@J2O^_4A*)#yY~{Z+Xp5Ll2s7565fH#x9ZKsguW73@NWH zudS>YGPJs4=+N@C2k~r2iKIBuXOG#IK4pM$oj1#A7)p`QXO_|@t+7Fv8!d2;2TkUx zS=|ejVKx1t`DMu5gX@2e8lyO_;%kb~Y0q{RKZZ^dhRa-}!WpihRx@5{8B*11XJP2X zIFBJ&Gv2WMo(nAAQG}n5aK5LS?w*+M{a)MqGAWN?4rTt?k~tafr=NG6(=d<*LI8DK z`o{OUle3JdNT4$3xU&e?!Zk~l?DbO8VCNsm)PMHcE5j8f9o;YFB#QM7W zX;af~b2(17w*t~Qc2C3`dXIa}l7`UVy^-UC=ZWH*Gf`Z#Cu>oBuWF0^Y-qfY5Q28c za#96*S?wP!KFdAJdtRy?Rq(t*QZcj{u4Ty3Uu{$m3g<($qI*4LA&H{FzTmY;9(<K5|h&y;!OX-e5h!4N5K*))l&tDj_zT&>y8}v!p{|MT-f0q5#dY zhcbt)MPCu+=RfzL7Vy6abo6>dB)7yiPr}t-!N!j1;FA)dlgcUnQ)@ zQKk5V@GB7`>P((viK05~V^2YyD@C2BL>P)`KqN6&WMp33Pg*ubBSY@fs63>%s z$DodvxjMm&5aFV-ny53S!V*PwI>fd@oeiSSR#As^hB{OH_%B{NpG8q7*qO!JnU6PW zXUe&j3{2bHL;PdOCh%CY9!@t#O5^LM)s11c?YG8FD@CRZtdoR7UPbyg2n>ORL#%%nZ}%t{e%k0e%w#UU??32l}XS<43sY#PV?7)w5=xt*Hzv@ zaw8XPeNZO0?&SFz(uUA*nm_()?hKm7vy9p?j3UYxbk=33XTusCWcF5r%p@6T@gob! z_%VGa6yo?1a!Pq*`Z#0!&>7GV*6uh>xRi*fMc~3HVSMuV7AfvbrRs_8E&Z9P{dB z%xjyK4|D5lCXtYOST(7+Lz*h9TdL=_+FY4lLxA@v_1p%igH``l(pqWdX| zA(7`@!k&Tg?on|+q%+*lJ{soMVALf7t$HJ%(%;ESY1@t(MKXr^xAitC6js;$2k4G zNc7q)dXdi1YhQo=lWs-S>&qz0WSe@;0uAbD-;I_G>ea@(UoNaf&i(%M^0s#A#Mh`!(w9^s+%)e(;{eQMnl8~vFibEo6J>qbR&zH zc9jrdRvwBvYW(^F+0imP1Yi~u^)zc2iyyP2^~k;V(pYtZyt<`!PEAX7bE`ufbBPat zC;5t-o^_(do7vGvxevMznZr%*_j_&c%cMMsz-Iku$zc&Mqm@DwM*`N(f2Vtf@Kdp-YUBke_K^5maTP4v99Cfz@ufB6J0L+nYg5k z4=ovt?Qdp?;>ASWhci((+ar6jrW#=1u&2d-HU_Ols*I_w7`|bJRki~3zj&uP4ZQ`) zX$na#Rv)k`g+>heeK^$sIXSEONkCrB$t&g!e>2 z1ft_?+yAq14e(Yw4Hcr}>P)O1*HNrJ`yR`JY+Uuq4yLuGZ_oZ4%EtRPGIJ`A2B=1O z-{t z2`|DCJI~q6l%_BbnDZ$&B^j9KEEdd@LNL$4`d@%K`%=ni_&?^p15V27YX8n`Usmd( zRGFoUwB2Qa1>3s=t0J&;6e-KH1s3T=L`4CySL`*m#HdL$##mwnJ7U*pOcPDarzV?>+b2bIW^lk3)rv*m?1Nf_X*?%=1td z9dhy8D8txmhHAz)3_$ktuunoKxDZc0WxB!0>MG2jo7?PUh^OipP}S8UvgOu_FTjvu zB%W&Y^Z!N^Bar>0d*E7r5JDl$N%lwtLOLq~Y4qcNL&tdPOY+uFsYg-mTST44+Z}b7 z5?R?v!YNPT3QI(>#HX_%m~N!sb#o9v72(=iP~ z_Liq?TI;Z{WwALvPM$bQqhQ9YTNTlddi)` znTXItn*Qw|p{IP+Ymp=iJY`GzO(cDD8x%w!IyP@}P|#Bz>}^-|<)Y*EOspN(Q9Na{ zKmRipq^F!pJ2>-R>A@`}K*DZ5LwEBW3-aHw9&8dJ=kMIH8VXa4 z^x&3_;6kr6Ger;f;an$hW(u5CUZ-WJEv*a^)P*-c%@51#w7f5vX9i$ig7`0nxx-f^ z|FW|CY~do9moWY#=GiGQFG5)~sU=fUhF)j3c%A)lymGcOkISB*Jmi|tRMOhAXSNd7 zjGsPx;tXUt%~U%IF^aOp^(=YZkw@2aK=haBt9P^Y!1esnxE@Vsxt=9oI&^eBU$IO# zlJUjnb1dG_X=y*F?9FgBw|G+}U3iG{rlzk`fy%@7$O%#HMgJ)xM3trIIbx_vw8oYB(B!G(C)IOklq}qrkV9Fb(+Ocm9(C#& zeE~Y%COW+!I+4!O>4b3(9d-J54a#&#q0k? zXhb?oqZRi$bkyjZ8kE`JZg`Y~N?P%bQ11oe zkq>pV+O2=q&5yk~vJ8abbx!))nBmpW`9>*>LZoU|`}6;}GJT_OLYbe6GH)7XG@Y%C zJN~n3a#%*EIKf^wG|9ED>0v7#V$Sz=rsM`u1agyebzI)KxVZu+Nxp4ka(F0+teo!} z#=o(iqI13%p-70l`Ra+ckK-1nD0jp*ddEmU2RM$&1V>(#FqRl%%{LBn=6rwaVFO)H z0znLc4Rk4X#8zjN^QB0N6MfeD^FLj8p)#anE@PTmhd9t@v6<7QbE*Y7Utb>@3@vc& z%eFo`=lcs3L2EEC7L)14TIT<1;9n&JV=9q!EfKIzR<^=0Iatna+XZ+@RUHR5~$_FwPLe>mNoMg-0&6Y+iP-SLkpja4{IrT4cS!NXJ~s`%)Mb82;9 zkXi0`BhXNa)Z%*1|6;vOF{5#JEEfcwpUOn%9FN>Boi~hkxGzE`V>Qb-3rT`x!z!mN z_jUXh$q%!L03>Dsr>VSZ4BFGTZ-?C_NF<@L;YEjg4J>-Ghm$lq=Y@pEGwJ)^OcEMI zVI?$9bjLroue z%qK+z^RXK9>XevIsK36q^`er!oGa1I~02yJ33F8|p#7#}dETy6J== zn+|gXF{bJl{T0HHpM_8m*_U5Q7(zNL4B527p<@`5OWu0aMN!>`=i?I%Ytzk6S$fIU z9c5cUjp~@AFw@Y9Up;-wF!Os7WKfQHr<48pk1-tm+R6-%uK8u*HK2|Q~dQ0 z!<~L|2JPUSj+9P*Y96dj$23=W@|#ln$x{zTA^M!T0Y2x{NmlvoXZ8y@d786pFH&!wlwW?R(^dt8!EZT4!Bc)FPM)H!i@Mg=G*tTB;bsZ8ugI!!mJF_n!YtDDDEwYl7c zm7MX@oio7EJP4s6nf`3%`WGFX5k|k$Kl*F=+8-SmROMeD)}pn$MG|C7`l@h8{&aWy zdX!6?lq4FRj`fd(${85_Ei}4cH2SS*L^?~O)BXH+T%!W=)>}YORBJR3pQxPE{rPXa zwf&o}Lu-%4%`?;2h}+kv*Uy+;S6?#|17%6H1R^L#)Y;^RTG%uBbyg zOPwuq9W$_k3dvi~oKRG&BQt?5e*Pz}&VRTBrrRa5AluVd2RR+6qxbS@ZQ4N+rTE(X z)j`GPc2IOLw0TIh*(=(R&eCQp&;P`G9JMKmqD()M@kOsI+HCdX-?%pa^{}r-Z|{IM zTlYa5lM{^=iWJdi>!S`TYBM)oi2p1bQ=5aMzkxRQi8j9xZAfQnv+YTTj@q;%Z*6ZB)paAG z;WoGap$+1SZE_qDq2adlwLxfzmgXmSqeT#8WvI^Z_rJuu&5-Etq0ED#%dD zybG{;hDP6kM!yh^-Vu#RXKA#3u|r3V_KTuSUt1%|4Q;>3E*q**xLTZ3N9_oZ8!}VJ z>ai_Njm=})o*w6@@Gt-&VOo9Q&nd=RIJ8MHO(Y-VLZolbx};@(pF25Aq<^;S{}}1dq}I++zC|dHws*Gc z|2UgmQ!MSeRu$*qTB(jaSyeNhx3$zypItkMDug%9nCqTyoVVn-JzlR zGwC-x&M3*<()=8E{9A-Jx0Yp`T+n3DK=X6_^$%0;6`1o)JyiB1R;WWYj#15{%MqTp zeMJvz;A!wAUztyyi~NV>8ku*y#KW05Jpy4q`E2@*IFtD#QCRcIbN4tX*vfwHwMxr| zty?p(wb9YC^?8FG?rdeVXb0!~o^d~$xk-0ZyTthrfu z50q7R?3+HAFQq0YKIE?=ZXMuPfXFe&6&My*13pFEb6O|W#KM-%U zq4S5K42Qsl`WQM#kW0cLj3LNj3=8Y~I$6>$Juj*Bj!vccDl`N? z=I0GjrZ|iCkmCOQXCY69V*v@XxcE?JpA0|$y%^&)x+Kz{^Rl)^s!zzFuaV2X%ZMEmH~mxpsgqwr^OXq4P)lZSt?xKHBG|KcQ%ghN%D zq$+^q-XtlHhCght}KUq%; zg1EeDb0I%<@y7c`2p6f$3%`)cTpZkME~5VIHH&qx`Dx0`??p%0(lqA3K3>MUwJ=}Q zB$yWmV7>_RznJDZw-a;s+uwWM#eC77g1IRLW|?|j^b*RjgEpyVP;&wLCp@R4_ZeZb za-4R=m5ohf%A3cGZ8LPhM6wR?y}#!-x*zya3L{&zsEex|NsNYvMc+d-{747|kb-$n zqG8fm(eTC0f5j0p8g5VC14zaf2lb_RQ^xg9Sq|1~@faDs!DtGfMuTI}I`x}4Vl#qI ze;~>6P8OLmXkSw5$mO8@GY)5ZUor-gXZ1jmmK8Z@le;yLT;i|)nmSs9L3p58DWEf7iu_lsGZ$=!$<>CuwLifJ#fSsCl=~kznlz{A%sw29 z|21?#%>pOYuzjgN{qFnX>SvZjF zq$IMQ-9u6OHIRh@$?i@f>(wn31r^ESSFk5Edj3_K!d^KmQX4c~D7+SJe>Y=YeFL|GClz zw+rzOB9i?*knEl$vV)65yh(;+p9PYAkVH15D8%bxN%mONn|p()>&Jg_X;mtuZ_;R=hYyF2G(GS}JaJuBOdMy1m*9PH zJfN~?hSx2OmE_Ejk;f=}@@35ouiK7N45?ZYLn?`uuKNKBFj`ugBFK@6^mV&M)KVSf zFsxo*5YoYS5k6wK%UN^p>wNv=BpdQWI>`o-O{qw(ACV;G#JrH+=>d|HQj+-j|G1Qm zxgouY4kQ~L5}D*)e}x#NInf}Z?DhWoXPoCrIU$cxHF%!r@RZhm{fAa-&*h+^79W4l z27Iu9@V%jZl6srMAzZwq`EGFd`Xv7NABS>sD8%b931z)QBH4%=)+b3hMTPWf@!~vu z#AvK+#0|H{XxQc!;T=I0<3qpLGTEZgI&XL_R+4Rw4dGN>#%Xgme1%eMb4wFVEN$+4 zT>odATb82D4K$*@=kI@7rrR7R7jsHNdYZ+LLZgVzi;=>o^4CA&RL8ap={!BCo(ZD5 zL)3~>9akLEnOIOg9Ypm`64hK(%TquDs;2^|ZtR*wb$nsSd(u!n8ALTHiE18t;1h=G zi6E-2NmTRE33nN)T|rb2BvCCuKip}kb_P-T`#<6~T!^m7WEHIOcmUN+{gPzWqq99` zs2&TV^4I_4vKFB$J!+^P4Wha@Nmc`T(<6rJkwhvPZEo`Ye_YmLbf||xwO<}SY%-FF zAHT`>f6H`+FM?mnd61Gla}C@!fz5@{PEjf!_QqtEAU03;vjFq z{AVKmsc$$aeUrZ)HFX}Rtt=p3;THOTYOp$m29o!_WhTlnuv@J@#m4%*pvGd0e$hem z#gcw$zy15Ap>DoWl5UqTmXs>H#h?GIW|e8)A((-)ExuSv#*kay1`1Zj>cq-`b!q5Z z{wXk5@p)Ka1xAEfi1)zUd=-jd>&@%R^O4Q->H^Jec9oxWGTx-2T{-s;EyYjg`;gkQLJn?l`NpA&W0B&vH3h$tE3 zpRN_Hu9ke7cM5)^zRPp*XI_NKw9z=tGPbg^%{8zHYDizm5&`GXEKQfrLr!$@8IVUz zv+xG4@B8(?N;Tt~HTdqd2o}Tfw;CIYe7Cz_8brqXG2K^)jMFKDc04kIb+R0&&;PGe z-$5XSv7bVzANcXVF*Y-(_cF7n&$v_}vl!06)&`#8EluqQjDOhF)~0A`WrE??qV!tb z)EIk?jC5=?=jp=qtp@j(23yDqekKx6ixn7uZqQEhK(89 z%%}R`wRm(>bH&JNWd6-^UT(MbH9#QX~35 zSED|oLf+ZNXPI8cVjxr6Ipt^xa2hP+FRoJ%s6 z6RW82iwv>d9e(~l-Xi){guERn1>5aNUk+@SbhYviz=d+=;f+Rd$0tcT^)C;37vt9& ze55ancP1u0r|4D@Bj&r)kN@Io4Hyyf==o7i=NM7t4cN%l;&0?i-J=o~;%cUX17%2u zx^q{ON&~xuaGjR!xfg0VRTZPEtIJ2WIi|7ay0`pn7a=0(M+YYf8ic9Dxkk{rTF}U* z#pM;1)&8AbsTE3c5yvM8x-3c1L707s6xu96fV@sem;v#=oh=>W`rd z@inx#%isSTH`q|`a#cTsV6fHn@=Oet2&RN;>8++>v%8r8q3bo$XtNQ~Kf-2@iOv4# z+l+)H+KM+}zT^@>OV!8Ky%aM>objYyWPr3gmZgN^~Tom0& zrW3`k6vlWANxIk?zUhkI9w0VgT%%1jIxh^vIEYLQ~cdk%2K z(vB6;Kf#VW#g2dS?MOlroo?{nGtD8S9S@0iR!ScRZdB3Enb?sL3htfbEeOVXn4VvB@3w-`y6l6|cr%kRT7ITRyVu|U5;t18 zceRWTFxu7oU^E|23H7p#c5i(C%O+2wjf{Q(qdg%;`@lCE328K$*4+CLjK(?O`rtX> zy}uX3uGe!w=B=8%=R%&Fa3}*)nFId3LyTY*zK!5rf!S3UpM0^DjIlo-i;^5;*C&p# zZ=(=A%FmA%R>q2y-HbV~nFBtWi$4o9NF1b{nJ9z=yt?hpU<41TvWN;nTS%#>7~RrT zS?R~5sUHoi6qTHYHx#|kU;i|UR$#Qb+{3Lm`Ytt0_FoOB&6o|gi#8H@Zxa5uI)>yn zrAs|LKtx|!vOsyeGii=#Zd8f`AqY|Q{-F*KZZN*jJ3;nKLD3V8qUmLjY(-m*qW50_ z{z?z~pP>dnRj3mqd_6uCkG}H}FeLiq!tTehNtpg*ZOnpA^ z21>GPZbb(Y>iiGOt4tG~f7JMo87#3gm{AoAZ6@lTzG&es-v`L!qem;#|@d_sW-ng5E9 zH1sVk=n^_M4oyUtDAZjpy_{smuI1ARe6ahN$WawzN0&z+Kca33oe>%fm<3R{sC8vi z^GUNdt!~CY@}je5)YjsU+{KNn%R?hjhDIqmp?TBf=9Npog4+75ZgjT-vW%+tj?axoP~WgR64{~e$Q`ZV3}D^zrk1gzDl7fmXUb7g7fjFj2UH^#6Q*U* zaZ$Y1gm}QJsLxcqVa`W5|Btzc5S`?2xNTSm;Ul?iI0L0{;Enbm>lVZQQGfr3cAp_k zq0td=eS|*LbPTVL`0fo=q*9_N^{DUvja04!{m84AYjvPheJ1VHQc9ZXV@0qTo9V_B z%~XcB$A*brH?o-`hzzUChle7-7;B^~DUI|o|NP5FHc}mO9tesQjr1|b|7@fi6C3Fo zutSADcAHb76nw>S{KQ5&CwI8Yq(0O3efltm82@o6{}HNz!@ZNYW@+CC~B^p3R#m9eWjf(09P^rzo{>65Duz*zI<1c^l`9lHQfBa6R65b>#oZFw4u!JCXmJ2)24$FJ^jH9q{l2x8uXoVCO}m!O1B! zkVd%E&;OsyMo10Xim`Um|IrorrJ-(cjJ2zMtTdXe=FK5GMU&md^B=6SlM`#~FyMiq zclr76O&n~%3XJmc6Ps*bv%K8XUS8J1auFhNVR<$RDKgrDH<|hEUo%sSkd`^!I~6=x zc^i~>YMybv<&_l})7yL=Tm&`To>_(XqlP085t@~mU2Qk#=89n!ldAmC{&G(>NmJz&QN>`s_SfnPMecc(ObunlZ#yRKd?V z8S__mcZ(NJt0oLk!}~O&+LM6FbV zwscA)3k;g44s_5k`&Qvy6VHTPgOLvYbaYzUCZH*yT3XOx5$4B#rjaufQWgDB)q778 z9x~-0hRdjiAtcc@A@ujZ8baFa@F={BK2g-7qFpnwnY*%O%!WEmBke{siwvTt{q=8y zh<2;=t`)l>?R#w|c1uQ7LhZCHG2PS8IQ2x+jf#E*(>*1o`^YyP2}vaB%BR0^2x+<_ zqWhyUkPex*{WCG0R4|vDjNPDRYP*s9mNLEsinO<9=>O@h<8~YAT_@RIWZtgJ#BK?2 zN~o9J-k!n!XG<83Hahxe80~2>+Mj)+k&s3^18;caXZ`pu-rl_E0dlq05*0ll6Qd=p z+8V>*&G;SLIzVhO1TH^_dth1*8({r-ez2d(J=qXly{^jfwsR z#(PGL_ZQ!IBqY(V4ASS;I7m5lU*h4Yr(U9YM0w8_W-t(??RyF#9Ix&w1Ak>_1)CZh z8nG*?vCSi+Mn@Sac01!gvZ#6L9c-F7~%6L?^dJe^S6M1 zJYq$}ObfCABUV&wM||iPL$xes1^3|m-$XFh5v$1xcJFrKE__2L%E1fOu*U;B#@wd_ zN346A#6qX&h!q;N6|wHQG@&%q4USm%?2MIW#7f>YLzpt~vJ&=tK0--GtfwSKtdbRc z!QcO-yEIsV5hGTbJBwLCjgi=hF%0gpte_5oDJ{Gc|8uf}FWB>+BCdp}YyWiXzwv>( z@Uwz1aQ+h?J?L7NdV5e!;2AQ#LOZ(GO#e@ZvvG8&F-7K#uPK22KVGX$3qHfUi~5s4 z(t-=&=u+g9X!wlW8pTp1JqyIX+}#mNxAvg7QtncQTU(i_TO)t+ep2{va`-dv_5^m- z=^5SaYD|GfUp{u^7z{gY-chRyrsrJ)%!JwBkr^1vO}kMYNs_!W*;WafgmJmGpCCq( zL=wd5DTNFb%%NE`35FVZ{QqIfQG zQ6}xcQfjPG>eVeV_RH}No57Qc`NlHlUI*xxi3vmeLXHOIq5xlHA_LK@yTOP9(W$`$ z(X0OXk5f4i!T;u{F`Yu?RxM!X^AeT1y^^%PYlb>&+Vj4`XarCfli&CFG0cd($;9uGa`T)Vp4;Mr{E3L zulI6Lb3m*eCpo`LHO^FgdZ2Su4)rBtpSjcj!)2+Nr+QecCy(ve1pqDOYbPizj0ayzQr44hwBy<$qIWh#`048oA^{Y&I<;-gEb0s8zB z=Llno-4ipb<$S6^);@cG|CiAeSGcGhUZXra4KA@!*rx>^XCZv{V(DK#CYIimXlaB+ ztcqV^|4%H2HF@L0Rdr|JM;4IQ@@6i$u(h0?qP4UdMsFU3GK|7c*LmXzLy344jz5V^ zGt6?{_|2Ow4T`z3QjOm4hCk!O)`0mBYOp1R26SmK#hZ+O*?za^C_I91(eCu?MDt-E zOM0tYLTRWQ9EHDiWUMq?W%mg91w`R*twu@qjV*~a#&j=WzQyxjgn7Eo8%M~9FxfX! z=8YrxAt`Ts9lj(pfB%nWhOV6x+Tv}72WI7+j!T@42w<9kI4h<+L(r`V1MlNxUf)}VVDow7Ceo`X z1KIGFG~e5Qa`-dNR}<|fm-ivf*R4qV(DeA*``0BjiKLNMCX|In-Cz6u-!w~Rt44V@ zsQL(-C3X|vkn^bWh~=*O6$JWFxND}Nj=qaKEEFutRIMAP_< zKmHq-v|2QZNvghB6M4SwnOH4h+qy9vsx@_j6aeB0Qo?BvNdEc9xaCHB-xJFr&-cAd zESCaO3H8&$Xb0Hfw~-{{jg5W+<2@(F`@}aM2}!g?2t?UOm~0N(B8bMq3?^ot_wvF}#E&-o^ZXBL37_U=u-K z*!9#y@eK<|1pO}KKgOTiQpBIF!e|-FFoNEuBk1z~vk3ZKjDIl=@=pO8_3^5qw6mqb zdp!R^4bDiRfke>n4MiCaq-W>|8XB|}LBHqk|Mp9x#)2d0_s)s&W&};%0TJ|j-2cG{ z`i#UHW4;JA_MU(L+cq6Rmy?knLH9B{FSA+D_lp$*_6igb4VDW@X@sBsW21^90?4F?7xN zhGFZ5PO6@*T6v*PISCRoijX-SqL`(4!w`S$7Zc@2$R|=%7(Y*;_sD@+`2Q2tcb7|< z(?srdF}Z$Fd_R5A5v%}o{XM4Zr&j?6r4tnY(O>^H70d{wHrh)mea$I?;$CK`O^-h! zg4pR%Gt7npf?|M=@3pXk;`h&U)a6sl&hqd$Z~fFV1jVzA%iT9PB~;7~P~QK_k;W1B z(CDWaVfWCw2#P-qmr)NxI2hl_2>YiBhma%eGVet>tAi2t#lj5QHAaT?^zld7a?LzF z8heCoXc}9MUfT8vU}ydIfBgKvX&n{l_dFh;}pi#&C_aJ&(OM{R06AjKup@9SeANl?N932EegSLWz zkNo_nUmBJV4gx;95b_ydoWm4zck&Jh0zP8=%SLrhVvS7!5e)s&uY}dvx-EEQZlBqz4*&W8!jEKpA>LsA=TSCuwFtgpn+I2}XBlhoOe7N;G*mY) zZmJylE!#1jr`$%seqSO30y|#7y%q!KM+O`DSY7;zyG+{WK+Es?SX~asFvDLCjY(R8 zZy1A|L`e(`Ce!KVz$pCx#_#?#Uy77q`!DxGEM395x|MdsFXqMHr{GO!Zwa(>^?fc| z;RGJ=jx<-D;S!I>&As8H8Y`=st6JKeb`k$NVJyr)_V<6qnLp^&nw#Fh{0YN+WOY?z z^QdooHYKAbgW6~A=YQkOf8k9Cc{@=I%y(+$&5K7iRyU7odpINV^$E;BNixer9p!M)s+?1)h%rN2KkRwtiasEJS~E}Up_)W~k}#i) zPgLk9^#3mNN5FKK0lrJ~t{zo6wyJs5x7`D8Qo!UZy#LDe5103&5py9SMC`*OYHZ$% zl6XJX-dvFg-urNqK6%Tq_18zhn@;fDL?EczK@=@%#y zvGix*G80RakVIm?Pd7V+wBJ!tjI$j{2m2kkkM_f*STXydic@a+hJ^Su#IoOKog7TG zUx|mO7ak<`+q#eTi(@LGinjee+wO>@{SJ@*9rk-s?Du!yek7#*PQx2@{n?`qA?9AS8PbT)mc44$XOwc($&!vC>%aM7i&GLWu{h#R#wA_AtwxE7)88$C(%fxaC zXiBJ-T7b8-xxf4Ke~r5{BqY(Pc*ACY|1`;F6QdEOI+JdZibgWA z8J_G4n@s>c4VD{}SjKPh)ePd0P~~&C{nKLYap#)pt9V??8JSqj#!^C!v=lMd=l|nq zO>=qCf52QXi@E+0E;C(}ge0P?gSq|@aR}MOj)|5jb4@6c1ZA0+ODZ+Kijfy&F{0X1 z#+N{0)f0}t{u6Iw9XvcK^C8 zgf!Y2cmw4Zvy+TADcTnS9Ow4TEqR1QO;y zdW?@7t|OnAs|{DnC+21_ETn)`LjANbX>nir>tAu>MWbKBc(01_zVwYpLK1B?NWb*= z{~AKJxXICen5U5rcNq1{#CTY3@ds5r6L(gvv0lNq-~B`|iw&T!{QO_sbe)(W(5A!k z&smw6E)}2>s;8xi?Y_FgsVw%^@zJkfyVt~aU-`BpA&Iu&jfVHtOAaAzHznF1m-Lbj zi;n#>v7M>q$y3Kq(ocCcrYrn*(-BL9!SnS(2M--wXI#N(uFXQLJ3AB8rNC1{!*U&C^*i{-xdEk{CH?rekfpM?%mTCOfS0B1BwcY=xz$i#A8R42B&nwhie zkFG_>jn>x8GUfx9Z;wU@sYPJj=)j@@yWkE%6OCZc=Q)`eFbGx&71a`@{r#(}BbDv1 zHu_Ik@eQ%!KYc5bkVM-J+<(QIM&4KllLc&*B8 zKu>|vLsMuHzlIiN~9YHG&eb36;hlob~;knztz(+i5UmSW8=$H>ZPv=8f$h?>j z<&2pc`17Chbk-gkw2`$h4*TPuUmEHLX6=i^t>8w3@(+pwCz>6E4Ov0)-yaFYe zusV;UZLa>={K%DUzzjS7zx#AD%EwPM_qy+VF;#_sF%=k_R}MbR5w!RrG*5pAID7Gj z*$qHr16>aGKTO1*dMilYtNgWJXm|lWP#|;$K3QnGSi?kMbS}%im(5{x93u2y{$F)2 z(<8`|CtXcdT^>OKJfa?!tEq}$&{XHZXo__1no51GmASR5B6j~&;w@IZ_R9Q9IqoG` z$+uS}T#JPsOby^A#9Z_R-m^5f{?jM~hsX)s+&6}y}^)pxD9%h1hnv9Jf zM0IN3OYn&*&vWyCJfZfihnsmcf1FUOhaU{6m5jU|#ZsiGmFJFs^m@Y?8|z^VYMo=$ zN^f+cd-v6P(x{c+(=i*5COuUxd&h5A)dkvUqbkeCVpM8#u|OA04@Q8QuqZZ7b4hPZ zM)6uR-+(5H>P!`tKCxB8!f+VRlf|KIJDl>1#@WD@MdfX1ydaMJfXIi z17aWgFU(drAO>idc0&-UlqgCS(EoG!LVKJLPpJLGYf$w=DHTdJWYP{SrB)lI3i~?j zdH2$lBjE4IeYP55n5Sp4G&>m>Q?{sAc#9*y2EOA*-h5TxMbw+02?88n19r{26FyOA zMcn_-G3o5YF-g3B5%iCFeX{}Z54FvI0kt&-7aavI9D^@NF$Qzorlb>4hEc=?dhNdf zHpCC?%>T<#L=pU7OapU}edmg?wXO!mMWVrlDKwBMqIej}u%%t7qX=lwRuqB5I&NvG z8yrOx`{zF{)KLU^2SgFYT>oW_U6@#77k~(cE`CDuRlR)h#;QjTAW@Bsh?Okq4v0l9(^;p+7dqaf9GmTwu@B3O9&WY zDyUmAW^r{zWmTKUnZSj{X77W+_sh3i3@+;khpzNqm(5rhN?$iVCLP8mLO^9{-V23) zON@UrzNx|Gpsm;`!2liCV_{E5lEJA*I)fdCm@eLvU_(hK;7x7f?SBQnp|oX`#&(Kf zloYQBT~tQfMrKmRu^modo*CY$tCOfclR#*pbXNCVXH7H{32>mTg<7bNz5>0j+% zvHF+s)&swPJ3X-Ft=s&0{*-dX&r8@jEe$$w{}VO1B!vdztvmb%W$3Lh(cT&wwB@Zk(*N6~p>D9Z z?l>k^dKLT~c?WpwjtwZu8oMO1#>{r@;*Q&d)y3LduO=gUYigWwyS5QF3qR}a+85&! z_0@5=%|$Q$DBlw8T5a%SB7$=>2~t_|HZK*jVDtU`+pt#e+~>|_Sj@W zFojOT8+FjxkAHPl>is4=ku)+}n9=)RGQB^&IxsK|@6E#hCov;(9>iSdjObvcda`Tc zktlb!lR^^5MNL@OEf;x}j+% z{CytY6!#yo=MFuuBs|^qXio22(=D2twwpu_R3-LjnTRPjqDuYtzeBf4iVCxrQ?;&f zvZ8qQ3_p<8r@uW0m17#|bNrajqAv)f{~&?BD5T|6Rg1nuOo0Mde%FH%=nFz)Q{a!& z7a973Mts85UE}({hHr>Jw|59|kt`_2v!_aMV52?0?1)@hXFPGC6yAP+JfWu>8f6dF zQ}x0H&j+eLuyC2`tNMXOK_^w9pPGc{oQ*B7sP+>g6^lm?YaBHa4=~^h_lu2e7+cP# zTTPo(JGZ{BW=bva6m(OCnG?^NJ53XJS4HU)^XYxj$=HuXvHK%IM%R|kc`drc#D~G* zcq&d|r^_JmtReb2j~bSw&%r+Esi}LY{W7QaA2ElM$q2ZBeiS0f@`|x-R~gO>jh{Ms z_LRC`{%853$}kgzuo_GCafbAnWNPu1BF zj%jHsuW0yIeG>qesV?c0BZQtgYsSR7qv|J5#KN;_Z+%p0`W&!6Y;%}XGkLad?h)11 zFlk=gQeLsRZ7r~=22n!|x2n)|{nh^I)57L4)Adycq)*7ClB0Y5B%^;n)lG9r=YA+2 z8B2JmAMP3Lo<2SLYvKb4acyLe^clc*U@(jOqjYE6U-itGIM&xnRWC!F#sh8Pg7uS@ zxYnp8cTv65CyqUzRFk$>2c}PoisO+$cdLYE(@FJ7pABmcY_P>=1}^8&^d0#qTlCp< zF68IvAoR10%42ctG&y- zE4?ec%e|{cBe`49M-A49_Hm3oICJ9o7&|C^;)F34*FdoHzQZ7kGyQ3E2pDC(8`Ecz zVwlAxAZa7;$zy%Ln;NF6_j#Nhkr8>quyvpso<6xd&JN9pFg959RF#HsU&q<<^vMIp z*^2Zz1dp?$43m8tXGf+_>yEQk=@Ygw&f-pQ?J(203O)5ei#wXrW{`TM#r4=}b4VU) zy=yY2j`g=vJ`_*O-nYRP4_r-~I&rkcBOueJZELv2vpCb{lXASpLr~LZkYd2avl!DR zPabhc8{*7I-0JiRQ;)bDZI_x6x4)nN@$(5Gb-Fi6soWg=K3RE_a!r0FjR|YSnXyI~ zN_UH=36Mt7G9@eApZkB94B3&G4B>(dX5an)B(ryunBwfatiRmhd4%epv(IUx5~iQ5 zg9gO|ra6+hDQ#l(zu1)aD`pz7Xa2w8GUl5MA&JeVwBiB&`j>&oHJ_DuilD;!q~4VF zYM}<1&YD05+DN*l`|1p6>`iIaV_UHNU)ARA7oy`de4;7_u5k>+O=&rJ*m>-#vmIfh zXBv!Y$1b#y&OdTL82P{I@aN4nSKxM=*v&OBWK!7(qS7kmL zJJ4CoD^g@J**wAMp#EagD`FY?w9eHRAGAOqUJ(HCpwn!KHxS~^?caQ!0P$9VcvT9B ztpf2`l;LX4ReH5%3zsAQPgZLNdzJ>BcR;9m;4hPyw6u&oxLGu~I)w(ZS~J-1|5xkP znl0qS1S4L_*Hab`=K2p;Yy8qsH+Z#X@cXgSNGo^W0-gb@H3xShXehYpm!Z~{LSRlRrKH@Jqo19pDBdtNchnxXxZtk;}C|#2E)P*l$&AaH5@Ri2nB}_Wx&WBN(mLR zV}oJK9Z8H0j){JQ*Z_Bs>e%3$aG8k>NJ!_q4R2U$*bf{+T5DEx2%h;!Iy~TLNG8@| z6gRzoLJiVvH8asMXfxDi8uw5V%_6*kXt=-rtK9&WGs2;@JXx-dHUYN)nNc=9qa`CM zp>|r93_-(x=vaWJn=~FR#rw4}-FOu)Go~XUiOl)U;^BK8LYi)NbSU;klMd%Mhh}0r zgxxTm&QBV$pNXns9J_^QBDemH?-OdTBvov=RRPe^W^` z+T`&e80|M=w2*H!5|Zc)yn(XZ-~Vk0X|y@fVM^(Jj5xnJEEA(~6=!Jw2cd)RzZNT0ia{&aJ z+bfw^Edfgjm9m>#rGNfCZn3)YIk4F8#9}$V#Yjkt$zV{q-9br<9Ty#;R2k_o7>vlo zVp!H^AH_7x1HZZN_q#T%73lJO50^{+{>YCc!Wj1;)nNM1P{IcRn;w(ZC)*tn6|8W0Rkw*7j%7+ppD%_`aw_-^nB)m{Dqa8mqds(ld8NVK__(! z@zFWhorb$U8OEfgQL0pM&C6gzL9!F3YHgCRO$UVd^lpWlXm+PBjJ-Hvy{hI#kt1`Y zDz<-yv)3esvp$$_1WeqmoXn@!jAGOCl=oUD9mH*VC2Ydhd9~tE-2dx(l&Rx$(La7K z{Ug`!A0)Kg@u=@RgzO)4qnKj!B^}&xc_#hC_`YMO&X}YZhD|3aXu}=Hk4ByWaSeE* zpNvi(|4-qDN#7m6o{8O3Kq{erwmTl}`hSjP)5hn)c<+eu@_gfwkjCSLmq_1DGT!mg z3ZC4-?>HY>k%{qQE^x*yW4OYAV3W*ahjEESdfJ%e@&8os4OKsq1aQc2WMa1jG$m9^ z3!?f^eN6KBKYe^YY_?Zymhan)ge2Nx2&*{M} z8l!1%~1wWT-alw*5xc~j-E#Ura*hE+8-VNP3K z-ca69RZ-bkUY^11rx48X>b@ld8asv8QqeLBJ0jb>r4Ib5qf8FRRXKX{^67WIQ{vZ3 zof6hi(qYU7(S(jHJ%sn)AO|8@s~v9E3KiQC<@Af`nk=3xJ>&~3Yh_M^?Dw;0 zMnaU8wK}vX&@mReHaKf_=n-O}YjxHN8nl(QI&?)sX>^R>tkt2{#Y(S2`zCLiAxs%~ zS(_mbeHtY>%wL;0%wK?lF!Z5*{$CICU7+kvl5xt}Vn&6Or z?mawMy#8=D8*FDjRs38UjE#Fig=yEqlp0#)5vMx*d2aMMZ;@Q$k8`7oFsyOKdHUpv zQRO&iSqVpjgLX1Hq?1>Si={|kt-m{B>D2~#KT`F27d6!SQKnwaSL;cm7LL~1aW2n| zK0Ou+(`vV?v4}O{lSemK<3i@DHg7Y~1=Dk`0cJwdcbevsp0Q9|W{rSqpBt?iXsZ;W z6Z$dl13*3 zW6BowqQ7?3tAU&Ov9}<8)^|ZB2yku`*fr};_(YwJ9}D&zbFNApb2x;u0Q8Ufb#oHw z?oYOJif@gv<4*?{jzBlM7L385pi&u)ZtqGSUl!m&&QNn~> zvC@nZZURq^C@G?Z34caO*4Pb+H6|yKYD_~iM6aYZy)AFshd8Dl(?X!mpk@tO^Yy> z3e^9`G;YQ>Tw~hYLDhG_-+1U6Hk-F0o?bj1w+d&L17B%mZonr@bmWOKxhwDu<(@{l zthM7gXyw3QhAzt>2w<9k@co~ur>zKPo?voRU$lpD@D<}A(yJ!}+3=Rk!%_2MN^}54T{9L5-%T zZ33DSs%2*$j-KNP;}m-4_(Dvf-=pp&YJ6e1jMZ!i2je@LLLYsxL&!LH4e}%AsbQEx zzfq`Brqk{+P=_5HukJTxKx0p#o6DQ<{H9Uga^TpWLQnM9KTX?XX!!>`XRv-4?SrrCm@cA6bM4#b*5AG62-!d);o zdWdoTJE#_A*2%iCF=$0)7(Qkv_|vIhpXk&%M!^!tApaK|#LUwr#yNm&1gK@7bDajHBtXI=utI|Sk#DIne_ z5MM+Yj3P1ua_JuyNdo)oWW%Y*^ns;>3ovBU)B1 zZ)#q))bO^YL#(iF%)^{LvW>z1xcdFQZh4@H(^M%Lk%_d+DaT2r-w{yh zQ#M(ZuFeIeI=$q+jrg$I<&^J9rQe&P(yLIGke?99_p0UaVLU&bgM)<(SqfArZmlo_ z5Z`ph&BV~wvHi0b-5Y?mZjgmG^fI`s`p?d+$A^Wsj`>gaqWe>z-Hx(ATZj8Ew1Mx} zy~qRFSTAZ-1toF3p_G3ryGfOmu}VT4_5auE@}Ws>P(rV49liU}B2{ruq<#)CY$wQvwNTp|mr(NZZAkFI&3? zn^UPOu-Y#40CJL(R8|M3UKk^1Rnj=7Dr-@dCH}+ULDw)KuI&5Pp92QKgz}7d8YYhw zZcGx&%M(G;LZ|){!dU0`2h{nro>rZ=>j!EzYuO_YS>u%yMhpyYEloMsMl%! z`iGVaTf-IfmNWwhm?+!i7ICzCyn<5yWa#~le=ReKM!^dd6!EgFd8xGFjhSanX za<^qqHNt8qfFtZs3lOO}h*H!2^-qnekPa%wrBvUi6ho!k2+2u0K|tK&fS^)`VVb3L z1w0(_W98<;aIZI}f&*)WH>Fe5m^UAnG$ZEM5w){`67x2{?_yk*te6%A-! zuwDiDk*CksfBu_kt@qflhN+RZ`fc+iUh))mXTE3SEqvGJ+lIQKL12MenN?D14}m8&y-;Pd3G%RmkyMCSN z62%c!smjxVJaxl*3$w2>YEzp`tUjCLKLZKV$fl=hb(l?CefCv0IXZVp-R8}cM%q<* z^Q=~Cd>?C_#6E*xISkmy0~y4dIf=z5%<=a>83h)BL993SN^Iuh>~cZQ7g>nUoa4v; zhRqt=4m}fFwWik%fZC9QSaZ&8jy7dBmp0n4T+3ZN=Y5+zKLYXros=1XT2-fZPQ%Te z_4WYxBzW^~)R9 zt)I}ed{*m<<_J;>#T8X}zGKxG-24U3RTljy79p2>F_gXgDv&t_ZDRj8&-`nxrAgNy_?!wdX}a&N%v^fH-;{=KpLq<{ZvV z_)E3>?B09u(G~AfyvgMpaA6$%Y>GH~l9c(e0DV@^^gQUCtEUhnz`fW{X`_R1$11v(l*xH_)<+? zct6c&8EYZef7psXNCEdEVf28&{UDY`C}8Vm4X#s#RvN+Tz9HQ;Py#PIji7yv8Mn(K zY%Yljh20DA{kJ6)h-@6*P_urvgN|(+c{r0r>Vu1y;y7G%5X}ZeF5;W&{pX(;J=x4- zoJv)pMGlRGoTLi`h`;~)gP6xC)FNuZB_N@&yCv>HT98mEOqo#Fy&x%}u!#G=S^FOZ z)c&I7R_zyl0*LCqj#oc`53BYU?U364I7RKtOmWeDV&9Kh`+BB$Xdl^UcNp$XiQBib z?7Peq8~puWMouSK|JW*3Kb)!FY%JEG`7VhHeWqcOh?_&1h4?C#jJn25MwPmunkfxp}~sJ z;$!3H3)b@g?D^eiif5_q|3@>$5wS?=u$pzP4fv;yRSuiDvZ=KpJJU!6uiBncC5?Xn zH#VgwNvBHuSzx2z|24UuM*fg@XX_S>Y2MHSRd$>*aw2Akxl;oH^XL)ZHJzTO+FXmjXR_^)-8ME#{v`Fu{qyVd&yv(nfF}MTHshZ$PUK`WM zC4(IaHMaeRI}I7;50)Sm!;nWbU$V_%e2>kzt$H16DrU(phkP2yy)v^g6cD$px3kxb3fVuw)!7|kCe54-BA#<+~)T@149F1FOa|H3r0T{f3eH72Q*lAMMZ zhc}wpQh)v5kW-tQzOww^2iJ-@^@wx7mgfx#7hj#Xlhj#It?DaTtf#7WIZ>UE?lll* ztJCWEzapv=<5H@slhh$O)#-#c)M;G^5DbYv4hV@_Pq#uMbrY!6kX?7*iVrI!YW+UA zFd+UsMMxxb!`A0S-Opo@UXhOUM#dvOdv16&=Z1&@Ii28(1G1hpw(uX$+;CYxQR4Fe zC6-OFmH3=WeBA9p)~Kb#vQ|;zixf)8V6n_!|N25T%iJ)}kDGh(!cyN$#;13fJ({4| znj0?j^B;x{TXKv|X9UeLHmoCUBx`PXLMJh#A@)FAOxv*C@MX;nPdHQz^F@GRPFP?W zMhjBwtB#uvE|y_VVEDnEMqOdTi_O@fP0ZJdR^ca#pZ@3MN)(7RFO3|w46XF z?pai3bHm@1Wi&TjA^DHK2HkYDulyoauXjiH!M(b19M+#ehn5N@?788JhaElH%wwEN zRgvYJpplT%H_6;^1=oLBt3@%7v7Z)En_Cn(H{7e?X-BaabHk$4bHm>x%?(#BfiTv7 zQ9$jl#P}bpeRUIHs;)O&&1PcN{>q<#3%W|*4k>D19O24WMXe4owc2a7M#j~$9pRUB zgix4$hp(6;+{=GBj&K#@KPu57K#5hzf5es0jTg94xQ1EH^(3-M_d+jYrxFOgi)7%u85zMYZm3`drI^?uBx;yF( zMe0-8Bw5apx6d;54|4E!?@;#l60>Bbc}$ zxLyk`#UmQy1C$XAP&V%M(;af+&TogUbOv`j3wK_YaX;?}CT<8e&j2k|?JV3zFhJS3 zH?aTHv&7qXL5WQ6>#%h@t~UP6W;@v+^FI+x+z@P@;#aClEZjygKsmT&{%7GX+)tJf z>(qW0?t<*%K5?95Fye+_b3;_A+Rwsm1Ot?f`$Sp)ws04h{RqZX@Mle9xxHncfmk>-F}iQq{r2Z3F|9 zgZo2=oVeRzh}CD)>r_VzcX4)cpS0K!OxzG$uLYN?juvhsnB%_XzfZc^At&w<%o_9| zwK~OJ0OxzG$uLYOl z97x;&8^HkO;NI*Mdvc{uXW{7@!>7pE~5k-T43{A`I>WEZm*4i+l4jM=)_iu-S4|st&Ml z8^Hi&j)-p2(H(HOI0@uw-F3bHty3J9CG3= z?XHx5dTX8PZsG2dW!%4Z1QRy|o5#kMs_qtUBN(7;+^2VU$ceja4|q^x?;aNJ((K|s z{ZdCTaYJyu7F?=&Sh$T~fO2rZ>W~xn{>WMCgI{&3r-i#~c5!bx#t}^15NuAIx!d84 zU?x&6f3oFVhn%<%=%v*D#@@Xw-1}!2_tvl@n7AR>+!yVhH!^|&%EpbO+4itS+}(OB zrJpEWr+Qns56Cj^haADg4Z-Ff?NZg-Y9~f8KsmU-cF2jlJ9gdb*Mdt`#Im;$3{W=i?PU%*arY|2 zGKj%lX5j|cY{s$MuXO|yHw2qoN=j9kh1&=QC`mIy_EZn`bi~Fo%M=)_iaD8u9bgAlR z;WmN+%Eo=xM2DQX`}CK4cI#As3-^Ip#{GmNn7ARhUSIHAs`^{FjbMPXai1M>$cZ~L zK(4y2Qv)pAeX@)D?2{eA#0|mq`i|XFHNe7c1Ot?V`xg#5ahDBLDq{NOKnr&yySUG( zbOaMO1e>dHO4UFMw-F3bHtuuQION3LcaTzL2KOKfcUhKkf8+=zZV0Z|f=ksP3%3yr zP&V#!2Rr1%-R~f|gR@Q@WZ~|cUEJs1;s_>g2(Ir-gO#d-EZjygKsmVIb;yak|6sZP zvQ7=QaQDkD?(>d!1QRy|*Y`8H2V1y}V1Tl5pLdNzPTT_ymMbXh)WH_+{@KNSerHE8 zaYJyuzWcIN9c21skQ4Vo!;~6ia1XO^ z56Uj?3&%Nvi5r6Jwct`U%xWh_FhJS3FWlsi6ZhcZN*!cy54Uh1lx5tXIf98Bg6p;5 zQe~eUGlBui#(hzRLr&ZWBg`FaaF4KX56&*`i|%v;6E_6cYr*chX(O1iH{Z;+VeR&! z4?)h=<-RG_cR9yEnTz)m5q)`2RkyvoMJiuK@MG*@-7MJyZMTb$3Xa`8H5E@)v8;M<1O(r4b(KoYW9#U;HFR?x19g6R?UW1sQMvnhNY^)>U~-;LD{YN z5;^}ETYU=ai$qeaW)ICS?j3Ufg@t=qrK%rla93Kmwcv1}i+hK^|G{6S3O=EdaJb0DeJS_<#G4u0Nx*9M@GRrr>j)<95vh-GXu;uP7x!g-9CG3g zSj`@hUEG&l?Fc6B@=>aOglQ+EtahRWhubCKe#;>z?ts3)$EGw;=ZELF&OP#>F<7a_bqF|;r(3PR~+S#)7}BA*_Bzw zz0(m)+#~&M&h7?fEjZjh0rxi!IdKQ9W{=D+?ki7n1QU1FSh>-?RE@RTi5490;Nreg z&i`5MBw#hWD!aIGg{kexhOl@UaR;nskIpXct8a1y6Ze=yQD=}798%JfcqVX zoVWv4v&Uo?_cg~kf{DBOFu5YXR2^pF)`G)bT-?`O<&YD1z-o4Nc5z?Z(Gg7CV-J^y zR+g&6E!4=NVL;>oYublp7x@i$If9A&aBsY-KTMF1w~%Ym;qD2@|LQOx@_<$C!?TP0drKX` zM1F)fLGIE=JD*@7*P_FmWn1n1dv`bth&*6j`-to!zp<|)n8-b^R@EONHm|jiYtiAJ zF7g|3|A#g75_!PNwwER3l>h+Wa--aTXT{<*-jTS4*AwVRTF|xZa4%4jTAa&G2PQ!e zSlq73F7%t+@sCE2dPe~|QaIW#9%Vt-vctXcgAF}#{VyD}z8%dj^qZ5{|HgYq1A0`T zA8kR`vcpU&S9q!{oza@G6?-@zC z$~_^=(38jiTJIP@pCHhWv7l?&__t&ATix|fc2lf3r&{Q)%`Ws?&v2@nZT(1Z5}?-# z^hp+UEjtWcc7wU~X@>)$2ds4;nO*4LKim;a=tp^z0sTmUKG}k~KF9`Vaj4Z#)+lu;P7mcA@`3?*Fj-`b4h|(2o}A zbry6jJKR43I-dWOFeC*mdQZ$Q^xNF|FB{A;-c&%JDA1=`(6#LF02lgg_as3NSoc0A zyU>4lup^kzCwbEV{TP8h&4R9FhX=aQe~9~U609Du^gSud(B=L^tHDh6rUUvUfj-@W zu4RV@xzKM_Wd|o+FqIrVh_Ws`sV{^qCfPEjv8eg?@+Je=PMz=NPaCUYA|yckZ7AeX2JL(CY;H zEDO4p9X{BFey5xNAoPG`@Tpmb{%R8RY2Iu=pDNI2ThO)a@DLaJkGeP<2t8mWd|Gy) z|L6ioFdNKtZw{bO6X5^=D!JjhIcHWPZ#LNTF|xZ z@GuwpkIzkl9Q#$ykGFD_+7mJgQ??wvMQd#Rq=8#Nwq3| zm%sm|Uu;zz4@^KqG%wETJK>G#>0QkKa8zgw-0+Cn_cMlB>TVPJUnd9J651Cuo{e(odaC9)$d94e|Vx^amw@F$7L7#&yxM$ zTyG(uA1BZkTF|xZ9N@B{|18n}{5etdSJ-q2K4`zX^Szho^OV3j}(j1zpR|>5u^Z&yMnh9_UI=3P&)ZFZPxI zdV@e;VnNrkbGUwPS^dGp{ZBbT>*tHJ3;h?#`=1)UrGUOzpf9zcYuPyr#VzQ+xYSXe zRu5Q0Z_FD0q@ z;KJ$Dz!cM|Q7A`7YX$2-eTmd+V9if;n_ixCn_e!6$-sxp{ZE8DFbM9iY`B+!Qq6w# ztq4ABxc?EjgHymg9AyFbkurfhm~gvszxNOgcOdZP-#85_HCnNCN@7KCl$o=@=kb5BS zgg9GI7_U~vC2_ni^g@vH71~!VA^@Ls6^H7sI4G$r-ld=X;q=9`bYCPZ>5Gs5OMq7d0N;u6 zKL$KZ@1qXC;*x*h!vemuMu1nP04{xTC(l1HgH)yaVg&UOONL~(FEXhJOYM9!Ca@U| zfPFE?f5?dQbP#=OBJ&?4Rf@!@IQ^JV-4jtl7aJo*{q;{%lr6rKc2G+bZK$97SD1z=0s zW_iM2|I)hZfnb=Q(emX;%X(rqEnW=UnJ5GPtlp!0ip7fKNIPP(+lY5 zy+T*~mT((yz7s#>p#T|(9iIGBkQ|6=(mne(WE7Qc=j)% z*3q%9I6`;DK}lWlRed|W(-)sej31smM1UV10Q|Z67H|_ke0u#m=eoe3+bX~(r2sB{ z@wp!e@JX@wfti23B9Yy`Ci*;lOi;dwv*KS_r> zbQ}lxj)MXj6^U7%A7;zZ@xv0|ML8Gir!vd)%N*jpkW=ngqAy2Ar{}LgQFe(*0WItK z`>mE`;)mXE-f;}rSfkVPzX2C^iK)6}rHmh*_w)Z#WAQ_Qxmdp-RG1=u*fT{*X899pKw;)iY_?O!_P!M`wW@L$k~L}HgcKN0EEgQRo)i_&MO z=u8~`F|zOZOr+0_#Sew4;)en!e&F&Z83>XW#t4$xiAY!hkn#f7KPTc(GwIRs!-L;{ z?M{4S)zAx@1@3Vv;7a`P!p#ErxR}$b$SDG!fmqpdk-W z@m~zLi654f^k*GdxL+J4xaXz7Ek5hTxq^FMEPf~i>`(}-j#Ncf{D5&S8}Y-7&pQI? zvjQFm0D(fhx|ayvt5t~=Kjg=QhEV*^ebz#Q*{J`&Ha>)YP?mZ8 z@X`y83iPHW{!Pk#REZTo{6FTt15Sz}`@ee=EW2buK*BD9iUc>YK}CCeSrr3uG3&DI zE{YN)DvAhV&f(Nk&wx2+J;R-XD1s=#fEjc4?u_So`qTgSTlH$XdU^(X)cbt?<7b$4 z=H+_z>eZ|2s;~+$1~j$vhxI1@>&9b1O;8G-$*gTjbSr;YKQSSQ$sc+t7WFxWY4e9~ z0_DGfBNwL5A3llX59>MqW3W6VjZ3e8-*RbPx%~OjC*Om}mP>D7{gaVnMhcf64p{W* z4MRk&8Hvb|pDJ>+4Mh%4O-Mi@$A(!V?u<0zZb1Ee6yR0l$TPJ7E)!_vW7p@i+M8&`%X(OF4J4s5p%;)X_oPG@Ct6=r(DjQ@d|dUwDt z3{)389a5XgDyOG2)326KQ=4&E@vobal1t6;h2+hty`&Nk5dd&G=%w z5G9(Cl+?lX;d8;Y8HsN7`4>ke&`qE3qn;A4&uO2o&#x^@^!c@A==1GU_W89#f&Dqvd$9XnWDj}5pEi|F{$6%iea6tgi~U{(__ ztAUP?Dy5E92_35vI!b(cxii=@oX$!UpI$Dr;*;|utm7PX;dC^E9iLu408|*C7Nm$z z`v4Y=`0@#&;DUrTi&Z0*2aRYgudLk_)(u(>a+`J<4jn;~WDV&}5z_sY?aBk&MMx)% zWs$B4NQ-?om5O~9q_NMY@s@qu>miiW>w@PV!-o~WHytbXIU$98q`122N>SnjH8Ek= z?r0)pZ$!w%tfo!kFEu%0c(f)*o8AkNVZSCB6&}0bo`M828%4#|p&-Cn58CLJej#Lf z{I;O4W}bks=W?&?9Rj7eolGR^jYyQ>6Nsy(NsJ?wxUXCoB13UWMunj+xSk+EMu|&g zTU_S<9P+fuDDD7&s90{qm3DmP2Y}MgPe{{_ulBOqQKt=mePzT5kg?kF)!jjbetvR_ zcI2oJ8@`J54<*#eiL{|zsNstKKi%NZbmL)h5&bJU(Or8KzRw_Sc$<`#&4PDg9J3rh#qP{JPXUi)u|5*>|{@;L5??~~Bu9nm@7-Gw(=_4S$%I$do$ zOtF=pUhH&AZKnTS+Ayv7*Ud=DC2e^9`;hcBV;6HNML(xgYBTDzfpr9{)c6M1zu4up zyU>iJ)JB`pzTQHck?1s|45@FJ=ig3E3}?INTT#4HXkYWIX8`cHlJzPN2d*?7w6Xc6_G{-Cd%8swEQf<5irQ$-KNR8D-wOnU2zvJ z`UihWur*!rSk)D`Rb5ec#DT^9AL)RW3Lrr;^+M|71Sx3HhP~!Ma}4B;PDS7%Wlncz zt2wQ+r#4NSdVJH|sWWE`n=*6u^rpFN59&pN(wHCaY7=eyu1z%Bm_)Xm=EDUxa+_;` zJOUu%YQV+ zCegOoCh@DHesqqFoHr?zsABq$SpQ7~vb=He_-qeh(T_g*)<#xC!)?LP(2hfciV@U} zD1!F!;1C*{VT))|Lil*Hjiv^ay+)~D-K!g{o zee&m!DLO%*p#TYuR1%#&G4*eyxN6SZ`NZQ;#ziCwhXheoN!J%&TZzJ_rv4oy3U{dI z-%O%#xCB<>S&71@J0~PG(P=?GVfyFvwG)ND=@SL%cb}e;palE0p<|8TNlv!z;p`Ly zr)``ZCu?d8Ls1_vvijYp&)6i|vhfonx;Q%}S%AoFe6jUsIW}?|hKSt&qH;$-DXrzA;MBHOZ> z`iE+4CF|}0L96Q*RWf|c`i}z5-3YaIG#65Jakfd~(B+Gi>C%#Xv3tNXQ5Ium$|*i~B>Mlz1O=VL%^e zh%GS+Yt1v~&6qoT0p;xle9|!T{%F%t@+RnXafT$xOJqyle^lGZO5SpSr1HamnEIz~ zjE#_Y>@c|)tqTs^Po!O?#tq14Al^{+AJ-=M9079>c^qPcx+SI^I(Bi9K_bF#g7k6% ziJ_?oNZaMmMfh)l&YRRk#_3$3Gp9fXl?ZA|5?}whZGu{>WPj=6tI~iy){3wH48}ib zppwTe$Jbr768{Y5ejuj1>rD!wMNKLp!I_E!oLBvUV>KD0GZ+229t@bVp<^1x$C zx0L-+FA@~@`BxDCX_j|kFeB-p)EN8$iuNC0KDm{Ju6{xWEv^+)Wi9sHmzUd}HE$5U^HQGW{cYRqlRR+-ZsJK_|kcF#XgVW z!!p4)oc~a!r&GuzUF=(v|2*v=y5+Kq4K}x8c5!+K(TyIVj^QBUFuWnfw`Tv7MrJm; zo7pGY)oF{f@e}*y4K^{wCBD5F;xRUAY!4vZc@G!XFM^5rzkOWn@pKw{e2e&(SQT`p zlfZ2wSKW&b%O2k`|D`=vDSNOus$Xow8oZ&>cRK*aA&lE*u~zbJ^NCG;d9ItQ2sqBk zPPW$$B<&Z8NDF;8%aX`_iBQjU96vw+M6d5I6}?s^>m_dW9p}Fkc~goeYz8rkZ1Uet zY(n=!oWXM>kAHP($UPlzj~r(^XFzp{ehn=tX(<~zxx9MH(2C}kCj4(nQ%O@rX<2hg ziPO<(KYC)_zV#C)G}cb28#M~#-97M6TTc7PZa>ZP-=iTGn@+P_hK6t88F__dS?YLW zaSE6-ptQNPvbtn=^U#vgipfK(%1WjTEpKV9B1;I-T+>qCQf-Knu;}w|_WywRz0l_f zo|CD{)X-<2?AZF?9`p(0_AMP=QdUw^Ryw@0ytJaU#IPhgamDZZhFC5}94w&9DO~aU zQ2=Gv+?3p8((vCOWi@=xRiNS=eA|B}p2(T=fsIjgq3La{rjTCYdnz&@VLz#I9d4Q%Zg_I;rb7Ccr956al8Kv6Y*Y+ zis$+u2J!wgUf8|L!n+bLpEkqW<%0(KMQoG-F=#wAOiZhRvBfb9U>n*)5Z&&l%R#(saz+*4e{G zw$7fs0KTxtycx}`_7y<|cMtQg>l8VMx+~G%xlMDLr%t84)Qb*){b49Cv>7$Jp?=T$ z@%W-0f8sAPdby5U2bj)IPTgL$1W+%HssY_1EVYMRv8tr9rlJJD(wgGXK3uXi6tn4XvtaEiq2k0xx5r$!`NB zl9lND<5nQeZt!|?Hy8^bSm(##3G2|s$$NiwD5b~7Eq6Q~N*UY9l`S2Iw5JGt>d#20 zyAUJYk=}l?nzy&(?U$WFM@Z>l(t?_!sQpucsQqfP+7iKj8e&B-_d&?&_=B%K4Ifqn z`)M4gFoL}q4G|99m-hQo^6J)X`FJI>WXR%c$KAATuM1Mk+?NB{okG;;%u?0 zJP29UD={|h`@6=bGBq}lB_o22P5<|3h$Y9SLV&2z9Z(8PTK4}q|D{8}nWklb&a+xJ zmoqx&tHUM~;lpa#p9g~qTXu7bUO5o3XxX1l{eQFSl@zMIs~$Bdbn_GQ^jrWFVLL%G z`EP*KK|fX19*JKFix@Jj{cQB#oEX+R1oamM)m`X+AeP$f%z}X^R7d>67a_z*zi|Fb zBW+GLl0>CnjQ;ONJ=>iNLeT#gqyM{!!LOY(ihGcgugAX9a9CpglVa?PdUgh#{tp}d z&Q+sd8!Uh|AK5yGCQ~ma4>88SoQC=X8vxj5Y;8kB{rJ%s4SPCm_OBhEG$8i#w(tv5 zf5iLIhh{X%4XFEL~uSl}W*S3_pzxp^>T7GyF0 zvGcr_+<7=4qfxSsOxVW^nFsMOWVPOj5tC;jb7~N)Ybawn4oT|~vmnc?f9QV4_f@yl7lea3pjyU@-jz%-2|HY0`ay^hjtl-ULGZqZVlcfY=pEF=f8Ydr_{6E_1vW zby8F{xw^Eh0$ETZ^Nx;iIZjt^gHS$PQ_tBkM7`E&;~b5R$y;&wM4F3Js+ES$`lKG)}UPRV~epoJ5x#S^pDV#$n~J#5+?`An0gbUrw*9 z<{%0v0O_B(^nW(;{Pc~Sdk$dO$n&L3^(lm3Ah!L+net93kAT)%=I72JU^Yx z-&<7LeMu^p3>@52vv!y~1;PsI;fD zq^z{1tZd7z)ZQs*sB0Wq+feJ*PN;{S3w>%#HaiF_ng0u`dvPjDgZ~+JbUj681&;q2 z@<)S59BJ%4ng0sj1x{h!VWgc{TO!<{nn#v5RHjO@D zLLE_bX&Y{Ty4paLtGkoK2$ri6ZeqEb=)ypy+J)|J>5|_yBoo`^0RL0yvQl*URCFPp zrAz)kAw23b$p=hg&jQ;;PeuyzW&KCn%WcAVwaJ_BG-i_pCL{Azcq^$Xuc|34YuRFW z!(qFfgs>t0k}Zfk3&bz-a8xM7R>Ye%@s=qy!^_I-3QTkyu!!qvM?t&N5b?3tQdB|y zq|Fz>bj`G&-JuCktl&9cI=Rj&)MXr}IH%HSqZjas&;oez3t5Te|C1PrNaE7&#snb@ zY*}1;*#nnuDr1%4(oME}Oj~6$LXxbIE^tDmi&K)0XQ>2fu~fk!A<_(&HRNtvN?NSo zG#k{#@mS8=-f@9D9G8iTxm=qeVNMniq6((4+oHx3~6y|CQ)d(CfnlAmNx?K%Djqw+X3`6J#8r%~;m zg+?>t+uu+pWd<;4)S)6oo!QeIYOVSsKop#0jU6V1U}pjU50$TTidEK#l(DgH z%zk6XjIJNuFd??vw!pJOc|JbT0v#U;QLj}l z`>g1Bu;V=&e=vk+$Ij6y1tkz2JIVf6@oak8quz9x(?Q4SnOHl}QM{~^?Eetz^s>3M zgBrykgz_@!=bd=|YtYZ}qutLXDem++fS42?mnJFhTxcc5?k4hh^TWf7@If|`6nEzM z$3#BWlf5EN=ZPZD0tq{zMBWfOUI-npvTKf^`o-Ui@kTp$z8Wy>$H%J~I|tk0j!V$C zcrM4BPeWi`6f5YwK7lY3jyV>;Y!f6PR#B{=^A8EgSqd^w4b4EXied#_`T>eIJ3iTF zqD2?x|1`(K6k7BLF=)}HMOZDQ7HFAlY#YNkgp@V6vm?Vk_D$joQKQSH2@I&A+Ok&= zUTihGEKfi}4Mlqw(BZ2)-qQ44J^~cgSeUGav{=_RR?{o%I}^Wq?;%hifKchm{!c1t z9(^w8iv2Ia+)T{`hxgr&DWc^-U1x&|Ew(7dtjdT({prt(lmnrCRF))!4A5gXH#&hc zcEL4^1jxwzj>xr$b2YZ(aKxx4H)|BVy6|~ zFg@Q6XJM+<*;<~3iAYn?+R{{7R=(xh1Ex-O$y33eph!NnC3%_nr+G0&KhXathV0cY z05BE&C!z$Wg2b~@^sW9Gl94HT6JZihl`!@krg`9A++=;W$JtyeoA-G(S6qFtcusx7 zb3rC4WF!i8J3YjV3LWHs35A{&g}xMph-WF(?THW`6`Je=rajU4qCyAZ4Wo5q{0leG z4mci1O@nvC@i?&+!ah(c6if!$Yso~E>D~}xL}d>4{{>~%h%)~YWr$}f)1CE?KtL+f z>;p#a9ssDVO!tRFu=KV;c&cB`8+YU30C8K#D3h2C_5dSGnXQXMjHt{Z{y(71Gos8t zL>b~)%4~g52#?CN_<%_~;K7zT5^uE3)~AMGF=2C0f~%>nvmafpm(zm@(!TW#^&?qf z`>pBHiNaF`@U40NcVKX)OHW~lZR|4bs3 z^ote{NqU6+pXt(v-tkW3b|T5~I$fGvS&8tm<;6LsOV0^qCM&RS583}Ip^53ziuU)K>!%?H%X>ZluB~)bO=rsvc zINB{qF>;7Fy}15Iah9mjt|;Yb*XvfBTQRsef9+m!wQw^sGykW|OVY{wFN;dMA4%nh z5j$@$R7B=&xc*O>m!^>UK)}M>+f)OFU1zD&if87=BuP zz^Ir4P~{H$;}g2{&I!R%m%)9U#vbIbi_<5*E;#y@x=2~A1JKDqH0ga*hzm72%zq!6 z{9QD8Uo;_}rAcqu|16fKCR2RE5RET(fXR4M6Ept{T4hLI>F0YoedB9_O%v2)-`WWc zjeFIjitE?c>h__hk(+4M$BSrnxc?EfS}t0BBw7*A(yGsKAv|hzgb$c(TdPB~R();= z!LogZ;z8nG1oilE7J^bNUt^6OT1{wxLiM`#$+-mCh&p{u{aYK5jnwFW2z8zibv_hz zh-azO*VKPC9@RP02MnO~i;X0CNZ&aTb%qr=joT1(H>W7RIvBb|od#W$?kG%*GH-?$ zQJG2p2Tl<;~M&iK4 z3H9Su9Owc9RvhSeOGqZ_(&T>(T~>%LAB!%;vvqkVghySD@&OYX7Nlxdg!ylvw_A%I zb|)XCQ$C5R9Ui2EHYM?@p;^Nqh*h;Kni?Wrg1OqC31V!Dq}5api~bY>Wi|92?^wx) zAjh$p$dOwqgryo<^i_yBtD*0EM@Z=e)zBkSmgBI;AnwSIf+bm6)zD%y|5csCWi|9} zUQ?&0640kn=@Y953vtV?#Vc%mRFQo(`P+7taklsifO4E&%5fIQ_j10hUIZA8H$rIt z&LMit#6C)tqLHQOQuO4>EfrYOE8pVK#!PIDP?ln7|Cu4`EQUVo;Z%lgDK3hkhy7mb z8=h7%w7;o;>)6CH)nnc?r?C@xLZfN%&2OmFR+4vZTN~1ddDnbu9SRl=fQN0jogRXn zh3}Rk@~L1^!^oHCV_Et-^AKI?YMB8kS)GciX>=dFt z3FFln3dXn_ts;^PF!{f>JL~F~dmAOMfFv80B&DU5Q{YLJzv@Y=h4OrS!j=PM|EHt{ z%;~RSsaLIPAg5mwKlH}!*=&7EK?y|1fxCz7LErkDceK;kQGDy@OsuV`Th<sTd$ZEfk?1DXE^o%@||=VtZ+qYY&JgZc5&H2Lv%82_2cTYDLQ|QqDuRNBuWR=Be*>dOR%i_WO&>X-IDakg5OZhM9y&c_v}9ZLKAW3^Mf}4OCj&=x5ccq)}_zhs$%}-}H z?^%*EQV#9`AV!_j(?p%YHC7MS)sO9u`NqU9iIjt}|4&8AGg9S^cbo>vgoO$yox(lg@%tN|}%x9&Lc@p5@ z+(Rk>!>DtXM4dc$Ywph5o(?+_yYrMw#oC>xcMaCojhi@XJc^q#%jQ0B@{-Uq_VAWE zjXRLVE{?Y}zD{kNT8mJO1az_xRff(Av7st0{&!Giy{Ph?s6sqTm7%DAS>jQZ z={{h(5REVP1eswEeK`b6ReE}7%ZVZ*9PjM-s^CdYsFJ8oQ6MrAWrpn*Vnk(H{r`h9 zuZS}LC(01sQfAov5FVA8;p2Bz(g3JZMkZ{-O#LVDwO-yCa^?soY-hw*rmNE#%A_dj z3M(A_*9i-ObU(He zN^nxM+7@icN>nPnAS4=_X{!GpsPuxU^dC`)c$P|~t3!BHX|@lTZbajY;eq1?Rr)pr zOO^V1r#p?q2G4&UmHEVCW^jXh_Z&);b#7?V=tAN;r|3>HiFph_9m5aDQ*uC2JsqVHYap|XP?#Yr5qimZP)9+b!28ybyh{@RmC2ZA{<|$Zy z|MeZP>w!heEqKGls!jb{OUt6~TGqc6brgLYf3Nj9=_OC{Ga9`b(umW|GG>jc<_?X{ zh;M#Fos?y|SreZBaJspG60|W5KsbqgHR17(C5gJQRDON;>pGD{3K}~<5Tag-g2pEH<|0}j1r79OgsfPz&rAv$ zl)@@#>=GXT=wao|qEyKNI-Z`1wYz#oNAa*-!s99M~1}-$Y4L7c^$B{_!thWfcc@W&NL}-?LK`2Sl7*Cx|#_tAYlO z4$7gxcoDz=9cObZ=C2#pkL4!;MCM&N{!!+$)5-j}O+}S{ThF`bR-qyVja^y)qRi){ zmw7}Rk@*}YvzJol5kqWlO6Jie&x{DmJYt;4d`=pfM_~VF!p_;31KT-m(P`g=Wgc;v z$b4Q3nWbPc0{cHzcR$Z*1vZzSO9sd@7P~qFR41rtsjO}(tJv~79g>jI`vzs-K_F%o zEOzT3V#6e4rvEdNkXKy*qW1E$Bq7AJl91gF4&gBgInD=+TIUC-^58@9i4kJA(?YO} zktcVO6}nm;f{%HID*LL*Q-*cl#o$3cqlMSkAzqA2a;TlqE{; z{(6Wy7s#%}hA^!sqU3Y&9VC+|Nhz!-=}ihr!A)<=IQE;;wqE;=Y3)Eqv9^c)Z*>7i z$rEUY(4&wNalFp}%P4uC>f*yfnSHw#4eaI!X>W_a62HvXO{WI}ubE zaZ+7VA>!1s{zGxj7jaTW$=a)JZpGl>{B-@eM}(V1$y%&`DVft;$xu-HfkmbL0#bSD zw%ytZ6_MG`7nv_i5pfO!EV`*b6fjIME|iFq+&QKWwG!`jTDt&G4@ul1o)vfe7eaW9I}3cks7H^=AH~4NZ0C)U|;+?}|EYL>=N;>eTNR5|QdG z^Z_$~XnZkF$vmL`s1PjGIoW&3X{;bDY9KGgR|gdxsH3*=DQ!xHi_zw#5GQIgw=NIb zyd&D=i8jQuwArIDghy>o^Z~PN3T^hN4#85Jr8q&Makyx+;Wua_713c~ZT7e>#EII> ztILKqZ;3Y9q7CtEZOr%=be=^%VAL!Dpmy8r8E$`Sa|&|;6h~CstjkoJpxQPB6s-Ex zp2tMAnP2BZn>R!oSF|CXrOlpZ|6icZNj_lIb{BvuZCE-&$JsO7{?z7FKG0f47N}3X z5MLXlhVZmRX&1E{yFRtoh=?-B*5yE%&7w?>C__9;nZ4$P@aSlZeZUkGjW2c^iEDc; z3&B#E(|D*!HDM9gUW~7dj%x>u8NW~ccoV6y`fJCvQ5^q+5suDwTwN|SdRsKg6^)2z zX*A00e+`;vi4U0mwnkDJ8fEssDp;2lx4S*;fisn^!&KT=*;^cPa=(35+UwD$Xk;~!^}OYtP4Dpb|;Nkm+T#>c5@>v%Lv zr{l;J1}Z23GMw!sUcqs%BrqyAy}Y%T2i7pDK}p*zed zGf4mA_HrYIHKQDTUq}l2*&y#Wc^m>d-j<2A10BWB#&il%r=OijJA@wFla4-y&%bjD zr#t%Ma7XV9+TdlV=Yb5Hu+CL`zpsBh9C2ChdCOYpuGhAngsEgN)${L(BSw{y$IDE zZwhVZKNqV*9y#z`t%2+Z!N>j!^w=5~C%1;!XB^v~Q};_!*k@bKYFvc~bqVdG#)7rk zE=QwwZjBscLn9H~pd1lFc`nQ4s!lt%It_}M{{~#1cik$5Qk8@<0N=s})m zaD7Tg*+Rb`FKorr&#L&FJZ3Y8k3W!PW5a>wDT8H$$$zyKIcz`6DXiKugMnm&8c1SU zkx2omTf_E5v;M28Pr|T$E@47Tgd)jAv;SAaa@hV9PfF`OMv&x0B}r>Z3!Z)+zU9Lc zIAFdcl5o(3eJ7gzznVIS?W;JsReECB)|1;=0OP+3Kv#%3sfO(X%=oV_%nc9Q-2H)eKETX> zu2jPk``prjuIicX@ew*lnJ9vGV3wFJKs#lC%PJO5k?_mXqCtNZg95Zuw{=-AB3fx$ zG&BAOXs7jeIaMZFNm?}O&xjYGonGv+WBg>J~vev!Ba4 zCDDeaMcX%mc2-}PRT`oViJ*xC9EA8c3NY%K8}~Z=)NC}I6~W?y&VzhoZha_Iy2CAOow)G_Xyg#+qhi4Bif*}XlDOsAmn+yTyE?o+74;a zPKj_kzo*L$OGMi~Et*;X47gp;!}YlZ9BA96MKkMv0osLIyFPm`&;~}(BqKRw5QJnv zxFStJI0WL4b4RBlMd~$ z?IUPcba43;6VZC7MPvLAbGs7oEW+_4E!rg!w7(R(J~MWp^-7EOTm+<{zqIFG+7L9*5yif-KCt6`y=r^UF#10*Y}satoYHpeDTl0Onj* zshlgED#>(w(!Yh(RVr0ta(-SLSEcaz_|TyPd&jXBqhqB5iS?x zy53?C128vILlS%Hv#3uR49{8 z{X0va*5-rr^Val;v<1$RU;+``qNt`;oUzdvfJJ=*~I^= zR5gN4cqyRyvb<$7`#-(qs^n2yyrtOz0KMg!6yCyA0*Q6=08!)`9DSjV4a2@^t{G;J{^{+{|D`L zIywZgjrV8lZ+v za(e$8KM84|jsF{HsGFyioJUZ+2q{f=#HHZRLGwo5ZA9z92wb4mFG6+38+JTmIUv~> zHzqemtpL0$q;7DIP^3VJVz$M5yPQ17EyCuT7x6#xQq1yjqR42x8=Q&w4=)mvh@woopGx<}p(!c_OB zWOdH~BJ6SG2H~vKRh<9qEyZutcUlhq%vBf7;MzwlGdJ5u38uJ%NzP*_0%f;B;l){- zR^a2ok1{pj4Y{V8_P@bt#W%C?-6=(Q`W=5eHEz-GJNl(bc$obE24}kPIF&qT$32Qr zrp<=>{P%?4bJh%*v7bVzqs;yfZERLh*J9Ao+h!_w!b=9W*5YZ};%Y}-g~p_--JHVJ zibaMM(%?5MS7X)dHowC4oaOjE;cNt!zIBwze{WXKb^(4FiXfB1*^cfl&USOMvz-Ve z=znxrfAYTFufT6~wx%5XnV*3H5~t^}snFqza936bXKN?VAGIf*fyGyxf2dd6V!rSv z1>rt63iE8p^OxZ|%2{5$-2Q{@a`oB2RhB}f6WI*XtfmbQvEr)0@pbuF6?oqTK%rBY zpH)mHG$N72VAE!V5NWVG{Nd1*cukH!JQI7_$G>Tl!QHlo$*|ePzF!Qa9k-Y-5#(@u z!eG-({a<@84K|y1Ty9HYu(N27ObnJROd-`$JB!3-(@p(bbEC~psKc%V?*p+}J7Y5< z5@`wE5Oex<5u4rVSIU;2R>!Z*#Ae|W0JYVi=C3;%g_)!Un!)j(gHvF)IlSsj*=;25 zH_OCsQGp7noZ=;xn{j=JA1$}At^k(%P%KwqEJs8logsq08A7DD-Q{BeHJf&npXrtw2YP4COgv6oI=6id~=m4Yb{TqyC#3`g+ zw$Wx!3^Ak8PO8H*kKQL@v<}8-M5NJVz@L2{5INxAl77ISeV-7&MGg3zq@0U$V7zd4 zSq!2w;LkSmpIg*`UxZ)EF_dQElP@cop2PVMv&&nO2mIFn2-ll4ETkBDpNB2!dcgmC z4*qPLL6(Ts)OcxVTgu7gJ%bV4b&87|*U=bF71b3b&1Ke3fJm|n`czUfySZD36fH%} zY~x)dC&(bPyC^F*VLEJAWhBx*7H=4G?y({Iyi9OfrRc zF~&i(8+m-+cJx!Id9bgb(%N^zJv!a^IeE z+<8KSOuOL?`_6mCn&rA2cbtA3bCXu5=du37-gaw>-X`IDG#%Pn$t(wD)AjQpN5Qq~5wSqA?jb9DT&9RKGyluO#;*k-FO@;D!M z-Wzn?C=K9pHV6fdJ;PSOVPrYq+XN=iz#ZjyoBpSPWknGpaMC^W*nM771pYI`_zt^NyJ|wYh0->*#sYrp-|}v~F7ilc3UJ=IoZ%*{v;g zGpEg)K0`x7KFWwFP*pk^QEsxm-zi0uQwEprf876tjLu^Z7JE26Q62h$?%$xhe;mR( zyr98UXpuy3!C@i#+=l(Uhch6yN2B5bIA8q=*H_bNpv4xGgyrJSD7O3|XD5H|~s|*r9LD+1WubfORLARpF-hX8XzL zh_~o5E>`eQ(~PJaja5`)*Ny6Rf(Kb0>N5s!nDd02ZJiJlvYl4mFRz{i#P!Mh>L!h+ z6rOINJxF?rCciK*M4nG;e8h3En2w6;laKxb996iIqbRk|#DC?dE{{mL*PASU3Z*7z z;-{8U;-(AVfKqhR+f%ry3~vj65yfw(n<9nGa;_ zlb*=^k95-8lbv)HNTWedWc@oaiB^oSEyYiC(*1KvolM#@cI;CtWn%mje+UUsi-3EN zhucV0>I35+o{0pg)lztp?u<9|Tg3iP1Kf~o02vh*EwDyK=PF=1m;HOe<@g|gG~pty z|F8+~OwojrV=dzPCv&Vj)&98+96Ev=3x9UPTM_(&R}xOz#nPY`lV|6f$^$0j!_wfS zNut4BDKro#Jjty8-=%UaXt1Rm>m)P(G0;#qeU5d~JK#@0yNhwPCuwttPT{hPx&Fl# zyDPcHMgR#-x_GdVx|4$qNP$ouexl3v*DDFzgjN#BEJxN5CgN{Ad+0b9;!R@4{}nN3 z5+)dz<9st+37m)X&2m-PoKjxif|-kbNo|TKjRO#QaB*;lWD!`n4VeXA7VIh9;ab2Y zJrmk8MVkeRrxoLx0Q~LLxy66f#g*Ls4Rb6xJ;BYC?oTH-w$oUS2-z?W|6TaU+?h$X z<237&vl6VCLo7iPJcsq>MsFnh5z>VhbR>w^6k1YEe339f{;OP2H#I9E_xc!s@<@NO zov)xmpOo)SbM}~fxrFbZmcg!lL8X9pBzB?IjNV)W~5F%}MpTCRj z9-rd)yJTWB)>wn9)wS8srZK8+lxC8a=oGX6MXN%)b@Wz=-B9&im5JS=5*1QA#Y;l} zDVM_n4E=Yjz^Q^bAao?(4xg~nDSsC$-K{oe(7a3VWC9*g;DdpoXA_8Fu)c|4os0+l z>)L26`%!-j>r|tEz-uWUsISY@SoS0JHdbSV*C}|GxGq;?*-sPOSZ4oU!0S{zQ(c#% zvFxX+ZLGBsUZ>$9`MPY4Wj`iwVV%b3U+AI%uhVggLtU1}vd?j_v8p0iXW$X`I=VG1 zZ$HFtV@;1>or&W->VQNnKG?;+z)j6ImZ|>-5}t*VGjM4vM6ggybaAe=!m^LF zwz0VXE!X|CBUml0Gx|iZF2q5!IFD9g*+x7vX4BoRF%p?88xQtTQ867vl(0oJ6Xy>_bRxtS2H^f5J)HI8a+**=K3nSZ4oQ z;8B<0fKZ$hs<7-MLM^N_P5u*LU5b;2ap17Rvd6 zU5V>gaSN-$vM*q@v1UfF{(_q{aiONdvhUKgv2KiDUFEHE{Vz1u7ZNwN#{b$_od1P; z%GKV}uK&5l`dqNOhq2D?9KpKAd&>1c(^#Jg7E^qS*V%hUu&(u1y8fpc>r=tv*k)s$ z7{OZRJ?Y|pKV=R3YCjvxtbYZ*aUHG!!);&+>to^7C5hLk5nk8hk~-W~r?5T}tj=Mq zbNWTFZopk|xDrlb*|))25}sq$e*y__#HDMvdre{4*RI)EXGeJ5gv$Z__cfM%J)n(c z_J0JtZub7>`tND1_e8Ju_+K0A`v|XFyvJPsU5)jwU=@b3&NcJ@fY+_wqptss#(GDv z3X-r){u^N3<~`#2Z)>c#1*=^W7W1F5_ulS3?E0HE)@H%VPr_Os!MelytLwj|vEC9a zCPt63Z*MC)Gy((BtkSwh8yGF3?^X_&1 zS2Wfug2e>M#u^pDy5GCU^;tWGgqsA536hOoM;Z*XNF8@LDHWOpq+B3wMfO{mr}C^`F;R&kGh4Bpa(Wg7tUrCf9#XW8owK=DoE{ zkZi0gBUq1nH@f~>jkQ*=m>}6$8zWdxcsIB{cO*lvX9bH1l7)3qn+Vo&?|Ro?qp{Wq z784{JYeWQVg?F9nuhv+r1&axijWs8N^`y7V^`FsL&j=P1Bpd722-ZsPTGwBtu~rEd z6C@k!?FiOW-ZifOw8nZ`u$UlOSQmGTU_I?!?fOq?tfvHv36hPqPXud~ca`g})L1J8 ziwTmAwIqV|jQ1DUe^O&TDOgO9Y^=XVuvU9ly8a4{wL-9%AlX>|h+wVpu5kV38f&>= zF+sAh{sjw|*MCA|Jt0_3kZi1jB3Ns^KfC_p8tZYvVuEC2ogcw^&b!R@|E{tA zE?7*Eh{bFLb$~xT4J?)}?@lgV4ibQmgltfUdA<$@%%V{NfI3X4tYTeuiihL>g@=-Z0vw-2czi zB{13p{s=jh^$5owk%`f8Xd8-GiTgOT;c~WMxU64{;V4d%h-Bu!LG`LLo|w?5u-D^O z%uEcILZm|Kr|fVVp0O*U3Ou}c-#nZfrr!sp6@t+ zbG%pDswR%No*3H~+JZPORO3O;74ImyUFk3$KbL``uwp!O8$l?lCB0#Avr;9Vi=)K3)G&(ey<3m}F*X-^gU%R%2!eGLO<&vzY%enNn`c%xZ6;X}Q+ z?3394Vjib4KW^zW^$w#eYOK{Sy~C*U(}xTJAx@1s(M2O)!TL9gxc8)A{kr00vCutg z^$QwoY4z)hdy=4`Zu-@)D>f&fx%x%gG(#47;MEIn8A7gX3rG$j_aqM?=Ku(*TscTc zsSXWNAe4unXzo99hmq1BG$~^is0TZYHVC(@<@h7C!{|zkf1~h~Izb}81ApD2L;1z{ zKwFp{Mpu4jE8sA&TvphK$ZAH*Pn%`J~+0O?Un{WObA?@$(2YEqwB-zlW%Ef6+Q_pxA*< ziTy>B5#>^BOrg~?ifLp^mJBOkdo}m}gk#!pUN5N5{zXhXJJXm(`lP*5=>I81pBs2y zKwjojt6Q91+9>Hu%bQ9u;@FqbMA{;Q7@x*WSVrC>17iVPQagyM)ET2yy5;f%*X|nP z%Kba{c{p%&S8|0?&6)UtrBsbp>RL1Zy%P^>==0`YbauvH#oWX-P`z|I=<*ABoID2r ze36KZLf3u(IXMd5n|>5p#`sU4zL%pAf^W97YZtY{C)GxvWt9nNj4KS)IDa?g1B=Tz z{&QTpH+fvS2}tNn%gp>wjVp@r2$pV{k)55>#mUXg+vCL&^!~V8#oX~l5YrtGaVYVA zcth&zs>A?V>dpwAd+w6B`(~HSyg8ftlCU8j>i=2ALcEx6Rptosxbi26i!)sJ6(pxK z+?T=`iogNdU2o?9_o-cBV-dBhI=h}T2|p*?p$LC7fgpUnSmQo!vT+&8dILxKg+NJ~ zaXr^R=nnTKyTb`W$~^zC3=2|uPBn4q4wG`wu`_UoEN6p)j%O-Bg9pWs?tMdO{^vG9 zpI5wNrIFFSkIk%=^QZ<%ODo@SyzN5nEX1tR5~oq!QCH@aXyWli$J4VUPQv^|i`>Bc zpLV`8+0M*P3Gg2)WzyTvuHI(tnfTF4UvKOKD)f~HQuvA_;5XI+h6(rsYWvxj%p!sW z9DjDgTScNjc>CFnk6Ie^97Zl9C*93Lfvo}SU)12i6dLpeEKG4zHeeX~9#jc9G}uxC zep6`@G}KL>fZt^Dp9j_UGty3zfZuc`=(5EgOl~nwkSXSE!uSET{p?E;%EM3W87a4) zky&K>*_A@kV$`JxUJ@!Cm?tZ7Sq z@0MRe^tsRKL4P+n?iuyH-SSlgjYprijQobpX%s~Ur(4bQZx5;N+6lkt7k@txZ?wy; zX8-3yYSR;MAKkXbMPSBF75Gp`zfK-rT`|0@#ykp9I=O0SRZS~S4&`fdWlL*mc{QE@ z3?2PiOS6-Q@VU*jf52xzwLb9&d=_dx6%|vet6R5xEvU6+5B^eEyE=)Fng0iT23G0w zi@;}5#(Y>_ReZSOL0t~T8+NM&X2fn9v2YByj1}}3I zaBVe6kclw5W5)=iLFKMDUo)Do8I_cm)|6G7yUJoRI$W3>fKN2k9T!I!?O5ibXod~M z_)v^uUGA9Ta$T|z|GwkD5jKNM^%-FByOZOyfq(OU<2!X7P$3`Tb(fj{1Wq?>n2Xyl z6|d7X<)v=*OI$KDK=?3~fjit~#=n5qPD5Q>!Onfyf5`RgWJDhXv%5P+j94>w;A;uCt^vogY_Vh7h-ieGoZM|?K8!VP+j^c9mLpp$_#)xA4Jm<-?E^*CXG zW1JbEiFS+MtnXgLZHX$hZO?v$NEPtXH5aGo%_4qF`CygneqySK<^ge zB_Z(s8za1SDRw>78W;j+#^=@1X>XzuZjaOVt$8YO=n`SJYmtk^Gv(<=#b=f> z1bWwjjuiqQSRCOtqMz$=a*cj9^*3<~LZG)yxJd|n;GYq0yY+RwS%5+aoE4v2x)A8y z3Q|@Ge9+8)125UVkL%3^6gu+U_zc4#(7OW!7%LGZvh}M{a6s6SBmM!iNZz- zZ4Y~3#j?#q$z>bynunW27FD)!I9=VtJ4bH2!zo+mWdDvCb|L5LDx3_$Cvt&@*N3=q z&&r0nuGn_5%mo1UtaQyPX6-^F5{YR75Fau1e=Q~p0T21RbITll$DWnl^D__=O~gXL z$XZ+jSv#R#FH&Ts+p_`&!Db(dBf;tsv;GlSZcp!Au^jfSoSTW|QixPY{S-4X-lK&P z<87?l3dXxmjJK6B9ubMO7;hNwQB(g8jQ3aHgT0B@;`m-B#xqqEb|h^%=$ZOiHIiRk z%oqQ_TYT_VUh?`*GlN=VfBKWO)jGPdJI$9;=I(?S;B-Fx&NFwr<93L?qHF zc*AUundjdFvpwwB@)&OXex&2qW@0uB?ew|Cv`w2XXA7naHcC;R$y%nrbqXoN?mEgl zUre`$9PiyYfO?_^x+qJD5{EFIqAP8rXmdP(8$O;LqR+ijuO<$Ez+NdH{t(+d8SN{e zAQVCref*q|Alxfe$)Q4RwuGYRXhq|-ldY(XRgZrG`W&ksNFJ-YX-hoO-da-8i$bT~ z;XjHW2_QAvC(1yDODYeiSW=lPgjoOMqR_)!6yho&+lp#3^TVqX{~^IYcv0vHjDMp* zR_z38cMQ24JL1db_^>tjRy24dg$8{A3#VA#12CN4Jil3T1w#C$?XJj zr(}}Ot+*i}T`RujO3dllhfA+^JhyXp632`deHICuBk_r5f6~;y9+o47v)nVtDhogF z$LNs_v+&2G33z{>pU`HSb2q* z15SXUEpe78lE>6ZYHTk}9vfz9R zt?<P=*0=34#`;bHig{4Y@FEG)ll(0n$;S4ZoE;c*Z?VYjDE{U^}p94x)Oj9;whHvK|odx%KfGak?=sNJbT}p~7b(KB3gA<0HJz z!}`l}_!VW_=i>81%>uk|7bF$h5$I$fp>x&S5hmwj@#T3yU?!Zwg~bT!U}9!;iq&9Y zBiY3>dxI6nsz;N@Dw&l$bF|2!W6&~$aqjGI?7S;;Fp&-!gKsF2ajo(VtmT8JCd z>X+)aMp}I%*eZ(Hr^RGieX;kt99)aE`t=Oc>UjD{THVXZV4$_r z>YCx@O&FTDc%DD>eid&Sa#ox5Kdome2K%<&#c~QWhMbG_kQ0ZnPOD)fMVlEd{Cmy8 zp$4c$b6<^~)R;^ABduPaNpokER#OlwtzPqfNRV+5WJeSb8#_`ED0)X`V{2DBgHlu+ z|5?s|>G%&PJAO9+!V=H6SV^hFjKlfq$(4`bg8(ulKFj?-91{PQA}OVNlF*%^$=?#$ zURE3Ad{m?ZZdNT)n}P5@D%~cM5%D7t@o#BFT-(ML@hXz|lY4Mad{`o`9U>w=oIw9P59aCm3k7Mf%MDxfuy)X8xp|Ci8#p&w!*;J(1jE`vD1t ze(n(=^|=07-;`IA{89;Yz3noPv=LZLfONSJm5@#nkS#GOfq4S(V4 zwn|55Io|1dJ}{-Ss;Z)-h%FV zo6);R_tnJLoBh85@eNp%TCHiXRE&LSMBA)?26Dg9OJ7C>+Ry46mRnnzO3TXaIc#dVS?IUq-Y_U4 z_lrGs?he}PH0_qk>Xxz!`%Kl;v==AQJ~cx7rS3Xk2kqZ*54SD%yAj$iW8LK`NC4VT zDY?rknp&${D}TK$j}6Pc(d7Suy*J?kHKxm;y^P$on`bu@Q*WZraaAfQ>ME@GV%H6F1R(Vu{(9%)A82moa%STR7e#S zJIOg-EDo_`9{qA%59HA|ktg!#9$CdyMk5kQ?DwL{|8)D%evkPhkpK`6_8a*d?WeaB zY5QemXulUZ{_B9wcuV_r#a(rcTZ{dk{f+huWGbYJw*6i*>wnq|wBM$>p0MA|V!xio zenh1Gm>{C9U%DbBEA96;zaA+a@o;QpeJ1v^HmGRJLD7_#-jH~zV;=#U%PB&6^;fTHp&XK?Yzq57E!GXQd)-%eb1=hKQ@wb7)t04@ z8YwQ)MK?Va(wgRawQd`j>sB$>HpW~;B$8AGH<|rUK^J}8-_y~TBcdv}XC~&7Mh%uN zbJJOjQY|j#OGIJQqxD`f>mPxSbw{Z}KN15Udp;A3C5cl=y=)(Q<%fvTUaRX3qunM( z>uroiL>g_Wmhx4ue*`gvKK6va7m_#PVWoVpOpJy)5jG?xEn4Pn!EnK%CBzANWn8{io%;oLYr0IIFN}(RZ!g{Y%DdP3gR00)J zJ$t?P_1PhgwA~wZePO#h#CCm+?TAR*Eyf$&`|GBoEKORTH+BihL(}y{jk&QG<$>32$lU8f@)S}z+w0!ABgBxFd$X<|EO)0^ zuAi|S5ox*8@rLEz_%MV>%RTAu<2Y*AKicv4$;5K0w!UfeflA6T6+#4KDn**2#d_21 ze+c4TFAvv8DzWg?^2SUIm`c6&WiMkw~X# z#J9};w_vnb>0_p$>bFSf_RYkKkv(wQgzeIpkfJ5h63WvoGMV+yz=Ydi*(Q+?zmSOu zQ_56G743v>b2$Ffew*uXRymHX10d{IZ0tuw+D{U?&8Gex*zYMH(|HwYkkIXyiT#3& z&DwAU>BbE34+dLGB1;{c!{Z-&Z*M+XRzxj4JXw~B<-*bwQY*zrT<-1Gh|9fQ*B>Uk zM@-h=n2d;)$=>GrcX+IQ+8^yWD#XBKqcb;|xwH*Um8r?zVf{0Zh9>L7MOxJXV6vAo zH<^*9kXqR$dx!PEfET^&ow{vdvU|m3+ZvM*ktSnlAKm+%jUhxXU;N2?14T*VVfo^X zwi&F&qwF4U5%BX(mB7H1h@JNZ+=E-RJf`VAf7w=sc5`?>gs<;hu zXHoQAt!TV(HmkQ9ZkS{X0T;jc{uVug)Sl*le z)8JBEgbI1&mbRLbB9#7$oX6yrwtT3Z>KG%MBF?pIsc(4G7r2!su3}5R(2Y01>0ikw zd>>r&BDbU);?RSs0iwk4N*p3r(qdO)BRj05IT-iRZXc}(H3?5x8o#^KxFfx)+}T|R zhd5+)6hvdHbL5WD)IaVQqRtbR)_XYOW)R8mgClMb1=F-iMlp|U$x?#*$CrlKGUiot z1M@Cqi{O5qRx5tYBYo0di8qY)@xMd#dBW05yUOVYw>Z0E3JK{;D_hGkCdD{mDb_Mr z8Z%)zY^-7zdv{#D&iJ5-eZtZwSpN(X1}q6@IUjfnoyL6a2;M?%vUp12Fd~>komL`q zl}|(UKbI#geS!zr^rhFBt4#j`96`8};~2crPM?PIKbI#g-S4$Hjk}3bEt&X%rPMJ( zK2-m5dHdNi>*fmfy&fhC9docmvy zL#$M*{-q!ZBJ6Pg;~e61%>NP^Umw2WX}S=U9SG0a)~$nV;`$6vUlOn|9tF&_cQ>GPf`l} z;}}~3hZ)4B-do5sVL=x~qqqL28ANFjvROWuu1+gx95s4m{eeYl?9JqkSA^{U%^k1w zi&UwL3mIW5#F(-v)s@H(>?i(G{-$sl#w9}IbZCSwsrG=}V~Rr~TR z7?OQ|Wpdw_{`KYUR{zq`y6Y{CtATCBx-U0?3ZwO^6tPaC^_P79fzf)Eiq;<8POJGp z-lO#u_CF`0b7}-1?_8T@YO}4!80i|kZAqYLx5p*IaL*4Yz`s*$U=$VLj9_XZr)?e=mNVeEB$t}i5E8wkPA10($sc7wy5XEG*B<0f? z9)9424KI>7A8(Sm!6xHOa`6NwE)Si&q+ zPiLr7`)_|dgQ6X(#ruZ)A8FoI4kG{`&o~=r%Xh-x*ks%(JDFt${Xg4V3?k$suKSJY z|Em%^vd$qd_W#lB|Jg7M|6T0-0gTC1djGd3|5Z(?cALD)&C90T^os8N@!CL(scD}G z9%$ol&qx@NSqzsbR{3LatR}l=ljD!c&!CM_jvc+PI><;*72Gscb&w9K>J(%vydlhY z-2cTOvRZXP)slo&zsuhL?`lNg)$({ozMG8;uBQPC&oWF?+t;fmmsC|(G}+JkMfXEt zp_WeyO`K3a9xYc`do3uC$3b{gmSqt>IXLc%O|!c>AR!hX?ENR7#KR1=R|%6tG^2mF zi!jQ?^Cm~(S8SEn=RU+@#D~T;qwzQpPw8|uh>*=>VKpzpDi4pFaIo16AMtI6IVD*| zkD{r)M!4-S9L@7TL2I?alP5>w*In@upJQ7mj~w;W2DN_yJ~@a_e7|2rueNyZgd;V4 zVtRZo;uGe?o7!czp zS%^CS=^bIyZYO=j3Dh}8s}qNf>_spqWYyj&JPe;xB76#l>4PP}XLiPX%ws{dcMBg0 z)c^S+!l!VkK2--#x9C%KVwo*zi-5A-J>W$?!s~|vBfQ!V(Z{lZmp+y)173-RVa4k{ z;d2;1;SN7M72(wZ$4D`msWXR+#Nw5_ey^C_3o>MMxMucWbN*-0lRDxQshRi{r-00i z-&i_1)x+>;b7ocT9m2_QyC$MqCmbO)Lvx$)o45tX!_?j`+?wzSL;l!4!maZT`iMIm z4>LbLw^WPK%9!_q5t&Ix{&7)+VHcb#bvz)@k@evtv06@dau?9aK$_|&*8iCX1x&UYs1Iy`W9S20V(FqCBk6>!+6RP_ z^sk@H{_lWWHynwy6j112OSMX6IKH*DYRlWd(UHh{q zH0xfZPYOgUogAMRS_!-o2i@fh3kie&cXou$*8TJWl?a0;#b<+XOExA9`b@dW!|?eg z!ly@H*INPz@L7^6pF~38Gp)Dy{Cr4+PtQI{2On0&X(Fu;_{h_d1;R%%q@Pzt`1Hcz zN2dY`8Pcio^@%(kSqKtjB0l$vS^o}vZW|ncv=|WZxyA7rX`f5%dSh?$+Xx@&vcH)1 z?|@ftY~^Q?iY|L%d|siaBXfXG29omqV(PyElRntfzX%XW`4+`zlH%#eTw$XdsSrD! zlcPi59{TtrG|~y0RY_$_bM=(Up(UjilZRHNe(b@1_*#Xb4PZnjy0aEKX8k|Vsb7IU zwicarp<-0oR9049Q(~_&rOb_F5~0|y*tdqkpkJjc|udjiIH;3LHNY;KCapSsCjX-#JhC^uvy|h7Xa9jGa##&Ga-#g zq*n#;=a7)BMqcJ^mYce;Sz>crC1jjxRR-EHPToC-=(K4*jGQ@enD-#)3hh@k; zK4ORKF$_PeWq4N05a&U#Ok^N+t1WU({}1eN14fy(T86bshKe{3jAbIjb72|sPL9~& zMhrrPfCDM<#_ zzZs)~wz#>Ye&`T(Sgm9zYi_A3X|1xCwCxa);TAkBw@TY#m6jpS!*ZF} zVMUS*tbd2?aBIFkvH^B@Uda&ap}I_Dcp)r9e)otBw`IHDliChXDj7@!X+g#z)A&0I0*$3Iy3W+2Ydk@?GGsAA);11cA1A<8c*9)nJ_8kw(QDNhjW6ZJ-n4AUl>b5X7Or>K44&A z=zsw-%g5r?MD-sJ7nrF24_}ky#X@$LP1c!&RP<^dBd+$m9P4;z=Bli?1m}4-m$ce* zp-7Hfe_x`t6>MmLU|SU<782qyk52MkcWNm|4XTO~wn#W!5gI=3Cp;=fjZxJ+!Er_6b72DMX(OThI95-J9LE(ecOTXVBheGEbwx zv*RplQ4H=~p{l0-tyNtHJM#w6#w?10Co*5vQ_^^<+M*OvA?u0E!cO~#l;(u=y}E&z zkls&Ln2-+4Dy9w^k-_(5ydh$z^FxTV)@py85~qx2Vzj74h15>*q7`6)&KHLG(RAc zi#_L0bev-1VKA7OiN%7|1Z}XqUpE-Vkosgd=08E>(O}$&sBff2ee$hL3>FrpkSf{r z$?oR)H|;32*T;1`!d?%Hy>>MABBIqjyPN&bfrmZsW5K;Y@z6c@&%|CyD?Qq1ZMM)o zgXad9f?X5daQdxFBX(oSR3B1|?zuS=yQL7Rkowu(bL;Cu3~9Vi>ITDjkBIRG8{-j? z#^VHtlD-i_q_eH_Q6=A&czAg5fJ}^+=n~@_wBg#ONj^wC@uVsxkq+8pYDgOT+n>F+ zgAV$3CU#4brjTkWMpPfF_t+F-Mw@+F_y3rC4>+lctbMrdBxYa+h9nt!1XNUDGBYS} zr<;)=nZbZz7$zu+fPguHIj%XZ8FK=2-c?-n)iviFFs^BJb@RUauHW-io!ed2VYrR# z|M#`OrpN9&u}+;jb?Q{zouRWwL}xn(IwK%~))>G6Jsd#l?Byu-v_=pPXUzA?MrZbZ zl2IC<)6b$rzov{Y{((0aa1dkw_rJMleI?C$Dk5!y{jXQJp2DRrrCcHTM49eswZ#L1 z{l8bZMnZe88B0;-Y+i{YUBv^3fI8PmuOzRLguQ_ig}s+Gdmt6d!&d ztiTTn_WxhjOB2H95}vGD@HrXbXC~1vRG+4SFOADW#%%gr|I;T2^mDvlr{O;VUufXF z?2J#yKPVypYxsuz=MDC{fLsxt_Sg-9cL~xMJ@9o;tl@gY$oi^nR6o@p=k^EU3D$$v z5Y&tggNDo0wm?wuf$Cy%UA65kb(M>Qvg?Y)wWC{Ws<1dU4_axdYN;;=jV|2MNQE0?{ zimK|0s?NzGm~Yr)N^@*Bo`u#p3tSiU#4e8}9tOO>+9o<}O7rB#*tD4q zGn%JPn=-X|kA@iwD1Sj;RT3UkKh@X9G-LLj2}Ikl4xIW3!xEP#vu7-riHAU%ekkar zdTa@}E^&Yuy;+4CmO8s}zH}ojG&~>-Z*nxwGHL?DgQ8Gp^6Uk8(nym2`>EdHksu2r zNF3GoSBR{`3Xjq&UB%cit6|383sE)+?F~?U!z1Xadd!$TYf6$F1JyR+QK0Zfqj01c zr22)&00WxYu$Mg~2CM$z!J&j%3uZP?o;JT}+N@a9d@5mx8W0`<{0I^3)-Yp!qlO-; z28M^GI{@^o1v6%7;9+V|*1%2k78uGAH8?zQvi~dCRt*Uc&6?Ec3>i6PYN&>+Kq-86 zMRn&ctU%=jBYlJ#79I}q7s{0X#RAmCUxUIUXsztU^(tCN( z)P?)-k$^4gp|%YV3&Wbe@{jlb4E$v@FQ#Xs3U)nCuY zvlaY9`L}8g%eQ3m^luLjn`}LpJ*7IgJp1Vr*N|(c72;7B>E=%<*9|+}q)ebvWf=;7 zGc%|>3p*65&ZkhR4Gx7V-5e?p`pXhNF^NivW+-@f7L`ilP>{*fsMKtRg3UaSN;!Hc zEEy+KDTNJ%B>hY(C1#;8WSUB)dMp%<40EYe?u9~;ZZegkyioAzXHzL84TT}ibn34$ z?jcM#nL3|Jb!jN#0&PvGQq~&^I@u>;MwLq4P~b^Zs#K9?Kc~v`y+Z+}Osev5>rlYX ztm?1VorLPREtppQJHi8JnpZiec9?l}$6)^}SpIuzjXwix>RkMeixXz#nf@N-eA=7;#%l?4>iKL^V%Ks!ci)}fWDhjU6 zV^x^wDQ3z@*TqXdYNrl|8uL-##6}|@^(a#d*bf^WpaEt`_=RnXUMw6pog`f(SZ|Xx@O!kTmR^yGH8jUseF!9AK2yIAHy_sF=Hf9=B(=uK$Xp*X=7)G|{LjrayH^0rO21`#aQIO zr{RBfK~(;|{PpqqZEu+dUOPT1!2j9;DEgLP`C-}GPgEiDnGACoS#sxPl7 zkD%HrqO!S;f=WrcP>NqA-%u6i?yJP|Mk!S~=mjdAodu;= z1owZKd54bfkN#TKT*r>Y@Lwx_E(BuZ=Qas$4mjfH{RfE&gBMo4!W)D1eOS1#stxm6 z2EAwH)ku~kMXXBJza_}htL^M>P|dj_)`o1o8u1hNlXyd{%9|bhys+xc3HXKRZ&4Gv z=-#NoBQtO=pwmkfb;EGZHG`VA(uGyURYCo?wgk*WIqD0X?=c5NaQV@3MrGlI!re|# zhraws;;yQ9ZH?&*tG>p$$!5Kig}7_kRxkviN{T#@f_|!=?NBX)P?ymB^fU{h+On|& zOQ;oseuIOa7gk+Y1qVnoSfHx1(OBrwYh74XTr+${~^MyTt!+7{}-2_U>&J+FPncGJ}e1pFBS>jOe29r3AGQS z4L#1AI!d5{bP^@h{voL~#LXNfjO!b3%_xDmIisYB62?tJOZM2C$vrj>Kv49!MS|)L zx=Ww}B1Q=evD03GVu&^2XMF|A75F5Y!2fA9Y7so&HU63A;z<;M`Df;9$5z$SR*$Mt z`@ume^HBs*PJf+AO)ecpHa7xdTT(qrHV-tc*3FAc+>UQp&3SGQJZ%nttC=3QlR6>K zIW@aV%h{Lp39m%=R%#@NFKRmWF3Q3Vm{G^N zm9{|rz{oSybl|77H67z8INYqmBl!0Aub2D#P;_-Y<{0#7IBAo(DW+I^)%0UU+zWDPz3GY61y?&VnO zS7ZTWub~%LRa7Ii47S|U-ev^+lu_2uyFBm6Spgr>)xQ`DicwG)-^IrG!eBPBC6r{Z zp?3|=KO5t2z{p592xF@Izcwzw8=(1|IT^%?KHC(?UQXz zkUo1mK0V}g!9Zk>b$Z}r0`ET{i*ILQ5%FJqqSb3d_4MfH{{g%MNGE z9(=^%%gQTO)t75$cEGZ+O=Ax_Y}u04b#=I+x2C$TqPDWWqOPuPT>ZGQtJ_ztUb5_f zF>Q-=URC}#0<$YVaMgh;+uPd?Y+2c|EP^;GPhU=jd=6L~i#Me3dH$VBHd4%mRqTuh z;OIwp^ie}F0L1~SVG*>?p@ak-Bj88G!4V|nW1du#3XufEuXy@WVGc9B!V#cziHnf+ zc$0WA{=er`uc3WHvx|ocISrK-5s`>d`m}!<<@d=QDg(V?m}Qje3_z;>3x41Ru{DYs zc>aw>`Oh@HUX8Y3xgog!;GY~t1APdbi^?npfnAiB&{huS(75U-58;JdT#jS7{bwd_ zQU4Vm^WMvVR6qZvX*fP?ZhsPP|DA^0lhGF3Mz=v5_U^y6TjIl2<8En*>QbEG*Gv1S zvQ4V2j9ua#q5oZ}+k3mOEG^l9@6pYNEjxJG@*|c-u;l1IXVp=y2OiS0Y|PT-tu4@# ze)P(Ce3GYN{x>|88J^6_av18=(lX*{qSQ07&;W^v_dh12<0Z0?&S2`Jx>&XgIlwu= z2`ZJWv8p&YyOPQ=UD=4PEFP&}8m`4SM=k6#3AvvcIT9+&*DtLY?K6!^y!?D2Bz+#^ z{r|Mke`o0P81kR-J~w{;nx_{fS!nD_V1b2>$W7Dd;^!xKHDaMn;O8szdDY6WylzQ; z9{-=w&rhCf7zP^T1sGNGMQLP6O@{wVV)*F4OrUNpBb|JyFkH|H!*c$0L_wV4s)AzR z8J1V2>}<_N=JtVCwg-7XXMR2ufUG9Y`u5pp^e zso9%12tRSQn3y+6SuUZfETVQ&qn58cU`)%3mevE?#~j$wwtV@DF$XR=ynW1}OUEo( zK4$sK)-khJw$5)~+13sWBq9!D%1DP$PmK75HS#EwKB~rY7*!njIb?#MBFmIEk)?YY zSt4i$S*DyWB)U`8U9R1ns;4miW7l+N(1P})mNsaa@+n$U)!ma-%~1%_QziZxq1C8K z$JIC{NsSTI3-?*ozH&*+(j`Z?tX{Hw*_e*yD-UT|O}V`aASBYMf_Puy(ICFtR@-P* zV|P!dc(p!>Q@Ug7eKy|0Pi(v)x?}2}0ZwOFnx;F%8BXgbBue8$wWdo6SQwU9;tbUh z{QtyIO$vIa?GH56eQC1p#TiaR{2zCQB~OEZ>aqUa@9<&ymT5fyPKJA>VOY{()1DQE zd+DHeBWz)G4=(~lIzPfswq!7iE}!psfqWI+xDy^gsUPRpM-8?{l?fq{PWp~EE)Jvaay)+ooyYDTi zjPaB7wEp;sdf6_ozsk4tQpQzo&XTqW7P7fZmSHdj^CppT2?SYom-t79DVL@iJNhnBj$f^T~vS8Kfl~`;T!N@W3$yl3ll7o|iEvQt@JG0qfRp|;Apag56t5Jw# zGoG{2XvyMe`sjyQQ1yHnb0RoF#hhlc{*&g^H?uj-i0< zLT|G+IeME5;|eu(b1DP)Y^?uuk88E#Aq}3|z$cXGr=ayUPbdi}8?$1k5I8^}`Tr3H z0gXE&f<*8@V6n^D>%al+vR@jz94cJO{cmXVBkXJuj01661V5&8{bSu2ZF63*u@rKJ7^<&D({sB4oSN>RM$-{63xVeb{;$8*bOf3nU)srpGPhY+>aTT8 zEEy)+xo4myb=^N%*QW?+$Nv|Q^e(0NiRO1uUa9Kl4EwyMcdxqN{RZt+n)HsSGq7Uq zJqm&mi2}cwIFd4@ak9Bjw5$wB$dC*#JZj4t9|aXqd8czY8761|e!LCd&vb-)=2lDdA{`7oTnp!{)B0s+l1B_=B+Fu#j~>Bq2gi0RDV4d67L zL77cw{y{=w5KX5G2Wfd{rZfNYczX;}w9jZd^PfUXn$DnP)0qz-Snd2T1eJ+JNV+00 zojRC~JH?_Fel-$*tCqAcTiv|0W!37*ZA+V$9MT@aCQH|f>dLB`YFy=157b02i-;z! zf9X^V369{nA&il(Kx@39|I8FHsL2?SiitGQ|HTm(K4BwH7bONJB0bYY|IeNolDTIV z;pV8kQoaC5Yve$PRDJ`^_tC8ygntlCzMcKh5#EO8$ZNiXcNy^XA} z!ba9djkX4=8D5M!2|5*Th`EQyHHdhg1uQ9d|e*vU=rE$uG z_+SGOVz2okL|GakB<9~s^8X{sR6B&=RGk~7ULcz~b+P z#o@^;O1JF&jZmMErdw(Z^@YLucY?MpA1X1nHN@Se$mRdWm74=#A`ng%vi~_0CM0u0 zT|mZ#t8Hbyi!aq}w|vK*v6Qv&*TSx!hFz8)6Hze#`|)jxF0^j;;3`#EtOo)9R>~T_ zUgBH$2snlK&R0BGD2o7l%7#Uc{qX(fHY~`QL?`=j|Ce29xTB3@ht%v+4a4d91RpMU zAWZ|ZqLY1YbTHA(5g`}?Nm|F7(R<@aFK>ivC#{QuN6y)Qmw-yr`#HJ+oZ#j#N4 z=z1jO=31k?%p^`pUC4DUz(&#~Q@hT*-?7X7eakpFL_ zx$1oN{J~0L=*CHl~vpL#jv(vP#R{^|JSLS@TQ0BJTcA+IV$3 zIE=X7WS{_B*mSnFT1M z_+K4dO_;WfFmFyg2Wggu=tjiApdWM&?0*b3{D;cNqG&Tki13<<9|*82F;I zEM~f)YH;V49gJ3c-~um6i=i(5*&)w4x*{Sq_kYK?S^Zq65A*EchItfYw`& zbHL9RaGk%XQ4x5=&?1^)K=z0`VDCc*|2p8`4->(_OcOyMokWaneEt^=XJ#71i5L;u zd?7JY8xAY-t2!&TYD4|Ew&j`HbjpDx!`iux`45`T%w*G%tVUZf|Leh3MAFrP>D0k= ztO*6$$((brvKm$7$I5E74;5|=i3YoW$LJ0UOsrE1m}AM9mDOku);|XE-8SOR>`r>7 z{Q>aBo|&DdXJkmX2kZaYx@YD3`hq&o^l(%HISXha$MrK3{%Wp;AENZi}O z_&@Hhib!|?{WDcHEN^g9zo5E|ladT-G$9sopB+~@m|CEvMfIw=iuOB5)w>;>(S3H% z|0Vc&!p71^)u)Vc^2z}u;vBH815R=F7$Q;IFkI8(1O>Q#!J1XcrzOq-PW)FyaazW! z=4vC(co!4J@$_r}V)hQ+0m1s;m=48>W9g&Dr-(yvibD&8&pF^nh{IvknB0ARO@jk_ zS*tU3ExuIOE*SR&KG;AE@W64vLIa$e#sDQzabU3inHx`36zOG0Rbts;B`SJ+#6$%a zB*}@29?Mb^6$jG(Da71NLcDDY@fQm5`IH&=CJFIx5n_HCAtVr35!mQwkLJ}2A4sg&^pwr@LdSL|@N5ekSO0p6aOM>~|@W4bxSCI8+ zH+)%%iX|_LV&-R1%o6PX#?w%mkQ#7(*YknJ>fM7_|IFT9pcPZ9sqi26z9cFRsuzhC z#1j?WJdLmvlEphQHBoU8)_=P179_JMwsp{jLcKLjx72_GSn)v*3iZ}_q9R{sNs)@!c+mVJ{NmLvhA_X1$VyaLPQXie7L`?^qI6zaR*8zGLH^rtR|fkkD3Y_A!*CH_f)DFYa0g3; z5O$(s=`86MgNbGy$J9p^S-uGp2{?U|Br2Ak69*3x6(yQPV_s3FL`CU7nmuMTOZP#d zq9|>mqI6zLqGIX4r1v{A^!_1(tll?{@T+03{s-(@y?+Sf9}bnJX?kBA;UR4z*3!5m z?5@SCN{D4S!qTJZ2qAE?BP`t~#StFzxCpT{lMtJ1A+#gh=Hb6T?+URjUxZkeMhJ0) z%SMS1%i@l(OJF2}VI*-6miS9MLRyIB2$u!%pFu+liKFQS3yGuQ0Mtsd9O1I{4i9vM zJ%V8bGPWaJ_KqlKSq8-{L;d44!003Lh2&6;Sop$q_^^6+IpaU}?h37#w2p8&$3JI? z6>&#c>S@kP^Q}%yb%e{05f)b@vnY;m`ISQb*fiZz4Nl>rmTwg5$7vlPJeLm|qdBe+*%U zG8qc74Pj*GHE6*1Z{o5CeDkD$J_ayL6)!<5AZ#+bZa&r!W{g0Q zh3`X=|JD|h&^LxV`VCEHsUnMS_}i_@_a7a?)Hev5L;9|JHiW^H&G*Wk9dO!7epjXP z4BuTXzVo&&-z$UscS7GF+^h-rQC+QR#SjKl4&R@r@LkXiWsipMZWiD9TbJ)u`#FTE zZxA+j()Ur_EWQn4FlF<-isPT2C1EFBijg=rd>31M0}z zZ$lVN*?g~Va=>XPg(WCqFnpI-e0SN(e81=rCf^`zuC(u?N-Vw&VK8O$eOR#rPQH0d zB=V0Om)$MC3%4%chn?aOCf^`zu9WYix?6l3!eGkb`zZ&Ue0S}Eyqw{?hsAf%*5&)~ z2@YZM4Z_Wua39sf;@c1gQ#Rj+AM1dV?{1i4_1#vpRH?;x*R9O=Ume2a8-$xR;XbO= z;@c4Byiflv>xLt$0nT;9vB?v+tQ(G4V8s^dOhBrOK76e=KCIZ{h@*jpvBlanvwIEN zVuC#4YGHJ3e3?^_ZkdzsEOVGTAcBBh=0C{p+6?R-iTQsT+MCCxU5LNb%JHinz=y@| zk&OSy?#XG`ortzz_sE&T?#X0V2NK1a-R;0%0ixPOw6K(ipb8kzVsd2g-gBVd(IzJ0 z`^YPV?~^m}jrl*$cg_Yts$Pv>orMpZ?@xs9)6($Wg0|rMD8|3^cc<8JsUK!$NGcnOeY-7_Eo|4|fQDVhE-?fd+!`>|F$u2@HbYZ-eV1fu5N2JP!EF zR99;`>9YQdrO?bAq5oZ(=*uI@(v%gZ3;oExWYvy7(h;Ey8Ym2=^kKNeJ)v?}1So^E zJF$o^kN%y5i7SdYCO0_nG~+>VAZyNJyf}EU=Io_OH0E{9wC3!2QbPEXV8~t5t~q-i zk+SAIW*LZaD4v$Vq>nk*GHGukq*QfRzdRiumPsFTAFyyJo}0#`*$yUh4A(zOxA_Tq zs|tk0M1~g|%)*l-RN>T2uC+d99AfH%>7 zD46#**pChJe+GL4&QmPMtfueEiSb%`n8?%_%}smFnugo^XH9P0D~szt3^h^`S&n7> z3&*gbHWy*ic>i40ysOA^u9l?(=b&*hcIP(}+OjYi3{4#u)V~?*yhGv&%By1oG{HfI zxZgT1sDC*(Lnj}H_&@Hq^sazf`;V)(0|u*;j|=Kw&dc1%$Ng41`8<7#4@{q~pD!X3 zOE~URn_3xveXgispfnI)dp!65Xqe|^;Q9C+EG%7NQaEDw>4wFEMHZ~b)Bn>jFG$0) zjJM+tm43QF+uS;Y!9($Y+2(HSC)^og9x~feRn=Bq-r9LT;ey!xC*SS$u`gpuw(VH*dI*Q~rs+&Y9sGu=+i z{DffrpAcx6|Ey}(*NfmL(P#B_s-XnJ`WXW8AsW;8Ku68l1W+R-k?4fG9L^}wA7i~B z(X}E`FOi6FwnSe!aFpn^DB9>)4NbK~3-Jl9t?A>mWfw*8obu)plE8BUp9wD!E_H!K z`q>^@nq7f`R79FJ$2*iL&G)gMkmeeZrl&|lI7^x}%>Q$+8=X^{*Q03T497qk*uT}5 z(v8Xy|{8*U2A!3Ri}5m3TmR|<^=dY@jwUl9F#xRlR8oJ z#bzuiXH(|?P3K}pR=n%)Ej{6v<&9Te5Jl=hQhPdnVX~xP)gU91K%5Wah}tp#^#rl0 zyNIgF)h^j+Dz}%Q71Sqj|0`~QISgNjy3A2a@3dpF|y!=F&vYPi23ku+8A2#iEcP~po6?8H2jUfR5fo)My4o#X)d;dk}mM9 ztFJB(J&EIbKawOlk>})d9Fk>V<37Jt=5WZOP=JdJ;gL+SQ@S0BRTiEY(~~fe;Ly6<+xP zc4C$p^;Bp4D+W+kQtDh{=Zo!RMm_Z+hm%90?bFLtvwj9_jE@XK;E1RZkC~Y{XZEbd zS$I%P$lIh~!@2&+LTNTW(T}I)IMmj{BHriC&^{s=E%>pNCpqeJgwN@2sg65-oacu`&d#9J2Ld@R&+k*FSd{WG>9X=X>`#t|4 zIcB?1`3Hrc)Wa*=a6t~l$WCAGa6u#6)jv};*NCpq%tqG+yJcjjzvy76kzGYSVA?e= z>RQJ5>Dd2iNR07U+%Ya;@fqCzVOV@YhOqdINmf|wy+tYmo^7SOCK(Bf&u9Y{hQ$}A z35$iDGfo$FE@UVVm*a%;0iyMS=pv2C-Hs{)LG<$(uc8gz+(q1u@;K3sS03dl%!dz` zZ74P#XaJz{0ARTWz|)o!p@(j@jkFB%{}JmzUrE9L5q&(`iD8HAIDWB(<5@H1Fze`xa1PG46xOZ{H_;VaL z#L&2XHpj~naehSKB&mDbfu8SF$RYr2Jon7m3fw zaF&BUNRSL!BdK8iGxk7nev0*lIM<6feMKC?S>pWaX@^CM^L7+%^b#CR9dTIy=Gxp$ z|7Aw_Zi9P15n)`>X1MI6Fe;+z}AKSm7{=bb3p=>3U6 zoCEO(RCaE#|8Lszw%zlCEykIAnk8=AWt=plv59B6w>)Y(1@uT$Ci3%m|4BlPjG8{+ zU7+<84bkH3dLj=sbQzNoyET!Y7sS6tszq?*>(~bRGA~Z#0Vlt6IN^=@-}}c)AP!EB z*POIf*Di(=Y1?ch&MGfrJ{)i8_xy;1onh1a{=u@+L)dh1cyq}nY$6xrKx91sCWi}# zO%rIeI^O{q&(22My>TJ2qwNc@|6}zf!=|g*M(?kqsUAKugIrLKwwz`z*VD|OWU56Y zHjo+Q0_=b35a99*Gsp#pTQi8sfWAFyC$dE>GJ{;e{U6SHSEQLi0(MycM0TzacGAsy z7kqA0D+2}f?uJnx2{qyQ!Xn}MiVQqoxTA%oTRyS;t*K*w!6H0gNdHfsuS&!7LbQcR zUC8){L+2{hj&IiC(Y#0a(`8|e^xAZ7QuUP;xWu!yqEmxqLJC8A>g@SVdgiJJK2o7G zA*Bx;N_0Sfi6I5=U&(QV8LzuIAi`M==pwFv6CBb3y&FXvT`_^C+92dm+eHmdTS{}J z->900lfm2Gd6@H6uNd>*BQz zP0IDR7>X_Y%SEnYk&AGaTo*s=z_E|skD^VkE!Tb`*B4G(%5}6qMO-9u?o-0c#hg3j zn%@MG@RVAHOSK?ELL$&5jQ_bjg_x90mG-xY63Y*ijOb zT=Ia^mI58)@1~l!V_Qt(yM-4BF^LG&WVR4$gp7!DX_-Tc;`}3qol5^Q5vQw&LpV#E zOBw$r)If1QjG~Qaiz6Q4(o>wa6z5ofn%va@k1#E~IN}lXg}v;YI&fuqgiC*LC{db! z#!w>TUm?G`t3kw0r&iC3%G`-u`;B3Yi_v zsQ@EZ7Vokk|E(GEcYxQfMf z)#ZyjI-TU0%z6u~5VJVy%iZzM{OD^wE`ZmnfTM2trP>c@lAcI(1?GQiK+lExZ)QGR z(}YAz!>d2QPR#UbFxD%W|4FzD#$eY_YR6qPN!$CJwub`#3>;e2c3rO`*nGi3$-QP} zmlZt;xQSn+y|TYUgYo@`{;P8K3evusjkLKwv0Kt!iTKZwb`fT+>vWsI+^qQi%IloA z6=>Vdzd$ZN1Sc05PFkvANv)NewWK$Nlhycy-mg*)b`~kDz!6Ek(YrTJ=yJS<8d71$ zHT#}CkiO{ps*@caXj!+=m+09H5+0n5t_^le*H?Y&V5enWt2wZBEkpZi-v5yp+IP7_ zTO#GFIsTd8yDCGZe07r*DVym2H+6RxKwy&TgEtg^^-;jWNcrkCk#fM!Rl?5II=X)% zZBFZItpAeoaH{Wff%(#Is%hmU}s|OkcgBSg?qmuVMU4 zJ+Di{^WJESzPL6YZRmW~sdjv`4jU{HggAO~pNL{C4ubM6d?2{FNLhlZYgzx4Aapmv zkAKDbA^flm^y@!vc+KWnTc4+`te5$`>@ssKX7yXQ$dPkWZPv&wZOvRdtI908?f_~uyY+<4%kWOa<2c{rlwua3-3P0EhLNQ8|eSZ^K}_`zG19| zrN`d^tzIg<@n^vzJl_ESuX(;94bO|v7AAGWk!VA8-=Ny@4Xbjt<*uu~{dqE<$IF2k zs$9>M5jMo;E|@kCIZjz-b4!)9#Pi(nl|zr7=YM0^RQB%_&r>Fzhj5nXxiR9v(erGI zqK!`Iqp9{hd*Tx^-5AvWBxD-yzaZNgyD0yK@G{{XIZ7iZNA&`GH=rj85#`2v9hNA{ zzhi?S%B>>GAQ6RdwkW~=hhdMRd=f<)ecc3_YEfiHxT&8*gWWTNo=uBA0iG?qD7Y~T zqQp~F9Lyvl!rZjRAw*&RFE$*)+%3Wk7hwoz33Jm!4jhH~G>SI*ZVNQk!nEQO{c;oX ze@V7BlCi9=YrQqEy5?ny7^J3E&FIY|M#+!fg$w!@M4)EG-Gyb_hvi{F37;eK!>zM zrdxJ$=uxJhWBno1O(Ii&k%@4YOt-W;aFpruDBAQO7+>_ji|~d_x480CrtSUbqyz#o zJ-0$X|m7?Wcu0R4Kfv~*Z|0Mv&b|+WFnj;)2%`MS7I!E5k;F)TPE2Pyw#PT zGVQ?GQ?J*sCunBRP_7Ppf;2z&6Ebj;uwt-k?Frr*#6N~LisZ$HLZmxHq@f}b;VhAE z3+jIoB7GS}8(puBrn;M$?nXD=R_myZB8~E2krEwj+`baNo3L>kh}5+A9Fu}$y*nvM zC*5{~LyZ#U#D+nl^&-(Qk%(}%L_z&uLZUxJ(MHcPXlhGzdyxMyZ7I=?{!6NPC`mw~ zm$H?}-M5_zES5yK2kYNNALYjI_$>c+k!XlWL^w;L+c!9x;86N1iZ*=+#utauLcGyO zx4-AKr9?aVFRJEYY>TAgi{T|gQW1TWsN$U>RLB`9-7(pr#Qw>P;T~rHHj!qqNJBVF znmdkl;3&=4QMBouMw&ZraoSRvo&D!kb3~+hK5J>>yS9z4H0ygglqgMpY#^k$MWh)h z(h$y)X8m*rj?#P+MH{{FKvUf}GQF;M`=8Q`=I&`3Nnmy z_7KWd)V5b)?XkI%Ze6->XKJMJXLC$>dcm5%=&4n1^khB@c`mFN3BuR3j*E@stpLP-+^SuxF zc$W4=$_{zjv^=5a4di=X>8p6Ig?Jv6(C{?i!A!TX2pkwwT++`u-@9+Ig4o=cs!Eb92dKS2TvsQ(1bNo#vc8+^*< zYhm99C&VoIjC=m-U}sUs2TbMbr+Oftu{6B<%_g5gF06dUy;%PzT$`eaO=9)ic+vJb z*=W1BAIK;scJI3mcAD7rY@@w07||vsBm6$*{}Lk{Nr5!^jQhC%!Q8+N8FB;n?QZ1; zOxkyYI+#X4GLjp(Zw0V0pK)WF+(5t%>z~NZjpA^;JTi9Sh z)HKgC&i;%AloU#MzW)N@`KAm!-;ecwoafvt0IAko8exKfSOT8k5uR^J!?Q#o_x~Gh z)_~_*R68Ko;V@Vk5{%}4NJJs0>S2IqJ+$J;WaoQyh(cz~&`%QA84HejD++m_-66y% zq)TiB9Lqf(8Y1mGLZT4DSy9LX%>O4CWEApU6m13(j4#^yzIek4KJa^|E#2frIO(Ss zWDTk`m(R7;X^a!!CvRXI z=U}AxoQJvp!$E$F9^`&nKl$)HG@{SBCByiAc$GDNjh|e->XLpC#u~p5Uj{7nleeZB zzX3bg|IzH+D(s~5lMiG6C+;m&8K|hYuJ7|lp(Z}(kpkiQ)(kv9g8Gj*mXf21<^NuF;{!?TnkJwpG_p>wdWA3DV7cM40c!7}gg+NmNHV0&^r0=7_`q!{TQHo+CA7fFVQy-BV=gS0UB~3?}rzK<@j@xYS-t33Mr90 zQajq=j1qN??Er}$5Q%mWi3n#&^qYGeI7;+G6m4`~15I`3>?1Td|GU=qi2sRfT%k_v zlkgI;c71O1yaltG8|Lddjbc>CKIFE`rqeUFTS>pV5tpi7K{uD)< z5d`CleoB@Czg_RNrL8^ce=I9)q;x(GFAh>V5J#8sX=!L`q$1Kh7Oejh=2jfr8PYr= z((EkK1STrp^s(6vhm_{WDB289Bh6z${F^Yh-(V4BYA+zo2U{SGq(qyNr1`?(ihWZO z8wF_|6lq3@G=#JI=J8Q=$!N-pf{+M7nLBE6wA< z{!gNBy2rMMH1~@%+lw@Wv!(gg(E_FUa};fKiW5z>rAcVG(d~aq^BC^CFjqSvH2g5U zGzbl0Y4P0d0uZu7!;QiIPePa;u^l1ILn6$MA`IayVKzRLBFtZ+Xj5i)jrg^VpF3?S z%;T8i&3#YsYafLd#`v{8XV2TcabDo5_W);>UweY{f5PX`*h*tNK_WyNdeYfRBqE$8 z(GzDnaO|GHM$u+lTOvsfJ@J6u)?>=;et(hNt%B6hA{`RdR8}rV*w^{;7u3nt*HbVl(06cN3ka9-4mTA15ma@Nrx=eB@6M#FBFRU6B9Mrhy}j zdFqe;GO1}m%57Piv`>9`MOG;{lC)B8Pjde|-gjOZb%$BJv7O+^(v9wlb3)GrK6;uu&3GL2AUH4Kl!N5EZECefAZI=<~kxoT)b8* zzoVkMrmnoby3CbTfON?|x=4n!I&)kSc zTvFbiVR`n<|5(ejoaZ5^y7%gHx4{R=$nxwN+CP_PccfXK2|LgB5O(g+OUf2EjU|Yl ztrkSd(O3PkWzhQ6l-4(UUZrkb znP@#saNn2G`WEF?;Kzde3Owm6AAhmwKxzJ2y!mqE3(N5>96PO9zu1bJ0@J@QiMKpd zx6FxaiY&%54bji<+5c4ZooVzlT#y_or0%4C^#05HxgKYXuHFQ>(}r@e zc>r>I3djdJ-mV&C*9;)R`X@o)93+f$yo^TnO)UuNpeFf z3~oW`g;s|W1LB^sF_=+)Vj4weNnU%wE0-t@;r@~)mpLTQmv z0kK;X$%{e$+eozt^Y+bb<7^Yky!~R3|2Aza(6+mOF|-_`y1>fCnw7;>71am_JHJeU z^Y$lzvgYlVg8grUp7ZwItXR+*#=LDR7D5S{&Ad%64i}}Mn_mj*Ukn#>5uDuM^P}lnU`MvP7vLnW?tGJM4{nV zO3{XH;eI_YanSvJq^BKB2|kL<)hqZH%?T2KO-TX!#|V#kEZiv(-76y)OTjoU<7_bGIz+N4m0`~8S$2<_JHDv^2{kKc) z|Au+YR1$1{3YhfdtLIxLk#j#?!u_dgv;x*&hdEHbIh$q2Sj3Ya&@{%TmA!w((e# zNU-S{!5II$9nht($Kp(aP0I-OK?+!5ACL8d1e=-yCNanx%>U49-IwgO#Ejlx{O__; z)Z1g_9`Q~|!7G;i2J#>A(5355Xsy2KnB4^*mSw+jUJCB6y*!qw6ZdW@xC54g`9IP7 z-FkY_Mg!KE5$r!H)QU^J=wt&nITP5MH7Q^vJ-lelfW{>i_!Xd%Q=3VD%ZnQvF~5JdcO%309X8EY<%F$n|(51i{8-0(&Rb|DnD$ zT5G^+GlHf1zd@eI`wd8~CIw8E0`H{yzro7W*Ku^u$A<$)7+GMx6P*9MpO?F-^Y9*% z2+QmR<~y&%4T=j)#4lZj;FGUy@D>;M4)Z^BarY;?I0=>BW&V#b*Mn(7C0UQZyNi%| zP=`v4&Tr}B=@`BUAMs1-*gOCU1jdUNCCM$ z&(r02068rk$a`H!(^oc}(oSGJ7dlnILKf6f>QTDNuebSf55A7_C2pj!&m?dX<|46TncK{cd6-BIXauTXRN zVJ4^}QlQpj?0jILKF9>MJ_Tw6hR^#3>itYmpQS+Ei4pXkfqE|k)Q7?TM?!~pVGzA* zpx(^{73_Z}pzg*vf|q4~y^{&*loSr{!BBeJK)sy_>d_Radoh~cGEi?NLrJ9c;g2a$ z_hCT238<2MeAr+F5fA)Pu>OBod*BFusZJc!us!fc!T#sNdQ8#K??-FCED!t<^Pe13 z4`&!tAN?8)IHn#;v+7*}WEj35{T6MMZo_S|HOSfG2d+z2^fy^NvbN;6UkEfA965k&O@z3LYNHZls z5}W^+vP~N6(}!)@*6Crf|n+S%ZriBYQldD@E=}8Cc4?g z@lV}sOrsk~^KZIBh;F2Av~#}OKTGHDRUodF=HK*Oym{dDM}qfs;{F9KDelH(aZd*w zi2F%jL8Zmjl<&b+*hb!^dH6Fw0uMc+ro5xBy0Y`@Ya-|&eH%*zn7t+q7iO1GH|K($ zJT(blLH|Fd+VRbyz<4GOl`;>i>b=M>yZEKW(D*cn{~uF_2#vEygL>R8qW3P((*K_H zOKzGWF^*Fx`_mI0Wiy5P84G9hQR8Y9&g5jFYbH_IQN%5#_UWg~jrcuoY-}uJ>$4#KeIg!^j)jv+V{D(b05y&6iDY9t4(K5NXRiFD zJuJEqzj1K2_q0Crsz{77`)Y&7QH|7UatF*k2yWM+TE(3h(nNvO3d)URo+#nkrMXf2hU zVv43IiQc~C_)nNL_0}g=0lhsgdaDTZMnD3cZrJ?8AP10o+cvr%`kin~Rdl~>^p^HM zNTa^IFzO>u^!YHlKLquEi7{G@x>>Eh>1u~;)R&Q~VhXAWiWa{L;=jar?Hj9v7B`9( zD+4VOkU%FGZoi`cOQ@e(93F)s=%V6Q72Q7@En=sUxAyR~tI-|i;ei;g15}qi8s;UB z*fqh3B`X{hP~KO6a@e8m)u0koUwrMWQQ0UjiJ4+brOAlOzGnQFker6MO{@wkdqPxJ z6{w7Wmdd`q$N{9vMns$C00>mpyag)b{ToJQ1=*_Xn_&MVVQ;ml(A0B+uXfx5l?BWc zQz~0!-~7svoGR-VtA@&cCn~EBR7OCmOeXnnnE&M@|GUhS{5RhU^55x6o(q!){7*gK zjVQ~2R3`av2gd;zQAY5t!StDfPrj@G`rF;ml9T-Jk|%kt7h!ha9w4ZU4oUk#|5N-{ zi1%;v@MmEbkzwf2li2YZxF(Bv+f!u`GnTvQw9x*=DwTb2wH(D z{6v3&6m=t-TaXhPG0nBNRuXX^g*O!W-CYiT?nB?hi)r+o4*T(9nk+;Oxf7jWgiMH_ zf2?;H;a2pGyp3~L4{admVk2m{C4w#J5+mpzx&Bo;664zTe1$$S#8=}J9b1YI{o;#3hTkE9H%#tgZdXrlVH0*YO2ZIQVbCUy zrTg-gZS5=D+hWU?9(Kqw(-QoVM|gpjj{5q#%Eg^O9l$7jE;z*~`H}S>iOIbJQDG1N z63kRlXr1H#5;Ok8U^e~*J%%&jiU}4?fa5F&yJhoj4&_^jL-$&#eF zwVz5IvUF=V`M7Igf{4{>#0u}$h@ZH{as0Fb_~|&FO5->pXb0W?6vRJI>4}JWgETPjyjV8hauZr@_#vAJVv$Ot#SIALc`*=Xc z_UzIsJ|H6tp>V8$`XK5KgZ?j3+hRN()A-)U@myV|Hyy`ww}PPxRZDRwG2XC z#^F&bLbYdOrD<*;0)3ufhJ$ z(=<{Yb8ZWWG)DT@3xStL`gF399tw2m(7)d2$WFqySK}ud=^lC2Dx3Zc?fi5yUzGp1 zp#DKmnO*}t?gKu`@bsZlSX4_k93VwY=83u|-jMHa9RF0nW627TY4LCOThpSt3Q+2d zho(WTdft>S{2S{Z*oDue=|Tyz{>J@323gPOvbk$HcO-%={8@^(c3oR}1L5ySSQ7Lq zCY23K&+35>iYOBN{YsJG*)$S}5&nH6+Hjh9RtH(v0vchHelcX@i!FoJ^nZ41h?_ac z`bV(-d6p(S@LJ&E9G%8w{}HTzo=xtt2>?Qu{v%lbJfk-dt|cO}Nt4~yY#{3;?JPty}?(FLFL#JD*uged5J;|aobh2hj7KVAp~ z``?BDQz0Xxi(s@Ir;Al|QBf9M&4djeM$L=g%tYyB$h;AGJUdQK_QM;7_rIg#{mPJO z9khBn7q|MXHZCt%fT<1+HJr3b+!RwRO=y%5yWAnhTuA>|4Rre?*+YiBCZ~*I7(fD@ zD1fg!fYjM`(H5y1?@-Z}Y;?v1YvLK}Ms4WR5J@+3v&eAt?{g^1F578TL_1q&H zy`^$gOzAW&QQg1abm&oa17fvM-BY5v+CX&#B+v%6jLtS-u~0ok)28# z7#jzrJx%stZ{q@`5s*qd32%&w|HK?XDs6{otK6V+fQq(eqcj|_wjX+E3@)z|#pV1Q zD30v32up7{EKzZ1`Fn}t8kN6SHi}EbQZePz#Khu$p5hRs@&?7~puGPP<<;esnZZgx z0YX^vGm@l!TH`}W~j4>!Nzyb?b2iKkHaDMHa<7G;{x_l3Uj;2oO6!$)I0oUGP7 zv>6_1S}UvDtL-bds@izgman$f`nIaJI!-j9d3ZF5hrg!q@bPHR=kf}A=rBA~)m68) z*R)%y-(SYVMwf@2qf&U-w6o_QWOz78c&KW_LtQ#QWSos2SW(4(kiN+IzZ4!m*~#-6 zvq2B-nuqqqRjqaHHMXh$GClm(8W4F!+kCf zd9458vLG=oJ{#pBo2_|RYmU_VAR;LqXpZ9{w=G^A9vU94I`LFRrhwvR7BXOb;(6@et%c6ZY`c zwjS;k)_Q2xJS?uR?x?G4-+Fqu%jE&3oO0IX3{(yta%9ah}&#@6gSf>AJt`D z3J>2zJb#JdVTs|Pu6=P^WtDwO_m}m>OTt4>d}7`#90?wnkokS`^4{#ql@xOQPX}UV zcTTL|@~W0?8U0{aZdo?*R!C(fqJYG%TuI@}4kNr%)%VYnM`>X_dY*|g!mZGPC}R$U zpuVArqOriQw4mITyIUcVkw_RJtcu4S%#>`sUAdB?TdahU*_E5M^yCr{Pb7>=ie3bF zjDeoZm@q0S`cX9WoK6@)f=&`fC0&OlwT8GE6GkOnXU1DIVMN?iLk?-+mDn&Z>5BZ9 zwy5WlEsDEeP;}SJ1eLZZpaLQ$jHvFdxhtpT2Tt;s1MG=ixohzaspV|MKlZL%Nw;dN zFN$=J-}B1jv=H=#?8=pN+uIgEF|dEc{{pfBCwJw(_^$@`cciyfg7*Q?3JkF3oNZ^`4biBiY2JTuI4|4q1A&2LF87!GT!kXY17hv7Q7S zsI7Yt{~7nqU9V3kp?@gVEoxF1&3#*2>*ChR)=ux_)D6Qq*9>Yx?sc~;l!fMv>?!l2 zIJ5DlNblGxgCx1?0Dq=xuAxlWqL^t^7EUPa4?!K)rdLUiaSnQJdi~(9ka-51UMseO zAqZ8j%0#Fh>m90mBEW+_HoXF&WYa5L2UtRJhaNgB4emd92IYm&xvHVJ-WE%JUMepObI4Jz=+@y>h1Q@! zHl!PRpY5>H0`_WAFRJ(HD^*o?eOy2xh**3_@2?%?xFJVf$ou^CF$*}vHiJGK>o{pb z^hDhgZ>X`)RMGP@$$Dmd$_Ci~o`yfoef?MAFE!+-?daUB0JhJsfQ7d5QW{&45WLSl zXu}ZvCB0|6k#R&K1jnDHc>fpnY)ks?WJ%CJVwB*QmIl0Fs z00?Yu6V$)Ir1xw$5)s+t$Vj_qyAeO6?Acy{FNrM3|7v6y*ty<)c-VzGgm8lL&oOSd zvZJyCl{KASSsH;C=+AqcW;8b8d5F71Xpqw%+z>2=e0No67*Hq3*ba`jPZ(mV`rsHoprak_`#8`8aubm`R=a^B5_ z!VKJ2W`Ybz1l>QV|1$lw7D3<_yp&nr^Mp|I1LG#b^^;-Q;5IvKCH-0dWt7VRmzzbS zqHS_FB6ir?inIp}ho5(HZ^4c(Sz%rgI`99p!_GSF5p3}H7O#k_f%ne+1?&`4FzYBv z$$&!~YTP9n9IMAB$TM_d*kY^CDPu<)z)|?_B)q}qfSVjZYHi0Tj8k7-iY>PGY_!HZ zjJI%4m+Z6{mIl^4EKzfJ`U^#K*kW6ljpov@R809aG3o(27#N&?GL}i@4T+71@}3pt zjSrMZKmx51z@Yxk08)87MKST{YrWc3v?Ckk1typ6t}b~In;Wz$MRRxg`^e)+VRQRr zqq$^$iYb|GbAwp_oEUIa+tAo9P}}cCZMy_&BOujwGTzYl;DHVx)wXjKV^$xPIZ#Cp z$VP2=a3yUn{!m1-jL4%aPj`9;tR?n!$G1VPp5pLhV_?Y~f|$ zf(n5guYnu0fU!4rJI1wFVE*m=DSbvpnJ|VnI11u@!9V#YCJKg6%+aibK}@6QO&#gU zh*@2j;m{!eZS>fHS@B+O9BVT~YWdAhI4BIvLk9YI+ceKP}S{e&URj1oYH zo4IU3C4wGyCh#*Jdnq{{V|WQo47=TmKTXx+KWFv63_n`&=P<^9j6YvZ6MrrdM1MgW zrQ7@jTY2UGSp*%y{5K6e=rusWG5AY$tLyp?d{`1hE)WS`OCy0q(2-#M_nMBNAwg#m zbmVhjqzepMV+dxBpv!RnMOrh0ChiOoblI+G$sT(xxyN|x5_+s`KSA}Xj-X$$Bj~}V z>anj=^=ODSQ5>S+=^UcD!A7GNA$GnW&mzPJeqV`a5$5Y5R#9J#x`9@E7pSAX3LE%* zEv{{^sH!ZFz+-mP7ojq2W*yv?h0Ek@AH&_YLH+lusvX}jZ2fgl+<=O|mGAWg{|?2h zBIHwngJcfG8;G_I@?Wn?rDKkFIZ5T<=b7{#xljxLJyrd3x}-VB&>hbGFB*{UN5!D{ z%lOkUF!Vn(J}z7b7%rWlc=#y}eonQcqoCiN9gLv(z@jWV7!koNb}oC_OtW5wGDjd! zlsQ^K@rXVSJL|CG`^-OF?r2AVa=3B1;lxd5rRy=e?uP?=%3`woKAU%#5{uVseazQmfJ)bW5; z>boPq#=qA5w0$M8aE5&&&HP03grnMi4^inGamSz21@j0kx;w6Kj_4!`{s%LHv1ws( z`*VfGH!`t!m(8M1qy4+=^{0WwW|8%;Wbv&uEG|J?viMK5VG!_^PNQuytkC3Ge1KOe z|6vYy>mGe5*qXIHzI%BMy?jYH$32Qut4ZEgd2;aqLg|0v zlnxO}SCSF~gBd0;I9CS-r~UE=CX=LWZR|7wWI4{>(N?NQ41P7g&%`HsXr~ro?Hyfn z%Glu#exrwHo}l~jp>Hhjs0vZ&$iRVXVIrO($RA5V{?)D??q%1wU)Q)p6(|w+DS}sU z?;NcE6O4boi|4;+aKC79hbcTFZblcx8{j@Xh3{`_Jv>@e^ZlyEU5}c_^776PS?WGL zDGB#$DY(By;TTIlq32-=n`w!Tr5wt|bKDTkvop@-0)=7{>o}fbS$bK#5<*v^w9dhR5jN&Y2S20dF;UENI%SPS)si9|s)fUk z<6u)~=trqHBYxH^q&$%E!i3P|rxkDTGGi2qFgP=8~isKeI0LVpKsq5g2)SX_VT z;>}+XW$CY+^M8EO#7V1eIb#`<>gQ@ee-e{st-l0M#T3!jUqx4kB=xs#Y!dYMqUdi@ zpg#gqe+&@O*A+1bkoqf+qDEBniZf~lXQMxy(t-V1XVegUr_~&pN^?f7q=NMiuI9@5 zZ0cSFz!|l-O*9`)b4lD3Q!KR~-ePl=!TCR9I@HS_qoML+_{u31&E|C1P^mC-|#xquZ3fmy0GQNZrc0GEpy2j}}YOIn^y+JL7wy}4!(O42Y#gxmovFhs_VzjZ5 zu_%=GiYP4_D2;$r+DUkm%l92X+E`U|8GKbN|DvE1#U=JH$WDu4 zsb-nO5*1g)yDqijuxa*QHi}EbQZePz#KhuiK6VIFdE3QeP~NMeyjY+-0upGg;k35D z14!jnN0AThOE_E_xI7!>VW%aSRPn^&rbaVRyKMfdMKZHk7`oMAh^njRQDdz-Z2i2S zjq1`dR7~;gt)Fp0{X=53ZXcTr-MuEdn;htlfYjX@ykU6bnjBWCyPD_HZ2}c` z&7eZE77t6vPpjc+d{FS z=&j@OX&|NW`Ox<)mNzap#gxjrW42`1?vCY9Wjn>DLS=7>%BBV?BcP?SU4!$_36<4H zkx0}`LS?J7S6OgLB-Sq3s%%&0KND_*Dy#Q5Nem2?ZOUF{0XM~z%2wHgVE@}lPL=H( zn+BD=Eh?K9sEmMAnG^_2IND*7+ZVs}zd}-yaM-^1s!;c9=mLRo5q%+$<27(o7Di)l zUyN%T*Mdxhn2DSr-3`> zLf_4tvG;@Z&$~Kf4+%QU*!#i!7qo`BnKSnO??wCX>Wn>cQw=$!ftPhc!2dH^GGX;@ za>9zc+E8?ZR$DWZ-lIf-w3JY*CctQfoIPWr_VOwd-9&kKTIV6l}+yM zv3jH12mE!rX+HTw1Mehh^tNcjcqKZQ$NaC&9WXdO=zslRbuK?fkReaL6SBU%Jc8tR zL_I2ZLKZ>fbO*qwiZGWvroT_pTo+jpzxXlv`bge4X?`Ug_x8!uYa^5Hqr?cN1n?3q zRWwy>{9NaF`}cl0q`5<5r+$a|SL3t~b1`<~t`s8G3HPi_`m3=~MzZowm5X9AMR zS{P33Td|1*@?Mk7W1)ciG?UP8udK(t51pUtrf<4g@Cxq9!Tg`V-Ft$kGqu2duEE__ zUxyRpo!_KD+%G2KzCMNTKG@{oUxcTdmv`Xt;Brr_>Z zf|lW+(5pG02{8S42L8Qd2c+~u{^<>l=*?({nO$i+Q%mlVE-4l>V-1K+1=+;wdj z^wrMAy6O48DG7Iw|4;Bete=N7uUaSP8{GA{wYRc*^SD(*V%x&*A2|4V zNcm6xp|bLVEgZU4Y{8JVY+=THhb&JiKjh<(e6Zi!k)16h#99IIU<)(ub@220+dFCy z`ci8PR82NY3mwQQ(w=ST&#ZFjHz16B(8taHy4!G)a&a~s;0!Z(*QmvX%)9{T=|bL0 zb|HO*KIET60NHEk$Fe|8k`H%}rO3>MYau)0V=o^F9XrFmO zIXkF-`H%}rjXT4Fa`t54WsiNB++&O*VX?E@h42UTnLuSND7WHeCPsdc%+YWAGLudC zhT6&5h<}{ROy+o2U*zt>Ur#RD?m&E?FT56fLV!8j*#amoCm;8}LDB&@J?tQSQ<#N6 z#Gni3ev#7PYmgT~YJ+7NdRx4C+N{Zqdqwm@ESrl=;D-M@j~Ae3)2(B{wg*)~+?sLq z_{aw!ZOO1sbOvq|7eRr(yT!30r%cK1`6kV>Kh*u2C)ef&{=w9Gi!qmo%{EA zhqmLJ20WnQP!!B@_~C|Pl;))RNC;;rZVz|XP&LGswAE+PAT)c+-U z>*IfVcLaW91M$}LPX!iw>yOj;I`P)?Z$le;>yNd!E~VKSR_Lwq!Z!xtRxa3W8g5BY za5Sm>{Nzu)@L@^N#Pk1@U{e|i#9KElMH~9xP1;*)ewnwzE6ut0Ci;K7HN?&At(%^W zx2Cry?hM|#=_j;gk8MisF*&ufptqp~=2 zU~%qMrPzShY%|)HMLo$o;pX;A4bi{P`2*&@%@pC&@lNQJSERO$ND2D%@uC@YU z5}kuL`e2Wb;zDxtuf?SCCm$O@`auP7qnmQF=mW#1!F`;;zh_YYq)kPiW*;9tOrixm zjPkIeEbwD*Zq|%!kL{2V>Crud{D)y?ohtIS#Y5kl^Js9s@=wd%VrGgdl%_KZpHS>x z;~d3u*K2faI(EI@Au$viPR}W0#~MHa$*$L4!TQ%Q$w*?O|E*jWja{#A3pJBrW|}Od z2_lKoEMV+iul9;OC*8`&s5Ug!;#>mz4R1^DZH$_5>r=uGhj~|I>8nTy*;5ya`M1{6f$b*$uJY zGzKlGOt1^rK~TEukCWXsXCG){A@={K;ZL(1Iv#(i&vO2B8a}M)Y2hD$h11igX{IN! z>3zD1N1i2?Qv#{U;O27m;e zt=RUxTDl}?jlRen1MK@kyftG0;?58Q?E5#gq)~m5++$M!1V!)HPf&fXZ6RXE06UvQ zJ0tBwJDOM#_V3K|N_b#rf&M?BBT6QUiFTQdRxLtI@DP)lde8p|T;ykBg39W;4#WhT zImz2gmGT;zhFH_I*|VDGE@)iP7y*$jIkZFe78w@z?-^$?hzqXiib({&sz$GCu1Z5T ztJvTSAX!8{WioUdbpGt^chPBIc7?EjxQ@dNNJVPz7hPEsJ$XLi|__VEkhlSsI+_i{Y7b$6|N=M0Yx~arE*kE=`<~Bf%Jm? zU(-oc-PqU+sO~*c-HbqW1SF6Yh_^iBFiO=;h#m~JQL2;XXa0!qn z^hQ?p!y9re4$i+OjO{+ZSvBX99UMYv&PH!Z>=aWjO-Pis_(?~0Dy=+*t3LhrMQJkw zr4f)oC*e&lng8Sx$Z$!eO^hP68Ado1h#!@W(y&~D(vk|q^D-G6D5NbAZ{_*l1WPoy z`~5|tI3(H@Wuv$>EEQ8e)q=MuuMJ+$B1q*`#AZQxABgg11tQ;6MWMoUeUExoZAEo;dwZ~bgrI-)xO#4knj19QaD_%4Hq?TGPWY7Kh8Pf# z#9oF>`yA0{g3N>eA9L>kCqfX4XIKO!gNTSO8v#W!A_l;$ zCkD(}Fz0yYggIl*IiH@V;u%jpf6r6(R(JLE%rfiw{eS*_pSf9?H@|Pa_14?f)p(sl z48P(Pma+8N$YT+FS(W0$&MbKtKBoQ;_}qn8HjL7IMzxv`2fO04oA8iDWpjQ+pS$r| zh8jS{@RdFv1S2}Q{`gS5Mt}16N&k?=u z9q4+a0Tstr`n(b!B~Z*pf(#k$j}KyRj_H2{irt6Z1J(F-6u#1Drn|Db38bE`Q{09M zCo$ukmm{j(zr9P5;_k>l#Y;a4sd!a^4EZpHK(o0cBfK8i&h^IN*WK}zKDT7M;uS+0 ziOgge&%HUq@WE|eZ%;sA81I=rLp_YStyi((z#)sh@PS_QO#L5tY-54zaj;{1P&|D$ z=6Nv1WVkSpu$XsvM3;y1U2LROVKFv+CXxN{iW_qdIZ4Kt_hp3J!`QCDUy+$+C~n0K zjTMDW754F8(d`<@OcOBGu^&l3u@1sRJaoX62%kqfyWVs_!9&wEpVGP}yuhU8tzh9q z8*@RxO0);u9bxlmC)b;x)tQh!o5V!>i>M=JJFpxT%hC-Pixs^&GdwYHbX(Ws zmPl|L_ouj7srDz~B&qfw-v41~7^I|U+PEHXJ0R6gOP^cHRQm@=S*dnG?+C+Zb6t;P z5CeJd^cjXzZ7c%>NJLWYf}{yiM7zM-kwv zz0zlrG}SWGs-TdbWe?hpjc?mzX;Yx&wF*!s>k}MF0&Qu%7o%M zk5#y0BqjdBk41OQj3s_?e4pWrua>t z+!of8!9twk)QL!D|7Q?rFM175Z7xN^da|K4c1t2s9`#eq#CS{0`X?}6X?!w__r4f! zvN0YJiL?qI7;lN0|1=_v7mFR?INK2qPmmqa8sqsxV#+ifZEaF?%ShaRT#=H<09|V8 zzrb!Uc{uD--Als&ZETI*5~RtaT8fd_Z0RwfxZwaTi%)^gJ`kHtF*YM2k=Eb?n=QRR zgh-pkW6)xI;^9H#RjsiZPZFM7J7up4wbfI#(SV)2T}JY23V87kd?d>*8y}K~qx5AD z?GqC9!jOmPykExR|wjb%Qr&H~f?&skP5q+>3GYGP;o;y6b2P6UoROY+T6GVS zf=~!i^kDXXg2cjp#Q@NERHN`ac(BNE7-3}2tv7IN zXMCxP_nZF0C+}9^{?}L#R`)ht-lA}Z4xY+Fq6a?G{aV5PC%a#tr@rmB!qh*XtJ`kS zprzYxhx8D88fd7S`nKC4aUYtu-AJ2e$RH0~z3`#L2$)0o{2%M0&lBsSwE__8e}fbV zaqA7u{a0?3RQiJ^Wu&bBb9cS-)xRX$D7kWH%NN;q0nT~zrwez%7kuG1;TsyRUZ(hw}1Vgrk0|7#6;w|yOP3u+}Fx?TC-V_MoD<~jsKMRFkL#2=|vx^ zOSM@NG9n8UmK2xZ31hnlktEM4gUj$g^wE&fnddI%UIFl6~~)XEmeN zW1`ltA^Lo#;#2k{GdbGuOhq-SO~~I+){OTLH`*^*jrcD~T6sp~kN+A{|=vYWkDLzvHgV?cfL}7?NpQ+gFEfM;Ng(ZJU zENHe?EF2M9|F{LnT<0+s%tp3W#zLUhBB+NDUzHnDi_cWtP>w7}tL^2Kx5j8`4`KV6 zidDyl_*Wy>+{JZ@Dpl}IMMG;Oz%vz)-F~KG70*B9#Pdnw#KUmQa1U#J~Ao}RlnqFE~7>dgP+%>5Vd(p~+Or9sYm za=CNOV|U}r*5GT=;L9W$^aCtRv8F3vIE8(wN(B%9ipgJHykF0YkOtaIfJVfmE)~`+ z@S#~Mc%Z{+B}u8U=3GG1V_znEtO7_B)AWD8V9o?75TckYvy<-9_3#5{dR!!N0X`se zq`Cj8$T+i+{qE)?qiTVS_fnmnt+G;iMR@}@05)6CyCzA3crTUOrJJ}dN3Gxg+|C?| zIafrzPK@`3!)k*&Wj1ueo0;%8-b>XzjD=>(CUpR~xDHrIww z^5`}p`dmtnimi5>0nC7CKdf%sia5Z6-`pY5D`_`H`vJ6*wjYi@J0uOa@@|LC(SG~k z1k5tMvm`yw%^bB=-~j>Gydn1(qtouZ{r4#bx= zi5+(vq~tXGRgy`J;YmWTizZ+BvqO4&d5`Z5M9w>3r4aG>B1^=s`%uJ(d;i_TmnGuy z-2X%oHzyHs2w)-N@rR0to0-n>B%SNr>|v`y|DX z)i{6Jyn~u3A7g_Q2MN`6A*y`wyHB;Nlb>J|A``Ko(@l4d5siI~(6*Uqx1^f?*6-xz>AekB=wZta{4RYz=Sunf``<dq@+kpJ1lUREgg_VDe0qkBl%<3elMr1@a?)Ei zi8ci`i7kq{*4+P4$th=cGEs%k+L+BJXL5RccF=mRU2P+)EOu*<#o958aS5;7O%#K6 zM+l8>Xc3JbgjLu6%SKbV@2o^eivv#XX@j<{1BZINsk4MJ(bK64qO?CINnTM0oV;&{ z#6vcTu1-~uVf^%N#pKCc|8Nzi9ME-k`p39QGF6!ouLV$64?^<6x06=M4bVUWW(nav;Nbd1>jR`ri|3dT>|YCv;NV@ zN2Z57UiqfODgOv*xK5DOdE>>Ltok{98Wt{CHg5r@sNJfU&Z`^JuyFZ;WlL7>Hg-Yd zygEN2D`zpYlkoPM_^CA^;u}DGwwVsvVmeSJCl28y7EfIn!r^RCoj6rV&Oh~X8&nOF z+qj#02tABSp*F!kJ%lIfXGCoN>qjYAAgACGV|_95PLM0YS4A&`4<&sO4PlJ<$RuPoMKzlY$x-e zrE@6jHozsm$hwaEKYVsdR-Z;!r#L}YB3rVqJKaWBvW^2tvf;GO-2YXwX1@r1cJ1TT zs&l5hI(-uObV}g!ADd71M%ZKsL2R3dKBx2km##|%oe}5{6%=|}Bj^nC{>ROZ8A0b*Bgn6sID+H~w15a4K{fcm zif5SqhYrhI;DY_iBn9m3lm%f~(o)q#y+}~}ddBa*Ynq_fhc?l!I7!D627V>76{lwo zwUJfTv=>2=#Lt;cHmItaeBrmu=h6@|$8Lm=q?t3H3xO`S_30Tll0RuvWNRioi|ao> z)K%4#zNlx7w~=#ZClggnf7T%((8;!_sZFv2gheo%b-RtMGWzX7Mn~0DucsX9Mihhg zRS2yMc(#Z}bZ|R+I~y&>gI4DzI$D;5XHT#}bLjv-A?@Py38J)ri%UXg5Nk%VX`{s3>!IT4J3|CR+Zod*PQb}2$T{}94t4Z`Z|MciLvQT z3L47GIfpbv-g86sUmMEn)9LCAPLMZY!a6rp|0#J>O<3oK>c1TF@#7=4C#A|dH&p+r ziNvn}+F~M6RTf=Kc)3e(?)$cZa+%@CP?ddsbU^0R&2L(=Y^k5?luJm4B*Z@F4GfXl zoSxhgf#)^Y$T|7&=WdA+$jT4?Jns~Mvir0-Mfd5vo2>4W`vkC@GroV|7<^gXr}KFJ zKe|r?MkVP!^%71$YoP{=a{Egg=sYawDF-h?@g@~yI)7WCR->pM&wc=#3wQbBlFE?c~Z zM7+^Sl-&yO6z@}Cyu~6Ooy&_5!t;Xm0yWw<$@!SAyDGTQK9&p0MEsgW@kPH2IR2^M z=tKZ9>IEwRKo^ZkVpO^kQeSX^exO9CQx82mr4spp7E*7-e~`6ZdgluF!Ja zzG8TS<_oy=f54ZHBKS;4HF9#Z``C^Wn_Sr6Cr>|20+mERh>b5C>qFAUq`WVnc@aVk zbKycj(J*6D80Ny$06@d+k;E|Dfe;LH;opVT9!iPMU=`ER)(+98`e&*DB)HMzb039z zFo;v=ago&j5P*7+cpT8YEIlr&5Iy!tp~potM2~Sv^e6x^=yB2EBGfqQk?Vlfc-Lii zZR2jE>p}cY;B5urh6ut0$K~|VHoK?eV-Q;OzDV-lfN>m0V3AVqi`)9Np|=udBEfN4 zdSA@`7yUObh29rW0s#FtA&K5{&-mhH!fFEb&fW;U3mvx;IAv#da@=hEK}b>IVyXW? z5h?&WwF=MrbfJQxTLfrcwhG^g3KLSOa0&N+Q-w)MRFGkJ3H^@=9x!y>dZ;knRpThf z?Gqh#mo!Jf_jX*)54qXBlMK5{&i3Kyw)Fr^5w4e*@jr>-ssxz{TIcnl+Lrapz{W)e?gKaB6y2BZ&)QdPg6S2POb9>S^t4D zbtdH`I&at{I!{ZX^9JUB>O4J(&I7?7Y3WkNKOHe$DdB`f7Brt;z%uxfd7xx8HjVGEPX<#PWYV$gYOphXJ3 zFE{ybrg9#InMkE^S$bc-R`i~kLhs8t|55K*N%WTcqL;rYYRpo4=Y!P-4i#nUlrowC|DPqI ztUIp2{%zpRcXp^z-7}VFPRj)(q>hw)mmvs@u!T3fPU-5PbnH|&WHgZgHH>XVo zPcm56v=l9G)x#tf)Ht^e+Y0hg17z_HYY>kb48ez5nE2OPaPSQ7C!1Ni zJN+^&CGr8EK@Ev2!{?e_z7aU~IDBFpK7hb`S5pojh76cr!}^~dUd4m16n5MUzJ9_E zJi8mJ z_&<@|YMg3{hKieu-}+?bI&NUC3~r8ODDYy)5?`dfcA`joVhU-mU1Uk?K2H)i{&Uu4 z__CzE_AHThZ4zmZ0W77x8!$BRTE=Lv4;r1REWR>1J(#4_phD_H<|5s#U zn+P)1x-?RMCpR;xAg*FcJClmU?scaBp|n#chWnJdJGoZuLJvqzbfWHu4+VM6Hx5Kp z#?h{lzm>Tr_y@yn~wtFP=Ym-on8PmoyBXxTIlf(~?FcAcssOI&NV7 z$Jp15x**Q>H}{*n<9>4xXJzzq28AH1(dB#5<(wqCAW{E)7(*M@lDb~Ij2Pc>@9}Z2rM~Hg6MIcC`V}$TmBBL^! z+Ii%gZIDx8yQf9E%ATYt?zqX+|B9}ISivkAEVTSmkE3{@0iGgH~+maWAu7bM!HTC={T(zc6YJ(*<~?J_W{my>O& zSHR7-*wv7aC#*8(Em+3?)E9}B;{#o8VgB>SbS}sAj1qZoPj9Eh3eJ3=9IQs3q##DW zh3j9RirRVLQ|a!MM2t>!qK?7`LfqQXMps%*L7}fEu$`RV2}9QqQMrg%c zp(nRsZmkc|q+DGTh z3Wn2#sl(~EJdyN5hEtA*aJsmq>E$-F{=HC5FCMt17-qL|{mWr?VL~{$I|2!YzwLP2 zNQ(6(GIh8ekn`FG;e?dVAmfEdC~}6RGl`5{uxRU1G=KNpR?nCqq zHXR)8@||L*FddO6E?;KJY*a2oG}2}g6@Py)WVgwPinLNMX9k%4@Bg*Y)s6H!_}H1+ z$lF6dc4j!k*fNz8SqR)dGDLvEoss|?XAzLNynPJ_FfK1n5|@(vZ@*laT&&`9HsTU< zaXzF-mi%vj$%kHuD4q?d6j6EmZ-8V}UYrn>>eevKb4OnxrN$gc;ck&BX!=HGV##q? zlnWP^`I6i2ko_N0uPnB`)H`wFRrtQCtlW0Tb+$BWYerhW7E9mpCD3W@+Z=a6-E7yQKVI{@3l z+lPjI*0eV-x-0xIfOC>8jm4ZMidFWITi*E&&eR%TJ)wGcyx3hG_gYW@A{0o5*IoT= zljjt{aV^f_6qu+6eLovL_eG#{ADn7mb;mxqKmk2A3_2G=@Wx#ihR}KQp4_{+__@E+ zSIr#0Dn>y)Oi77Uj3B)K;ZKXXo72@pLg(%QKxgP&nmTmu9xE&^W$5H`tdzABI(M)1 z;TJ0Sl%aF?Wq@SpT$&g&}R5(}eMqP?q#V56!-HC%^TW3hv=V8L|0_#2! z5-;4BW6P=b|Mr_?yIrUm70FpI4Dv!qM;%?ks>-$c+wmubT10g zrH+%a?{sH^|#BjXK@yr1z;PoI|PX|^y3 zfb}oKTEB&2J^peItbf;R){tnu{~%Cdv|gDcS|w!eUnhcI$u3+k_O-H9d#5C5-3%C@ zGNW2dS3#1@fqXgH#RN~4MN<7fPR^odqQT*kgYw=|@Y91&S zC9Y1Qgw*^GaQ_RfeYMjBB{oBeOw$PKgGQK2O(btO5>b|NNk$TK`{M!Qe?>-h!F)1Z z(FOC#&;<&ah;$u1@&NDu1~Sl)*`V(-cz6*^4D(>N80P8}hIw$HWf+%v)meS$RCIW3 z!#udB80K1K7_O4_Mepx~4}^Mfp$K&?Z(u0z=D14o&JfObBAqEnM#QT{#A_2pl-PRk zDdGM|61SjgR@}|{_YuFtt7EHgxWk*t|MZQU0b~*oNjB2|jG{*pC6S@9ajI>sH-YKA z^xDq6MP^0O#zREh$CHSA3SeQZjb{FT+;8#bGP>zO?ANneyw>n_)zAP0F1y96a*bYz z6y$??wAuS8{(Vc$UP>m_$wNb-0pm+c#~9Bt9sd4K8!I`5r2}gQkX4;Lbc{_&g}G0u zJ8lq!G!k^Cwf*pcnI3w?M>iRxjbc#)Dyf>in}Q6o3C-R%$(y~8Mw-137lIkR|9A@T zKRns;e!Hh3qI2iY6Ie=E-hY_)znBQ0N#cF(Yk*o0pD$`X<9Ac{#eztAe>bO}W#96G zRyR}0LuQ6YGD3X1z>l?+PHi!xth-ZbS-+fdlfgw9QODo|F(0Y6(R0^hiuc7F6c#fd zbwta}O42lR+5cjADfLye(idxxQ)#Q;B3&gfX&QBT|IeqZnw37?7Swe|X>}i^|9$iv z59+QLueduo{n`ex3Tw3i4zK+Gy-#tRbIkuoeF{AHmHIKTzovDLoZa&Dc2KEoD4`N*G-AZ z6^M$l+5ez)Q;chP+nNJyTEdNagsN^5h^4mhfMYOP?STurWG#XE=tN+&lLG9iJ)j)}=fxy}ldn~HYJphbMHM)O7$}u3?fZD@VjmuEP}|^B zEJdGs5|9j?7ZXEAngmaMA*8f+WL;%KrxKxKO(?KV>Lq*&O@gP#g;-SE{vE6wWK9UE zIbcXrFsn)M^c5k(x7dVRiaYVl)1LsHo_RTmXJkk}(?(dltUNOb=9$&crQ4DX>1WD) zX!?agl)^91%myU=@^YeIq>Oy#NFk;DLdN@1CYq{$R=)(lV51A4P1e&8AXFJC>GPRq zLX_%YrN&`SZ5c;rZ)aH8!70<{v&?_~1mE@{n@V?QSY(V-Rt_Lh=h;aibgHv|f2rE~ zJN?5#;KX+IaJ5#4a*~Ru^K7X8Q%~SG4R>nGwK~JYl&FrYW&`<_X3pDcEs^Q>~$Sr07d(z_qN8edkq`&<*KFaTdo5`ePq zf6nxOU-j4hZB-Rt5>)Zly1!>XuKQ8nN7ntDBJ2L=82?n_)f7r>!umI8UVKj_EXoL1U5|o41?!&r%BfR*=9}D2Wuj~A`F)9d}>SSNfNT|-^BetJ{h$j z^2v1NjwzEz=p%(}t@}5b`CrTM0X*Ig@Gj$qm$mNS^t%}5^%RDA9{q3MFp81WuE*E` zpkjIV`M4P7O=Xy*>;C8Gh(>Sv>;CqxBG?(4`A&?k`=4JgBEFd@q9nKH9}@1#PD3h# zq=@3@KNap@`Ro2{)pST$_h*IH{h2@}0T~J}(Ep60R3{_7fc{5dEVb_6{O;H9gpKtA z_J66N@J*5^l6C(J-2Y7Xf8($F^V*6BmpW~&b-$Ga+@OGQ@~y31{AamIP{0HcS@-um z8vovkb^nX(e+1HPMttS8l{d8YciLJaS${%0u!a)$y8p$)ZAvQ4eM;S(wpMO}MuN`V zBBQIpj+)w;juT4g3S2z#!@y1#AGb$`#JBkTT`b_6qe|C_`(NAN)kvQK?lDsAVU@tCbY`yZd<~|U|BvU^B;_s#SAV^>NT5|^shs+BwNzyH@?UQYBou_HMD&rqC=77c; z(C{~&PH=K@=hnFaLf&V;0DEBpkfjU_h{53-} zx8&B0Ayi0ij(=npD^7CrumMKPooC67KmW=&d6V^Dz?gD_u|B4~yOU?htr!HDd^6%l($z|FG=cwxh%Ir8q7- zT5`AfYvq3HwuszdT&oy&cRE^fYsL^NB=>hAblSUJCoJN%+?_1B+y1q3zkOhcG3^b; zwTf|fr;{bOW(=Whx!=Avgig8JW3ACn%iYT04(%=_b?H+gDOVo&^#sXK*-1QK!a+b6s3AD4>n+J!Qy?Jf$9b^DH*w+ z61h91k{k2CFL%a`z;xc4ocj~LEVwOJeT;HbMV5g#H20yI{^!b?~`P zySzZ&u;~A@7D|{E>i?Gnz8qPQWS3sD(vRXxSMB{PLn>^B3T@1k-rs3wEi}U_z!;p} zNkwe={9YtC)P1j>&=Ah%1p=GFHo0qm(K0YRd%;@U~%c%rq_()`Zd=o%9BzmTZtdC!? zBFjC3=u>asK4CKb_y@u5@8pEU{>sfnU2#o}tvmHwu z*!wf)f2PR5Oj}_IrrHl@IC;s(FH^M{iB2=h%>R#_d~{BcjJB1$(dD0%8V~E!HhYzIK_zl zVtZ@EI=?~;=lZ*kc^_Zai2Z{3k0Z7+$%x$(un5I3%=}lW$_(0Y8)d_iz=l@Y-)j>NK3w@RTsz1#{uS$ACavNAG~PEu<)hBo zR?IQ>K%7Vh(aiq=Vn0`TsB^Y84lG2`?JG0?4^Pxhs*kVC{9m0U{HPv==f61rI}p3t z`3{bDVMhGGivSbwArSSaZq_Vzq^p0(KVybQjXsjQ-^0xwN0ZRves=!JCi zz7)#)3-wb&)T{B3@>2A+dO#K&L)bBsCs2$TS37OSgt4P)$4(ekGlSgHe6yG4MovZ- z-v17CnT%#>jd!L~JDlvG%b7}-=F*08ypXdnZC&`D0$A$neM8jwdZ6c8$zN{<44oVkC%bF_@ggVT z_RUr{w*vn9@5@FLkN_g{H^W5cSPGfHnPpLF_amuneblJ~g^C2`H{Ab2nUfvXD%0UN zvi}J%jJXYnoVDI`r*?b#soL?TXEF*J8p^9GiyK;wxeiYIY4O@o)p*>mdTI>_<;Chi zgp78?2X6beD+FRtXl8sJSY;e9Xz?B;xAsoF9LdGt5l7tp?H(aKzCo>qv{fS;UV-Q3Qek>GHvX%u@fgK$r?a~oQy`Fgt$?osyKG` zd5?=mm7)>xER8ms^`Eu`H5wfQjOy3|RB1F5->}-|F(Dq*=oD{nr*^ zYlBAWc}%5EJt&ZiXtVj&5GQIgG>+3hyvIZvzHfs5A)c*`ss93Pb|*||IEpskA^sEm zbE>zO9EA;S_G+~@Ojt^)*ekz~i?1GoQsq%DSv zcOyDya__ZzV_FgSUXyr5aT#6?*HDxWaT1nN)GlKjnH!p%!;X^q%|LH-C z>fGVGd_P0jIZi>=2qzb1Bre%B$baAjin*5wjl@VqZ1sI-pHLI#zb^QF5A3>AZ)YmR zez{iIDzlrAHKfh`4;6qJ1wwio?FLOcO-uQt#Zuot9+H|SiIcl+De>6%%>RCRb6G#m z^Tx}*LU?R^YgEi;(~pFiIpBsLCWMId?ZDr8izEx|?s$tbu@ffo1>KdEg|uo~lX#}1 zSdyLS^8@#P1^N^~jJv%i+4c#2nv_0i%?(1_iohSS|1BV{!s>IV46Esj=9ek+N3MSy zHEp1HLSryl@$}<@5WOPU=U7f;1tf@qW4azL^@TVHuA#Wa;lkqSQ(&=_X~YNC|Iyt4 z(HV~K>3-A0OOkgb^)Zgu{J&b?@FYJep?>m08nJ{L?X8pK0*%(CZ+=6alx0r+N%sH9 zT+9;cF>0M@96&gkQ-69oB*|h}

    2B)#Fvz;M_O9XB5 zhSjFewjVSGQue946Kmmot8YWg?(Yq#@A;6tp zmOHZuz(~K2B@!%)S#qPo$<&VuC$_n$a60ctguC5S?djEDBY*BJN6(?r_qb~$t|VNo z-4n4y-Q!wM!E~A-_a}B7H!NZUHy$)=Sb@ zB8_0Q^<13N-W3P1Yx5S&>A5&1y(CT{hIH+cdGiQ;uM25t^lZRi!~`qe1!*D?^<>7^ zkwgw0puJ%6teV=&dCL|qsH@3<1T9esjPdF@)dXGS0RZJiOG}50ZZ8MOlNA{%2&-3A zb$I3CWeck6tLvA|tFE0>e>gRQ-UDas(o8{VLS_+^cpthgXvAV_*DabyD@1jD&HULp2% zk!wS@dZ@6MMCo6+i>s@aEv}qZb9AVex+PF=Ap-ZGRb5qs-F4ZlnmIM~yjJyW zn)enAaKD*LW&@shmzSMnFZEcVGpFimmWKD1Zgm&!552Jn_6AaBZ-D{#i8j}|r5DZj52g{fEAhd!%Q+PvOmj@C_+aYWM;S*u!YIhR42}<`F|iGP zkcY7mu^#{D4yHc40p3l21*00Y2IK(ch2ZW76VN_|;*W0p6XfsC2YHgY(&r*K7>vx( zpY9;>hbRObCV?HoB;X2`6ZM~7p;SRWgl8R@+((j2JT`HMcdSegg|EUq@|5xS?daf7F|L&j&y6K&HDP--#R1*tzU)>kq>iNG++KElv zE?>VyPr)M!;ZzFu{;Qe8C2I}YaqB0JZJa!H?D$C@ZvB+gr%X$YJ$)>FmQP-h8aH7o zU{uvHe|K@~&EI6lt>3BAu=hp6I>xy5_*uH8N(t9%fZ7e10WofUj9Z@zeai#Hh;i#< z-1?T49=Oueo(kq^)YUJ z*1_ei)d4_QLRXJ*>py`0z#mM!kF=0?e;CcJV_5VivcHsFDU}zr~za4;}mH>kG%fN591;4$> z!EY}S{B}UU$N248P9QmvoCoQ_1m!t|fLO-?$$6;geVB{i4m9Dt1i!tElK^#hAAxV( zm%y}OWm{Y0 zdaWjPn=`4V26nxkrHS)U&t?E}B2e>)n6$clM33n!atw&drjF)QnK^L!QUK*<+s(5m zzyj3z5unJ?bJ1hKPz{LN)NK`HlQLTOqOke8mJ|j+cmEyfq+S$7Go|Ya4tw$&Z`A$e z=~oIWl6l{M;F{-ON@? zqxJd>TAV(`+QOXwv}o(Bq~3>vFzr7h%fLD0Cy?y(M9mwi7m(w#(hC=GH0J#0CHlgo z9>GBv17DC5E@3~q{fm<1GAaQTTp=Y~$D%=kROz>Tnyq=QEh{a3 zmk6^ZNn{>SG3Lp=qV=utuN;SN-X^_*<2mBd&)=5#2iT8Z`;Pd(%jwbY4@&y)QT#;Y ze^}xlN$TlIA)1(oRF8}PpGfM4f{KYpf8mMo-vQSja$uPJP~x7!v(cn}E|THk1z|XN zk+WbVy(E<|X~jfe8aqP$3H@`e4#-Fz zT}MLmDmW1;&dp|;KC~?Tx_*OXdz0IVVf#z*|4IhW5wm7iQFCfmSo+ssw$!g_JVAyM z(T(p)tC(eDBD(E&qM`RW9^LkV`2Qdyf=a1vT$2i5YX^BpEd3|_XDS}#TV`q!di8IT zod~j0IR76UasIz<{g1xc3+Mky<9bNy^bh?{(j6$ZFpzl#nKa%qG;7c>F!7nbicB_Y zeS5;l1C@aD6JY#6xyCXIj6@$J-$-(!Cs;;b$y*3M0QWcXmeEfV_2)zr0dy;gU^OS| zYib)%pk?P$ub5?8Zbi?%)~GMD9I2>vO6c4Y#GBO2_W?H7FSo!vy8nYK^Z5Q zVj25L0@#KHvlcADZ?7$5xFliwaMD!E7$HeuA10h18d}DIl4cYaGtDwaOPVoAgI`Bh zFRrPwjB%2NbvAjdWlWSb7{B?m=Pjs&v{NJn>cHD1;L0)%rt&6_vy4M{+!-_24}Bjd zc`Mit9nKVg71sPuF1MCZZOjI9gfWMuMyJ+DtBznlI&hw}Vm|xPTMNX$5H=c2%7tyI zrsj5f<-l0X)6!t_ipl7;B~s9loDu3fS~51UAKh_`_>aY}u29hY#?zbEtXp$b^NNmD zmT|n4wT#kELANyW05_Jyzj7Q~=Tp!DDJg-Cr77ruRpN)Sgk0!=lf}>0(iAkUMf|NC zkEWd>er$wDkEV5qe;xbLg44tg#*;kyJ-B6@A!C6}hbd^%rzHWdY4hmE;+C;N5`2bB zKpW4M1mF_sLka5YE9Wn?jPoVcXDwre45`mi`l)E=g`%^I*pC)|LGrU1F%>QRqWCZ4 zc(m{eNq;5#(ZVlF{8j8n3%??MaDXI<{-UXIb;>fnCd!2+flATr>qNP@x`1+-K4dBy zf1_mm2B$(>H;eyf_M>UHi2qj0xIr3mJH<>xBfcd<-N}A5VvD4_oBe3Sz2c{BH#H57 z*e3PUw!3m1W*$P(lm{f$cJ`ww-xdE3_CxCLiT@$?L*7Tk|0w$*@At+3IQt>*lj8pY z`U)(X5c6pX_@M+q&SxdyISGKEFG#?Pmho+A;!6?*nO~MRz9In-_otHfXA%H;e<1U7p%3+ zJidxC^SfXnPctAAWMq3p7$8^qqst+B+KCI?&#|CGAwp+*i`> z2g|6bwS}%7TJWdU*4*ZK`l}x~$MG&DjxcVU>G79Lk@TW1bAR&y(ie_lg=NhxP4c5K zG`U=wJk2skNs|xa0Nml2VFU@>>gi9H7ia8bAtvOAq!)Tq(tDkuY=H|+AlO-5qR*VS~#zY=rq*&%m z-ZV^D6PV$kM>DIX=V!y5IkIxGBsyFY)mY}CykeQSN}=Q!IrDk>F>56NBWI!H$MpuK zz{shSfO=TQya=pm>{vBsX7kEf*y7hTu1=UoLdNl<$0p6AExcS?)Ym-9Tq?xwP7?0{ zO&YPgOxEc+$4T0R?&8#!p4u{xHIKszXdW-k!QehY8r&!W==vsU-wF=EvEDpU0#-@@ zy1rQgPJ;E<(Yz+r-sMb47o%G(>{jC3Ga+3L%Uo@)DKwL2i+MKQCZ>~5TMARVjria# z^Au(aGhz6k>nC6kc8IQE`A`6cpO`)-nLY@>Xgfnx1Jj2BFrLKrG0FBp0LIe>DFe0- z1z zPw(azIVH~8=H*h>6%v5H`;uh+vIL+_S4)boNC2jDapsJ)%xlcAVNW$-3_(abPY;w- z&8=hVQf*9NPdn#y%e>CqL@BR#r$<_fgn6U0VvOPQq$~po^Cs*r9j&r{Smw?8&wb6! z<~OA}sPR^j@-_*88t;&RZ((B1sjjVF48wC)S>?Q=DjVv`DxtKy2@A@uQ)t(mWiv|xT(z)^D;9QPU$6@+f?Zscu!{>2c5(5+ zE-pOS#YG3ZxZq$H7aQ#2?t)$1R~Tm!I+(?7d7^s|ejJG(fRvx@^ayErYA zOGjaLadu@FM^<)mQe_wCK6Y{3V;849c5#AZ7sncQahhQlhZuHox?tDoXF4>nAEyC! zaR^`+wm-Wtn%RY|%q~o1c3~N_3qz4zr~k+-WWUpoWCXGwwjaAN`PhYF$1cn^c44it z3nPtPm|^U~$YK{}5xcO4*oE=JE({j9eQ5(=7mGK$SXSXqR6Qa={5e1{@dUnS)O$FwLdB@}4i%-#DHi-Q zPAu`lIWz=BsyHKWyRM=yAeYjWX5KbtRH@`V`CuPAoY}|9O>mKQ>Un*XX>QXCDb}NJ z)W?KNTp9MRrNlmK<2uLdNOLxMJ%#FX!+9?X$6gway_#a}J{~3E&Ee?nVeg}y+L!uC z_7C=QKO0W|a@hN!-N#{}{K)-gG!;mYalIFGe<+t}idWywJeMYQ8>t?oJUNBHl;)(fEnK>gIeo5GSRoL4J?+n{(q`AG^?!Vo9Yw5k_uX7SFE^>$$tFsX= zRtoXr*Vr+iV*W+*hs3{VKE?ct=2Hzj%x62z7jpA2Zc^nuy{_yY|044m5>3X$@*?%@ zM%&s+21a2v>^T7=DDprdh-oo^zYqbVa54r77{9Gc2%+LMU;toKR|gTHqDvFx zQ>jIhEDBAIR@ulCasEG~0ENoukO6ndm*)p8`E@Ux{|PV^Km;C3nCi$vwFzoQk7~0U z)Qk@IUv`?bE4r>4)QlcoW;duALsdtyq~#mLq?w;6@ERAZQGtj}xi4~q$BHga@2?2e*1lVfzG4ALHjD%y{(GeIi$GD7X z!E{a~cr!nWKyR@_pf?+YgAR8k6Y;qs@=B(U&t!Vl13l`}Y>TK~5g1O5zUmvH`jwvQ zR|ZtiReM~H*;mmkyRLgppxaOK>plr}ul4F)+eO{oSVlt}l6dF1J0r%Rw~o7$LE~00 zqBqrk&JY_MAjHJJz(!DDF~v^`7a{*VnWXlF+Z*|Q^t?Ach* z_6RL#XS-F5h^XQsXF=P5Zag<2qPysv3%*rZbk4H%2ZB zj1jLIkn*$hh7h^;_nu`EMNBHsM&&Y&_hUj6P)294hZMJ;N9CDGNp zr`+I@*btdJxFkM_sTrWBb;sC~?r>TbRYKnjfhC4BM_)3*o(0A1^SB8h#%lc|lWe1x zy}o)m+*fY#&nCf8+lTB0{R^RW)~_|tP27^2EiSKUXr_Q25^N zRA-=*DZ|?seO{?xT#YXFJt_A0Qmle?A;rGessHL1`$vDo{Smc%=(Xa*EUh3qTVr~3 zoyMVDr$N`HbC&r{F%Atj#hB$a#h69p>K~b%#^HGicd~CA>T0zhOW|{+@Y?jk=XM(P zUW1H9e&LHYV-7?r$2dyLTAE(gQJuyKep$!)WgQpNxw~wkuR`Hiq>RsJsq$i}k}qw8 zy16){8!YrrHyf8jH%8OW>E=sPI9&mU6#u18<66Hx*Z75Bvsv|{#2CNnjleykaYvR$ z+$HtglU~nVp+;Z{J;TKLe@4Xl|FA>v|0s0+Cp<%ir8@mXp;4a#t!nS)iD2AE8oEBL zp<>>SFfU1uo7Gup%#X7)<|opapQktGC!xl?<|_^J^&aE?aD62<(-V6YX47h679rjg zA>I}tjCY)AYP{KLeBk%od!6b-nf=Cl?mXFJi}82P1)=`r3;CzbYWFQzl4v9#=AVAd zKSN{HN?*cE)1CaLnQijp!o}_@YZw(~qB9(EbUmUhiZYXFQOYJ^&?B@4>5+cA@_Wqw zUY-5nEAo6r4R$FWu@EtLUWrLp#{ODs7IvD0gwES6Y(Q~^QY{MD!z*A9_{uzAS+3%C z_lw)Tp~T!LP#H964(Ho^uK_ASVV&kkM}pzVa$q0}Vn;{h813g6O{I>DNHfk)Gj6jQ z@9UcP3uKzOtl>d*YKb|y(>yp2yN)>#9rZ6qPods*o~>j|?g7#j8R6(6Gs|LWF5n?* zF6Ku1tGc-$I8V$a-as>#V4j#WLi0oq`L=l!G}UmQ$vRz<3p_sCO?T+3(s1-<)f`+zvdMU= z$hbaT##2Kwp6SbYCdqiV$Vif%oqY{1&Z3udq|S{}r?*;|=LA;^^8#!t9c8DT`S#qJL|twx^W^BTw7vp;V>~Pr5ttYR&y(dBE_A) z9QgW?(H$LhdOGOLq64vj^XTVW=_b40Q_g|9m@Kv}2A{{!D+cu7S}fQ>>Liwz8H%<1 zSS!zs=0^b4h2S~Hrq7@hX8b5XoYmH((jj#1%*K*S+*3)>qAKT6?qgYSlcr5DHdn#*F`9|by%eln4+HC5wu+6*jHGq#wsoQ^juy)w*7)*8hg;l|GK zW9L9?(k&X`4(=U*cP(ka6y&FVe?&^NhqJY}t@(jYZZe_#z{u?GomD6$9~b1<Qgji=!$E?Eik5fz96!4D9@VU|`$#0|!M_gm({IgThNGyHlz8qB;-bpbu7E z;xVRkV%`?>rQ3KFE;WCwM?0}Z$4toW`9tqEUy^}UidUF_=0pcF|I{t|=jJccGaq=j z`BOLZt5G$-?$-Q9WX-QjW|r>%?>~q1@07+*{o(NOeEz4Np^vQeJAMAoqlIU|_3AY} zQu2HM(m;3zPRX^P}$ ze&`eLLtP)yLAyf84AKPE2fylF%?KFqe4KUgJ&Uy6>3#(C)KgPNz2lY<3sehf-Afsn(rh=-SGj9QqF7O*N zo{BA^)4nyev#O6?wi86fq_qvvTI&(JGg`P)-8(=eT(qQeUd`;9>ct_gX0B(DFq@b} z>ey4I1_>ei)-I@BwxqUZ(UNc%+bS`0!Gd|!m9-><`^hqQXT(Pg2l4sdv!V-of@%(@PxwAjl3+Mk7MEK7c82zEkJZ9j; zuTkk&m3US}4v?70NeC@xKO!fDm(Ego^qoEwE#LTiq5Iu+a}h8Sy=34+{FHJ|BR2;( zB@%rC`89p)3=Wx5cLEv_FdfUp7J;|-A2Lt7Na8}NbY~doR2OsKvYGJL% zR#}D+7Iz2S=jng{Q--e*rPDB?mievBF_btVjl({#N<#y#-*?&CcPV9rv`bVZBPW$> zUKeC>-GmVOxLx@%5_2|sWFXBJ5?GM`bgHA zn3yvr=ETL`m=&qE0C%os%sfD>t!rGFs)8p44Efd;eCtS^-jPtl1=FW; zT?c@mn$IBA>VrmP1HXD7o9Vq4)@VJk8EsfTGfaVeQdVD6^(P4ZTLs0 zj)^&AV$Sfon{AxL#GF~V6Vm|}b5Fta)G;w9gHKkU0#X}+D4gBax@KWx8=gl2loI>` z5g>+G=>5IT0XDSMh3cZd>htPiS<0Oi)LWm^a7Ue7S+>LIX^^vLasID}IRBsV;-+LT zod3s&yk2XthhD<$4+^-KMrwkkE@uo!bpW?;c)p0V;@?_7l3j>k~u! z`FKqG8O2lx7MN3;n^Ns#R<}0d*f3^JswLG16x$VmBwy9KVtUQeW4g()mARtqObDo~ zfU=4Idx*5D9#RhjX;VEyvv{hd9utcjz?+3IaCsK9aXWhFja#v6pHNSdT?S;9>Sjg&!Np(3@In7ef$-)6RO=IPFi{LasWubtd37)gDminmf0} z=d5fCJ6&tJw<&43)RrEm_aF+rJ$d^8eO;N*nC%_l_uNki@Nj)MI{JPjSOHxgxz{xb zeE`W)vPLLXXT{c@f?V9h1nk%npcl^njcLzE#NG)JTqZQg zNk(L>s9Bnj(5t!KcwW$5vMJr7TdD19xd5#3ZQ^f-G1c6@q^0?kbtyy6i4EX)J=(wI-?OlRQKwdyOdaYph6~6>BYhtq_o3$B8hlu9uN_1N+gG z?y_lqL*GQbx0&;UHT0X}zlF|!=*-)Mx%75CLc&HkH?_Wsk9_zMNHbZ?vg6iRU{Sr3 zdiJiQKAKBH)_X*q_a^m;1r;Y>y7I*ne~9NQXc~I`eu?`wp1LPBnA>a=PA5|Dd}!(K z3Z3Z=&Vs@4JwAErhuDvi@Q5`0QTAgPd|$NnIQ!9?Pl_K$Ug2DQTJnR%4Sp!@S!w8V zIhoA#3wXA~=P{o13p!TOC8Vyi^pEsQee{d^$66gXqAqi(2#`;u-|=QMq+}AOr^b!2 zeuV}HXeWdU_Gi-YpR*r5_Np`lL=lLG-ro>E*dC0PQ1vgR2XVecH2lAo=-;F>=YSoR z2jOoyAr$^Qspoz6L*XBYLV1@U_S8R#|IZu`h5uF3|Bd`ahx(yZ^bht!;r|l9ubo2$|UGWm7bL} zKyr<-2{}N{H-N{QVJERN@j&~pq^<#9YALpIW48?I8e@k6rA0>|(!T7yBH$*f!Y3cEK*T33jm`u#4rMT`cqL zVtHp5%R0MQ&e_E>POgDv6>eXe0PJG$vWsB_mw^29@iTeMYWV24$n=6;e))YWDZdDY zwU+rSdVQ53f9i|Mljc@M00BN9#3zMdgAkH@sr3Hj1s2OwOwFK)ks%N}$BA_qF6NTg z_6aZuf-OLvLAV70Di6Xb2#|H0@5DMaT^aTgNXiVmpC`xz@H?P11e^D8*!zrl^?M@q zJK@YmQMp~T-8gvZy~d$AsUeP*{sPimalfsKejiUia^Dk3>h}mI#8`0_i1T@NjJ+BC z$=H_|B8Pz*JF2BQa z5gz;wi-@g(I37?)ibKKgz)2lF0bfPfG(IhP!0%9M??(I%k_sA12s;J*XNk;=i^@ID z^c4HD)I|#P6#Ju^bETku`1)D?>S*rsCHf+W{RPvshZ}YQ9GgflE-BStbmbNj3mYhL z7ukE+W%l0oJ}3n+NW!YYNgcWG=0L;(_Ybjle{Jt!?}_vO%mNHu`FQyM7w`G$+j`;r zPg)Tfpp^n4y?}q4Xa%Gf1^@?0m;nyy1x0phBxceJLLNeFQDQIK=7P3$8Uva10{9xn z5I1O#nDpYDPIbOk0&$_$iJ7o0DuCmzZpia6|FeOKtN&%v3)=4SUVPU|=ou|VA_O@6 zdIae2$3{yLuDTm|>H@9uNuZ^0o8a|M05(|d6fH#}kB}x+YXJK$H!X$EV+&BJzcOAC zp@b5SC-!TGx^=PRrkBIkBXD;rSQqa+EkyqoLG|iE_zq0k z-6KRR^|C;4ro!NVGAhmj9WA}D@g7G8o|6i~%@tguI2>SqL9&7he9`=&n z$E_lYH=_b3IGNo5*VFq4I40^Cvw8pga<@kvkRc@M)T5Z}Vt=>XDDH~kY>_E71m(_f zMGuHM!_Dk7NO&e3P7U;3sRNODPDJK8;7Wph6lmidx40;Ri7`$a_eF7Y0d1Y$q`5$y zmnO}1iV09i_);x*q&gDCERE3o(oEDWz0`%hI?nHpr7mig#s=%Eb*Wo(nKupy$EN|m zf1%USXr~GLgn+pLS?;z)#^(ytL!R>er7as`I)YEeIR_lUxTCVoQTnO4rbJTq0x4ZTU@m#kY8MIw4EO@+Rpbz z+xc!WH4()GSy8Txkm^cz1p2sb`br0mO<(N~tSjA0=6jU{l@}OTR-U(Z=Nec?N0#%D zLsxO1w;H+&s5f?a>Bg%w9%iWix<53&-l;Ko-4SJAMF;U?Yjtad=`&ekw0y5Eo)7DS9ol@*+4pC_{nfg}Ap7B63L)vFNX`jg= zt)1ASUv@j}`A+>)LO#Zo#Ph*|alm!Kki%{Ig|z8)r%m3l_yy84Sgs6nBog~ABshirgJM9O&l8g_$!MkkT~(LV=8XiSvI|#QA^Wx;IKf=YK*_>L&hw<=!EXPl(}w zY7&*KFI%AV{lGxa`+% zN1HUpxkXPfh=nW}#Xo81^FN*X>C?^ge;!>##`ypAt7DgkAjbcvu|)6J z^$8O+#Q6UNlNsaxw=rEuR#@kZpLjuO9*}K}|4)lF-tFFb5b+rQpH!AZXNdYA2mk%g z!t+A;G5&vtEP6SHVFWu;jQ`Iw_iQ~4ZX^zO)dK!CCDiI}5rq=8i$o$Z1(1pfoT>AvQ*XEAEm;%VxSvHae|tw`TL=9nn!(L2RjLXoihl2ZV)N?N-jC9O!~sNC4-%I_ zAp!P-pWBY8SxRuNgVhkq2<)noj#cv~qJTVW+ATFq3f@Dj{j&d1z7pX2 zr#3Pup4v;RVZi6eioK6fl@+RxU;=U@_XA&bZ?#XMDpUJvH8s0Lxl4)h|2r@UeAM|E z|9|HpQe8kfjKqo<{~vI$^ME9heIf$0A)$Z`)(6XwpaE$Gm9>|3Ak6XNDBPS5K&Qf|BG7UqXAzJ<`+ho4Sqmq)sP9=JkY< z1R$8P*vLe(Qi)>!IXbzY%5MQyayrvyFvvs#r!Pey6QyylZr2?YSR zy;ipU@8JK>jyV56*ZHe=d*S>aw3?QQd`tkMaLQJY4?CV0w&gdurpDzYTz^#(S}6p`Bi9W`&n zQs5V=x706V{C{P_9M|gj+_p)&KBwn4t}>EaD#rhx)iWS>Q|A$obUuddYFKMpHRkTB zC^NFVy`>X6S*TidpQO5xz&ytR?9>wA*nUX`NZLNl?X|7UW@2euaB+*U%eS^!x`c@y zbZLzLA7H`BazyVEiN42tHZlHxY$G*B+ek6~e|EAe=qBRGoo)68AXU?#do3N~|F3Nx zBbN*zbo}X-{@C&V=R}c@&7F_ zRmS-LtH&IfYU*fhn?4fHK!@PT3?6l^PA$YWW!0)w)5%1g(ROO%YAeS77hHSXDByJ| zL&y04G5$Yb#Q;pF{sbI1!ti&1_WP3;nEeihe~ka{fwQR?|3Ak6kMaKj^&5fze|W_C z|GkHPzp59`|1tjmR&~KP^##V;1Xo9Mr@Bl)MO8DrSGsVY>dF8lse^5(QVyOay`6$r z140BNIq_g){C^lqc-vfU@?bgxxVUdvDdN8n*&|8C-s_D9|U&Fz)zI$E1sn%YupQow&0!%gZE zEjt5QwtrniUDblxy86YHHMRA1Bvwn~8Ui^?ES$TG$g=%V{=?)r)pPGd%(=b|z)UD9Q(pa$RsB^_xAnSD=-Xz0`l!?76*x7fX}BH-;0` zBjVIm9bP>@)Qdw@;>gOyRfktD=Jm+=X9L^_mJI5M-jJuzm8Yt*zUR`=2<@pf8Ov&4 zURKiqyZbHdopD7wSGPq)o~#RNhWwGB#n!@J8HU8W^YKbg+UK!Z8QjpI{azWi9~*FS z3W_Ooh1{jFOp~qKDkf{FNxjK)Ha1|$po$Hc*nn}#dtw8on@5eH!eRp^gC`)Up4fod znIr7Z8Zd^b=0-gKKYy>`%X{Jc&u8Pmaui6J=_4|eT=Yjp<>iut5^*ajCE1HcuGC1xh70_`>uQq`P#zy$=lm z9GW>U=-*l(JZI=n@9pT3I*?{+tt5eu@^hE&$n2A-vRSbFwc@<*_*l$5l%z;R_md_n z)vfYbL=iMYOrwaB;3MNKASUO{B4$RpEL1X2oP5WZd-Ro|(jUM_Lr4|@6~#rAPScsD7H8w{Ii@Zrq?M&lnc^n(dDU^tWNz{#rP4_KX{uM3 zQbx3BiCU=pqII=E?WX@STet}+=|jay3w@>)Dur5ZzKUwn=5MwdSk}Y|;yA$ z?)n{>+N8K(xh$7iA4WCTS?LcTYu_(?hlZ7xuBtt|p zDsqaX$0EBCttVZHIQnD=#JREv6ey9!NjkILbwr#j>yau$brI8&d4+`R$U-y3p z5*=igNoiw*OJ@vo>5KuF&X~rg#+@k*x1Cyz&qPHB?ZFr#O+is$f=P`LMm!&MIiXBo z$R5(%(%f+<1Fi#!J)am|@>Bv3d8%4h0GxBc+SYb(GqkjKw5=m75Ijh8%G2tAoLns) z&CK+qDg;<|*8kq;jhj*rGb!&SHu&%p}h=8+$+%fx2N6daRN{syomGvUFW^Kp%>2o&T_+0)4|1}+NF>3iGU7P z(}|_wkjTZR5$K4y@+TD5m;uc0up)IjJ@W?HidGY{Q?jI&7IW(`K^Mm45o#_Gd&Jy2 zE$vA)NvkQDTMG|Ipi?zBrP?#lQD?62NYU<=P{-m042;S7sg6~x*j?sE?Jn8CQnQsV zg`t~{SIY|3aq0wOix`IuFk3-^Et+OsV4N>kO++~X$&A!^&=6%QK~PDB!@@($h5#oD zW5p_Ap7D6zw3?WG)=61jDFNyhOP!=nrUhVi7f9G?f+SRnR%4>OC~IvAwN|=j!Sa)U zL1w8oHJO-T0CsPml!gNwbL%v(C6u;k%P8j7$u;!S#wsSwCF)XIT)#+jZi1yQmpOEW zR!NYQ@d`}yC+AFznmt5w^de)K(*{}Bd`8MEDxtn2A{^Dcq60I! zU_^~@G+v{=R;a$JzMfReN#uzZFjaWfxSqU|ETDNXZ_PL0T{+HDn}tZ}X03*)GCp&r6n20KqZcXU%@ihRm&`2A>|5*3cP>%3J5=9sH0%#(8L=g02^)lHoui%2? z-^M7lZi@}jaZsR=VCZz(9(357e$?> zenq?8+aSlxwI!ujlu*CPq#t@${kBlOquvv9vu4ejb^LBKF%NcIuJ_gNsjLq|%q;4U z6ncvK6Ldv4Q6D?U&tKGENs7NQMa)nP!i8mTYF(s(}@!`{=%fI;jhj>T#}S zyruiAEjXv?0T`_?8u+cux}*ky(Cz{2LxLUS12%>(!NTSxcfyl;w}J}Qc;?Vo7J+qJ zh>C_tyLL~iHwr3-{_Uc(M(l_C)ZH;0YcUQ}E9SMft~~;#OklToy9c8&>+&8@QQ82` znx%`gG52}q9+PdA2yUxs5?FdKU4~sk@6FnpXz6`LTl*#Hp%fH2$W8{c^ov^ z(g#Y9jRIRbx*+h2k)$3C?(jgj1>&TSCgd_4HHcPs3zz8RO6u_i6+_y zoBpfS7gk-#;!LshR1tbwQui;Yn0oLL7wt9=r-YJEv5C5XM&XRvHAh#^qG5x3G&F0f zr7I*;Wl|5}plOz#B`vK^>U}9_JZ8n=(o(RwQcGtqSX^B*rxw3hzyO^aZfPLS8I3tx zI$v6vnJcLm6jV(9^Nfejza6@S6qty0B5Qq8A5c&+=c-vF=Ktdjj>GI}khrBueIN&c zo9j4f$MH!$m4nc)jnWPra%j?^SLV!HFu!^+eg}e<;D+SjaG&|HJJS|0dJal2)ybFC zs|qSs?)JS)jvevmf{JrS>^Qf$-MCsD5yC1`dN>BxppNBgmHbk2(&(_XK@h5)|sM9jGV(c2qR~M=<+jo zg`Qi@FZL((xj5N|w8-H*qrZ)ET#{dz)OniwY(d4Xg@=Fhfq%{~sQB@Z_IPL9bm?@A z)Qd&XFJM?Ynj<**l__lN$L3buF47|FOZ68?eU~M59qD5NX6)!Jn_dPm3d1jU1S8rV1x^fKPSO$c%^d@~h>E{NmMr7zGORj{*4F$TR%X`lB{Hsr7R9SlHSQ^6SWKEk?-I5gvJiTia@Z{Q#(?Ol=iw@H052SHcwi_E{rbECjI6*R2I z=iIXXkZEy^PwGDvR6O>Nfye&t5j=>8H0XoBi8Oyt>MIK>-f6q7@{8|dY#tjw)jlK~VP~($^RZ#Kt^S57f zY~^)S)1l zF~S&$%N?T};^KZdAZ}B)Rgg`}=&D6wql|+{VWaU{26l2b>kMpY;{wxz0(W8$H?Z)M zg&0VlG$vwdT)*3GH=VeJ2g^ju5KGm7rTSPw#UC%||G8h!=Y?m|B+EEdhTCCDEK)Sw zCR@f#8E#cc{VWGfv5eU=+~yQi{B@vp->5}b7gQF1u=J0=xfts=k<83oxBBF$YthrP z?UMiF*P>WnY9r47vzsq2>W%ZiH)JE8A7viVGDCKisN9%u)ZzeVEa35h)pU`JqQy!5 zVp6|cP&v5t_%E+`O>;C>)}tl5A*o-dpvhPQkCj1oTvETyL0I`tkU`d1P+4%PKjuYubD zRE_byPW-3RZj9uo^KNWl1w=H)`llrOne0c~&XVM3vmcH5jQG!?>?r*_$qs9pNUSa6 zvwZ(#T#(d%qa0Jv)(b`Ki<0`Ig36O_IrGkg7P4tD1&#iq#9fA)kfebpR(x%Wt7gw0 zw2UwD(=7vrD~f{_uI7hB2A(a!kM>_9{;#ng4Z2nuz`M;9NO-*n48soT(f)6U|0Zex zyf;e&*bvUtT6*|iuD3*_G`(78DdQDBw2*`~;8h@e%=g)Fz zuWM>+UA>xgfM$*uxzsZLYW$7%)W6F^8HulHX-~C~`G0CE(BLojpuyQ%8@5I8aJPr8 z`-kyQY8`Gx`?R^Q!D%%-N3~vqrduXn3GJf>n1-bWDI&o$^CI8eOn=)ed@GkQ3ovrm zG`5{w*6qz>|#4&7w2(yarR~xXKr?}$g_*Zon4%y*~RghU7VTO#YvW3 zoLbq%QI%bsPua!!kzJe>$)y7#yEqlHi!&R$IG?eLLm9g`iLvYGi41D?!-{4XrZc-R zk=TVT#4ZdVc476f3)_ZWSTpRxieVSl3%jsd*oAq*F3b{kVUDm1BZFNS6zsxEU>DW_ zyRZt_g*Ct~tN?Z${n9qher&qzVl!pe(JyVD?8m0bE_O+FvG1{q4UJuFWb9%CV;6f8 zyV!`}_N4`bT}*a%F%98XDNXOO<6rtcVG&#mSe0LXpGw|^FZ3&rzKBlaYclH*iiGCj zB1Qvwy>abLLxD)Fn9_bV%CF_I3Ma>K+o=x6_D~4VYPJB!_@iOw-kkTo7E%|++ z&j_bJ%ZYUoUlR6iC$F9K%6f?W(1+8A=gI47TE7&|{AM`uZ^DV+Bd_144?~F!9ZGEE zJFz0AQ5;S@IGlJ`-gaXj5IP!V<@Xq6X5KbqfBHg+%>42a@)NQ6|=5-ne_EDy} z%@`A|Alw~BMYw`l==E-DHG>#ah9`VnvW^Fd@K5C z83Z#ot3>%drrEH=Omv#KX1kNu1cMBobU72(E5gxIi7%CRs{J&0xNy%?j=?$BRFHRK zZ~PUh9UE=p1)W%4q@LYq+vUZ_@L;n0l^0(}-%Pb2;{5;o;tO{4!ucOYLV59>Tw$I` z`Y*n}_<`c@n5JRa1sKky>bs@t!A|x4VXD0Nc@fMizTfU^7uuHHPkRYoDn^3WIYD{x zFC}6B^5PHWThi_Wra`BykNRUZMuY7WHrl&!%k5J(+JiuRXi91uU;FXbJZpafqr7BJTx$qp*fV@mY65|u%VvuhhT3TFD`FT$8|~q!yGlyi z4PWO3#WMNMjjWr$Q9#A$p{tBv-h&g?7i)M95IjtVkETP2MQuFj`GCl zOO@MY8|`wI!XCHL9)+5Zm9nVAP6!IoRH9QQdS!`jc7;HJCza~et`O@a<3Oad_qPwQ zN7y4t2uT7V^0l2LT_E68PY8?a=5D!kqdh_Np*C%_C!zjNNr^-7_3WT1=alI4O7!_9 z`htKcpDWcDx}sbm83!YsJ;okukF&>_`@Z=GoDxxKd=myf>U&ukl` z(5NXj=5`tj_at4^OWyrxsx7wb?0S0%LYHu;n^eTfcd{PoDW#O9AjK%mII{g{Z`!oc zUXCJ;k%AAvS8${Hidi0*adsnb9wJp!sgZJ}I!Q9`PqXeA`&j!p`*@UdvMUuQC8-ej zc@D&vFG!vd+HSCyVm29)if3OQ$qxS|uI!k6hKyV+E%^mb@=JK~!8SweAGo-pzQgQt z$=YfB#e(HVd$94L8}w0M$1cFLERPK{54M1C{7u2pn(UVYv13llgMu9M$|Z)KmS1iU z=u}4$btiMdscB+nXtg||xE3H?P;tvqTq>_TZ>z(u*rC-9b!w5?KsA6TOPv9`V_8gC z3sw1jF$A`xfshE^C_FTb(Drbr^M*j%Pr9A=oZnSXx?Pn?K-&Jw(_^C zpX^Yrevkdai)&&MBb@hg%vfN3rNM zDt-;ZVi={rGt||puv6hF_hvP{9EaF7Zo#^GDEeK)6W68Zt&*29v+c-IOM`M;m>zH+$}CI zqPQ_mBlm^SBQXey@?;RWO&V#sO&aMI(?6n^X^vC}p_qdsq&k?)$neUf4|WS25>a5V znKL5_oQY=c7i#8Ax43;HiVHS#Hkw(S*38*nGiSTS91u~=e5aX}Xyy_+IHZlRC2lhV zy}2Y2kBpey2Zow{j9c8)h~k1xUxucaIxB_S!^^y;FLR5jh$v=% zr|G++>1+8^fVS%nZ>`d6-Q2SxatEjD84+zd!&N4(v0TB$$lf>Js#vtIva(i$A*YYiF z&U#Cy-iqaj5+U+--n2RY?VE8i&*8F@LnIW_X}UKM^xj#<$}tImTf&EKwM~4q)O$GNaz1{7s#>f&*=c`$z zRzo`vCClxET_{}?Qud>wY`%VSl|2?Y9;FNed~N?Qr0pO0+WvvRqw0r5B7@$Ri_mZm zCA4@=@OQ!BQt^Dmw0a&(1x^8h&ER=gBd13c7o0*bL8C(*jk+`PC2tD7v+z?SClt;)vpc&3q5d9G12szUMXb zJ-3)EBZ>(&^Fz0pfhK(zvG9NB7U&yvk-cf?%w`%I&8!JF)4)8XkJg(b%i-QpgMC@$Ef zVVMKe80Ix;m|M)lUNOP7V~}Gx7y5(T&G49)!S9)%4i0v8;E!m-Gi9EL$Q;x`MT8D2 zpo61AI;e1Ec{ZZBpbln12m6FauQAKh!7R6!7bA+9?(FB-HjKK6{TzXGLlib}j5@c- zm%So`Y8~xZll#CpbHu{h*We2&8x3|ix186#a(rzE6}B`&VM}2coDfpjQn$FbBZ>&6SG+y`a!U{4j#^??V8fV)!E_Oy7 z%(sXV%*irR%yO&*m*j0~`iRwDAx0D#q zW~w4bIrJ2rjw1#hGvE%9jw8kktkRdeRn9Nma(?QU^V6_O@d(hZ;dNB_mZ#FULMrXQ z#dyap>}@~G+sN{sm*u@+7I)6$9n)@2Vf+@uoq*nMr^m}itm!HK2kyQS*h4;u*jGNt zw4jX-Tyry+*WJQ{VeV#6uZ*}$#0Pd%ZqEV;rR+n>k_Yd8c6vUS5lGK=G6Lx#8d(*g zGW$gZ(o>oA?4tTovrwIft4DgP#)eA=%e*aSzuU+fR;6Zt;P3kH+|ZS5{?NP4e)1kb zDc;B{$;|B7v886Ixm$YX1MfDA-OK}{Y98#?JS4K_!IGJ!`=9yekp91sw%juNE9dk1 z-#y=V^ZXz4{{e=+Tl`)zeNIfD6VvDPkg1cNS-HfPF?~);~eMe_7M zii!FESd&7*R;yTdG5=qXUGq|ml9>MwN}x~Fl;JUxAM^hyJUpi*b|)89qk<}mB z|AXS3PZ`_@cEKK!e!eQJ+W!py-=t|1mror#VPfy_|Ls(1G5=r8|Cc&F=KoXwv;2Q` z5$FHChX3oTUO4~9{C}-&>j8H>OH~8(OU>?r2*ZQYb7&D0>Tn;y8pKUafZOX5!8)CE zDjVpifJmpOj`{!kdekW~{~sZ-Wg90k|6iC>nW2nDS!vkc({oS3^wbu3b51AT8~}Y9 zD}lE=HX9INmfC87?$+GWkyM`s9%Tz)G3#3wwl%L|e8GBhtPWjPTTCO92uPc#7(dq&ks8reMr>7vkGe zz!RiG?gsi|TZ(wcn}H;L1_OsPpg3T4Cv+P&n+Ey4YAfyD_X(ke1GAn~ojKswa{}AP z{C}XF8Iw*8)nlaArQIDWnM>Kugg8*tJw#58`Tsl)u$oM5$NYb(_NF!=+Rbe&L~yCZ z{C`>Chs8XK&{)j>=kb=~`G0-H`G3RrZ~JU-od44o`7AxvC68ZE^Tqss^b)7b&=K?h zMI70J0NG)r@#!)DpN{$eV*bCCMA0xl=Kl-cW=CJWSTX-!DpN}l7Ff*x2UePx|1ak? zNzDHj^Z&*Ce=+|bjE|WAZ*61Ks?gTj<&k;ku0?VFUlMWtU-RJ|uk^yu20j|Ix#KyyR`r^Lln;{y*l|(=q>F%>M@hj(cPNKY+yf>~uQj z|BLzmXkHjG|6jK`0e6P~@5qSr|Jf&eVO1}j|6~3?a72NZFy{Y@`Tt`6znK3o=KoW6 z%>QS^{D114PIbOk0RSQqdI#xR-5>zeM7Y#F?C+TWPlyF${y!C9lZpBNfCm)w|KV!WBLoOBJjVQgLGrSg z|L;G}|94cx`F~TNXa5p9{}VU=JImY~d zdBWY~^3cTmfA_@ve+FYi`u0@J|Hn*a#D&F7WikJs5%d4ym5K)S8WCv4{C_e3pDYY{ zb`IXYnE%fwymamWjhO#W)iFot2X@T=M~B){^Zz6N-$}D>c$v=sX&?Wlz>Uau{&(YJ zH`YL`fmj2v24W4w8i+LzYarG@tbtequ?Aud#2Wa-YGCEJcKdC7{{O^wPb^rhfmj2v z24W4w8i+LzYarG@tbtequ?Aud#2V-o4aDdFUQuIQNUVWa1F;5T4a6FVH4tkc)w6o$`U)H6FPX?3@x;cAS`#lk(^JvT6T4Ok&Xy zKE~z0POoRe|N864T=NBgLZXc`Xuvn;-!HvxUsU_^FFz(-{M_F{;QDX;W1!F75G61OHW&!J#G)^=HPJ`9o{*u^{*cMyMT}AlhYc0fJeuM zKXD-#(h*%VMbf=1Av*RYU*|GN5qN!|EY8ufXpb&hfbLVujl=zS);PWDH zv^HAchl(Qbg)SWPiNKE(O*_6Q8mEns_<&ZySGshKHdf%3qBZb~qFfZ(IEfEv9lX<3 z#(XOAAUsbdRzR}B^`GvqoQ@R44?QOU}lK-Ps z+6SK{y5;do{*Tt^0Q|B{Dy^;L|7e{K!54|in3w#&rYHulOKY6AlK&&n+2{z(3xh4d2mHpwlISMvXCLN9}_ z%Oc@?O8(zabQQc#iY4=s|2G191$)`WpS!nGf|8FaL1AJk$ z=OzE|D0;KwtAzXGB>(RObOU^443F1I{@)4d82qBLoKGkDe^=34;GJ>&eL2bhy8+z< z-&QI6!Abt#Q=~iIWb8L5`F}5<1iY^@{|`ed!7uBC*Yi~Jf2`S z3B3zGtux+FQ_25vMDKxjHs}31mHZ!1={ESRF8Mr=O8$>mX+B4TmL1IlmL@>FM)Nyp zP}en|Xi{kg#OpMF9}Rf*WcGu4P{h|+4+Coc&L#djiX&c9JQd1{!ZKowjB{=b>xN_{&kCq~i z&`KP*AoWHMNIfDX5J$ubA%TS8g2bibgg{)7Ktdo;`Tt$7?WQqdOeG$gjq+!A#`DcT z-_ES(`)4+|0>DMTd^A~Bvuv$#@@v!R{BHT!l3e#PN;yUPHGmH1LoTYEuK)Cy*)CVy zdHxwYy#~ulIQT97@%ZgZ#CByV&8pLws*|gd6$6R^#eiZ!F`yVw3@8Q^1BwB~fMP%~ z@c&_;bo0AGl&C$9zE6YY^6~px=;Uw7@Mt8X-M9{;5((J$tX!-r^j#QreQBq<&z5t! zW;c1VgatfX&La&^X@K~ej1whrW84EDjQLO^FcU}wZ~%|2`A3h8`TeO8zkhvf!c{j83T`~3vjp;sV1WoY$BF`yVw3@8Q^1BwB~fMP%~pcqgLC+i|8T%+FXy?}0o%DbCqwb38_f*d6=zm@OjU{|b{KwBmf{T%*`2`v6K4idaiGL~` zPXrR-cp@B>0?^?T@IohmrJ00G(Lo)}R^7DaAW$sk)2pSTS&-Oj(5|6HU@vwPz_eK? z#&TD~DKlG4ucys?Ad@L?UywlT(9kL{jKS>?LmQjLYZ73-ZnV{PFPcwpB=gtKm@?90uN(PecW(1Wxgl@ zIiR6S;87gd0aBiX0Ug~tkU+Of9#`n-v7&qAaWGj-W^xy0dU>CYUV%aMbMZ`?`gb_H zo})cFn`XY4mK!i$^a=E$o1v_|J=AHgq%-D4DVR{>CM%gW#7HXXwPZD^@M zk!ec1j)w#~(N5*V8=JX&(M(ZylY!{hu*-458dls0n!Tkx4ird>qiGkTiX%L}QD=(9BXyE+nmUOuHFg0>@EQRv4}=$H7iAc2$5g-?V$S_&jMNTHO*@_t!h9gMEX!(V%`8rxDh3n- ziUGxdGz0o6%mLc(m?rxHrpUg;I$4|G^QND8UA^fYgvefIrQNh25qbyx4wv%0fiq+i zti8o=pqJLgTfD=dXcYllQ*KUU|FO8mHM?yvzO=9?m8G&ujY?{onTa?X`RK|1O&4f)wV~ z!?iV`eC@3NmoI76zxpCTIjz3_e?Xh>9Z6l8f5}YX`ZSIQ*t5_ofYb{Bn!ynPnm+L? zvS5}{Ir+)7kM=@1svy4hg1M2DX@rkv;m$lZULu*4NUmn26pqi}H~>vRsSP&5wM{92 z9)q>NttqV98xmSBgY~{G{kEXU>MWn7@a+~^PzlKvb!jZ+X-Z9$Nb}iluxO{7%7v+7 zNnzsCSea9hiHcK@Nv1EjEb(nC#B*l8(2%zCHM0cYkA1n+>e>o9@nE7&^t2#3{x}ZR zg{`VE`7zX^vq~Sq;kxK{V8>F5^6;HH3-`l#YzO#;LMFYIuBXS411|5MuvGm3`YI5O z#s^w8R`%3|9Hd6HwOeC(Pcah*r1Er?Z`z=x^XcJoAC}C>@U^hor{}bT(!jvfG}iky zBsV<7+V1}+J9WNeRkC70F`yVw3@8Q^1OF)o^e6B%#R$Gt<8 diff --git a/flume-ng-sinks/flume-hive-sink/metastore_db/seg0/c260.dat b/flume-ng-sinks/flume-hive-sink/metastore_db/seg0/c260.dat deleted file mode 100644 index 25f81fde784105363c372eba91f6175388235f63..0000000000000000000000000000000000000000 GIT binary patch literal 0 HcmV?d00001 literal 8192 zcmeI#u@b>R5P;#mn}{nEdX-A+0X&0YhQeq(1ZH$RfG6-+YQ4@Qur682oG3I3^Is;H z&F;-@?&~635pjAyhmUg}=lW#zdQRheIg<3p*NIlvMYU^-I@&zg%Qw$kSS*vh%Fi5J zHSdNZuO#5CmTJJ{onf-wck!@^`wyH wy1wVmRobsK&Sb3DTxM0C2mu5TKmY**5I_I{1Q0*~0R(<5Fp_@J@zR0f6#Y9cqTpT@tH{d|bj(Q4xw3HM?<81yW zJlgjPFMJ!7gNPWtsu9_k*cb+14|6(y)}xdDxLjyur&lI2@SBV_*Y(%Y+rF4RM((NL zT{bbk>&;ApCT?FV+Md~|m1BVb0tg_000IagfB*srAb`L>3S6#7BYXWa|AQUbS^D;^ zzTPb)sU8`eC2I}>dS}g3{y*Qy)RZn8ZR={R_p1F;=CZ15Ynu~^Z!SSZ009ILKmY** W5I_I{1Q0*~fu99(*|@z$K1^TR&n;~L diff --git a/flume-ng-sinks/flume-hive-sink/metastore_db/seg0/c281.dat b/flume-ng-sinks/flume-hive-sink/metastore_db/seg0/c281.dat deleted file mode 100644 index cfed875dfceab65004fe1b91cbefd660cea0ccb7..0000000000000000000000000000000000000000 GIT binary patch literal 0 HcmV?d00001 literal 8192 zcmeI$Ar8VY6oBE^iER>v#vwtWI0HeW&}bmRfk1KsZonnr0Ml>(Y%uf+c^G zy|wRkYxcQ4S&N9#%Z|jw)WtZ2dYsewvmA}|$K_NjOTBWDPS|Aha9w{Lt;e56kDh-j zdY4P|?|L)C=n%iJ7TuZoww7ao00IagfB*srAb{RQS&~GB8)4O?G9~wjOLRU?Q`!GItqmN?d$`hyVfzAbchAM`1#m1bdH?_b diff --git a/flume-ng-sinks/flume-hive-sink/metastore_db/seg0/c290.dat b/flume-ng-sinks/flume-hive-sink/metastore_db/seg0/c290.dat deleted file mode 100644 index 4d10db46799e9d59746cd4b6c1e17c2cf3a553b1..0000000000000000000000000000000000000000 GIT binary patch literal 0 HcmV?d00001 literal 8192 zcmeI#F$%&k6oBEEAhn2tp1{%7#YwNAxO5Ql1g`#-IlRrIyyRnZW?ltk@ieG4FLoYKmY**5I_I{1Q0*~0R(<4FqA=2 IV18We?qDStyZ`_I diff --git a/flume-ng-sinks/flume-hive-sink/metastore_db/seg0/c2a1.dat b/flume-ng-sinks/flume-hive-sink/metastore_db/seg0/c2a1.dat deleted file mode 100644 index 8e2ed6afe90b6e9bab82029f98a338a3cf455bf8..0000000000000000000000000000000000000000 GIT binary patch literal 0 HcmV?d00001 literal 8192 zcmeI$yA8rH5P;#cizFp%Kt~A`6*C}eBo?3sBr0eqk?5&tnSv>p0j7Y9ACVIR5-s{I z%49v}E?KxD-)~u`K-V?%Zx)KyfWW diff --git a/flume-ng-sinks/flume-hive-sink/metastore_db/seg0/c2b1.dat b/flume-ng-sinks/flume-hive-sink/metastore_db/seg0/c2b1.dat deleted file mode 100644 index 2a2969247718ad8c76b895d72b43a1f781e3cbd9..0000000000000000000000000000000000000000 GIT binary patch literal 0 HcmV?d00001 literal 8192 zcmeI$p$@_@5P;$9mTZ$e0uBoXg~t&L2Ez*=fdq#^F2VB#I9>*g1jie|tzDOH41y+q zlik|8ZkK&-m5qoPy^0apnAjKwUk`ISf0m<>{?NxX0$OL)Q~p2S$iQ4&HaeHpSnoyqnM`C})($o&5?@_{hyVfzAbbTe~jZ7z9oJ zP1cXQc9;F!Dk~8&`s6LLePa8t^Z78Q^Jmd&;g8FSW)}KnA}znkXmMSCZH?uR!&}## z3f^S{y{itU9~5!>D!r|lZ7MMq2q1s}0tg_000IagfB*sr{G-6`ur{*PFY`axk)5S) z-|F-AM3U;2!B(>JBA{_rp7Q_sMh52WqS2|y#yZAYek>iC7P-BS`ISU`aRGV+5I_I{ a1Q0*~0R#|0009IL_*q~ibGNsBpFLh?bT1nK diff --git a/flume-ng-sinks/flume-hive-sink/metastore_db/seg0/c2d0.dat b/flume-ng-sinks/flume-hive-sink/metastore_db/seg0/c2d0.dat deleted file mode 100644 index 60e3e4fc09f7c3c926027a7014a78e090ff9136b..0000000000000000000000000000000000000000 GIT binary patch literal 0 HcmV?d00001 literal 8192 zcmeI#u?@m75J1s$gd&3|vH=AJqGJG70Hs9~bW8w9bS%IGEQORM=%}dSaB?Ivp`f8a z-%&2ko#l&P7uku3)B8D+5YrHw!RpPN&hOQT(jQ-Qt!&C_W{djT+}q1H&#PZ7qrIxn z99=Z;CL*sUE&GO^{?Jk(g#ZEwAbKU1x>>0tg_000IagfB*srAbIO*cd&_%|}MI<$JIe0=ER`ByJn?qAI_j$H8{6YcpV!Ohv8 zl+pIj<>$UzHK=7BM;|;0IcVbYHni;--);C{fB*=900@8p2!H?xfB*=900@AxY2R*?OhF=L<^3tMks@`>1IV@J3GgyXa#t zJento;ElH!1$0OUDo@)&sVPY4SSVRJBriGLQR3BDQs# zvP+A)t@`{-c1f;{>FO?Yu4_{_%c{1n+xcW3*Y!trCxHe55C8!X009sH0T2KI5C8!X Y0D(Utz!&^;ZuDgJFR9n4Olt0x<(HR3 zzqD7jZgjV}8%v)e75n;{$*v+p2mu5TKmY**5I_I{1Q0*~f&UWNpSDhGJ5zYUj{FyZ z@7M8=FCD)b+mk(a%fpR_-)$f}$Mf(7jh+<3 diff --git a/flume-ng-sinks/flume-hive-sink/metastore_db/seg0/c300.dat b/flume-ng-sinks/flume-hive-sink/metastore_db/seg0/c300.dat deleted file mode 100644 index 2053e0105b40818ea4cc1ea3a6c1cb5edb9972df..0000000000000000000000000000000000000000 GIT binary patch literal 0 HcmV?d00001 literal 8192 zcmeI#u?@m75J1s$K*2&3Y>^6y0a$=0qM<`c2CTypbc_HOCr5$_1qBWI zw&I-cEIWQ(WGf<0pQ0sUOvBh5tlrFNe6L!R{y3d!WnET>wrHS-dz*aozWK#6+N=7^ z?5cS;7I_}hvTr!k?^`OQ5I_I{1Q0*~0R#|0009IL_%DIOY3F34ce4NcBt*TpTUtFT zbW>fQxl5hy6k%U-nQPBqdcBAU0R#|0009ILKmY**5I_I{1b#0tkx_Zb>Nq_=05p*m AuK)l5 diff --git a/flume-ng-sinks/flume-hive-sink/metastore_db/seg0/c31.dat b/flume-ng-sinks/flume-hive-sink/metastore_db/seg0/c31.dat deleted file mode 100644 index 813fe3b79dcc587ee2ff25e01832a8fe6e691e71..0000000000000000000000000000000000000000 GIT binary patch literal 0 HcmV?d00001 literal 28672 zcmeHPeatRJb)R`Zp6}21%0(@+K<%SQd^~pf`TGnUOp65Y>D-c#wG-fZBi2x656EpoHMh}yZ3$H-I1jI z!}vUK&pXfTIrDL5c6WZiIrFeXj4=-X@Y5yPAz2sI^?Dwpe{4QGvKzmi4_7XSe?7)# zl@f=_;d|SM-@4b;tpm5bP3PCjqKwnO4}UsNt&%Iyi)Kf}kfnQimXY-}$ZX(9?Wc6p|ft3eV z9$0x`<$;w4RvuV+VC8|82UZ?fd0^#%r3d)w>@4{5X_EhelBbkDn=n6IdiQ2$UdR9Z zM)nHG{-MQt^-|c zryih&H){hSBkB5y#h?c@{H8X@Bd8&?8Vp$c7BX<7!j*1hHJGqCB?A$PT)L^%V2i~$ z8F+pOeaLMFedt4B`cO*c7B;u`p$~7>eaLf12^iycxAvhAmuZ72RgRF3#b7XG@!NHi zd=V<)OPj%r#lK4iBJ-pp0$UA3==337gG@t5C>tv^oU`~!(}&P=L}D`-jahs(8F(@g zN?2#>7`*fcx=9&OU7iCsSnF&ZLzAx2O>*5*daiAHPkJoAR@YF9NO>OIlC3pNph<_x zK%_Eq{LsDKvYCk^t#^%UD+`{IK< z{uEg_o`4%7$JdGGb(<_+$>UqdLKLaYM7}+AO%|`_@vXX!G!u>}w;k1FaUPF9M;5#8 zx+aSYd3>9$BMXwuFYK*)J&$iUt;&PUFX0_(>G>OY{CRDWNJj<^yh<$=Z{qP6$U>+x zm5JIOz@~l_JibHMp`z4DBKvr}jmLMAh3f~I%!R#GZ|CtBwS^J|SZv>|O|80|$6q1~ zFO+_kBi(Q5`4v3AOV{D13M}m7aV3wxOcpRjU?G!jKh~OBbrp~A)^!Ad$co%Pf>-nS z9&J&;5EghtSo-mYJib@ADoTP}g!Ve#$K(5S9a%2m$pFhHOC8tq_wFu0!DED%GpWnpeW7j%JHb^Z09I;la8%gFDxZV6(;Nc>D-ixH9mvX!&Gswz!?gk7|n`ceBE;?x<#q zFY);6WFbm7lZ9X1wapfH^Y|OOj#MSG2<@}v7>mEDE!?1lM~S^v53=|%vhYGDcZIOe ztFN;7Te^-gRqpmOvbi7M*nymWX1?}*zyxFC4U{}!u-m)#W|ZBFF=zb=Ob@?}>Cx|? z>^w#J7hwLz%_#d%V9w@`G2J^A(`zrn^yYx%$50M0$Nb^TQ1;Iw`5Dr@3$J!Qg6YA2 zl%v0*^uMCy|AklA{uAZ;ttdDC1tsq=h#v2K9(6V!#hh_Kwr8X4UyHJL7-c8HtD^y? zhclF$mtju-B)aO*)yFWsaS+qPqm+Iprq@1*l0Ss9dp+f!Kgo5A6W%d74*zek zm&28h=&Ln3sHN-43jdqDjpfeu<}%fa|DVAgKHY9cA6{VXd;VR6|BXI8VB7U8{BQK( zk@3nxoFJ5L zvJD{`^k?j)TEFl->1I~_XR%))eH8?Oa%+%r@ACTq-G5bgz|Vwot&B*2#NwCh76oph z1Tlndp+o%{i+|1308?RmP0p-oQr0gqtKT?$5{=4M}32?AuL1b3EK@f zfq{D?8ALEGGS3>+Hi16;hW1hfcaNV}tMaD%VG7gxVr@_WwdA(O^?%XJ<5bK3MgthQOLPsTD1g7)<^%Q~=^50ZIr$_~QU#H%4|5o}-=Z2+ zSwsq`5X(r$=OhrhBQX+{9cMtnQ^|i*MGU%_{wW>4EN?aOrnRo{JJ<#{t21 zUxw-4pHa>c!n98!YVH3Jb4G_SJ$ePo;dd$j64JjE<={Oihv#7a=INw=Go}9>)16yL zUPC&c!u*XA%kRIKa<0al!6}&Te;MUE;p*PQnC|`_$$v(<@f6CHgZaZ})71{*z3adJli~ko5QtBYiy#bPG3{56)}}Z0U~ce#pu8w^@FnNKt`O{fMvJ^eR-VoX zNY|AVFTlrU|363qcxgGds|nVbs)~-qhfK*pc&8K&w{a0@N#_MFfp7 zcuaal=u$kw)Zl;I(}8EHt-swZga3mf3Cc1dyhr7Gi&7};K7N+v!>~=YKX>W}3lSB& z^fzY)|A!uIp19WKVRvfqe+XwQ!2P!N_j(5ZhssHmn_B(3&*1-1g_+|BTtTit({)gEQcvxEJ=RSk~HJ^nD1BmT~>jTgYoRs(l zeMZKo`M*A8&!zdWhklVK)ZgRu$L#0!)-M)mj(hXf85~pogq^fY-&+&{Hu~VzjLkdl z%+C52dtQ@%;^F?)4*ePX`6m6e2$7yUzWh1+g_=Ia6SeU{yvt6m>*EdEs^4R$)b;84 z({e_M_t~j+eHh>N@(1h}>-sQ@+x3U+w3QBC3|{ zP4A0Qk1e!*=4QY_Sj(TVmzeedW)$#1YbOZC19paKj}u7P=(OpN3_lo@5PsuX-;d)h z=IZw3D$P7AqMMAFFzrcw#|vyod@^UA)(2pg!5h`m{%MCvtuOpU`en6=*yR7|komg( znXe>x$Ws1n!~&!50&HIb(x&p~V;1W60CkrIrLFz*Ia69cb20$!6~JmLe{0AMuNDBA zlgXqqXvRK&%?+^@!j2{&*tvs;uz)!aax`bvsk0Yk-93^>(J; z2xad!lwGP9A_&mx0OdSISDz$FIdyE`qZ0TLyxQMS@*qlHk_1zLR zY{^di|Mb1zE04qfPtgAVHM?d?(?Rqd_#aUJ_74G=LR$C#pTTK*djF+aSxUFC;h%or z;D5mW9$*^U?+q6jPzMXx-^Aq|;sL<_u=@&lZ=>4(-&?=J4@>>^eFp#Ut&hv`9HGi>I;s0Iw#R8&F34b1^KR5V)5B)TXYV9%Q zPYwRxLm$GpDtx%)Q`Tk%|JU^4yhd53(4H_ucyRoAjlRMEHGM#lKE%T>$^s+m=mTr3 zOd)>W7#jSaIe`!7HK4(*_y3z*76iDTA;i|I7{khu}#Fq4ulAT+@1S*fID& zOT!>@9czDw2LESS2HUpFrC(F|{jtISxhSALa2K}Ylm6V``5cy_$_;Go@AVCy&vOq* zzR;#WoESWx7qAWPx;B2QJ2H44&ILFMg8eP+?~M(fFT%jZN03FoJ2QB`C{j3zV~vsT zjtxF9oic?pCbsg2eS^9fWmabCn5}Y*7w+e4(QC2whVC20wYDZmzx1o55?y&N?5bvk) zc3xx++oC~6Q-inD1P+{uT3Gnr?HP9s-Uj*?Vi0n?-`mO`_YL08T~$Inyj6c_ z@HXt%!Iq%)FwJ;u@OB2n1?Pb(;MU~-@yy_D@P3kcu1$Z^HFz6#12YAP)zaTdg|`bJ zLxC@_`F}DqcstKC2k^g*@0d;v-p;*HKs2-!|MyKiR2c;+oCLD%V9y5ZOj^%C%ycLN z-^TyWrY3$0!kHt`x7}Z}89STGFLPfy0TQ^a^Jm^O@m85r!f7w-iL&{C{jx4!mMQQZ zupntKe`MmbfMj=7=veh37%HX;;Fw0LLeFNfHR16)v`66%ld7yd+Dt9n%Hzv*AAq1z z5v~!b56u>z=kXP~4ks-k+`Ss*W{WR#bF>fQ0pM6fQ1Lv?7WeS@ow`*DLU)SgDb8k# z2ddM6X`187nd)9^ws?rezfX0*FI{C3L^oJ_{xFNL(iSP6xy9eyviRz2EPj{nhg(7{ z7(Bde7LQej6wAy5s%N?9)!dJ7Ri_jSCx=%31c3^>4>s>YI;)KT7Y}H}J|MgbSQK#S z=sH5zPowM~#GK&`nC=|Hbcb>Vgk48(p#1%WW;-Z%&t7l@o}i88PYnK-P8m921MSCn?+lJD zW)PLV#Bb6|X#csw|8R~U){EBn>u6^7|D_kGOh{{gM+W~(nZa)bz^-8@R0# zVykBe02u}jEOM*>w`T$XWa9hMTfT-HJG{j;0U*;90L1b{Uvme}F%ba11K|ZCwXI0U z9TNfIiyZd5maCPf2F*H#0f3*iln}aYo1w#jVE}w7fF{Eu0<8`mRSW=n2Uhsi`@N|> zlfGd9d>`&$VMVz8sfhsaLl1s-LD**8pu=?pfFJr0=3Qe5n`Y3tT!#SoVF+8eweaF5 zg9{9206U1e3aoMA^S{0vqz5Jh6~=(uM%13$}rKcFKCIz}j1$O3O^`YdC3Hs^*j z@N>s^Fp{W!#HIri0|5M?3p2>Jq@Kjkt~h@Lk}*{3YUOzUAM@br8U=TexY)UBYz~fQ%rF2I8WYJ#I2cxuzw&!jUSe zg_}1ST*`GE0GuPw;s0$Osmb6Pt`7=9r(p*dINfT}H5pvXbtHgSCbO_g|0c-We#nXHx!^lz%&>``@RWSCTvt^Lro0bob9u z_U=Ww_C3sByPf1=$|2UEj_n7}!F+yDslh- diff --git a/flume-ng-sinks/flume-hive-sink/metastore_db/seg0/c311.dat b/flume-ng-sinks/flume-hive-sink/metastore_db/seg0/c311.dat deleted file mode 100644 index f60c260f85bd9b0a9e1ac4fbc0967fb1521015d6..0000000000000000000000000000000000000000 GIT binary patch literal 0 HcmV?d00001 literal 8192 zcmeI$yA8rH5P;!x5`~no0Ud25Bq~+_(au1>-;g8#yW>(rWk*+^vw7BlSw$}3J(Y^1U z3f^S{gPTq!4~n>Rwb9nhcC{EA1Q0*~0R#|0009ILKmY**{!!p?+!|Tym-!#;$gVnX z-|F+tMB33SgS}+cML_GSddmOj8yT9bi^jb&o9G;C`MFHBFUo8&#oCjH3os&p00Iag afB*srAbVZcRFqIrAO@g9)UwmeL4+bQhZ0=*mU9p~z@+^}kAM`^)&z^QWS_ zTwrk5&5WWZzOJTS&-_q};XnWa5P$##AOHafKmY;|fB*#kk-+J^H?q~2g%5V(tnBTr zA>T}-nT`yOl2?O(R;xM{{`p3R=IW!JewnqS2RjRx%d*Va%{ER7YatMT00bZa0SG_< X0uX=z1Rwwb2>e7~Dr;YVTTNy!_+>2u diff --git a/flume-ng-sinks/flume-hive-sink/metastore_db/seg0/c331.dat b/flume-ng-sinks/flume-hive-sink/metastore_db/seg0/c331.dat deleted file mode 100644 index 85ee72b31368516c6de9fbe3516e0e3229ea2755..0000000000000000000000000000000000000000 GIT binary patch literal 0 HcmV?d00001 literal 8192 zcmeI$u?@m75J1s$kdl;)Kt~BQRLp=V5hG9o5*-bqVgfc`3T8;b1Qaj@ob5PrLO`NL z-;y2s&dJHIi)AAsM!Ri^8&fyNL&(Q5oj=Rg2!CA8G_%s4i*&;#qs4XowY8Q%j~_k% zR`e|w=->5WhEWl}uQs}x`K}gYfdB#sAbHq)$ diff --git a/flume-ng-sinks/flume-hive-sink/metastore_db/seg0/c340.dat b/flume-ng-sinks/flume-hive-sink/metastore_db/seg0/c340.dat deleted file mode 100644 index d99b11a3f2dfe2333989898bca9463dfb265ef5f..0000000000000000000000000000000000000000 GIT binary patch literal 0 HcmV?d00001 literal 8192 zcmeI#u?@m75J1s$K*mB8Y(Rx*8GwQbEE)>L8i0goSb&C>ozfsN105s4;lv~|p`f8a z-%*^)o#l&P7ukr2(fc{FFxxQKr&zD&G`^Q3Nq>BuX=PPZ`*BfEo4fe(&GY6K%Ve+e zGe=j=o3Y5NNyEOOr`tDFNFjg#0tg_000IagfB*srAn;!Thtt-`TF(^!@3Rnf-cA|o zNuh0Z{lJ{7bZe@y6)$tWym-^CYD5SifB*srAb%c`MXo7z5=i5{uUv--&9MZ&vF5D`EC d0R#|0009ILKmY**5J2E3fuStizH|39egYK>F4O=3 diff --git a/flume-ng-sinks/flume-hive-sink/metastore_db/seg0/c361.dat b/flume-ng-sinks/flume-hive-sink/metastore_db/seg0/c361.dat deleted file mode 100644 index b5c8f259d0d1fefc112f60325bfde24724aded2e..0000000000000000000000000000000000000000 GIT binary patch literal 0 HcmV?d00001 literal 8192 zcmeI$u?@m75J1s$kdl;$3Fs)HqM~2|L`_Fa4M=n}luW<|Ou-be0fLqV;B3c{69N(~ z`j&FB@0?h9T~5{_V)U{jaWQo<4xt|Bbp9+xE&Xvh)yh(@T%;W~89iLrUq|Ed$Kk!} zpNih)61|(=%pf|%@2f<2X1=ZDSRjA^0tg_000IagfB*srAn=a@`@_b_O1~`pU?*;t zy?tw_*9&Q;ErXrpWh0<*Rz4N}^NsY)*=JkxqH0?YwihswSy36gj5Z}sK0!nP0R#|0 a009ILKmY**5I_Kd9|T6S@Vl>_%i{&Jf-Z#s diff --git a/flume-ng-sinks/flume-hive-sink/metastore_db/seg0/c371.dat b/flume-ng-sinks/flume-hive-sink/metastore_db/seg0/c371.dat deleted file mode 100644 index ad11f01b25b6bd995142e4d731203e66b5792409..0000000000000000000000000000000000000000 GIT binary patch literal 0 HcmV?d00001 literal 8192 zcmeI$u?@m75J1s$kdn|b0v)1)ii&~>5H%AZIwU}%qoHI1HedoureFq`0?u|EIUyj? zqHie|`!3EYuZv|XB1SJe5*Je!;}Gg`PUp{J)Y2c9Gp(%k%0)WiA*0Q8{dF|9KaU<= zzbkr|OZ4t~GsCEff36Z;&wO9Wu|NO;1Q0*~0R#|0009ILK;Rz*j;EcGjec49!A{&P zd;8Wiymj?ljvvODX&o?qKH=k{-imE+&uS5XUFRwoxni0-h8-1t!_e7xbcD6r=>CXGu0G2p(1NR=oQH`a<4)15ciO z1)Xf0wzTOMuVr@0WOqLHzxyMXB=7_P7F|`_(zdPbys6{!J{tc{+JbxKG^NPHIC@Yg zbxGW*CoAsD8=}O^-p|cAf4bIWo2TF8w-8_&XjoTkI-`aNyO;nIU;<2l2`~XBzyz28 z6JP>NfC+3Cf!FtE3m)lA_4~a89h&VgowANE1h(5$d1!sk-1dmZE$G95dY?LUT5tl# zSr%mq6XBvKb2swx&{cs5U94K6GLa=np8?wG;QadPf>yB4(zGfpR1w4&N@Jl7Rv#QK z6#B_9#>AZ8BqU{7l2EW-Xh#dlOi23Efo1>Lg+2=@1HxmO8UO$Q diff --git a/flume-ng-sinks/flume-hive-sink/metastore_db/seg0/c391.dat b/flume-ng-sinks/flume-hive-sink/metastore_db/seg0/c391.dat deleted file mode 100644 index f2c4a79ec481c2ac609799bb519af085bc822922..0000000000000000000000000000000000000000 GIT binary patch literal 0 HcmV?d00001 literal 8192 zcmeI0y-LJD5XUE%c*%W%4KbER@Sfsg(+L*H|i8`vw+1j`#?I zg}#AKHpvmzDCq?G5A5vhO!mLKzf`+)MMN66+&a>CrLViHUe9s-S(=jmaTsysf}2IO zUVWtT;ynJ<%%4^p)k8pIw3=1j^EGm{=OU}$msNv% zoi{S4BW}lKbt$t%#s>$c!7l~fXuf|qJD^S4NR!kLMc~=O@nU=8#DSgoZeWYt(q!Q- zb_K1GK8>PW+9XR6@VqQz0l%eyMLk6z491xd6Q`v>kNS#0;*HZ#_-RXlJ`EIs*hzv! zgjq|00Sy%aH+5qng0Q8)u=p|XAOHd&00JNY0w4eaAOHd&00MtZ;E|?$oj;TQN&21K S)0>RnWxPk{#V%(rr*~gcd3yB# diff --git a/flume-ng-sinks/flume-hive-sink/metastore_db/seg0/c3a1.dat b/flume-ng-sinks/flume-hive-sink/metastore_db/seg0/c3a1.dat deleted file mode 100644 index 0040fce9fedc49b500026976e044b54d2bbdb43e..0000000000000000000000000000000000000000 GIT binary patch literal 0 HcmV?d00001 literal 8192 zcmeI1y-ve05XaB0l%!u060d-OCBkjKYKgfcVrD=}LNhP}5?dEWRu(1}o`a|8GqA9; z!TFL@Q3=fs|C4+<-~Amszs(oABO=AaT2H(qzUk_EGsp2)Wn1ZAr{`R`;o%YO)gr~4 z^L(edZPiKd^0>@5-X)2ncYY|{*ra8>?CGsqDoB9<2!H?xfB*=900@8p2!H?xfWY4( z@c4YM=$5~!zTbOfuiBFzUs>0;3y#Yzg9q|9jeyo@v#P%5D>_l@GTVQts_~h{*CmpV{l(`uRpMwi diff --git a/flume-ng-sinks/flume-hive-sink/metastore_db/seg0/c3b1.dat b/flume-ng-sinks/flume-hive-sink/metastore_db/seg0/c3b1.dat deleted file mode 100644 index f970c9eaf3740ec1f584aec8f2cab70e19d03646..0000000000000000000000000000000000000000 GIT binary patch literal 0 HcmV?d00001 literal 8192 zcmeI1zfOce5XNV@1h{{R4`I}XxE!#DiO*oJwlFzZOte-LyK5{>tZeKo?0o}$iF*d$ zLWcn^=7@L2w_Q_j=Q#eXZ7coba>11=9zM}dBU0R) z*T0(ARh>F#2UWiHEJ^Ht@T7FBNymQa>90B}NPz$dfB*=900@8p2!H?xfB*=9z~3Tp z|8%S9n!l+z-}_V^wWr^{vZ-$uT#v5|?#SO<1awB5UClXP(V<#b*}-dFEw3INevuP8 ztt%^@%f`+dJI|p#+O@e&Qf=ZuhjA9n!^{ME5}81kX7gpVJkm5E<&iJ?I4%m-Cw+RK zhN4f+tk8NM_UVT-5`7-eY^oF6rytQ+^jVmjT&G2!eoPb5M>fi|Hff)J!h~QT00JNY s0w4eaAOHd&00JNY0{@A?3-Nz%WyGJ0cS}!ei613CrppS+`^Cfa7g7XhtpET3 diff --git a/flume-ng-sinks/flume-hive-sink/metastore_db/seg0/c3c0.dat b/flume-ng-sinks/flume-hive-sink/metastore_db/seg0/c3c0.dat deleted file mode 100644 index 4d061cf0609ecd4b111d8f763b15c5658bf6caa6..0000000000000000000000000000000000000000 GIT binary patch literal 0 HcmV?d00001 literal 8192 zcmeI#u?@m75J1s$K*j|HB^?!_WdLS~XebbC01_PwFagWZ(vl_U7$F=^4k8l@DkStB z#kt&BzW8;Kt%x|iA0vyik8^XfdNZeSS&t=aq3P25}@J%KlH^aKu0ZZ4idH*cWNmXd;K zoXuatqkXUN!nc7eM8xR%8j+2ujbZThFsJiJF>2|L%aK-QdgdZ+zsYEGU4I>o?RSG) z$K4e?%O<*4y_jCm#OV^ z%iiAA*XxBe(_aQ_$;%%BjkEGD|DA87Z!RtypNndu*I3(+?J+5;2b&d1?k+(@009IL bKmY**5I_I{1Q0*~fgc5iGIcwTt^MfxGuieXjZk45o7=7~Aa(&|ZxC{9>rt@dfYT=K|v1aD_WFqab$!Kw1e{GHBkAr*1 zpNih)0^OSqrWY0Q`zpPwnXM}^76>4K00IagfB*srAbheKZoe|J&@T%=*ey3p z-@Y~E>xm@QD}#+>K-rbX_gV}2!JFWS)%KmY** a5I_I{1Q0*~0R#|0;1_|R%>4F)Jhm^H*DqcG diff --git a/flume-ng-sinks/flume-hive-sink/metastore_db/seg0/c3f1.dat b/flume-ng-sinks/flume-hive-sink/metastore_db/seg0/c3f1.dat deleted file mode 100644 index 08acdcee31e218c5d9c3b4f62ef38fb0f42c8e72..0000000000000000000000000000000000000000 GIT binary patch literal 0 HcmV?d00001 literal 8192 zcmeI0F%H5o3`LzHwH@IGY>Y55aR$W95tsqN#?p}!a05=kDL6wJ7&rkqNmHezAhDun zDX#M!*UH;s*@}qKWi4?&bv||>A8Q&vD{ILgw+qdzb>$-caL8zL-G8;t_UGws=&PcC zIbl@l$4sIo{#?Vmp827nv4H>tAOHafKmY;|fB*y_009X6BZ1>-Z)Br43lHqXMcLb3 zL%y9zGaVTelGlTP&Zw>m?|dU;bM2TM$;ub(?#18*9@Q?- diff --git a/flume-ng-sinks/flume-hive-sink/metastore_db/seg0/c400.dat b/flume-ng-sinks/flume-hive-sink/metastore_db/seg0/c400.dat deleted file mode 100644 index 1e8976f89fb63a0868c737d9434d15ecfdedc47c..0000000000000000000000000000000000000000 GIT binary patch literal 0 HcmV?d00001 literal 8192 zcmeI#F%E(-6oBDZLa-qY-h#=*19*nUK?mdR3Dm^R)xi^ZCmCL&Ffo_&2!d+4ca%$v)#Ylsj)009ILKmY**5I_I{1Q0;r2Le+WS6z;m_3Z_e Cb`~Q5 diff --git a/flume-ng-sinks/flume-hive-sink/metastore_db/seg0/c41.dat b/flume-ng-sinks/flume-hive-sink/metastore_db/seg0/c41.dat deleted file mode 100644 index 88245f03ee1ea1e2d76b88887d765eb26a209a34..0000000000000000000000000000000000000000 GIT binary patch literal 0 HcmV?d00001 literal 69632 zcmeHQd$1%|SwB59cV>FtGj}&jl7%c75=e4Qzow^02nZ5A_eV&_cvg3M`7!?>jv`cOE^P zd^26vhJ3SizwJ3a^P6*i=k(+Eo$q`tP75Iv_`_F9+PJiX#@|0!2HXGJdyN!Ne0vo3 zyaxV~LKOTDDe&X){z5P1<3W30{q}1=>Xi>*$%3oD4gaZ{_al*@??RdPrNSF0?B-%- zfEi#0m;q*h8DIvO0cL<1UB>Y z4;U3MkBNCv1i;0W`r>A7bEm$xTsvtJ;3CC7Tuj^$6MIBy^Vo`B)(ov`XnNJG8oD-s zr(yMsg{J9r7pzvh(W{=N8!cJAc)E`_CteW~mxw)3z0Iw~t=ejBZL3#9V*v4K8=UJS z#>I^>`Ay5JVg(m8-D)mW^rpSgDw{Ue1!~u7n2jo>&T6**3kl)6<0O=gvO@t(?E)r+ zyBGjy8YTrWSLoMJg%QOsW`G%B2ABb6fEi#0m;q*h8DIvUjSML77aLIThr}_E^D(gw zV)_Bxj+~Gso8l;xe_u4=GcOmv;9qcvY#20 z|GmA5zRube{ksYtRK9yCiVD#E7cbZ*IZ-xd8!>uiW?Q%=k^j2uVdxYEdfZLj>>Fuz9 zRX7R#yacXTPFd4Cp#iJHeU%5?vS1u{Ljz6=_tzz`W_KEn(QJiu+RPM%`+5niw=LT- z+F=1_fr{Ufz-HHg?lZ!oA_r95?bX<7wk^{Hmv1Ou*IZV(zb%1{s?&DNGB|?60nQ8e zj0AS{N<}k6JGdhWh+h*Gtn4h@tj0a4>xPPIzpOMHF&2G1xuHSpJBR{0LpOeUT!!Vqx<`1~p zk^fS0KQECDtI=^fc6i8tt+-$C*f3f!I4V|XJbzAczv$Ju)vjor{&77!8($FaS&wI{ zV^<9$JT|^4+%I{^nxVH1!@segz0O|}?h_K(>a-g=%qNA$^OuGDWe>S+J8i=eHd-}KAeY6 zy%nu`wJhI4oB9_liTwz-Gjgsv^MEY5A4_6&w9)UO%|@_1dn?-LyJh(+(84q%Vty01 zljqPTifBQ3qdkpQeIwfRf8!?;pOM=xZb$wZx8skZjl30Y_7VA+J7kH8R{55+zlkh|k|QIOd6A9w;vswpay^us$OVE>=v49|AC;jLL!l4UNRPZIL?;AL zkVZ*Xyf7-BCoTbMH`cee>T9*5wT;zYC9aJ{ zIwIS1s(7(K05{fG{8$rycu6u@C@3YD!7`bN~dUbKyFFz2CWB zLx@bkm5*P{05iZ0Fayj0Gr$atGZ6VFaTrwkABda4r|^Dp2<)xW{+p;l`6+4lpjDm} zOW-4TT-*%yw6xcujinGF_IJ29_D*Rx(9T^hAZ|YXFmA_xS=#HR{UMes_X>#NSJs6K zHZJN=v&W=8jlaID|Nroj+wLEn{}%_O{yzYp8Gb!O{jc^_f3&|fS^tL+aqnwo5V}i8 z$T#Rw6=i=(6yOfOhd|c3(IhJV9>)60RQHmxc~t$lVgTylV~}eSE1>cZLZRGS$BHQa zgHR~9*Rc|+f6og6E!tjPBX^EcLhbL35g&rgiIPF-9{@t3M9GSx?;ToNUput2zFOOW zbr|#}6+|g2E*1M^KMw-Y8iYoT{QWcom+}c>I<60G>wu-3Y zk^#c|Fl3e@aj|bA5QU>iLOgFG5QU*gQe5gE7|Y;3T3e>GlM|QuW7>zHGm{aQ2ZE4T z$%_5{8F+bnb@i0q@9CDNo3IcI9j{$yq1-H+3xY*NP9-wMX?Or|5(~j;O|0Y0sQW+{=fUF_kDLr|DO<-!~U1S-|#Co{I>sYWd46Q z>Oc83`da)U9#v17~{pzDkPMC9Kdj0bV!vz z;H8MpfbqKMv=)IkArOVsqGSSH0poQsNG(dL=miKsF(|ww)93&QKp~BnWEQo5017GN zLo$!je;mc6@lr(HAAmyY>Et|${s0sbFKQeWe*g-J7d45Ze;mak@uH?sEeD{Gcu_Mb zm;+EqXG}FG4g{jmQjQcbCTF}ZWm2*gwS@6F0Vt%NPN^7+6M#bECFSMd?O$SfgBf53 zm;q*h8DIvUEes@9P=SM<0~>!_+Sf^Ip`H7#v=2zzly*Pb*f*qoyR>vA zBqRDCX8m^)0rn6k&;N&$($2f%^5Vnmn_KlI@`M`1Sbzs$w>CW#@aq=^kM<0M!n*kJhZ8GQu2*#nzZN;+a2METY%zVb=f&g!6gKQVMpA?8E@Ix(n z2xL~26#4+XaE(C>3L{DyJpet7jryTOwGDDxVo6{B!&vb#$gPRa`}%(rh0-3#;@Q*x zqbQ{QO`!hw!sUCavz>^)!xk&|4izxb2BDr;Nh5Fr;l=}Ve(j;gcr}eWUy}ee41qYG_LfcaboY03Z z@V>sYya))850wCsP*ISBQ?B8(Eyw7Dwk|)761`tF=-RFNR(*YqP_=9U^?9EhIl2g6 z&!=&L479A-ZFe2J>aWM0HEU`yjtgesz=5%{b9AGAoH}5NNn9=i32|k2BLYNH9I8>< zS5$Gy%;n-z3FC!aO%ImE#WQ#mL8I^}nyO(rTtFkoQpIlBU4Q-T%#0UsMGfG#EL&?q zW`o`u*O_rkxWoo7ao*u`)ArgDafp@@N-sSNG*`D^)puyiZ~shjcf8?iSzX8J_*o)m zUqJomirbLLZO70ZryCyfN0nacm$u&3VA;NZGMT;3k1B4<8v1CDa zmRp9^32_1SClt3M>)dIYEvHisZ*l!&N-tqe*L18_r|f69n4Rax758?JXT8#e^a$aN zjnDeIa3FV2vuc?B>3H@!e?_>zAnV+4I?c9WhL>vK+_0+RcmJ;pq>tfB_{<#I^q-J+||lEQ(s6-C&>~i3?hJv9A4SkI=NO~_8!lG#PuHbaN#fn%q{d)^c?_1UpP?N z$jU4NFuT`B*pV3?XzDUm>_rbC%GB~Qn?>(|DiWIzK2EGa23-dJS83u>#tERpU!JZX z(?xUeN8cA~Z{DO_7zg~b>&9x_=B5qN>vKY(+5ab>%x*1cnZ+?0d@s|H3V+Rds?nrM%!7z{CTN+2~PTf{&J7>m271lXdxoN@^a zg@D8&yEt%a!EUB9lat;T@RDHW4R}mUed`p}4Uj9MqXilT>zocQu1xI#$j8y!0&cAO z@=0h$c`yOUg-_Su49TFYW%s+3Xr_D?-6{R=l6-&p9C}jV1SLtE?gUkcqZ_4notT=v zLJIvSy}>z^^Q4eQ=Si;~Q@daad32S)X{bLmXLJk|3g{!jD|g3gXm%5ZLMRy;i)qYE zk`^j@U8sjmDTz5rz%z(n-r3kDp8HEF%tr#lX8GXM9bU>|LK4UvwR~`@?k?pO&EwT^ zAl+lx&j>f`tz@aF=-w@=X*f=!={p@}#hu@y7(HCeX*HU@%VB2ReTwPf+LcPHt^49# zZ&ns6)efF{;MKI-9ax}7x`Vt^u{>a>({eP!*8sEo8{VaqC9qRAs#d4pbF%~g)dbb{ zoG6YzjC0-b*T|^*>!tRVtM2E1AM@eQ)iED_?0=;F2&&@Pov5W_*Wu@*|A{g1(MQlG z&Z8Pv--8-EdKY4<-;#C%ZQ=&}ZsLqwv44&H#UDvE{%y3W1Nd7I=?KWlTTt^S-zDwW z&_+Ip$jK5Sq~Vcl@cFai2H3twxL|8i+uw$MfyjPDgD2*}FJ`a79DbQ6Wj?>xOFzZ0 zN&7;y%2jAnzl2ttLz_N}HhZ_s{C8TG_w|4Ip0}JI(*K(i;{TuF`u|igX8}zq!{Xq>d*g=w20Kx znUp{OKY~J%F`0}%|38948ZVh#aQuIT)YF+laQweR>Z7cT0~kN19a0}XJkGKF&*(#%>9t1OiY<=ek@Rm;MK!koqW>437V= zl8&QXDlq=v)MzM~!1#YtBlS@(_nf#r<3=(A%m6dM3@`&ffebu5;{V@v<~QCkr2o%~ zYoIe4V;zzmEAa|O=KptB{qH^rfS~`Uc$j;@9V9%miE>Yzn-A3gWtwiu%QS#vceC)g zdrDpMsX+Z-rscrP zB=u9s1nPfVBk9~iE>Qp5G_6o52I_yCmI0%f2-N@ZU>VBTEv5qXzYS^N2nw~B3Do~K zZJm2DAGrSCE|WBOu^6cTZJK|v6ukc5rui33a`t}gmVF&PCG=5Aru_@3 z3u(NR3cH#67FX)9%7DCLBvQipcQ3oA4p`3kDeiUvPL+}$2XP)$@DZ_N^{G;>N{I6j#XS*-L%1#>&cg~mSvH_P z#Qyr2;+`7Efk)L6jw!6qCl!1k?Pz@fhuE%9Delh(;t<>QX$7BS8&Ds@JsP&_^TPeP zaU95uL~Pet`S{z>`T!2$%7FEGLf~qO030Ge=LJ4BcMJ#806p_FGS=s*2{-6m%nUFC zlNd-uFfJhen6$6M+4%GurM(et@(0pBDDCaiUV%38 zx6aiGbd*q0F$1DQVgzbHqNAZ?f^2}q4os05pr?nk9Y;l)hi%nywi4g??o0SG_<0uX=z1Rwwb2teQ;37m?(k*&Tge6SN|Wp8f{ z`F0}BbYyUpydDH}TJ@>$&o?qQrH}RoRo0Fk>?~j|t14qR+c>#e3xNOxAOHafKmY;| XfB*y_009U<;3onzS^N6pvV6P%AQ>&m diff --git a/flume-ng-sinks/flume-hive-sink/metastore_db/seg0/c421.dat b/flume-ng-sinks/flume-hive-sink/metastore_db/seg0/c421.dat deleted file mode 100644 index 65775ee7bfbd1fe9a4e81f326933d49be9138b53..0000000000000000000000000000000000000000 GIT binary patch literal 0 HcmV?d00001 literal 8192 zcmeI$Ee^sk6u|MxGuieXjZk3gY7=3JuT%Wi;?0i0q>HJx=TKMB~qM3z0nMm7jGFn{MUt44O4K00IagfB*srAbheK?yxqp)GzZt*pZv1 zZ{Oc{L6;HW?BpOap`f8a zza_u!EMNR}k*$b0?V)9HEaKQ5EZ>dk`rfuA{Bb+i%(}@A?a)Ard)s_7*M7N5?rJ}? zx@p`^M4m-@?u|A5z9&Nn0R#|0009ILKmY**5I_Kd{}MPHcTP6?Ci}n7;;MXmrR9@E zUg-IeyS~D?{&hs>8?kdI?JewM8f{y3d$X02B)(hZA@7T5XL)>{5De)RoQ z(Yst=aMzm|MMeC&TJLJ+`&x_%0tg_000IagfB*srAbfaii?* zUqil`NHaY$I7nU{1hkH-r@}tp$k5z;w6iF)_UOTupUJY!=my?>PIC1DdIS(a009IL YKmY**5I_I{1Q7T^U?MBOx_2t3FYvf6QUCw| diff --git a/flume-ng-sinks/flume-hive-sink/metastore_db/seg0/c461.dat b/flume-ng-sinks/flume-hive-sink/metastore_db/seg0/c461.dat deleted file mode 100644 index e6d98541fb79659dd82e6c495be7eda4795dd219..0000000000000000000000000000000000000000 GIT binary patch literal 0 HcmV?d00001 literal 8192 zcmeI$u?@m75J1s$h#Z%UKu3#&M8ynr~rwMhLQ=`fQo6D0XjB-!;T{-1VoL# zB|G+=lT%)2%SJ?u_E*dNX#JQDF`veC{wP}`{Bb$e%u2fx>Bdb)i|hJpYb}2o-Fo4z z49z7(^F^7>_1V$#Gnr_gmsvf;Yg8n^hX5l22q1s} a0tg_000IagfB*sr{3tM%W!UMgJwE}e#Vuq2 diff --git a/flume-ng-sinks/flume-hive-sink/metastore_db/seg0/c51.dat b/flume-ng-sinks/flume-hive-sink/metastore_db/seg0/c51.dat deleted file mode 100644 index daff727ca7de975b438b4c5f74c5244485d279ec..0000000000000000000000000000000000000000 GIT binary patch literal 0 HcmV?d00001 literal 32768 zcmeHQZLlO)Sw3g(z1=_WruczcR#{tWX~p-v=bV||o$cv! zs<14j@7BI|_Vn}gIsNvTnSP%0v1C6Xgu-9;V_`HfR)aiVErZw3il1YOYyWv69Jv<$ zLNyZ!qEI=nR+g;%rvJQB1O-o>cIG3zymBrq*!OMtW!jZWIPR#3bv9Id6rue`n}Ie1 zZ3fy5v>9kK&}N{`K%0R!18oM{473^eKbnEZ2KOH!*Foc@zQ3I??f>eP%aO;AIk0Jf zbnlP@#9BB3$9*MeOQBE63+dCM^dj-Q1HU`al2?*TMaik~o7LR;90V!VJ0!VagX|^G zk8Sqj=TqCx?P-}u(bOyQ82)Le@MrGKY@2M54c12fvq6#zks3s0Zae90YCHaX8aeZD zngw3SsBp3@_0k!mTH6ciCP^+rs%r;PW;=7B+TPSIyxDXf7ZImISqXoEx@9e>TO@fE zQqws1W4Fi}Y900$RD&cJBQ+}1EDrLZfjTx9R8w4Y`>V}Bn}Ie1Z3fy5v>9kK&}N{` zK%0R!18oNWA7_Bh$(e9@kFZGJkFURjulM2WvrzK`;Gsib+Edm4^aW(bidRyzXQhM} z7$!*eTdVS40Jj!GbSe6m8Xi==*JWF(Ap$BEb&w z(svvX^lPmF69|`T5r!oBIZjA}neVubHH=8|IxWJOB(LX$((!!9Yph{HlFPIR+a!4d z5***oeaCOCVTUAd)FSMXB;|yeANfwuRD%T;$y|$Ik|g7VB6EBvY^=c|Nv=iclH^(> z*lPEo3u(PhivVfe&zq!{r5;)*|#saswx*-MC(#BsXdi z;643zEdso!-=Rf-_w=nuu)`va-FcH2IDoEkK!-4dJN>J?Nx2`!Zr)VG5bndTX%XN) zyj!bDkfrx%5g<#q=`?8sz4h0%2+&&(a)Lk4ea}l9KJG_w4e!+=jA(MZAY?@xg-zD& zZkHzSk*vk*HXd2B5n_JTJFfSZ0CYxk8c{xq)bvFgTd|ID0Zs1WjJR;^&~X-xs$;x?ChzAAFL&cKUS4+{Bd5s+_;sZ4Rh_%dbzDu8 zyY(2?(By-%Ri$4P%llo|s&~=k9?pnoIa~=d{P|cx9pg7>a<3laH)--AJ;p(rd>9#a z2w#rwtob^AizfH!F@BpShk2`J^E9?S`3d6M<JFORugAEPCJ*Q_ z?xM*@_;o~4>c@+|Ti>cbAjyNgjv({`*O$4fXMCL`AJt=gqvpA|Ep9W=X5hP(fx`sm za+Ccmn(rnKAzNo-9I&|gw-_H}apQR`cK(G$;|myn2P0*1>#Ho9-ytp`z5N(phH;0* z&Fk6G&2urH#Nx&?7(c}#3_-JakVWehhVR~p@qCQWVUaw|j`j!ab?}dC47HP~%sL7)F8CF8hA}`{%=AS6ud6Sm{ugD- z;pgR``&#|4F@}#H%&K^kbSwCPjqI?<3dqq3lIrm0)PGZ_B(UJ*Glo1L&t##;CtGSR^s2HDoan`Gzscz0SvGk>N1 zBeJW7ACo;=_z875KJZewwaZ7+<4+VZ_7E5GvG31p*KV3v(j5{{3qK-0$7k`}cAdua zk4d0~pO8?eJ%b^MwD2Pm>$GPuCNnMkguDpxGpCGWH{5!7&w76H7zSLhJN(4?Q{gu+kl za1Ii8tx3BL4$v#L2;cy5-#AvR0epcAwQ2xw<035rxVo=GLJBTI7M3vSZ2d(;U%FU} z0DbAzI)ojtTYg510CvkIS_CjdUW0@g{Rbp>v%IwexBD67a3fO{GO|~G!D_^ z7xWmvN0ZBWs~k79ZBIV?)Yox4O|IY!aHRZV@n)@O{ApD~w6DJ>Ghh`=ky<{+4OFLJ z$9NBmy_=X)?OlNJ6jsvxM~tVjX#E?D{co|O{g1KOd4{JdN=%i`J9usC9_N!An_7HbX=jh}nF^38E^QLD}Ye(pRRs62d|6!0hwv#oUe~-}riukS}^nVBgJGPTI zoxf`e{jZ9*$my7$$d3ctDH_k;C1+^idt}P-nIG9s*?9gw`7tg0fV>d#iv6MACAJp6 zN4TC>><|6E(DSPJfzb1+_@U7As`!!6^NRhgKNfmk6+aPrUa`ORw}qZp#19Oi=N0?s zz!Z936>kYWuZr)Ivv~hhd%p~NRHLE{{4`q1!aBwqsgSD5 zjFbwk8pYt|rL(0ltgGV+DOQykS4y=S#z71-shSncTcuo8X1q=6)hJC1zsQ@F*o{)K zDl^_L6>AuH5!BH<1>qf1vW7uf*hMMTMg2X$MQT=M#;sDchIX1~McJ&Y4oKA+&Fv`m zQ&>}oN~~VN+#^-1GUHyUT0_?^GJh_8occOGBvq?2K?sq4lwTUsaejK`#Ejoi$G^_bGxsAqgZs#ayj6C}Bk zf57=gICI_RR(+8qzo^If3zEEvGr;E0T~|&!slSddk>n~p#+Pdz=G)>n1OJ;DFi_7M zsOPsXVS1J8?GC=~>}4h0h-uyK7K;`kMC4RQIYcP}S{K1>XBz#x?K9cpZz@Js7WMaRW8^25R!ER=3V(NBghG zcsh&SJ276)qV)|d|2)QnEN-Bl-#CY5hI<*@2dQfQswYAaztsP3ckkCHzq6S2zqxp|R{yKc|A*D|_!*~)?+X3z#$g7l_1WX}y4Cud6QTcA@!LZG ztKxTr{#V8C3jOcS;ym^O`8-lL{4pAf@qbnPM2!Ed;eR2}Vmo7L$Zj+xf>XYYa z;Robo#QXLvf>|KTC*b1`J&n8M`?T;q^8Gsai9voq3vZGin zs^YhWf9m@V46iq?McaXymY7df{80F(s`!!cPgU__;h(DFCsa%X@ro?8^TnWMz3uQ8 zDkgR)Gp?m#W`{E4Ix41i_%^Ht%6%y)>sxibw9LHF@nIc9v#Ik=X_+ZAZjzRnGUH}x znRz*UTDWnuopqbE%wQEm3~LPJmDkJM_eskPG;xsGFc#lr(R@H!X3C7arDX<-1&X*> ztl6n=)#s&Urp$O;T4u_OKa-Z3o8^%W(;*wL{7Jd46D+yZaXooI>aXK3<-$%s598eR zn~kHd$(5bH@4K+NLfWMDb$qi%L$t-;lNm5j0~@G;4b-Iu>R97Mre^!!Wcn7yjUaA4 zgI>U+7!P2)1mj*7SGD~hW-JTw7UC_$TZlK0FczQx&*$*-@%gHfuj+HIvAdOd{(Rp4 zs_N!SyN7EEI#_Q9>*dac`2?%iJcO~t7-BS7?BN=O>b(8_z3eKg4h2PS-3WjG8iwz` z7-MA(0$(X`{PF+0zIO2|s`{VWxH8~9D;-jC{y%J)x7-L&oMf&3*VqydpFdRb1EK#F z@uRNL|El<&(C72m4{UhAZ}J!9{CrjXQ0Q}2{7C3?Rs2}!^Ld$nE z6#85hZwY;_ith@2u8NoX9K0Y8)<@u;k!Nd1sHeDb0jD&q3T)t=IuF&U-_)SCT zb5*=4^tmeD68ijTe4fwYO5zOcE!ln_+us=reXfe12z{=K-xm5@6~80&c?Dm_(_kLi z(>Yi;Vtt1*gZ(?%`jpMS(5r>*l|C2%&_A7E>cP~0xZb>+mhe-!Q80HS+-dgZ<%C5I7^rtGmC-i6L zhjs?D^PAqkJ3@ad;&%+8KUMKPp+8meeVV)(KhW9K>)2oHgHZqUc$g-?gp9I?0~p$D ze*S-uCU4<&6h-0tivjTZIzCFp#vMRVV?0E~&K+4+Mo@+`ZyK4cPKL+p1!6SD{8wQS>CrJMx6xpXmXPtE(ChyW?{0U8N;jQvwKPv(`OR4_KKS#xWCz%tK zps1UB$)C#&PhcYEeCB%c(WJhPr{s<&X6*K7V{_=?=}M49nda?cZG##dLKq1%Es z18oMhGGO2;_bt@Yd`3X`5YxBaoW!-~A6f`b8|k75=L;_BT;QTX!*x zbp=NB3%Uefe}zR8RkVo;9A@}KN!Mku>QGc&iQZ-GXz#r&s<;Vt-odJCuwMVe?q^V= ziC#&6FSg-dF+R>BMQ5Qp7vR|O|8E_f-c;59;FZ91EejvvGRq;bH{;!H334cKqFUS8?@pAlM6)(sC`Th$w z(=)Ir-U-TRnlcf`^(XwF3*d%;p4k`*e?b*L6661>__6R8s(5c&xUM^mBM-djWBw0_ ziSQRx@!P^*P{r?v@qdm-y|-?^3D}K;kN>OUE#WVy;=96MP{mJ#zaa4JKhap*emVZH zith`5K@~6k1%YP<$J+J`gukGQABypRRs4<^|L1t)Hx2{U0=W~*)9O<-GOHPKRjYQ`QPYcb5{TW diff --git a/flume-ng-sinks/flume-hive-sink/metastore_db/seg0/c540.dat b/flume-ng-sinks/flume-hive-sink/metastore_db/seg0/c540.dat deleted file mode 100644 index b4955609b98341d56cfcc2c70b4f4aa6fffdefe0..0000000000000000000000000000000000000000 GIT binary patch literal 0 HcmV?d00001 literal 131072 zcmeIyKW|%A7zg0vCUu+=iYkGswgVkNY~26$b=4sQkw8MCN?ReZP$zK`w~^c0j-yIQ zbYWs)p@M-GJ^&jV0~-SpLSo_@@EPDbad1V`il~{&b1a|xo^yQe{hcRQ3F8n#t@=*d z`Tf4W-#^|Ru0K9k)qhRfzowr$yjHD@4ptvdF0WSKrRw30$>167$o1>jK6&F{&%?v5 z2Qh1_UsMxwaR{dl7mm0#maEAlZ+yg$O%WhKfB*pk1PBlyK!5-N0t5&UAV7cs0RjXF z5FkK+009C72oNAZfB*pk1PBlyK!5-N0t5&UAV7cs0RjXF5FkK+009C72oNAZfB*pk z1PBlyK!5-N0t5&UAV7cs0RjXF5FkK+009C72oNAZfB*pk1PBlyK!5-N0t5&UAV7cs z0RjXF5FkK+zzZ$#%Qv6?5q7I1)~3fi7Z$7Pl_w6}cw%OMV!mGOe7*MTZ1wbVRoBCf zaPyx}Sg41?{k3pDy!K9av$MN1y8Lc$+`YWf8Fkh=!)~|~E^PL8x>uszVRsO1S8LH; zXVBfgTaBw!3mXT!s$13X?XK8Fiy^ zXAo`P-P!04hS7t5wAH$w?G567vur0j>v#6X{Wux6<1((=RzJEC-QMl@MoAjC$~x)sn$c4`FX9Lkt&^Foe7O8R{B(}mu9(6pOjWvHq)ex z+j*Q7Sy7}#o*YwaX_l&fQd-YQ^`Es~p5_3x=009C72oNAZfB*pk1PBlyK!5-N0t5&U zAV7cs0RjXF5FkK+009C72oNAZfB*pk1PBlyK!5-N0t5&UAV7cs0RjXF5FkK+009C7 z2oNAZfB*pk1PBlyK!5-N0t5&UAV7cs0RjXF5FkK+009C72oNAZfB*pk1PBlyK!5-N z0t5&UAVA>%A~646_%ejK@57_2zZAX-;nZi7y7uR!u6;PE!;kxYC)}#)tKrQ9n~#3_ z`C{0u>Qe{fQ0!mNO-Hj!ZC0uX5FkK+009C72oNAZfB*pk1PBlyK!5-N0t5&UAV7cs z0RjXF5FkK+009C72oNAZfB*pk1PBlyK!5-N0t5&UAV7cs0RjXF5FkK+009C72oNAZ zfB*pk1PBlyK!5-N0t5&UAV7cs0RjXF5FkK+009C72oNAZfB*pk1PBlyK!5-N0t5&U zAV7e?izcx0-M96yTb-&lJy|`>Rn_7X=bX$Q(dO%+9vahy3$?Jk9y){0-j(Rypf~P} zy3x2Zh&J!;Y;*_1=s`c)YTeKF264Yxwv(OpJA31PoDAD>8CPwqAKi#<@AiA6R=Zuc z+F70@S(&tpq?t6MQFl1%^|#WuH`cD zE>^kL!ma1em8N+!EAq72Znny_XtuM~VXjH6#jr4q_0&|2S*lAXm8vYtI8U-xoR(=` z=5d~7$4hm@-_;ZW0t5&UAV7cs0RjXF5FkK+009C72oNAZfB*pk1PBlyK!5-N0t5&U zAV7cs0RjXF5FkK+009C72oNAZfB*pk1PBlyK!5-N0t5&UAV7cs0RjXF5FkK+009C7 z2oNAZfB*pk1PBlyK!5-N0t5&UAV7cs0RjXF5FkK+009C72oNAZfB*pk1YTT$+MV#I UI^A#k{nhaG!Aalx{Dw^aTW+nPfAI+4ul52lmf7-(+*L-&SYf2mp{S9*(3dnT@obWi1w$i%~%l8A|qbtN&A63!~AFe&4lk}8t`6D2h!AvTmG zOd@P5X)uYgrKHKEf^8)&CROMtX%i{9*j3VD;^DlKE|c^n(o3;n2F!pNFau`54445k zUcMk^{*H$s5Tl$qUI7$$iPbTLt S@=o$vas&2;|1Y^6^nL(h+Cdcn diff --git a/flume-ng-sinks/flume-hive-sink/metastore_db/seg0/c560.dat b/flume-ng-sinks/flume-hive-sink/metastore_db/seg0/c560.dat deleted file mode 100644 index b2861f8f03d7875b603e1ba30e59d2da6caf6987..0000000000000000000000000000000000000000 GIT binary patch literal 0 HcmV?d00001 literal 65536 zcmeIuJx&5a6ae5E5U{YcF(Kg!#G29r7^1z0P-_}`F5v~diit9C@LTkwP zHuLx1OlIdxuPLP*t%yO{kITN@HT8Bb=KDH2=|7kEQF*Sat75p0b`_Uj7K@^TZ2Ja4l7V|nMPi~Ent5+Fc;009C72oNAZfB*pk1PBlyK!5-N0t5&U zAV7cs0RjXF5FkK+009C72oNAZfB*pk1PBlyK!5-N0t5&UAV7cs0RjXF5FkK+009C7 z2oNAZfB*pk1PBlyK!5-N0t5&UAV7cs0RjXF5FkK+009C72oNAZfB*pk1PBlyK!5-N z0t5&UAV7e?nF+kj@22S`9+m5d4a(W%*E5@Xr*JWj{WtmJCp~(K`7k}C$M^&R0t5&U zAV7cs0RjXF5FkK+009C72oNAZfB*pk1PBlyK!5-N0t5&UAV7cs0RjXF5FkK+009C7 z2oNAZfB*pk1PBlyK!5-N0t5&UAV7cs0RjXF5FkK+009C72oNAZfB*pk1PBlyK!5-N v0t5&UAV7cs0RjXF5FkK+009C72oNAZfB*pk1PBoL&jO1ytA20$xqd$YB2^Xs diff --git a/flume-ng-sinks/flume-hive-sink/metastore_db/seg0/c571.dat b/flume-ng-sinks/flume-hive-sink/metastore_db/seg0/c571.dat deleted file mode 100644 index da6611056afd030f0971522e2791537cc6096f75..0000000000000000000000000000000000000000 GIT binary patch literal 0 HcmV?d00001 literal 8192 zcmeI$p$@_@5P;$97RDrxfWv}8;jsuXC>{Vq0wg#Lk|)3;cpw}J5-cwOw{~5+F$kLc zO?GSVx?T3URn{V6^vXtLV`5_%d_BzR{F#qR`r~q>m8D*(NXu_B+FaLPM{WC4|K4^_ z1@E$n&P{Ko8#HnIO40UAx1}5l1Q0*~0R#|0009ILKmY**{!w6m*ce&qm-!#;$j+K? z-|FktLekVDgPo+sK|t-Sc*_6h8|j&|%QhByHP(C9ek>E2=Czs4iNsfzAR>SO0tg_0 Z00IagfB*srAb`Nn0t1=5y~D@l_yU*jFE;=H diff --git a/flume-ng-sinks/flume-hive-sink/metastore_db/seg0/c581.dat b/flume-ng-sinks/flume-hive-sink/metastore_db/seg0/c581.dat deleted file mode 100644 index ad4a0d47bc034a370c1cd78613cfeb37f788ca4a..0000000000000000000000000000000000000000 GIT binary patch literal 0 HcmV?d00001 literal 8192 zcmeI0KTZNc6vn??Ki=S$9KU7F8oobM9*BcBXA%tRa(xJ(xr z7H9mg<$s9F-#(UOzbkW=6Rv;wF^xB&AOa#F0wN#+A|L`HAOa#F z0_RBJef4B8=grdh{Vj@&D!IG*zCcb!DP+sw6?WeUI6B+!OW*khb+h&A#Y>WnOT@i^ zdvudb+bL{HY&Z!6lo9s&gNOMfLIEZ;)=5!+7ClaflnNH*fIn{&sTpfWTL(N(CITWL p0wN#+A|L`HAOa#F0wN#+=SAQPRbFU~EBa68Z!q##ysNdoe*qmJIqm=e diff --git a/flume-ng-sinks/flume-hive-sink/metastore_db/seg0/c591.dat b/flume-ng-sinks/flume-hive-sink/metastore_db/seg0/c591.dat deleted file mode 100644 index 34255132cb5dc0a46bbb1b61fca60f510c99edba..0000000000000000000000000000000000000000 GIT binary patch literal 0 HcmV?d00001 literal 8192 zcmeI$Ar8VY6oBE^C2W%%0fz*I;t&WF3XKL590rL%a|2F+&19+$w7kF?!jNxR|;ahft4mI)9d{-T zUD3N-qJPty8AeU~bG7Jt=G$711p){lfB*srAbby^vl8zcH(B) z+qZ^#vyf(bWU!OGItXZ;RlCA}zL9|`e718atM=%@wx7zZtjzUp8z*NC3tku%~d<2IHJxYTKeO1qLqbSxkx8$GJ3eKzmCS^&%;O8 zKNY>pC3<(gnL%`j-&cw5%zRtPu|NO;1Q0*~0R#|0009ILK;Rz*_Q#EprG8oX!A{&P zd;8WSBR4vW` diff --git a/flume-ng-sinks/flume-hive-sink/metastore_db/seg0/c5c0.dat b/flume-ng-sinks/flume-hive-sink/metastore_db/seg0/c5c0.dat deleted file mode 100644 index aa4bef93f63c214b1ddfeacf13566a2d7fb86840..0000000000000000000000000000000000000000 GIT binary patch literal 0 HcmV?d00001 literal 12288 zcmeI%F=_%q6vpv4vWkUaO*CL)Wff8g7J^L#8!MX>fm|U6NCLqF1k%~%8o5DCW$z_& z3DGxicNuXMTZR9^K4-t@%&xy~R(Tf@r>8%fRx4JwOVQuXbiB@wo%rD8qjp}iZry+Q z+tu{u)hs($;|J2OPA8v#*$i9D5A(;o``2-)r)SpRTdO0DkKK6hOhN<@KmY**5I_I{ z1Q0*~0R+xVU^t%L%bPxPvA^#~S+5$=uSlZAf~2A|uiY|q%fC8y@+2+|+xnLE>mj)D zfx8UBtp!&hXxkJD0tg_000IagfB*srAbH>qs{AUA&KpGhU diff --git a/flume-ng-sinks/flume-hive-sink/metastore_db/seg0/c5d1.dat b/flume-ng-sinks/flume-hive-sink/metastore_db/seg0/c5d1.dat deleted file mode 100644 index c37d1b60d37e743b216f8a1a6f0827106570858f..0000000000000000000000000000000000000000 GIT binary patch literal 0 HcmV?d00001 literal 8192 zcmeI0t4;$^5Qb;Aba&gmbX({hNkAY#p#Y6Up!x^|2_%>XgCxP@5Ig|_i9#SqY6v7e z071YDkig7rx3i%O0!ij1`}dr0x;@=*)fqSh0HlkDBk4-!Bdg~b%|FNE8~$;*O_hDR ze1IhT64K_h{&kd#s!yBu?KWPLM&L%f!=X`_}d`M|| zG_G>%fqMZvFc?>Egiuq0Cg?enE&)P`*61s7nD|&xQeslUs*)KdRScArnFJUrahZe| zDe;&@7%K^gI4;(dgiJh4ltfH?tSgC`RIs6>#-xg~N)jdk&MB!g39+f9$t1#-k`|K~ z+e+F*FvgCO4wD*mm2{aT*i+JDQpdiMK9ffJGAzx2888E8zzmoHGhhbHfEh3YX5cRd zoEYd_g&&fyl8=(Nl2?)!l4p`b$vw$|%ju@>=pzavgTlf9)-9 G48H&{G(e{S diff --git a/flume-ng-sinks/flume-hive-sink/metastore_db/seg0/c5e0.dat b/flume-ng-sinks/flume-hive-sink/metastore_db/seg0/c5e0.dat deleted file mode 100644 index 0bd366af1c01ceb82ea63f45765850f9adf196c5..0000000000000000000000000000000000000000 GIT binary patch literal 0 HcmV?d00001 literal 8192 zcmeI#F%E)25J1s^C|X!(@D8@NCMF&M3#hz?dIvl0y?__t0#mEcDEsxA=Feg@%HHWxJ8Rb+n#)+vfo|S+HE;a5{p#}A zxfS+VCN1up+)O@oO<%Noe%si$m^uOoAbHV|dT$Q_5&0|R=k?egWI!jweR;?mK009ILKmY**5I_I{1Q0*~fu9S^ LW#OK=KO9FdV4)ip diff --git a/flume-ng-sinks/flume-hive-sink/metastore_db/seg0/c5f1.dat b/flume-ng-sinks/flume-hive-sink/metastore_db/seg0/c5f1.dat deleted file mode 100644 index 051499658057a3bebce70da5e57adaa0c9a4bf3c..0000000000000000000000000000000000000000 GIT binary patch literal 0 HcmV?d00001 literal 8192 zcmeI$u?@m75J1s$B$AYjKt~A`6$4OGP%#2EAkiTqN+w_freF$cxtdM0|$`cN)Q4G3B(17D@Ytb2ysA2;k_z*ws*%<6W7cI)IFNo zc6GOZey?7=s`no+`2r!N2ETY*QSH2H%gy+Dxew0Y93A&`ANz6_u6#~k-5B4r3h>48 zi(~%%$fW+C-G1(}?mv95(Z{_07wDg%gj|^P+^A16PXZgBuP#*{s60@4pz=WFfyx7w z2PzL#9;iG}d7$z@<$?bd54`Z*jbD?`!>1kB_m7dQ@ala>KL65@9@Q~B4^KW)`(5ek zhv9sVbjSdH@rNwTk!$2pN%3IRp){jTkp+R{=9z?-G=`sqg_J_MzOi?(*XVY(+ufb0 zSMNuAWQNrC?z{objBfxp_sC)yh=k*TD`W|P{rXnhinLTFGFZ)M zH|jKPQ0oenQszR~tB4JpEKA@_rbxz+7mNuAc^wz^cB{T;6@uqWRtfXuSSE|$z5Q17 zw2e(0C6MPEhe^*Wh`+{78`6DcQeq_O_`A{VMXl|QWn5e+;i$?_%wb*;IB^o-%Ta#v zV6e`UHSi$hi+;2d*+u)sBQ)2^Im~DoxH#8J*-($yjh0<*x?h3rj=_|2&k>U1EE~lj z2ea153YgXJwYImTp3R88O|c5e@==P$e&pp!?QFZGcsX?oK^eG?G0$-&Paz{_g;T(h^T11ZK#30fUe_+{A6B4H zG76~|-kx#hq+ZUQJP5^T$b_2*OpyR+Nvj5FbsxOXZX3w$nU0y`Q5?h<(rQSGQ7HHW7L zze|Aw$-I<;3r%2{XPxL)Uol%vbkmeZzFV;}=7o~t`fo52b^=ef_M>6DJ6blUL|suR zk7u!@DMX(E1(C1JLe$X8uB!?br>+DTPP5|!5;pZXc=blV)$I&nvJD~&M?R{6q36aB z>dXj)Xi?VPOSQ6;yhnlEI15vnW2QXb(m68I+)|l;x}hW2Gy|5elonSYD&Le1&qesTde+)u;2!wo`bd)S6`d;0Ms@72iI3kBn5B)GAm zx7|~kRjNYI{B%7h86j)`AfJWyW%3zlFJsT@>*R~j3i2!=D{qizp#32E1R=|RC!d7& zdAz#3NIp%-(yJtg_HpttLKc66cP?glzVJ_cw}o%u{R@}KrwEz<5&1Z@TX;VA2kf7F z2Jf7G6MJS~!FQXzhWE_;9Q$W_NIdsv)!rr_h4T%(r}jGb)E>u{{1)#gL;RfN30398 z$+_2leDSXO|N1fY|18`*;eh&Ik88nP!CdpW!tNOY7Z!~+#8O??ic^lT))Js_$^;W; z+9H|Xi+WAHu1|^43IYp7Aed*y;Bvdwwwd%=JYads1oO=hm=SwcGeWBd!gB-Ib-@nT zki1|?-x@S-MzG7<#xb%nNh=2;aYNWHpNNC>)o-aqVO+UQIZRqP@K}(*e&3A4LZI90 z+Zv{|11~5t%CJ%}`0-)#(9wiB+^@F>R-ve+1TRmZ%1F#)SocO`oMMWIn~C90eb*vk zYhl6nQbt*72E)z;h$w6g_qhd1OACHRUCJ^u3dgZ&zwl{kArR7o3d>BwT3;`G)?=rY zg&-9$`wBA+c4Vq{v^Q-eEC>-&js?CUL{VndreUCT>N{4EsRag^%7k)e24ba!x!B%1 ztuRnOk0npc`%Oq>m|0G>&S;4tWQBwp)r^DSKTKpcIa6y4EEB%unHdG+c8IcfrIs0D znF+~rGY~d^cYu(jF{#+tsMQ9y;4sCxIaEO6-3Jj0N(GBVqE!Zn{7kqml(>dDfXd;h z@*k?1XMxjN0}o{;UC&(HEs(iGMPtfUPKykZ1p#$WSi@nS!Cy|c>h0lfueEP8Qd(#T zia3>CW-cOO=hJH5vVmx!LC`P&WyV~9gGH@QGrDCT?pkK>Gr?1zqpEg%{WhA)*3Q$y zf-7@Z_~OLyJ4{lxW2Lo)q{s^Cqqb$R43+NF zZ9xWp>K7om8fY-BZVWnFMV@k6X+a?sK22d_87YvHkTUEIERuxQ4}`#K;)KE%G{WYV zt=no^M353R!oV{}pS?z?l>^~|4jK67NH`0PP^$(^peX5^)6tn~ghxx=_3vOMzy283 z)oX8HCBOC|ti4zNh4uF8E2Mz-3fAWq`vvyT9pL%wUzG~*5>jR#z&mHYho5cc zD)|yr^gqS^b5G&3wLhx%dDJOt7m!YVNbsxlPhdYD|M}&?56bm_?d$5R_+QgkOYQjk z0i`ZHo!$Jh765#a`yq$FF6r6IY2mop_-`ax8fI=L?WJk5`GS%eeAsDboR1Z<$&@pQ zJF@u|#S?iJGwzP#d@}EB18?(1MTV*?3AsDY>XXT*Rj0}3Z3T;Eo@CrLyY-avG}-*B zc4_9zjEix?oopDa`lr*U$>vLn#ysEk_~BYSnRX@uHQ9Vw+b0AEBK#>=EnAa`XRA=( zGbgK;l?N&hR34~2Pi^MysDJ94-+KRl0S>$?aR2}S diff --git a/flume-ng-sinks/flume-hive-sink/metastore_db/seg0/c600.dat b/flume-ng-sinks/flume-hive-sink/metastore_db/seg0/c600.dat deleted file mode 100644 index 83367f789f249b7cb94a069813adcb2198c5d29c..0000000000000000000000000000000000000000 GIT binary patch literal 0 HcmV?d00001 literal 98304 zcmeIw&uSA<6bIltsl+ja2?T)Fd6d(`%+D8RvHw+He~kaSeG*UJ z%}$T1rFvaAzPwXy{&97eoo&3@{yv)@&5vhavv~F)4q99Bo*yh(I}hUUe=lB=Ga^8M z009C72oNAZfB*pk1PBlyK!5-N0t5&UAV7cs0RjXF5FkK+009C72oNAZfB*pk1PBly zK!5-N0t5&UAV7cs0RjXF5FkK+009C72oNAZfB*pk1PBlyK!5-N0t5&UAV7cs0RjXF z5FkK+009C72oNAZfB*pk1PBlyK!5-N0t5&UAV7csf#WA|=f}5UIEcTisDIo2a5{Dy z^)4?bbGz7`XvgGg@ioudRl5>igqQJCoLvc>`fRfZy|6i%zA5{A{lR!tj{C#OWUn0W z9FEHUx8=^HoJkXrqa&)U<^m2~gXc%43(VNYp`A?=JK!5-N0t5&UAV7cs0RjXF5FkK+009C72oNAZ zfB*pk1PBlyK!5-N0t5&UAV7cs0RjXF5FkK+009C72oNAZfB*pk1PBlyK!5-N0t5&U zAV7cs0RjXF5FkK+009C72oNAZfB*pk1PBlyK!5-N0t5&UAV7cs0RjXF5FkK+009C7 z2oNAZfWU%4Yd72tp>?aO#m`hfr}{qChpBd}8os3ZG1cuy&)->M(| zG_7m(E-yuHbp!|yAV7cs0RjXF5FkK+009C72oNAZfB*pk1PBlyK!5-N0t5&UAV7cs z0RjXF5FkK+009C72oNAZfB*pk1PBlyK!5-N0t5&UAV7cs0RjXF5FkK+009C72oNAZ zfB*pk1PBlyK!5-N0t5&UAV7cs0RjXF5FkK+009C72oNAZfB*pk1PBlyK;XCu^j@FZ F{S98?ybJ&U diff --git a/flume-ng-sinks/flume-hive-sink/metastore_db/seg0/c611.dat b/flume-ng-sinks/flume-hive-sink/metastore_db/seg0/c611.dat deleted file mode 100644 index a8a8e86b5cbda5523623b1395c94dc0583997a81..0000000000000000000000000000000000000000 GIT binary patch literal 0 HcmV?d00001 literal 8192 zcmeI$t8Y_b7{~GVcM9$4y*<6(2XyPUZf#dCf*=qGBq0P*gFz62ge4#dh6Di#8WIEo zg+L$>gb)M*g&>3wG$9By0s`;v>E&ry;SXT%N%}oKpLgw9pJk%2**3<&K6rT2uyj7t z^?9?;e@%`*_^+3z?8;^P@Quk#e+c&G_WGYkY|(1#`Ol+cpL;KX&KG;aVD2UhUN86b zZ!B1GqsgB}9oLrm}|;LtJGqC60tjn36zJB|=FerQ%akNUKCCX=GGllnk;eaY`0} zN`ev~r;?=PkXK1j@+hdJDFqZ&GFH4WN-9}O1Z9rGOQcHl>J; z$}**dRh1P=8EYyXN(JjGtCT9bDr;6y#D>Z`rDQf$x|Fi%sccXxrmwO|shWXGk5a>y zN}p24P-Q@A;GoJDrHMl-LrRPPNA7SFa1(G7a1(G7a1(G7a1(G7*bNg1y)}U`p%>D} z(tFZd(rxK^>1pY4X-`_0##ryW^t1Gx^p*69^nvuY^qTa7^sMxRbSPbtrda%tewBWZ wzLq|fK9t^-UYA~yj-)50N2G0OAT_M{CjBIRBYh@)Bt2#>j{o+OUwtB4xk&iCYXO-4Yr`O+&wK#IKeye_c>)O3W{pyXaqL|FFHfw(}w|_lt zpW@~1r(d0qS{&*=9Q1Y`w_E+zg0sr&b9XqE4|y})p5o<){M{YON9X#V&i&s_`=|I{ z`{(@z0t5&UAV7cs0RjXF5FkK+009C72oNAZfB*pk1PBlyK!5-N0t5&UAV7cs0RjXF z5FkK+009C72oNAZfB*pk1PBlyK!5-N0t5&UAV7cs0RjXF5FkK+009C72oNAZfB*pk z1PBlyK!5-N0t5&UAV7cs0RjXF5FkK+009C72oNAZfB*pk1PJ_J0>!hq8`WZCIL7;E zr`zAp>&vk*l6djq_RAQg^~zXgGtcYeXf_qIr4<1J1PBlyK!5-N0t5&UAV7cs0RjXF z5FkK+009C72oNAZfB*pk1PBlyK!5-N0t5&UAV7cs0RjXF5FkK+009C72oNAZfB*pk z1PBlyK!5-N0t5&UAV7cs0RjXF5FkK+009C72oNAZfB*pk1PBlyK!5-N0t5&UAV7cs a0RjXF5FkK+009C72oN}E0xM50JpBoq1z+6& diff --git a/flume-ng-sinks/flume-hive-sink/metastore_db/seg0/c631.dat b/flume-ng-sinks/flume-hive-sink/metastore_db/seg0/c631.dat deleted file mode 100644 index 695438d365616889922343e8e5cdbc3086575b38..0000000000000000000000000000000000000000 GIT binary patch literal 0 HcmV?d00001 literal 8192 zcmeI0txm&G6vxl$wyxXywZXn2L69I2fI;y9NCawv2Ovlw!4M3RgrG=p2ow^5oI>yb z1i=Fk1W!N$=bW}X4J!yFIXCIQy}z^ew!f)+;1B?i4jzuAE18e1o@aFZnT~Jx$LSVT z_UZ5d;_OFAi_`qqQ7)?PTvl!qo_m&nrB^!PYHpE&`(;h)tPh&5K1&hPl?0C!gdy*r`P04l1Mah;VaKgLfi{zu^z2uGL2J9wp JJ-&$MzW@LXKFZIP2x7SQ^!t}1*k{} z3BjTdfDjMB6R=>(f+f5KNU&go!~$j#-=F~^?3Swi#-4Ne&sa16r{A90IqZfIqIgfX z#}C^1gLb?btdGay@$dcZFX6wPUXE9~hpV@eE;-$!@yQpG$vUD?L8_hjAmh!8Urumuog~FSPHGMbyr<-NiA%)d!4QQdbK-2{=;JuQhC=7U(<8X$ zVR%xwPr}Fl3*5n%E2*s2Y+*SQzw^u4DJU(6Gb5$cgQ%35f>LVknXzBz^rXMeQ@J7K za)n}krI0IDiluUHrC7$aV9@HZYIX+%1i(fXwM z$G?ZiU;NM0(|+ZuzXi@^M^A!(bASHdW9s_5oh!cvubQ})z|vQLhwj9iOu4#TGuWB3 zVq_Dr3D^W|0yY7gfK9+AU=y$j*aU0>Hi5l0fx8c`!(H>AY4rcT?>heM=&nz>wo3*O ziTW*rTP_%H1WbK2e(~skJX{AKhgUc=T;b#UCJyDt87x{a73j?X%2o)R-1rLz$Rc5unE`%Yyvg`o50?mK={4OI~RT_eImUt zy)E6AUXY%ZZb{qHhBVHGen`JaKS*CopGqG}??`V*FG|l#PfB~zd1;!(Pw6-5C+QpM tGwCDgJ?Ty9Woch}N_tG%lIEq3b>F3*rEjG#q>rU1+@--cpK2d}{s&wgS#AIT diff --git a/flume-ng-sinks/flume-hive-sink/metastore_db/seg0/c660.dat b/flume-ng-sinks/flume-hive-sink/metastore_db/seg0/c660.dat deleted file mode 100644 index d8d7104ab5cf8202b27b2c7de7d5e7407cac8bf7..0000000000000000000000000000000000000000 GIT binary patch literal 0 HcmV?d00001 literal 12288 zcmeI1y-LGS7>2*3B^K@AAZo0(f`ioG>ReF=2M3{og5csHh>5OgpRup zw_IHpuO;2kte27fi<7&gx9@V(!8RA)bRKr}N_c73cC`L!;nF85F@OOKU;qOczyJm? zfB_6(00aNaz}DVvTds7^>G^(J#`JE!@D?M5#)@jBPsYKsr_q_tD{>^q(vx7a@^!*) zOI5`)U;)ab@y(#6EFiHl>$ zj*cChxHy@Pj2%0=7=3xT{D?*WfaX2P`NH|UaDaz|gd=Z71ZE#^0c}XTnA!TGnft$| z+h6?8!+}|OZFX10n>`5T#m(`5x25afhA)0ip0#i;h1Qk1L2uzjmYiMB=?<5ySh5qa z6R;Do6R;Do6R;Do6R;Do6R;Do6R;DwT@(2D`90)~`AoC__xsW{M-Oj&%FYd$Kqzc3 z8GI0Tej#A#z4?=8|KlNj{Ft8M)pXS{w*m7;p2fWn2;HGa5Tys{QN-wX=rP3UA$lAMdYGO-lJ3%zNYNwo6g+yA?jcQ&(bLG#dc4SF8y^i_HRE&3Y0h&Fwl z?qh@AqL;8qZ_~@@&^P9e0=DRz^rCFjJ9J-m=v(xXbm`mlvh?UX^a}d)F1?CfdXHYi zeR`i>#{>E9%={LJ#AC# zj_X`&FSX~|Q|%}1JMDq?skW!}wK8ttO8Z^=Rr^JItbM6{u6?3yYrS#tQ#;aLXwS4K i+M)JPJJ9ZFn_72Vxz=84&$Xx8$8t3JnvWkfSN;Gwb6d0k diff --git a/flume-ng-sinks/flume-hive-sink/metastore_db/seg0/c680.dat b/flume-ng-sinks/flume-hive-sink/metastore_db/seg0/c680.dat deleted file mode 100644 index 0d8f4e36d2b465b0a1956ea29e2abfddcc0aa187..0000000000000000000000000000000000000000 GIT binary patch literal 0 HcmV?d00001 literal 98304 zcmeIuF^dyH6ae5iA(uSXv&Uf{hlZ=z$~9ic4+dZpxp$rZM93LSrwwZ^q?hdAoF_?b_S$iNl4~wI`>v z+cxfR*mG~8b^V?*MH2x61PBlyK!5-N0t5&UAV7cs0RjXF5FkK+009C72oNAZfB*pk z1PBlyK!5-N0t5&UAV7cs0RjXF5FkK+009C72oNAZfB*pk1PBlyK!5-N0t5&UAV7cs z0RjXF5FkK+009C72oNAZfB*pk1PBlyK!5-N0t5&UAV7cs0RjXF5FkK+009C72&@rE z$8j?vJ&jv=yj{ngDvs}QBadIIOnj=x7j^!ATyM7eb}jxynd45gj)(O;oUPg^wV{Xr z0RjXF5FkK+009C72oNAZfB*pk1PBlyK!5-N0t5&UAV7cs0RjXF5FkK+009C72oNAZ zfB*pk1PBlyK!5-N0t5&UAV7cs0RjXF5FkK+009C72oNAZfB*pk1PBlyK!5-N0t5&U uAV7cs0RjXF5FkK+009C72oNAZfB*pk1PBlyK!5-N0t5)0HG${H&&q!*E>IBw diff --git a/flume-ng-sinks/flume-hive-sink/metastore_db/seg0/c691.dat b/flume-ng-sinks/flume-hive-sink/metastore_db/seg0/c691.dat deleted file mode 100644 index d645a25567df1e37d63a5fc08924c9b005704689..0000000000000000000000000000000000000000 GIT binary patch literal 0 HcmV?d00001 literal 8192 zcmeI0yNeT16vodt6O+k%@_wxAu8%dkDrjXR*tv~`g<6PRP*%_c)9)}RQ9<%ZLlG?FRx z%HSSF^A`b2)!aOLjt8B>?sS9)QyQvcpcZnGF42`K8*`5+{G^tc3g_<$Kv_z6`@O-9 z-nHI#5kfbwZ5wXPfs4+?grW|hxWr=NX85IMwi{2oKs>GHzNTFt6&hbX5OKf}7 zXc(Axycsklw!K-j3@kX_9NH4w-aJ+eEIQr-_DXDfi&!-S>hwzr0Z2G$*K9fu^gy$x&_*j)6Yg2NJ9UR0@LU~AzvsL>G##cfc> zQA66!lnrc3Y){$5F#}f|ZwtpIw!Q7ikB_fD0iS?Rz$f4n@CoVILP%bjyvJKWD`9e&k1e9!th>xZmwv%bvwH0!q3!4Iv2kE~y_e#-hT z>+7tySZ}h9yf1cg}KM=YFU)eb$=ZvVO+;4Bec3mv7>iTfYI? C_0+Wh diff --git a/flume-ng-sinks/flume-hive-sink/metastore_db/seg0/c6a0.dat b/flume-ng-sinks/flume-hive-sink/metastore_db/seg0/c6a0.dat deleted file mode 100644 index 6ff147471cc4155494759477469f29bf66b0c192..0000000000000000000000000000000000000000 GIT binary patch literal 0 HcmV?d00001 literal 8192 zcmeI#u?>Py5P;!>kf()(iFOvo%GLoKfG5 z$^Qt)aR(fH9kLUVOn1L>i_T&*#D24;`J-N~a`Lp)&L(vG{xHIfi!00IagfB*srAbg-2d2p;`t1ldXuye~g zZ=VqRtw!PW`cZaymEA$}LKd==Qa2U?2q1s}0tg_000IagfB*sr{7PUhlQ8-;URO^i Cy%qQX diff --git a/flume-ng-sinks/flume-hive-sink/metastore_db/seg0/c6b1.dat b/flume-ng-sinks/flume-hive-sink/metastore_db/seg0/c6b1.dat deleted file mode 100644 index 1b9a62ea62b20a63b031c4a745e22725bc9dcf2c..0000000000000000000000000000000000000000 GIT binary patch literal 0 HcmV?d00001 literal 8192 zcmeI$p$@_@5P;$9mTZ$e0)}A0pm>N06wg3{0TLVriN#>hI9>*g1j`G+tzDOH41y+q zlik|8ZkK&-m6eDXy^0apnAjKwUk`ISf0m<>{G(}Xo9p`PXl;KQ-Mj9o z;9WM+zv;~kf+lWXE!v*hrj}!Y00IagfB*srAbhjGXH}e*;)Gb zt-jtYB&i-5Y$dA>0$OL)Q~p2S$k1F|HaVBoSnoyqiR3aZYe$BZ~pq|Dga)7$-LXC{qHF6#!8pK+FTQoC6|;lf-I$ z^v&O$5FKE%KuoAQ^%o$v0YnlP3AU4v#ke$&ijRiCXb6mkz-S1JhQMeDjE2By2#kin zXb6mkz-S1JhQJUCfn&lezcZ8q9SrMWGl5y`7+!$Mqe-x^0o6+~UPcJB9RRYK8A2FB z89<%=Q7{?;qaiRF0;3@?8UmvsFd71*Aut*OqaiRF0;3@?8UmvsFi;_2!ypf_IzHp| Fb^r@n76NN^Y=4#y2p9Dyrf&`3B1ytV7njX}`l zZ?d=cy>2i2+$t*(F?y9FvN5qS489)bbpEVHEB$eqYh|HVCerhpj5gQx*U{PjIDYiq zQ^C7zVsO`+83s+I% z?OT1lT}VZkmFzLAl+x@>Z(sQ~muYca| zzw0y0=4ibyzpf&Y$4$0zPgcqR=a1|1fV(>EbKmZkok6KV?M(2oNAZ zfB*pk1PBlyK!5-N0t5&UAV7cs0RjXF5FkK+009C72oNAZfB*pk1PBlyK!5-N0t5&U zAV7cs0RjXF5FkK+009C72oNAZfB*pk1PBlyK!5-N0t5&UAV7cs0RjXF5FkK+009C7 v2oNAZfB*pk1PBlyK!5-N0t5&UAV7cs0RjXF5FkL{4h1IZss5|!?qmG}{Gb*I diff --git a/flume-ng-sinks/flume-hive-sink/metastore_db/seg0/c6f1.dat b/flume-ng-sinks/flume-hive-sink/metastore_db/seg0/c6f1.dat deleted file mode 100644 index 785d62c5fc90e698a55fa93c1d408526403b63fa..0000000000000000000000000000000000000000 GIT binary patch literal 0 HcmV?d00001 literal 8192 zcmeI0F%H5o3`Lz*YCAF@W;Rw>m^cIj6K7xqBsLaC!~wVgN8ljLj4*NnaFV7)O?+L=yPn0irm=tk1Rwwb2tWV=5P$##AOHaf{3C(=abskqCkq$s#6|hr zSwp^=$Z~x$*hx`61hhugu5iycGBOvhW{)zP>TKK3^qZHttJ6eM8(e1|A`4&MJs)~67q>3& diff --git a/flume-ng-sinks/flume-hive-sink/metastore_db/seg0/c700.dat b/flume-ng-sinks/flume-hive-sink/metastore_db/seg0/c700.dat deleted file mode 100644 index 646e2f5b451dd35a05e287333ace6c87f19d7d19..0000000000000000000000000000000000000000 GIT binary patch literal 0 HcmV?d00001 literal 98304 zcmeIuu})M$6adgWE4!?O(u4&>LJYmO#)8t45GV)B*{b_xWVj()iljz2y;n^r}@<@ZZhFp4oewttD zosT{Y&rVBIiq)oRwe6Hxn`>x4C9OA2Yi*~br*jSWoU+k0t+$@?Y_6gGl!i=1fB*pk z1PBlyK!5-N0t5&UAV7cs0RjXF5FkK+009C72oNAZfB*pk1PBlyK!5-N0t5&UAV7cs z0RjXF5FkK+009C72oNAZfB*pk1PBlyK!5-N0t5&UAV7cs0RjXF5FkK+009C72oNAZ zfB*pk1PBlyK!5-N0t5&UAV7cs0RjXF5FkK+009F3Ng(aTc0_s^TX|lt;`mj?@vX#P zN<1m?+Y-Ny=k?WJk2ZhAILDoOA8*P!nC_Y-HKB+A0RjXF5FkK+009C72oNAZfB*pk z1PBlyK!5-N0t5&UAV7cs0RjXF5FkK+009C72oNAZfB*pk1PBlyK!5-N0t5&UAV7cs z0RjXF5FkK+009C72oNAZfB*pk1PBlyK!5-N0t5&UAV7cs0RjXF5FkK+009C72oNAZ afB*pk1PBlyK!5-N0t5&=G=bxb!|@F_l{qH> diff --git a/flume-ng-sinks/flume-hive-sink/metastore_db/seg0/c71.dat b/flume-ng-sinks/flume-hive-sink/metastore_db/seg0/c71.dat deleted file mode 100644 index e639c1cc40f4634e80579814fd9dd124be6e601f..0000000000000000000000000000000000000000 GIT binary patch literal 0 HcmV?d00001 literal 16384 zcmeHNOKclO82)xAe&qdVlIGQS`nXtfl2&zi#cS^-UBzA-uk9vCXi1y)1_)FNQYn4# zkPtVdUJ!?>5+`neDkPA&02d@AgajNafz%5ZKuFv;!2i!Wj-8}+CKW=eW~0omXZHX8 z`M;eR&-Z(A5r9Gee6-0jD$B-Vt-Z01uK%eW8>;X5u}Cc!=synht=(jhbNYPWGy|Fe&46a$ z;bP$POGhr_C5mZH-(P@9pVFJ6Tx;LZ&=w`>ZsIM(>zRNpuWDxNLf^oY@k6clh1yxr z*`%IS;NhuS-7~U|#QZOGv6dw;G1rDJbok4@m2)lM@o@x`siZts6UpRiGI4(O%1nW@=i5&}j~9fuw2Lo=Wm&Gy|Fe&46Y=GoTsJ3}^;40}mksALA57 z^%!0bWBYmd%kLZYzi|bJsbyhPw-96txkeeEWEnr5I5U$vpE!PYb)_cbk55gXl``J!KwKaKk+oIKjALjQoI48af5WFv1`?r|j54dC8;r z^TmSac;)Sf!(f-uCD4yv+bUZbN`(YPt7PT3k3k8MBj}b)+8HIY!|0Jro^q>BkSk;t zw{OiIzk*)Lg6AxiW$=xB2Cl6{>2>r8^q@ABSQGUc-PSgV-;- zJTA6RhJZPYU2-nL=kHk!=c7_P^{lREIr zmcp1OHiocUAcjb(kZa29wVa*f7?A_nS6ULxc8m&iqTRRECy&{OF&VD!&_dv4oq(oc zZu{#2g*%3EUM!=&Q?k|KjbTELdTF_Xz7JxLeBAz`bDCBmepsHBSL?$r?3L5F0fRNL$jg>)FoI4u;lV zq%-veQN82)kj^%*BFeGvN7|Ol$b7}iDg}QCX-A+PF}4g!6@|fPP9i-oKzkEAqaHPH zz4W0O&Gz0%X1F-;)1F^lZfarI~Q~3_&shmR&i2Mzk%Fp=M^2jwjLggGXRGMtn z_6<%_`7UNj^;R)Q*F|oP-$0hGyVzEI&1m8ct{3xog32hPnP0I-^A+|OAL8rS9mX+d z87KBW-_t%oDjxBl*l)YX9!--y8ecHJ@gmzA2{^>!AB>DoxfK`L7g>@OA6-ALe9)-> zaUJ`pdh-`iEB!Cgs1I9Y0`9H$AC7nlpcH?c z^?kVBkf2>ya7s$%gDmpHu~T)O(@M6zEb!^k$xiP;#SO5yC*sI{XQ`YhD|*tO<&2u} zjkAVlyQQY(_5K+k&MIDFy-rKS17uWRWyMZwM3o*CCGr|p_xro21jZ;1$#FvAs;j>P zm<-7#V@|~=E8c^(FJXPs3}^;41DXNNfM(!7W+2v&M=2xlEvxS64OZ!~oXi@$Ewcts zBSrNSd>y&Nx;=7}b$eumm3(AUW(w|dYuhbZuE_EL-zo0OjKEDs7OQL}j&W)J#vaWN uS;?Ev%bID%G5+HFjgPqAc#iW6Mp7#OKiL;P;d@B`b1tB+|0jQc@7%vK&DatE diff --git a/flume-ng-sinks/flume-hive-sink/metastore_db/seg0/c711.dat b/flume-ng-sinks/flume-hive-sink/metastore_db/seg0/c711.dat deleted file mode 100644 index e63d6e9b4057e7f16ee278c475b2de84906a865a..0000000000000000000000000000000000000000 GIT binary patch literal 0 HcmV?d00001 literal 8192 zcmeI0IcQX26ov13Z!-J7FG_~TeTN#1fzwWD2SNYxIn-ng<@qRb|SW7AFvcG z1+fyu#!9plY$g8xotbwoW)y7Ye=zTTb56clP7#&lo$>Ko z{Ku!~Tyxt!vLe;VAHlu3d;ZsBYX5d? zIB0VU9W1fvBm!DEWb;Ho8_O)Z6goI;a|&HqVbP<|lOr~#(8p00eF_5{vpI!X9A`10 zFo&VdDa`wS2;m3|gayI^VS%tfSRgDA76=Rc^8%?S(i2JDldd~oQ9i4DLbm0u{IlJ(JGH?{|tDGTp}k*DOh?2t@Q1#OGG!?2KnAH zfA7sOJ74-tDdlKIw939)_NQH4Kh4E_e~h#ApUZhvKC0@n7>3aXae2R3T#3f($D8f0 zI@I9J7rbRp|A0O5#XT6JO*7dVOryKhp^3QL&_Z;(fdQ0!|2?7KN5FkK+ z009C72oNAZfB*pk1PBlyK!5-N0t5&UAV7cs0RjXF5FkK+009C72oNAZfB*pk1PBly zK!5-N0t5&UAV7cs0RjXF5FkK+009C72oNAZfB*pk1PBlyK!5-N0t5&UAV7cs0RjXF t5FkK+009C72oNAZfB*pk1PBlyK!5-N0t5&UAn>0BCh4jAz4dDLasV=Z6!QQ8 diff --git a/flume-ng-sinks/flume-hive-sink/metastore_db/seg0/c731.dat b/flume-ng-sinks/flume-hive-sink/metastore_db/seg0/c731.dat deleted file mode 100644 index 85e7867903432ff4690b867a1bb2b60a08b1aa38..0000000000000000000000000000000000000000 GIT binary patch literal 0 HcmV?d00001 literal 8192 zcmeI$p$@_@5P;$9mTZ$e!V(0Q859zSL4e^IkU)aNAbA4bfVaV*k?<67YuBY4gP_UZ zWViON+hw0yWh){^uX02-CN_q_*TbC7pVerkKQ3olS?iUF^!z5H&2{~CbhbZ_9=Ure zc$ZBK?s_xBpo!bph_+|8Z{%1YfB*srAbQ6Vc9y<< ztFN~UNvcN%2g&M#fX-R{l>g5+QkbjDrbSha^BZ~pq|Dga)7$-LXC{qHF6#!8pK+FTQoC6|;lf-I$ z^v&O$5FKE%KuoAQ^%o$v0YnlP3AU4v#ke$&ijRiCXb6mkz-S1JhQMeDjE2By2#kin zXb6mkz-S1JhQJUCfn&lezcZ8q9SrMWGl5y`7+!$Mqe-x^0o6+~UPcJBI{?|t3?V=W z>g*#EM9zXi- zso-5UF}Ul^41*?aUnAO{*}joufdB#sAb z_N~6&E+nZQ85|_54+1)8^;7;o-^j=mE}PD(YOMFN{akWcRJEziiNrUTAR>SO0tg_0 Z00IagfB*srAb`Nn0ux!fy_eP1ya20$E<*qS diff --git a/flume-ng-sinks/flume-hive-sink/metastore_db/seg0/c760.dat b/flume-ng-sinks/flume-hive-sink/metastore_db/seg0/c760.dat deleted file mode 100644 index ca7d1c98807f49614d0bf1c4fe644924d0bccc44..0000000000000000000000000000000000000000 GIT binary patch literal 0 HcmV?d00001 literal 98304 zcmeIuJx&`z6ae72h8PQ>Kr9e}NXZQ%n>gY}6e&`sh$3i6L6D$80vBOPO_4sFC0F1G zNF0KOk`l)2wH9FlZiBwDcIM4{ewvvtenmuXezfB)_GoD5p~*UY}1rbMlkzCq!Ewecw>pE~$~J}CkO z2oNAZfB*pk1PBlyK!5-N0t5&UAV7cs0RjXF5FkK+009C72oNAZfB*pk1PBlyK!5-N z0t5&UAV7cs0RjXF5FkK+009C72oNAZfB*pk1PBlyK!5-N0t5&UAV7cs0RjXF5FkK+ z009C72oNAZfB*pk1PBlyK!5-N0t5&UAV7cs0RjXF5O{t9Ygd=Mah%?jm2Z0+FH+PT zdg=I3rbvv}Q|Cf5E0Y))w3{4&q}M6%~U;$qxH12S5?oOk$T!o)iXr|2oNAZfB*pk z1PBlyK!5-N0t5&UAV7cs0RjXF5FkK+009C72oNAZfB*pk1PBlyK!5-N0t5&UAV7cs z0RjXF5FkK+009C72oNAZfB*pk1PBlyK!5-N0t5&UAV7cs0RjXF5FkK+009C72oNAZ zfB*pk1PBlyK!5-N0t5&UAV7cs0RjXF5FqeB30y}fUH7b*{}l80VqTAT{by`#zyA`) zDXx{zY8C5DxvQ3}3Pl765FkK+009C72oNAZfB*pk1PBlyK!5-N0t5&UAV7cs0RjXF z5FkK+009C72oNAZfB*pk1PBlyK!5-N0t5&UAV7cs0RjXF5FkK+009C72oNAZfB*pk z1PBlyK!5-N0t5&UAV7cs0RjXF5FkK+009C72oNAZfB*pk1PBlyK!5-N0t5&UAVA=` K2~2k9FKz)OpGJ28 diff --git a/flume-ng-sinks/flume-hive-sink/metastore_db/seg0/c771.dat b/flume-ng-sinks/flume-hive-sink/metastore_db/seg0/c771.dat deleted file mode 100644 index cb59123de8ce1bac425013c90fe59fc1692cecb7..0000000000000000000000000000000000000000 GIT binary patch literal 0 HcmV?d00001 literal 8192 zcmeI0Jxc>Y5Qb;&V(#+&p_fm4ksxSq=?}1oA~qT@MKA_5QLs`G8^O}b)f(ZO0~;hwvleJ;EALW*5D0su7EpjpHyF^ph5TrCH}jt2a^c3sJ`$8C-xnItcjjY4mci&qr9o zXTR#~4@Yftx6DH7um+pMv2CgW*l%Wujy*ajKq!LOJ7{#et!AgU)oyfqD?9CaYp)5m z5z%GU5>*#ttlFX)!#JxEQH^7Q)hSU;V3O6Ss3tMRsw1i?OtYGhR4a-ZR+FOYV3yUC zsJfVAH7%+!%(I#i)i_SGnibUq&aj#j)g*eXdZL=b0;>g4O=FSOqNrxD#A-=WF%4x_ z%c7dW3ab@S&0>|+s;K7tf2L9s&;&FAO+XXS1T+CnKoige{+vK00yVR%eI-5;&xsqv s1)?>e^+J3go)OoHH6jd%ed0awgjgre4+tN`C*mD(8QT6*bNPPZ3ktdT4FCWD diff --git a/flume-ng-sinks/flume-hive-sink/metastore_db/seg0/c780.dat b/flume-ng-sinks/flume-hive-sink/metastore_db/seg0/c780.dat deleted file mode 100644 index c298dfd2b6a1efdcbf60a61c7c88486c1b013e08..0000000000000000000000000000000000000000 GIT binary patch literal 0 HcmV?d00001 literal 8192 zcmeI#F%E)25J1rZ4NzE^Xk%q*>j5}~g;rieZES2kiWl($R(b+YAccj_&a#o98%tyI zH(?lNfPvSoY(yl{!>!a}X0aJUzgg4#QLk1xdO6e1%6I$jG}NP~!wX-F#*gEd=lkA2 zVefKjec$Bj@~LmeqRsp^IQB4g1Q0*~0R#|0009ILKmY**{z%|BI8?IM4-Omb)Uy2T z7ec?)=w^EVC^@~#`A+kW%w-{Nv(G{R0R#|0009ILKmY**5I_KdUkQ{l_M^Au<@N+F CnHA>% diff --git a/flume-ng-sinks/flume-hive-sink/metastore_db/seg0/c791.dat b/flume-ng-sinks/flume-hive-sink/metastore_db/seg0/c791.dat deleted file mode 100644 index 352a9935d270d27331d4c0416c4d02aface3b126..0000000000000000000000000000000000000000 GIT binary patch literal 0 HcmV?d00001 literal 8192 zcmeI$p$@_@5P;$9mTZ$)91aTx#Y0S>cm^bp;AW5rG;hE|z*~^uknj|6YuBY4gP_UZ zWViON+hw0yWi28`uX02-CN_q_*TbC7pVerkKQ42vEcMDndVZ79=DPkmI@=#7kAZtC zc$ZBK?|L(%po!bph_+`|G;%BuKmY**5I_I{1Q0*~0R#~EM}gg8V`QaY=6|pwJ4@fb z)z{mFB-JB>tz`8%DA0lU(Lit*|+f_~sHs1Q0*~0R#|0 X009ILKmY**5cpYODhs!_f4-hxWDqYY diff --git a/flume-ng-sinks/flume-hive-sink/metastore_db/seg0/c7a0.dat b/flume-ng-sinks/flume-hive-sink/metastore_db/seg0/c7a0.dat deleted file mode 100644 index 9c5169f5fe5c39cf36d0c0357965dfba6c13e8ee..0000000000000000000000000000000000000000 GIT binary patch literal 0 HcmV?d00001 literal 98304 zcmeIvu})M$6b9fKmcSY*35uvt*a(7PX-xDH)Ig9>ZK5WdC?vR8SYcuBBUsp3_#$@3 z!dKA3N6@*mcQYHZj5XxDn>};p%ztzDejDC~5Muf^*LjLf7|QS_{{C& zbTSy8o-|AIxViN3ZnLRebCb?ip6&l0&QC_i!=D+SeoTX@&Ga1|Ojx^*((r!|o^YQL z0RjXF5FkK+009C72oNAZfB*pk1PBlyK!5-N0t5&UAV7cs0RjXF5FkK+009C72oNAZ zfB*pk1PBlyK!5-N0t5&UAV7cs0RjXF5FkK+009C72oNAZfB*pk1PBlyK!5-N0t5&U zAV7cs0RjXF5FkK+009C72oNAZfB*pk1PBlyK!5-N0t5(LJc0XvemxJz>Ahn6ZV$ud z)UC9;vdortyqlg$cP_^7Wj6aM^_{RCcG4#WmgH7F zxm=R>s>$-#=?D-YK!5-N0t5&UAV7cs0RjXF5FkK+009C72oNAZfB*pk1PBlyK!5-N z0t5&UAV7cs0RjXF5FkK+009C72oNAZfB*pk1PBlyK!5-N0t5&UAV7cs0RjXF5FkK+ z009C72oNAZfB*pk1PBlyK!5-N0t5&UAV7cs0RjXF5FkK+009C72oNA}E(PL`uo^=A zlI#0icXM6OwbRt_GuN-Ve#mt{*9T#F_|o5Q^*@K>)K9etPx89d?#dENt0O>w009C7 z2oNAZfB*pk1PBlyK!5-N0t5&UAV7cs0RjXF5FkK+009C72oNAZfB*pk1PBlyK!5-N z0t5&UAV7cs0RjXF5FkK+009C72oNAZfB*pk1PBlyK!5-N0t5&UAV7cs0RjXF5FkK+ p009C72oNAZfB*pk1PBlyK!5-N0t5&UAV7cs0Rk6IU~TWx!9Q|)cZC1| diff --git a/flume-ng-sinks/flume-hive-sink/metastore_db/seg0/c7b1.dat b/flume-ng-sinks/flume-hive-sink/metastore_db/seg0/c7b1.dat deleted file mode 100644 index c288cccf65da317adc3c152c784ede74a0e086c3..0000000000000000000000000000000000000000 GIT binary patch literal 0 HcmV?d00001 literal 8192 zcmeI$D{ND76vy%NYq$3H{eHh-Z0nwV3`h_Y2m}H#0STEa2}?%yNXA&g)C>+11cC&_ z1OkgeAP@)yBqSiv6Nm`}0)c>x|2=JY8di=b7^Yag1NbehwI}G zroTV9W*#5)jz5J&j`>dg-}lSZG|$eT z_?ELLR=Nv5zqw>^kDTKR0Ye`hzqtQ99@NI??yCPqZ`W(?J?4X4rE9d@+uP?gBJY9u zh<@+!bplY9(zkDJt#;N{cRSmwn;VUl&8_v$HhGPMBhsTu2LX{@O$HDYIikrRLLx^s z8A4d3Pm^IpMEW%uK~$up$tYqX1DcE>E;6XeI1(a5noJ-mGOWoYQX(UoOd%~Ys>w7m zB4e7&AS*JiNRJOWkqJ%ukr$cNq=SOUlqLfxicD)Vh?2;RCPOHT%xW@>ipZQMBdChZ zYch(O$bu$gI3=>E$v94nENL=zugjR}!8O=d7Da$J*H)J0Bc zGKVRVlbXz9T4Y_51QAWk=#->S7#1JmDwp diff --git a/flume-ng-sinks/flume-hive-sink/metastore_db/seg0/c7d1.dat b/flume-ng-sinks/flume-hive-sink/metastore_db/seg0/c7d1.dat deleted file mode 100644 index 8223520e0c45b8efa1bbc6df39208d3d7c33c575..0000000000000000000000000000000000000000 GIT binary patch literal 0 HcmV?d00001 literal 8192 zcmeI0yK59t6o>C`f;*3$ot<4{5)%`nu1PjgS1qgr8(Tq95YpI6v=NM8Cx{4wV3SI* z5V5hb)I!k4#!3XSwGgaqENlclk4(<&`uGFnT$pd}J>Qwf{@CVDxE4YH9XujoLs&=U z>ouDH+#Dn2zb;Rcxk^V8LRmF|9!~3jj^AI1+V4ExxTIIr?j?oAFLc7;+C#K&i1N(j z#!$@{o`5Id33vjYfG6Mycmke)C*TQq0-nJCn84lp*Adp~msR`wvtgN5v~wOA{gHLG zY{c}*;8sYsJ_P)6R&{Uf^AQI4pw;-@rsYo$5xvN{u&R~DJe~Q)qE~c|quSMDv_=vk z$Rss2#+3w_Rkm>@MN^wovvYrO_WGJkXaHWa1HfJbJ<*agaRORTJDp<9!T~!O? zrb<##13N^~WWrRJRg6hhF{Vt7vr4d2Rf1_#6Rc9qs7kTR)D)`>v#K)8nVM$R#BNnh z%$u5Jm1B>p9D7amShcWERSP{+3#Cd(j+929RvupC`oM2VVdjN)I5#kf(l2uc#|p8M888E8zzmoHGhhbH zfEh3YX21-Xf&XQobKXX1(KGeG?`NS*XOrKavh=MI7d)Sq3{D|P76R0xq`Ci{k1&sq zanKXEnEEot7)Cg*HI5U* zJXC&q85QOcR<+(LvjR?NY=T)a{-ar%0W)9*%zzm%17^Ssm;p0j2L8%`a{<%zMO)!? zc43O<>2f;H{O#@BNuEirNiIrOIPp!gCwVJ*CE1bOm#j%Hae`bAH*$O_xeN`e)7ZSe GKluP|Nq#E; diff --git a/flume-ng-sinks/flume-hive-sink/metastore_db/seg0/c7f1.dat b/flume-ng-sinks/flume-hive-sink/metastore_db/seg0/c7f1.dat deleted file mode 100644 index 02c55e3803673f1dbc044851b2b68d0426c68634..0000000000000000000000000000000000000000 GIT binary patch literal 0 HcmV?d00001 literal 8192 zcmeI0y)Fb%6vxk*WoCBgbC><3P)HOaR4NZZr%>%9phTh(jYcBTsB|KFiGoh0@BpIl z0HToSC7g3+XV0ufp_Frz`QN#}duQ&Q-&FU)4gersTpUSPvKV=Pkp}IsN?sW(_l72F!pNFau`5 z4445kU|f_7ZW8R6Ax1*5fdM)N@6ButSL#DRB%j5$|S&XB~>ON)|J$m zMA%T$U=m|fNs~!}EhQ}?7(-i0n@NHlB^@RyPAHjRQl%eY#SEAMGhhbHfEh3YX21-X z0W)9*{%F97p+ZIN??DN%CIuTJl12 N3D&cPugqQEd;;a-K!X4P diff --git a/flume-ng-sinks/flume-hive-sink/metastore_db/seg0/c801.dat b/flume-ng-sinks/flume-hive-sink/metastore_db/seg0/c801.dat deleted file mode 100644 index f5c89e632a2285634e084dc7b995fb617f3d1abd..0000000000000000000000000000000000000000 GIT binary patch literal 0 HcmV?d00001 literal 8192 zcmeI1O^B6M9LCS{&i#1r_xIeHJL8PsAJwGU$CQqFEI%uCtg%un7Y2ghLP(=lMMQ1d z)uPvKJ)Z{NKF`yX$o-O3y8l^Rp;hu{u(_y2ko=UW}Wcx#>dbM~UK z;t%(RHFFM`XP#%K@659jt=K?pAT|&ihz-OBVgs>(*g$L`HV_+#4g9|v_~_(YFvs1` z)c=0}x!LFL4&L)q?!CuRBbRrN4Bj{C?1O-L)$DNpdp^u+{G@K$KUy9=B<_REnEiM6 zw(eb4lW*O7KAC+=05jn3kp@nRLc*mKMHP9Jd?t~p0?L6?M8!J;k}098<0aHo6j4`E zK*NVK5-y_|I2E+Kla)*r?ZBy_6F7AY22KNAJ+y)$6=e+jOip%gq8B(VjA*!pQ58)r z@R__!wJ{br9gJ(ZgM}*ESmZM$H?trC1&~1+36;0V`Fc zSmiSnnJVJGz$xK=4VUnMiXtX_rYciqJQz3?tk!S^YgCl+kk8a)s*1INQ^Ug=uHg|C zRXpl5b(yN;vA}6yGH{xBT*n)DLPZnnRMfHFha0kC3sZs9#)iP@;7J{CW21@=tX0*5 z-Q?pP*>Vu}sle*OKCOADHmh>6Ej~6VlS8mu18W#|8}q2aZdX-@-Jz-h`;4k8?6ZEf zE4%ezpW~vaovPe+yHvTecKd-LHvspErUR>M>>h6oOKQm2=L2il*cSq;XY5R1jlk~J z59>;JQI$LLB_Hd_{-dxj2i5}EeVTV_zbY3y;A0~)IR^VmV2#7Rs(GhgQ#I!Pw#6$p z5F3aM#0Fvmv4PmYf6+kZka7P&%^W4)6b{@V|0d6yf^*K1-_UuT`T%*9Y>}hFDGhfIV6Ka-cp&rH)fACf1?9pogr zjD#@xll-3iiu{Cpk32~3Bv+ALA#Re_$gjz>sc;Liby@Y&V(I&dDx zH2Hns`4-OL)o)uyS-%(9;)Y%yUm1J?gWE3xw$0tvO@HTi;8bUG;=MfirY7iJz{~L3 z#PbGw@TJ7pnCW+69gbT;Ns2%SCxXhwaiB|Q@ls^?m&-&5Sb$C+2D3Z?jr;(HvwZ2} zhp;rmE9K*tU^L5DK7It_8D1woehkaAytn_$u#@>hNR2UdRDT>emt)QrB&T>9DNP;E zov>@Be#3lI{CC013?D|m{;$C9Sw8piyJ61^FL*u3K!#NuWO+=Nll2hR{oez7x9~bB z70#TEHBHBz>t`?QWB$(g6hei8v4W;!5+7C9zYq2^Us+jB)@P}MD3U4&vxJ79lseOS zh^4eX_rvPl^;b3ssq1-}X{sNSOmJ;m;#c7S^SL!RB8lr+3ZzP08M!UvDK9GY6Mp~> zGOuh@;23a1oB4xqsOCi&BaiWK<`2PP=1ZX~BP4b}$}rGY6}Wy}6;zl?6^FTsxj%>D z2=j@R0@oW-QpunQOBIxHVkZ4Jx{M`{_Xr%_o|i$Us&ET`6xNv6nM(>$QvXdJ#4?Yt z5l!*OX?gGR|>Zc=<7invW|dWaj0ULDqbk;}*)j{7z8JM`0m_EWG?K&^7Ob7Seh7 z6)?=l#)i0kz5H$nYrZfT|H{knfrxo)Od=E=XwCDr7vh>PY%Y}a^83ItFQPKDDyRM4 zv|qaWAz?m|mBmBE%df)Y%u8n?tDKiV08cQVS!b*&z5GEq&AhG@&Y1F!_Ygc;_s8VU zXr1}Sdl;T#UM1F99pe48IX*|=Y38krW1;=~>nNlt3*ss(Z0PXBY;NElNS|SW5lM{a zG9H=Df_ou-mIaYBs)!WbVj8FT7^Kg!K-*HA$arIPAEakkkXNp<5#1%4Teu(6=UHG& zlZhzD$AD(R1CX9&L1yEqh$_6JHVYnv^aU0aCK3hCgimlB(id4^jKrhYdn4$sb<*!x zpiEZUGMY}D5>=qx^iJ=J#A#N2Fi)@>N0P=Sw%yhP@=@iG2bY14lZSU3y#pV-BS>?i;U zFn$#jz`_OcbZ(JdSS7pkC6z90P=5R=F@u}z|C{`a73CMdAUn9i{u|^UtW&*mt|Bjnv$A41kkn7)3`>$~6IkE$`JJu-=r{S%7ey)b+|M(Z#FW&qB diff --git a/flume-ng-sinks/flume-hive-sink/metastore_db/seg0/c811.dat b/flume-ng-sinks/flume-hive-sink/metastore_db/seg0/c811.dat deleted file mode 100644 index 40992c68a47cca5d0c45542316693c4d93247c6d..0000000000000000000000000000000000000000 GIT binary patch literal 0 HcmV?d00001 literal 8192 zcmeI1OKeqD6o&VIdLQS$-@Wba%STIFptRIS6{>6$5k#P1Q2~_!CMG5fNCcxJjWIfL zrUP|g;y?x(qXQE)Y8)DkabRLh#6%~C5e}F*A+p!MXjcQ(nG>^5a=-JR@BI6ob60Mf z+*5bfIfw9)Eg^H{TCAL3>q7qf>UNF($HS#?=9%zHovW{u5H22$|8*;_cXsTH%iGki zc`u5|3t@-3H5XZDJ+GO*u+EvN;soLZ;soLZ;soLZ;soLZ;soLZ;soLZ;spL*6L{_I z=ittS&$ROU{zG>(9G$rCQy#g_sL18RBZHS+y80ks-L$%R<##^ZEPhg(Hm;tPM-NH3 zk!5#GP0okC+*tB;*yoegrvz|);TWlzL`;f8!c!@V8uBPvr%yTsl+9$s6j3pg6;nc0 zmzPk}P()or0SznXq+CYROkPX{EiV00Lm1bP z!-RF3(n+zwOiN4wlUgodN<)fi>$Ig)#6~k6F(qu$atWI?6ft9+e(994#Y|UB1+!YN zU`|6BTdgx7ohs(d42r4Y7A@CstA;9WvrbPsb=+=dNK69@W`@NyafdE%;7$!qY|~K3 zb}NraxrIeDqhi{)%gmUV4(`_FZS2s{f%lqP@H?$OF7cZcv{g8HP3bp&J zwn1tG@ViY-iW-E!pZ)OV^?Nkc;rD84z(1g=3ct^mPRY_9{DV9xX}_lMb_X~sl4fegd7k`~e2-joPwV&Wz&jro FuKvFDn;?Kp+qZLI|QE1c4v~2}?i_3<&}fG$aVb z6as-jEd+r;AqXJQgdore5_q3q*Zvw-_ygc=(r??(S=aViCVI@4F$VU^izf|A=QBM% zZ}$Gr-OF$M$A_owk!$wq86E|fq!d(`;TtGT(_TT`u%=v2KLdBJ)g3> zMl&2m%`$o>1@kD|b?JeJ!_tM^myxCa- zn1Fo@U&W*N2&)7rVMJ8+Q6h+{?59K#QwdUHh^vGsaU@iHN&-ohFeQnUN`#U^S|v(J zBcl?dWRO*fQ?kgZBq%xLRg#oE3MwfpUJyl5ClEjiqg_6RuN|lnv0hJmhgM%t{N)|1Z1|^3Tl_n*Rwn~dq zKu2YTQpBoCn^MA>N{3R$y2>i0g09M%6%??cvQ8HH=h-lsXQnY*HFHtTLiB?Z1$_xCPt-ZUMJ|Tfi;g7H|u=1^$Z#0&h*mn81$o zsq~@ru5?R!NqR}q_?D3rDN$y=}~D*nvojTe3O2XzLCC=ZcC4uE0aHd@5A{EzW{MWSs4HT diff --git a/flume-ng-sinks/flume-hive-sink/metastore_db/seg0/c831.dat b/flume-ng-sinks/flume-hive-sink/metastore_db/seg0/c831.dat deleted file mode 100644 index 37fd871fded433033709ed32397858f326754749..0000000000000000000000000000000000000000 GIT binary patch literal 0 HcmV?d00001 literal 8192 zcmeI0J#5oZ6ov0`z;T?!v7P_4&;l)iQho;p#K3?A0|Q8ip*s>njI_`j@ z%nS?+3>5u|t||s*1`xb^-Q-2LfgN<8X!CO*iw*b!&`ji<{ z)_^r&4Oj!#fHhzZSOeC8HDC=`1J=O*)xh(Yk3g&LJB{A&Z_v=4o!<8?7x$S$gk`s6 z@RXwQLcr9c@x`O}e9&2ZDQt9pZc+=oaf>~u6IOI*rnG}bW$uQ?l#+e-R z$~1Arn>J@!II2t=$Gquqrh^4#x>)q4=Q3#GxH2sodei4jn@%Xx!AWlhoatgonI2Ag zGsBramX#T}zpM>wz#6aytO0Al8n6be0c*e-_y-y&U7?()bVfKUjG4iA;f`=q_)+*) z_)7Rp_*i&Pct?0uctJQXOqlo~{3_fMeiFVDz81a^J`p|;t_ZISFA5igxsaIjO}H)m TEPO9~BfL!ai=QI9cK6nAS$)0w(HWtAi}B&T`R`|0R#|0009ILKmY**5I_Kde-tKXEG0&{G@l+IJ;%49qP=?X=9gs|QzpCi4)<&-!$QGdH0jfB*srAbBfdB#sAbW&^<`-x0LCco<4HO|f&7ziMM00Iag ZfB*srAb5t2iR_1zHkyd=jXmeeE9gXb|{d+s? zO5WuYotxfFFKH5XMR?^ zeQT`O3wfbO1{<-}K|tfI+7gmfhK}+1=UikNd!J&arnC3Q;H&N|B4=YK4fBODKdwas`P(Cs8QW z3WY);3WZukp%x)gD+urZ+1q_L|63ZJ{NH4LGxK|&*_nB=&Ae{QIS0SVh{!BCjq&`n z^T)q-hSGmLJ@03(`7L&?Iu60#+@Jq-6lTpXy!?4;M^JsGM_&XnN4ev)bzOyqcOAnEMkPWvvGCA4FgGI0WJ{L_^3E(pR9I-Mn z^1(!s$)li50mWc4k}0C3ObO*+_DH6TM41GYVD?I;f~qoA)Pl)MriN6R6!lm+VWD4jhQ$#43!yREttM!=CH2Jy!`)dSOeC8HDC=`1J-~wU=3IU*1*3v zklA*rbD1sj5_y3uFBs&uX;NAD2_DEcLP?z4(yP=DPkmI@_N|k235^ z-sKbhyWY$&X%e1mMBmeH8#xvTAb(B(NHo06)jUR1v9`DaJJ*f2@i=D z{g!gE@8X>Db+K$j#OUQl=40+-8e%=o>HJZTPWt0=u9ZqJE7FS(8EvlXuj6g|^Z4Em zyOMYL#NehkGfJ9-=UUPCw7XW01p){lfB*srAbT9cIIcr z%eTgQw~!ZlWUv?890a_bHM`<}zLBB1hO9l+Rd@8@+Am}oGR0Kyu5otJz(4>21Q0*~ Z0R#|0009ILKmdW?1SYZy8>(4ZJ^_e;Eu#Pc diff --git a/flume-ng-sinks/flume-hive-sink/metastore_db/seg0/c8b1.dat b/flume-ng-sinks/flume-hive-sink/metastore_db/seg0/c8b1.dat deleted file mode 100644 index acbc48a97704daa333e9c8bdceb954a3ae848ea6..0000000000000000000000000000000000000000 GIT binary patch literal 0 HcmV?d00001 literal 8192 zcmeI$Ee^sk6u|M=iX)(b1cyQ5aNGdL4LAj707JL|yteDojgJIN z{!RAQzPIjWKex(KM2uc;WIpCTrXkkToX#KRXrwTlg??H5!Or}w zc=^^?Zx-@Gj|?_qtAl{IvuaoT&o|OF=a99>vg(c=T>FVkL#7z&-8Ifm8W;#5fB*sr aAb-@Cs8PXJ+l^tQqE20f9L-0ow;{@Q{4;O0DyFHaU@;IV&wfrM)x1R@&*6cT&A8) zx_p2%AA~eHwSO(c3!>o%=O^pgQ-ya4n0cWJ=L?f$;|5aZ^!EmsHOzn+Fau`54445k zUaN*pFWmX(y4l(C{@gh>ShC8JCN z43)S{LX4DnOd^bx1VkJc6D1)N4^t%(6CbNeVkTv*DM^@Aa7;=w$ur3#$z90}$*$y*c@=o$f@k4NE}>(D_}SS2+jbn?Yea1Bf*k? zlfAX?t$W$et+Esmqn8_*kGYR&i1jq5^G7vW>5t2?R_1zHkxqQbXmeeE9dFwohPQ6m zmAuO*dRM)fLDD2V*NDESEgLx&2q1s}0tg_000IagfB*sr{G-5Tw=%NOFN;6enV%Ie z-x}-fLSE>R!CGv65b$=&z;L}{KtoKl`p{1EJm+sA*s#O57sJ^yiZ>@&Yhp!3<@U@-F{bDl4+ z>0g_3#F1OTE#MY#3%CW`0&W4ffLp*V;1+NTxCQ>L1#aEF0(04ZrpfpFk?Gqaz#=MvIHsVds z62OG)WB4i_#Ya>nM2RA%vWF5wTxBmMj)Y2>l0Z@=LP;W};!{#ct3)YjWK?3546-V5 zN)~}if)XI7lBDF2S4mOwD5#_<1r${>R=hAuDp^VdWtD*9qoR_dL{U}AQ(~y86ew}j zRf?1Z8Y(495>1sdC54tsg_6dCN|lnqK9w3Ji~TBfN`M0@4N4Ael_n*RMU@t%fF+eS zrHGEoBBg|7l_g3UD=HmI1*l(OlmtWheaud+_5nt@7> zQp1KypHjzAWk6}*pvnfNi9;$wO3VHWxrHS_(rxK0=@aQa=}qZX>3Qi{=?Upjx*$!l_%8h- xeJ_12eJXt*y)E66UX+fcC#6TEZD}AitobVaAblf!CVeP9W-g5X_*>qi?O$)DT9^O; diff --git a/flume-ng-sinks/flume-hive-sink/metastore_db/seg0/c8f0.dat b/flume-ng-sinks/flume-hive-sink/metastore_db/seg0/c8f0.dat deleted file mode 100644 index 4bc03d265892852a4c3b6c7a41b1946e72b8dc63..0000000000000000000000000000000000000000 GIT binary patch literal 0 HcmV?d00001 literal 8192 zcmeI#F%E(-6oBCeG05OxB1bS`b=R190vvGi8tNT9ij%kS0#4k((apg@Ut6jKY@8fS z{x&=w??DUShU`Qn(!-3{#>B>E@cm{@^G7{e<>2K~I~&&>np397KsQfaEf_y;zpn0k z_k>-`r1gFI#pDy$3`M)=w~a#&Q%3*+1Q0*~0R#|0009ILK;VxAPNTz3w)(>U2RpV| zzw-&c-)i&{y?+{=-{thGc_yV4vU<%zugcCeeFg#uAbthDFc<K)1Ri+gd*I1;R=y%`gWqBD`zyrb;V)7F>_5-^+Uu*O35yHC$OR&{zuLCD39sA>$x4cE1>Dr^G+V2U>m_(Zg6|Nq+Q!COquK&Y zt3^m`L!|lOX^~uQbn5NqW_N=;6cxM1V#$b$tBxGIX=ndve|YnF=V*LvaL3MJ|9JnF zBRB1A9v+QvUmf2%Ja*{l@gv7H&Q!iOP^%(GEb{BRB+|_eIK0zsSMl0mbBwuo16`uL zvBbBqLoC@ZsA`yTpN4sc-2iqU_ue$qW7CpT)O(F;OjZ6IyM?jm z6f^PA~(t8MaW+}F&- zd^0c(PyHJBS+biae2rXUGm^`BcJ$dO<=4v}&4-3wPC1rNFMkXlAKDZ6XYWS4c|*0^ z+-RxxwBSENTxvJ6?dcyGO3blYS@84GWqkOVk<%|(d@LWC27Q%OSXY4IoX7EPL`B#Z z`@9FA%lWXgZ8G8%AutrF2=RD6bWB8`w{|HgI7S}CRpaFqd|dFhHD1;?wx-fJ&oSpp zn~nU+X>W>G@!_4bM86h!wcUVkPO;k9Y(UE2>N0!M6Zq&54J`EC$SWxC4~bnj$zkql z*CBazd~|$xgrnZ>LqGwJ35`>+wcu+qEYsi2a%n}p8|IO}DxLx5b44G@i$qU|^!vmJ z%AU9$%CLBv5UIC|15h^cc=Er+4N%^VXC|L0#z2SvN&GI9NAb)=3XjD9THFNXwOGc! zDh5z)iMxb|-Y4DxWdn~yzJuS2ya=sF9*;-DpGJi6VR0jrXCVH<-=VFAF4isdKX^`P zw|E6{lf&ewJ%8de!cCBl!##_n+Ks|gaM_=b~aKXSjZ4YC9=#%7?h0E2Nv{f@ONZnw)BuB2NpD>!B4I3uh!S9osKmk zhQ(um7kpQw*FajibVk|`CoD!!>I2ITkXGp(Yix8Kz|=rPY5^A;tKQkH)_H>%=>#ln zAbp*Koi3V_V%I?#ogeEOovXJT>fhQtynRG) z&cS-Y%SKR0Y+zLna=p%Gqu$)ztXs_^;h#;~^@f7%k*>1iOuFZnNZQo~A7qG5`yli!-gXbCt+7_1gup>yYeN zPjKy6?eX^b*!b{pJgOhL<+f)ZzE$X)E4vvKlf=iaM-&IA-T!u>!3Tc6OxCV5_!xi- zysvR^qS-Q&4ZFnPV?heoYmIKBy|qSh-m&@HI+4vyg&yDbP1)0xl*l zhBWN4Aon*RyXjWfH=%7`0{MG?t3R1SIO6X9#o7QxqU&>(D6MTSYGi^VrZ`Tt1p|Q?Z zUuDQnb|JwB9oo;HMhoXaC#qlASro+psRUlq5~dTuVJ25xE>tooyeQ(7gBXUhPVMF% zm?dCRCp0t9Yw1p8+Dn&9=Ko2l0z3=(^Ws_JATa-TNEKi&a_rQ%@qOw}Wb)zX;dd9lF8&hA=OMzvrC8h0$7Ok=xEbCXXx;c5S*}U`|7!6r@WeHU zhOfIH_2U;$@&7Afj$Vjm`JuhQ4(RZL`G3j(S=SA6Q+3^}md6++NCRLQnC&utql6P# zcWHBw%E7FKY|@huh;zYV*T{(OQjyV#ySo^Iaig&xBsfVB{04p};_iYADa-C=8-pQ4 za0%|GuR2k87jXL+8gr9_6(MzE>MpQTa|?pbOOiTq`;l{xeGW1)@mJtstrEn1MkGIUEp2WO8VEtR&25Wk#<(6hia>+ zGDGMVJqP=VbVM|sWecc_N$8*_ZLCppwB)}Wgc$rBHd>7Xva_DyZ-aO|%LdHELgj5g z#R|wfTm%(tv1$PaF?p^X_yJCK&h)oKlzzLC9jo&h{GbS* z603pW%v&u+44wt-u+YLU4pV4OhgqS%&P9-*2=cg3W@RVoy<* zwr>mmM6X%UkR*T*VrkIsUie?1Tb3o{*vT*B`{eUvc?oj(!~=LdahZ4{u>A)RCw?<>^7xsk z5yak%tUlJlbD}@MPou9yr6BrbFFu+WX0qq2rzLaWT(#=o;C_c@h$GnPM6GIn3gC^m7bT8(ZCG3l{b8$750ySO)a_Iw=ZT z;QYQ$x&jwm&Y>&5;5QzVvcN)y&@FQm+w`knR!r)`IFsKlWU`P^WwXWQd?h!86(ghV z$AXN7V$u{ySKW!-fU@HPGb5j3qW$39S|*K-sYED5xZZhWHYd^2pUH=mt7tf z$GPQkd8MH8@IroE&Q^p>3a{JJzBzye4oQBeLEx5Xz-Tf==2;eWRa~O;ENED>ZC3s_ zeP;lxzToWoRvT{fu&tGkI8UV@Wm8irq!X|V5M6EC;G@p)*oPE6sRJx{s6X_c=K9vU z?^=JG0n!QB2CCOj8*o|WZ!1d>=Cfd-&1}@33Y^ngK!^yL&gna`XDH+6f%9xS_*^Za&}OHF)5GYEWAd;e!jmV#o6xlt)7*)_}mL?5YJ~p z<4iC>Pwp9T6=#Vj8$|P2*vP^OyWLf*AbjuRHHhW2@Iiw|d|YbL+uML!lxi@&D1Glc zS#A*BXPJTCgl4utYLVdn&C(#guYrdJ*G#?!MfLb zP<`*VV;lO=&gpn99Y*H7v_o?Kcgb=euA)yrglqECFU2|bbOE_^>Pz@(>JB_VbupI7 zkIHg{pC-d{<-fc;AhCgG#(yZwyHQ7oKUvQFe-1f(>u}9T=n0VEy9j=ILsQG`60o7itAm+w4 zu)$CWHq;UT4m!%)_1-4*)X557H_8DPLUgaWL6kjPF69cD0o?89(cU4P)%6;@L42Ks zj>+U4{wLDTvS8L<=K>IIXTjrrvh!R4;_u~3wm8hd2<3AHS{N={=K4CrFC8|BzmKxT zl}ZNh^YFO7Aj#}q>*2lOJ;3LWh{WY}O~9Shriqa6uOLd#G6B5|e#ImzyNrt1$%ru+ zMCn=Jz=;8TwWbC_-_e8!JqudZ+iIL{BcUxi5SeG8L+^r$Pe%v0EX@`jh}N?#K=U_S z&m`gT%hPDGo`p}pK($aNh#%fPx zR7vOdZ)>77YtJ$Pc|QCQE1^octYpAJQVlXh$Th&Nd@Ah^PISDO8}%!hAuvFX_yaX> zt?9%zV&eT_z5>`NI=N^0&~fY6neoOyY7zx!AtOn?PCIQN7S1vNfdLH2sR@P6x?X#$)l;|Px!s9otGi<4#e5&gan`dp zLET$`&rd@H5ps}*N103x7EF6|RnYrkoT{#8WotLYMX++?Jh*)WZY*4*?WrQz)UoCv z$(sdH^3qBPGAN|?8sN?&IAEW6Jz*0i*TT-pio2;!6ORj`<678x&k$h4CJwHJU69z0 zk`n@}xqPDETEImK?AY&Zz{I?@fJ+kCvD@2#pCsq-wSdbK*fD#t0WVA7e1Ehu%#3lm z({ub;t(k$ucl^QjJ^NaEHO`+eRdMcp@q4m-sVqxaroSRN{|jXKSS(YYKz5zFS(XXp z@X5E~`{YxRr6+zM=loxROgoXsBk?av=6_r={|oWc*hdg2*2izf!dOP%Acj!>wp{c7 zHN=cOPwoP6p=ACaLWFQ1F~jow1$q9$@8GvW-^5Qt&zJ8Xjo&psgx@lrfyc!!@IB1` z<9loV|EG~X@1NrTSBWd(bM5v2tV;xXG;aGQue?X+JwpWsBK|C7++CuE#7ZvE?`^f4 z(ul`EZWDRytQwjBe2I0tQGIA!IDsZcp}E1=@(ZD*_^itOuSe5$VBur+u3{}oA*cV@R5n- zvyj2>)%|;Vu(+lNx1p&Ei}M0!iRkLa7EVG4QNU}8gMpfNB&yHFr-eI{9_4mj(FoBr zL2O^sfYqhkLW0czk$si{NW3AU`NjhqI5B+|_@l&muoG!-6*g0VoUgaHxo2;0(gQqH z3!(PUB-+oib-p;)ZfkAhfX$Rw!mmsb_h*>`g@GE+>@T7jVXwAD^q*w~)(|Kw)4zpe z3@;-j1F(#M)+3dYYOA_-@CG&gBnt$9rp9J#VXPVSQcRL zM+>{FYn=wcpWzEn;s6UCMo@%@ok-UReTOr;trd|zpo89RZ>hSU>5Oi34QT`{bReqT z+L}26o#hKn;s6U8ye3D5eIOu(`N^_ToYCR!fULQPo; z5(-$*5MJ7FhX`yLsP;6X?7_3R?P&Oah&@*{Dy8@^ALdk|sa`)GIJWwOjj z&i`5bG;|bgh0ep{#(VKhV?UO_|KSMC|AUpu{Qut1oxgHA_&@7nLQKSPZY8yGmxv1i z#tdDg0qCGx`MAFzs~#J`LWk8lxPceU+iq52_sZ3twWmnZ^ETtjo+4@;04G+O#$fMU zcwPWx=$5e*)0kQoFdk+L*^+l)p6X)$aak<~U?NYLsml~JUTc-Ss+9m7WSBf?wl=rA zz16a7X19et@amk}$x+zerpF_#JruL1Pv6 zJKUC2WefV9DUHbyz=PnK5(lXSEOZ!sF~O{D)%P`2eoVB^ za||G{fMozK1>DCnbG1EUhxr>wFJPIN&DLC|q-~9rw1Pr0Uxu4Dyr)|(KjgCXG!W`)p4XQYg2Eub0(T92`$so3Bt?J)4RK-< zltb*txZ%8P{T|*iTMnrKEDI=5q6N$OZea+!uF{!E2hf7IMl}zW$;{qpK>!Qh;o4Gq z=y`2P0Vrn61GuN$``T)@eIouWWK8xucZ3_#U4fWXnJSw@Q|%Flv! za62R6C(>UmZ;vwBzPH=wm7R(CvrK^4W>>Z+@}bvNP|=4Z0G1gCS23;y2?53N0QMyq zc)Loog^~ooLWV5P?xE&tZ=ZFGouht)AwuOsBnGg|%#QAE0SE>!Gro}^0F(LIOm^t& zo=FbppB<`&0vu#qTEYv<2IW^mtrXyZJ46RJzt{q^S}wpb0C68<@+A0p1ZLCiYQ+Et z-l5qu0<&5#CLxaFB6C#MLF7DLeDZss#ibc<{^|Q+9Zfd9S5E zLKeOBILZG%g`9cuW;_Di0eE%#&GutfL#qH-W*mAZV?ZME zUmg~KE^K>yNjDuPs?UO+r)v`5XTi_cHHr3D3Wb$S4tM(WUgM(^w21Drkby~c_B9TG zu7h*ZRx;1XWr*{$jLg?`i0D^_l|d$#_4dy!jrEV~#PeC;^E4h}`7HSP8jo5NP#X6u zxq<%zWsTJgYB2x@8LF-TY6emOG8`-OuVzr|060eGT+N`C0dOqLy_!KS1>o42cQu1r z3&1e}kwkwrgIWy0!Jl(AgIW;4u`thS2DKu90}q3c?rH|LC_n>UXEg)q0j#bDK@gaW zwFuCEIRl9RENp0-h%IkSoun@wOBzhEP~A4b@y^LhCeCr0z{3PyX`LhptYo(jc;K0wvAc*?+E2Bas zH$KfRjaWYm8)vF;aKO1F(9->8s?3GBME+S8z&&%!;}QMOZ|6Xp^S`ZZ|Kwgi|L{%3 zp9PKc8S?QzJ+NTy-Q&j2p@EANQ}+UJst+lo0-u^L^kQ0SIzvtS7Y3t3KHv9LFdzXl zHz1n?al>O6RVLJIKgR@!0C?M=tra<{U+5$yvXKaoTd9mH`J(p@1~5kBR)PFgjidmu zu;&}2NdaIPfCOgdHXFB#Yk+f90z*0g3m@W}bdDt-d5q2`Jjno+Y`L7t`>Z2?wu3jF z_vA)R{3}2@0L#LBSAcW?mH~)zRD0WoXA5rkpJEGD_n(p)z%l_Xly=~xiCe}cAo|Y& zH|xLAp4=1dv7XYr$i(_t7BG>A`-SSwO<4A2H6)J}JEjfd{Q5R5#mDPj4AmEi^0RC} zUyzN}+QHgm!H)S8i0_xT`@>3cds(d`YaRhfHiW^URbWU22n#kR#&DK9$mtmsU*VBG6X@D3|Wnp z(IJTOv+$wI5x(tC0Dt~Kq@QKLjkz%vu@U)aS%Ef%t2ukNU*WP(C5}B)s{mN|Q`F6W z`5MvwLT055d&qj<(68Usi1V|+=b;2d`&sbd^*JVs)iQulAvY}La5H`PyUt-umT9}@ zn5&jJ&VKu7PfPLX7n}E1N3cJ{h86>NE2Y$FcSn^R$}$( zq>JEVKtxtli*$j(_Gr6O!u|a{auB%Cm{n@_Ws(J02IfmaNETojFq^8qtMO$4mWi3J z=D~r?D?qXU3*4-~y{j=gGsyxh3(nnv(Va;a(6?cA2gVRevH;5lkO^$7R!?AbXi^1O z=+m=I{w#8Y@2ExJl9g%g%h`CJ6&<1N!482?8twW@oT>X3n@t zngGkhOlM|vHPQoE;AZ{pote?mNDE+DaPG{Ejz$Xt^lez3nbF-y8DQA}PpLjJ>ClYP zljH%Gh3O9Mvi}K=no2MsX`o*zmxiTncuu9qs0nRo_g1UU7Z6DmU}1wMCgrw?fa^9@ z1$8$yH_`-H_znXyBljgufMo#Gq{)T{E{`Lyxt*i{7BcV~>2@8pki-Bjc&o)z&-J{W z4SDe7we)&q=CBJOl#Aa*ZohaJa^}S=kTa)0jVwBS1le_Z5qWj$Epq<9fo1YP@JRA5 z&$p`TtMwTalN` zGK)vTpTRT3hh(`Z`Ttw-TMI4xR_GzA0?7LdLRaIrjDJA&!RX^}0sar)75B*U8ay9g zpTGCrpPl0WyIl7FyMWH0EzRfuu`bTZ^KXopM;d@0zIFAz;+@*&j`;tuP|6Hp3hwlV z4C4JPY~TrSqxr^GyAHdUnA`1A)bMP)#Qj+&AY5A}{B}q*CUJ>KKg$BdrP(D0lxTbY z1S0)~0`9}$v)3IMfViZ|bcpkF@#k13MvR|jU@EZs?<7NXpM`9uExKtJ@qLyF{j`e+ zKMUNfzh>G+bf0Bm?zBtY>(6PRQ#P!A>=lNl_&)1*z-53=U{NUljLT3<04ngXrgG6+ zAjFIdo`1o;K}$UVpu&NMp&y|q`;_}VtVzxKbFd*WC~S>~HK~bz4nEvmhWHLoFL23@ zaG<25)KIhk95YCd!Ho560XJrY=+tlwHSe#1ZarMUjn&xDiP{(P@RYA|$(tc#-L)Rp zWDv7wf#dbne{>M>dzJxJd+XsK{wsqFV)ZO^WJ+^9I*4d}wtx@IDtRx6fX=1+=pf?s zEOe;Wnlf6 zXYL6Di}~;utw#qDo!2!1S)4fpxa`}6F+#;R;`1yM(97UgK!{W0i*E*GKCqADBFL=( zod4(`BJ?`-wi>5fT!ysSjLSslS?JKapyD_$H+is_TWkgkYGJ?mm_Ckcp!u7vXEOXc zZoZG#l~x<62lZ+E0=-8E5vymx&;RHk{5g8BrRU%r{?hr9|Gy78|Kb*M?DWr&v8V5q z%>OCKpHp8$z70?Q!}21z=KuZpK6wzooA?=iD{&7VPn7X}{LA=0e!J8HE<%LZhw(_P zhh_ACabGGjVi};{pp|lp8nGc3{wd^~HDW|8Gt}3#pI0er=2R!6KZ9Th*}o+!`9H8y)Xd3F zHo7(B`#-T#)QB4uMy1LqS1{*XoI@SdUF(sRqDIr4*fEWQ5be>u%YRfWLXTjr8tpC_bQQOHkLBC)^Nd2}vu05HL z4ebWKW6q>GKEVzNjk>d@mf=H_XLO%jDQa213ClCwM^}nkj*pHhkmj>1MJ+G!3xz@s zy4AMQs+_wO)L!%9l}^<1H+pR8a^&QTA4j&lxGmo=Ad80m|8VAi4d>od58{#3i;?Z8 zu0l?p{0x4YJc`U5*8c-S;ytq5MBW~M5E*{_&k!?y6*6#G{|^YUZA6GA5i|NWJU_Y@ z&yRdtmUrT(k;^3WzaPIFeg=MP;bASU2GRSi^M6qnSHa<(@ts{dDmotE)K3!8l{)kc3hK4D)=n%vi_xBXh~5V6_@+%aEmL8p-- z8Ww8QA$dy++kDb1wv&@rRUmd)f(x~dzhB;lc2e=2ThMMNY90Tt1Y51&*TH7D6SZz% z2bC^Y<+5c|=fyH&JmlD9L)n2-%fY(q&9SSs`5f@M z^iMU3uTy(V)$mu|3{tIF=#ZArGsjMvRS~4;OlCZZlTO|@r9-~>X^>jQLPuIL#~KGx ztXLMz`s-W(60KP9b1I((30H%Bwou8~c3^V30-D07L7LTII4HqmjJ6u=4p^h#X^VPHP0M7DOD^B^C+Hb z4xfcjzd%dyR8#mY`1$otgG4Kql{w{3gM=%V34OH_$6|Ob$$ROSzeV&riwoQ}X=e*O7-OPvED?i}8Kp!}#6A)9_pIU*hrjYvlWD z@ja~kNB(~Td3fwHJQ96CmN%kC5Rp3zMqZDKK%{}T!Vlv6@Sn>vi{-)>WqF4zFTpZ& zzbwbHOkioeU6$8jDSjf$dt_wY{gelDeF9G&m4=An-uofxrWS z2LcZS9tb=Tcp&gV;DNvcfd>K)JOVs$(>u@oe!%}90T(K$V&H+m1Azwu4+I_vJP>#w z@Ic^!zypB?0uKZpI0X*`{Qnf37koAFK;VJE1Azwu4+I_vJP>#w@Ic^!zypB?0uMYw zJP`2zN63W=su_47@Ic^!zypB?0uKZp2s{vYAn-uofxrWS2Ts8Q0slV*=LKI4JP>#w z@Ic^!zypB?0uKZp2s{vYAn-uofxrWg5Dx_W{}FPbf@%gH2s{vYAn-uofxrWS2LcZS z9tb=Tcp&gV;DJ-{K*0Y`!Fj<~0}liq2s{vYAn-uofxrWS2LcZS9tb=Tcp&h=Bg6y9 G|NkG$!B*A) diff --git a/flume-ng-sinks/flume-hive-sink/metastore_db/seg0/c901.dat b/flume-ng-sinks/flume-hive-sink/metastore_db/seg0/c901.dat deleted file mode 100644 index 92d1897f93ab5006502be6361903478eccd27c23..0000000000000000000000000000000000000000 GIT binary patch literal 0 HcmV?d00001 literal 8192 zcmeI$yA8rH5P;!xio|q`Kt~A`qM%{|3MxjR0B0LVP6$Y} z=(m)Medol=*X3j_B1SJaG9Pmv(-7-vPUnx;sHHzHCt6wRWkuR?lhMO<{dF`Re;nL8 z;i=?ZKGD7E&GeH)!oEuMXWDHg#{vNa5I_I{1Q0*~0R#|00D*rL*dI1VR{CY}2Rrk# z;_X{wyjg|@7Ap4cGuo6mg#-}+1Q0*~ a0R#|0009ILKmY**eh?VRJnX)V7sDriqb>9R diff --git a/flume-ng-sinks/flume-hive-sink/metastore_db/seg0/c910.dat b/flume-ng-sinks/flume-hive-sink/metastore_db/seg0/c910.dat deleted file mode 100644 index 76bfb9919ad103e51e7b37131650afb09937d8c9..0000000000000000000000000000000000000000 GIT binary patch literal 0 HcmV?d00001 literal 12288 zcmeI1yGjE=6o$`A5+jNx8ZkGt3L?00pSXw`;+q!};Zyq$YuCcu#{^`Z`)w8)mdWqC(bM~F{fh^xxO1VNk}938}=kD!h!xQL63Ac)}PCgKw~I667_ z0OH^Sh=Wfc=zmVyp4O@lASaM7`M)Gh@^#7&`v8DurNv@qnf>SU zkzKPo0CD#uG&eWnKbPSLGN+$hUu>sW_5Bu$=~uJCxxSmE=M9kW=^PC>9cTe9parym z7SIA(KnrLAEuaOofEM_V1x_yy5cbTQb-(ZDp>9U4p?9qenN(zLGi7iLPH!UM$GzUg z-S2#aS$s;ZJJ~r(Qx7y3umSDP5&EpB2ea3Mn{ZA#z9w1hK!3jF(YJh z=!qF6e$+#7K;1jFrV$j8DeKyqJKDgNc}sjEe;^5t$qo#l&Ph zEQ!gJ@v$r>AroLlOo2>@RWT(p5!S?%$;4O}Q!xf3Xo#tjiJ>W`MkWtqV(MfP92e6d zQ@{x^O)|ywPgQCGEuaOofELgKT0jeE0WF{f{$~Me6B6_Ny2@N&wwNIiKbUXKXXXR* qmU+cIV;(Yhm~G}7bCEg4Oqf7~cjgQ8k$K0wVJ^c~+IwZ@eCZ1<$w7ty diff --git a/flume-ng-sinks/flume-hive-sink/metastore_db/seg0/c930.dat b/flume-ng-sinks/flume-hive-sink/metastore_db/seg0/c930.dat deleted file mode 100644 index 4e5fa8b7a46429ed8362cfdade4421650a22c748..0000000000000000000000000000000000000000 GIT binary patch literal 0 HcmV?d00001 literal 98304 zcmeIuu}&0G6b9gP0dX}X3PKPyv9Pq+UDim9g)uyUQLHQo2^1tiA%iuAy|zAvzJib7 zTi94y%AMKST;ifmEXg;UIrq%D`|sWP;$1|feAm}?VkPq+G4}ZC)|4{z&QuY4JICDCQj{jzQ`7sY>`uU!0T(I_@<>9{$z945r zfB*pk1PBlyK!5-N0t5&UAV7cs0RjXF5FkK+009C72oNAZfB*pk1PBlyK!5-N0t5&U zAV7cs0RjXF5FkK+009C72oNAZfB*pk1PBlyK!5-N0t5&UAV7cs0RjXF5FkK+009C7 z2oNAZfB*pk1PBlyK!5-N0t5&UAV7cs0RjXF5FkK+009D*PvFUq@4Io5-z&B6_8_k2 zZe{9C=cXpjU~WDq*V4DLd##`Q+1QHL`IBzUMz7sljn!DG-tHV7@9!N|+Xp+x)#mQO z*8bsMEX4J}Xt@4p^mwD+kDDcPyK82#WY)T7mP+PM$4t6aGIzUW25Gru?sduxhJ{l{ zfB*pk1PBlyK!5-N0t5&UAV7cs0RjXF5FkK+009C72oNAZfB*pk1PBlyK!5-N0t5&U zAV7cs0RjXF5FkK+009C72oNAZfB*pk1PBlyK!5-N0t5&UAV7cs0RjXF5FkK+009C7 z2oNAZfB*pk1PBlyK!5-N0t5&UAV7cs0RjXF5FkK+009F3DUh~f6p*d0~PjNfdBvi diff --git a/flume-ng-sinks/flume-hive-sink/metastore_db/seg0/c941.dat b/flume-ng-sinks/flume-hive-sink/metastore_db/seg0/c941.dat deleted file mode 100644 index b403439df87f360775c1ef1f87a23a5fc117d809..0000000000000000000000000000000000000000 GIT binary patch literal 0 HcmV?d00001 literal 8192 zcmeI0O)mpc6o${4sdoB3ZB=ROV<#ely^X{VuqY80BGE;oEEr9!5a~u@X~Wh+M8tm) z#Lm*Mz@0N~Pbp_AmwVn%oNw06=s3%_4@0-Dr<@*Ua-rt9>i~+Fdjw zp1B+Vw=JQW-1NoPyC9>VUY&0RZ*~0^6~^Dp4QIM03A`_)Ih~^~HSK5unt&#t31|YE zfF_^`XabsmCZGvu0{=0Av&#d7ee-4Q{rxNyO>eB{TMIpvpb(CjBZDS5or8cMpE@tM z_xT93_!Ly5^Q}?a+%2<^3M|2DYs_rn=}5wUHA{@xopS<&0_45jjh($(b;nz;Z|r#s zoApX zt9emP;}EL@qUvIi)qMIhoh{PMYTWp7g1^g znt&#t31|YEfF_^`XaawfKdO`L;z5J2PNW$hENZ6B`y diff --git a/flume-ng-sinks/flume-hive-sink/metastore_db/seg0/c950.dat b/flume-ng-sinks/flume-hive-sink/metastore_db/seg0/c950.dat deleted file mode 100644 index 56aeaa9f552de3d354a2b5321026bfbdff30289c..0000000000000000000000000000000000000000 GIT binary patch literal 0 HcmV?d00001 literal 12288 zcmeI1OG*Pl5QeMG(?>u-$ZKLuWC(%-3A$8J(1i;ji@0(nW}%1?GzrAV&I_mq&^ven zFW^EEJc7%30wd{m74#rnx6S~HA8J85`4%tt723|Gx)x#asKVC9(m|# zm7S*RPRl>J-4t&gceSB0G01s&zW;D_buF2Cw^{n+e5}le>ko|ab(a6ycwi?)5C8!X z009sH0T2KI5C8!X009sfEdpy>w~N%`d-mu1O_HPbLf=!w`V=LEquiMmZ`vJ~BYuc# zv`bYI$*>2Q5kFTh`6u%#surCizTaD&f=h|(ij9HMN zNxS3NCyrfr>|<3SD61>d!JWx6ExvkX#VPY|*{OKFIbD$H>76qzv$yQ_io7n!27AT4 zX$7BH6trjcv=vP&*ru@~0|5{K0T2KI5C8!X009sH0TB2{1jGYbJUQP`mgAwycT~Pa z1y*KcPF2+1w6v%6c6z^YDR==dAcz-COk{NI;9#6^V{p(!jSeO_m^f(SNaEti z$mqz(=wuif85!C1KW}Nj=A*i~n7>J%w$FJ=+qVfxkGvKUxNCd~$P_t>!SPY&?*BTw z82!hGJ$K}#yM`jsAcA{w_xP`u{J5hJmfyeV`;oszv2g5eurl%@W1i2~47bM|@yajY z7w`-C1^fbj0l$D>z%Sqz@C*0_`~v^i0-wLWg}ic~Y4H1gQ`X$0d*?po(mB%?skA#~ z@Ik`iiGZ>9hEE>+&WEhxXWwR?o*hlN+rYh%N79ogXUF361@S3~#F2Z8>BR6n0VK&H z;VeSyAH7h(Ndq&K1ADk_8D~4=e3{2 zl=bYhm{woVK8G3W+2=8`SE< zVm(L4GH{ksrzDUy!}L`K4Twx&l@KfpCBTD00Iag afB*srAb!iQVM$HECXMBo z$zVlIp8I668Z$oD)RehT4(n4m<~SKgr@u&|gAC~90pqX~o27-MvCBS2y!7R91rz>pYsU0fWE ziB3-bC&oczqH)mw0?%`L+fv1H5+~#LhW>hb?)~+g`yrPVc_1R{UUnpWCVd`^=J|tj zZoD5KzfAw}<5hR2?p|q;z2-&g7I&Y!9=~tMUHQmN>N z#yImVo{QXj!foV+T$h{ixssm~iN3fkrXrb=$<@tTy;@tZ)tAoSudd9mZf@7=TRTe^ zYirdNX@7s@uGe&UBO$Jl)=YS##~K;!i=-m0ZOks7nmv6X-qh~=tBBIk(_`PRvz08(GaA)mZ3kGrTbv(Q=VvUlP!y99+p+%YatA}jdzN93R z@v;FAnQ!#Vv7;L8hs-wybSnJT%C@UPoepnISYxR1r;$Ve0R#|0009ILKmY**5I_I{ z1Q0*~0R#|0009ILKmY**5I_I{1Q0*~0R#|0009ILKmY**5I_I{1Q0*~0R#|0009IL zKmY**5I_I{1Q0*~0R#|0009ILKmY**5I_I{1Q0*~0R#|0009ILKmY**5I_I{1Q0*~ z0R#|0009ILKmY**5I_I{1Q0*~0R#|0009L4O@W>tvgrPE{bS4CvLC;*?0G56ZvL!g z^vPzThyI#JJ}>)g?)g0DGjAEa@wx1C&@%DW=Ubnzkx76=GbC&Ya=WCy5WTWAE I{C)ZJPt6lu82|tP diff --git a/flume-ng-sinks/flume-hive-sink/metastore_db/seg0/c9a1.dat b/flume-ng-sinks/flume-hive-sink/metastore_db/seg0/c9a1.dat deleted file mode 100644 index bde1b52cb9d2d78079218344510a8850631647a9..0000000000000000000000000000000000000000 GIT binary patch literal 0 HcmV?d00001 literal 8192 zcmeI0JBSle6o&6Vkz^j3nY`Ax?z+3`W~1vS8{?xAY%FX93$a+(%0{#jT){>Ju@MEE zEkqCm(Zbe3J8f(%EEFp(?8HJ_3qkKWC&|gWYAtdG^5xF|-8(b+C(Vqz5&=xR@ z%yA6;c;?4H2HP+3A9qiBbJ2f7=PJWV@F(~Cf891;ke`0z!TNbwHFhix+MoOvdSfS% zc^k;*46km;Y{wd~2CM;Vz#6aytO0Al8n6be0c*e-_#Yd%b@wvdn*U|P=lfOXpPJp_ zO~19XDUqO9@~;f8yKwX(;J2gE;^A{X+%n!t8r>gQdi4B55N{<{>6 z-Vy|^;P(**kU^ws3nd|9Q6UmlB~lT}nnaYuNHvKmNswuhP*O%N2aHfrm6A$P6_ui< zDkqhpE-FJqRn@DYhzVAHuvJx^R0&N{C2UhQK`O+gs1Vy#HAzL75*1-u)g-AHJ4D5p zQ8h&>!K^0Jl$0?ilN0QeK^ePr&J4{-v0IZ_N;1sL~Eh@(zRc%rg z>}BQe#i9%n?2|zmOR}nhj;>nttB|{{CQFo5+_EMeN~*4>Ntcou_REoT^i}mp)p0;n z9S2qQNi}dtR73unMr*(tum-FFYrqX!?5g(Z^ znU9&bnQO#Ciy0Gxubi(ik8-lkyve-A`8?6RVLl}m>aOP3eq(-MzG6OOK4M;Ao@Vx$ tb3}Y+erCRAK4IQrUSXbL9%pu#HKrrFFUKjUE6P}g>&`5HdT{7Q)}Y;)p%-|$G3So9uG9NUfRLaudTJat9{Oq$Ye zw3=}?Y0RievdLgpjmsvBff|oZfH^g3GENdhH5oQ8=GA1`co?Y(*!WmblVg*@qMDFR z8cS;OY%*9@6S2u+MNNTCfK@dmHaV=RDYFT&uBJi;bI?#zWfMYEO^r<+Ce+l~L^!FY z!KQ#yYMN|{@t(AuGyI@b`Ooo4??gvx8uKVnQ^nnyEiZQ{K#if%zv>5EREb`%=Nry zcw@|rSH1z?fN#Jz;2ZD__y&9fz5(BWZ@@R;8~DE(xO4v+%vJlH2JiRh%!(Zy-S#aP zwwa!Y#qE;8EfWqG0>+*VFCM(-!z|)`&n8~>XJzRjun%&^toLVRwLl&$cwvw6V7N*E z6R~52%7jQb6G$e3q%ujQoQX&#g|sqhWSoggCWEXpS>&AAA(N+yj- zWips@CMTIJI?CiQ?Mz-WdCVwNz)oihk||{q6R1!rnDgBrTZ)XkzZb;&f$k}^$mz?p_*T4q_9wpnqeDVYg# zP?<>_a;7DjDXc2f!C`0Gl9|R4WoCMROHbc`Z@@R;8}JSI27Cj)0pGyCZy;JXMPs5d zVdOjciF`{wCm)md$Srb%JWrk`Pmo=*O(uoG5Arklj(kNvAs>*p$?N0=@+^6hTqZkY zR*0YESMme-ntVz=B=3@&E9{RW8p BTMYmJ diff --git a/flume-ng-sinks/flume-hive-sink/metastore_db/seg0/c9d0.dat b/flume-ng-sinks/flume-hive-sink/metastore_db/seg0/c9d0.dat deleted file mode 100644 index 06e5d765a89a1275711b03dd581b44a6ae0e5ad9..0000000000000000000000000000000000000000 GIT binary patch literal 0 HcmV?d00001 literal 8192 zcmZQ5V_;xl1Y!_i0+VcD5}AO>BZ~pq|Dga)7$-LXC{qHF6#!8pK+FTQoC6|;lf-I$ z^v&O$5FKE%KuoAQ^%o$v0YnlP3AU4v#ke$&ijRiCXb6mkz-S1JhQMeDjE2By2#kin zXb6mkz-S1JhQJUCfn&lezcZ8q9SrMWGlNF}Fnh%?c{IPVumN>SGG0aqb6x2X*q<4D<|*^$Zw<7zBb-OB9kS6?{`m5`#V6(djsQW``<$prMw!3b38si0y7xY%)IC58de zx8$?^zt5Im=fe(wXs=pgV`^ghf7h)ge_hVi*=Wy!f!`#0xvqaU6La_d;^Fk* zEGvwkI>jO#MFX;dP diff --git a/flume-ng-sinks/flume-hive-sink/metastore_db/seg0/c9f0.dat b/flume-ng-sinks/flume-hive-sink/metastore_db/seg0/c9f0.dat deleted file mode 100644 index a12ac62e399dfb1194b9392355cdc471ec074ebd..0000000000000000000000000000000000000000 GIT binary patch literal 0 HcmV?d00001 literal 8192 zcmeI0%}&BV5XXn6K|uTnns_2TdeI7E)cO$I> z_yQie>CJ-&owiWWK&(g2Y|?*cXEOiY{iT;Jv;iQ3k2@q!qn-{*uRfT=_@g_%rAJPS zSaJO7wELRHCy9&4{M<-Q4XY?A&Ghr0mm7AsIp1;Pj9(n3-kLuxJgS-UM1Tko0U|&I zhyW2F0z`la5CI}U1pW|#$C-yWXyJx?_xmg)a2DLBgQN87LBbE@D0a?8p8DzdE5;I> z!x_Ms1go$jteVP9R;hEf&ZzE@un^YZR17W;2?;b<3){SJh%z&ZmfqG4Uh8inDv#qamS8cg z(N_wJoW%A`(89Kt5G>Y5g-CY;2#r^t|5=tZOzrl l3ml=|!7(a_9n>@&;NhQP8}$};Q7>T+bqms{IBxDc@1M9+&n*A| diff --git a/flume-ng-sinks/flume-hive-sink/metastore_db/seg0/ca01.dat b/flume-ng-sinks/flume-hive-sink/metastore_db/seg0/ca01.dat deleted file mode 100644 index baf01f3b7346555834e0627716800e85124ac928..0000000000000000000000000000000000000000 GIT binary patch literal 0 HcmV?d00001 literal 8192 zcmeI0J5R$v49EXDq&97Nijj?l1;SH2i-go7#DG-s8eo7c9BokORcWZi#=?l$d8~X6 zJ_R!qV&ofulQdC51%(y)q+EPaLZ(!Vb z4z<}Rv_E?AX8}9d3oV)VKtn@`Clb;EvzXCt#gcV-R8>}09&257$$Bk(nm5wI|t)OS;dIWx!#EMP9F-svfL&Wvw&@Hfpg`|#K1W#5}Q8kWSyG{omS~q$G7BQbRNI^U2 zXSJ^}deAE{jme}jwv<~`>$MZ@zd$eg{^VSeu4jAn-F@f2@Gs(>2`~XBzyz286JP>N zfC(@GCh$K9JYt2q<_BssK@IgP^~wiWCV3e1B;N<}9tu?KEApfnLXqS=PQ&sK2W}=m E0n8WY0{{R3 diff --git a/flume-ng-sinks/flume-hive-sink/metastore_db/seg0/ca1.dat b/flume-ng-sinks/flume-hive-sink/metastore_db/seg0/ca1.dat deleted file mode 100644 index 6b5e3a2fcbdd5d1fb4ad70b9a5df479eb06fc66b..0000000000000000000000000000000000000000 GIT binary patch literal 0 HcmV?d00001 literal 98304 zcmeHw378~Db!J4}M|aPCYc!gBMp8qaUDaI5s_vSZ()6Y3?wOHxmv(nol?1{D86>oT z1p*`xAYeHb5QoGmaY)?u8nBle%waHKbL<*qcG;CdoU3Kc;bs398Ihe?U0wZ3Xg}-o z7o~4P)miz!jC>Ut75}?JTqlH(_(!jes*S1E-U62P|^|?J@HDEPhHDEPh zHDEPhHDEPhHDEPhHDEPhHSn)R10R3S`(G!n#oZWw{~R%dzb01QpZUDwgGdb2JtS@x z3C|MH^__;T5Pd1K^6&J~H|zfsQ4gUfQ(|6Rp^v>>9TOsaFWwj{K}cK@5l6(Ka)w^H z_HZWC%nY}irP6S|)hy%B%p(3<%8q6-A|(>DW2b79mHM~{3wg+s5*Ci5#4KmBd5CI~ zn9}faCRZFTlry7ROd*@iLChAhsZl#oYt*L3YU5+m6Z4Z(vyd0w=*f!+r{e2YCPVV_ z!>wE{H(V~|sh4YZD|FNo6BSNXi5V?tGw9axa3-6}4lfs5+2O@pyGUa2D&$+GY+1xa zWPV=VS>$|AL`*niN<@CKIl7!_EkQ(KX}FYK8iAbrh%2JGyqI4c&4^86!+3RJVtTAH zSD&6*m~Yf^U$M9+EiRmKB`sST$rUn7xt`L}A~`lUwlH^YO7$w5^yDOjQ&VzAip$wd zb{Pup(W|70k4;Zb)~4proA?n=N>Vr{l$3I|RLbN_y`^jrgY&bs#=^-)Wom9g^(eX3 zla&(ANhPb9EsmBmOBew?dV~>hYGPrcGCQ|$s#a;tRcjUMTXef8FD;x?O5RerP|Req z7>U_TemK+0j}9-DTP6L~WixFY7XuPk?qP;M{hnk!#m zX}C1nqLGEy(nzy~k&Da6pl-Umh!}8Q;dN(FIF~9ZrR72?lWq4_6~4HcN~1ElaJqIm zdXvC&qo2Vc;ha`-3QPG>+|~MWAYx&vGFhXqZl@<_gO+FiXEk6oU^QShU^QShU^QSh zU^QSh&{G4U--sEU@GEM=r$rrv{pDf|bosrCtA9|Oz>&H*iBG>HPT}|y=t$%My_a7H zqZZ*;(UH)8IwBq;p6*`mMc?}F$Nl_Y-XKQtiDBmMNz|^v|A{At#7%zsuQ3U4*}p>i zR~S6xIgbw$hX=uJg2VUb6ks@tn`DV^9Tv`$U1+h{20I6>>cQGJilN!LMt$n!!bBa+ zO;NxY7|l~$5sSsqLN?c1L|jD2#x-(-aISHY?QFh~&99G~)yR#)`C~qEP^2qoPAg>gLC^{KHrvd6{t3g_8; z39!MCgcFsC*_wi%FP!J_;jozyd}@ATLZSBw=f=L#bB%fT+7}4txqLKPeX0|Ba@jAO z=kcMq>$wUIW%GO<5YF=zJX^>#3)w>NS&b~JwZ?c&U;ChNZsLPsuryaaB%GW30?tm< ztMnb?7vQz--Hx;t%ef^ok*;mur0H8v!U;@*uj3M;o?F!SBcm$b$toWSgN2VY9}NjQP&@6CK6 zT_)iKX1}-aC3KyH6PN+t%9qh?5>8+Wd>dcHs!2G3Iq>a#e3wZ$fqC#9d?#uJ8xsa{qlFcZFukHvRK*t+t%=s7WgIq}_m5uFox z0yE=#6ddzmvyjg(_M92<)y>u}neCk zQ59nFX7N}d20kwyhxezb9g3=UOk9E^F*;{pLOdGn3q(bT^q19<4TOq4gLrK$qTW~N zee^|iX7n?JjZTo9=n%D$uh5amt%QwC3I}b9n0ENHVjS)B>3H}OQ3IF$m15M#z@~+K zgm@I%`{;am7qwvKm!fTo(>QV!9T5-EIq8>@{PYGwrB9RO)Zfr`Q?FF*9DSPFOjk*M zjZn##(K*S}boFGCjwe1%S5MqPSC8LKa$+AO$?-p=^WzwY__RP*kN+cGJ@#WdBHmAO z#C3Gnz~Fp(4gS?f{>;z+#rMTw9Dbhq(BCBefB4=T{;v;3J=TvgqK~b?|Kn=JJV+Nl zAB#!GeahhbKP-}bAf7QbJt&KT+%HmmDAu@o!6pxiv=0YAaIa7W{348jmQ74g)*6@? z+$$dw1AGbeY)?&lhs|P} zPePF*K-P|!!5~m72#VM)w(~`-9*3cAVh10;Mzl?6huG;Cg1Cqt_tEWf3hffR_!7F@ z#!hjNkB24^3eoXIhIWfXd=VJUZj?x4eu4~Op*`YoU+DQ6ER$*m4SMk;7tRTvp@*S? zFqhiGTwO(TBtby`kdKG{+{hd6X-a~H{t;gQo&zOdn&zP-Xs2uWWySuOi%&P)l@bJi zZMt?}mH;&s!LU@HffZ6O(37B=t^v4z5&AbFxm#*mT{(kzsvL#$X!Aw4=pOu#dht{g z-}73jt#m9jzCob<3uWhZQrqZQ0`%p{TttgC=QgP=bSyN!16)3^PlS86jI3l5G&9SEqPvU+{x zTTHXEg{|7wQpZ4DMqYX9D;U4KP(Zixp*f zi?#?n2no{@r)n^y zOuAN-@Q^kF1kueVn7-0Qqz7G5MPx7x{69@+RuVQ z%Pw^4#57%ZgJ%4Ffi*)34QQ^<0%L`HY2{313^56_`b!F>HQ8qgK-UR3J5w1`40lkI zd=@zPNQF|G-m|bE2eUJXGC5HnGacV3c6XUl*7%+=IA<@PokMPs`L>6|o=$kXJX~BV zWrmkpi%11guk0wYl*$6%cD*{;s9mNelJjN8g|^D2a%Qo2E{dNNhDGEendM_UC?O)M zVlu~v`w1biahd01JLw?s30YEjRtgRtETrI!QxHIOlYA^6h{+$o%GhP|^+s)c!A;!R zAur)ez?TIHr>19B&JU~rj(gjMk<2pkkUR?CEyD`f6Z2DJZoEV~fdnr;7Ef2BHak7v zfR9c;o@wMjjqxQw>7E2Pxhsvtr*Xaj3>a9GYxotw9h`&i(`o6{Tzos9D`xV&Q^Iij zQ!t>wPpB*i^U^utf=8Bf<&g|B+IpBjI>29-&Pf;C8qGIzrQVrQ{>KEyi^>`4oZ_Q9 z9uyLARyuXQ0RN*x=yTG!l#lLwSV+R<(mCx)SZwW9c^^DQTX6v+^nFO!48Qc^BT8LwpUyWAQvzYyKOz2I|$3CEe!;LOr!E??+5aOfAA?%Rx<>UrLR-@dULc(`jI_G@Ci zv1cov|9>JT41YMH0222QXHE{Q_G)$fe#OaONUS&(R_!IkpQE2veEx1aGxbx|9;G9x zHl3fklcMO8Kce%~N?PI?lApes&WwMM&WT?`^5YMXw0KdS`A&7t{~$ThN$MdY4)8wm z*K|(g6?9Hyp3aGEB?;kg();jjs$Ee0{|Mbt=svni=zYZPL(2LPQgTWh4|%K~8N~Xe zp8VhS|M#6Muck5=n^k%CYx;A$IRWKRAP`_32?!VTd*wA#kf}gKz&bMUdlZl{KUsBM z>&ZZTz&cWtFK~9OQJo0|5c+U~ySr97ZA#2|yY4JsiL)!gB01c@XC(DvkOaCALc#HT=V;w}_6O zZw|x)taJHb&o>18|LZ_${X`BYGuSo;A^@0R6N9oT;QY__jsU@EuguR)!|mCqQS}A9 zA*G!c@cyqO1EW141Is`}ib2YrKny^iQcMbn+#Lu4SVxL_{;66A`9O_Eb_9X|){#IS zJXP-lHn7V4gu0QP+DxDyoZc#>cnZG&H!5bAHWjc$;K_GIm;td(oaZ0Jy*q#>U!%Qw zXS>+T$74BeOs6_1X{$Kj2SK-ya>3WqNQR5Ja+00UJp0v_CdIuL`2TvTt8j16Bi8 z16Bi816Bk7!WxJ_O|ktPwXt6ipO1ZrSbFSwV&Ji&fM-7XOU3;Els=7KPoGBB@yG+j z_9Gu4Y~)E4)gL)RZTR~t#A1YbLYe|mp)@PDk+^hYa^06qEt zgD7&R)?TY{x#3o66jn|3(#7#;?Ka;)$-X(m9U?n5^Jn3)?j5^)Y@+5xuSWK2($9i} z#CbFd>?d-b=K3r!`i|V_`sDni*Y`cz>(9m`0Njt?tzG{td~n4b*{q8#TQ@`b8`S#XlT!%^_n}fP1AlBo~}-$)zkehagiE#i{)maoNM*Y zI0o^jcu&{gmrRI@`A~FRL8*csE{tJ0hmqwOH+I@H`Q%6Nv zUzq_Vib^2>CQ>?eOpNl;^r!_Q%~Qw4qxf(*hEZ-p4V2V{;?aB{o>AqR)uSYJk$6o1 zU~N`NT`XLeG)vXc*Yw#b_*C_!(=qWl{-r&%@d?MJ^8^j=7@^&Q%E2kAmgV%==>f{8L1Q?af7vMQbojavYPhg2~JE6WqH3&!vcS)U{ zz>)w#UTeE>^RI+=S6w*|N>nL>1WFD9lSe!W%FS z11|b&r7lV{T5c^P#lrm1ba(Os9mtJ%mej>)SZK_-ov9IjB6T4emH>Tu^AJ^TFo`WJ zG{*0$#E55`nPcomf& zp3jHs3!Cv*MdgQ^_~l=M1Rx2fj#W4$i1{kl1qitHgxUFQInu6%W(UI6R z$%(02194SZApT;K6F)_AVre=v`Z;RjUnV(;S1Z)5BnL%)a6G(7JUo1aj)dQ=nEx## zGkRLJLv()RZ>SC5OXoyhOCI~k<#c9b51o^IDYdCV)t;uSC%;5@t^NOQm4PSHr=cTs zMfpQz1$Y--S3Xg-!z5q)gk*|)=m_{fWdwNqf8)_#eJQ~I&nf@^TX9bC#q|FVdi;Nb z_pT@Z|NSb|tI@?0fk5!_+AO&;V=-+4;NtZ9JeJVhpM~?*=LT&Y{r#c!>OPjz)IX0P z;^j=+{4R7a=3HaE6CjBfheokGm;mol?o#$LA=23orol5=a z^sFwj7~i5(oLNA4SF7`5r)zVnj+|C^e5*L31_w(7x*jCW@onOm3+Gh4#}CO2@pa#R z1v|94fK@Sn_2p5Ds_KsIcCLx=+I6dAo^eaLz*>Mk2*&tO%=)fxbO91(-GT>6>^|T| z2j@9ghyFFzr-0-jHhXY2m>Df$?+R2L(5v8X0n+#(SAiwkVuD|;6MS&Ts7>NG>m(n& zYSE`M z^HG&X2MDvej-UILOwLcEyq_B++`kB8 zM?r-@H$Ipp0xH0_uQ%{Wqq^T)wEhb@l)p06ulw>HgWga5s`=nAuKmkuz-qv1z-qv1 zz-r)Ou7NP>B7w=z6Vnb|p`!HvPQ8C6*$d)|^T)oVYzMbc8&&p;*eT-kpjq(A$5lnZ z=zk#pdlX9{w3kr}8v(VdI$-3CvKPEc*$Xx+E5lbv8frq}y7Ig9seHSN{J)ay0`fSu z;^&I<|G8?fqw`nz|8IWqj3+!Xlv5Pf}X8{9M#gAx*JqwBDEY%}71l*+Q z_k-1I@OWIi>RDI>n^V0z_%UZHNKP1En4?5fY&@6HvHE>UL6GvvI_m0sQcV7SwzZrs zWJh`rRH)tT6cJDBkbM>qbf`WxGe5U*V!AP@vXkQjI%=PVhb6i`OZ#8<=d`L&`vxNCZUp%FQ^>dkG2}${6xmtTm zM9`^~`UjB#9i`8ODl%H55&|gM>Zb9tpqO4<9&1WIMh10kJ_{b8!I88M%x6JCVjI)* z$gx1p&Bo-c8|aeMW;2$m@zj}}J88^s(3Ue67WYlN7r53lDpnESsI6u!IK~|M9vrlr z#8G!z=R7UwyKf(X_-1W$%Qg!m?$viPLq$u z0}eFmFu>7Yd7*S#e5k&#@!3^wkxrYB#yp7&(>L#yHl^#oNIF;YC3N%IRbDKetN0?^ zP6T{*m6w?Co9rlK6s+a5tK4cLa9Cp8!PF6WQF*C}z+uV2g9MG5z>CVuOaM+FVB@o^ zyxfG~GLbhd zGzI|B!BO~X6M4gu;9lR0{Hgp$6L7;q<2$3?>+NK(F)Xl`eqtkRGx>fP&zol~`x2g9k>pnI~{2Vd)@Xv#*)A?VfbHLScJp64j ziT1U0JbaG0dU&5Y^A?g2mvm^79t#mx}ZHnp-y;8MjsSn`)XyreU?Et$EQ0D*D$`+8L_wfG%B|fa) zUq}7H**gFKj@J}_>F574OxF=N<6rP~e(+E4uMYmd;`Lvt1h5AGkE>6mz`rkKiI5j-d%*~N<(X`=x3|1w`5(VXn+aIpRm=bQ z#X7B@1@{x<3$=ZqR45lS*;emPuNuaQgf*+V4tft4Ki1ufuqmdk{rty-Tefb5BY@U?feeANpt96T<3)Jg`_!jFrgDd z`a;qiI+!%1vb2;bl?z@)$L@xKV70Y37&^(q(yx?TF3sIds`5rdBUx}fSsLzDTYHn? zkSqb1AxOaLM1soghEB5NfLM?mkBr`7_+&q5kBuUm#ocM5gi!M3!m`PH>aPFph8g4` zxI0etE5m1)K{~=y)Dz2+0UZPkGj|(a$wCJVGxr!$$%5(&dxn|!8dk|dV~$&8n0cRJ zl`IKe4KwdI)RHB_?L^Qp^8u61!IH41VdjG-FM}nfyJ6--COd;Aqr))sVUwBB2iSWQ z{?sI9lrkgD<#OAsTf09Bm`(!K|0-WKSs5dXxm8b8Gdqp2&=`E3MxiQf!)`7HOM-iS z&nQ$)n2knQXsF$16sqFa$4$auf!)prj6&6v)VUZe30~**o`7kob1_(G47q?&s5+4N zcf^q6uO_}4{|2#ZY<_zR?=L5=n|QtYbRY59*bh|uZq;5xZTg9dZ!gk0$?s5`+Nn7I z8&!J+G414DV#Ns+^Pl=2G41r73Uxa%;IxXU4?jR0JN##KJbXR%Wbm6LVNliN8~Cpz zdGOy-JG4u+XVp2o>70Q-AUOlCr=F+(I~|Yyf{vs9KiZh8*cZK)&WSvV&WU_PIQXil5Q>*#8d_ zI`{v3`t8?d{QO^BBPj8LY6J#f(cqunUmg5^rRrbIiv;M&|5>{z;irMt1pX-7PbcqsuoP&2tl+W^Xi!e|DSSbH*;)!b`H2NAH~KF*gI&_)0j5P{XRr)y{7(MItu zV}VEvYHL7wxxAFgWqWrW_w9PC5T1@Pl-Q^Z0xU4@+Xb6|{)8#}&w^t@@Ok$V8%)i= zR$(zSvebLkPgA+7m}YJ}b-tj9>j%H!vDDNBrt}{NA7E=b6eD%e)c@;Sg5i6IO!YsG1Z;swfgt(|^zr@FUL8a* znniABZrONByL)(P3QNa+2O@p8&%i+rBck8cf%(R>e_sLK~+i8$N^oN zj|HWvbykfKMLQ6On2H-XVe0a+u#}*suzHL~_UW2@Tr@>&)Ms_h zOk}^=(~WDU?m2(*~rRqVYhIW0wzmkvV&Mis>u{6gwv zEEb$r(t!JST&(;rsdKVe5=;j?CjNlb`B*GC7E7x)9Q@zjg3Z3M8n7C$8n7DZ)Bpm8 z!Q^9KQ0+^}%77{W;LI@<7ZCk6arNl!#Hgcf;^c6Wf>%d=qB#7!$eIwjn$C~kOy|T; zC?DPMvf1|RAw=(&50m&Ap zzOj&~QZPU7E@p=`X=CAFDmYu8J5?Eba=1kAS^V-=F0X2;hnlPI1B3AoN%`$ zA1oXOCQLH!o$is&pQ{I{U0lu;TI-}76LElp4NzLx-Y1=p@Ud8y64o5tALYaG z^=i21Wc;CY{(>)HZ71W8r1O`2IT+~F5zonZzjXd{-)PUX@?+_IT%lV_`Q>t}Xf`wJ z?%kkk`>DlGr1J?rnpD`qEPg7TPx1xm%X`G)XVUo;A5E;ab0hPgOXt%{LOWkBEVqg# z2BiD#V+aKuOTYAfN9-G`0jmM40jmM40jmM4fqx|$2;Zc<_mh-+kFDZ~?SF&p3!&SH zlZXC|;{4myzJ1>z<{$YaeH#8X9gmEvxPWI+)I<0$=~Fo2$wKj1`ZW4DdLR7@)&7Po z0?}vD`SLZ&F0epXl!q1bzhBu1-cMKW^#5PE@5T4|`Mr-%m&zU|S*1mrh8h%`l49^+~CJ}%ofmkhcHs)Rbp!WT<&_PyvKxYH6phOdh zVXR~{BOzu3U+2Vfkc5xjrpzf5PLrsmB)DG?LW)E;fPLcxMOoT{iooxxVy%b8qp zxKO5Wp-g5mvslPxbFiGD*A=*+r?E-gq_#h8(3UF}el6=+LL0GILSPfBok6h>vR@^& z{fY&|^Kzn&LV724b(3&Po3L1D7-gnvXU$XvDLUpPyiFUlxKgGk#=A(_DxHcub2Qt< z%;G5hOdzYc)V<o<3jTsq-u2w_R#3EjDwbCjOfPu3X*q*Ulca8n7C$8n7C$8n7C$ z8tAHl*cr+ih;5`c`ZdMRU#H>)=7{H`%76oI|1HJ#-$eX8@_2RRJQW@A9o61ON5WUA zcAv5u+)JE4_6CXwh<%&d&|64yNW}$&u;&EW{(W?&d>0*;SCJffA-xwrRm}fGn*Y-g zFaQ60uWY|3!2j=2_5Uu%Il&i21$29L@c)Fz`s3caHTeI4TG-{vBgAChz2eS;)nz6ItfdKW?fF|rLc+gfbq`5i^+Qn9|K{NHf z@IG6Csd&f2Zx>rtTzr=h|CHwIEIFno0OWYody5V{VF`c^R|51?8#QZZLA%;JHfi48 zzXW6N*sMu=ezDX>H1-N%T{pRXlVbL=snw4*Vc9G+ZXak}^X@Sw zB%1}*7xv;V9xL_c7%ViN_Eo9XkCS?b43>nhsnwTAy-fy7gxiT=YPBQvMj0##Yo=CL zq~0%sCB_|0oi*>OQtz3;l7VRh8ucQ8#-!dhqYtnb05mT3mKnLiVyoO5=^X&n{ZaTo zQg5BnuMFR#@J~|jp23pQ`6$3vHtqi0bM7MWDEwOL4Kw<=u=gnZMmm@AFWl`>_^osr zdG~*Cg>=sFWdt9Es+7*ze!=EZsD`9- z&IK_#u_F{^=oG!Xr*V%nc1ULAd*YNu2?Kzumz1=YTc zm~doDdDmmgyZ%|lv~O0estXt%P@Z{J=P&UY#kxO6TsmG8?g$vc$CzD38u+tIG(?byE% z%s;8>8m1h=rf#44$2l#(R z9K@B6uhMyX)e-kV2meoctUv0#TZ8|*_5VuC1?&yiHd94+q6U`dW}@pfc1#^=o?(GO zZC$XJ#tw0OxZkIkNXnFpY_*Gp#nKAB?6xy-AlTaZuEe)=L>ww{plilDJO3bEy9rAS zxzgQ`Z4-0t+$#t9-GHJ3A>YpTWNy?NsAj*=s8#3d6XP)cB+u8j6qXF+vAdhnox;8X6DU8SwznrS`?P6=B?fBn z?`?OUwyUtPu&cOOW8W~v09a@&R9v*@6i@Eg#uXO6dkTH>fGHlx{2pmbg@u(a9Mm=z zRu{0+G`m734{7TP9P;W#PWkL`w%9HWFD{iRyM>(cnWZcOiUhtq+6|#Tfs_z<=EqLe zCMzl~AaqDuT3BMBLaJ)!ic;Guz}C83Jp(K}zI^|+_vvT|78dh|7f}$}tL-W*BwZD; z2ztnJ**8VEHH+L$%F~* z*Y*=XFhAVN<#NO2QXX6Jnc*>t;Q`&AsOuCh?4EmI)i=5(61pX%(d{v9L1A4Lqv>RA zs@70r(jAS*<)OZ3RI}iA{+m4Pg4^vZs;wJiQ}^{Y6dlZKe}_E6N2BYj(%SEoNBIKG zsl967dw07Riz=ApB!u z@8P=%6@D_E8P1cO(ETJKbQgUZdV;c0TufJ&KcREvJ#@ajlFpGA6Wfp9K%a_VP*21M z>3qch(-F`AKe#pd=m7seAok(&!}u3`$>5*fUmg5^#p}Of3FyiHSsSEd?VvVT*Nd+B zPR;-O0&5Rle3xebEU=&b$G7YF|3a&|n9b!)3A^sgtu70u>MorEz!K1H$y42|((PzP7$`97qr=IWAK@=eVCUz$3=}sV9S1>ugx=k~E79P)QXLfsaho<^00ni-;tW*!)uDO0cVDDKo zX%8cX*2riMX$QSF(xNNF%x7p})r3&6zyT{|bx2cw7IaN3W!1z{u*9skQdT!=BLEAJ zX)a);tZvYj02W#+&p3mtCYXY?Jq*UYx8LVsZ3$ojS5J>ebpgRbbEMqN6-=^1_a7H{ z?R=RI)z#*D@Auihu^O-%uo|!$uo|!$`2V1R_$lJm*m#+20bf(LfL9S4kIfMqj|~zV zM-~9sc=XluY4n_G_fQ-8j$-~Q+CMT!`2vwWBq#hmV*BCSC|@9aj%)-drG~47?xjyd zcdGU(b>uj;@<)pQ->urK>3l>GpcX*<1tbUc|6Tr1%)f*GZ@lXEHDEPhHDEPhHDEPhHDEPhHDEPhHDERH&}xAA|NjGQl$VA8 diff --git a/flume-ng-sinks/flume-hive-sink/metastore_db/seg0/ca10.dat b/flume-ng-sinks/flume-hive-sink/metastore_db/seg0/ca10.dat deleted file mode 100644 index 3683d197c21e2bc2ff8f1da52e21e91ba223341b..0000000000000000000000000000000000000000 GIT binary patch literal 0 HcmV?d00001 literal 8192 zcmeH~y$%6E6ot<+))Iw;PNfh^51>|WL8fGn)Fa_X+mEjy^j|f}@4GKc diff --git a/flume-ng-sinks/flume-hive-sink/metastore_db/seg0/ca21.dat b/flume-ng-sinks/flume-hive-sink/metastore_db/seg0/ca21.dat deleted file mode 100644 index ed3c2cfe8f921402f33f2b136118c683c7e68788..0000000000000000000000000000000000000000 GIT binary patch literal 0 HcmV?d00001 literal 8192 zcmeI0F%khW6o%gmc0+|5sFZXX;|z+e96^m4okqDQ&}+3yCs3Mk1DzxAl5AFHg#++s zCU2AfPx3Zjmq#liV(F$S8xtGD;QdgO@uO%J`Qvmd&O*8-YWqbj$z}evHDc~{a=tq~ zILiwCn|yc}B#G>|IrS#uKmq{}009sH0T2KI5C8!X009sHf!`vK@77F9xs!k2*QqA6 z=G!g3Up10cN(NifWg#FsDtGyJKGT3NPHpuH*OVSiEg+*=;h1N=Nvuv1QdL$7SbmaI x(l13B#UCz<(LewMKmY_l00ck)1V8`;KmY_l;77Vqi`5yohN$;FJ}M% diff --git a/flume-ng-sinks/flume-hive-sink/metastore_db/seg0/ca31.dat b/flume-ng-sinks/flume-hive-sink/metastore_db/seg0/ca31.dat deleted file mode 100644 index 83b43f8269f35ca880bfd72fe0306b8f1e082efb..0000000000000000000000000000000000000000 GIT binary patch literal 0 HcmV?d00001 literal 8192 zcmeI0KMnyw6vn?<_74>88QQGCJAscoqsL=1zen~G{1IkQPga#?>hJ4wCN zPPZ@~yDJD3ukumlSdxVEK~8U;pfE%NBtQZrKmsH{0wh2JBtQZrKmxx-V7ZzpOyo_b zzE2}1tIXRgjs7Y)uCxpm@Scr;#M$$dsq+;|>KsP1l}OdC2i*&3pcyH;tk;RHqd3=s z3%k`F_(SiwzfVFIG!+f&c>6&nDC9Nv~t(NXa;`2EkCvv~3HvTnvjHS^ctxSLt000k&O0SZun0u-PC1t>rP3jC44A-lh0 zEide^?@jduAi65{`=S44IdW&gI0ofZO>!3lbkWD~y(8(q7x@OP!D)MyNd}UzN8`nG z2?tiM(`rTwHq82b9t>bc0SZun0u-PC1t>rP3Q&Lo6rjK_feC7INfjl1VZPG1JpuID BAqD^d diff --git a/flume-ng-sinks/flume-hive-sink/metastore_db/seg0/ca51.dat b/flume-ng-sinks/flume-hive-sink/metastore_db/seg0/ca51.dat deleted file mode 100644 index 453dcfee86aa85148a1ef98c66c580ad96e16375..0000000000000000000000000000000000000000 GIT binary patch literal 0 HcmV?d00001 literal 8192 zcmeI0u?@m75Jm5ZltifzOHjf96pR2;Dn_6Nq%?>IIyyRf=3t7f09(MtiK8eO20-s5 zJLms%w){Gu>_kLuR55UK?B?#|{jO&DtBIccb-6HSVS;x6R? diff --git a/flume-ng-sinks/flume-hive-sink/metastore_db/seg0/ca61.dat b/flume-ng-sinks/flume-hive-sink/metastore_db/seg0/ca61.dat deleted file mode 100644 index aca129793f3813762772775024f493fb8248bf6e..0000000000000000000000000000000000000000 GIT binary patch literal 0 HcmV?d00001 literal 65536 zcmeIuuTH~290%}UH`W#u4nqQh2LM4Z2oN|D$Px^035H}K$UvZG@Hn0T%bQ?%8j^AJ z&$5id3-Gz7*Sp`hch`R6AtJJLf45%M=T*PhwfSN#jb}|;D*rk?Pst?RWyG|7DNBc^ z`Jb)dm2T`mzSQ+;8`qzi4WCmX+l^RngN|o)W;yLhln4+YK!5-N0t5&UAV7cs0RjXF z5FkK+009C72oNAZfB*pk1PBlyK!5-N0t5&UAV7cs0RjXF5FkK+009C72oNAZfB*pk z1PBlyK!5-N0t5&UAV7cs0RjXF5FkK+009C72oNAZfB*pk1PBlyK!5-N0t5&UAV7cs z0RjXF5FkK+009C72oNAZ;J*oMZEwEDefsOH`SX7C(y095(B{iQUZe$2QO=syJ({m- zH_qZ9`>3mHqbAMMbC5pdVt!pVZyZ%gME;ouMIJ>AMprl2w~@!n@a*&?E$bhT$3N@+ z$clOoJplp)2oNAZfB*pk1PBlyK!5-N0t5&UAV7cs0RjXF5FkK+009C72oNAZfB*pk z1PBlyK!5-N0t5&UAV7cs0RjXF5FkK+009C72oNAZfB*pk1PBlyK!5-N0t5&UAV7cs z0RjXF5FkK+009C72oNAZfB*pk1PBlyK!5-N0t5&UAV7cs0RjXF5Lgs=i@h{GjJvwH IGg*EA0!!96JOBUy diff --git a/flume-ng-sinks/flume-hive-sink/metastore_db/seg0/ca70.dat b/flume-ng-sinks/flume-hive-sink/metastore_db/seg0/ca70.dat deleted file mode 100644 index bdfb446aa13779a444ba5a5f39388ae92439a3e3..0000000000000000000000000000000000000000 GIT binary patch literal 0 HcmV?d00001 literal 65536 zcmeIuJx)SF5CGs=LQn~%rLC0>v8JRZgy1=}5NnR$B|L#g@eZB=E135e2|l$o=G)Ew zyqVq1+b^CXqU5OBpq@wdd^~me<5=$ZZ955n`?<`_W0T!h%Vmy97C)~(erUbrZ{K$x z&7Ey`wK-Q-&lxDwF517=HGeDIe^rzK0RjXF5FkK+009C72oNAZfB*pk1PBlyK!5-N z0t5&UAV7cs0RjXF5FkK+009C72oNAZfB*pk1PBlyK!5-N0t5&UAV7cs0RjXF5FkK+ z009C72oNAZfB*pk1PBlyK!5-N0t5&UAV7cs0RjXF5FkK+009C72oNAZfB*pk1PBly zK!5-N0tEh(N04W_5NHk2q228;ehz)=#;B3c{69N(~ z`j&FB@0?h9T~5{_V)U{jaWQo<4xt|Bbp9+yBmHqX(aKVk`p5X(fDb znZNgDn4K@ZrId2CB3fnNE&KDXuAk>(emuoR`sQ*Tm1R|37sD{xATI9}i(jJg`sr?e zsLrhGqt(8=*hC^v>TL5^-uc(X%|~Sk5FkK+009C72oNAZfB*pk1PBlyK!5-N0t5&U zAV7cs0RjXF5FkK+009C72oNAZfB*pk1PBlyK!5-N0t5&UAV7cs0RjXF5FkK+009C7 z2oNAZfB*pk1PBlyK!5-N0t5&UAV7cs0RjXF5FkK+009C72oNAZfB*pk1PBlyK!5-N z0t5&UAVA=s34Bau4vS&N9#D<6@KiH%|K^)RRNXEAE&kIRWxmU^Wk9lyzFb6tNOjqS7Ht?Ql& z-enWLtKLjMXyW!&qV1V(D>)VjAbSF|yJx^FP>;oweS+ z)z|BVq@_m&J4wrffW}$*l>g5+GB9VCZJ&y2toOYARAw?SY6qJWi7zfeL;wK<5I_I{ Y1Q0*~0R#|00D+$cMzU~wv-|k*0{%uX!~g&Q diff --git a/flume-ng-sinks/flume-hive-sink/metastore_db/seg0/cab1.dat b/flume-ng-sinks/flume-hive-sink/metastore_db/seg0/cab1.dat deleted file mode 100644 index 278ace9134b769969d8640d58067ae992ee53b13..0000000000000000000000000000000000000000 GIT binary patch literal 0 HcmV?d00001 literal 8192 zcmeI$p$@_@5P;z;OST2YBjB)LP)HsC0fuKlAb{IRP5ODc}M~&H$%?*;liaLV_j# zCY|oiY&+@ay0Q`xqn90-i@A$wi1jq5^G6u9^vC5`D+|3!k#^ih)K;l1OZ zO5WuX-J9M_KWXCky+qfu+`QyiAbXKq%! zeQT`O3wfbO23sksgMh|a^;G=NH_|g#pKYB()gC?A_7jpCAxRLnL*UV@2f=DGv8ElED%5d0R#|0009ILKmY**5co%d-F|Iksb3a;uoE}S z-o7=|>xDGaBZIBvtb8i`=NsvptIxL1Mb#cX*!B~d7L~c)ZR6zR6GQ|MKmY** Z5I_I{1Q0*~0R#~EL0}{^zx#IS9bTk1F8Tlf diff --git a/flume-ng-sinks/flume-hive-sink/metastore_db/seg0/cae1.dat b/flume-ng-sinks/flume-hive-sink/metastore_db/seg0/cae1.dat deleted file mode 100644 index 8fb96b5b55c3fbd0dd36c7bbd2927ae81fe62cc1..0000000000000000000000000000000000000000 GIT binary patch literal 0 HcmV?d00001 literal 8192 zcmeI$Ar8VY6oBE^iETmQF*qbB6o){d;07oJkl--jH~}|6aRg3*$H6J!wOyBP3=%B) zo9wN9uiMK$x5`>Xj9zvmE~YNVA=KlX&Y#7or9UpmT3PCqi?qWgqs?{wbu_j=4If?q zRP-*F=-u^Z22m5guM%C)d|Sz}KmY**5I_I{1Q0*~0R#|0;2#AJ$BmJdep&dzPTVYe z`_@pe7t&0R40e*22LX+<@~QBjZ=`RoKHIt#ReSVc+fQU#R2F)-jgyN{5D`EC0R#|0 Z009ILKmY**5J2DufsxGp?#})^djXjZE;RrE diff --git a/flume-ng-sinks/flume-hive-sink/metastore_db/seg0/caf1.dat b/flume-ng-sinks/flume-hive-sink/metastore_db/seg0/caf1.dat deleted file mode 100644 index 86593054652131fe04a4307f30881fc6e91d9a62..0000000000000000000000000000000000000000 GIT binary patch literal 0 HcmV?d00001 literal 8192 zcmeI$Ar8VY6oBE^iETk~1RN3+iUS}KD2{*z5*!9eLW1N5oPskTxB&@BP5`g%x^!cZ zV9DQPZ|!^CUiP_FmLg*GvLkUZbukX19_MuaEJiK;aXHq?T(4ZD9X1(luIsO(vHfvy z@A#*pcezCOrZ>}%n)rQ{=z8YsN{$5r2q1s}0tg_000IagfB*vjD6l=Oj4bra!Vh-h zX4%`fhI+k_W_o0>k-R(zXq=T#h5vjbJ#+Ed)}yG}qX*l5BGaO>)4Oe)oP2_a00Iag bfB*srAb&fTb@> diff --git a/flume-ng-sinks/flume-hive-sink/metastore_db/seg0/cb00.dat b/flume-ng-sinks/flume-hive-sink/metastore_db/seg0/cb00.dat deleted file mode 100644 index 0d62a64c8581ba65f54de36db98de95a2bd99d17..0000000000000000000000000000000000000000 GIT binary patch literal 0 HcmV?d00001 literal 8192 zcmeI#u?>Py5P;!>7}RKCqAM5+8yh=^AQ37Hmr!?b4l8?SZ~-efa03eqy>|}+L0{}l zOnwN*arfZh>yVv@M0&U_wwPF~hq7OMmczyYG>oS19uwf(bwS}Ukk>M<5$qU*=er7HJa+LcUFSxT;ZW-+f;kRgBo0tg_000IagfB*srAb`M61Qs&& Jb54i#`~{=<7U%!~ diff --git a/flume-ng-sinks/flume-hive-sink/metastore_db/seg0/cb1.dat b/flume-ng-sinks/flume-hive-sink/metastore_db/seg0/cb1.dat deleted file mode 100644 index afc543aebf85d6225a074117ea81b231a878ccaf..0000000000000000000000000000000000000000 GIT binary patch literal 0 HcmV?d00001 literal 16384 zcmeI%d7Rec8prYLo~il$W^cBpX_}^Knx+~F9YWFl+#@7qsbtTVWJ@I>oorE(BwMl^ z2Zs)hljS%#AxX9nLPAKAByG}uuIKx`kfZq1>%3miyk32J_q0sz>-purN-hu)r(cpE z4bTj9`~3L*+rOsXW*`6a@=m{bxPH(1`Xv@=^y3e5`pNa%zx9zmct_tmbjZ{X`TBOh z7G9|RO210qZM#phgP)Jj`8%I@2R|+KQtAX!Cy+XU)Cr_cAaw$%6G)vv>I70JkUD|X z38YTo|Lz3deZ4GChU;(J_ItNl()3%~e)S)+ll{LzhOX{ua+SnC{Y}6QWxw9fi@uW% z?mc^Tx_zDe%xU^N=_SY7E7$AS$I70JkUD|X38YRSbpojqNS#3H1X3rEIsrR@=~Arff}LMcv$oHYU+n*_WTUR!zuxw1 zcXEOmk*@4xwqO742d4kDoBzD@|GdNNRv)D$cX$^`OvI8&WJ(s1CD}x_luRnn4ZDb+-^v?5wb4N)VliPlm})JhwojkG1&N;{&R)B$N3vI`MO zdm@%yiA?E0WXWzsw(L&i$R0$lbR_cRPei`#Ni>qZhyv+E6w2O2k?cbh%f3X3bS4^0 z7ot@5BbvzmM45CYnu_;Sx%dRtOnjg!#Me{J#UDUbitk3X5Pu+5CH^3)rTBxXYVqBv zR^kt#YQ*=TT8lrFsukapY9qcE)mHprR6Fs9L+KgfkDwy)M^dr)-c+XeK2(C#A4@e7e;idH{&=cT{0UT%_!FsO@%^b1@h4G@#h*--ia&*F zBK}mWO#Eq-p7;PtPyBRBPkbPyCw>N{Cq9VM6F-yE6F-a66F-~M6F-O26F-;I6CX_J ziJwR5i4URl#D`LP;=?FC@$)G?@!?dR_zNKY>qS3;(zCyi(zCya(zCyq(z72)>DgaG z>DiB>^z8pk>DiB_^z1LC^z1LA^z1LE^z5&o^z5&s^z6q_diGaQdiGaSdiG-}J^OK# zp8YkHp8a@A&;D9U&wc`>XMY`~XFrkBv%j9wv!6ui+226v*-xhQ>~Ex6Ie!yXD1{voQd^AA&{&i{jI z;`}32ne+donmRw7DtG=-s+seTQ5DWVPBnM_398che^D)*f0C+l{@+wf=bxggoqw9r z_x&@JzVH7->HB^LrSJP^DSh8RN9p_ic}m~+Gbw%Fzd-FGeiqeU{EJYAJ`1xceHLD# z^jVlg>9g=MrO(1Els*fuQu-{sM(MNgI;GFTTuPsXHz<7;-lX(dc#G0!VIHN=!rPQS z3-3_+EWAtUvoN31XW>0cpN01+eHK2T^jTOy>9g=5rO(1bN}q*~D18<_ru13(gwkhW z5v9+zr>D4oK$lultKrBnEh(kZN>bPB5}ox&PQr?8gNDXgP(3g1&Yh4qw9;Ri~muz}Jk zY@~Dwn<$;aW=f~9h0-Z(rF070sHQu)(Eak|%oUO)lDQaNfWC>&MxQ~aqxYkCpp(&Y z=xB5}dKP*z+7IoC)}#BN9nc!I94$bjWaBO98gv=D2%V3L5_BOt51oU~Kp#V=p?9G-q2tlZ&=Kf4 z=&9&2=wWC#v@^OpT8mboMQCQS8Qaix=yLQk^nG+LItzUYeFVJ^or+FEuSQ3q!_Yx! zf3z>!15Kcv(DrCGT88GM4U=uS30;LQML$O0L0>_iL!Uq&MDIp#K_{SBpckQo(E;dj z=n?2a=zeHNv@KeRmY~_FOSZv!bOpK?U4Xuc&PJa>r=$0ycc7Eeap-7tIC>U(GTIOA ziPoe0pdHW}v>Yu!qh!;!pli@&=puAJ`WiYDeG+{by$79wPDIC`Bhew~K=cH(H`*QT zitdTlp;c%pnv13=nV8RbQ*dWdJ{Sxy$l_Jo-3L9D-S@AM|-1( zpb2zuv;*22tw4*>Y&0!dw*mbQU5b8!&PQKIXQ5A{)6oaeyU?4_3FwvRNOUMV2t5hy zhxS6dp4A63}?A2qlA zKdQ9-KWbt7e^h1rf7H_U|ESvb|EQJi|51(Y|50n(|D#&l|D!gx|3__Y|Bu?){vXxZ z{vYjP`+uBa`+pqS{vXG-|HqlO|HoOj|Hs+3|HnDD|Hrwu|HpZ@|Ht{Z|HqAN|Bnl7 z|Bnl8|Bs7o|Bs7p|Bp*-|BoBn{vVgx{vS87{XZ_V{XcGM`+r<+`+wZb_W!uT_W!uK z?f-G5?f-EL+yCP#+yCR1w*SY~w*SYiZ2ymIZ2ym2+x{Qd+WsH6vHd@8Yx{rP&i4Pf z&i4OUZ&V(G&(s@Lg3=pRp!7ynPw9>707`FE-6*|L9Z2bo>L5yQR0mUfqv}rSjp`6e zZ&W=fy-^)X>5Zxw7Yx`1quRV&= zdu=~T@3lu$dapf((tGW(l-_HPqx4>TJf-*A6DYmco=EAvwm+r!+LI`~*Pcx2z4jDJ z@3p5=dapf=(tGUyO7FF&Q#yr#P?k>N3`(akh|(#XN$C{MqI3#pQ#yrnD4oK&lultV zrBgVM(kTp~bP7W$ox(6mr*J-{Qy5O^6fU523L_|;!iAJh;UY?>a51G*7)j|AE}?V^ zqbQxipDCTfXiBGWDWy}mjM6DwPU#e`pmYjXQaXh(luqF)N~dr&rBfJ7=@iCMI)!T} zox*rZr*JK$Q) zFm7<+P2)xv-ZE}-VV-fb3vU~@xbTi~s|)WMx4AH%4C2CjM(4u&#xxf`Fs8e(z}Ucr z4~-37SZK^};Ui<@!pFwgg-?u`E-W%;x$vnm+l9}JIWBx|%$2a%m?z;2W4?qX#zqpp zG!{r$YAlrSm9a>|GGnoXuZ<-VmKz&O_{LZ&VTG}Ygl~;y5>^_UO8CxLE@73inS|BG z3JGhB%_XcgR!UfBY$4%$W0i#U#+DL(Fjh<0U~DB}qp?Q9CSz*}n~k*+wiw$;*lKJm zVVkj?)QhoB>YZ^HsZTSum-=+$u2SE?*g@(W8h4ZW4CC%n9~t+M`qGi-HuF`<);@>dIWkP z+8NyqtwEcj`DlY=;|=Jy=wkE(bT0Y=`Xu@gdN+DAIv%|g9gd!f_DB1m-O>Hgj%XXS z0xd+NWTVaKD)dWqA^H|N8+{sm1icrXf?kJSfnJE7gPwx+Lk~q0=w4_Y+5#;>^)c)Tfwg0E~|37g5&q{kd{l6QX7kdI?(KTBzZKe|)RtaRlf-B4t-x%ywFwf%Yg==)RA zT`n-V>t;q#6JJ-;u4lfh#c&`10SG_<0uX=z1Rwwb2tWV=|486?+8SBw%fbgcaaQ*B z){t)|(o9DNd&#RoK&w@q3jcf~Lv!=dPOr?`(Sw}@%w$n!>}DG$g|!d}KmY;|fB*y_ Y009U<00Izz00e#_Fp;INzZ|af7gc>NtpET3 diff --git a/flume-ng-sinks/flume-hive-sink/metastore_db/seg0/cb21.dat b/flume-ng-sinks/flume-hive-sink/metastore_db/seg0/cb21.dat deleted file mode 100644 index dc181754e444bffa7c9a5b7dbd105dfcdaa4ce9b..0000000000000000000000000000000000000000 GIT binary patch literal 0 HcmV?d00001 literal 65536 zcmeIuyG_JE5CG6w2euIj1t0+(kRnk8;*(IvLF|MaiXf#4dLSjI2##HQeL5Y6h>SNU zkH<569{I(6M5KI`XvW-*c{|PX?Ov|eHFnxRK3?R>DqnrXFds^Jd4B#i7T3S-eU$g* z7{;8_bS_;!mijc}U>+>o$DRGKFwr7FfB*pk1PBlyK!5-N0t5&UAV7cs0RjXF5FkK+ z009C72oNAZfB*pk1PBlyK!5-N0t5&UAV7cs0RjXF5FkK+009C72oNAZfB*pk1PBly zK!5-N0t5&UAV7cs0RjXF5FkK+009C72oNAZfB*pk1PBlyK!5-N0t5&UAV7cs0RjXF z5FkK+0D=D|aCCh59C!JvQ~mO0th#Rw^L%&EcG1K`^y?bS)#hJ)tx}w&x2fJgslBqa z%j>V>X7jtPXUeTD^UtXv6Cgl<009C72oNAZfB*pk1PBlyK!5-N0t5&UAV7cs0RjXF z5FkK+009C72oNAZfB*pk1PBlyK!5-N0t5&UAV7cs0RjXF5FkK+009C72oNAZfB*pk z1PBlyK!5-N0t5&UAV7cs0RjXF5FkK+009C72oNAZfB*pk1PBlyK!5-N0t5&UAV7cs Qfj+tCy$uFExxX3IG5A diff --git a/flume-ng-sinks/flume-hive-sink/metastore_db/seg0/cb31.dat b/flume-ng-sinks/flume-hive-sink/metastore_db/seg0/cb31.dat deleted file mode 100644 index 130efcf474f892b00aae8676aa1ecd7ad2a50390..0000000000000000000000000000000000000000 GIT binary patch literal 0 HcmV?d00001 literal 8192 zcmeI$Ar8VY6oBE^iERl6kHH}UgMmXJP#ggbBsdUAPQVQ~#ZJKy2%G|5+jZ&2Ai~ zSwp^=NHe`MI7(h!1hmeox59tEk)gTzXlGnz?bU-VzmR2_u@74~DeR1f00IagfB*sr XAb diff --git a/flume-ng-sinks/flume-hive-sink/metastore_db/seg0/cb40.dat b/flume-ng-sinks/flume-hive-sink/metastore_db/seg0/cb40.dat deleted file mode 100644 index 7ba3154f2e826ceb1961e41512cf26d67ce4274f..0000000000000000000000000000000000000000 GIT binary patch literal 0 HcmV?d00001 literal 8192 zcmeI#u?fOZ5P;!JkSJ;)=nkm_2e5F2fTnT{bq7bWwXkr2)Pj{8xPgU*-n*Azh(7ED z;eU{Om%AjFd|hNCB9R_$#TFBb^-%TeHO(L8YLtVgnRZsbJ8;`bkG>A?_*yW29KXEW zxBd=$mP^a)(yPlSz8Q)(^V{Ii&D0S<009ILKmY**5I_I{1Q7TmfnvD3$y#5y`oWGZ zYrlO$)o(OfiC&*Z=U2Hm)4VGS$t9Dy#@uqZvk*W40R#|0009ILKmY**5J2FU0#h0L Jar@KZ_yq5d7>NJ? diff --git a/flume-ng-sinks/flume-hive-sink/metastore_db/seg0/cb51.dat b/flume-ng-sinks/flume-hive-sink/metastore_db/seg0/cb51.dat deleted file mode 100644 index f8da77f809411c0802f2c12b0d840cdbcd67486b..0000000000000000000000000000000000000000 GIT binary patch literal 0 HcmV?d00001 literal 8192 zcmeI$yA8rH5P;!xiX;*ZBhb+S6%_>|fJDUz)PO_>1tk-(0aGvq3(&v@ba3$_aza3& zMZcw7>^mn`zAh(g5ixodBe5~HF%F>~=XCz`qn7@-oM>gKS1!^Hn~WZ=>#w8n_|x#- zaZg3>vWf0ZZ)Ol3;`UXdJu}}{ax4%)009ILKmY**5I_I{1Q7T~fx~fQWTjsgey|fe z%ig{<)a!*b)0V+r^0E=oI4hqD|M^Dx=Hjxgsjn8ThoToSlew>qZAP0CXO|!%fB*sr bAbBb~pKEMa1Z3N8)1YVjMy}&guMFj9U8Ra-x-`Ub#p+JY=-FuD_1P_NU>a z>vu)(a*5tuZ)OlR@y}JF>zQvWITi>YfB*srAbh(gJ>5;)s^70^{aaQgM|M^Dx=IXPpc~P}T54QbOW<_PJciT8QYhWON00Iag afB*srAb}ab;JF0$UjmQh5L}{c+bwB61%u>G z)9G|(+ihQLA|jQuk7mfLA12^^msy~Jbr>s0@|8LIU+hiSe$SgoUpr^sE63yvC?`yB0 z<+A-v+|G`>y6U1_`_IbV*pLYjAV7cs0RjXF5FkK+009C72oNAZfB*pk1PBlyK!5-N z0t5&UAV7cs0RjXF5FkK+009C72oNAZfB*pk1PBlyK!5-N0t5&UAV7cs0RjXF5FkK+ z009C72oNAZfB*pk1PBlyK!5-N0t5&UAV7cs0RjXF5FkK+009C72oNAZfB*pk1PBly SK!CuX1snm>xIRt{dK+F7}7&zuH&bai;k)q?Tk_~rSw zcX!xlnY36px|n?8nxSZCej6ORnK}XpAb z&9_hR{aT}u==x!FD$AX%=55I&m6^<6dseO>LjVB;5I_I{1Q0*~0R#|00D+$fOl9om I9CnNQ6MVE6;{X5v diff --git a/flume-ng-sinks/flume-hive-sink/metastore_db/seg0/cb91.dat b/flume-ng-sinks/flume-hive-sink/metastore_db/seg0/cb91.dat deleted file mode 100644 index 74bb2b02f1579d78bb651f17c9fa851e42159483..0000000000000000000000000000000000000000 GIT binary patch literal 0 HcmV?d00001 literal 8192 zcmeI$D-Oay5P;#?ina*`iNv7@C=`c4pg2M`kl<>N2sAg~6g>rJfK$M9AI-KD5-j;Q z>2!By+f6^$k-dl*z3P$JnA#YJP>*vufBMl$e_XD#vehdW8HPWsAu8Pt01;>Xixc~qF diff --git a/flume-ng-sinks/flume-hive-sink/metastore_db/seg0/cba1.dat b/flume-ng-sinks/flume-hive-sink/metastore_db/seg0/cba1.dat deleted file mode 100644 index d17d52c93a4ead4264b63cd554aa8efc44a83d07..0000000000000000000000000000000000000000 GIT binary patch literal 0 HcmV?d00001 literal 8192 zcmeI$Ar8VY6oBE^iERl6i^Cy7p*REr#Szdzf}5G-1Sl-0;0`2&Bfu%(wOyBP3=%B) zo9wN9Z{5p2x5`#Tj9zvmE~YNVA=KlX&Y#t2r9Up`T3PFri}b@oMw{#U>*#EMnm&ep zSM)BI7~S<|CQ%drTqC-kdC|zRKmY**5I_I{1Q0*~0R#|0;2#C{hnUcYCGY)YxnP_;^(&t8(Rtem<1S^78y^EUsT4d<^d^ z(T};L>0G*g9_q7*qj|7!A9v<{VWLHV009C72oNAZfB*pk1PBlyK!5-N0t5&UAV7cs z0RjXF5FkK+009C72oNAZfB*pk1PBlyK!5-N0t5&UAV7cs0RjXF5FkK+009C72oNAZ zfB*pk1PBlyK!5-N0t5&UAV7cs0RjXF5FkK+009C72oNAZfB*pk1PBlyK!5-N0t5&U zAV7cs0RsO`;NTY`&1vk)!wkQ zE9-CLZu`5hXZmt)hxzB!kO>eVK!5-N0t5&UAV7cs0RjXF5FkK+009C72oNAZfB*pk z1PBlyK!5-N0t5&UAV7cs0RjXF5FkK+009C72oNAZfB*pk1PBlyK!5-N0t5&UAV7cs z0RjXF5FkK+009C72oNAZfB*pk1PBlyK!5-N0t5&UAV7cs0RjXF5FkK+009C72oNAZ RfWRLJT*PX6%T2y~{Q{uRFaiJo diff --git a/flume-ng-sinks/flume-hive-sink/metastore_db/seg0/cbc0.dat b/flume-ng-sinks/flume-hive-sink/metastore_db/seg0/cbc0.dat deleted file mode 100644 index 46cfc9ecf8e5a5881042f2acc3197b63bb5d1ff7..0000000000000000000000000000000000000000 GIT binary patch literal 0 HcmV?d00001 literal 8192 zcmeI#F%E)25J1scV$i6CiQa+A!~=K;60!0cYHeZTQS3Z|6&JAb2HwEJLT7i`K+uhy ziOGjB3^NM@uS2#XV)U?E;$rGzGlYJ#runnlTIJ|vuAQ~-j_hfo$54m&d@UM3j$d6L z2Y-iqmQAa5+0EuN-;6~!^V{Iq&D0S<009ILKmY**5I_I{1Q7TmfpUDj%SK-~{9q?8 z>%M(L=(if3RM%(bQkMs3&HGZwQgT^n40}b Jr<`_s^8&_r7t8HrM@E>ui6SK8C(3 z`j-<%cm0@2)Wn}_c-J%EH#9a7fB*y_009U<00Izz00bZafqx`$JnxKb^k(6Kowz7_ zyKBg|6KSR+gM;MtAfPjUeGvw8f=V_YHd!_Db9JGJDy)Ki7a^{G3T9nG><^I=L-2VOOvwvTS zcE}}->(b3jUtdHVPlJW)uxD-;CRzjt5FkK+009C72oNAZfB*pk1PBlyK!5-N0t5&U zAV7cs0RjXF5FkK+009C72oNAZfB*pk1PBlyK!5-N0t5&UAV7cs0RjXF5FkK+009C7 z2oNAZfB*pk1PBlyK!5-N0t5&UAV7cs0RjXF5FkK+009C72oNAZfB*pk1PBlyK!5-N z0t5&UAn@M=&dyI?PW2y_IYKS-Bec=34dHJG*fENMTX%eqs4Xowe^-i&8m^# z6@AME#y5SKX;j3&*BV{TeAkMxKmY**5I_I{1Q0*~0R#|0;2#AJ$E}gIo-F)eCvKL# zoi*gUi8RwIgT3Ud26mzCd8l;_6_ft@d9 diff --git a/flume-ng-sinks/flume-hive-sink/metastore_db/seg0/cc0.dat b/flume-ng-sinks/flume-hive-sink/metastore_db/seg0/cc0.dat deleted file mode 100644 index 226872006d7e7681f9732c7e2268f2312c42abd0..0000000000000000000000000000000000000000 GIT binary patch literal 0 HcmV?d00001 literal 8192 zcmeI1y-or_5P;_d`8^c1va&H+ZL;?R90n5+qGABigF-C~aPS0bv@<4_))u~qrSS<& zd;v=n-$9){Zjr#!8#8-h*qynL-TN{P0G9xu;Y*AeKSumG8f1Mmhx6}XjHRb88(29X zRMYY%9XF53#gzA?Dt~fYD-QC*&kByY{(*Uo0}Kla-rqBJqo`1d43GgbKnBPF86X2> zfDDiUGC&5%!2dFE|9Gvz1wNDfexCvzSBn!*DeGf}a0JO^?QJCejPnrGp@kPjL$C&` zU2a$o=N{v{@32fKV=>cd=W!{9k?(nqgFgX8++(-0U4vN&6&nqR`7Cl`2-f4Q@8xpL z!uti=#;arGITr-75{yr5+F@qaHW<(Kx^h#??pr+Av{YL3pOymKmp?&Y+~H=qb@S= V>cAb&4@QE!k-y}P` zZ)P|9_KRIalpJ-8nmlguY3lo@wOoI8aaR8IbCsQ^wp-QHU5;s9KB+%0#NhQ0@5hh! zOw}K4_m%Y_3uVz)hv(+bzb+m=s!D(W0RjXF5FkK+009C72oNAZfB*pk1PBlyK!5-N z0t5&UAV7cs0RjXF5FkK+009C72oNAZfB*pk1PBlyK!5-N0t5&UAV7cs0RjXF5FkK+ z009C72oNAZfB*pk1PBlyK!5-N0t5&UAV7cs0RjXF5FkK+009C72oNAZfB*pk1PBly zK!5-N0{=|lW$}0)&-tj*J#5shZht(p@1Hep#<_o1UcbxJdO3+rY-82sZLS3Y0t5&U zAV7cs0RjXF5FkK+009C72oNAZfB*pk1PBlyK!5-N0t5&UAV7cs0RjXF5FkK+009C7 z2oNAZfB*pk1PBlyK!5-N0t5&UAV7cs0RjXF5FkK+009C72oNAZfB*pk1PBlyK!5-N u0t5&UAV7cs0RjXF5FkK+009C72oNAZfB*pk1PBngK7nP-+n?H1`?oKrn-;|Y diff --git a/flume-ng-sinks/flume-hive-sink/metastore_db/seg0/cc11.dat b/flume-ng-sinks/flume-hive-sink/metastore_db/seg0/cc11.dat deleted file mode 100644 index b9381ede540c69c57607d6fb2fb980afba87fc08..0000000000000000000000000000000000000000 GIT binary patch literal 0 HcmV?d00001 literal 8192 zcmeI0yA8rH5J1lEH!!$t$%0 diff --git a/flume-ng-sinks/flume-hive-sink/metastore_db/seg0/cc20.dat b/flume-ng-sinks/flume-hive-sink/metastore_db/seg0/cc20.dat deleted file mode 100644 index 1d2d401df636c956e968696f400cc301deb14443..0000000000000000000000000000000000000000 GIT binary patch literal 0 HcmV?d00001 literal 65536 zcmeIu%}oMP5CGtLgn$Q*-r#^85YHYhfEX!&gc6i9CD4Qo*or++gPwGDmyHA#xf=3K z@_u&az0AHZ))7&1RMD&FK|LQ%ZT&Ep`|o`mrO$rOv+~kZH`Q{RW0cKrs*gXSv;F<< z^Q*bDX|Fct%F{LzW!7f9*ShDW#@$zC2@oJafB*pk1PBlyK!5-N0t5&UAV7cs0RjXF z5FkK+009C72oNAZfB*pk1PBlyK!5-N0t5&UAV7cs0RjXF5FkK+009C72oNAZfB*pk z1PBlyK!5-N0t5&UAV7cs0RjXF5FkK+009C72oNAZfB*pk1PBlyK!5-N0t5&UAV7cs z0RjXF5FkL{$^_nKkCS-KN0t4TOUad9#6348+|$k|b!LXVb`f@qx0 z--M&RD_rau{;WnT{c%}nWu;dtGVq&>HrMsn(b@hod1UUX z;9WK`y6er1gC=fYBif$nzL8^r00IagfB*srAb1u3;AkH=Xuc@;g8F?W;Xg{BHggbXmMSCZLQ@m<7dyG zs=Uhu`VSq5WhEWl}uQs}x`K}gYfdB#sAb#^}GxlK%CxxBS5I_I{1Q0*~ Y0R#|0009ILK;S2Vu`K=m<9Tv@0b@5U-2eap diff --git a/flume-ng-sinks/flume-hive-sink/metastore_db/seg0/cc61.dat b/flume-ng-sinks/flume-hive-sink/metastore_db/seg0/cc61.dat deleted file mode 100644 index b3cc1e607032c28ccf10b1ec2f3a3db919dec0db..0000000000000000000000000000000000000000 GIT binary patch literal 0 HcmV?d00001 literal 8192 zcmeI$Ar8VY6oBE^iERSG5$2Fgp*RErg~Or{K!QVnt6P`Rdym`^s*yyF?BHxp&sXS{;WqQ{c$1IOVG0jB@GoDP^P0u0cSfVQNW@@ zLi#~IE_XIZUKiPkNTkbLv5kq1&EWgZn&yw{YL$bxxpp?LJ1}1(U8xT5xmqxO9KX7j zdw0VAmPxC9lbgwBt{IAU=C{G2!_*N#009ILKmY**5I_I{1Q7Tmfz$AKm#zL`|AHOc zto!x|zTawe65T(JE_FFwYu=ZIR diff --git a/flume-ng-sinks/flume-hive-sink/metastore_db/seg0/cc81.dat b/flume-ng-sinks/flume-hive-sink/metastore_db/seg0/cc81.dat deleted file mode 100644 index faee883e230e8756fd1eee7a0cbc18754f0265c3..0000000000000000000000000000000000000000 GIT binary patch literal 0 HcmV?d00001 literal 8192 zcmeI$Ar8VY6oBE^iERSG5pYOQBqV4w0>u$f2q3|MnB)Z9fGc1)1!sU$z-zlM-54ZT z@;BLA`(C$ZpWBnIh#0->NL)-^j6&a|@DD;Md8O-2vb_1DpQ{CV`~ z`KO|HxkUf2H#3Y5@%w7gotf`zITi>YfB*srAbdiu$Y0IFHylMor&Z?)vf4-4{x%q78P*!d0!S({?vM4Jnm(ixg)hCDuAbn-a5 diff --git a/flume-ng-sinks/flume-hive-sink/metastore_db/seg0/cc91.dat b/flume-ng-sinks/flume-hive-sink/metastore_db/seg0/cc91.dat deleted file mode 100644 index 37e65667d806a8a54d5be4db795e91c9bf33c1bb..0000000000000000000000000000000000000000 GIT binary patch literal 0 HcmV?d00001 literal 65536 zcmeIuu}#E45CFiv4{Re5PyiC7WCSEg6oC*VR6+0%J0XV-NWKz83lu>SJm>TI^?D2m zA+whE*0*P`Jn;|_Dc?gZN^VNtPE&t-FW2iDJKH}#Ugpj+-(5sM9ZGq4e*QIP$G`4< z4ByMqmz>jhE?qqh^=ZVx)Rl*Xb=B=()DbCaTSnogA zy^XJOBUy diff --git a/flume-ng-sinks/flume-hive-sink/metastore_db/seg0/cca1.dat b/flume-ng-sinks/flume-hive-sink/metastore_db/seg0/cca1.dat deleted file mode 100644 index 482bde6b9706dd38acadadb22db10bb52ce95463..0000000000000000000000000000000000000000 GIT binary patch literal 0 HcmV?d00001 literal 8192 zcmeI$Ar8VY6oBE^iERSG5#|UA6pBM2P;dhj0!VNeBqu;&IR&TS3~&S#yteDojX{DX zf0Mnn?{$0G=T_N>h|$ZA#KqLbID~qf)A_R;jr7OmL@O)3a*=M>WVE@izmC@S=h40J zPet!?iQY|bW*9Z``)bkk%y+dM3j`2A009ILKmY**5I_I{1pZOraNHVM>z9Qe?8MEo zw{H#gW+Bb=$Y3vdbr8@xtDXw~`9=oj>a(3wS+z$Gw*6FQWo4;%+c>%S1Q7uQ5I_I{ Z1Q0*~0R#|0009Jk5E#qC?=Jd}`3v?iE$RRO diff --git a/flume-ng-sinks/flume-hive-sink/metastore_db/seg0/cd1.dat b/flume-ng-sinks/flume-hive-sink/metastore_db/seg0/cd1.dat deleted file mode 100644 index d919a1b0e320bd23ef9eb70d43b5189acb5d9ec9..0000000000000000000000000000000000000000 GIT binary patch literal 0 HcmV?d00001 literal 8192 zcmeI1zfQwI4935tkfi@;3lckGKujQ5SxPHVs!#%H23Wc?0}>kx5(_g!cb=gm8#4RL|*JS^htr_`8HUa5uVga3OcH{e2zUYA=4frH#)9JYPlzHwdPKfWqCh%J%sVHk_ACea_xGdeB)w ziqlMaG>Qx*bmO}+CGe7yB-}kf3677z#XL%)s2iYc6FTjIDrHoxvX}N%si5kj3|~s( z=`Msvy5C)_a3a_Md RtIj>O58!+O>G0`x?-%c{mUjRE diff --git a/flume-ng-sinks/flume-hive-sink/metastore_db/seg0/cf0.dat b/flume-ng-sinks/flume-hive-sink/metastore_db/seg0/cf0.dat deleted file mode 100644 index 780d64a60020caae0a1bcced4cd7a860c9606d86..0000000000000000000000000000000000000000 GIT binary patch literal 0 HcmV?d00001 literal 24576 zcmeHOTZm*w8Lpn`p8L%1PO_WLWj7Im@xdi^J5_ZgFLUV)GSpXSN;G0{Z)mYL8`;}q$w;p}@O@II3yLvG1dk5}!FhYyN{;qe=J#*@l6+K2DjTjg)Fk)cD zz=(kn10x1T42&2UF)(6a#K4Gw{|yFy`uXqu1APR3OJCnVfNqC(cMtr!&cGdaqdftC z`L5Qlj(qwTxSm9t@ZX+mH_;aAOv0^qqkFM22IDNfgbB|tNs^08I$`1xwlAYImZjMC zD~w7x&SYi3ThZdqLmMQ)6sHuEl%+&qG>%$Z_A8Y8PK@EZ4ZV~zRbJxSWcE=R&N!hO zyUtzIw%}V+4xlJZ0S_)K9}M5F3ndlKOl7}8fRwmrUTy+#+5rU46vufGpdm@BaDGyN zj5TL)#sSQAT4=0`N@jq;Of#jh)|LGR8Dz~(+@AG=kZ>3=xP*~WJqTH5QjLqevfmV% zSY6)P-&$`t@Y-n3(U!^rV`7JE0927Njmv6KK-=ol?$YwoPIrH6X?y8P1VDu59YDo$ zm^C;!@K_yyXVER!1H8OWEQL*`Mti{~61ZKDfWJwzJ#Z&g1P<9;9YkEAT!cL*pz-!An(xgweOM8e!ZqblMr0t29%Dg+Y~s zal)!WEuzyf_V!n;fLK99zHLB}ZR)`)S3&}|N1nNXpL3ggRJiixVm2W|vn*Y4Wx+SSed_3j6n z`(A|3J9sjeT9Y&gFCj9Ae6kwe0-D>oy1g6C-0cn;rVtWH7KSEEgGnHPzKHG`61uJ} zZS2MPJcRCWP?ai_CV3cYk(Zd{)hu2>clMy#aN!{CZ(Lp3zp}IyWA8gXsbvWcE}V*r zL=#MOHPSgW0}%JFY(~wKyBx4oCR`H}q)-^lFfrA@>_A+-32EL12aOa3_`MzYx zrl^JnQ@Cf{_`!G4!OH|_3sQ#RX$~H+tcG_6oq|MV<7VW@NA&NI%Y~QFR}os6KwpDw z`cd>eLUX@F&q4bh^eRHLPowW3H1oE-KeJ-{OuvX;fOZ~z0imhK(APmw|J=14J&Vx9 z^Y(8$|FS(hrR~%H5qb&QyU{lg8vnTMIrfgdKlY%#ZhgzO9or&p`=AHVeIDi;+dq5$ znEtv>#=v(rzHF zNS!=)22Hs<_wqVzD*2EvHVBMUMOk%~4q#Q&M37V30OmC#eO1$CBDO zyPk3YOGyhwMUcT-Y3(UAf0)+Bkmm~vGA~QO79=mJlh@vjF5Eb;jlp`x8#>D&Hp$?u zP3k1JhdKCQVjBZ>(Lq&-A&SZ{t?^Ks&u?FgfFe0nR0T(8tt9+3TIeU?*JzYW?A0O4 zf&sW7kF^qUJ9Q2c@faJQ^)@a_SlUQ~!?9K-4k^L$OgsjkFGi#pOB79m+%?75eBk+F z1SzFZGz$(qSw|}Pn&zQYh@sCHBQmZfzzfn>D`P%~ZXL*&V<7sX1kI>YG!HVU>ZHt8 z3)-q{KV^=A=L->%CJIBGSUJ*mcS0X@36quGlgqbp2(V`k5 ztmYgdw8cayNJOxBD1c9JweSr3Cg}e^TLnJ<5P~i3bKkW(e{Rt#`Ps+Oi=hACu-7x# z>HlYJJM}m8WoRF^{U=|um4eBZZ7257YoPysU_b4gx7xn_QQMCH-YWm`b*KMdxAlUt z3$|@NZNClu&Q>Bo|HHHF>ofO!?#W~N|Bhz*-xvW2Ng4EJwdP_kNj@?cW1NMT|1g-0 zX1IFY6qI7Exd>^Lypjb(v>`SW)P*9D&*4^H*OCu4E z*4G7ctQBW0xUvsRbq4xX*Qjiw*?nlQySdWc?-f^@ZjI>gJSY>*ITJJ#=OI@%iUou^ z)SkAlZp0N7eO(}9jAa}NZ$g<2d$ldBBbb!7DIB&W}-QRQ|lqhv#f(EXI5UO}< ztB*wR)Ygn960#^^Blf!BrTD`lI^0@A40PYI zAr@myFYGl=GAvkdj0wZ37J+~}=sD7e-7Q|)Q=xJt7!OT?S^#^xFW&8XO|KGIy%(I_ z6l}dbxQ8vqvc0VZ8Y?Bj+q{@67#F;H-V9h);N~#_`5gr%O#~K6aOl+nc(mjY9aO!l zmy~EOl3>WMg%1d{iLjGl9$$=wy{I>k*lLkhMQGp*D3!`+3VRqSZzQ1eyikXUa@y(W^5H;`omVYPrd2du5Bm3ZPokaX?s2K zxb4~bg}nnO-Ql|ZoV`E(Py5Z|AGCV}#(rq`1&p1yYQOcAeF}QZwGX&m0Pyx>zkYl2 znErq74fOv!bRP2Rum7_Yb`pxTdPdf@E9_{2_2GDr=as+8QiDZc%T!%x;EfNV#Y_h; z{4+q)Sl2Uj8u{aOT)DQ0(Gm)@!rEP_-6!; z70B?Zhg!%WXdNJnuxh7wumFnHLdo#K0;S-joJ^_%@1R+w7deei%6X-qNDeb1T$SSt zPVz}{($EFR_nNBMme>3xFETD=Ft!#1vPLKuRrdM8Yuq zkZRvE1Yq-{e4TTC0aHU%!dV_fm@xtgO5oJw07BS{=4{-ELSG7iBZQ1+d9a9>LOE4s zftBnb0F&l}(3b)PlZIzH2rw^5rm{dHGX!As(~vD+4}cR>L}w!u ziND&D0uzs*FM;y^zP;bMV6WSsMqdQw|3|C*$9L`Z*z2|yFm~2f6IxH$p6EBW$^h&C zwuRR#3-|x&nEwBXI}ZOo|LKvob>#ZUWn1`3icOg&^dP&Wa88(+0^FW`8J!wtlszse zcp*|s`+{wFzhf38iAYG=*w3|)bi3(r3QKVGAPXJ-U<(lT{u@gBLveV&AppIfYcffZ zasU%J6_FlpWgq@v0GO#FH?U!#m#z%&Hw2(xPHf`lQ0f4R0#Z$SP@En9U;qdPV+lPd z&JOQ40MO1;oA5OPcw|71E=LTE7#J}yVqnC;h=CCUBL+qcj2IX(@PEv}*gaPHw?5(Y Q|68s-SpR?Qh2@w319}xYIRF3v diff --git a/flume-ng-sinks/flume-hive-sink/metastore_db/seg0/d18e0.dat b/flume-ng-sinks/flume-hive-sink/metastore_db/seg0/d18e0.dat deleted file mode 100644 index 56231b126cb3b9cbbd317f52bd97b5de56c088a4..0000000000000000000000000000000000000000 GIT binary patch literal 0 HcmV?d00001 literal 4096 zcmZQzpdSbTT2%tX%nS?yAX`O%n1_Lpfdjg#Z8m diff --git a/flume-ng-sinks/flume-hive-sink/metastore_db/seg0/d1900.dat b/flume-ng-sinks/flume-hive-sink/metastore_db/seg0/d1900.dat deleted file mode 100644 index 898f6a132934d4669c14e79f776f79ff413cd2b8..0000000000000000000000000000000000000000 GIT binary patch literal 0 HcmV?d00001 literal 4096 zcmeIuu?YYm3`Efr8xac&7qE62N3n2!j9{m=)jvp)BM^AfzjS*wZM>BC*J6|86!!9n hJDuI%88x=Mr9l7z1Q0*~0R#|0009ILKmdV0HW~t>Aut*O iqaiRF0;3@?8UmvsFd71*Aut*OqaiRF0;3^-5dr{ivjWoq diff --git a/flume-ng-sinks/flume-hive-sink/metastore_db/seg0/d1930.dat b/flume-ng-sinks/flume-hive-sink/metastore_db/seg0/d1930.dat deleted file mode 100644 index 898f6a132934d4669c14e79f776f79ff413cd2b8..0000000000000000000000000000000000000000 GIT binary patch literal 0 HcmV?d00001 literal 4096 zcmeIuu?YYm3`Efr8xac&7qE62N3n2!j9{m=)jvp)BM^AfzjS*wZM>BC*J6|86!!9n hJDuI%88x=Mr9l7z1Q0*~0R#|0009ILKmdVBC*J6|86!!9n hJDuI%88x=Mr9l7z1Q0*~0R#|0009ILKmdVBC*J6|86!!9n hJDuI%88x=Mr9l7z1Q0*~0R#|0009ILKmdVBC*J6|86!!9n hJDuI%88x=Mr9l7z1Q0*~0R#|0009ILKmdVAut*O iqaiRF0;3@?8UmvsFd71*Aut*OqaiRF0;3^-5dr{n+ygiO diff --git a/flume-ng-sinks/flume-hive-sink/metastore_db/seg0/d19c1.dat b/flume-ng-sinks/flume-hive-sink/metastore_db/seg0/d19c1.dat deleted file mode 100644 index b29acd7c7d269c4e081209de8e5cb34a3311e85a..0000000000000000000000000000000000000000 GIT binary patch literal 0 HcmV?d00001 literal 4096 zcmZQzpdSbTT2%tX+zbo?Fk2ZJ7#TQV95k8{%I8882Xh}(%!$V)J}NdE0;3@?8Umvs gFd71*Aut*OqaiRF0;3@?8UmvsFd71*A%GDA0A&;dNB{r; diff --git a/flume-ng-sinks/flume-hive-sink/metastore_db/seg0/d19d0.dat b/flume-ng-sinks/flume-hive-sink/metastore_db/seg0/d19d0.dat deleted file mode 100644 index fd69350ab4492229bf299ba5c03a36d8750aadfa..0000000000000000000000000000000000000000 GIT binary patch literal 0 HcmV?d00001 literal 4096 zcmZQzpdSbTT2%tX%nS?yAX`O%n1_LpfdjAut*O iqaiRF0;3@?8UmvsFd71*Aut*OqaiRF0;3^-5dr{n+ygiO diff --git a/flume-ng-sinks/flume-hive-sink/metastore_db/seg0/d1a91.dat b/flume-ng-sinks/flume-hive-sink/metastore_db/seg0/d1a91.dat deleted file mode 100644 index b29acd7c7d269c4e081209de8e5cb34a3311e85a..0000000000000000000000000000000000000000 GIT binary patch literal 0 HcmV?d00001 literal 4096 zcmZQzpdSbTT2%tX+zbo?Fk2ZJ7#TQV95k8{%I8882Xh}(%!$V)J}NdE0;3@?8Umvs gFd71*Aut*OqaiRF0;3@?8UmvsFd71*A%GDA0A&;dNB{r; diff --git a/flume-ng-sinks/flume-hive-sink/metastore_db/seg0/d1aa0.dat b/flume-ng-sinks/flume-hive-sink/metastore_db/seg0/d1aa0.dat deleted file mode 100644 index 9ae5d87652f235806d34d38d8317614a4de4b769..0000000000000000000000000000000000000000 GIT binary patch literal 0 HcmV?d00001 literal 4096 zcmZQzpdSbTT2%tX%nS?yAX`O%n1_LpfdjBC*J6|86!!9n hJDuI%88x=Mr9l7z1Q0*~0R#|0009ILKmdVAut*O iqaiRF0;3@?8UmvsFd71*Aut*OqaiRF0;3^-5dr{n+ygiO diff --git a/flume-ng-sinks/flume-hive-sink/metastore_db/seg0/d1c31.dat b/flume-ng-sinks/flume-hive-sink/metastore_db/seg0/d1c31.dat deleted file mode 100644 index b29acd7c7d269c4e081209de8e5cb34a3311e85a..0000000000000000000000000000000000000000 GIT binary patch literal 0 HcmV?d00001 literal 4096 zcmZQzpdSbTT2%tX+zbo?Fk2ZJ7#TQV95k8{%I8882Xh}(%!$V)J}NdE0;3@?8Umvs gFd71*Aut*OqaiRF0;3@?8UmvsFd71*A%GDA0A&;dNB{r; diff --git a/flume-ng-sinks/flume-hive-sink/metastore_db/seg0/d1c40.dat b/flume-ng-sinks/flume-hive-sink/metastore_db/seg0/d1c40.dat deleted file mode 100644 index 9ae5d87652f235806d34d38d8317614a4de4b769..0000000000000000000000000000000000000000 GIT binary patch literal 0 HcmV?d00001 literal 4096 zcmZQzpdSbTT2%tX%nS?yAX`O%n1_LpfdjAut*O iqaiRF0;3@?8UmvsFd71*Aut*OqaiRF0;3^-5dr{F)&r{m diff --git a/flume-ng-sinks/flume-hive-sink/metastore_db/seg0/d1e10.dat b/flume-ng-sinks/flume-hive-sink/metastore_db/seg0/d1e10.dat deleted file mode 100644 index fd69350ab4492229bf299ba5c03a36d8750aadfa..0000000000000000000000000000000000000000 GIT binary patch literal 0 HcmV?d00001 literal 4096 zcmZQzpdSbTT2%tX%nS?yAX`O%n1_Lpfdj events) + throws StreamingException, IOException, InterruptedException { + txnBatch.write(events); + } + + @Override public RecordWriter createRecordWriter(HiveEndPoint endPoint) throws StreamingException, IOException, ClassNotFoundException { diff --git a/flume-ng-sinks/flume-hive-sink/src/main/java/org/apache/flume/sink/hive/HiveEventSerializer.java b/flume-ng-sinks/flume-hive-sink/src/main/java/org/apache/flume/sink/hive/HiveEventSerializer.java index c233d3d860..386484c636 100644 --- a/flume-ng-sinks/flume-hive-sink/src/main/java/org/apache/flume/sink/hive/HiveEventSerializer.java +++ b/flume-ng-sinks/flume-hive-sink/src/main/java/org/apache/flume/sink/hive/HiveEventSerializer.java @@ -27,11 +27,15 @@ import org.apache.hive.hcatalog.streaming.TransactionBatch; import java.io.IOException; +import java.util.Collection; public interface HiveEventSerializer extends Configurable { public void write(TransactionBatch batch, Event e) throws StreamingException, IOException, InterruptedException; + public void write(TransactionBatch txnBatch, Collection events) + throws StreamingException, IOException, InterruptedException; + RecordWriter createRecordWriter(HiveEndPoint endPoint) throws StreamingException, IOException, ClassNotFoundException; diff --git a/flume-ng-sinks/flume-hive-sink/src/main/java/org/apache/flume/sink/hive/HiveJsonSerializer.java b/flume-ng-sinks/flume-hive-sink/src/main/java/org/apache/flume/sink/hive/HiveJsonSerializer.java index a75073f732..0311a5b7d9 100644 --- a/flume-ng-sinks/flume-hive-sink/src/main/java/org/apache/flume/sink/hive/HiveJsonSerializer.java +++ b/flume-ng-sinks/flume-hive-sink/src/main/java/org/apache/flume/sink/hive/HiveJsonSerializer.java @@ -27,6 +27,7 @@ import org.apache.hive.hcatalog.streaming.TransactionBatch; import java.io.IOException; +import java.util.Collection; /** Forwards the incoming event body to Hive unmodified * Sets up the delimiter and the field to column mapping @@ -41,6 +42,12 @@ public void write(TransactionBatch txnBatch, Event e) txnBatch.write(e.getBody()); } + @Override + public void write(TransactionBatch txnBatch, Collection events) + throws StreamingException, IOException, InterruptedException { + txnBatch.write(events); + } + @Override public RecordWriter createRecordWriter(HiveEndPoint endPoint) throws StreamingException, IOException, ClassNotFoundException { diff --git a/flume-ng-sinks/flume-hive-sink/src/main/java/org/apache/flume/sink/hive/HiveWriter.java b/flume-ng-sinks/flume-hive-sink/src/main/java/org/apache/flume/sink/hive/HiveWriter.java index 4a06feb9e8..aa8576e327 100644 --- a/flume-ng-sinks/flume-hive-sink/src/main/java/org/apache/flume/sink/hive/HiveWriter.java +++ b/flume-ng-sinks/flume-hive-sink/src/main/java/org/apache/flume/sink/hive/HiveWriter.java @@ -19,6 +19,7 @@ package org.apache.flume.sink.hive; import java.io.IOException; +import java.util.ArrayList; import java.util.concurrent.Callable; import java.util.concurrent.ExecutionException; import java.util.concurrent.ExecutorService; @@ -68,6 +69,9 @@ class HiveWriter { private boolean hearbeatNeeded = false; + private final int writeBatchSz = 1000; + private ArrayList batch = new ArrayList(writeBatchSz); + HiveWriter(HiveEndPoint endPoint, int txnsPerBatch, boolean autoCreatePartitions, long callTimeout, ExecutorService callTimeoutPool, String hiveUser, @@ -117,7 +121,7 @@ void setHearbeatNeeded() { /** * Write data, update stats * @param event - * @throws StreamingException + * @throws WriteException - other streaming io error * @throws InterruptedException */ public synchronized void write(final Event event) @@ -126,28 +130,43 @@ public synchronized void write(final Event event) throw new IllegalStateException("Writer closed. Cannot write to : " + endPoint); } - // write the event + batch.add(event); + if(batch.size()== writeBatchSz) { + // write the event + writeEventBatchToSerializer(); + } + + // Update Statistics + processSize += event.getBody().length; + eventCounter++; + } + + private void writeEventBatchToSerializer() + throws InterruptedException, WriteException { try { timedCall(new CallRunner1() { @Override public Void call() throws InterruptedException, StreamingException { try { - serializer.write(txnBatch, event); + for (Event event : batch) { + try { + serializer.write(txnBatch, event); + } catch (SerializationError err) { + LOG.info("Parse failed : {} : {}", err.getMessage(), new String(event.getBody())); + } + } return null; } catch (IOException e) { throw new StreamingIOFailure(e.getMessage(), e); } } }); + batch.clear(); } catch (StreamingException e) { throw new WriteException(endPoint, txnBatch.getCurrentTxnId(), e); } catch (TimeoutException e) { throw new WriteException(endPoint, txnBatch.getCurrentTxnId(), e); } - - // Update Statistics - processSize += event.getBody().length; - eventCounter++; } /** @@ -156,7 +175,13 @@ public Void call() throws InterruptedException, StreamingException { * new TxnBatch if current Txn batch is exhausted */ public void flush(boolean rollToNext) - throws CommitException, TxnBatchException, TxnFailure, InterruptedException { + throws CommitException, TxnBatchException, TxnFailure, InterruptedException, + WriteException { + if(!batch.isEmpty()) { + writeEventBatchToSerializer(); + batch.clear(); + } + //0 Heart beat on TxnBatch if(hearbeatNeeded) { hearbeatNeeded = false; @@ -190,6 +215,7 @@ public void flush(boolean rollToNext) * @throws StreamingException if could not get new Transaction Batch, or switch to next Txn */ public void abort() throws InterruptedException { + batch.clear(); abortTxn(); } @@ -221,6 +247,7 @@ public Void call() throws StreamingException { * @throws InterruptedException */ public void close() throws InterruptedException { + batch.clear(); closeTxnBatch(); closeConnection(); closed = true; @@ -335,7 +362,7 @@ private T timedCall(final CallRunner1 callRunner) throws TimeoutException, InterruptedException, StreamingException { Future future = callTimeoutPool.submit(new Callable() { @Override - public T call() throws StreamingException, InterruptedException { + public T call() throws StreamingException, InterruptedException, Failure { return callRunner.call(); } }); @@ -354,7 +381,7 @@ public T call() throws StreamingException, InterruptedException { sinkCounter.incrementConnectionFailedCount(); Throwable cause = e1.getCause(); if (cause instanceof IOException ) { - throw new StreamingIOFailure("I/O Failure", (IOException) cause); + throw new StreamingException("I/O Failure", (IOException) cause); } else if (cause instanceof StreamingException) { throw (StreamingException) cause; } else if (cause instanceof TimeoutException) { @@ -384,7 +411,7 @@ private interface CallRunner { private interface CallRunner1 { - T call() throws StreamingException, InterruptedException; + T call() throws StreamingException, InterruptedException, Failure; } diff --git a/pom.xml b/pom.xml index aad8be6ad8..6d334c170e 100644 --- a/pom.xml +++ b/pom.xml @@ -51,7 +51,7 @@ limitations under the License. 2.4.0 0.7.0 1.0.0 - 0.13.1 + 0.14.0 2.7.1 2.9.1 From 4e06f6fe7af751672925967ce45feb18a92c4888 Mon Sep 17 00:00:00 2001 From: Roshan Naik Date: Mon, 6 Apr 2015 17:23:15 -0700 Subject: [PATCH 212/341] FLUME-2657: Upgrade to Hive 1.0 (Hari Shreedharan via Roshan Naik) --- pom.xml | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/pom.xml b/pom.xml index 6d334c170e..4a25a69d84 100644 --- a/pom.xml +++ b/pom.xml @@ -51,7 +51,7 @@ limitations under the License. 2.4.0 0.7.0 1.0.0 - 0.14.0 + 1.0.0 2.7.1 2.9.1 From 4d2a34e931554baa1c1b255d95540a46354a521f Mon Sep 17 00:00:00 2001 From: Hari Shreedharan Date: Mon, 6 Apr 2015 18:14:04 -0700 Subject: [PATCH 213/341] FLUME-1934. Spooling Directory Source dies on encountering zero-byte files. (Grant Henke via Hari) --- .../avro/ReliableSpoolingFileEventReader.java | 6 ++-- .../source/TestSpoolDirectorySource.java | 30 +++++++++++++++++++ 2 files changed, 34 insertions(+), 2 deletions(-) diff --git a/flume-ng-core/src/main/java/org/apache/flume/client/avro/ReliableSpoolingFileEventReader.java b/flume-ng-core/src/main/java/org/apache/flume/client/avro/ReliableSpoolingFileEventReader.java index 27e9c1ebdd..d54f415d29 100644 --- a/flume-ng-core/src/main/java/org/apache/flume/client/avro/ReliableSpoolingFileEventReader.java +++ b/flume-ng-core/src/main/java/org/apache/flume/client/avro/ReliableSpoolingFileEventReader.java @@ -252,8 +252,10 @@ public List readEvents(int numEvents) throws IOException { List events = des.readEvents(numEvents); /* It's possible that the last read took us just up to a file boundary. - * If so, try to roll to the next file, if there is one. */ - if (events.isEmpty()) { + * If so, try to roll to the next file, if there is one. + * Loop until events is not empty or there is no next file in case of 0 byte files */ + while (events.isEmpty()) { + logger.info("Last read took us just up to a file boundary. Rolling to the next file, if there is one."); retireCurrentFile(); currentFile = getNextFile(); if (!currentFile.isPresent()) { diff --git a/flume-ng-core/src/test/java/org/apache/flume/source/TestSpoolDirectorySource.java b/flume-ng-core/src/test/java/org/apache/flume/source/TestSpoolDirectorySource.java index 89e7c8c3c5..fe530ff4d7 100644 --- a/flume-ng-core/src/test/java/org/apache/flume/source/TestSpoolDirectorySource.java +++ b/flume-ng-core/src/test/java/org/apache/flume/source/TestSpoolDirectorySource.java @@ -283,4 +283,34 @@ public void testSourceDoesNotDieOnFullChannel() throws Exception { Assert.assertEquals(8, dataOut.size()); source.stop(); } + + @Test + public void testEndWithZeroByteFiles() throws IOException, InterruptedException { + Context context = new Context(); + + File f1 = new File(tmpDir.getAbsolutePath() + "/file1"); + + Files.write("file1line1\n", f1, Charsets.UTF_8); + + File f2 = new File(tmpDir.getAbsolutePath() + "/file2"); + File f3 = new File(tmpDir.getAbsolutePath() + "/file3"); + File f4 = new File(tmpDir.getAbsolutePath() + "/file4"); + + Files.touch(f2); + Files.touch(f3); + Files.touch(f4); + + context.put(SpoolDirectorySourceConfigurationConstants.SPOOL_DIRECTORY, + tmpDir.getAbsolutePath()); + Configurables.configure(source, context); + source.start(); + + // Need better way to ensure all files were processed. + Thread.sleep(5000); + + Assert.assertFalse("Server did not error", source.hasFatalError()); + Assert.assertEquals("One message was read", 1, + source.getSourceCounter().getEventAcceptedCount()); + source.stop(); + } } From cfefda167ea42aa5d68e15ac55bc2d6394c2d48d Mon Sep 17 00:00:00 2001 From: Hari Shreedharan Date: Tue, 7 Apr 2015 13:18:32 -0700 Subject: [PATCH 214/341] FLUME-2654. Clarify error message in MemoryChannel when byte capacity is reached. (Johny Rufus via Hari) --- .../main/java/org/apache/flume/channel/MemoryChannel.java | 8 ++++---- 1 file changed, 4 insertions(+), 4 deletions(-) diff --git a/flume-ng-core/src/main/java/org/apache/flume/channel/MemoryChannel.java b/flume-ng-core/src/main/java/org/apache/flume/channel/MemoryChannel.java index b84a3e0a4d..6575d10b61 100644 --- a/flume-ng-core/src/main/java/org/apache/flume/channel/MemoryChannel.java +++ b/flume-ng-core/src/main/java/org/apache/flume/channel/MemoryChannel.java @@ -120,10 +120,10 @@ protected void doCommit() throws InterruptedException { if(remainingChange < 0) { if(!bytesRemaining.tryAcquire(putByteCounter, keepAlive, TimeUnit.SECONDS)) { - throw new ChannelException("Cannot commit transaction. Heap space " + - "limit of " + byteCapacity + "reached. Please increase heap space" + - " allocated to the channel as the sinks may not be keeping up " + - "with the sources"); + throw new ChannelException("Cannot commit transaction. Byte capacity " + + "allocated to store event body " + byteCapacity * byteCapacitySlotSize + + "reached. Please increase heap space/byte capacity allocated to " + + "the channel as the sinks may not be keeping up with the sources"); } if(!queueRemaining.tryAcquire(-remainingChange, keepAlive, TimeUnit.SECONDS)) { bytesRemaining.release(putByteCounter); From 91ec5794589bf3711cca2a251a511fa360e5ac30 Mon Sep 17 00:00:00 2001 From: Hari Shreedharan Date: Tue, 7 Apr 2015 14:46:10 -0700 Subject: [PATCH 215/341] FLUME-2613. Add support in FileChannelIntegrityTool to remove invalid events from the channel. (Ashish Paliwal via Hari) --- .../apache/flume/channel/file/EventUtils.java | 41 ++++++++ .../flume/channel/file/TestEventUtils.java | 44 +++++++++ .../apache/flume/tools/EventValidator.java | 49 +++++++++ .../flume/tools/FileChannelIntegrityTool.java | 99 ++++++++++++++++++- .../tools/TestFileChannelIntegrityTool.java | 75 ++++++++++++++ 5 files changed, 306 insertions(+), 2 deletions(-) create mode 100644 flume-ng-channels/flume-file-channel/src/main/java/org/apache/flume/channel/file/EventUtils.java create mode 100644 flume-ng-channels/flume-file-channel/src/test/java/org/apache/flume/channel/file/TestEventUtils.java create mode 100644 flume-tools/src/main/java/org/apache/flume/tools/EventValidator.java diff --git a/flume-ng-channels/flume-file-channel/src/main/java/org/apache/flume/channel/file/EventUtils.java b/flume-ng-channels/flume-file-channel/src/main/java/org/apache/flume/channel/file/EventUtils.java new file mode 100644 index 0000000000..ff5242a1fe --- /dev/null +++ b/flume-ng-channels/flume-file-channel/src/main/java/org/apache/flume/channel/file/EventUtils.java @@ -0,0 +1,41 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, + * software distributed under the License is distributed on an + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + * KIND, either express or implied. See the License for the + * specific language governing permissions and limitations + * under the License. + */ + +package org.apache.flume.channel.file; + +import org.apache.flume.Event; + +/** + * + */ +public class EventUtils { + + /** + * Returns the Event encapsulated by a Put wrapper + * + * @param transactionEventRecord TransactionEvent + * @return Event if Put instance is present, null otherwise + */ + public static Event getEventFromTransactionEvent(TransactionEventRecord transactionEventRecord) { + if(transactionEventRecord instanceof Put) { + return ((Put)transactionEventRecord).getEvent(); + } + return null; + } +} diff --git a/flume-ng-channels/flume-file-channel/src/test/java/org/apache/flume/channel/file/TestEventUtils.java b/flume-ng-channels/flume-file-channel/src/test/java/org/apache/flume/channel/file/TestEventUtils.java new file mode 100644 index 0000000000..c72e3f2149 --- /dev/null +++ b/flume-ng-channels/flume-file-channel/src/test/java/org/apache/flume/channel/file/TestEventUtils.java @@ -0,0 +1,44 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, + * software distributed under the License is distributed on an + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + * KIND, either express or implied. See the License for the + * specific language governing permissions and limitations + * under the License. + */ + +package org.apache.flume.channel.file; + +import junit.framework.Assert; +import org.apache.flume.Event; +import org.junit.Test; + +public class TestEventUtils { + + @Test + public void testPutEvent() { + FlumeEvent event = new FlumeEvent(null, new byte[5]); + Put put = new Put(1l, 1l, event); + Event returnEvent = EventUtils.getEventFromTransactionEvent(put); + Assert.assertNotNull(returnEvent); + Assert.assertEquals(5, returnEvent.getBody().length); + } + + @Test + public void testInvalidEvent() { + Take take = new Take(1l, 1l); + Event returnEvent = EventUtils.getEventFromTransactionEvent(take); + Assert.assertNull(returnEvent); + } + +} diff --git a/flume-tools/src/main/java/org/apache/flume/tools/EventValidator.java b/flume-tools/src/main/java/org/apache/flume/tools/EventValidator.java new file mode 100644 index 0000000000..10e677d30c --- /dev/null +++ b/flume-tools/src/main/java/org/apache/flume/tools/EventValidator.java @@ -0,0 +1,49 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, + * software distributed under the License is distributed on an + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + * KIND, either express or implied. See the License for the + * specific language governing permissions and limitations + * under the License. + */ + +package org.apache.flume.tools; + +import org.apache.flume.Event; +import org.apache.flume.conf.Configurable; + +/** + * Event Validator interface to be used for validating Events + * per custom logic + */ +public interface EventValidator { + + /** + * Validate the Event in a application specific manner + * + * @param event Flume Event + * @return true if Event is valid as per App Logic + */ + boolean validateEvent(Event event); + + EventValidator NOOP_VALIDATOR = new EventValidator() { + @Override + public boolean validateEvent(Event event) { + return true; + } + }; + + interface Builder extends Configurable { + EventValidator build(); + } +} diff --git a/flume-tools/src/main/java/org/apache/flume/tools/FileChannelIntegrityTool.java b/flume-tools/src/main/java/org/apache/flume/tools/FileChannelIntegrityTool.java index 1030442ab2..7abb7ebeb8 100644 --- a/flume-tools/src/main/java/org/apache/flume/tools/FileChannelIntegrityTool.java +++ b/flume-tools/src/main/java/org/apache/flume/tools/FileChannelIntegrityTool.java @@ -22,15 +22,21 @@ import org.apache.commons.cli.CommandLineParser; import org.apache.commons.cli.GnuParser; import org.apache.commons.cli.HelpFormatter; +import org.apache.commons.cli.Option; +import org.apache.commons.cli.OptionBuilder; import org.apache.commons.cli.Options; import org.apache.commons.cli.ParseException; +import org.apache.flume.Context; +import org.apache.flume.Event; import org.apache.flume.FlumeException; import org.apache.flume.channel.file.CorruptEventException; +import org.apache.flume.channel.file.EventUtils; import org.apache.flume.channel.file.Log; import org.apache.flume.channel.file.LogFile; import org.apache.flume.channel.file.LogFileV3; import org.apache.flume.channel.file.LogRecord; import org.apache.flume.channel.file.Serialization; +import org.apache.flume.channel.file.TransactionEventRecord; import org.slf4j.Logger; import org.slf4j.LoggerFactory; @@ -39,6 +45,8 @@ import java.io.IOException; import java.util.ArrayList; import java.util.List; +import java.util.Properties; +import java.util.Set; public class FileChannelIntegrityTool implements FlumeTool { public static final Logger LOG = LoggerFactory.getLogger @@ -46,6 +54,15 @@ public class FileChannelIntegrityTool implements FlumeTool { private final List dataDirs = new ArrayList(); + private EventValidator eventValidator = EventValidator.NOOP_VALIDATOR; + + private long totalPutEvents; + private long invalidEvents; + private long eventsWithException; + private long corruptEvents; + private long validEvents; + private long totalChannelEvents; + @Override public void run(String[] args) throws IOException, ParseException { boolean shouldContinue = parseCommandLineOpts(args); @@ -85,12 +102,39 @@ public boolean accept(File dir, String name) { // this will throw a CorruptEventException - so the real logic // is in the catch block below. LogRecord record = reader.next(); + totalChannelEvents++; if (record != null) { - record.getEvent(); + TransactionEventRecord recordEvent = record.getEvent(); + Event event = EventUtils.getEventFromTransactionEvent(recordEvent); + if(event != null) { + totalPutEvents++; + try { + if (!eventValidator.validateEvent(event)) { + if (!fileBackedup) { + Serialization.copyFile(dataFile, new File(dataFile.getParent(), + dataFile.getName() + ".bak")); + fileBackedup = true; + } + invalidEvents++; + updater.markRecordAsNoop(eventPosition); + } else { + validEvents++; + } + } catch (Exception e) { + // OOPS, didn't expected an exception + // considering as failure case + // marking as noop + System.err.println("Encountered Exception while validating event, marking as invalid"); + updater.markRecordAsNoop(eventPosition); + eventsWithException++; + } + } } else { fileDone = true; } } catch (CorruptEventException e) { + corruptEvents++; + totalChannelEvents++; LOG.warn("Corruption found in " + dataFile.toString() + " at " + eventPosition); if (!fileBackedup) { @@ -106,6 +150,7 @@ public boolean accept(File dir, String name) { } } } + printSummary(); } private boolean parseCommandLineOpts(String[] args) throws ParseException { @@ -113,7 +158,17 @@ private boolean parseCommandLineOpts(String[] args) throws ParseException { options .addOption("l", "dataDirs", true, "Comma-separated list of data " + "directories which the tool must verify. This option is mandatory") - .addOption("h", "help", false, "Display help"); + .addOption("h", "help", false, "Display help") + .addOption("e", "eventValidator", true, "Fully Qualified Name of Event Validator Implementation");; + + + Option property = OptionBuilder.withArgName("property=value") + .hasArgs(2) + .withValueSeparator() + .withDescription( "custom properties" ) + .create( "D" ); + + options.addOption(property); CommandLineParser parser = new GnuParser(); CommandLine commandLine = parser.parse(options, args); @@ -137,6 +192,46 @@ private boolean parseCommandLineOpts(String[] args) throws ParseException { dataDirs.add(f); } } + + if(commandLine.hasOption("eventValidator")) { + try { + Class eventValidatorClassName = + (Class)Class.forName( + commandLine.getOptionValue("eventValidator")); + EventValidator.Builder eventValidatorBuilder = eventValidatorClassName.newInstance(); + + // Pass on the configuration parameter + Properties systemProperties = commandLine.getOptionProperties("D"); + Context context = new Context(); + + Set keys = systemProperties.stringPropertyNames(); + for (String key : keys) { + context.put(key, systemProperties.getProperty(key)); + } + eventValidatorBuilder.configure(context); + eventValidator = eventValidatorBuilder.build(); + } catch (Exception e) { + System.err.println(String.format("Could find class %s in lib folder", + commandLine.getOptionValue("eventValidator"))); + e.printStackTrace(); + return false; + } + } return true; } + + /** + * Prints the summary of run. Following information is printed + * + */ + private void printSummary() { + System.out.println("---------- Summary --------------------"); + System.out.println("Number of Events in the Channel = "+totalChannelEvents++); + System.out.println("Number of Put Events Processed = "+totalPutEvents); + System.out.println("Number of Valid Put Events = "+validEvents); + System.out.println("Number of Invalid Put Events = "+invalidEvents); + System.out.println("Number of Put Events that threw Exception during validation = "+eventsWithException); + System.out.println("Number of Corrupt Events = "+corruptEvents); + System.out.println("---------------------------------------"); + } } diff --git a/flume-tools/src/test/java/org/apache/flume/tools/TestFileChannelIntegrityTool.java b/flume-tools/src/test/java/org/apache/flume/tools/TestFileChannelIntegrityTool.java index f24ae561b0..ac4dac4314 100644 --- a/flume-tools/src/test/java/org/apache/flume/tools/TestFileChannelIntegrityTool.java +++ b/flume-tools/src/test/java/org/apache/flume/tools/TestFileChannelIntegrityTool.java @@ -58,6 +58,7 @@ public class TestFileChannelIntegrityTool { private File checkpointDir; private File dataDir; + private static int invalidEvent = 0; @BeforeClass public static void setUpClass() throws Exception{ @@ -96,6 +97,45 @@ public static void tearDownClass() throws Exception { FileUtils.deleteDirectory(origDataDir); } + @Test + public void testFixInvalidRecords() throws Exception { + doTestFixInvalidEvents(false, DummyEventVerifier.Builder.class.getName()); + } + @Test + public void testFixInvalidRecordsWithCheckpoint() throws Exception { + doTestFixInvalidEvents(true, DummyEventVerifier.Builder.class.getName()); + } + + public void doTestFixInvalidEvents(boolean withCheckpoint, String eventHandler) throws Exception { + FileChannelIntegrityTool tool = new FileChannelIntegrityTool(); + tool.run(new String[] {"-l", dataDir.toString(), "-e", eventHandler, "-DvalidatorValue=0"}); + FileChannel channel = new FileChannel(); + channel.setName("channel"); + String cp; + if(withCheckpoint) { + cp = origCheckpointDir.toString(); + } else { + FileUtils.deleteDirectory(checkpointDir); + Assert.assertTrue(checkpointDir.mkdirs()); + cp = checkpointDir.toString(); + } + ctx.put(FileChannelConfiguration.CHECKPOINT_DIR,cp); + ctx.put(FileChannelConfiguration.DATA_DIRS, dataDir.toString()); + channel.configure(ctx); + channel.start(); + Transaction tx = channel.getTransaction(); + tx.begin(); + int i = 0; + while(channel.take() != null) { + i++; + } + tx.commit(); + tx.close(); + channel.stop(); + Assert.assertTrue(invalidEvent != 0); + Assert.assertEquals(25 - invalidEvent, i); + } + @Test public void testFixCorruptRecords() throws Exception { doTestFixCorruptEvents(false); @@ -226,6 +266,12 @@ private static void createDataFiles() throws Exception { Transaction tx = channel.getTransaction(); tx.begin(); for (int i = 0; i < 5; i++) { + if(i % 3 == 0) { + event.getBody()[0] = 0; + invalidEvent++; + } else { + event.getBody()[0] = 1; + } channel.put(event); } tx.commit(); @@ -244,4 +290,33 @@ private static void createDataFiles() throws Exception { .invoke(true)); channel.stop(); } + + public static class DummyEventVerifier implements EventValidator { + + private int value = 0; + + private DummyEventVerifier(int val) { + value = val; + } + + @Override + public boolean validateEvent(Event event) { + return event.getBody()[0] != value; + } + + public static class Builder implements EventValidator.Builder { + + private int binaryValidator = 0; + + @Override + public EventValidator build() { + return new DummyEventVerifier(binaryValidator); + } + + @Override + public void configure(Context context) { + binaryValidator = context.getInteger("validatorValue"); + } + } + } } From c77f1ac469aac712e533cce7748683f1d2cc131a Mon Sep 17 00:00:00 2001 From: Hari Shreedharan Date: Fri, 10 Apr 2015 11:48:20 -0700 Subject: [PATCH 216/341] FLUME-2649. Elasticsearch sink doesn't handle JSON fields correctly (Benjamin Fiorini via Hari) --- .../sink/elasticsearch/ContentBuilderUtil.java | 16 +++++++++------- .../AbstractElasticSearchSinkTest.java | 13 +++++++++---- .../elasticsearch/TestElasticSearchSink.java | 15 ++++++++++++--- 3 files changed, 30 insertions(+), 14 deletions(-) diff --git a/flume-ng-sinks/flume-ng-elasticsearch-sink/src/main/java/org/apache/flume/sink/elasticsearch/ContentBuilderUtil.java b/flume-ng-sinks/flume-ng-elasticsearch-sink/src/main/java/org/apache/flume/sink/elasticsearch/ContentBuilderUtil.java index 70d0b8689a..de0acf4e2e 100644 --- a/flume-ng-sinks/flume-ng-elasticsearch-sink/src/main/java/org/apache/flume/sink/elasticsearch/ContentBuilderUtil.java +++ b/flume-ng-sinks/flume-ng-elasticsearch-sink/src/main/java/org/apache/flume/sink/elasticsearch/ContentBuilderUtil.java @@ -55,18 +55,20 @@ public static void addSimpleField(XContentBuilder builder, String fieldName, public static void addComplexField(XContentBuilder builder, String fieldName, XContentType contentType, byte[] data) throws IOException { - XContentParser parser = null; + XContentParser parser = + XContentFactory.xContent(contentType).createParser(data); + parser.nextToken(); + // Add the field name, but not the value. + builder.field(fieldName); try { - XContentBuilder tmp = jsonBuilder(); - parser = XContentFactory.xContent(contentType).createParser(data); - parser.nextToken(); - tmp.copyCurrentStructure(parser); - builder.field(fieldName, tmp.string()); + // This will add the whole parsed content as the value of the field. + builder.copyCurrentStructure(parser); } catch (JsonParseException ex) { // If we get an exception here the most likely cause is nested JSON that // can't be figured out in the body. At this point just push it through // as is, we have already added the field so don't do it again - addSimpleField(builder, fieldName, data); + builder.endObject(); + builder.field(fieldName, new String(data, charset)); } finally { if (parser != null) { parser.close(); diff --git a/flume-ng-sinks/flume-ng-elasticsearch-sink/src/test/java/org/apache/flume/sink/elasticsearch/AbstractElasticSearchSinkTest.java b/flume-ng-sinks/flume-ng-elasticsearch-sink/src/test/java/org/apache/flume/sink/elasticsearch/AbstractElasticSearchSinkTest.java index 48eafdfd94..2f8fd6dc0d 100644 --- a/flume-ng-sinks/flume-ng-elasticsearch-sink/src/test/java/org/apache/flume/sink/elasticsearch/AbstractElasticSearchSinkTest.java +++ b/flume-ng-sinks/flume-ng-elasticsearch-sink/src/test/java/org/apache/flume/sink/elasticsearch/AbstractElasticSearchSinkTest.java @@ -121,13 +121,13 @@ Channel bindAndStartChannel(ElasticSearchSink fixture) { void assertMatchAllQuery(int expectedHits, Event... events) { assertSearch(expectedHits, performSearch(QueryBuilders.matchAllQuery()), - events); + null, events); } void assertBodyQuery(int expectedHits, Event... events) { // Perform Multi Field Match assertSearch(expectedHits, - performSearch(QueryBuilders.fieldQuery("@message", "event"))); + performSearch(QueryBuilders.fieldQuery("@message", "event")), null); } SearchResponse performSearch(QueryBuilder query) { @@ -135,7 +135,7 @@ SearchResponse performSearch(QueryBuilder query) { .setTypes(DEFAULT_INDEX_TYPE).setQuery(query).execute().actionGet(); } - void assertSearch(int expectedHits, SearchResponse response, Event... events) { + void assertSearch(int expectedHits, SearchResponse response, Map expectedBody, Event... events) { SearchHits hitResponse = response.getHits(); assertEquals(expectedHits, hitResponse.getTotalHits()); @@ -151,7 +151,12 @@ public int compare(SearchHit o1, SearchHit o2) { Event event = events[i]; SearchHit hit = hits[i]; Map source = hit.getSource(); - assertEquals(new String(event.getBody()), source.get("@message")); + if (expectedBody == null) { + assertEquals(new String(event.getBody()), source.get("@message")); + } else { + assertEquals(expectedBody, source.get("@message")); + } } } + } diff --git a/flume-ng-sinks/flume-ng-elasticsearch-sink/src/test/java/org/apache/flume/sink/elasticsearch/TestElasticSearchSink.java b/flume-ng-sinks/flume-ng-elasticsearch-sink/src/test/java/org/apache/flume/sink/elasticsearch/TestElasticSearchSink.java index 3e11726893..78e166548a 100644 --- a/flume-ng-sinks/flume-ng-elasticsearch-sink/src/test/java/org/apache/flume/sink/elasticsearch/TestElasticSearchSink.java +++ b/flume-ng-sinks/flume-ng-elasticsearch-sink/src/test/java/org/apache/flume/sink/elasticsearch/TestElasticSearchSink.java @@ -51,6 +51,7 @@ import org.elasticsearch.common.UUID; import org.elasticsearch.common.io.BytesStream; import org.elasticsearch.common.io.FastByteArrayOutputStream; +import org.elasticsearch.index.query.QueryBuilders; import org.junit.After; import org.junit.Before; import org.junit.Test; @@ -100,7 +101,8 @@ public void shouldIndexComplexJsonEvent() throws Exception { Transaction tx = channel.getTransaction(); tx.begin(); - Event event = EventBuilder.withBody("{\"event\":\"json content\"}".getBytes()); + Event event = EventBuilder.withBody( + "{\"event\":\"json content\",\"num\":1}".getBytes()); channel.put(event); tx.commit(); tx.close(); @@ -110,8 +112,15 @@ public void shouldIndexComplexJsonEvent() throws Exception { client.admin().indices() .refresh(Requests.refreshRequest(timestampedIndexName)).actionGet(); - assertMatchAllQuery(1, event); - assertBodyQuery(1, event); + Map expectedBody = new HashMap(); + expectedBody.put("event", "json content"); + expectedBody.put("num", 1); + + assertSearch(1, + performSearch(QueryBuilders.matchAllQuery()), expectedBody, event); + assertSearch(1, + performSearch(QueryBuilders.fieldQuery("@message.event", "json")), + expectedBody, event); } @Test From c3eff4bffcaf6c952ba13efef456d6a1a40f89cb Mon Sep 17 00:00:00 2001 From: Hari Shreedharan Date: Fri, 10 Apr 2015 12:04:43 -0700 Subject: [PATCH 217/341] FLUME-2664. Site profile fails due to dependency issues (Johny Rufus via Hari) --- pom.xml | 7 +++++++ 1 file changed, 7 insertions(+) diff --git a/pom.xml b/pom.xml index 4a25a69d84..fe7242f4bf 100644 --- a/pom.xml +++ b/pom.xml @@ -388,6 +388,13 @@ limitations under the License. + + org.apache.felix + maven-bundle-plugin + true + true + + maven-javadoc-plugin From b68f5991d45fd4f89471233c2e3318fe252b5788 Mon Sep 17 00:00:00 2001 From: Hari Shreedharan Date: Fri, 10 Apr 2015 12:24:57 -0700 Subject: [PATCH 218/341] FLUME-2635. Documentation for zookeeper dynamic configuration in flume (Ashish Paliwal via Hari) --- flume-ng-doc/sphinx/FlumeUserGuide.rst | 25 +++++++++++++++++++++++++ 1 file changed, 25 insertions(+) diff --git a/flume-ng-doc/sphinx/FlumeUserGuide.rst b/flume-ng-doc/sphinx/FlumeUserGuide.rst index 576cba1c6c..9a532517f6 100644 --- a/flume-ng-doc/sphinx/FlumeUserGuide.rst +++ b/flume-ng-doc/sphinx/FlumeUserGuide.rst @@ -234,6 +234,31 @@ The original Flume terminal will output the event in a log message. Congratulations - you've successfully configured and deployed a Flume agent! Subsequent sections cover agent configuration in much more detail. + +Zookeeper based Configuration +~~~~~~~~~~~~~~~~~~~~~~~~~~~~~ + +Flume supports Agent configurations via Zookeeper. *This is an experimental feature.* The configuration file needs to be uploaded +in the Zookeeper, under a configurable prefix. The configuration file is stored in Zookeeper Node data. +Following is how the Zookeeper Node tree would look like for agents a1 and a2 + +.. code-block:: properties + + - /flume + |- /a1 [Agent config file] + |- /a2 [Agent config file] + +Once the configuration file is uploaded, start the agent with following options + + $ bin/flume-ng agent --conf conf -z zkhost:2181,zkhost1:2181 -p /flume --name a1 -Dflume.root.logger=INFO,console + +================== ================ ========================================================================= +Argument Name Default Description +================== ================ ========================================================================= +**z** -- Zookeeper connection string. Comma separated list of hostname:port +**p** /flume Base Path in Zookeeper to store Agent configurations +================== ================ ========================================================================= + Installing third-party plugins ~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~ From fc03456a1e4b1059b7fc6f29d96f9d047e28badb Mon Sep 17 00:00:00 2001 From: Hari Shreedharan Date: Fri, 10 Apr 2015 12:35:11 -0700 Subject: [PATCH 219/341] FLUME-2645. ipFilter.rules property name is wrong (Johny Rufus via Hari) --- flume-ng-doc/sphinx/FlumeUserGuide.rst | 8 ++++---- 1 file changed, 4 insertions(+), 4 deletions(-) diff --git a/flume-ng-doc/sphinx/FlumeUserGuide.rst b/flume-ng-doc/sphinx/FlumeUserGuide.rst index 9a532517f6..78b139efda 100644 --- a/flume-ng-doc/sphinx/FlumeUserGuide.rst +++ b/flume-ng-doc/sphinx/FlumeUserGuide.rst @@ -732,7 +732,7 @@ keystore-password -- The password for the Java keystore. Requi keystore-type JKS The type of the Java keystore. This can be "JKS" or "PKCS12". exclude-protocols SSLv3 Space-separated list of SSL/TLS protocols to exclude. SSLv3 will always be excluded in addition to the protocols specified. ipFilter false Set this to true to enable ipFiltering for netty -ipFilter.rules -- Define N netty ipFilter pattern rules with this config. +ipFilterRules -- Define N netty ipFilter pattern rules with this config. ================== ================ =================================================== Example for agent named a1: @@ -746,15 +746,15 @@ Example for agent named a1: a1.sources.r1.bind = 0.0.0.0 a1.sources.r1.port = 4141 -Example of ipFilter.rules +Example of ipFilterRules -ipFilter.rules defines N netty ipFilters separated by a comma a pattern rule must be in this format. +ipFilterRules defines N netty ipFilters separated by a comma a pattern rule must be in this format. <'allow' or deny>:<'ip' or 'name' for computer name>: or allow/deny:ip/name:pattern -example: ipFilter.rules=allow:ip:127.*,allow:name:localhost,deny:ip:* +example: ipFilterRules=allow:ip:127.*,allow:name:localhost,deny:ip:* Note that the first rule to match will apply as the example below shows from a client on the localhost From be4ae294ca549648f785e7eea7564ee95112130a Mon Sep 17 00:00:00 2001 From: Hari Shreedharan Date: Mon, 13 Apr 2015 15:57:30 -0700 Subject: [PATCH 220/341] FLUME-2595. Add option to checkpoint on file channel shutdown (Roshan Naik via Hari) --- .../flume/channel/file/FileChannel.java | 5 ++ .../file/FileChannelConfiguration.java | 3 +- .../org/apache/flume/channel/file/Log.java | 22 ++++++- .../apache/flume/channel/file/TestLog.java | 35 ++++++++++- flume-ng-doc/sphinx/FlumeUserGuide.rst | 1 + .../tools/TestFileChannelIntegrityTool.java | 60 ++++++++++++------- 6 files changed, 102 insertions(+), 24 deletions(-) diff --git a/flume-ng-channels/flume-file-channel/src/main/java/org/apache/flume/channel/file/FileChannel.java b/flume-ng-channels/flume-file-channel/src/main/java/org/apache/flume/channel/file/FileChannel.java index 61c353aff9..ed2b996f88 100644 --- a/flume-ng-channels/flume-file-channel/src/main/java/org/apache/flume/channel/file/FileChannel.java +++ b/flume-ng-channels/flume-file-channel/src/main/java/org/apache/flume/channel/file/FileChannel.java @@ -100,6 +100,7 @@ public class FileChannel extends BasicChannelSemantics { private boolean compressBackupCheckpoint; private boolean fsyncPerTransaction; private int fsyncInterval; + private boolean checkpointOnClose = true; @Override public synchronized void setName(String name) { @@ -251,6 +252,9 @@ public void configure(Context context) { fsyncInterval = context.getInteger(FileChannelConfiguration .FSYNC_INTERVAL, FileChannelConfiguration.DEFAULT_FSYNC_INTERVAL); + checkpointOnClose = context.getBoolean(FileChannelConfiguration + .CHKPT_ONCLOSE, FileChannelConfiguration.DEFAULT_CHKPT_ONCLOSE); + if(queueRemaining == null) { queueRemaining = new Semaphore(capacity, true); } @@ -286,6 +290,7 @@ public synchronized void start() { builder.setBackupCheckpointDir(backupCheckpointDir); builder.setFsyncPerTransaction(fsyncPerTransaction); builder.setFsyncInterval(fsyncInterval); + builder.setCheckpointOnClose(checkpointOnClose); log = builder.build(); log.replay(); open = true; diff --git a/flume-ng-channels/flume-file-channel/src/main/java/org/apache/flume/channel/file/FileChannelConfiguration.java b/flume-ng-channels/flume-file-channel/src/main/java/org/apache/flume/channel/file/FileChannelConfiguration.java index f8c037884d..5c3c48ff7d 100644 --- a/flume-ng-channels/flume-file-channel/src/main/java/org/apache/flume/channel/file/FileChannelConfiguration.java +++ b/flume-ng-channels/flume-file-channel/src/main/java/org/apache/flume/channel/file/FileChannelConfiguration.java @@ -98,5 +98,6 @@ public class FileChannelConfiguration { public static final String FSYNC_INTERVAL = "fsyncInterval"; public static final int DEFAULT_FSYNC_INTERVAL = 5; // seconds. - + public static final String CHKPT_ONCLOSE = "checkpointOnClose"; + public static final Boolean DEFAULT_CHKPT_ONCLOSE = true; } diff --git a/flume-ng-channels/flume-file-channel/src/main/java/org/apache/flume/channel/file/Log.java b/flume-ng-channels/flume-file-channel/src/main/java/org/apache/flume/channel/file/Log.java index 0e9171e0ea..247c287930 100644 --- a/flume-ng-channels/flume-file-channel/src/main/java/org/apache/flume/channel/file/Log.java +++ b/flume-ng-channels/flume-file-channel/src/main/java/org/apache/flume/channel/file/Log.java @@ -128,6 +128,7 @@ public class Log { private final boolean fsyncPerTransaction; private final int fsyncInterval; + private final boolean checkpointOnClose; private int readCount; private int putCount; @@ -158,6 +159,8 @@ static class Builder { private boolean fsyncPerTransaction = true; private int fsyncInterval; + private boolean checkpointOnClose = true; + boolean isFsyncPerTransaction() { return fsyncPerTransaction; } @@ -254,13 +257,18 @@ Builder setBackupCheckpointDir(File backupCheckpointDir) { return this; } + Builder setCheckpointOnClose(boolean enableCheckpointOnClose) { + this.checkpointOnClose = enableCheckpointOnClose; + return this; + } + Log build() throws IOException { return new Log(bCheckpointInterval, bMaxFileSize, bQueueCapacity, bUseDualCheckpoints, bCompressBackupCheckpoint,bCheckpointDir, bBackupCheckpointDir, bName, useLogReplayV1, useFastReplay, bMinimumRequiredSpace, bEncryptionKeyProvider, bEncryptionKeyAlias, bEncryptionCipherProvider, bUsableSpaceRefreshInterval, - fsyncPerTransaction, fsyncInterval, bLogDirs); + fsyncPerTransaction, fsyncInterval, checkpointOnClose, bLogDirs); } } @@ -272,7 +280,7 @@ private Log(long checkpointInterval, long maxFileSize, int queueCapacity, @Nullable String encryptionKeyAlias, @Nullable String encryptionCipherProvider, long usableSpaceRefreshInterval, boolean fsyncPerTransaction, - int fsyncInterval, File... logDirs) + int fsyncInterval, boolean checkpointOnClose, File... logDirs) throws IOException { Preconditions.checkArgument(checkpointInterval > 0, "checkpointInterval <= 0"); @@ -352,6 +360,8 @@ private Log(long checkpointInterval, long maxFileSize, int queueCapacity, this.logDirs = logDirs; this.fsyncPerTransaction = fsyncPerTransaction; this.fsyncInterval = fsyncInterval; + this.checkpointOnClose = checkpointOnClose; + logFiles = new AtomicReferenceArray(this.logDirs.length); workerExecutor = Executors.newSingleThreadScheduledExecutor(new ThreadFactoryBuilder().setNameFormat("Log-BackgroundWorker-" + name) @@ -791,6 +801,14 @@ void close() throws IOException{ lockExclusive(); try { open = false; + try { + if(checkpointOnClose) { + writeCheckpoint(true); // do this before acquiring exclusive lock + } + } catch (Exception err) { + LOGGER.warn("Failed creating checkpoint on close of channel " + channelNameDescriptor + + "Replay will take longer next time channel is started.", err); + } shutdownWorker(); if (logFiles != null) { for (int index = 0; index < logFiles.length(); index++) { diff --git a/flume-ng-channels/flume-file-channel/src/test/java/org/apache/flume/channel/file/TestLog.java b/flume-ng-channels/flume-file-channel/src/test/java/org/apache/flume/channel/file/TestLog.java index f7f1afa3b2..801d925956 100644 --- a/flume-ng-channels/flume-file-channel/src/test/java/org/apache/flume/channel/file/TestLog.java +++ b/flume-ng-channels/flume-file-channel/src/test/java/org/apache/flume/channel/file/TestLog.java @@ -22,6 +22,8 @@ import java.io.File; import java.io.FileOutputStream; import java.io.IOException; +import java.nio.channels.*; +import java.util.Collection; import java.util.List; import org.apache.commons.io.FileUtils; @@ -35,6 +37,8 @@ import com.google.common.collect.Lists; import com.google.common.io.Files; +import javax.ws.rs.Path; + public class TestLog { private static final Logger LOGGER = LoggerFactory.getLogger(TestLog.class); private static final long MAX_FILE_SIZE = 1000; @@ -56,7 +60,7 @@ public void setup() throws IOException { } log = new Log.Builder().setCheckpointInterval(1L).setMaxFileSize( MAX_FILE_SIZE).setQueueSize(CAPACITY).setCheckpointDir( - checkpointDir).setLogDirs(dataDirs) + checkpointDir).setLogDirs(dataDirs).setCheckpointOnClose(false) .setChannelName("testlog").build(); log.replay(); } @@ -465,6 +469,34 @@ public void testCachedFSUsableSpace() throws Exception { Long.MAX_VALUE - 1L); } + @Test + public void testCheckpointOnClose() throws Exception { + log.close(); + log = new Log.Builder().setCheckpointInterval(1L).setMaxFileSize( + MAX_FILE_SIZE).setQueueSize(CAPACITY).setCheckpointDir( + checkpointDir).setLogDirs(dataDirs).setCheckpointOnClose(true) + .setChannelName("testLog").build(); + log.replay(); + + + // 1 Write One Event + FlumeEvent eventIn = TestUtils.newPersistableEvent(); + log.put(transactionID, eventIn); + log.commitPut(transactionID); + + // 2 Check state of checkpoint before close + File checkPointMetaFile = + FileUtils.listFiles(checkpointDir,new String[]{"meta"},false).iterator().next(); + long before = FileUtils.checksumCRC32( checkPointMetaFile ); + + // 3 Close Log + log.close(); + + // 4 Verify that checkpoint was modified on close + long after = FileUtils.checksumCRC32( checkPointMetaFile ); + Assert.assertFalse( before == after ); + } + private void takeAndVerify(FlumeEventPointer eventPointerIn, FlumeEvent eventIn) throws IOException, InterruptedException, NoopRecordException, CorruptEventException { @@ -479,4 +511,5 @@ private void takeAndVerify(FlumeEventPointer eventPointerIn, Assert.assertEquals(eventIn.getHeaders(), eventOut.getHeaders()); Assert.assertArrayEquals(eventIn.getBody(), eventOut.getBody()); } + } diff --git a/flume-ng-doc/sphinx/FlumeUserGuide.rst b/flume-ng-doc/sphinx/FlumeUserGuide.rst index 78b139efda..43ca5db2b4 100644 --- a/flume-ng-doc/sphinx/FlumeUserGuide.rst +++ b/flume-ng-doc/sphinx/FlumeUserGuide.rst @@ -2638,6 +2638,7 @@ capacity 1000000 keep-alive 3 Amount of time (in sec) to wait for a put operation use-log-replay-v1 false Expert: Use old replay logic use-fast-replay false Expert: Replay without using queue +checkpointOnClose true Controls if a checkpoint is created when the channel is closed. Creating a checkpoint on close speeds up subsequent startup of the file channel by avoiding replay. encryption.activeKey -- Key name used to encrypt new data encryption.cipherProvider -- Cipher provider type, supported types: AESCTRNOPADDING encryption.keyProvider -- Key provider type, supported types: JCEKSFILE diff --git a/flume-tools/src/test/java/org/apache/flume/tools/TestFileChannelIntegrityTool.java b/flume-tools/src/test/java/org/apache/flume/tools/TestFileChannelIntegrityTool.java index ac4dac4314..a11126dfdd 100644 --- a/flume-tools/src/test/java/org/apache/flume/tools/TestFileChannelIntegrityTool.java +++ b/flume-tools/src/test/java/org/apache/flume/tools/TestFileChannelIntegrityTool.java @@ -97,10 +97,21 @@ public static void tearDownClass() throws Exception { FileUtils.deleteDirectory(origDataDir); } + @Test + public void testFixCorruptRecordsWithCheckpoint() throws Exception { + doTestFixCorruptEvents(true); + } + + @Test + public void testFixCorruptRecords() throws Exception { + doTestFixCorruptEvents(false); + } + @Test public void testFixInvalidRecords() throws Exception { doTestFixInvalidEvents(false, DummyEventVerifier.Builder.class.getName()); } + @Test public void testFixInvalidRecordsWithCheckpoint() throws Exception { doTestFixInvalidEvents(true, DummyEventVerifier.Builder.class.getName()); @@ -111,15 +122,24 @@ public void doTestFixInvalidEvents(boolean withCheckpoint, String eventHandler) tool.run(new String[] {"-l", dataDir.toString(), "-e", eventHandler, "-DvalidatorValue=0"}); FileChannel channel = new FileChannel(); channel.setName("channel"); - String cp; - if(withCheckpoint) { - cp = origCheckpointDir.toString(); + if (withCheckpoint) { + File[] cpFiles = origCheckpointDir.listFiles(new FilenameFilter() { + @Override + public boolean accept(File dir, String name) { + if (name.contains("lock") || name.contains("queueset")) { + return false; + } + return true; + } + }); + for (File cpFile : cpFiles) { + Serialization.copyFile(cpFile, new File(checkpointDir, cpFile.getName())); + } } else { FileUtils.deleteDirectory(checkpointDir); Assert.assertTrue(checkpointDir.mkdirs()); - cp = checkpointDir.toString(); } - ctx.put(FileChannelConfiguration.CHECKPOINT_DIR,cp); + ctx.put(FileChannelConfiguration.CHECKPOINT_DIR, checkpointDir.toString()); ctx.put(FileChannelConfiguration.DATA_DIRS, dataDir.toString()); channel.configure(ctx); channel.start(); @@ -136,15 +156,6 @@ public void doTestFixInvalidEvents(boolean withCheckpoint, String eventHandler) Assert.assertEquals(25 - invalidEvent, i); } - @Test - public void testFixCorruptRecords() throws Exception { - doTestFixCorruptEvents(false); - } - @Test - public void testFixCorruptRecordsWithCheckpoint() throws Exception { - doTestFixCorruptEvents(true); - } - public void doTestFixCorruptEvents(boolean withCheckpoint) throws Exception { Set corruptFiles = new HashSet(); File[] files = dataDir.listFiles(new FilenameFilter() { @@ -193,18 +204,27 @@ public boolean accept(File dir, String name) { } FileChannelIntegrityTool tool = new FileChannelIntegrityTool(); - tool.run(new String[] {"-l", dataDir.toString()}); + tool.run(new String[]{"-l", dataDir.toString()}); FileChannel channel = new FileChannel(); channel.setName("channel"); - String cp; - if(withCheckpoint) { - cp = origCheckpointDir.toString(); + if (withCheckpoint) { + File[] cpFiles = origCheckpointDir.listFiles(new FilenameFilter() { + @Override + public boolean accept(File dir, String name) { + if (name.contains("lock") || name.contains("queueset")) { + return false; + } + return true; + } + }); + for (File cpFile : cpFiles) { + Serialization.copyFile(cpFile, new File(checkpointDir, cpFile.getName())); + } } else { FileUtils.deleteDirectory(checkpointDir); Assert.assertTrue(checkpointDir.mkdirs()); - cp = checkpointDir.toString(); } - ctx.put(FileChannelConfiguration.CHECKPOINT_DIR,cp); + ctx.put(FileChannelConfiguration.CHECKPOINT_DIR, checkpointDir.toString()); ctx.put(FileChannelConfiguration.DATA_DIRS, dataDir.toString()); channel.configure(ctx); channel.start(); From a508d953162a2b5e771ce5c3c8ee40e02c3fe3c7 Mon Sep 17 00:00:00 2001 From: Hari Shreedharan Date: Tue, 14 Apr 2015 13:34:15 -0700 Subject: [PATCH 221/341] FLUME-2668. Document SecureThriftRpcClient/SecureRpcClientFactory in Flume Developer Guide (Johny Rufus via Hari) --- flume-ng-doc/sphinx/FlumeDeveloperGuide.rst | 147 ++++++++++++++++++-- 1 file changed, 133 insertions(+), 14 deletions(-) diff --git a/flume-ng-doc/sphinx/FlumeDeveloperGuide.rst b/flume-ng-doc/sphinx/FlumeDeveloperGuide.rst index e3b60e6701..f948778e63 100644 --- a/flume-ng-doc/sphinx/FlumeDeveloperGuide.rst +++ b/flume-ng-doc/sphinx/FlumeDeveloperGuide.rst @@ -277,6 +277,116 @@ properties: request-timeout = 20000 # Must be >=1000 (default: 20000) +Secure RPC client - Thrift +'''''''''''''''''''''''''' + +As of Flume 1.6.0, Thrift source and sink supports kerberos based authentication. +The client needs to use the getThriftInstance method of ``SecureRpcClientFactory`` +to get hold of a ``SecureThriftRpcClient``. ``SecureThriftRpcClient`` extends +``ThriftRpcClient`` which implements the ``RpcClient`` interface. The kerberos +authentication module resides in flume-ng-auth module which is +required in classpath, when using the ``SecureRpcClientFactory``. Both the client +principal and the client keytab should be passed in as parameters through the +properties and they reflect the credentials of the client to authenticate +against the kerberos KDC. In addition, the server principal of the destination +Thrift source to which this client is connecting to, should also be provided. +The following example shows how to use the ``SecureRpcClientFactory`` +within a user's data-generating application: + +.. code-block:: java + + import org.apache.flume.Event; + import org.apache.flume.EventDeliveryException; + import org.apache.flume.event.EventBuilder; + import org.apache.flume.api.SecureRpcClientFactory; + import org.apache.flume.api.RpcClientConfigurationConstants; + import org.apache.flume.api.RpcClient; + import java.nio.charset.Charset; + import java.util.Properties; + + public class MyApp { + public static void main(String[] args) { + MySecureRpcClientFacade client = new MySecureRpcClientFacade(); + // Initialize client with the remote Flume agent's host, port + Properties props = new Properties(); + props.setProperty(RpcClientConfigurationConstants.CONFIG_CLIENT_TYPE, "thrift"); + props.setProperty("hosts", "h1"); + props.setProperty("hosts.h1", "client.example.org"+":"+ String.valueOf(41414)); + + // Initialize client with the kerberos authentication related properties + props.setProperty("kerberos", "true"); + props.setProperty("client-principal", "flumeclient/client.example.org@EXAMPLE.ORG"); + props.setProperty("client-keytab", "/tmp/flumeclient.keytab"); + props.setProperty("server-principal", "flume/server.example.org@EXAMPLE.ORG"); + client.init(props); + + // Send 10 events to the remote Flume agent. That agent should be + // configured to listen with an AvroSource. + String sampleData = "Hello Flume!"; + for (int i = 0; i < 10; i++) { + client.sendDataToFlume(sampleData); + } + + client.cleanUp(); + } + } + + class MySecureRpcClientFacade { + private RpcClient client; + private Properties properties; + + public void init(Properties properties) { + // Setup the RPC connection + this.properties = properties; + // Create the ThriftSecureRpcClient instance by using SecureRpcClientFactory + this.client = SecureRpcClientFactory.getThriftInstance(properties); + } + + public void sendDataToFlume(String data) { + // Create a Flume Event object that encapsulates the sample data + Event event = EventBuilder.withBody(data, Charset.forName("UTF-8")); + + // Send the event + try { + client.append(event); + } catch (EventDeliveryException e) { + // clean up and recreate the client + client.close(); + client = null; + client = SecureRpcClientFactory.getThriftInstance(properties); + } + } + + public void cleanUp() { + // Close the RPC connection + client.close(); + } + } + +The remote ``ThriftSource`` should be started in kerberos mode. +Below is an example Flume agent configuration that's waiting for a connection +from MyApp: + +.. code-block:: properties + + a1.channels = c1 + a1.sources = r1 + a1.sinks = k1 + + a1.channels.c1.type = memory + + a1.sources.r1.channels = c1 + a1.sources.r1.type = thrift + a1.sources.r1.bind = 0.0.0.0 + a1.sources.r1.port = 41414 + a1.sources.r1.kerberos = true + a1.sources.r1.agent-principal = flume/server.example.org@EXAMPLE.ORG + a1.sources.r1.agent-keytab = /tmp/flume.keytab + + + a1.sinks.k1.channel = c1 + a1.sinks.k1.type = logger + Failover Client ''''''''''''''' @@ -459,20 +569,29 @@ full Agent. The following is an exhaustive list of configration options: Required properties are in **bold**. -==================== ================ ============================================== -Property Name Default Description -==================== ================ ============================================== -source.type embedded The only available source is the embedded source. -**channel.type** -- Either ``memory`` or ``file`` which correspond to MemoryChannel and FileChannel respectively. -channel.* -- Configuration options for the channel type requested, see MemoryChannel or FileChannel user guide for an exhaustive list. -**sinks** -- List of sink names -**sink.type** -- Property name must match a name in the list of sinks. Value must be ``avro`` -sink.* -- Configuration options for the sink. See AvroSink user guide for an exhaustive list, however note AvroSink requires at least hostname and port. -**processor.type** -- Either ``failover`` or ``load_balance`` which correspond to FailoverSinksProcessor and LoadBalancingSinkProcessor respectively. -processor.* -- Configuration options for the sink processor selected. See FailoverSinksProcessor and LoadBalancingSinkProcessor user guide for an exhaustive list. -source.interceptors -- Space-separated list of interceptors -source.interceptors.* -- Configuration options for individual interceptors specified in the source.interceptors property -==================== ================ ============================================== +===================== ================ ====================================================================== +Property Name Default Description +===================== ================ ====================================================================== +source.type embedded The only available source is the embedded source. +**channel.type** -- Either ``memory`` or ``file`` which correspond + to MemoryChannel and FileChannel respectively. +channel.* -- Configuration options for the channel type requested, + see MemoryChannel or FileChannel user guide for an exhaustive list. +**sinks** -- List of sink names +**sink.type** -- Property name must match a name in the list of sinks. + Value must be ``avro`` +sink.* -- Configuration options for the sink. + See AvroSink user guide for an exhaustive list, + however note AvroSink requires at least hostname and port. +**processor.type** -- Either ``failover`` or ``load_balance`` which correspond + to FailoverSinksProcessor and LoadBalancingSinkProcessor respectively. +processor.* -- Configuration options for the sink processor selected. + See FailoverSinksProcessor and LoadBalancingSinkProcessor + user guide for an exhaustive list. +source.interceptors -- Space-separated list of interceptors +source.interceptors.* -- Configuration options for individual interceptors + specified in the source.interceptors property +===================== ================ ====================================================================== Below is an example of how to use the agent: From 94f568b96afc79d7850de64e47dead1c7352cef9 Mon Sep 17 00:00:00 2001 From: Hari Shreedharan Date: Tue, 14 Apr 2015 13:41:01 -0700 Subject: [PATCH 222/341] FLUME-2148. Windows : Add flume-env.ps1 (Roshan Naik via Hari) --- conf/flume-env.ps1.template | 23 +++++++++++++++++++++++ 1 file changed, 23 insertions(+) create mode 100644 conf/flume-env.ps1.template diff --git a/conf/flume-env.ps1.template b/conf/flume-env.ps1.template new file mode 100644 index 0000000000..8bf535ad00 --- /dev/null +++ b/conf/flume-env.ps1.template @@ -0,0 +1,23 @@ +# Licensed to the Apache Software Foundation (ASF) under one +# or more contributor license agreements. See the NOTICE file +# distributed with this work for additional information +# regarding copyright ownership. The ASF licenses this file +# to you under the Apache License, Version 2.0 (the +# "License"); you may not use this file except in compliance +# with the License. You may obtain a copy of the License at +# +# http://www.apache.org/licenses/LICENSE-2.0 +# +# Unless required by applicable law or agreed to in writing, software +# distributed under the License is distributed on an "AS IS" BASIS, +# WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +# See the License for the specific language governing permissions and +# limitations under the License. + + +# Give Flume more memory and pre-allocate, enable remote monitoring via JMX +$JAVA_OPTS="-Xms100m -Xmx200m -Dcom.sun.management.jmxremote" + +# Foll. classpath will be included in Flume's classpath. +# Note that the Flume conf directory is always included in the classpath. +$FLUME_CLASSPATH="" # Example: "path1;path2;path3" From 4188e4dedb4af0792bfac3086e3b2a7d415f2248 Mon Sep 17 00:00:00 2001 From: Hari Shreedharan Date: Tue, 14 Apr 2015 16:21:26 -0700 Subject: [PATCH 223/341] FLUME-2665. Update documentation for hdfs.closeTries based on FLUME-2586 (Johny Rufus via Hari) --- flume-ng-doc/sphinx/FlumeUserGuide.rst | 6 +++--- 1 file changed, 3 insertions(+), 3 deletions(-) diff --git a/flume-ng-doc/sphinx/FlumeUserGuide.rst b/flume-ng-doc/sphinx/FlumeUserGuide.rst index 43ca5db2b4..8d91cec0a3 100644 --- a/flume-ng-doc/sphinx/FlumeUserGuide.rst +++ b/flume-ng-doc/sphinx/FlumeUserGuide.rst @@ -1728,10 +1728,10 @@ hdfs.roundValue 1 Rounded down to the highest multiple of th hdfs.roundUnit second The unit of the round down value - ``second``, ``minute`` or ``hour``. hdfs.timeZone Local Time Name of the timezone that should be used for resolving the directory path, e.g. America/Los_Angeles. hdfs.useLocalTimeStamp false Use the local time (instead of the timestamp from the event header) while replacing the escape sequences. -hdfs.closeTries 0 Number of times the sink must try to close a file. If set to 1, this sink will not re-try a failed close +hdfs.closeTries 0 Number of times the sink must try renaming a file, after initiating a close attempt. If set to 1, this sink will not re-try a failed rename (due to, for example, NameNode or DataNode failure), and may leave the file in an open state with a .tmp extension. - If set to 0, the sink will try to close the file until the file is eventually closed - (there is no limit on the number of times it would try). + If set to 0, the sink will try to rename the file until the file is eventually renamed (there is no limit on the number of times it would try). + The file may still remain open if the close call fails but the data will be intact and in this case, the file will be closed only after a Flume restart. hdfs.retryInterval 180 Time in seconds between consecutive attempts to close a file. Each close call costs multiple RPC round-trips to the Namenode, so setting this too low can cause a lot of load on the name node. If set to 0 or less, the sink will not attempt to close the file if the first attempt fails, and may leave the file open or with a ".tmp" extension. From 275329ca6c5ebd5212879696fcd0d19f691c1b60 Mon Sep 17 00:00:00 2001 From: Hari Shreedharan Date: Wed, 15 Apr 2015 14:54:44 -0700 Subject: [PATCH 224/341] FLUME-2670. Modify dependencies to not pull hadoop/hive related jars (Johny Rufus via Hari) --- flume-ng-auth/pom.xml | 1 + flume-ng-sinks/flume-dataset-sink/pom.xml | 2 ++ 2 files changed, 3 insertions(+) diff --git a/flume-ng-auth/pom.xml b/flume-ng-auth/pom.xml index 292731dcfb..1f0ffbace1 100644 --- a/flume-ng-auth/pom.xml +++ b/flume-ng-auth/pom.xml @@ -65,6 +65,7 @@ limitations under the License. org.apache.hadoop ${hadoop.common.artifact.id} + true diff --git a/flume-ng-sinks/flume-dataset-sink/pom.xml b/flume-ng-sinks/flume-dataset-sink/pom.xml index 92f7021655..ae6a93e797 100644 --- a/flume-ng-sinks/flume-dataset-sink/pom.xml +++ b/flume-ng-sinks/flume-dataset-sink/pom.xml @@ -109,11 +109,13 @@ limitations under the License. org.apache.hive hive-exec + true org.apache.hive hive-metastore + true From 364752a43f8ad7f85fd232351a179db017cd2265 Mon Sep 17 00:00:00 2001 From: Hari Shreedharan Date: Wed, 15 Apr 2015 15:13:56 -0700 Subject: [PATCH 225/341] FLUME-2673. Remove unused import in TestLog (Johny Rufus via Hari) --- .../src/test/java/org/apache/flume/channel/file/TestLog.java | 2 -- 1 file changed, 2 deletions(-) diff --git a/flume-ng-channels/flume-file-channel/src/test/java/org/apache/flume/channel/file/TestLog.java b/flume-ng-channels/flume-file-channel/src/test/java/org/apache/flume/channel/file/TestLog.java index 801d925956..b1f59cdb22 100644 --- a/flume-ng-channels/flume-file-channel/src/test/java/org/apache/flume/channel/file/TestLog.java +++ b/flume-ng-channels/flume-file-channel/src/test/java/org/apache/flume/channel/file/TestLog.java @@ -37,8 +37,6 @@ import com.google.common.collect.Lists; import com.google.common.io.Files; -import javax.ws.rs.Path; - public class TestLog { private static final Logger LOGGER = LoggerFactory.getLogger(TestLog.class); private static final long MAX_FILE_SIZE = 1000; From 019ab5207f2d2f0e7b9113b855b37d20dce664bf Mon Sep 17 00:00:00 2001 From: Hari Shreedharan Date: Wed, 15 Apr 2015 16:17:25 -0700 Subject: [PATCH 226/341] FLUME-2658. Upgrade to Hbase 1.0 (Roshan via Hari) --- flume-ng-dist/pom.xml | 8 ++++---- flume-ng-sinks/flume-hdfs-sink/pom.xml | 10 +++++----- flume-ng-sinks/flume-hive-sink/pom.xml | 10 +++++----- flume-ng-sinks/flume-ng-hbase-sink/pom.xml | 10 +++++----- flume-ng-sinks/pom.xml | 10 +++++----- flume-tools/pom.xml | 4 ++-- pom.xml | 12 ++++++------ 7 files changed, 32 insertions(+), 32 deletions(-) diff --git a/flume-ng-dist/pom.xml b/flume-ng-dist/pom.xml index 9f7c4f63ec..c1a7a1b9fd 100644 --- a/flume-ng-dist/pom.xml +++ b/flume-ng-dist/pom.xml @@ -68,7 +68,7 @@ hadoop-2 - hadoop.profile + flume.hadoop.profile 2 @@ -84,11 +84,11 @@ - hbase-98 + hbase-1 - hadoop.profile - hbase-98 + flume.hadoop.profile + hbase-1 diff --git a/flume-ng-sinks/flume-hdfs-sink/pom.xml b/flume-ng-sinks/flume-hdfs-sink/pom.xml index 83f8bec62e..f52beae259 100644 --- a/flume-ng-sinks/flume-hdfs-sink/pom.xml +++ b/flume-ng-sinks/flume-hdfs-sink/pom.xml @@ -108,7 +108,7 @@ limitations under the License. hadoop-1.0 - !hadoop.profile + !flume.hadoop.profile @@ -134,7 +134,7 @@ limitations under the License. hadoop-2 - hadoop.profile + flume.hadoop.profile 2 @@ -162,11 +162,11 @@ limitations under the License. - hbase-98 + hbase-1 - hadoop.profile - hbase-98 + flume.hadoop.profile + hbase-1 diff --git a/flume-ng-sinks/flume-hive-sink/pom.xml b/flume-ng-sinks/flume-hive-sink/pom.xml index e5f673a608..9e3ab26c32 100644 --- a/flume-ng-sinks/flume-hive-sink/pom.xml +++ b/flume-ng-sinks/flume-hive-sink/pom.xml @@ -44,7 +44,7 @@ limitations under the License. hadoop-1.0 - !hadoop.profile + !flume.hadoop.profile @@ -61,7 +61,7 @@ limitations under the License. hadoop-2 - hadoop.profile + flume.hadoop.profile 2 @@ -84,11 +84,11 @@ limitations under the License. - hbase-98 + hbase-1 - hadoop.profile - hbase-98 + flume.hadoop.profile + hbase-1 diff --git a/flume-ng-sinks/flume-ng-hbase-sink/pom.xml b/flume-ng-sinks/flume-ng-hbase-sink/pom.xml index cc2bbee50a..f5a6cf06df 100644 --- a/flume-ng-sinks/flume-ng-hbase-sink/pom.xml +++ b/flume-ng-sinks/flume-ng-hbase-sink/pom.xml @@ -114,7 +114,7 @@ hadoop-1.0 - !hadoop.profile + !flume.hadoop.profile @@ -154,7 +154,7 @@ hadoop-2 - hadoop.profile + flume.hadoop.profile 2 @@ -185,11 +185,11 @@ - hbase-98 + hbase-1 - hadoop.profile - hbase-98 + flume.hadoop.profile + hbase-1 diff --git a/flume-ng-sinks/pom.xml b/flume-ng-sinks/pom.xml index de12891e4a..84083f59cd 100644 --- a/flume-ng-sinks/pom.xml +++ b/flume-ng-sinks/pom.xml @@ -55,7 +55,7 @@ limitations under the License. hadoop-1.0 - !hadoop.profile + !flume.hadoop.profile @@ -64,7 +64,7 @@ limitations under the License. hadoop-2 - hadoop.profile + flume.hadoop.profile 2 @@ -77,11 +77,11 @@ limitations under the License. - hbase-98 + hbase-1 - hadoop.profile - hbase-98 + flume.hadoop.profile + hbase-1 diff --git a/pom.xml b/pom.xml index 72a6f17154..e2a7119f33 100644 --- a/pom.xml +++ b/pom.xml @@ -1401,6 +1401,21 @@ limitations under the License. test + + + xalan + serializer + 2.7.2 + runtime + + + + xalan + xalan + runtime + 2.7.2 + + From 8443062c8809e3ce4bee864e61d9837c5a243287 Mon Sep 17 00:00:00 2001 From: Hari Shreedharan Date: Fri, 17 Apr 2015 12:26:58 -0700 Subject: [PATCH 231/341] FLUME-2679. Make hbase-1 the default profile (Johny Rufus via Hari) --- flume-ng-dist/pom.xml | 3 +-- flume-ng-sinks/flume-hdfs-sink/pom.xml | 6 +++--- flume-ng-sinks/flume-hive-sink/pom.xml | 6 +++--- flume-ng-sinks/flume-ng-hbase-sink/pom.xml | 6 +++--- flume-ng-sinks/pom.xml | 6 +++--- flume-tools/pom.xml | 3 ++- pom.xml | 6 +++--- 7 files changed, 18 insertions(+), 18 deletions(-) diff --git a/flume-ng-dist/pom.xml b/flume-ng-dist/pom.xml index 11fff24459..218c6b8b02 100644 --- a/flume-ng-dist/pom.xml +++ b/flume-ng-dist/pom.xml @@ -87,8 +87,7 @@ hbase-1 - flume.hadoop.profile - hbase-1 + !flume.hadoop.profile diff --git a/flume-ng-sinks/flume-hdfs-sink/pom.xml b/flume-ng-sinks/flume-hdfs-sink/pom.xml index 0cca037742..1aa281f051 100644 --- a/flume-ng-sinks/flume-hdfs-sink/pom.xml +++ b/flume-ng-sinks/flume-hdfs-sink/pom.xml @@ -108,7 +108,8 @@ limitations under the License. hadoop-1.0 - !flume.hadoop.profile + flume.hadoop.profile + 1 @@ -165,8 +166,7 @@ limitations under the License. hbase-1 - flume.hadoop.profile - hbase-1 + !flume.hadoop.profile diff --git a/flume-ng-sinks/flume-hive-sink/pom.xml b/flume-ng-sinks/flume-hive-sink/pom.xml index 768763d330..e273f1ff4c 100644 --- a/flume-ng-sinks/flume-hive-sink/pom.xml +++ b/flume-ng-sinks/flume-hive-sink/pom.xml @@ -44,7 +44,8 @@ limitations under the License. hadoop-1.0 - !flume.hadoop.profile + flume.hadoop.profile + 1 @@ -87,8 +88,7 @@ limitations under the License. hbase-1 - flume.hadoop.profile - hbase-1 + !flume.hadoop.profile diff --git a/flume-ng-sinks/flume-ng-hbase-sink/pom.xml b/flume-ng-sinks/flume-ng-hbase-sink/pom.xml index 698efa8df2..575a61dc34 100644 --- a/flume-ng-sinks/flume-ng-hbase-sink/pom.xml +++ b/flume-ng-sinks/flume-ng-hbase-sink/pom.xml @@ -114,7 +114,8 @@ hadoop-1.0 - !flume.hadoop.profile + flume.hadoop.profile + 1 @@ -188,8 +189,7 @@ hbase-1 - flume.hadoop.profile - hbase-1 + !flume.hadoop.profile diff --git a/flume-ng-sinks/pom.xml b/flume-ng-sinks/pom.xml index 5622c4568c..00791e41ad 100644 --- a/flume-ng-sinks/pom.xml +++ b/flume-ng-sinks/pom.xml @@ -55,7 +55,8 @@ limitations under the License. hadoop-1.0 - !flume.hadoop.profile + flume.hadoop.profile + 1 @@ -80,8 +81,7 @@ limitations under the License. hbase-1 - flume.hadoop.profile - hbase-1 + !flume.hadoop.profile + + + 4.0.0 + + + flume-ng-sources + org.apache.flume + 1.7.0-SNAPSHOT + + + org.apache.flume.flume-ng-sources + flume-taildir-source + Flume Taildir Source + + + + + org.apache.maven.plugins + maven-compiler-plugin + 2.3.2 + + 1.7 + 1.7 + + + + + + + + org.apache.flume + flume-ng-core + + + + junit + junit + test + + + + diff --git a/flume-ng-sources/flume-taildir-source/src/main/java/org/apache/flume/source/taildir/ReliableTaildirEventReader.java b/flume-ng-sources/flume-taildir-source/src/main/java/org/apache/flume/source/taildir/ReliableTaildirEventReader.java new file mode 100644 index 0000000000..951b7867bd --- /dev/null +++ b/flume-ng-sources/flume-taildir-source/src/main/java/org/apache/flume/source/taildir/ReliableTaildirEventReader.java @@ -0,0 +1,347 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, + * software distributed under the License is distributed on an + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + * KIND, either express or implied. See the License for the + * specific language governing permissions and limitations + * under the License. + */ + +package org.apache.flume.source.taildir; + +import java.io.File; +import java.io.FileFilter; +import java.io.FileNotFoundException; +import java.io.FileReader; +import java.io.IOException; +import java.nio.file.Files; +import java.util.ArrayList; +import java.util.Arrays; +import java.util.Collections; +import java.util.List; +import java.util.Map; +import java.util.Map.Entry; +import java.util.regex.Pattern; + +import org.apache.flume.Event; +import org.apache.flume.FlumeException; +import org.apache.flume.annotations.InterfaceAudience; +import org.apache.flume.annotations.InterfaceStability; +import org.apache.flume.client.avro.ReliableEventReader; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +import com.google.common.annotations.VisibleForTesting; +import com.google.common.base.Preconditions; +import com.google.common.collect.HashBasedTable; +import com.google.common.collect.Lists; +import com.google.common.collect.Maps; +import com.google.common.collect.Table; +import com.google.common.collect.Table.Cell; +import com.google.gson.stream.JsonReader; + +@InterfaceAudience.Private +@InterfaceStability.Evolving +public class ReliableTaildirEventReader implements ReliableEventReader { + private static final Logger logger = LoggerFactory.getLogger(ReliableTaildirEventReader.class); + + private final Table tailFileTable; + private final Table headerTable; + + private TailFile currentFile = null; + private Map tailFiles = Maps.newHashMap(); + private long updateTime; + private boolean addByteOffset; + private boolean committed = true; + + /** + * Create a ReliableTaildirEventReader to watch the given directory. + */ + private ReliableTaildirEventReader(Map filePaths, + Table headerTable, String positionFilePath, + boolean skipToEnd, boolean addByteOffset) throws IOException { + // Sanity checks + Preconditions.checkNotNull(filePaths); + Preconditions.checkNotNull(positionFilePath); + + if (logger.isDebugEnabled()) { + logger.debug("Initializing {} with directory={}, metaDir={}", + new Object[] { ReliableTaildirEventReader.class.getSimpleName(), filePaths }); + } + + Table tailFileTable = HashBasedTable.create(); + for (Entry e : filePaths.entrySet()) { + File f = new File(e.getValue()); + File parentDir = f.getParentFile(); + Preconditions.checkState(parentDir.exists(), + "Directory does not exist: " + parentDir.getAbsolutePath()); + Pattern fileNamePattern = Pattern.compile(f.getName()); + tailFileTable.put(e.getKey(), parentDir, fileNamePattern); + } + logger.info("tailFileTable: " + tailFileTable.toString()); + logger.info("headerTable: " + headerTable.toString()); + + this.tailFileTable = tailFileTable; + this.headerTable = headerTable; + this.addByteOffset = addByteOffset; + updateTailFiles(skipToEnd); + + logger.info("Updating position from position file: " + positionFilePath); + loadPositionFile(positionFilePath); + } + + /** + * Load a position file which has the last read position of each file. + * If the position file exists, update tailFiles mapping. + */ + public void loadPositionFile(String filePath) { + Long inode, pos; + String path; + FileReader fr = null; + JsonReader jr = null; + try { + fr = new FileReader(filePath); + jr = new JsonReader(fr); + jr.beginArray(); + while (jr.hasNext()) { + inode = null; + pos = null; + path = null; + jr.beginObject(); + while (jr.hasNext()) { + switch (jr.nextName()) { + case "inode": + inode = jr.nextLong(); + break; + case "pos": + pos = jr.nextLong(); + break; + case "file": + path = jr.nextString(); + break; + } + } + jr.endObject(); + + for (Object v : Arrays.asList(inode, pos, path)) { + Preconditions.checkNotNull(v, "Detected missing value in position file. " + + "inode: " + inode + ", pos: " + pos + ", path: " + path); + } + TailFile tf = tailFiles.get(inode); + if (tf != null && tf.updatePos(path, inode, pos)) { + tailFiles.put(inode, tf); + } else { + logger.info("Missing file: " + path + ", inode: " + inode + ", pos: " + pos); + } + } + jr.endArray(); + } catch (FileNotFoundException e) { + logger.info("File not found: " + filePath + ", not updating position"); + } catch (IOException e) { + logger.error("Failed loading positionFile: " + filePath, e); + } finally { + try { + if (fr != null) fr.close(); + if (jr != null) jr.close(); + } catch (IOException e) { + logger.error("Error: " + e.getMessage(), e); + } + } + } + + public Map getTailFiles() { + return tailFiles; + } + + public void setCurrentFile(TailFile currentFile) { + this.currentFile = currentFile; + } + + @Override + public Event readEvent() throws IOException { + List events = readEvents(1); + if (events.isEmpty()) { + return null; + } + return events.get(0); + } + + @Override + public List readEvents(int numEvents) throws IOException { + return readEvents(numEvents, false); + } + + @VisibleForTesting + public List readEvents(TailFile tf, int numEvents) throws IOException { + setCurrentFile(tf); + return readEvents(numEvents, true); + } + + public List readEvents(int numEvents, boolean backoffWithoutNL) + throws IOException { + if (!committed) { + if (currentFile == null) { + throw new IllegalStateException("current file does not exist. " + currentFile.getPath()); + } + logger.info("Last read was never committed - resetting position"); + long lastPos = currentFile.getPos(); + currentFile.getRaf().seek(lastPos); + } + List events = currentFile.readEvents(numEvents, backoffWithoutNL, addByteOffset); + if (events.isEmpty()) { + return events; + } + + Map headers = currentFile.getHeaders(); + if (headers != null && !headers.isEmpty()) { + for (Event event : events) { + event.getHeaders().putAll(headers); + } + } + committed = false; + return events; + } + + @Override + public void close() throws IOException { + for (TailFile tf : tailFiles.values()) { + if (tf.getRaf() != null) tf.getRaf().close(); + } + } + + /** Commit the last lines which were read. */ + @Override + public void commit() throws IOException { + if (!committed && currentFile != null) { + long pos = currentFile.getRaf().getFilePointer(); + currentFile.setPos(pos); + currentFile.setLastUpdated(updateTime); + committed = true; + } + } + + /** + * Update tailFiles mapping if a new file is created or appends are detected + * to the existing file. + */ + public List updateTailFiles(boolean skipToEnd) throws IOException { + updateTime = System.currentTimeMillis(); + List updatedInodes = Lists.newArrayList(); + + for (Cell cell : tailFileTable.cellSet()) { + Map headers = headerTable.row(cell.getRowKey()); + File parentDir = cell.getColumnKey(); + Pattern fileNamePattern = cell.getValue(); + + for (File f : getMatchFiles(parentDir, fileNamePattern)) { + long inode = getInode(f); + TailFile tf = tailFiles.get(inode); + if (tf == null || !tf.getPath().equals(f.getAbsolutePath())) { + long startPos = skipToEnd ? f.length() : 0; + tf = openFile(f, headers, inode, startPos); + } else{ + boolean updated = tf.getLastUpdated() < f.lastModified(); + if (updated) { + if (tf.getRaf() == null) { + tf = openFile(f, headers, inode, tf.getPos()); + } + if (f.length() < tf.getPos()) { + logger.info("Pos " + tf.getPos() + " is larger than file size! " + + "Restarting from pos 0, file: " + tf.getPath() + ", inode: " + inode); + tf.updatePos(tf.getPath(), inode, 0); + } + } + tf.setNeedTail(updated); + } + tailFiles.put(inode, tf); + updatedInodes.add(inode); + } + } + return updatedInodes; + } + + public List updateTailFiles() throws IOException { + return updateTailFiles(false); + } + + private List getMatchFiles(File parentDir, final Pattern fileNamePattern) { + FileFilter filter = new FileFilter() { + public boolean accept(File f) { + String fileName = f.getName(); + if (f.isDirectory() || !fileNamePattern.matcher(fileName).matches()) { + return false; + } + return true; + } + }; + File[] files = parentDir.listFiles(filter); + ArrayList result = Lists.newArrayList(files); + Collections.sort(result, new TailFile.CompareByLastModifiedTime()); + return result; + } + + private long getInode(File file) throws IOException { + long inode = (long) Files.getAttribute(file.toPath(), "unix:ino"); + return inode; + } + + private TailFile openFile(File file, Map headers, long inode, long pos) { + try { + logger.info("Opening file: " + file + ", inode: " + inode + ", pos: " + pos); + return new TailFile(file, headers, inode, pos); + } catch (IOException e) { + throw new FlumeException("Failed opening file: " + file, e); + } + } + + /** + * Special builder class for ReliableTaildirEventReader + */ + public static class Builder { + private Map filePaths; + private Table headerTable; + private String positionFilePath; + private boolean skipToEnd; + private boolean addByteOffset; + + public Builder filePaths(Map filePaths) { + this.filePaths = filePaths; + return this; + } + + public Builder headerTable(Table headerTable) { + this.headerTable = headerTable; + return this; + } + + public Builder positionFilePath(String positionFilePath) { + this.positionFilePath = positionFilePath; + return this; + } + + public Builder skipToEnd(boolean skipToEnd) { + this.skipToEnd = skipToEnd; + return this; + } + + public Builder addByteOffset(boolean addByteOffset) { + this.addByteOffset = addByteOffset; + return this; + } + + public ReliableTaildirEventReader build() throws IOException { + return new ReliableTaildirEventReader(filePaths, headerTable, positionFilePath, skipToEnd, addByteOffset); + } + } + +} diff --git a/flume-ng-sources/flume-taildir-source/src/main/java/org/apache/flume/source/taildir/TailFile.java b/flume-ng-sources/flume-taildir-source/src/main/java/org/apache/flume/source/taildir/TailFile.java new file mode 100644 index 0000000000..99683da48b --- /dev/null +++ b/flume-ng-sources/flume-taildir-source/src/main/java/org/apache/flume/source/taildir/TailFile.java @@ -0,0 +1,163 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, + * software distributed under the License is distributed on an + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + * KIND, either express or implied. See the License for the + * specific language governing permissions and limitations + * under the License. + */ + +package org.apache.flume.source.taildir; + +import static org.apache.flume.source.taildir.TaildirSourceConfigurationConstants.*; + +import java.io.File; +import java.io.IOException; +import java.io.RandomAccessFile; +import java.util.Comparator; +import java.util.List; +import java.util.Map; + +import org.apache.commons.lang.StringUtils; +import org.apache.flume.Event; +import org.apache.flume.event.EventBuilder; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +import com.google.common.base.Charsets; +import com.google.common.collect.Lists; +import com.google.common.io.ByteArrayDataOutput; +import com.google.common.io.ByteStreams; + +public class TailFile { + private static final Logger logger = LoggerFactory.getLogger(TailFile.class); + + private static final String LINE_SEP = "\n"; + private static final String LINE_SEP_WIN = "\r\n"; + + private RandomAccessFile raf; + private final String path; + private final long inode; + private long pos; + private long lastUpdated; + private boolean needTail; + private final Map headers; + + public TailFile(File file, Map headers, long inode, long pos) + throws IOException { + this.raf = new RandomAccessFile(file, "r"); + if (pos > 0) raf.seek(pos); + this.path = file.getAbsolutePath(); + this.inode = inode; + this.pos = pos; + this.lastUpdated = 0L; + this.needTail = true; + this.headers = headers; + } + + public RandomAccessFile getRaf() { return raf; } + public String getPath() { return path; } + public long getInode() { return inode; } + public long getPos() { return pos; } + public long getLastUpdated() { return lastUpdated; } + public boolean needTail() { return needTail; } + public Map getHeaders() { return headers; } + + public void setPos(long pos) { this.pos = pos; } + public void setLastUpdated(long lastUpdated) { this.lastUpdated = lastUpdated; } + public void setNeedTail(boolean needTail) { this.needTail = needTail; } + + public boolean updatePos(String path, long inode, long pos) throws IOException { + if (this.inode == inode && this.path.equals(path)) { + raf.seek(pos); + setPos(pos); + logger.info("Updated position, file: " + path + ", inode: " + inode + ", pos: " + pos); + return true; + } + return false; + } + + public List readEvents(int numEvents, boolean backoffWithoutNL, + boolean addByteOffset) throws IOException { + List events = Lists.newLinkedList(); + for (int i = 0; i < numEvents; i++) { + Event event = readEvent(backoffWithoutNL, addByteOffset); + if (event == null) { + break; + } + events.add(event); + } + return events; + } + + private Event readEvent(boolean backoffWithoutNL, boolean addByteOffset) throws IOException { + Long posTmp = raf.getFilePointer(); + String line = readLine(); + if (line == null) { + return null; + } + if (backoffWithoutNL && !line.endsWith(LINE_SEP)) { + logger.info("Backing off in file without newline: " + + path + ", inode: " + inode + ", pos: " + raf.getFilePointer()); + raf.seek(posTmp); + return null; + } + + String lineSep = LINE_SEP; + if(line.endsWith(LINE_SEP_WIN)) { + lineSep = LINE_SEP_WIN; + } + Event event = EventBuilder.withBody(StringUtils.removeEnd(line, lineSep), Charsets.UTF_8); + if (addByteOffset == true) { + event.getHeaders().put(BYTE_OFFSET_HEADER_KEY, posTmp.toString()); + } + return event; + } + + private String readLine() throws IOException { + ByteArrayDataOutput out = ByteStreams.newDataOutput(300); + int i = 0; + int c; + while ((c = raf.read()) != -1) { + i++; + out.write((byte) c); + if (c == LINE_SEP.charAt(0)) { + break; + } + } + if (i == 0) { + return null; + } + return new String(out.toByteArray(), Charsets.UTF_8); + } + + public void close() { + try { + raf.close(); + raf = null; + long now = System.currentTimeMillis(); + setLastUpdated(now); + } catch (IOException e) { + logger.error("Failed closing file: " + path + ", inode: " + inode, e); + } + } + + public static class CompareByLastModifiedTime implements Comparator { + @Override + public int compare(File f1, File f2) { + return Long.valueOf(f1.lastModified()).compareTo(f2.lastModified()); + } + } + + +} diff --git a/flume-ng-sources/flume-taildir-source/src/main/java/org/apache/flume/source/taildir/TaildirSource.java b/flume-ng-sources/flume-taildir-source/src/main/java/org/apache/flume/source/taildir/TaildirSource.java new file mode 100644 index 0000000000..97ca43bc58 --- /dev/null +++ b/flume-ng-sources/flume-taildir-source/src/main/java/org/apache/flume/source/taildir/TaildirSource.java @@ -0,0 +1,331 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with this + * work for additional information regarding copyright ownership. The ASF + * licenses this file to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance with the License. + * You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, WITHOUT + * WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. See the + * License for the specific language governing permissions and limitations under + * the License. + */ + +package org.apache.flume.source.taildir; + +import static org.apache.flume.source.taildir.TaildirSourceConfigurationConstants.*; + +import java.io.File; +import java.io.FileWriter; +import java.io.IOException; +import java.util.List; +import java.util.Map; +import java.util.Map.Entry; +import java.util.concurrent.CopyOnWriteArrayList; +import java.util.concurrent.ExecutorService; +import java.util.concurrent.Executors; +import java.util.concurrent.ScheduledExecutorService; +import java.util.concurrent.TimeUnit; + +import org.apache.flume.ChannelException; +import org.apache.flume.Context; +import org.apache.flume.Event; +import org.apache.flume.FlumeException; +import org.apache.flume.PollableSource; +import org.apache.flume.conf.Configurable; +import org.apache.flume.instrumentation.SourceCounter; +import org.apache.flume.source.AbstractSource; +import org.apache.flume.source.PollableSourceConstants; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +import com.google.common.annotations.VisibleForTesting; +import com.google.common.base.Preconditions; +import com.google.common.collect.HashBasedTable; +import com.google.common.collect.ImmutableMap; +import com.google.common.collect.Lists; +import com.google.common.collect.Maps; +import com.google.common.collect.Table; +import com.google.common.util.concurrent.ThreadFactoryBuilder; +import com.google.gson.Gson; + +public class TaildirSource extends AbstractSource implements + PollableSource, Configurable { + + private static final Logger logger = LoggerFactory.getLogger(TaildirSource.class); + + private Map filePaths; + private Table headerTable; + private int batchSize; + private String positionFilePath; + private boolean skipToEnd; + private boolean byteOffsetHeader; + + private SourceCounter sourceCounter; + private ReliableTaildirEventReader reader; + private ScheduledExecutorService idleFileChecker; + private ScheduledExecutorService positionWriter; + private int retryInterval = 1000; + private int maxRetryInterval = 5000; + private int idleTimeout; + private int checkIdleInterval = 5000; + private int writePosInitDelay = 5000; + private int writePosInterval; + + private List existingInodes = new CopyOnWriteArrayList(); + private List idleInodes = new CopyOnWriteArrayList(); + private Long backoffSleepIncrement; + private Long maxBackOffSleepInterval; + + @Override + public synchronized void start() { + logger.info("{} TaildirSource source starting with directory: {}", getName(), filePaths); + try { + reader = new ReliableTaildirEventReader.Builder() + .filePaths(filePaths) + .headerTable(headerTable) + .positionFilePath(positionFilePath) + .skipToEnd(skipToEnd) + .addByteOffset(byteOffsetHeader) + .build(); + } catch (IOException e) { + throw new FlumeException("Error instantiating ReliableTaildirEventReader", e); + } + idleFileChecker = Executors.newSingleThreadScheduledExecutor( + new ThreadFactoryBuilder().setNameFormat("idleFileChecker").build()); + idleFileChecker.scheduleWithFixedDelay(new idleFileCheckerRunnable(), + idleTimeout, checkIdleInterval, TimeUnit.MILLISECONDS); + + positionWriter = Executors.newSingleThreadScheduledExecutor( + new ThreadFactoryBuilder().setNameFormat("positionWriter").build()); + positionWriter.scheduleWithFixedDelay(new PositionWriterRunnable(), + writePosInitDelay, writePosInterval, TimeUnit.MILLISECONDS); + + super.start(); + logger.debug("TaildirSource started"); + sourceCounter.start(); + } + + @Override + public synchronized void stop() { + try { + super.stop(); + ExecutorService[] services = {idleFileChecker, positionWriter}; + for (ExecutorService service : services) { + service.shutdown(); + if (!service.awaitTermination(1, TimeUnit.SECONDS)) { + service.shutdownNow(); + } + } + // write the last position + writePosition(); + reader.close(); + } catch (InterruptedException e) { + logger.info("Interrupted while awaiting termination", e); + } catch (IOException e) { + logger.info("Failed: " + e.getMessage(), e); + } + sourceCounter.stop(); + logger.info("Taildir source {} stopped. Metrics: {}", getName(), sourceCounter); + } + + @Override + public String toString() { + return String.format("Taildir source: { positionFile: %s, skipToEnd: %s, " + + "byteOffsetHeader: %s, idleTimeout: %s, writePosInterval: %s }", + positionFilePath, skipToEnd, byteOffsetHeader, idleTimeout, writePosInterval); + } + + @Override + public synchronized void configure(Context context) { + String fileGroups = context.getString(FILE_GROUPS); + Preconditions.checkState(fileGroups != null, "Missing param: " + FILE_GROUPS); + + filePaths = selectByKeys(context.getSubProperties(FILE_GROUPS_PREFIX), fileGroups.split("\\s+")); + Preconditions.checkState(!filePaths.isEmpty(), + "Mapping for tailing files is empty or invalid: '" + FILE_GROUPS_PREFIX + "'"); + + String homePath = System.getProperty("user.home").replace('\\', '/'); + positionFilePath = context.getString(POSITION_FILE, homePath + DEFAULT_POSITION_FILE); + headerTable = getTable(context, HEADERS_PREFIX); + batchSize = context.getInteger(BATCH_SIZE, DEFAULT_BATCH_SIZE); + skipToEnd = context.getBoolean(SKIP_TO_END, DEFAULT_SKIP_TO_END); + byteOffsetHeader = context.getBoolean(BYTE_OFFSET_HEADER, DEFAULT_BYTE_OFFSET_HEADER); + idleTimeout = context.getInteger(IDLE_TIMEOUT, DEFAULT_IDLE_TIMEOUT); + writePosInterval = context.getInteger(WRITE_POS_INTERVAL, DEFAULT_WRITE_POS_INTERVAL); + + backoffSleepIncrement = context.getLong(PollableSourceConstants.BACKOFF_SLEEP_INCREMENT + , PollableSourceConstants.DEFAULT_BACKOFF_SLEEP_INCREMENT); + maxBackOffSleepInterval = context.getLong(PollableSourceConstants.MAX_BACKOFF_SLEEP + , PollableSourceConstants.DEFAULT_MAX_BACKOFF_SLEEP); + + if (sourceCounter == null) { + sourceCounter = new SourceCounter(getName()); + } + } + + private Map selectByKeys(Map map, String[] keys) { + Map result = Maps.newHashMap(); + for (String key : keys) { + if (map.containsKey(key)) { + result.put(key, map.get(key)); + } + } + return result; + } + + private Table getTable(Context context, String prefix) { + Table table = HashBasedTable.create(); + for (Entry e : context.getSubProperties(prefix).entrySet()) { + String[] parts = e.getKey().split("\\.", 2); + table.put(parts[0], parts[1], e.getValue()); + } + return table; + } + + @VisibleForTesting + protected SourceCounter getSourceCounter() { + return sourceCounter; + } + + @Override + public Status process() { + Status status = Status.READY; + try { + existingInodes.clear(); + existingInodes.addAll(reader.updateTailFiles()); + for (long inode : existingInodes) { + TailFile tf = reader.getTailFiles().get(inode); + if (tf.needTail()) { + tailFileProcess(tf, true); + } + } + closeTailFiles(); + try { + TimeUnit.MILLISECONDS.sleep(retryInterval); + } catch (InterruptedException e) { + logger.info("Interrupted while sleeping"); + } + } catch (Throwable t) { + logger.error("Unable to tail files", t); + status = Status.BACKOFF; + } + return status; + } + + @Override + public long getBackOffSleepIncrement() { + return backoffSleepIncrement; + } + + @Override + public long getMaxBackOffSleepInterval() { + return maxBackOffSleepInterval; + } + + private void tailFileProcess(TailFile tf, boolean backoffWithoutNL) + throws IOException, InterruptedException { + while (true) { + reader.setCurrentFile(tf); + List events = reader.readEvents(batchSize, backoffWithoutNL); + if (events.isEmpty()) { + break; + } + sourceCounter.addToEventReceivedCount(events.size()); + sourceCounter.incrementAppendBatchReceivedCount(); + try { + getChannelProcessor().processEventBatch(events); + reader.commit(); + } catch (ChannelException ex) { + logger.warn("The channel is full or unexpected failure. " + + "The source will try again after " + retryInterval + " ms"); + TimeUnit.MILLISECONDS.sleep(retryInterval); + retryInterval = retryInterval << 1; + retryInterval = Math.min(retryInterval, maxRetryInterval); + continue; + } + retryInterval = 1000; + sourceCounter.addToEventAcceptedCount(events.size()); + sourceCounter.incrementAppendBatchAcceptedCount(); + if (events.size() < batchSize) { + break; + } + } + } + + private void closeTailFiles() throws IOException, InterruptedException { + for (long inode : idleInodes) { + TailFile tf = reader.getTailFiles().get(inode); + if (tf.getRaf() != null) { // when file has not closed yet + tailFileProcess(tf, false); + tf.close(); + logger.info("Closed file: " + tf.getPath() + ", inode: " + inode + ", pos: " + tf.getPos()); + } + } + idleInodes.clear(); + } + + /** + * Runnable class that checks whether there are files which should be closed. + */ + private class idleFileCheckerRunnable implements Runnable { + @Override + public void run() { + try { + long now = System.currentTimeMillis(); + for (TailFile tf : reader.getTailFiles().values()) { + if (tf.getLastUpdated() + idleTimeout < now && tf.getRaf() != null) { + idleInodes.add(tf.getInode()); + } + } + } catch (Throwable t) { + logger.error("Uncaught exception in IdleFileChecker thread", t); + } + } + } + + /** + * Runnable class that writes a position file which has the last read position + * of each file. + */ + private class PositionWriterRunnable implements Runnable { + @Override + public void run() { + writePosition(); + } + } + + private void writePosition() { + File file = new File(positionFilePath); + FileWriter writer = null; + try { + writer = new FileWriter(file); + if (!existingInodes.isEmpty()) { + String json = toPosInfoJson(); + writer.write(json); + } + } catch (Throwable t){ + logger.error("Failed writing positionFile", t); + } finally { + try { + if (writer != null) writer.close(); + } catch (IOException e) { + logger.error("Error: " + e.getMessage(), e); + } + } + } + + private String toPosInfoJson() { + @SuppressWarnings("rawtypes") + List posInfos = Lists.newArrayList(); + for (Long inode : existingInodes) { + TailFile tf = reader.getTailFiles().get(inode); + posInfos.add(ImmutableMap.of("inode", inode, "pos", tf.getPos(), "file", tf.getPath())); + } + return new Gson().toJson(posInfos); + } +} diff --git a/flume-ng-sources/flume-taildir-source/src/main/java/org/apache/flume/source/taildir/TaildirSourceConfigurationConstants.java b/flume-ng-sources/flume-taildir-source/src/main/java/org/apache/flume/source/taildir/TaildirSourceConfigurationConstants.java new file mode 100644 index 0000000000..616527606b --- /dev/null +++ b/flume-ng-sources/flume-taildir-source/src/main/java/org/apache/flume/source/taildir/TaildirSourceConfigurationConstants.java @@ -0,0 +1,52 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with this + * work for additional information regarding copyright ownership. The ASF + * licenses this file to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance with the License. + * You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, WITHOUT + * WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. See the + * License for the specific language governing permissions and limitations under + * the License. + */ + +package org.apache.flume.source.taildir; + +public class TaildirSourceConfigurationConstants { + /** Mapping for tailing file groups. */ + public static final String FILE_GROUPS = "filegroups"; + public static final String FILE_GROUPS_PREFIX = FILE_GROUPS + "."; + + /** Mapping for putting headers to events grouped by file groups. */ + public static final String HEADERS_PREFIX = "headers."; + + /** Path of position file. */ + public static final String POSITION_FILE = "positionFile"; + public static final String DEFAULT_POSITION_FILE = "/.flume/taildir_position.json"; + + /** What size to batch with before sending to ChannelProcessor. */ + public static final String BATCH_SIZE = "batchSize"; + public static final int DEFAULT_BATCH_SIZE = 100; + + /** Whether to skip the position to EOF in the case of files not written on the position file. */ + public static final String SKIP_TO_END = "skipToEnd"; + public static final boolean DEFAULT_SKIP_TO_END = false; + + /** Time (ms) to close idle files. */ + public static final String IDLE_TIMEOUT = "idleTimeout"; + public static final int DEFAULT_IDLE_TIMEOUT = 120000; + + /** Interval time (ms) to write the last position of each file on the position file. */ + public static final String WRITE_POS_INTERVAL = "writePosInterval"; + public static final int DEFAULT_WRITE_POS_INTERVAL = 3000; + + /** Whether to add the byte offset of a tailed line to the header */ + public static final String BYTE_OFFSET_HEADER = "byteOffsetHeader"; + public static final String BYTE_OFFSET_HEADER_KEY = "byteoffset"; + public static final boolean DEFAULT_BYTE_OFFSET_HEADER = false; +} diff --git a/flume-ng-sources/flume-taildir-source/src/test/java/org/apache/flume/source/taildir/TestTaildirEventReader.java b/flume-ng-sources/flume-taildir-source/src/test/java/org/apache/flume/source/taildir/TestTaildirEventReader.java new file mode 100644 index 0000000000..18968838ab --- /dev/null +++ b/flume-ng-sources/flume-taildir-source/src/test/java/org/apache/flume/source/taildir/TestTaildirEventReader.java @@ -0,0 +1,492 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, + * software distributed under the License is distributed on an + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + * KIND, either express or implied. See the License for the + * specific language governing permissions and limitations + * under the License. + */ + +package org.apache.flume.source.taildir; + +import static org.apache.flume.source.taildir.TaildirSourceConfigurationConstants.*; +import static org.junit.Assert.*; + +import java.io.File; +import java.io.IOException; +import java.util.List; +import java.util.Map; +import java.util.Set; + +import org.apache.flume.Event; +import org.junit.After; +import org.junit.Before; +import org.junit.Test; + +import com.google.common.base.Charsets; +import com.google.common.base.Throwables; +import com.google.common.collect.HashBasedTable; +import com.google.common.collect.ImmutableMap; +import com.google.common.collect.Lists; +import com.google.common.collect.Sets; +import com.google.common.collect.Table; +import com.google.common.io.Files; + +public class TestTaildirEventReader { + private File tmpDir; + private String posFilePath; + + public static String bodyAsString(Event event) { + return new String(event.getBody()); + } + + static List bodiesAsStrings(List events) { + List bodies = Lists.newArrayListWithCapacity(events.size()); + for (Event event : events) { + bodies.add(new String(event.getBody())); + } + return bodies; + } + + static List headersAsStrings(List events, String headerKey) { + List headers = Lists.newArrayListWithCapacity(events.size()); + for (Event event : events) { + headers.add(new String(event.getHeaders().get(headerKey))); + } + return headers; + } + + private ReliableTaildirEventReader getReader(Map filePaths, + Table headerTable, boolean addByteOffset) { + ReliableTaildirEventReader reader; + try { + reader = new ReliableTaildirEventReader.Builder() + .filePaths(filePaths) + .headerTable(headerTable) + .positionFilePath(posFilePath) + .skipToEnd(false) + .addByteOffset(addByteOffset) + .build(); + reader.updateTailFiles(); + } catch (IOException ioe) { + throw Throwables.propagate(ioe); + } + return reader; + } + + private ReliableTaildirEventReader getReader(boolean addByteOffset) { + Map filePaths = ImmutableMap.of("testFiles", tmpDir.getAbsolutePath() + "/file.*"); + Table headerTable = HashBasedTable.create(); + return getReader(filePaths, headerTable, addByteOffset); + } + + private ReliableTaildirEventReader getReader() { + return getReader(false); + } + + @Before + public void setUp() { + tmpDir = Files.createTempDir(); + posFilePath = tmpDir.getAbsolutePath() + "/taildir_position_test.json"; + } + + @After + public void tearDown() { + for (File f : tmpDir.listFiles()) { + if (f.isDirectory()) { + for (File sdf : f.listFiles()) { + sdf.delete(); + } + } + f.delete(); + } + tmpDir.delete(); + } + + @Test + // Create three multi-line files then read them back out. Ensures that + // lines and appended ones are read correctly from files. + public void testBasicReadFiles() throws IOException { + File f1 = new File(tmpDir, "file1"); + File f2 = new File(tmpDir, "file2"); + File f3 = new File(tmpDir, "file3"); + Files.write("file1line1\nfile1line2\n", f1, Charsets.UTF_8); + Files.write("file2line1\nfile2line2\n", f2, Charsets.UTF_8); + Files.write("file3line1\nfile3line2\n", f3, Charsets.UTF_8); + + ReliableTaildirEventReader reader = getReader(); + List out = Lists.newArrayList(); + for (TailFile tf : reader.getTailFiles().values()) { + List bodies = bodiesAsStrings(reader.readEvents(tf, 2)); + out.addAll(bodies); + reader.commit(); + } + assertEquals(6, out.size()); + // Make sure we got every line + assertTrue(out.contains("file1line1")); + assertTrue(out.contains("file1line2")); + assertTrue(out.contains("file2line1")); + assertTrue(out.contains("file2line2")); + assertTrue(out.contains("file3line1")); + assertTrue(out.contains("file3line2")); + + Files.append("file3line3\nfile3line4\n", f3, Charsets.UTF_8); + + reader.updateTailFiles(); + for (TailFile tf : reader.getTailFiles().values()) { + List bodies = bodiesAsStrings(reader.readEvents(tf, 2)); + out.addAll(bodies); + reader.commit(); + } + assertEquals(8, out.size()); + assertTrue(out.contains("file3line3")); + assertTrue(out.contains("file3line4")); + } + + @Test + // Make sure this works when there are initially no files + // and we finish reading all files and fully commit. + public void testInitiallyEmptyDirAndBehaviorAfterReadingAll() throws IOException { + ReliableTaildirEventReader reader = getReader(); + + List fileInodes = reader.updateTailFiles(); + assertEquals(0, fileInodes.size()); + + File f1 = new File(tmpDir, "file1"); + Files.write("file1line1\nfile1line2\n", f1, Charsets.UTF_8); + + reader.updateTailFiles(); + List out = null; + for (TailFile tf : reader.getTailFiles().values()) { + out = bodiesAsStrings(reader.readEvents(tf, 2)); + reader.commit(); + } + assertEquals(2, out.size()); + // Make sure we got every line + assertTrue(out.contains("file1line1")); + assertTrue(out.contains("file1line2")); + + reader.updateTailFiles(); + List empty = null; + for (TailFile tf : reader.getTailFiles().values()) { + empty = bodiesAsStrings(reader.readEvents(tf, 15)); + reader.commit(); + } + assertEquals(0, empty.size()); + } + + @Test + // Test a basic case where a commit is missed. + public void testBasicCommitFailure() throws IOException { + File f1 = new File(tmpDir, "file1"); + Files.write("file1line1\nfile1line2\nfile1line3\nfile1line4\n" + + "file1line5\nfile1line6\nfile1line7\nfile1line8\n" + + "file1line9\nfile1line10\nfile1line11\nfile1line12\n", + f1, Charsets.UTF_8); + + ReliableTaildirEventReader reader = getReader(); + List out1 = null; + for (TailFile tf : reader.getTailFiles().values()) { + out1 = bodiesAsStrings(reader.readEvents(tf, 4)); + } + assertTrue(out1.contains("file1line1")); + assertTrue(out1.contains("file1line2")); + assertTrue(out1.contains("file1line3")); + assertTrue(out1.contains("file1line4")); + + List out2 = bodiesAsStrings(reader.readEvents(4)); + assertTrue(out2.contains("file1line1")); + assertTrue(out2.contains("file1line2")); + assertTrue(out2.contains("file1line3")); + assertTrue(out2.contains("file1line4")); + + reader.commit(); + List out3 = bodiesAsStrings(reader.readEvents(4)); + assertTrue(out3.contains("file1line5")); + assertTrue(out3.contains("file1line6")); + assertTrue(out3.contains("file1line7")); + assertTrue(out3.contains("file1line8")); + + reader.commit(); + List out4 = bodiesAsStrings(reader.readEvents(4)); + assertEquals(4, out4.size()); + assertTrue(out4.contains("file1line9")); + assertTrue(out4.contains("file1line10")); + assertTrue(out4.contains("file1line11")); + assertTrue(out4.contains("file1line12")); + } + + @Test + // Test a case where a commit is missed and the batch size changes. + public void testBasicCommitFailureAndBatchSizeChanges() throws IOException { + File f1 = new File(tmpDir, "file1"); + Files.write("file1line1\nfile1line2\nfile1line3\nfile1line4\n" + + "file1line5\nfile1line6\nfile1line7\nfile1line8\n", + f1, Charsets.UTF_8); + + ReliableTaildirEventReader reader = getReader(); + List out1 = null; + for (TailFile tf : reader.getTailFiles().values()) { + out1 = bodiesAsStrings(reader.readEvents(tf, 5)); + } + assertTrue(out1.contains("file1line1")); + assertTrue(out1.contains("file1line2")); + assertTrue(out1.contains("file1line3")); + assertTrue(out1.contains("file1line4")); + assertTrue(out1.contains("file1line5")); + + List out2 = bodiesAsStrings(reader.readEvents(2)); + assertTrue(out2.contains("file1line1")); + assertTrue(out2.contains("file1line2")); + + reader.commit(); + List out3 = bodiesAsStrings(reader.readEvents(2)); + assertTrue(out3.contains("file1line3")); + assertTrue(out3.contains("file1line4")); + + reader.commit(); + List out4 = bodiesAsStrings(reader.readEvents(15)); + assertTrue(out4.contains("file1line5")); + assertTrue(out4.contains("file1line6")); + assertTrue(out4.contains("file1line7")); + assertTrue(out4.contains("file1line8")); + } + + @Test + public void testIncludeEmptyFile() throws IOException { + File f1 = new File(tmpDir, "file1"); + File f2 = new File(tmpDir, "file2"); + Files.write("file1line1\nfile1line2\n", f1, Charsets.UTF_8); + Files.touch(f2); + + ReliableTaildirEventReader reader = getReader(); + // Expect to read nothing from empty file + List out = Lists.newArrayList(); + for (TailFile tf : reader.getTailFiles().values()) { + out.addAll(bodiesAsStrings(reader.readEvents(tf, 5))); + reader.commit(); + } + assertEquals(2, out.size()); + assertTrue(out.contains("file1line1")); + assertTrue(out.contains("file1line2")); + assertNull(reader.readEvent()); + } + + @Test + public void testBackoffWithoutNewLine() throws IOException { + File f1 = new File(tmpDir, "file1"); + Files.write("file1line1\nfile1", f1, Charsets.UTF_8); + + ReliableTaildirEventReader reader = getReader(); + List out = Lists.newArrayList(); + // Expect to read only the line with newline + for (TailFile tf : reader.getTailFiles().values()) { + out.addAll(bodiesAsStrings(reader.readEvents(tf, 5))); + reader.commit(); + } + assertEquals(1, out.size()); + assertTrue(out.contains("file1line1")); + + Files.append("line2\nfile1line3\nfile1line4", f1, Charsets.UTF_8); + + for (TailFile tf : reader.getTailFiles().values()) { + out.addAll(bodiesAsStrings(reader.readEvents(tf, 5))); + reader.commit(); + } + assertEquals(3, out.size()); + assertTrue(out.contains("file1line2")); + assertTrue(out.contains("file1line3")); + + // Should read the last line if it finally has no newline + out.addAll(bodiesAsStrings(reader.readEvents(5, false))); + reader.commit(); + assertEquals(4, out.size()); + assertTrue(out.contains("file1line4")); + } + + @Test + public void testBatchedReadsAcrossFileBoundary() throws IOException { + File f1 = new File(tmpDir, "file1"); + Files.write("file1line1\nfile1line2\nfile1line3\nfile1line4\n" + + "file1line5\nfile1line6\nfile1line7\nfile1line8\n", + f1, Charsets.UTF_8); + + ReliableTaildirEventReader reader = getReader(); + List out1 = Lists.newArrayList(); + for (TailFile tf : reader.getTailFiles().values()) { + out1.addAll(bodiesAsStrings(reader.readEvents(tf, 5))); + reader.commit(); + } + + File f2 = new File(tmpDir, "file2"); + Files.write("file2line1\nfile2line2\nfile2line3\nfile2line4\n" + + "file2line5\nfile2line6\nfile2line7\nfile2line8\n", + f2, Charsets.UTF_8); + + List out2 = bodiesAsStrings(reader.readEvents(5)); + reader.commit(); + + reader.updateTailFiles(); + List out3 = Lists.newArrayList(); + for (TailFile tf : reader.getTailFiles().values()) { + out3.addAll(bodiesAsStrings(reader.readEvents(tf, 5))); + reader.commit(); + } + + // Should have first 5 lines of file1 + assertEquals(5, out1.size()); + assertTrue(out1.contains("file1line1")); + assertTrue(out1.contains("file1line2")); + assertTrue(out1.contains("file1line3")); + assertTrue(out1.contains("file1line4")); + assertTrue(out1.contains("file1line5")); + + // Should have 3 remaining lines of file1 + assertEquals(3, out2.size()); + assertTrue(out2.contains("file1line6")); + assertTrue(out2.contains("file1line7")); + assertTrue(out2.contains("file1line8")); + + // Should have first 5 lines of file2 + assertEquals(5, out3.size()); + assertTrue(out3.contains("file2line1")); + assertTrue(out3.contains("file2line2")); + assertTrue(out3.contains("file2line3")); + assertTrue(out3.contains("file2line4")); + assertTrue(out3.contains("file2line5")); + } + + @Test + public void testLargeNumberOfFiles() throws IOException { + int fileNum = 1000; + Set expected = Sets.newHashSet(); + + for (int i = 0; i < fileNum; i++) { + String data = "data" + i; + File f = new File(tmpDir, "file" + i); + Files.write(data + "\n", f, Charsets.UTF_8); + expected.add(data); + } + + ReliableTaildirEventReader reader = getReader(); + for (TailFile tf : reader.getTailFiles().values()) { + List events = reader.readEvents(tf, 10); + for (Event e : events) { + expected.remove(new String(e.getBody())); + } + reader.commit(); + } + assertEquals(0, expected.size()); + } + + @Test + public void testLoadPositionFile() throws IOException { + File f1 = new File(tmpDir, "file1"); + File f2 = new File(tmpDir, "file2"); + File f3 = new File(tmpDir, "file3"); + + Files.write("file1line1\nfile1line2\nfile1line3\n", f1, Charsets.UTF_8); + Files.write("file2line1\nfile2line2\n", f2, Charsets.UTF_8); + Files.write("file3line1\n", f3, Charsets.UTF_8); + + ReliableTaildirEventReader reader = getReader(); + Map tailFiles = reader.getTailFiles(); + + long pos = f2.length(); + int i = 1; + File posFile = new File(posFilePath); + for (TailFile tf : tailFiles.values()) { + Files.append(i == 1 ? "[" : "", posFile, Charsets.UTF_8); + Files.append(String.format("{\"inode\":%s,\"pos\":%s,\"file\":\"%s\"}", + tf.getInode(), pos, tf.getPath()), posFile, Charsets.UTF_8); + Files.append(i == 3 ? "]" : ",", posFile, Charsets.UTF_8); + i++; + } + reader.loadPositionFile(posFilePath); + + for (TailFile tf : tailFiles.values()) { + if (tf.getPath().equals(tmpDir + "file3")) { + // when given position is larger than file size + assertEquals(0, tf.getPos()); + } else { + assertEquals(pos, tf.getPos()); + } + } + } + + @Test + public void testSkipToEndPosition() throws IOException { + ReliableTaildirEventReader reader = getReader(); + File f1 = new File(tmpDir, "file1"); + Files.write("file1line1\nfile1line2\n", f1, Charsets.UTF_8); + + reader.updateTailFiles(); + for (TailFile tf : reader.getTailFiles().values()) { + if (tf.getPath().equals(tmpDir + "file1")) { + assertEquals(0, tf.getPos()); + } + } + + File f2 = new File(tmpDir, "file2"); + Files.write("file2line1\nfile2line2\n", f2, Charsets.UTF_8); + // Expect to skip to EOF the read position when skipToEnd option is true + reader.updateTailFiles(true); + for (TailFile tf : reader.getTailFiles().values()) { + if (tf.getPath().equals(tmpDir + "file2")) { + assertEquals(f2.length(), tf.getPos()); + } + } + } + + @Test + public void testByteOffsetHeader() throws IOException { + File f1 = new File(tmpDir, "file1"); + String line1 = "file1line1\n"; + String line2 = "file1line2\n"; + String line3 = "file1line3\n"; + Files.write(line1 + line2 + line3, f1, Charsets.UTF_8); + + ReliableTaildirEventReader reader = getReader(true); + List headers = null; + for (TailFile tf : reader.getTailFiles().values()) { + headers = headersAsStrings(reader.readEvents(tf, 5), BYTE_OFFSET_HEADER_KEY); + reader.commit(); + } + assertEquals(3, headers.size()); + // Make sure we got byte offset position + assertTrue(headers.contains(String.valueOf(0))); + assertTrue(headers.contains(String.valueOf(line1.length()))); + assertTrue(headers.contains(String.valueOf((line1 + line2).length()))); + } + + @Test + public void testNewLineBoundaries() throws IOException { + File f1 = new File(tmpDir, "file1"); + Files.write("file1line1\nfile1line2\rfile1line2\nfile1line3\r\nfile1line4\n", f1, Charsets.UTF_8); + + ReliableTaildirEventReader reader = getReader(); + List out = Lists.newArrayList(); + for (TailFile tf : reader.getTailFiles().values()) { + out.addAll(bodiesAsStrings(reader.readEvents(tf, 5))); + reader.commit(); + } + assertEquals(4, out.size()); + //Should treat \n as line boundary + assertTrue(out.contains("file1line1")); + //Should not treat \r as line boundary + assertTrue(out.contains("file1line2\rfile1line2")); + //Should treat \r\n as line boundary + assertTrue(out.contains("file1line3")); + assertTrue(out.contains("file1line4")); + } +} diff --git a/flume-ng-sources/flume-taildir-source/src/test/java/org/apache/flume/source/taildir/TestTaildirSource.java b/flume-ng-sources/flume-taildir-source/src/test/java/org/apache/flume/source/taildir/TestTaildirSource.java new file mode 100644 index 0000000000..f9e614c332 --- /dev/null +++ b/flume-ng-sources/flume-taildir-source/src/test/java/org/apache/flume/source/taildir/TestTaildirSource.java @@ -0,0 +1,283 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with this + * work for additional information regarding copyright ownership. The ASF + * licenses this file to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance with the License. + * You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, WITHOUT + * WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. See the + * License for the specific language governing permissions and limitations under + * the License. + */ + +package org.apache.flume.source.taildir; + +import static org.apache.flume.source.taildir.TaildirSourceConfigurationConstants.*; +import static org.junit.Assert.*; + +import java.io.File; +import java.io.IOException; +import java.util.ArrayList; +import java.util.List; + +import org.apache.flume.Channel; +import org.apache.flume.ChannelSelector; +import org.apache.flume.Context; +import org.apache.flume.Event; +import org.apache.flume.Transaction; +import org.apache.flume.channel.ChannelProcessor; +import org.apache.flume.channel.MemoryChannel; +import org.apache.flume.channel.ReplicatingChannelSelector; +import org.apache.flume.conf.Configurables; +import org.apache.flume.lifecycle.LifecycleController; +import org.apache.flume.lifecycle.LifecycleState; +import org.junit.After; +import org.junit.Before; +import org.junit.Test; + +import com.google.common.base.Charsets; +import com.google.common.collect.Lists; +import com.google.common.io.Files; + +public class TestTaildirSource { + static TaildirSource source; + static MemoryChannel channel; + private File tmpDir; + private String posFilePath; + + @Before + public void setUp() { + source = new TaildirSource(); + channel = new MemoryChannel(); + + Configurables.configure(channel, new Context()); + + List channels = new ArrayList(); + channels.add(channel); + + ChannelSelector rcs = new ReplicatingChannelSelector(); + rcs.setChannels(channels); + + source.setChannelProcessor(new ChannelProcessor(rcs)); + tmpDir = Files.createTempDir(); + posFilePath = tmpDir.getAbsolutePath() + "/taildir_position_test.json"; + } + + @After + public void tearDown() { + for (File f : tmpDir.listFiles()) { + f.delete(); + } + tmpDir.delete(); + } + + @Test + public void testRegexFileNameFiltering() throws IOException { + File f1 = new File(tmpDir, "a.log"); + File f2 = new File(tmpDir, "a.log.1"); + File f3 = new File(tmpDir, "b.log"); + File f4 = new File(tmpDir, "c.log.yyyy-MM-01"); + File f5 = new File(tmpDir, "c.log.yyyy-MM-02"); + Files.write("a.log\n", f1, Charsets.UTF_8); + Files.write("a.log.1\n", f2, Charsets.UTF_8); + Files.write("b.log\n", f3, Charsets.UTF_8); + Files.write("c.log.yyyy-MM-01\n", f4, Charsets.UTF_8); + Files.write("c.log.yyyy-MM-02\n", f5, Charsets.UTF_8); + + Context context = new Context(); + context.put(POSITION_FILE, posFilePath); + context.put(FILE_GROUPS, "ab c"); + // Tail a.log and b.log + context.put(FILE_GROUPS_PREFIX + "ab", tmpDir.getAbsolutePath() + "/[ab].log"); + // Tail files that starts with c.log + context.put(FILE_GROUPS_PREFIX + "c", tmpDir.getAbsolutePath() + "/c.log.*"); + + Configurables.configure(source, context); + source.start(); + source.process(); + Transaction txn = channel.getTransaction(); + txn.begin(); + List out = Lists.newArrayList(); + for (int i = 0; i < 5; i++) { + Event e = channel.take(); + if (e != null) { + out.add(TestTaildirEventReader.bodyAsString(e)); + } + } + txn.commit(); + txn.close(); + + assertEquals(4, out.size()); + // Make sure we got every file + assertTrue(out.contains("a.log")); + assertFalse(out.contains("a.log.1")); + assertTrue(out.contains("b.log")); + assertTrue(out.contains("c.log.yyyy-MM-01")); + assertTrue(out.contains("c.log.yyyy-MM-02")); + } + + @Test + public void testHeaderMapping() throws IOException { + File f1 = new File(tmpDir, "file1"); + File f2 = new File(tmpDir, "file2"); + File f3 = new File(tmpDir, "file3"); + Files.write("file1line1\nfile1line2\n", f1, Charsets.UTF_8); + Files.write("file2line1\nfile2line2\n", f2, Charsets.UTF_8); + Files.write("file3line1\nfile3line2\n", f3, Charsets.UTF_8); + + Context context = new Context(); + context.put(POSITION_FILE, posFilePath); + context.put(FILE_GROUPS, "f1 f2 f3"); + context.put(FILE_GROUPS_PREFIX + "f1", tmpDir.getAbsolutePath() + "/file1$"); + context.put(FILE_GROUPS_PREFIX + "f2", tmpDir.getAbsolutePath() + "/file2$"); + context.put(FILE_GROUPS_PREFIX + "f3", tmpDir.getAbsolutePath() + "/file3$"); + context.put(HEADERS_PREFIX + "f1.headerKeyTest", "value1"); + context.put(HEADERS_PREFIX + "f2.headerKeyTest", "value2"); + context.put(HEADERS_PREFIX + "f2.headerKeyTest2", "value2-2"); + + Configurables.configure(source, context); + source.start(); + source.process(); + Transaction txn = channel.getTransaction(); + txn.begin(); + for (int i = 0; i < 6; i++) { + Event e = channel.take(); + String body = new String(e.getBody(), Charsets.UTF_8); + String headerValue = e.getHeaders().get("headerKeyTest"); + String headerValue2 = e.getHeaders().get("headerKeyTest2"); + if (body.startsWith("file1")) { + assertEquals("value1", headerValue); + assertNull(headerValue2); + } else if (body.startsWith("file2")) { + assertEquals("value2", headerValue); + assertEquals("value2-2", headerValue2); + } else if (body.startsWith("file3")) { + // No header + assertNull(headerValue); + assertNull(headerValue2); + } + } + txn.commit(); + txn.close(); + } + + @Test + public void testLifecycle() throws IOException, InterruptedException { + File f1 = new File(tmpDir, "file1"); + Files.write("file1line1\nfile1line2\n", f1, Charsets.UTF_8); + + Context context = new Context(); + context.put(POSITION_FILE, posFilePath); + context.put(FILE_GROUPS, "f1"); + context.put(FILE_GROUPS_PREFIX + "f1", tmpDir.getAbsolutePath() + "/file1$"); + Configurables.configure(source, context); + + for (int i = 0; i < 3; i++) { + source.start(); + source.process(); + assertTrue("Reached start or error", LifecycleController.waitForOneOf( + source, LifecycleState.START_OR_ERROR)); + assertEquals("Server is started", LifecycleState.START, + source.getLifecycleState()); + + source.stop(); + assertTrue("Reached stop or error", + LifecycleController.waitForOneOf(source, LifecycleState.STOP_OR_ERROR)); + assertEquals("Server is stopped", LifecycleState.STOP, + source.getLifecycleState()); + } + } + + @Test + public void testFileConsumeOrder() throws IOException { + System.out.println(tmpDir.toString()); + // 1) Create 1st file + File f1 = new File(tmpDir, "file1"); + String line1 = "file1line1\n"; + String line2 = "file1line2\n"; + String line3 = "file1line3\n"; + Files.write(line1 + line2 + line3, f1, Charsets.UTF_8); + try { + Thread.sleep(1000); // wait before creating a new file + } catch (InterruptedException e) { + } + + // 1) Create 2nd file + String line1b = "file2line1\n"; + String line2b = "file2line2\n"; + String line3b = "file2line3\n"; + File f2 = new File(tmpDir, "file2"); + Files.write(line1b + line2b + line3b, f2, Charsets.UTF_8); + try { + Thread.sleep(1000); // wait before creating next file + } catch (InterruptedException e) { + } + + // 3) Create 3rd file + String line1c = "file3line1\n"; + String line2c = "file3line2\n"; + String line3c = "file3line3\n"; + File f3 = new File(tmpDir, "file3"); + Files.write(line1c + line2c + line3c, f3, Charsets.UTF_8); + + try { + Thread.sleep(1000); // wait before creating a new file + } catch (InterruptedException e) { + } + + + // 4) Create 4th file + String line1d = "file4line1\n"; + String line2d = "file4line2\n"; + String line3d = "file4line3\n"; + File f4 = new File(tmpDir, "file4"); + Files.write(line1d + line2d + line3d, f4, Charsets.UTF_8); + + try { + Thread.sleep(1000); // wait before creating a new file + } catch (InterruptedException e) { + } + + + // 5) Now update the 3rd file so that its the latest file and gets consumed last + f3.setLastModified(System.currentTimeMillis()); + + // 4) Consume the files + ArrayList consumedOrder = Lists.newArrayList(); + Context context = new Context(); + context.put(POSITION_FILE, posFilePath); + context.put(FILE_GROUPS, "g1"); + context.put(FILE_GROUPS_PREFIX + "g1", tmpDir.getAbsolutePath() + "/.*"); + + Configurables.configure(source, context); + source.start(); + source.process(); + Transaction txn = channel.getTransaction(); + txn.begin(); + for (int i = 0; i < 12; i++) { + Event e = channel.take(); + String body = new String(e.getBody(), Charsets.UTF_8); + consumedOrder.add(body); + } + txn.commit(); + txn.close(); + + System.out.println(consumedOrder); + + // 6) Ensure consumption order is in order of last update time + ArrayList expected = Lists.newArrayList(line1, line2, line3, // file1 + line1b, line2b, line3b, // file2 + line1d, line2d, line3d, // file4 + line1c, line2c, line3c // file3 + ); + for(int i =0; i!=expected.size(); ++i) { + expected.set(i, expected.get(i).trim() ); + } + assertArrayEquals("Files not consumed in expected order", expected.toArray(), consumedOrder.toArray()); + } +} diff --git a/flume-ng-sources/pom.xml b/flume-ng-sources/pom.xml index 79de5fa283..f52695656b 100644 --- a/flume-ng-sources/pom.xml +++ b/flume-ng-sources/pom.xml @@ -45,6 +45,7 @@ limitations under the License. flume-jms-source flume-twitter-source flume-kafka-source + flume-taildir-source diff --git a/pom.xml b/pom.xml index 64decfd2e0..d3d64b8a4f 100644 --- a/pom.xml +++ b/pom.xml @@ -1231,6 +1231,12 @@ limitations under the License. 1.7.0-SNAPSHOT + + org.apache.flume.flume-ng-sources + flume-taildir-source + 1.7.0-SNAPSHOT + + org.apache.flume flume-ng-sdk From fff13b5e0aa6cb2f81aaeeb46e440120662fffa8 Mon Sep 17 00:00:00 2001 From: Johny Rufus Date: Thu, 20 Aug 2015 08:11:13 -0700 Subject: [PATCH 251/341] FLUME-2763. flume_env script should handle jvm parameters like -javaagent -agentpath -agentlib ( Philip Zeyliger, Neerja Khattar via Johny Rufus) --- bin/flume-ng | 12 ++++++++++++ 1 file changed, 12 insertions(+) diff --git a/bin/flume-ng b/bin/flume-ng index 380169fdcc..e28723d2a9 100755 --- a/bin/flume-ng +++ b/bin/flume-ng @@ -315,6 +315,18 @@ while [ -n "$*" ] ; do opt_plugins_dirs=$1 shift ;; + -agentlib*) + arr_java_props[arr_java_props_ct]=$arg + ((++arr_java_props_ct)) + ;; + -agentpath*) + arr_java_props[arr_java_props_ct]=$arg + ((++arr_java_props_ct)) + ;; + -javaagent*) + arr_java_props[arr_java_props_ct]=$arg + ((++arr_java_props_ct)) + ;; -D*) arr_java_props[arr_java_props_ct]=$arg ((++arr_java_props_ct)) From 318da208844d02ed7554724ae526cefe94dd894c Mon Sep 17 00:00:00 2001 From: Roshan Naik Date: Tue, 25 Aug 2015 18:38:14 -0700 Subject: [PATCH 252/341] FLUME-2754 - Hive Sink skipping first transaction in each Batch of Hive Transactions (Deepesh Khandelwal via Roshan Naik) --- .../apache/flume/sink/hive/HiveWriter.java | 10 ++++-- .../flume/sink/hive/TestHiveWriter.java | 32 +++++++++++++++++++ 2 files changed, 39 insertions(+), 3 deletions(-) diff --git a/flume-ng-sinks/flume-hive-sink/src/main/java/org/apache/flume/sink/hive/HiveWriter.java b/flume-ng-sinks/flume-hive-sink/src/main/java/org/apache/flume/sink/hive/HiveWriter.java index aa8576e327..46309be061 100644 --- a/flume-ng-sinks/flume-hive-sink/src/main/java/org/apache/flume/sink/hive/HiveWriter.java +++ b/flume-ng-sinks/flume-hive-sink/src/main/java/org/apache/flume/sink/hive/HiveWriter.java @@ -88,6 +88,7 @@ class HiveWriter { this.serializer = serializer; this.recordWriter = serializer.createRecordWriter(endPoint); this.txnBatch = nextTxnBatch(recordWriter); + this.txnBatch.beginNextTransaction(); this.closed = false; this.lastUsed = System.currentTimeMillis(); } catch (InterruptedException e) { @@ -117,6 +118,10 @@ void setHearbeatNeeded() { hearbeatNeeded = true; } + public int getRemainingTxns() { + return txnBatch.remainingTransactions(); + } + /** * Write data, update stats @@ -212,7 +217,7 @@ public void flush(boolean rollToNext) /** * Aborts the current Txn and switches to next Txn. - * @throws StreamingException if could not get new Transaction Batch, or switch to next Txn + * @throws InterruptedException */ public void abort() throws InterruptedException { batch.clear(); @@ -332,8 +337,7 @@ public TransactionBatch call() throws InterruptedException, StreamingException { return connection.fetchTransactionBatch(txnsPerBatch, recordWriter); // could block } }); - LOG.info("Acquired Txn Batch {}. Switching to first txn", batch); - batch.beginNextTransaction(); + LOG.info("Acquired Transaction batch {}", batch); } catch (Exception e) { throw new TxnBatchException(endPoint, e); } diff --git a/flume-ng-sinks/flume-hive-sink/src/test/java/org/apache/flume/sink/hive/TestHiveWriter.java b/flume-ng-sinks/flume-hive-sink/src/test/java/org/apache/flume/sink/hive/TestHiveWriter.java index 174f179ff0..41bf0f6813 100644 --- a/flume-ng-sinks/flume-hive-sink/src/test/java/org/apache/flume/sink/hive/TestHiveWriter.java +++ b/flume-ng-sinks/flume-hive-sink/src/test/java/org/apache/flume/sink/hive/TestHiveWriter.java @@ -174,6 +174,38 @@ public void testWriteMultiFlush() throws Exception { checkRecordCountInTable(3); } + @Test + public void testTxnBatchConsumption() throws Exception { + // get a small txn batch and consume it, then roll to new batch, very + // the number of remaining txns to ensure Txns are not accidentally skipped + + HiveEndPoint endPoint = new HiveEndPoint(metaStoreURI, dbName, tblName, partVals); + SinkCounter sinkCounter = new SinkCounter(this.getClass().getName()); + + int txnPerBatch = 3; + + HiveWriter writer = new HiveWriter(endPoint, txnPerBatch, true, timeout + , callTimeoutPool, "flumetest", serializer, sinkCounter); + + Assert.assertEquals(writer.getRemainingTxns(),2); + writer.flush(true); + + Assert.assertEquals(writer.getRemainingTxns(), 1); + writer.flush(true); + + Assert.assertEquals(writer.getRemainingTxns(), 0); + writer.flush(true); + + // flip over to next batch + Assert.assertEquals(writer.getRemainingTxns(), 2); + writer.flush(true); + + Assert.assertEquals(writer.getRemainingTxns(), 1); + + writer.close(); + + } + private void checkRecordCountInTable(int expectedCount) throws CommandNeedRetryException, IOException { int count = TestUtil.listRecordsInTable(driver, dbName, tblName).size(); From 67189ca84d24154150fa2ca4194b3b8d79400bda Mon Sep 17 00:00:00 2001 From: Johny Rufus Date: Tue, 15 Sep 2015 18:35:28 -0700 Subject: [PATCH 253/341] FLUME-2672. NPE in KafkaSourceCounter (Rigo MacTaggart via Johny Rufus) --- .../kafka/KafkaSourceCounter.java | 2 +- .../kafka/KafkaSourceCounterTest.java | 63 +++++++++++++++++++ 2 files changed, 64 insertions(+), 1 deletion(-) create mode 100644 flume-ng-core/src/test/java/org/apache/flume/instrumentation/kafka/KafkaSourceCounterTest.java diff --git a/flume-ng-core/src/main/java/org/apache/flume/instrumentation/kafka/KafkaSourceCounter.java b/flume-ng-core/src/main/java/org/apache/flume/instrumentation/kafka/KafkaSourceCounter.java index 1cb911d765..ad0ba2cf10 100644 --- a/flume-ng-core/src/main/java/org/apache/flume/instrumentation/kafka/KafkaSourceCounter.java +++ b/flume-ng-core/src/main/java/org/apache/flume/instrumentation/kafka/KafkaSourceCounter.java @@ -31,7 +31,7 @@ public class KafkaSourceCounter extends SourceCounter implements KafkaSourceCou "source.kafka.empty.count"; private static final String[] ATTRIBUTES = - {TIMER_KAFKA_COMMIT, TIMER_KAFKA_EVENT_GET}; + {TIMER_KAFKA_COMMIT, TIMER_KAFKA_EVENT_GET, COUNTER_KAFKA_EMPTY}; public KafkaSourceCounter(String name) { super(name, ATTRIBUTES); diff --git a/flume-ng-core/src/test/java/org/apache/flume/instrumentation/kafka/KafkaSourceCounterTest.java b/flume-ng-core/src/test/java/org/apache/flume/instrumentation/kafka/KafkaSourceCounterTest.java new file mode 100644 index 0000000000..4a712656dc --- /dev/null +++ b/flume-ng-core/src/test/java/org/apache/flume/instrumentation/kafka/KafkaSourceCounterTest.java @@ -0,0 +1,63 @@ +/** + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package org.apache.flume.instrumentation.kafka; + +import org.junit.Assert; +import org.junit.Before; +import org.junit.Test; + +public class KafkaSourceCounterTest { + + KafkaSourceCounter counter; + + @Before + public void setUp() throws Exception { + counter = new KafkaSourceCounter("test"); + } + + @Test + public void testAddToKafkaEventGetTimer() throws Exception { + Assert.assertEquals(1L, counter.addToKafkaEventGetTimer(1L)); + } + + @Test + public void testAddToKafkaCommitTimer() throws Exception { + Assert.assertEquals(1L, counter.addToKafkaCommitTimer(1L)); + } + + @Test + public void testIncrementKafkaEmptyCount() throws Exception { + Assert.assertEquals(1L, counter.incrementKafkaEmptyCount()); + } + + @Test + public void testGetKafkaCommitTimer() throws Exception { + Assert.assertEquals(0, counter.getKafkaCommitTimer()); + } + + @Test + public void testGetKafkaEventGetTimer() throws Exception { + Assert.assertEquals(0, counter.getKafkaEventGetTimer()); + } + + @Test + public void testGetKafkaEmptyCount() throws Exception { + Assert.assertEquals(0, counter.getKafkaEmptyCount()); + } + +} \ No newline at end of file From 2cc93b777865195a9bb2329946a14f301213ea3d Mon Sep 17 00:00:00 2001 From: Johny Rufus Date: Mon, 21 Sep 2015 13:43:50 -0700 Subject: [PATCH 254/341] FLUME-2095. JMS source with TIBCO (Michelle Casbon via Johny Rufus) --- .../java/org/apache/flume/source/jms/JMSSource.java | 11 +++++++++++ 1 file changed, 11 insertions(+) diff --git a/flume-ng-sources/flume-jms-source/src/main/java/org/apache/flume/source/jms/JMSSource.java b/flume-ng-sources/flume-jms-source/src/main/java/org/apache/flume/source/jms/JMSSource.java index 1d7f60f7f9..c1cc9cf5e9 100644 --- a/flume-ng-sources/flume-jms-source/src/main/java/org/apache/flume/source/jms/JMSSource.java +++ b/flume-ng-sources/flume-jms-source/src/main/java/org/apache/flume/source/jms/JMSSource.java @@ -220,6 +220,17 @@ protected void doConfigure(Context context) throws FlumeException { initialContextFactoryName); contextProperties.setProperty( javax.naming.Context.PROVIDER_URL, providerUrl); + + // Provide properties for connecting via JNDI + if (this.userName.isPresent()) { + contextProperties.setProperty( + javax.naming.Context.SECURITY_PRINCIPAL, this.userName.get()); + } + if (this.password.isPresent()) { + contextProperties.setProperty( + javax.naming.Context.SECURITY_CREDENTIALS, this.password.get()); + } + initialContext = initialContextFactory.create(contextProperties); } catch (NamingException e) { throw new FlumeException(String.format( From 3fccd241d92bc5b70d216a2111e28897cf93c5df Mon Sep 17 00:00:00 2001 From: Hari Shreedharan Date: Thu, 24 Sep 2015 22:45:04 -0700 Subject: [PATCH 255/341] FLUME-2773. TailDirSource throws FileNotFound Exception if ~/.flume directory is not created already (Johny Rufus via Hari) --- .../org/apache/flume/source/taildir/TaildirSource.java | 9 +++++++++ 1 file changed, 9 insertions(+) diff --git a/flume-ng-sources/flume-taildir-source/src/main/java/org/apache/flume/source/taildir/TaildirSource.java b/flume-ng-sources/flume-taildir-source/src/main/java/org/apache/flume/source/taildir/TaildirSource.java index 97ca43bc58..8816327abc 100644 --- a/flume-ng-sources/flume-taildir-source/src/main/java/org/apache/flume/source/taildir/TaildirSource.java +++ b/flume-ng-sources/flume-taildir-source/src/main/java/org/apache/flume/source/taildir/TaildirSource.java @@ -22,6 +22,9 @@ import java.io.File; import java.io.FileWriter; import java.io.IOException; +import java.nio.file.Files; +import java.nio.file.Path; +import java.nio.file.Paths; import java.util.List; import java.util.Map; import java.util.Map.Entry; @@ -151,6 +154,12 @@ public synchronized void configure(Context context) { String homePath = System.getProperty("user.home").replace('\\', '/'); positionFilePath = context.getString(POSITION_FILE, homePath + DEFAULT_POSITION_FILE); + Path positionFile = Paths.get(positionFilePath); + try { + Files.createDirectories(positionFile.getParent()); + } catch (IOException e) { + throw new FlumeException("Error creating positionFile parent directories", e); + } headerTable = getTable(context, HEADERS_PREFIX); batchSize = context.getInteger(BATCH_SIZE, DEFAULT_BATCH_SIZE); skipToEnd = context.getBoolean(SKIP_TO_END, DEFAULT_SKIP_TO_END); From bd80c5e67deda4ec146f19fc92ff0c3bff61a982 Mon Sep 17 00:00:00 2001 From: Roshan Naik Date: Tue, 29 Sep 2015 14:43:34 -0700 Subject: [PATCH 256/341] FLUME-2804. Hive sink - abort remaining transactions on shutdown (Sriharsha Chintalapani via Roshan Naik) --- .../org/apache/flume/sink/hive/HiveSink.java | 4 +- .../apache/flume/sink/hive/HiveWriter.java | 59 ++++++++++++++++++- 2 files changed, 59 insertions(+), 4 deletions(-) diff --git a/flume-ng-sinks/flume-hive-sink/src/main/java/org/apache/flume/sink/hive/HiveSink.java b/flume-ng-sinks/flume-hive-sink/src/main/java/org/apache/flume/sink/hive/HiveSink.java index 6fe332a37b..d93bca3b3d 100644 --- a/flume-ng-sinks/flume-hive-sink/src/main/java/org/apache/flume/sink/hive/HiveSink.java +++ b/flume-ng-sinks/flume-hive-sink/src/main/java/org/apache/flume/sink/hive/HiveSink.java @@ -320,6 +320,7 @@ private int drainOneBatch(Channel channel) sinkCounter.addToEventDrainSuccessCount(txnEventCount); return txnEventCount; } catch (HiveWriter.Failure e) { + // in case of error we close all TxnBatches to start clean next time LOG.warn(getName() + " : " + e.getMessage(), e); abortAllWriters(); closeAllWriters(); @@ -462,8 +463,7 @@ public void stop() { for (Entry entry : allWriters.entrySet()) { try { HiveWriter w = entry.getValue(); - LOG.info("Closing connection to {}", w); - w.closeConnection(); + w.close(); } catch (InterruptedException ex) { Thread.currentThread().interrupt(); } diff --git a/flume-ng-sinks/flume-hive-sink/src/main/java/org/apache/flume/sink/hive/HiveWriter.java b/flume-ng-sinks/flume-hive-sink/src/main/java/org/apache/flume/sink/hive/HiveWriter.java index 46309be061..ec30c98515 100644 --- a/flume-ng-sinks/flume-hive-sink/src/main/java/org/apache/flume/sink/hive/HiveWriter.java +++ b/flume-ng-sinks/flume-hive-sink/src/main/java/org/apache/flume/sink/hive/HiveWriter.java @@ -216,7 +216,7 @@ public void flush(boolean rollToNext) } /** - * Aborts the current Txn and switches to next Txn. + * Aborts the current Txn * @throws InterruptedException */ public void abort() throws InterruptedException { @@ -253,11 +253,66 @@ public Void call() throws StreamingException { */ public void close() throws InterruptedException { batch.clear(); + abortRemainingTxns(); closeTxnBatch(); closeConnection(); closed = true; } + + private void abortRemainingTxns() throws InterruptedException { + try { + if ( !isClosed(txnBatch.getCurrentTransactionState()) ) { + abortCurrTxnHelper(); + } + + // recursively abort remaining txns + if(txnBatch.remainingTransactions()>0) { + timedCall( + new CallRunner1() { + @Override + public Void call() throws StreamingException, InterruptedException { + txnBatch.beginNextTransaction(); + return null; + } + }); + abortRemainingTxns(); + } + } catch (StreamingException e) { + LOG.warn("Error when aborting remaining transactions in batch " + txnBatch, e); + return; + } catch (TimeoutException e) { + LOG.warn("Timed out when aborting remaining transactions in batch " + txnBatch, e); + return; + } + } + + private void abortCurrTxnHelper() throws TimeoutException, InterruptedException { + try { + timedCall( + new CallRunner1() { + @Override + public Void call() throws StreamingException, InterruptedException { + txnBatch.abort(); + LOG.info("Aborted txn " + txnBatch.getCurrentTxnId()); + return null; + } + } + ); + } catch (StreamingException e) { + LOG.warn("Unable to abort transaction " + txnBatch.getCurrentTxnId(), e); + // continue to attempt to abort other txns in the batch + } + } + + private boolean isClosed(TransactionBatch.TxnState txnState) { + if(txnState == TransactionBatch.TxnState.COMMITTED) + return true; + if(txnState == TransactionBatch.TxnState.ABORTED) + return true; + return false; + } + public void closeConnection() throws InterruptedException { LOG.info("Closing connection to EndPoint : {}", endPoint); try { @@ -346,7 +401,7 @@ public TransactionBatch call() throws InterruptedException, StreamingException { private void closeTxnBatch() throws InterruptedException { try { - LOG.debug("Closing Txn Batch {}", txnBatch); + LOG.info("Closing Txn Batch {}.", txnBatch); timedCall(new CallRunner1() { @Override public Void call() throws InterruptedException, StreamingException { From a2f55e180f80b1daa707df9269375e1a7c69e6b5 Mon Sep 17 00:00:00 2001 From: Jarek Jarcec Cecho Date: Wed, 30 Sep 2015 09:20:29 -0700 Subject: [PATCH 257/341] FLUME-2751: Upgrade Derby version to 10.11.1.1 (Johny Rufus via Jarek Jarcec Cecho) --- pom.xml | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/pom.xml b/pom.xml index d3d64b8a4f..ade1ac4a32 100644 --- a/pom.xml +++ b/pom.xml @@ -1023,7 +1023,7 @@ limitations under the License. org.apache.derby derby - 10.8.2.2 + 10.11.1.1 From d8d97db4be954fc15e4632d6c9ae5dd8f46c189c Mon Sep 17 00:00:00 2001 From: Jarek Jarcec Cecho Date: Wed, 30 Sep 2015 09:34:31 -0700 Subject: [PATCH 258/341] FLUME-2734: Kafka Channel timeout property is overridden by default value (Johny Rufus via Jarek Jarcec Cecho) --- .../flume/channel/kafka/KafkaChannel.java | 5 +++-- .../flume/channel/kafka/TestKafkaChannel.java | 18 ++++++++++++++++++ 2 files changed, 21 insertions(+), 2 deletions(-) diff --git a/flume-ng-channels/flume-kafka-channel/src/main/java/org/apache/flume/channel/kafka/KafkaChannel.java b/flume-ng-channels/flume-kafka-channel/src/main/java/org/apache/flume/channel/kafka/KafkaChannel.java index 80a122db9e..c83d4f6029 100644 --- a/flume-ng-channels/flume-kafka-channel/src/main/java/org/apache/flume/channel/kafka/KafkaChannel.java +++ b/flume-ng-channels/flume-kafka-channel/src/main/java/org/apache/flume/channel/kafka/KafkaChannel.java @@ -177,13 +177,14 @@ public void configure(Context ctx) { throw new ConfigurationException( "Zookeeper Connection must be specified"); } - Long timeout = ctx.getLong(TIMEOUT, Long.valueOf(DEFAULT_TIMEOUT)); kafkaConf.putAll(ctx.getSubProperties(KAFKA_PREFIX)); kafkaConf.put(GROUP_ID, groupId); kafkaConf.put(BROKER_LIST_KEY, brokerList); kafkaConf.put(ZOOKEEPER_CONNECT, zkConnect); kafkaConf.put(AUTO_COMMIT_ENABLED, String.valueOf(false)); - kafkaConf.put(CONSUMER_TIMEOUT, String.valueOf(timeout)); + if(kafkaConf.get(CONSUMER_TIMEOUT) == null) { + kafkaConf.put(CONSUMER_TIMEOUT, DEFAULT_TIMEOUT); + } kafkaConf.put(REQUIRED_ACKS_KEY, "-1"); LOGGER.info(kafkaConf.toString()); parseAsFlumeEvent = diff --git a/flume-ng-channels/flume-kafka-channel/src/test/java/org/apache/flume/channel/kafka/TestKafkaChannel.java b/flume-ng-channels/flume-kafka-channel/src/test/java/org/apache/flume/channel/kafka/TestKafkaChannel.java index e665431a00..25b9e408a7 100644 --- a/flume-ng-channels/flume-kafka-channel/src/test/java/org/apache/flume/channel/kafka/TestKafkaChannel.java +++ b/flume-ng-channels/flume-kafka-channel/src/test/java/org/apache/flume/channel/kafka/TestKafkaChannel.java @@ -35,6 +35,8 @@ import org.apache.flume.sink.kafka.util.TestUtil; import org.junit.*; +import static org.apache.flume.channel.kafka.KafkaChannelConfiguration.*; + import java.util.*; import java.util.concurrent.*; import java.util.concurrent.atomic.AtomicBoolean; @@ -149,6 +151,22 @@ public void testNoParsingAsFlumeAgent() throws Exception { channel.stop(); } + @Test + public void testTimeoutConfig() throws Exception { + Context context = prepareDefaultContext(true); + KafkaChannel channel = new KafkaChannel(); + Configurables.configure(channel, context); + Assert.assertTrue(channel.getKafkaConf().get(CONSUMER_TIMEOUT) + .equals(DEFAULT_TIMEOUT)); + + String timeout = "1000"; + context.put("kafka."+CONSUMER_TIMEOUT, timeout); + channel = new KafkaChannel(); + Configurables.configure(channel, context); + Assert.assertTrue(channel.getKafkaConf().get(CONSUMER_TIMEOUT) + .equals(timeout)); + } + /** * This method starts a channel, puts events into it. The channel is then * stopped and restarted. Then we check to make sure if all events we put From c8eb221c6bbeb24a6e5073eb89e69e7cd2a10366 Mon Sep 17 00:00:00 2001 From: Roshan Naik Date: Fri, 2 Oct 2015 16:07:12 -0700 Subject: [PATCH 259/341] FLUME-2798. Malformed Syslog messages can lead to OutOfMemoryException (Phil D'Amore via Roshan Naik) --- .../apache/flume/source/SyslogTcpSource.java | 4 + .../apache/flume/source/SyslogUDPSource.java | 4 + .../org/apache/flume/source/SyslogUtils.java | 100 ++++++++++-------- .../apache/flume/source/TestSyslogUtils.java | 46 ++++++++ 4 files changed, 109 insertions(+), 45 deletions(-) diff --git a/flume-ng-core/src/main/java/org/apache/flume/source/SyslogTcpSource.java b/flume-ng-core/src/main/java/org/apache/flume/source/SyslogTcpSource.java index c117813925..bd87151738 100644 --- a/flume-ng-core/src/main/java/org/apache/flume/source/SyslogTcpSource.java +++ b/flume-ng-core/src/main/java/org/apache/flume/source/SyslogTcpSource.java @@ -93,6 +93,10 @@ public void messageReceived(ChannelHandlerContext ctx, MessageEvent mEvent) { } catch (ChannelException ex) { counterGroup.incrementAndGet("events.dropped"); logger.error("Error writting to channel, event dropped", ex); + } catch (RuntimeException ex) { + counterGroup.incrementAndGet("events.dropped"); + logger.error("Error parsing event from syslog stream, event dropped", ex); + return; } } diff --git a/flume-ng-core/src/main/java/org/apache/flume/source/SyslogUDPSource.java b/flume-ng-core/src/main/java/org/apache/flume/source/SyslogUDPSource.java index 378d48418f..47993dd46c 100644 --- a/flume-ng-core/src/main/java/org/apache/flume/source/SyslogUDPSource.java +++ b/flume-ng-core/src/main/java/org/apache/flume/source/SyslogUDPSource.java @@ -85,6 +85,10 @@ public void messageReceived(ChannelHandlerContext ctx, MessageEvent mEvent) { counterGroup.incrementAndGet("events.dropped"); logger.error("Error writting to channel", ex); return; + } catch (RuntimeException ex) { + counterGroup.incrementAndGet("events.dropped"); + logger.error("Error parsing event from syslog stream, event dropped", ex); + return; } } } diff --git a/flume-ng-core/src/main/java/org/apache/flume/source/SyslogUtils.java b/flume-ng-core/src/main/java/org/apache/flume/source/SyslogUtils.java index 42e3f7136c..5a9f4c85e1 100644 --- a/flume-ng-core/src/main/java/org/apache/flume/source/SyslogUtils.java +++ b/flume-ng-core/src/main/java/org/apache/flume/source/SyslogUtils.java @@ -285,55 +285,58 @@ public String getSyslogStatus(){ // create the event from syslog data Event buildEvent() { - byte[] body; - int pri = 0; - int sev = 0; - int facility = 0; - - if(!isBadEvent){ - pri = Integer.parseInt(prio.toString()); - sev = pri % 8; - facility = pri / 8; - formatHeaders(); - } + try { + byte[] body; + int pri = 0; + int sev = 0; + int facility = 0; + + if(!isBadEvent){ + pri = Integer.parseInt(prio.toString()); + sev = pri % 8; + facility = pri / 8; + formatHeaders(); + } - Map headers = new HashMap(); - headers.put(SYSLOG_FACILITY, String.valueOf(facility)); - headers.put(SYSLOG_SEVERITY, String.valueOf(sev)); - if ((priority != null) && (priority.length() > 0)) { - headers.put("priority", priority); - } - if ((version != null) && (version.length() > 0)) { - headers.put("version", version); - } - if ((timeStamp != null) && timeStamp.length() > 0) { - headers.put("timestamp", timeStamp); - } - if ((hostName != null) && (hostName.length() > 0)) { - headers.put("host", hostName); - } - if(isBadEvent){ - logger.warn("Event created from Invalid Syslog data."); - headers.put(EVENT_STATUS, SyslogStatus.INVALID.getSyslogStatus()); - } else if(isIncompleteEvent){ - logger.warn("Event size larger than specified event size: {}. You should " + - "consider increasing your event size.", maxSize); - headers.put(EVENT_STATUS, SyslogStatus.INCOMPLETE.getSyslogStatus()); - } + Map headers = new HashMap(); + headers.put(SYSLOG_FACILITY, String.valueOf(facility)); + headers.put(SYSLOG_SEVERITY, String.valueOf(sev)); + if ((priority != null) && (priority.length() > 0)) { + headers.put("priority", priority); + } + if ((version != null) && (version.length() > 0)) { + headers.put("version", version); + } + if ((timeStamp != null) && timeStamp.length() > 0) { + headers.put("timestamp", timeStamp); + } + if ((hostName != null) && (hostName.length() > 0)) { + headers.put("host", hostName); + } + if(isBadEvent){ + logger.warn("Event created from Invalid Syslog data."); + headers.put(EVENT_STATUS, SyslogStatus.INVALID.getSyslogStatus()); + } else if(isIncompleteEvent){ + logger.warn("Event size larger than specified event size: {}. You should " + + "consider increasing your event size.", maxSize); + headers.put(EVENT_STATUS, SyslogStatus.INCOMPLETE.getSyslogStatus()); + } - if (!keepAllFields(keepFields)) { - if ((msgBody != null) && (msgBody.length() > 0)) { - body = msgBody.getBytes(); + if (!keepAllFields(keepFields)) { + if ((msgBody != null) && (msgBody.length() > 0)) { + body = msgBody.getBytes(); + } else { + // Parse failed. + body = baos.toByteArray(); + } } else { - // Parse failed. body = baos.toByteArray(); } - } else { - body = baos.toByteArray(); + // format the message + return EventBuilder.withBody(body, headers); + } finally { + reset(); } - reset(); - // format the message - return EventBuilder.withBody(body, headers); } // Apply each known pattern to message @@ -441,11 +444,18 @@ public Event extractEvent(ChannelBuffer in){ case PRIO: baos.write(b); if (b == '>') { + if (prio.length() == 0) { + isBadEvent = true; + } m = Mode.DATA; } else { char ch = (char) b; prio.append(ch); - if (!Character.isDigit(ch)) { + // Priority is max 3 digits per both RFC 3164 and 5424 + // With this check there is basically no danger of + // boas.size() exceeding this.maxSize before getting to the + // DATA state where this is actually checked + if (!Character.isDigit(ch) || prio.length() > 3) { isBadEvent = true; //If we hit a bad priority, just write as if everything is data. m = Mode.DATA; @@ -460,7 +470,7 @@ public Event extractEvent(ChannelBuffer in){ } else { baos.write(b); } - if(baos.size() == this.maxSize && !doneReading){ + if(baos.size() == this.maxSize && !doneReading) { isIncompleteEvent = true; e = buildEvent(); doneReading = true; diff --git a/flume-ng-core/src/test/java/org/apache/flume/source/TestSyslogUtils.java b/flume-ng-core/src/test/java/org/apache/flume/source/TestSyslogUtils.java index 6da173391b..be4598eb5c 100644 --- a/flume-ng-core/src/test/java/org/apache/flume/source/TestSyslogUtils.java +++ b/flume-ng-core/src/test/java/org/apache/flume/source/TestSyslogUtils.java @@ -263,6 +263,52 @@ public void testExtractBadEvent2() { } + /** + * Test bad event format 3: Empty priority - <> + */ + + @Test + public void testExtractBadEvent3() { + String badData1 = "<> bad bad data\n"; + SyslogUtils util = new SyslogUtils(false); + ChannelBuffer buff = ChannelBuffers.buffer(100); + buff.writeBytes(badData1.getBytes()); + Event e = util.extractEvent(buff); + if(e == null){ + throw new NullPointerException("Event is null"); + } + Map headers = e.getHeaders(); + Assert.assertEquals("0", headers.get(SyslogUtils.SYSLOG_FACILITY)); + Assert.assertEquals("0", headers.get(SyslogUtils.SYSLOG_SEVERITY)); + Assert.assertEquals(SyslogUtils.SyslogStatus.INVALID.getSyslogStatus(), + headers.get(SyslogUtils.EVENT_STATUS)); + Assert.assertEquals(badData1.trim(), new String(e.getBody()).trim()); + + } + + /** + * Test bad event format 4: Priority too long + */ + + @Test + public void testExtractBadEvent4() { + String badData1 = "<123123123123123123123123123123> bad bad data\n"; + SyslogUtils util = new SyslogUtils(false); + ChannelBuffer buff = ChannelBuffers.buffer(100); + buff.writeBytes(badData1.getBytes()); + Event e = util.extractEvent(buff); + if(e == null){ + throw new NullPointerException("Event is null"); + } + Map headers = e.getHeaders(); + Assert.assertEquals("0", headers.get(SyslogUtils.SYSLOG_FACILITY)); + Assert.assertEquals("0", headers.get(SyslogUtils.SYSLOG_SEVERITY)); + Assert.assertEquals(SyslogUtils.SyslogStatus.INVALID.getSyslogStatus(), + headers.get(SyslogUtils.EVENT_STATUS)); + Assert.assertEquals(badData1.trim(), new String(e.getBody()).trim()); + + } + /** * Good event */ From 67ed62aa18df3675b68369d0d00c8f0dcbdfb970 Mon Sep 17 00:00:00 2001 From: Hari Shreedharan Date: Thu, 8 Oct 2015 18:28:31 -0700 Subject: [PATCH 260/341] FLUME-2781. Kafka Channel with parseAsFlumeEvent=true should write data as is, not as flume events. (Gonzalo Herreros via Hari) --- .../flume/channel/kafka/KafkaChannel.java | 33 ++++++++++------- .../flume/channel/kafka/TestKafkaChannel.java | 37 +++++++++++++++++++ 2 files changed, 56 insertions(+), 14 deletions(-) diff --git a/flume-ng-channels/flume-kafka-channel/src/main/java/org/apache/flume/channel/kafka/KafkaChannel.java b/flume-ng-channels/flume-kafka-channel/src/main/java/org/apache/flume/channel/kafka/KafkaChannel.java index c83d4f6029..c0c1c6678f 100644 --- a/flume-ng-channels/flume-kafka-channel/src/main/java/org/apache/flume/channel/kafka/KafkaChannel.java +++ b/flume-ng-channels/flume-kafka-channel/src/main/java/org/apache/flume/channel/kafka/KafkaChannel.java @@ -261,21 +261,26 @@ protected void doPut(Event event) throws InterruptedException { } try { - if (!tempOutStream.isPresent()) { - tempOutStream = Optional.of(new ByteArrayOutputStream()); - } - if (!writer.isPresent()) { - writer = Optional.of(new - SpecificDatumWriter(AvroFlumeEvent.class)); + if (parseAsFlumeEvent) { + if (!tempOutStream.isPresent()) { + tempOutStream = Optional.of(new ByteArrayOutputStream()); + } + if (!writer.isPresent()) { + writer = Optional.of(new + SpecificDatumWriter(AvroFlumeEvent.class)); + } + tempOutStream.get().reset(); + AvroFlumeEvent e = new AvroFlumeEvent( + toCharSeqMap(event.getHeaders()), + ByteBuffer.wrap(event.getBody())); + encoder = EncoderFactory.get() + .directBinaryEncoder(tempOutStream.get(), encoder); + writer.get().write(e, encoder); + // Not really possible to avoid this copy :( + serializedEvents.get().add(tempOutStream.get().toByteArray()); + } else { + serializedEvents.get().add(event.getBody()); } - tempOutStream.get().reset(); - AvroFlumeEvent e = new AvroFlumeEvent( - toCharSeqMap(event.getHeaders()), ByteBuffer.wrap(event.getBody())); - encoder = EncoderFactory.get() - .directBinaryEncoder(tempOutStream.get(), encoder); - writer.get().write(e, encoder); - // Not really possible to avoid this copy :( - serializedEvents.get().add(tempOutStream.get().toByteArray()); } catch (Exception e) { throw new ChannelException("Error while serializing event", e); } diff --git a/flume-ng-channels/flume-kafka-channel/src/test/java/org/apache/flume/channel/kafka/TestKafkaChannel.java b/flume-ng-channels/flume-kafka-channel/src/test/java/org/apache/flume/channel/kafka/TestKafkaChannel.java index 25b9e408a7..319e779abe 100644 --- a/flume-ng-channels/flume-kafka-channel/src/test/java/org/apache/flume/channel/kafka/TestKafkaChannel.java +++ b/flume-ng-channels/flume-kafka-channel/src/test/java/org/apache/flume/channel/kafka/TestKafkaChannel.java @@ -167,6 +167,43 @@ public void testTimeoutConfig() throws Exception { .equals(timeout)); } + /** + * Like the previous test but here we write to the channel like a Flume source would do + * to verify that the events are written as text and not as an Avro object + * + * @throws Exception + */ + @Test + public void testWritingToNoParsingAsFlumeAgent() throws Exception { + final KafkaChannel channel = startChannel(false); + + List msgs = new ArrayList(); + for (int i = 0; i < 50; i++){ + msgs.add(String.valueOf(i)); + } + Transaction tx = channel.getTransaction(); + tx.begin(); + for (int i = 0; i < msgs.size(); i++){ + channel.put(EventBuilder.withBody(msgs.get(i).getBytes())); + } + tx.commit(); + ExecutorCompletionService submitterSvc = new + ExecutorCompletionService(Executors.newCachedThreadPool()); + List events = pullEvents(channel, submitterSvc, + 50, false, false); + wait(submitterSvc, 5); + Set finals = Sets.newHashSet(); + for (int i = 0; i < 50; i++) { + finals.add(Integer.parseInt(new String(events.get(i).getBody()))); + } + for (int i = 0; i < 50; i++) { + Assert.assertTrue(finals.contains(i)); + finals.remove(i); + } + Assert.assertTrue(finals.isEmpty()); + channel.stop(); + } + /** * This method starts a channel, puts events into it. The channel is then * stopped and restarted. Then we check to make sure if all events we put From d6bf08b54e467a6bdc6a5fc0edd41c51200e9da1 Mon Sep 17 00:00:00 2001 From: Johny Rufus Date: Wed, 21 Oct 2015 15:18:42 -0700 Subject: [PATCH 261/341] FLUME-2632: High CPU on KafkaSink (Ashish Paliwal via Johny Rufus) --- .../main/java/org/apache/flume/sink/kafka/KafkaSink.java | 6 ++++++ .../java/org/apache/flume/sink/kafka/TestKafkaSink.java | 2 +- 2 files changed, 7 insertions(+), 1 deletion(-) diff --git a/flume-ng-sinks/flume-ng-kafka-sink/src/main/java/org/apache/flume/sink/kafka/KafkaSink.java b/flume-ng-sinks/flume-ng-kafka-sink/src/main/java/org/apache/flume/sink/kafka/KafkaSink.java index eada17ca13..38b854b55a 100644 --- a/flume-ng-sinks/flume-ng-kafka-sink/src/main/java/org/apache/flume/sink/kafka/KafkaSink.java +++ b/flume-ng-sinks/flume-ng-kafka-sink/src/main/java/org/apache/flume/sink/kafka/KafkaSink.java @@ -98,6 +98,12 @@ public Status process() throws EventDeliveryException { if (event == null) { // no events available in channel + if(processedEvents == 0) { + result = Status.BACKOFF; + counter.incrementBatchEmptyCount(); + } else { + counter.incrementBatchUnderflowCount(); + } break; } diff --git a/flume-ng-sinks/flume-ng-kafka-sink/src/test/java/org/apache/flume/sink/kafka/TestKafkaSink.java b/flume-ng-sinks/flume-ng-kafka-sink/src/test/java/org/apache/flume/sink/kafka/TestKafkaSink.java index 80f764f729..72117b197f 100644 --- a/flume-ng-sinks/flume-ng-kafka-sink/src/test/java/org/apache/flume/sink/kafka/TestKafkaSink.java +++ b/flume-ng-sinks/flume-ng-kafka-sink/src/test/java/org/apache/flume/sink/kafka/TestKafkaSink.java @@ -168,7 +168,7 @@ public void testEmptyChannel() throws UnsupportedEncodingException, kafkaSink.start(); Sink.Status status = kafkaSink.process(); - if (status == Sink.Status.BACKOFF) { + if (status != Sink.Status.BACKOFF) { fail("Error Occurred"); } assertNull( From 0e40e831108f34b5bc6d01bf1ffa9efae330ea82 Mon Sep 17 00:00:00 2001 From: Johny Rufus Date: Thu, 22 Oct 2015 18:35:38 -0700 Subject: [PATCH 262/341] FLUME-2593. ResettableFileInputStream returns negate values from read() method (Miroslav Holubec via Johny Rufus) --- .../ResettableFileInputStream.java | 2 +- .../TestResettableFileInputStream.java | 23 +++++++++++++++++++ 2 files changed, 24 insertions(+), 1 deletion(-) diff --git a/flume-ng-core/src/main/java/org/apache/flume/serialization/ResettableFileInputStream.java b/flume-ng-core/src/main/java/org/apache/flume/serialization/ResettableFileInputStream.java index 622c09fb68..618913ef6b 100644 --- a/flume-ng-core/src/main/java/org/apache/flume/serialization/ResettableFileInputStream.java +++ b/flume-ng-core/src/main/java/org/apache/flume/serialization/ResettableFileInputStream.java @@ -224,7 +224,7 @@ public synchronized int read() throws IOException { } else if (len == 0) { return -1; } else { - return byteBuf[0]; + return byteBuf[0] & 0xFF; } } diff --git a/flume-ng-core/src/test/java/org/apache/flume/serialization/TestResettableFileInputStream.java b/flume-ng-core/src/test/java/org/apache/flume/serialization/TestResettableFileInputStream.java index 2c559db8c4..631bdfe387 100644 --- a/flume-ng-core/src/test/java/org/apache/flume/serialization/TestResettableFileInputStream.java +++ b/flume-ng-core/src/test/java/org/apache/flume/serialization/TestResettableFileInputStream.java @@ -87,6 +87,29 @@ public void testBasicRead() throws IOException { in.close(); } + /** + * Ensure that we can simply read bytes from a file using InputStream.read() method. + * @throws IOException + */ + @Test + public void testReadByte() throws IOException { + byte[] bytes = new byte[255]; + for (int i = 0; i < 255; i++) { + bytes[i] = (byte) i; + } + + Files.write(bytes, file); + + PositionTracker tracker = new DurablePositionTracker(meta, file.getPath()); + ResettableInputStream in = new ResettableFileInputStream(file, tracker); + + for (int i = 0; i < 255; i++) { + assertEquals(i, in.read()); + } + assertEquals(-1, in.read()); + + in.close(); + } /** * Ensure that we can process lines that contain multi byte characters in weird places From 8bb556604047974775eb2da4c5c1686d89fe62d2 Mon Sep 17 00:00:00 2001 From: Hari Shreedharan Date: Fri, 30 Oct 2015 16:36:40 -0700 Subject: [PATCH 263/341] FLUME-2712. Optional channel errors slows down the Source to Main channel event rate (Johny Rufus via Hari) --- .../flume/channel/ChannelProcessor.java | 146 +++++++----------- .../flume/channel/TestChannelProcessor.java | 69 +++++++++ 2 files changed, 122 insertions(+), 93 deletions(-) diff --git a/flume-ng-core/src/main/java/org/apache/flume/channel/ChannelProcessor.java b/flume-ng-core/src/main/java/org/apache/flume/channel/ChannelProcessor.java index 1cce137f78..f2612a660e 100644 --- a/flume-ng-core/src/main/java/org/apache/flume/channel/ChannelProcessor.java +++ b/flume-ng-core/src/main/java/org/apache/flume/channel/ChannelProcessor.java @@ -20,10 +20,14 @@ import com.google.common.base.Preconditions; import com.google.common.collect.Lists; +import com.google.common.util.concurrent.ThreadFactoryBuilder; + import java.util.ArrayList; import java.util.LinkedHashMap; import java.util.List; import java.util.Map; +import java.util.concurrent.Executors; +import java.util.concurrent.ExecutorService; import org.apache.flume.Channel; import org.apache.flume.ChannelException; @@ -57,6 +61,7 @@ public class ChannelProcessor implements Configurable { private final ChannelSelector selector; private final InterceptorChain interceptorChain; + private ExecutorService execService; public ChannelProcessor(ChannelSelector selector) { this.selector = selector; @@ -77,6 +82,8 @@ public void close() { */ @Override public void configure(Context context) { + this.execService = Executors.newSingleThreadExecutor( + new ThreadFactoryBuilder().setNameFormat("OptionalChannelProcessorThread").build()); configureInterceptors(context); } @@ -153,7 +160,6 @@ public void processEventBatch(List events) { for (Event event : events) { List reqChannels = selector.getRequiredChannels(event); - for (Channel ch : reqChannels) { List eventQueue = reqChannelQueue.get(ch); if (eventQueue == null) { @@ -164,74 +170,26 @@ public void processEventBatch(List events) { } List optChannels = selector.getOptionalChannels(event); - for (Channel ch: optChannels) { List eventQueue = optChannelQueue.get(ch); if (eventQueue == null) { eventQueue = new ArrayList(); optChannelQueue.put(ch, eventQueue); } - eventQueue.add(event); } } // Process required channels for (Channel reqChannel : reqChannelQueue.keySet()) { - Transaction tx = reqChannel.getTransaction(); - Preconditions.checkNotNull(tx, "Transaction object must not be null"); - try { - tx.begin(); - - List batch = reqChannelQueue.get(reqChannel); - - for (Event event : batch) { - reqChannel.put(event); - } - - tx.commit(); - } catch (Throwable t) { - tx.rollback(); - if (t instanceof Error) { - LOG.error("Error while writing to required channel: " + - reqChannel, t); - throw (Error) t; - } else { - throw new ChannelException("Unable to put batch on required " + - "channel: " + reqChannel, t); - } - } finally { - if (tx != null) { - tx.close(); - } - } + List batch = reqChannelQueue.get(reqChannel); + executeChannelTransaction(reqChannel, batch, false); } // Process optional channels for (Channel optChannel : optChannelQueue.keySet()) { - Transaction tx = optChannel.getTransaction(); - Preconditions.checkNotNull(tx, "Transaction object must not be null"); - try { - tx.begin(); - - List batch = optChannelQueue.get(optChannel); - - for (Event event : batch ) { - optChannel.put(event); - } - - tx.commit(); - } catch (Throwable t) { - tx.rollback(); - LOG.error("Unable to put batch on optional channel: " + optChannel, t); - if (t instanceof Error) { - throw (Error) t; - } - } finally { - if (tx != null) { - tx.close(); - } - } + List batch = optChannelQueue.get(optChannel); + execService.submit(new OptionalChannelTransactionRunnable(optChannel, batch)); } } @@ -253,57 +211,59 @@ public void processEvent(Event event) { if (event == null) { return; } + List events = new ArrayList(1); + events.add(event); // Process required channels List requiredChannels = selector.getRequiredChannels(event); for (Channel reqChannel : requiredChannels) { - Transaction tx = reqChannel.getTransaction(); - Preconditions.checkNotNull(tx, "Transaction object must not be null"); - try { - tx.begin(); - - reqChannel.put(event); - - tx.commit(); - } catch (Throwable t) { - tx.rollback(); - if (t instanceof Error) { - LOG.error("Error while writing to required channel: " + - reqChannel, t); - throw (Error) t; - } else { - throw new ChannelException("Unable to put event on required " + - "channel: " + reqChannel, t); - } - } finally { - if (tx != null) { - tx.close(); - } - } + executeChannelTransaction(reqChannel, events, false); } // Process optional channels List optionalChannels = selector.getOptionalChannels(event); for (Channel optChannel : optionalChannels) { - Transaction tx = null; - try { - tx = optChannel.getTransaction(); - tx.begin(); + execService.submit(new OptionalChannelTransactionRunnable(optChannel, events)); + } + } - optChannel.put(event); + private static void executeChannelTransaction(Channel channel, List batch, boolean isOptional) { + Transaction tx = channel.getTransaction(); + Preconditions.checkNotNull(tx, "Transaction object must not be null"); + try { + tx.begin(); - tx.commit(); - } catch (Throwable t) { - tx.rollback(); - LOG.error("Unable to put event on optional channel: " + optChannel, t); - if (t instanceof Error) { - throw (Error) t; - } - } finally { - if (tx != null) { - tx.close(); - } + for (Event event : batch) { + channel.put(event); } + + tx.commit(); + } catch (Throwable t) { + tx.rollback(); + if (t instanceof Error) { + LOG.error("Error while writing to channel: " + + channel, t); + throw (Error) t; + } else if(!isOptional) { + throw new ChannelException("Unable to put batch on required " + + "channel: " + channel, t); + } + } finally { + tx.close(); + } + } + + private static class OptionalChannelTransactionRunnable implements Runnable { + private Channel channel; + private List events; + + OptionalChannelTransactionRunnable(Channel channel, List events) { + this.channel = channel; + this.events = events; + } + + public void run() { + executeChannelTransaction(channel, events, true); } } -} +} \ No newline at end of file diff --git a/flume-ng-core/src/test/java/org/apache/flume/channel/TestChannelProcessor.java b/flume-ng-core/src/test/java/org/apache/flume/channel/TestChannelProcessor.java index 0656596286..924c998b76 100644 --- a/flume-ng-core/src/test/java/org/apache/flume/channel/TestChannelProcessor.java +++ b/flume-ng-core/src/test/java/org/apache/flume/channel/TestChannelProcessor.java @@ -20,11 +20,16 @@ import com.google.common.base.Charsets; import com.google.common.collect.Lists; + +import java.util.ArrayList; import java.util.List; import org.apache.flume.Channel; import org.apache.flume.ChannelException; import org.apache.flume.ChannelSelector; import org.apache.flume.Event; +import org.apache.flume.Transaction; +import org.apache.flume.Context; +import org.apache.flume.conf.Configurables; import org.apache.flume.event.EventBuilder; import org.junit.Assert; import org.junit.Test; @@ -79,4 +84,68 @@ public void testNullFromGetTransaction() { Assert.assertTrue("Must throw NPE", threw); } + /* + * Test delivery to optional and required channels + * Test both processEvent and processEventBatch + */ + @Test + public void testRequiredAndOptionalChannels() { + Context context = new Context(); + ArrayList channels = new ArrayList(); + for(int i = 0; i < 4; i++) { + Channel ch = new MemoryChannel(); + ch.setName("ch"+i); + Configurables.configure(ch, context); + channels.add(ch); + } + + ChannelSelector selector = new ReplicatingChannelSelector(); + selector.setChannels(channels); + + context = new Context(); + context.put(ReplicatingChannelSelector.CONFIG_OPTIONAL, "ch2 ch3"); + Configurables.configure(selector, context); + + ChannelProcessor processor = new ChannelProcessor(selector); + context = new Context(); + Configurables.configure(processor, context); + + + Event event1 = EventBuilder.withBody("event 1", Charsets.UTF_8); + processor.processEvent(event1); + try { + Thread.sleep(3000); + } catch (InterruptedException e) { + } + + for(Channel channel : channels) { + Transaction transaction = channel.getTransaction(); + transaction.begin(); + Event event_ch = channel.take(); + Assert.assertEquals(event1, event_ch); + transaction.commit(); + transaction.close(); + } + + List events = Lists.newArrayList(); + for(int i = 0; i < 100; i ++) { + events.add(EventBuilder.withBody("event "+i, Charsets.UTF_8)); + } + processor.processEventBatch(events); + try { + Thread.sleep(3000); + } catch (InterruptedException e) { + } + for(Channel channel : channels) { + Transaction transaction = channel.getTransaction(); + transaction.begin(); + for(int i = 0; i < 100; i ++) { + Event event_ch = channel.take(); + Assert.assertNotNull(event_ch); + } + transaction.commit(); + transaction.close(); + } + } + } From f38a52102cf7a09e9727aa7a1321a86dcbe84c0d Mon Sep 17 00:00:00 2001 From: Roshan Naik Date: Tue, 3 Nov 2015 13:00:12 -0800 Subject: [PATCH 264/341] FLUME-2835. Hive Sink tests need to create table with transactional property set (Sriharsha Chintalapani via Roshan Naik) --- .../src/test/java/org/apache/flume/sink/hive/TestUtil.java | 4 +++- 1 file changed, 3 insertions(+), 1 deletion(-) diff --git a/flume-ng-sinks/flume-hive-sink/src/test/java/org/apache/flume/sink/hive/TestUtil.java b/flume-ng-sinks/flume-hive-sink/src/test/java/org/apache/flume/sink/hive/TestUtil.java index 1fd60bc5be..107789f168 100644 --- a/flume-ng-sinks/flume-hive-sink/src/test/java/org/apache/flume/sink/hive/TestUtil.java +++ b/flume-ng-sinks/flume-hive-sink/src/test/java/org/apache/flume/sink/hive/TestUtil.java @@ -76,7 +76,9 @@ public static void createDbAndTable(Driver driver, String databaseName, " clustered by ( " + colNames[0] + " )" + " into 10 buckets " + " stored as orc " + - " location '" + tableLoc + "'"; + " location '" + tableLoc + "'" + + " TBLPROPERTIES ('transactional'='true')"; + runDDL(driver, crtTbl); System.out.println("crtTbl = " + crtTbl); if (partNames!=null && partNames.length!=0) { From 88b3fee10f1ec10dc33872710a4d4084c86b5e7d Mon Sep 17 00:00:00 2001 From: Jarek Jarcec Cecho Date: Wed, 18 Nov 2015 07:37:01 -0800 Subject: [PATCH 265/341] FLUME-2841: Upgrade commons-collections to 3.2.2 (Hari Shreedharan via Jarek Jarcec Cecho) --- pom.xml | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/pom.xml b/pom.xml index ade1ac4a32..15c086b509 100644 --- a/pom.xml +++ b/pom.xml @@ -1017,7 +1017,7 @@ limitations under the License. commons-collections commons-collections - 3.2.1 + 3.2.2 From 0421fa2ab1eb9575b34bbb2f44e8c6d83842eaeb Mon Sep 17 00:00:00 2001 From: Roshan Naik Date: Thu, 17 Dec 2015 10:35:56 -0800 Subject: [PATCH 266/341] FLUME-2801. Performance improvement on TailDir source (Jun Seok Hong via Satoshi Iijima and Roshan Naik) --- .../taildir/ReliableTaildirEventReader.java | 4 +- .../apache/flume/source/taildir/TailFile.java | 129 ++++++++++++++---- 2 files changed, 101 insertions(+), 32 deletions(-) diff --git a/flume-ng-sources/flume-taildir-source/src/main/java/org/apache/flume/source/taildir/ReliableTaildirEventReader.java b/flume-ng-sources/flume-taildir-source/src/main/java/org/apache/flume/source/taildir/ReliableTaildirEventReader.java index 951b7867bd..5b6d465701 100644 --- a/flume-ng-sources/flume-taildir-source/src/main/java/org/apache/flume/source/taildir/ReliableTaildirEventReader.java +++ b/flume-ng-sources/flume-taildir-source/src/main/java/org/apache/flume/source/taildir/ReliableTaildirEventReader.java @@ -195,7 +195,7 @@ public List readEvents(int numEvents, boolean backoffWithoutNL) } logger.info("Last read was never committed - resetting position"); long lastPos = currentFile.getPos(); - currentFile.getRaf().seek(lastPos); + currentFile.updateFilePos(lastPos); } List events = currentFile.readEvents(numEvents, backoffWithoutNL, addByteOffset); if (events.isEmpty()) { @@ -223,7 +223,7 @@ public void close() throws IOException { @Override public void commit() throws IOException { if (!committed && currentFile != null) { - long pos = currentFile.getRaf().getFilePointer(); + long pos = currentFile.getLineReadPos(); currentFile.setPos(pos); currentFile.setLastUpdated(updateTime); committed = true; diff --git a/flume-ng-sources/flume-taildir-source/src/main/java/org/apache/flume/source/taildir/TailFile.java b/flume-ng-sources/flume-taildir-source/src/main/java/org/apache/flume/source/taildir/TailFile.java index 99683da48b..eabd357b5e 100644 --- a/flume-ng-sources/flume-taildir-source/src/main/java/org/apache/flume/source/taildir/TailFile.java +++ b/flume-ng-sources/flume-taildir-source/src/main/java/org/apache/flume/source/taildir/TailFile.java @@ -28,22 +28,21 @@ import java.util.List; import java.util.Map; -import org.apache.commons.lang.StringUtils; import org.apache.flume.Event; import org.apache.flume.event.EventBuilder; import org.slf4j.Logger; import org.slf4j.LoggerFactory; -import com.google.common.base.Charsets; import com.google.common.collect.Lists; -import com.google.common.io.ByteArrayDataOutput; -import com.google.common.io.ByteStreams; public class TailFile { private static final Logger logger = LoggerFactory.getLogger(TailFile.class); - private static final String LINE_SEP = "\n"; - private static final String LINE_SEP_WIN = "\r\n"; + private static final byte BYTE_NL = (byte) 10; + private static final byte BYTE_CR = (byte) 13; + + private final static int BUFFER_SIZE = 8192; + private final static int NEED_READING = -1; private RandomAccessFile raf; private final String path; @@ -52,17 +51,26 @@ public class TailFile { private long lastUpdated; private boolean needTail; private final Map headers; + private byte[] buffer; + private byte[] oldBuffer; + private int bufferPos; + private long lineReadPos; public TailFile(File file, Map headers, long inode, long pos) throws IOException { this.raf = new RandomAccessFile(file, "r"); - if (pos > 0) raf.seek(pos); + if (pos > 0) { + raf.seek(pos); + lineReadPos=pos; + } this.path = file.getAbsolutePath(); this.inode = inode; this.pos = pos; this.lastUpdated = 0L; this.needTail = true; this.headers = headers; + this.oldBuffer = new byte[0]; + this.bufferPos= NEED_READING; } public RandomAccessFile getRaf() { return raf; } @@ -72,20 +80,29 @@ public TailFile(File file, Map headers, long inode, long pos) public long getLastUpdated() { return lastUpdated; } public boolean needTail() { return needTail; } public Map getHeaders() { return headers; } + public long getLineReadPos() { return lineReadPos; } public void setPos(long pos) { this.pos = pos; } public void setLastUpdated(long lastUpdated) { this.lastUpdated = lastUpdated; } public void setNeedTail(boolean needTail) { this.needTail = needTail; } + public void setLineReadPos(long lineReadPos) { this.lineReadPos = lineReadPos; } public boolean updatePos(String path, long inode, long pos) throws IOException { if (this.inode == inode && this.path.equals(path)) { - raf.seek(pos); setPos(pos); + updateFilePos(pos); logger.info("Updated position, file: " + path + ", inode: " + inode + ", pos: " + pos); return true; } return false; } + public void updateFilePos(long pos) throws IOException { + raf.seek(pos); + lineReadPos = pos; + bufferPos= NEED_READING; + oldBuffer = new byte[0]; + } + public List readEvents(int numEvents, boolean backoffWithoutNL, boolean addByteOffset) throws IOException { @@ -101,44 +118,87 @@ public List readEvents(int numEvents, boolean backoffWithoutNL, } private Event readEvent(boolean backoffWithoutNL, boolean addByteOffset) throws IOException { - Long posTmp = raf.getFilePointer(); - String line = readLine(); + Long posTmp = getLineReadPos(); + LineResult line = readLine(); if (line == null) { return null; } - if (backoffWithoutNL && !line.endsWith(LINE_SEP)) { + if (backoffWithoutNL && !line.lineSepInclude) { logger.info("Backing off in file without newline: " + path + ", inode: " + inode + ", pos: " + raf.getFilePointer()); - raf.seek(posTmp); + updateFilePos(posTmp); return null; } - - String lineSep = LINE_SEP; - if(line.endsWith(LINE_SEP_WIN)) { - lineSep = LINE_SEP_WIN; - } - Event event = EventBuilder.withBody(StringUtils.removeEnd(line, lineSep), Charsets.UTF_8); + Event event = EventBuilder.withBody(line.line); if (addByteOffset == true) { event.getHeaders().put(BYTE_OFFSET_HEADER_KEY, posTmp.toString()); } return event; } - private String readLine() throws IOException { - ByteArrayDataOutput out = ByteStreams.newDataOutput(300); - int i = 0; - int c; - while ((c = raf.read()) != -1) { - i++; - out.write((byte) c); - if (c == LINE_SEP.charAt(0)) { + private void readFile() throws IOException { + if ((raf.length() - raf.getFilePointer()) < BUFFER_SIZE) { + buffer = new byte[(int) (raf.length() - raf.getFilePointer())]; + } else { + buffer = new byte[BUFFER_SIZE]; + } + raf.read(buffer, 0, buffer.length); + bufferPos = 0; + } + + private byte[] concatByteArrays(byte[] a, int startIdxA, int lenA, byte[] b, int startIdxB, int lenB) { + byte[] c = new byte[lenA + lenB]; + System.arraycopy(a, startIdxA, c, 0, lenA); + System.arraycopy(b, startIdxB, c, lenA, lenB); + return c; + } + + public LineResult readLine() throws IOException { + LineResult lineResult = null; + while (true) { + if (bufferPos == NEED_READING) { + if (raf.getFilePointer() < raf.length()) { + readFile(); + } else { + if (oldBuffer.length > 0) { + lineResult = new LineResult(false, oldBuffer); + oldBuffer = new byte[0]; + setLineReadPos(lineReadPos + lineResult.line.length); + } + break; + } + } + for (int i = bufferPos; i < buffer.length; i++) { + if (buffer[i] == BYTE_NL) { + int oldLen = oldBuffer.length; + // Don't copy last byte(NEW_LINE) + int lineLen = i - bufferPos; + // For windows, check for CR + if (i > 0 && buffer[i - 1] == BYTE_CR) { + lineLen -= 1; + } else if (oldBuffer.length > 0 && oldBuffer[oldBuffer.length - 1] == BYTE_CR) { + oldLen -= 1; + } + lineResult = new LineResult(true, + concatByteArrays(oldBuffer, 0, oldLen, buffer, bufferPos, lineLen)); + setLineReadPos(lineReadPos + (oldBuffer.length + (i - bufferPos + 1))); + oldBuffer = new byte[0]; + if (i + 1 < buffer.length) { + bufferPos = i + 1; + } else { + bufferPos = NEED_READING; + } + break; + } + } + if (lineResult != null) { break; } + // NEW_LINE not showed up at the end of the buffer + oldBuffer = concatByteArrays(oldBuffer, 0, oldBuffer.length, buffer, bufferPos, (buffer.length - bufferPos)); + bufferPos = NEED_READING; } - if (i == 0) { - return null; - } - return new String(out.toByteArray(), Charsets.UTF_8); + return lineResult; } public void close() { @@ -159,5 +219,14 @@ public int compare(File f1, File f2) { } } + private class LineResult { + final boolean lineSepInclude; + final byte[] line; + public LineResult(boolean lineSepInclude, byte[] line) { + super(); + this.lineSepInclude = lineSepInclude; + this.line = line; + } + } } From a8dd873588a282bb832c923010a485ab39ccf37b Mon Sep 17 00:00:00 2001 From: Roshan Naik Date: Tue, 29 Dec 2015 13:02:13 -0800 Subject: [PATCH 267/341] FLUME-2806. flume-ng.ps1 Error running script to start an agent on Windows (Liam Mousseau via Roshan Naik) --- bin/flume-ng.ps1 | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/bin/flume-ng.ps1 b/bin/flume-ng.ps1 index 1638bf1a16..8cdc331b82 100755 --- a/bin/flume-ng.ps1 +++ b/bin/flume-ng.ps1 @@ -166,7 +166,7 @@ Function GetClassPath ($cmd) { Function GetJavaPath { if ($env:JAVA_HOME) { - return "$env:JAVA_HOME\bin\java.exe" }ss + return "$env:JAVA_HOME\bin\java.exe" } Write-Host "WARN: JAVA_HOME not set" return '"' + (Resolve-Path "java.exe").Path + '"' } From e6416a0707a8be539a163bec9d7b12f7afbc36cf Mon Sep 17 00:00:00 2001 From: Jarek Jarcec Cecho Date: Fri, 15 Jan 2016 08:23:14 -0800 Subject: [PATCH 268/341] FLUME-2718: HTTP Source to support generic Stream Handler (Hari via Jarek Jarcec Cecho) --- .../apache/flume/source/http/BLOBHandler.java | 101 +++++++++++ .../flume/source/http/TestBLOBHandler.java | 170 ++++++++++++++++++ 2 files changed, 271 insertions(+) create mode 100644 flume-ng-core/src/main/java/org/apache/flume/source/http/BLOBHandler.java create mode 100644 flume-ng-core/src/test/java/org/apache/flume/source/http/TestBLOBHandler.java diff --git a/flume-ng-core/src/main/java/org/apache/flume/source/http/BLOBHandler.java b/flume-ng-core/src/main/java/org/apache/flume/source/http/BLOBHandler.java new file mode 100644 index 0000000000..a8163639a4 --- /dev/null +++ b/flume-ng-core/src/main/java/org/apache/flume/source/http/BLOBHandler.java @@ -0,0 +1,101 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package org.apache.flume.source.http; + +import java.io.InputStream; +import java.util.ArrayList; +import java.util.HashMap; +import java.util.List; +import java.util.Map; + +import javax.servlet.http.HttpServletRequest; + +import org.apache.commons.io.IOUtils; +import org.apache.commons.io.output.ByteArrayOutputStream; +import org.apache.flume.Context; +import org.apache.flume.Event; +import org.apache.flume.event.EventBuilder; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +import com.google.common.base.Preconditions; + +/** + * + * BLOBHandler for HTTPSource that accepts any binary stream of data as event. + * + */ +public class BLOBHandler implements HTTPSourceHandler { + + private static final Logger LOG = LoggerFactory.getLogger(BLOBHandler.class); + + private String commaSeparatedHeaders; + + private String[] mandatoryHeaders; + + public static final String MANDATORY_PARAMETERS = "mandatoryParameters"; + + public static final String DEFAULT_MANDATORY_PARAMETERS = ""; + + public static final String PARAMETER_SEPARATOR = ","; + + /** + * {@inheritDoc} + */ + @SuppressWarnings("unchecked") + @Override + public List getEvents(HttpServletRequest request) throws Exception { + Map headers = new HashMap(); + + InputStream inputStream = request.getInputStream(); + + Map parameters = request.getParameterMap(); + for (String parameter : parameters.keySet()) { + String value = parameters.get(parameter)[0]; + LOG.debug("Setting Header [Key, Value] as [{},{}] ",parameter, value); + headers.put(parameter, value); + } + + for (String header : mandatoryHeaders) { + Preconditions.checkArgument(headers.containsKey(header), + "Please specify " + header + " parameter in the request."); + } + + ByteArrayOutputStream outputStream = new ByteArrayOutputStream(); + try{ + IOUtils.copy(inputStream, outputStream); + LOG.debug("Building an Event with stream of size -- {}", outputStream.size()); + Event event = EventBuilder.withBody(outputStream.toByteArray(), headers); + event.setHeaders(headers); + List eventList = new ArrayList(); + eventList.add(event); + return eventList; + } + finally { + outputStream.close(); + inputStream.close(); + } + } + + @Override + public void configure(Context context) { + this.commaSeparatedHeaders = context.getString(MANDATORY_PARAMETERS, DEFAULT_MANDATORY_PARAMETERS); + this.mandatoryHeaders = commaSeparatedHeaders.split(PARAMETER_SEPARATOR); + } + +} diff --git a/flume-ng-core/src/test/java/org/apache/flume/source/http/TestBLOBHandler.java b/flume-ng-core/src/test/java/org/apache/flume/source/http/TestBLOBHandler.java new file mode 100644 index 0000000000..f770d51171 --- /dev/null +++ b/flume-ng-core/src/test/java/org/apache/flume/source/http/TestBLOBHandler.java @@ -0,0 +1,170 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package org.apache.flume.source.http; + +import static org.junit.Assert.assertEquals; +import static org.mockito.Mockito.mock; +import static org.mockito.Mockito.when; + +import java.io.ByteArrayInputStream; +import java.io.IOException; +import java.io.InputStream; +import java.util.HashMap; +import java.util.List; +import java.util.Map; + +import javax.servlet.ServletInputStream; +import javax.servlet.http.HttpServletRequest; + +import org.apache.flume.Context; +import org.apache.flume.Event; +import org.junit.Before; +import org.junit.Test; + +/** + * + */ +public class TestBLOBHandler { + + HTTPSourceHandler handler; + + @Before + public void setUp() { + handler = new BLOBHandler(); + } + + @SuppressWarnings({ "unchecked", "rawtypes" }) + @Test + public void testCSVData() throws Exception { + Map requestParameterMap = new HashMap(); + requestParameterMap.put("param1", new String[] { "value1" }); + requestParameterMap.put("param2", new String[] { "value2" }); + + HttpServletRequest req = mock(HttpServletRequest.class); + final String csvData = "a,b,c"; + + ServletInputStream servletInputStream = new DelegatingServletInputStream( + new ByteArrayInputStream(csvData.getBytes())); + + when(req.getInputStream()).thenReturn(servletInputStream); + when(req.getParameterMap()).thenReturn(requestParameterMap); + + Context context = mock(Context.class); + when( + context.getString(BLOBHandler.MANDATORY_PARAMETERS, + BLOBHandler.DEFAULT_MANDATORY_PARAMETERS)).thenReturn( + "param1,param2"); + + handler.configure(context); + List deserialized = handler.getEvents(req); + assertEquals(1, deserialized.size()); + Event e = deserialized.get(0); + + assertEquals(new String(e.getBody()), csvData); + assertEquals(e.getHeaders().get("param1"), "value1"); + assertEquals(e.getHeaders().get("param2"), "value2"); + } + + @SuppressWarnings({ "unchecked", "rawtypes" }) + @Test + public void testTabData() throws Exception { + Map requestParameterMap = new HashMap(); + requestParameterMap.put("param1", new String[] { "value1" }); + + HttpServletRequest req = mock(HttpServletRequest.class); + final String tabData = "a\tb\tc"; + + ServletInputStream servletInputStream = new DelegatingServletInputStream( + new ByteArrayInputStream(tabData.getBytes())); + + when(req.getInputStream()).thenReturn(servletInputStream); + when(req.getParameterMap()).thenReturn(requestParameterMap); + + Context context = mock(Context.class); + when( + context.getString(BLOBHandler.MANDATORY_PARAMETERS, + BLOBHandler.DEFAULT_MANDATORY_PARAMETERS)).thenReturn("param1"); + + handler.configure(context); + + List deserialized = handler.getEvents(req); + assertEquals(1, deserialized.size()); + Event e = deserialized.get(0); + + assertEquals(new String(e.getBody()), tabData); + assertEquals(e.getHeaders().get("param1"), "value1"); + } + + @SuppressWarnings({ "rawtypes" }) + @Test(expected = IllegalArgumentException.class) + public void testMissingParameters() throws Exception { + Map requestParameterMap = new HashMap(); + + HttpServletRequest req = mock(HttpServletRequest.class); + final String tabData = "a\tb\tc"; + + ServletInputStream servletInputStream = new DelegatingServletInputStream( + new ByteArrayInputStream(tabData.getBytes())); + + when(req.getInputStream()).thenReturn(servletInputStream); + when(req.getParameterMap()).thenReturn(requestParameterMap); + + Context context = mock(Context.class); + when( + context.getString(BLOBHandler.MANDATORY_PARAMETERS, + BLOBHandler.DEFAULT_MANDATORY_PARAMETERS)).thenReturn("param1"); + + handler.configure(context); + + handler.getEvents(req); + + } + + class DelegatingServletInputStream extends ServletInputStream { + + private final InputStream sourceStream; + + /** + * Create a DelegatingServletInputStream for the given source stream. + * + * @param sourceStream + * the source stream (never null) + */ + public DelegatingServletInputStream(InputStream sourceStream) { + this.sourceStream = sourceStream; + } + + /** + * Return the underlying source stream (never null). + */ + public final InputStream getSourceStream() { + return this.sourceStream; + } + + public int read() throws IOException { + return this.sourceStream.read(); + } + + public void close() throws IOException { + super.close(); + this.sourceStream.close(); + } + + } + +} From af63d38fada97a06c542ad875ef31ea3e74d53cc Mon Sep 17 00:00:00 2001 From: Johny Rufus Date: Sun, 17 Jan 2016 10:25:41 -0800 Subject: [PATCH 269/341] FLUME-2704. Configurable poll delay for spooling directory source (Somin Mithraa via Johny Rufus) --- .../org/apache/flume/source/SpoolDirectorySource.java | 8 ++++---- .../SpoolDirectorySourceConfigurationConstants.java | 4 ++++ flume-ng-doc/sphinx/FlumeUserGuide.rst | 1 + 3 files changed, 9 insertions(+), 4 deletions(-) diff --git a/flume-ng-core/src/main/java/org/apache/flume/source/SpoolDirectorySource.java b/flume-ng-core/src/main/java/org/apache/flume/source/SpoolDirectorySource.java index 0b11fc9b04..3fe947dfdf 100644 --- a/flume-ng-core/src/main/java/org/apache/flume/source/SpoolDirectorySource.java +++ b/flume-ng-core/src/main/java/org/apache/flume/source/SpoolDirectorySource.java @@ -45,9 +45,6 @@ public class SpoolDirectorySource extends AbstractSource implements private static final Logger logger = LoggerFactory .getLogger(SpoolDirectorySource.class); - // Delay used when polling for new files - private static final int POLL_DELAY_MS = 500; - /* Config options */ private String completedSuffix; private String spoolDirectory; @@ -72,6 +69,7 @@ public class SpoolDirectorySource extends AbstractSource implements private boolean hitChannelException = false; private int maxBackoff; private ConsumeOrder consumeOrder; + private int pollDelay; @Override public synchronized void start() { @@ -105,7 +103,7 @@ public synchronized void start() { Runnable runner = new SpoolDirectoryRunnable(reader, sourceCounter); executor.scheduleWithFixedDelay( - runner, 0, POLL_DELAY_MS, TimeUnit.MILLISECONDS); + runner, 0, pollDelay, TimeUnit.MILLISECONDS); super.start(); logger.debug("SpoolDirectorySource source started"); @@ -168,6 +166,8 @@ public synchronized void configure(Context context) { consumeOrder = ConsumeOrder.valueOf(context.getString(CONSUME_ORDER, DEFAULT_CONSUME_ORDER.toString()).toUpperCase(Locale.ENGLISH)); + pollDelay = context.getInteger(POLL_DELAY, DEFAULT_POLL_DELAY); + // "Hack" to support backwards compatibility with previous generation of // spooling directory source, which did not support deserializers Integer bufferMaxLineLength = context.getInteger(BUFFER_MAX_LINE_LENGTH); diff --git a/flume-ng-core/src/main/java/org/apache/flume/source/SpoolDirectorySourceConfigurationConstants.java b/flume-ng-core/src/main/java/org/apache/flume/source/SpoolDirectorySourceConfigurationConstants.java index 895433e411..505369764f 100644 --- a/flume-ng-core/src/main/java/org/apache/flume/source/SpoolDirectorySourceConfigurationConstants.java +++ b/flume-ng-core/src/main/java/org/apache/flume/source/SpoolDirectorySourceConfigurationConstants.java @@ -93,4 +93,8 @@ public enum ConsumeOrder { } public static final String CONSUME_ORDER = "consumeOrder"; public static final ConsumeOrder DEFAULT_CONSUME_ORDER = ConsumeOrder.OLDEST; + + /** Delay(in milliseconds) used when polling for new files. The default is 500ms */ + public static final String POLL_DELAY = "pollDelay"; + public static final int DEFAULT_POLL_DELAY = 500; } diff --git a/flume-ng-doc/sphinx/FlumeUserGuide.rst b/flume-ng-doc/sphinx/FlumeUserGuide.rst index 897a2ca14b..0f8461d8e5 100644 --- a/flume-ng-doc/sphinx/FlumeUserGuide.rst +++ b/flume-ng-doc/sphinx/FlumeUserGuide.rst @@ -999,6 +999,7 @@ consumeOrder oldest In which order files in the spooling direc directory will be scanned to pick the oldest/youngest file, which might be slow if there are a large number of files, while using ``random`` may cause old files to be consumed very late if new files keep coming in the spooling directory. +pollDelay 500 Delay (in milliseconds) used when polling for new files. maxBackoff 4000 The maximum time (in millis) to wait between consecutive attempts to write to the channel(s) if the channel is full. The source will start at a low backoff and increase it exponentially each time the channel throws a ChannelException, upto the value specified by this parameter. batchSize 100 Granularity at which to batch transfer to the channel inputCharset UTF-8 Character set used by deserializers that treat the input file as text. From 7962ce63bc1cfdfed6b54b8b211e785ccab350f5 Mon Sep 17 00:00:00 2001 From: Hari Shreedharan Date: Mon, 8 Feb 2016 17:40:35 -0800 Subject: [PATCH 270/341] FLUME-2875. Allow RollingFileSink to specify a file prefix and a file extension. (Ralph Goers via Hari) --- .../formatter/output/DefaultPathManager.java | 108 ++++++++++++++++++ .../flume/formatter/output/PathManager.java | 63 ++++------ .../formatter/output/PathManagerFactory.java | 82 +++++++++++++ .../formatter/output/PathManagerType.java | 43 +++++++ .../formatter/output/RollTimePathManager.java | 66 +++++++++++ .../apache/flume/sink/RollingFileSink.java | 8 +- .../flume/sink/TestRollingFileSink.java | 106 +++++++++++++++++ flume-ng-doc/sphinx/FlumeUserGuide.rst | 53 +++++---- 8 files changed, 460 insertions(+), 69 deletions(-) create mode 100644 flume-ng-core/src/main/java/org/apache/flume/formatter/output/DefaultPathManager.java create mode 100644 flume-ng-core/src/main/java/org/apache/flume/formatter/output/PathManagerFactory.java create mode 100644 flume-ng-core/src/main/java/org/apache/flume/formatter/output/PathManagerType.java create mode 100644 flume-ng-core/src/main/java/org/apache/flume/formatter/output/RollTimePathManager.java diff --git a/flume-ng-core/src/main/java/org/apache/flume/formatter/output/DefaultPathManager.java b/flume-ng-core/src/main/java/org/apache/flume/formatter/output/DefaultPathManager.java new file mode 100644 index 0000000000..176db7f9cf --- /dev/null +++ b/flume-ng-core/src/main/java/org/apache/flume/formatter/output/DefaultPathManager.java @@ -0,0 +1,108 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, + * software distributed under the License is distributed on an + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + * KIND, either express or implied. See the License for the + * specific language governing permissions and limitations + * under the License. + */ + +package org.apache.flume.formatter.output; + +import java.io.File; +import java.util.concurrent.atomic.AtomicInteger; +import org.apache.flume.Context; + +public class DefaultPathManager implements PathManager { + + private long seriesTimestamp; + private File baseDirectory; + private AtomicInteger fileIndex; + private String filePrefix; + private String extension; + + private static final String DEFAULT_FILE_PREFIX = ""; + private static final String DEFAULT_FILE_EXTENSION = ""; + private static final String FILE_EXTENSION = "extension"; + private static final String FILE_PREFIX = "prefix"; + + protected File currentFile; + + public DefaultPathManager(Context context) { + filePrefix = context.getString(FILE_PREFIX, DEFAULT_FILE_PREFIX); + extension = context.getString(FILE_EXTENSION, DEFAULT_FILE_EXTENSION); + seriesTimestamp = System.currentTimeMillis(); + fileIndex = new AtomicInteger(); + } + + @Override + public File nextFile() { + StringBuilder sb = new StringBuilder(); + sb.append(filePrefix).append(seriesTimestamp).append("-"); + sb.append(fileIndex.incrementAndGet()); + if (extension.length() > 0) { + sb.append(".").append(extension); + } + currentFile = new File(baseDirectory, sb.toString()); + + return currentFile; + } + + @Override + public File getCurrentFile() { + if (currentFile == null) { + return nextFile(); + } + + return currentFile; + } + + @Override + public void rotate() { + currentFile = null; + } + + @Override + public File getBaseDirectory() { + return baseDirectory; + } + + @Override + public void setBaseDirectory(File baseDirectory) { + this.baseDirectory = baseDirectory; + } + + public long getSeriesTimestamp() { + return seriesTimestamp; + } + + public String getPrefix() { + return filePrefix; + } + + public String getExtension() { + return extension; + } + + public AtomicInteger getFileIndex() { + return fileIndex; + } + + public static class Builder implements PathManager.Builder { + @Override + public PathManager build(Context context) { + return new DefaultPathManager(context); + } + } + +} diff --git a/flume-ng-core/src/main/java/org/apache/flume/formatter/output/PathManager.java b/flume-ng-core/src/main/java/org/apache/flume/formatter/output/PathManager.java index 933cc942e3..5a3066a1d5 100644 --- a/flume-ng-core/src/main/java/org/apache/flume/formatter/output/PathManager.java +++ b/flume-ng-core/src/main/java/org/apache/flume/formatter/output/PathManager.java @@ -16,58 +16,35 @@ * specific language governing permissions and limitations * under the License. */ - package org.apache.flume.formatter.output; import java.io.File; -import java.util.concurrent.atomic.AtomicInteger; +import org.apache.flume.Context; -public class PathManager { +/** + * Creates the files used by the RollingFileSink. + */ +public interface PathManager { + /** + * {@link Context} prefix + */ + public static String CTX_PREFIX = "pathManager."; - private long seriesTimestamp; - private File baseDirectory; - private AtomicInteger fileIndex; + File nextFile(); - private File currentFile; + File getCurrentFile(); - public PathManager() { - seriesTimestamp = System.currentTimeMillis(); - fileIndex = new AtomicInteger(); - } + void rotate(); - public File nextFile() { - currentFile = new File(baseDirectory, seriesTimestamp + "-" - + fileIndex.incrementAndGet()); + File getBaseDirectory(); - return currentFile; - } + void setBaseDirectory(File baseDirectory); - public File getCurrentFile() { - if (currentFile == null) { - return nextFile(); + /** + * Knows how to construct this path manager.
    + * Note: Implementations MUST provide a public a no-arg constructor. + */ + public interface Builder { + public PathManager build(Context context); } - - return currentFile; - } - - public void rotate() { - currentFile = null; - } - - public File getBaseDirectory() { - return baseDirectory; - } - - public void setBaseDirectory(File baseDirectory) { - this.baseDirectory = baseDirectory; - } - - public long getSeriesTimestamp() { - return seriesTimestamp; - } - - public AtomicInteger getFileIndex() { - return fileIndex; - } - } diff --git a/flume-ng-core/src/main/java/org/apache/flume/formatter/output/PathManagerFactory.java b/flume-ng-core/src/main/java/org/apache/flume/formatter/output/PathManagerFactory.java new file mode 100644 index 0000000000..4dbe08314a --- /dev/null +++ b/flume-ng-core/src/main/java/org/apache/flume/formatter/output/PathManagerFactory.java @@ -0,0 +1,82 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, + * software distributed under the License is distributed on an + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + * KIND, either express or implied. See the License for the + * specific language governing permissions and limitations + * under the License. + */ +package org.apache.flume.formatter.output; + +import com.google.common.base.Preconditions; +import java.util.Locale; +import org.apache.flume.Context; +import org.apache.flume.FlumeException; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +/** + * Create PathManager instances. + */ +public class PathManagerFactory { + private static final Logger logger = LoggerFactory.getLogger(PathManagerFactory.class); + + public static PathManager getInstance(String managerType, Context context) { + + Preconditions.checkNotNull(managerType, "path manager type must not be null"); + + // try to find builder class in enum of known output serializers + PathManagerType type; + try { + type = PathManagerType.valueOf(managerType.toUpperCase(Locale.ENGLISH)); + } catch (IllegalArgumentException e) { + logger.debug("Not in enum, loading builder class: {}", managerType); + type = PathManagerType.OTHER; + } + Class builderClass = type.getBuilderClass(); + + // handle the case where they have specified their own builder in the config + if (builderClass == null) { + try { + Class c = Class.forName(managerType); + if (c != null && PathManager.Builder.class.isAssignableFrom(c)) { + builderClass = (Class) c; + } else { + String errMessage = "Unable to instantiate Builder from " + + managerType + ": does not appear to implement " + + PathManager.Builder.class.getName(); + throw new FlumeException(errMessage); + } + } catch (ClassNotFoundException ex) { + logger.error("Class not found: " + managerType, ex); + throw new FlumeException(ex); + } + } + + // build the builder + PathManager.Builder builder; + try { + builder = builderClass.newInstance(); + } catch (InstantiationException ex) { + String errMessage = "Cannot instantiate builder: " + managerType; + logger.error(errMessage, ex); + throw new FlumeException(errMessage, ex); + } catch (IllegalAccessException ex) { + String errMessage = "Cannot instantiate builder: " + managerType; + logger.error(errMessage, ex); + throw new FlumeException(errMessage, ex); + } + + return builder.build(context); + } +} diff --git a/flume-ng-core/src/main/java/org/apache/flume/formatter/output/PathManagerType.java b/flume-ng-core/src/main/java/org/apache/flume/formatter/output/PathManagerType.java new file mode 100644 index 0000000000..4f1fa9399b --- /dev/null +++ b/flume-ng-core/src/main/java/org/apache/flume/formatter/output/PathManagerType.java @@ -0,0 +1,43 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, + * software distributed under the License is distributed on an + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + * KIND, either express or implied. See the License for the + * specific language governing permissions and limitations + * under the License. + */ +package org.apache.flume.formatter.output; + +import org.apache.flume.annotations.InterfaceAudience; +import org.apache.flume.annotations.InterfaceStability; + +/** + * + */ +@InterfaceAudience.Private +@InterfaceStability.Unstable +public enum PathManagerType { + DEFAULT(DefaultPathManager.Builder.class), + ROLLTIME(RollTimePathManager.Builder.class), + OTHER(null); + + private final Class builderClass; + + PathManagerType(Class builderClass) { + this.builderClass = builderClass; + } + + public Class getBuilderClass() { + return builderClass; + } +} diff --git a/flume-ng-core/src/main/java/org/apache/flume/formatter/output/RollTimePathManager.java b/flume-ng-core/src/main/java/org/apache/flume/formatter/output/RollTimePathManager.java new file mode 100644 index 0000000000..6883a9c13e --- /dev/null +++ b/flume-ng-core/src/main/java/org/apache/flume/formatter/output/RollTimePathManager.java @@ -0,0 +1,66 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, + * software distributed under the License is distributed on an + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + * KIND, either express or implied. See the License for the + * specific language governing permissions and limitations + * under the License. + */ + +package org.apache.flume.formatter.output; + +import java.io.File; + +import org.apache.flume.Context; +import org.joda.time.LocalDateTime; +import org.joda.time.format.DateTimeFormat; +import org.joda.time.format.DateTimeFormatter; + +/** + * + */ +public class RollTimePathManager extends DefaultPathManager { + + private final DateTimeFormatter formatter = DateTimeFormat.forPattern("yyyyMMddHHmmss"); + private String lastRoll; + + public RollTimePathManager(Context context) { + super(context); + } + + @Override + public File nextFile() { + StringBuilder sb = new StringBuilder(); + String date = formatter.print(LocalDateTime.now()); + if (!date.equals(lastRoll)) { + getFileIndex().set(0); + lastRoll = date; + } + sb.append(getPrefix()).append(date).append("-"); + sb.append(getFileIndex().incrementAndGet()); + if (getExtension().length() > 0) { + sb.append(".").append(getExtension()); + } + currentFile = new File(getBaseDirectory(), sb.toString()); + + return currentFile; + } + + public static class Builder implements PathManager.Builder { + @Override + public PathManager build(Context context) { + return new RollTimePathManager(context); + } + } + +} diff --git a/flume-ng-core/src/main/java/org/apache/flume/sink/RollingFileSink.java b/flume-ng-core/src/main/java/org/apache/flume/sink/RollingFileSink.java index 9cb3370883..b97d404a03 100644 --- a/flume-ng-core/src/main/java/org/apache/flume/sink/RollingFileSink.java +++ b/flume-ng-core/src/main/java/org/apache/flume/sink/RollingFileSink.java @@ -33,6 +33,7 @@ import org.apache.flume.Transaction; import org.apache.flume.conf.Configurable; import org.apache.flume.formatter.output.PathManager; +import org.apache.flume.formatter.output.PathManagerFactory; import org.apache.flume.instrumentation.SinkCounter; import org.slf4j.Logger; import org.slf4j.LoggerFactory; @@ -66,13 +67,13 @@ public class RollingFileSink extends AbstractSink implements Configurable { private volatile boolean shouldRotate; public RollingFileSink() { - pathController = new PathManager(); shouldRotate = false; } @Override public void configure(Context context) { + String pathManagerType = context.getString("sink.pathManager", "DEFAULT"); String directory = context.getString("sink.directory"); String rollInterval = context.getString("sink.rollInterval"); @@ -81,6 +82,11 @@ public void configure(Context context) { new Context(context.getSubProperties("sink." + EventSerializer.CTX_PREFIX)); + Context pathManagerContext = + new Context(context.getSubProperties("sink." + + PathManager.CTX_PREFIX)); + pathController = PathManagerFactory.getInstance(pathManagerType, pathManagerContext); + Preconditions.checkArgument(directory != null, "Directory may not be null"); Preconditions.checkNotNull(serializerType, "Serializer type is undefined"); diff --git a/flume-ng-core/src/test/java/org/apache/flume/sink/TestRollingFileSink.java b/flume-ng-core/src/test/java/org/apache/flume/sink/TestRollingFileSink.java index 07fa6443a0..bf4ed1ff58 100644 --- a/flume-ng-core/src/test/java/org/apache/flume/sink/TestRollingFileSink.java +++ b/flume-ng-core/src/test/java/org/apache/flume/sink/TestRollingFileSink.java @@ -172,4 +172,110 @@ public void testAppend2() throws InterruptedException, LifecycleException, reader.close(); } } + + @Test + public void testAppend3() throws InterruptedException, LifecycleException, EventDeliveryException, IOException { + File tmpDir = new File("target/tmpLog"); + tmpDir.mkdirs(); + cleanDirectory(tmpDir); + Context context = new Context(); + + context.put("sink.directory", "target/tmpLog"); + context.put("sink.rollInterval", "0"); + context.put("sink.batchSize", "1"); + context.put("sink.pathManager.prefix", "test3-"); + context.put("sink.pathManager.extension", "txt"); + + Configurables.configure(sink, context); + + Channel channel = new PseudoTxnMemoryChannel(); + Configurables.configure(channel, context); + + sink.setChannel(channel); + sink.start(); + + for (int i = 0; i < 10; i++) { + Event event = new SimpleEvent(); + + event.setBody(("Test event " + i).getBytes()); + + channel.put(event); + sink.process(); + + Thread.sleep(500); + } + + sink.stop(); + + for (String file : sink.getDirectory().list()) { + BufferedReader reader = new BufferedReader(new FileReader(new File(sink.getDirectory(), file))); + + String lastLine = null; + String currentLine = null; + + while ((currentLine = reader.readLine()) != null) { + lastLine = currentLine; + logger.debug("Produced file:{} lastLine:{}", file, lastLine); + } + + reader.close(); + } + } + + @Test + public void testRollTime() throws InterruptedException, LifecycleException, EventDeliveryException, IOException { + File tmpDir = new File("target/tempLog"); + tmpDir.mkdirs(); + cleanDirectory(tmpDir); + Context context = new Context(); + + context.put("sink.directory", "target/tempLog/"); + context.put("sink.rollInterval", "1"); + context.put("sink.batchSize", "1"); + context.put("sink.pathManager", "rolltime"); + context.put("sink.pathManager.prefix", "test4-"); + context.put("sink.pathManager.extension", "txt"); + + Configurables.configure(sink, context); + + Channel channel = new PseudoTxnMemoryChannel(); + Configurables.configure(channel, context); + + sink.setChannel(channel); + sink.start(); + + for (int i = 0; i < 10; i++) { + Event event = new SimpleEvent(); + + event.setBody(("Test event " + i).getBytes()); + + channel.put(event); + sink.process(); + + Thread.sleep(500); + } + + sink.stop(); + + for (String file : sink.getDirectory().list()) { + BufferedReader reader = new BufferedReader(new FileReader(new File(sink.getDirectory(), file))); + + String lastLine = null; + String currentLine = null; + + while ((currentLine = reader.readLine()) != null) { + lastLine = currentLine; + logger.debug("Produced file:{} lastLine:{}", file, lastLine); + } + + reader.close(); + } + } + + private void cleanDirectory(File dir) { + File[] files = dir.listFiles(); + for (File file : files) { + file.delete(); + } + } } diff --git a/flume-ng-doc/sphinx/FlumeUserGuide.rst b/flume-ng-doc/sphinx/FlumeUserGuide.rst index 0f8461d8e5..423e0cf8a3 100644 --- a/flume-ng-doc/sphinx/FlumeUserGuide.rst +++ b/flume-ng-doc/sphinx/FlumeUserGuide.rst @@ -28,22 +28,22 @@ Apache Flume is a distributed, reliable, and available system for efficiently collecting, aggregating and moving large amounts of log data from many different sources to a centralized data store. -The use of Apache Flume is not only restricted to log data aggregation. +The use of Apache Flume is not only restricted to log data aggregation. Since data sources are customizable, Flume can be used to transport massive quantities -of event data including but not limited to network traffic data, social-media-generated data, +of event data including but not limited to network traffic data, social-media-generated data, email messages and pretty much any data source possible. Apache Flume is a top level project at the Apache Software Foundation. There are currently two release code lines available, versions 0.9.x and 1.x. -Documentation for the 0.9.x track is available at +Documentation for the 0.9.x track is available at `the Flume 0.9.x User Guide `_. This documentation applies to the 1.4.x track. -New and existing users are encouraged to use the 1.x releases so as to -leverage the performance improvements and configuration flexibilities available +New and existing users are encouraged to use the 1.x releases so as to +leverage the performance improvements and configuration flexibilities available in the latest architecture. @@ -1153,7 +1153,7 @@ Twitter 1% firehose Source (experimental) Experimental source that connects via Streaming API to the 1% sample twitter firehose, continously downloads tweets, converts them to Avro format and -sends Avro events to a downstream Flume sink. Requires the consumer and +sends Avro events to a downstream Flume sink. Requires the consumer and access tokens and secrets of a Twitter developer account. Required properties are in **bold**. @@ -1165,7 +1165,7 @@ Property Name Default Description **consumerKey** -- OAuth consumer key **consumerSecret** -- OAuth consumer secret **accessToken** -- OAuth access token -**accessTokenSecret** -- OAuth toekn secret +**accessTokenSecret** -- OAuth toekn secret maxBatchSize 1000 Maximum number of twitter messages to put in a single batch maxBatchDurationMillis 1000 Maximum number of milliseconds to wait before closing a batch ====================== =========== =================================================== @@ -2119,16 +2119,19 @@ File Roll Sink Stores events on the local filesystem. Required properties are in **bold**. -=================== ======= ====================================================================================================================== -Property Name Default Description -=================== ======= ====================================================================================================================== -**channel** -- -**type** -- The component type name, needs to be ``file_roll``. -**sink.directory** -- The directory where files will be stored -sink.rollInterval 30 Roll the file every 30 seconds. Specifying 0 will disable rolling and cause all events to be written to a single file. -sink.serializer TEXT Other possible options include ``avro_event`` or the FQCN of an implementation of EventSerializer.Builder interface. -batchSize 100 -=================== ======= ====================================================================================================================== +========================== ======= ====================================================================================================================== +Property Name Default Description +========================== ======= ====================================================================================================================== +**channel** -- +**type** -- The component type name, needs to be ``file_roll``. +**sink.directory** -- The directory where files will be stored +sink.pathManager DEFAULT The PathManager implementation to use. +sink.pathManager.extension -- The file extension if the default PathManager is used. +sink.pathManager.prefix -- A character string to add to the beginning of the file name if the default PathManager is used +sink.rollInterval 30 Roll the file every 30 seconds. Specifying 0 will disable rolling and cause all events to be written to a single file. +sink.serializer TEXT Other possible options include ``avro_event`` or the FQCN of an implementation of EventSerializer.Builder interface. +batchSize 100 +========================== ======= ====================================================================================================================== Example for agent named a1: @@ -2284,19 +2287,19 @@ This sink extracts data from Flume events, transforms it, and loads it in near-r This sink is well suited for use cases that stream raw data into HDFS (via the HdfsSink) and simultaneously extract, transform and load the same data into Solr (via MorphlineSolrSink). In particular, this sink can process arbitrary heterogeneous raw data from disparate data sources and turn it into a data model that is useful to Search applications. -The ETL functionality is customizable using a `morphline configuration file `_ that defines a chain of transformation commands that pipe event records from one command to another. +The ETL functionality is customizable using a `morphline configuration file `_ that defines a chain of transformation commands that pipe event records from one command to another. Morphlines can be seen as an evolution of Unix pipelines where the data model is generalized to work with streams of generic records, including arbitrary binary payloads. A morphline command is a bit like a Flume Interceptor. Morphlines can be embedded into Hadoop components such as Flume. Commands to parse and transform a set of standard data formats such as log files, Avro, CSV, Text, HTML, XML, PDF, Word, Excel, etc. are provided out of the box, and additional custom commands and parsers for additional data formats can be added as morphline plugins. Any kind of data format can be indexed and any Solr documents for any kind of Solr schema can be generated, and any custom ETL logic can be registered and executed. -Morphlines manipulate continuous streams of records. The data model can be described as follows: A record is a set of named fields where each field has an ordered list of one or more values. A value can be any Java Object. That is, a record is essentially a hash table where each hash table entry contains a String key and a list of Java Objects as values. (The implementation uses Guava's ``ArrayListMultimap``, which is a ``ListMultimap``). Note that a field can have multiple values and any two records need not use common field names. +Morphlines manipulate continuous streams of records. The data model can be described as follows: A record is a set of named fields where each field has an ordered list of one or more values. A value can be any Java Object. That is, a record is essentially a hash table where each hash table entry contains a String key and a list of Java Objects as values. (The implementation uses Guava's ``ArrayListMultimap``, which is a ``ListMultimap``). Note that a field can have multiple values and any two records need not use common field names. This sink fills the body of the Flume event into the ``_attachment_body`` field of the morphline record, as well as copies the headers of the Flume event into record fields of the same name. The commands can then act on this data. Routing to a SolrCloud cluster is supported to improve scalability. Indexing load can be spread across a large number of MorphlineSolrSinks for improved scalability. Indexing load can be replicated across multiple MorphlineSolrSinks for high availability, for example using Flume features such as Load balancing Sink Processor. MorphlineInterceptor can also help to implement dynamic routing to multiple Solr collections (e.g. for multi-tenancy). -The morphline and solr jars required for your environment must be placed in the lib directory of the Apache Flume installation. +The morphline and solr jars required for your environment must be placed in the lib directory of the Apache Flume installation. The type is the FQCN: org.apache.flume.sink.solr.morphline.MorphlineSolrSink @@ -2334,11 +2337,11 @@ ElasticSearchSink ~~~~~~~~~~~~~~~~~ This sink writes data to an elasticsearch cluster. By default, events will be written so that the `Kibana `_ graphical interface -can display them - just as if `logstash `_ wrote them. +can display them - just as if `logstash `_ wrote them. -The elasticsearch and lucene-core jars required for your environment must be placed in the lib directory of the Apache Flume installation. +The elasticsearch and lucene-core jars required for your environment must be placed in the lib directory of the Apache Flume installation. Elasticsearch requires that the major version of the client JAR match that of the server and that both are running the same minor version -of the JVM. SerializationExceptions will appear if this is incorrect. To +of the JVM. SerializationExceptions will appear if this is incorrect. To select the required version first determine the version of elasticsearch and the JVM version the target cluster is running. Then select an elasticsearch client library which matches the major version. A 0.19.x client can talk to a 0.19.x cluster; 0.20.x can talk to 0.20.x and 0.90.x can talk to 0.90.x. Once the elasticsearch version has been determined then read the pom.xml file to determine the correct lucene-core JAR version to use. The Flume agent @@ -2588,7 +2591,7 @@ Example for agent named a1: a1.channels.c1.transactionCapacity = 10000 a1.channels.c1.byteCapacityBufferPercentage = 20 a1.channels.c1.byteCapacity = 800000 - + JDBC Channel ~~~~~~~~~~~~ @@ -2796,7 +2799,7 @@ The disk store is managed using an embedded File channel. When the in-memory que the file channel. This channel is ideal for flows that need high throughput of memory channel during normal operation, but at the same time need the larger capacity of the file channel for better tolerance of intermittent sink side outages or drop in drain rates. The throughput will reduce approximately to file channel speeds during such abnormal situations. In case of an agent crash or restart, -only the events stored on disk are recovered when the agent comes online. **This channel is currently experimental and +only the events stored on disk are recovered when the agent comes online. **This channel is currently experimental and not recommended for use in production.** Required properties are in **bold**. Please refer to file channel for additional required properties. From de6ecf48568761de829c8d8caf9bc41f9bf9e64c Mon Sep 17 00:00:00 2001 From: Roshan Naik Date: Wed, 17 Feb 2016 14:11:44 -0800 Subject: [PATCH 271/341] FLUME-2881. Windows Launch Script fails in plugins dir code (Jonathan Smith via Roshan Naik) --- bin/flume-ng.ps1 | 12 ++++++++---- 1 file changed, 8 insertions(+), 4 deletions(-) diff --git a/bin/flume-ng.ps1 b/bin/flume-ng.ps1 index 8cdc331b82..4778717b7f 100755 --- a/bin/flume-ng.ps1 +++ b/bin/flume-ng.ps1 @@ -319,10 +319,14 @@ if ("$pluginsPath" -eq "") { foreach($plugin in $pluginsPath.Split(";") ) { if ( Test-path "$plugin" ) { - $pluginTmp = - ( (Get-ChildItem "$plugin\*\lib") + (Get-ChildItem "$plugin\*\libext") ) -join "\*"";""" - if( "$pluginTmp" -ne "" ) { - $javaClassPath="$javaClassPath;""" + $pluginTmp + "\*"";" + $pluginTmp1 = (@(Get-ChildItem "$plugin\*\lib") -join "\*"";""") + if( "$pluginTmp1" -ne "" ) { + $javaClassPath="$javaClassPath;""" + $pluginTmp1 + "\*"";" + } + + $pluginTmp2 = (@(Get-ChildItem "$plugin\*\libext") -join "\*"";""") + if( "$pluginTmp2" -ne "" ) { + $javaClassPath="$javaClassPath;""" + $pluginTmp2 + "\*"";" } $javaLibraryPathTmp = (@(Get-ChildItem "$plugin\*\native") -join "\*"";""") From 109ec30725a4c665a2ccf5f40af8a0e455cf4166 Mon Sep 17 00:00:00 2001 From: Jarek Jarcec Cecho Date: Tue, 23 Feb 2016 08:17:34 -0800 Subject: [PATCH 272/341] FLUME-2886: Optional Channels can cause OOMs (Hari Shreedharan via Jarek Jarcec Cecho) --- .../flume/channel/ChannelProcessor.java | 13 +++++-- .../flume/channel/TestChannelProcessor.java | 38 +++++++++++++++++++ 2 files changed, 47 insertions(+), 4 deletions(-) diff --git a/flume-ng-core/src/main/java/org/apache/flume/channel/ChannelProcessor.java b/flume-ng-core/src/main/java/org/apache/flume/channel/ChannelProcessor.java index f2612a660e..7b2de7c78f 100644 --- a/flume-ng-core/src/main/java/org/apache/flume/channel/ChannelProcessor.java +++ b/flume-ng-core/src/main/java/org/apache/flume/channel/ChannelProcessor.java @@ -26,8 +26,7 @@ import java.util.LinkedHashMap; import java.util.List; import java.util.Map; -import java.util.concurrent.Executors; -import java.util.concurrent.ExecutorService; +import java.util.concurrent.*; import org.apache.flume.Channel; import org.apache.flume.ChannelException; @@ -62,6 +61,7 @@ public class ChannelProcessor implements Configurable { private final ChannelSelector selector; private final InterceptorChain interceptorChain; private ExecutorService execService; + BlockingQueue taskQueue; public ChannelProcessor(ChannelSelector selector) { this.selector = selector; @@ -82,8 +82,13 @@ public void close() { */ @Override public void configure(Context context) { - this.execService = Executors.newSingleThreadExecutor( - new ThreadFactoryBuilder().setNameFormat("OptionalChannelProcessorThread").build()); + int queueSize = context.getInteger("pendingTransactions", 20); + taskQueue = new ArrayBlockingQueue(queueSize, true); + ThreadFactory factory = new ThreadFactoryBuilder() + .setNameFormat("OptionalChannelProcessorThread").build(); + this.execService = + new ThreadPoolExecutor(1, 1, 0, TimeUnit.MILLISECONDS, taskQueue, + factory, new ThreadPoolExecutor.DiscardPolicy()); configureInterceptors(context); } diff --git a/flume-ng-core/src/test/java/org/apache/flume/channel/TestChannelProcessor.java b/flume-ng-core/src/test/java/org/apache/flume/channel/TestChannelProcessor.java index 924c998b76..c2a57487c4 100644 --- a/flume-ng-core/src/test/java/org/apache/flume/channel/TestChannelProcessor.java +++ b/flume-ng-core/src/test/java/org/apache/flume/channel/TestChannelProcessor.java @@ -148,4 +148,42 @@ public void testRequiredAndOptionalChannels() { } } + @SuppressWarnings("unchecked") + @Test + public void testOptionalChannelQueueSize() throws InterruptedException { + Context context = new Context(); + context.put("capacity", "100"); + context.put("transactionCapacity", "3"); + context.put("pendingTransactions", "2"); + + ArrayList channels = new ArrayList(); + for (int i = 0; i < 2; i++) { + MemoryChannel ch = new MemoryChannel(); + ch.setName("ch" + i); + channels.add(ch); + } + Configurables.configure(channels.get(0), context); + context.put("capacity", "3"); + Configurables.configure(channels.get(1), context); + ChannelSelector selector = new ReplicatingChannelSelector(); + selector.setChannels((List) channels); + + context.put(ReplicatingChannelSelector.CONFIG_OPTIONAL, "ch1"); + Configurables.configure(selector, context); + + ChannelProcessor processor = new ChannelProcessor(selector); + Configurables.configure(processor, context); + + // The idea is to put more events into the optional channel than its capacity + the size of + // the task queue. So the remaining events get added to the task queue, but since it is + // bounded, its size should not grow indefinitely either. + for (int i = 0; i <= 6; i++) { + processor.processEvent(EventBuilder.withBody("e".getBytes())); + // To avoid tasks from being rejected so if previous events are still not committed, wait + // between transactions. + Thread.sleep(500); + } + // 3 in channel, 1 executing, 2 in queue, 1 rejected + Assert.assertEquals(2, processor.taskQueue.size()); + } } From ffb52b9e656df51e5e6881cfc8ed851a89cdc2f1 Mon Sep 17 00:00:00 2001 From: Roshan Naik Date: Thu, 25 Feb 2016 17:31:24 -0800 Subject: [PATCH 273/341] FLUME-2889. Fixes to DateTime computations (Roshan Naik via Hari Shreedharan) --- .../src/main/java/org/apache/flume/source/SyslogParser.java | 4 ++-- .../apache/flume/serialization/SyslogAvroEventSerializer.java | 4 ++-- 2 files changed, 4 insertions(+), 4 deletions(-) diff --git a/flume-ng-core/src/main/java/org/apache/flume/source/SyslogParser.java b/flume-ng-core/src/main/java/org/apache/flume/source/SyslogParser.java index b61f7459ec..c8245ff536 100644 --- a/flume-ng-core/src/main/java/org/apache/flume/source/SyslogParser.java +++ b/flume-ng-core/src/main/java/org/apache/flume/source/SyslogParser.java @@ -325,10 +325,10 @@ protected long parseRfc3164Time(String ts) { // flume clock is ahead or there is some latency, and the year rolled if (fixed.isAfter(now) && fixed.minusMonths(1).isAfter(now)) { - fixed = date.withYear(year - 1); + fixed = date.minusYears(1); // flume clock is behind and the year rolled } else if (fixed.isBefore(now) && fixed.plusMonths(1).isBefore(now)) { - fixed = date.withYear(year + 1); + fixed = date.plusYears(1); } date = fixed; } diff --git a/flume-ng-core/src/test/java/org/apache/flume/serialization/SyslogAvroEventSerializer.java b/flume-ng-core/src/test/java/org/apache/flume/serialization/SyslogAvroEventSerializer.java index d1cbcae5ff..896eced96b 100644 --- a/flume-ng-core/src/test/java/org/apache/flume/serialization/SyslogAvroEventSerializer.java +++ b/flume-ng-core/src/test/java/org/apache/flume/serialization/SyslogAvroEventSerializer.java @@ -190,10 +190,10 @@ private static long parseRfc3164Date(String in) { // flume clock is ahead or there is some latency, and the year rolled if (corrected.isAfter(now) && corrected.minusMonths(1).isAfter(now)) { - corrected = date.withYear(year - 1); + corrected = date.minusYears(1); // flume clock is behind and the year rolled } else if (corrected.isBefore(now) && corrected.plusMonths(1).isBefore(now)) { - corrected = date.withYear(year + 1); + corrected = date.plusYears(1); } date = corrected; } From caa64a1a6d4bc97be5993cb468516e9ffe862794 Mon Sep 17 00:00:00 2001 From: Jarek Jarcec Cecho Date: Wed, 9 Mar 2016 11:05:01 -0800 Subject: [PATCH 274/341] FLUME-2891: Revert FLUME-2712 and FLUME-2886 (Hari Shreedharan via Jarek Jarcec Cecho) --- .../flume/channel/ChannelProcessor.java | 151 +++++++++++------- .../flume/channel/TestChannelProcessor.java | 52 +----- 2 files changed, 98 insertions(+), 105 deletions(-) diff --git a/flume-ng-core/src/main/java/org/apache/flume/channel/ChannelProcessor.java b/flume-ng-core/src/main/java/org/apache/flume/channel/ChannelProcessor.java index 7b2de7c78f..1cce137f78 100644 --- a/flume-ng-core/src/main/java/org/apache/flume/channel/ChannelProcessor.java +++ b/flume-ng-core/src/main/java/org/apache/flume/channel/ChannelProcessor.java @@ -20,13 +20,10 @@ import com.google.common.base.Preconditions; import com.google.common.collect.Lists; -import com.google.common.util.concurrent.ThreadFactoryBuilder; - import java.util.ArrayList; import java.util.LinkedHashMap; import java.util.List; import java.util.Map; -import java.util.concurrent.*; import org.apache.flume.Channel; import org.apache.flume.ChannelException; @@ -60,8 +57,6 @@ public class ChannelProcessor implements Configurable { private final ChannelSelector selector; private final InterceptorChain interceptorChain; - private ExecutorService execService; - BlockingQueue taskQueue; public ChannelProcessor(ChannelSelector selector) { this.selector = selector; @@ -82,13 +77,6 @@ public void close() { */ @Override public void configure(Context context) { - int queueSize = context.getInteger("pendingTransactions", 20); - taskQueue = new ArrayBlockingQueue(queueSize, true); - ThreadFactory factory = new ThreadFactoryBuilder() - .setNameFormat("OptionalChannelProcessorThread").build(); - this.execService = - new ThreadPoolExecutor(1, 1, 0, TimeUnit.MILLISECONDS, taskQueue, - factory, new ThreadPoolExecutor.DiscardPolicy()); configureInterceptors(context); } @@ -165,6 +153,7 @@ public void processEventBatch(List events) { for (Event event : events) { List reqChannels = selector.getRequiredChannels(event); + for (Channel ch : reqChannels) { List eventQueue = reqChannelQueue.get(ch); if (eventQueue == null) { @@ -175,26 +164,74 @@ public void processEventBatch(List events) { } List optChannels = selector.getOptionalChannels(event); + for (Channel ch: optChannels) { List eventQueue = optChannelQueue.get(ch); if (eventQueue == null) { eventQueue = new ArrayList(); optChannelQueue.put(ch, eventQueue); } + eventQueue.add(event); } } // Process required channels for (Channel reqChannel : reqChannelQueue.keySet()) { - List batch = reqChannelQueue.get(reqChannel); - executeChannelTransaction(reqChannel, batch, false); + Transaction tx = reqChannel.getTransaction(); + Preconditions.checkNotNull(tx, "Transaction object must not be null"); + try { + tx.begin(); + + List batch = reqChannelQueue.get(reqChannel); + + for (Event event : batch) { + reqChannel.put(event); + } + + tx.commit(); + } catch (Throwable t) { + tx.rollback(); + if (t instanceof Error) { + LOG.error("Error while writing to required channel: " + + reqChannel, t); + throw (Error) t; + } else { + throw new ChannelException("Unable to put batch on required " + + "channel: " + reqChannel, t); + } + } finally { + if (tx != null) { + tx.close(); + } + } } // Process optional channels for (Channel optChannel : optChannelQueue.keySet()) { - List batch = optChannelQueue.get(optChannel); - execService.submit(new OptionalChannelTransactionRunnable(optChannel, batch)); + Transaction tx = optChannel.getTransaction(); + Preconditions.checkNotNull(tx, "Transaction object must not be null"); + try { + tx.begin(); + + List batch = optChannelQueue.get(optChannel); + + for (Event event : batch ) { + optChannel.put(event); + } + + tx.commit(); + } catch (Throwable t) { + tx.rollback(); + LOG.error("Unable to put batch on optional channel: " + optChannel, t); + if (t instanceof Error) { + throw (Error) t; + } + } finally { + if (tx != null) { + tx.close(); + } + } } } @@ -216,59 +253,57 @@ public void processEvent(Event event) { if (event == null) { return; } - List events = new ArrayList(1); - events.add(event); // Process required channels List requiredChannels = selector.getRequiredChannels(event); for (Channel reqChannel : requiredChannels) { - executeChannelTransaction(reqChannel, events, false); + Transaction tx = reqChannel.getTransaction(); + Preconditions.checkNotNull(tx, "Transaction object must not be null"); + try { + tx.begin(); + + reqChannel.put(event); + + tx.commit(); + } catch (Throwable t) { + tx.rollback(); + if (t instanceof Error) { + LOG.error("Error while writing to required channel: " + + reqChannel, t); + throw (Error) t; + } else { + throw new ChannelException("Unable to put event on required " + + "channel: " + reqChannel, t); + } + } finally { + if (tx != null) { + tx.close(); + } + } } // Process optional channels List optionalChannels = selector.getOptionalChannels(event); for (Channel optChannel : optionalChannels) { - execService.submit(new OptionalChannelTransactionRunnable(optChannel, events)); - } - } - - private static void executeChannelTransaction(Channel channel, List batch, boolean isOptional) { - Transaction tx = channel.getTransaction(); - Preconditions.checkNotNull(tx, "Transaction object must not be null"); - try { - tx.begin(); + Transaction tx = null; + try { + tx = optChannel.getTransaction(); + tx.begin(); - for (Event event : batch) { - channel.put(event); - } + optChannel.put(event); - tx.commit(); - } catch (Throwable t) { - tx.rollback(); - if (t instanceof Error) { - LOG.error("Error while writing to channel: " + - channel, t); - throw (Error) t; - } else if(!isOptional) { - throw new ChannelException("Unable to put batch on required " + - "channel: " + channel, t); + tx.commit(); + } catch (Throwable t) { + tx.rollback(); + LOG.error("Unable to put event on optional channel: " + optChannel, t); + if (t instanceof Error) { + throw (Error) t; + } + } finally { + if (tx != null) { + tx.close(); + } } - } finally { - tx.close(); - } - } - - private static class OptionalChannelTransactionRunnable implements Runnable { - private Channel channel; - private List events; - - OptionalChannelTransactionRunnable(Channel channel, List events) { - this.channel = channel; - this.events = events; - } - - public void run() { - executeChannelTransaction(channel, events, true); } } -} \ No newline at end of file +} diff --git a/flume-ng-core/src/test/java/org/apache/flume/channel/TestChannelProcessor.java b/flume-ng-core/src/test/java/org/apache/flume/channel/TestChannelProcessor.java index c2a57487c4..b37b823036 100644 --- a/flume-ng-core/src/test/java/org/apache/flume/channel/TestChannelProcessor.java +++ b/flume-ng-core/src/test/java/org/apache/flume/channel/TestChannelProcessor.java @@ -23,12 +23,8 @@ import java.util.ArrayList; import java.util.List; -import org.apache.flume.Channel; -import org.apache.flume.ChannelException; -import org.apache.flume.ChannelSelector; -import org.apache.flume.Event; -import org.apache.flume.Transaction; -import org.apache.flume.Context; + +import org.apache.flume.*; import org.apache.flume.conf.Configurables; import org.apache.flume.event.EventBuilder; import org.junit.Assert; @@ -85,9 +81,9 @@ public void testNullFromGetTransaction() { } /* - * Test delivery to optional and required channels - * Test both processEvent and processEventBatch - */ + * Test delivery to optional and required channels + * Test both processEvent and processEventBatch + */ @Test public void testRequiredAndOptionalChannels() { Context context = new Context(); @@ -148,42 +144,4 @@ public void testRequiredAndOptionalChannels() { } } - @SuppressWarnings("unchecked") - @Test - public void testOptionalChannelQueueSize() throws InterruptedException { - Context context = new Context(); - context.put("capacity", "100"); - context.put("transactionCapacity", "3"); - context.put("pendingTransactions", "2"); - - ArrayList channels = new ArrayList(); - for (int i = 0; i < 2; i++) { - MemoryChannel ch = new MemoryChannel(); - ch.setName("ch" + i); - channels.add(ch); - } - Configurables.configure(channels.get(0), context); - context.put("capacity", "3"); - Configurables.configure(channels.get(1), context); - ChannelSelector selector = new ReplicatingChannelSelector(); - selector.setChannels((List) channels); - - context.put(ReplicatingChannelSelector.CONFIG_OPTIONAL, "ch1"); - Configurables.configure(selector, context); - - ChannelProcessor processor = new ChannelProcessor(selector); - Configurables.configure(processor, context); - - // The idea is to put more events into the optional channel than its capacity + the size of - // the task queue. So the remaining events get added to the task queue, but since it is - // bounded, its size should not grow indefinitely either. - for (int i = 0; i <= 6; i++) { - processor.processEvent(EventBuilder.withBody("e".getBytes())); - // To avoid tasks from being rejected so if previous events are still not committed, wait - // between transactions. - Thread.sleep(500); - } - // 3 in channel, 1 executing, 2 in queue, 1 rejected - Assert.assertEquals(2, processor.taskQueue.size()); - } } From 5293eba9a418180b42b3138c0c0b5aac38361f7f Mon Sep 17 00:00:00 2001 From: Jarek Jarcec Cecho Date: Tue, 29 Mar 2016 08:10:31 -0700 Subject: [PATCH 275/341] FLUME-2897: AsyncHBase sink NPE when Channel.getTransaction() fails (Mike Percy via Jarek Jarcec Cecho) --- .../java/org/apache/flume/sink/hbase/AsyncHBaseSink.java | 5 +++-- 1 file changed, 3 insertions(+), 2 deletions(-) diff --git a/flume-ng-sinks/flume-ng-hbase-sink/src/main/java/org/apache/flume/sink/hbase/AsyncHBaseSink.java b/flume-ng-sinks/flume-ng-hbase-sink/src/main/java/org/apache/flume/sink/hbase/AsyncHBaseSink.java index eac00f67cd..c1ff0c41eb 100644 --- a/flume-ng-sinks/flume-ng-hbase-sink/src/main/java/org/apache/flume/sink/hbase/AsyncHBaseSink.java +++ b/flume-ng-sinks/flume-ng-hbase-sink/src/main/java/org/apache/flume/sink/hbase/AsyncHBaseSink.java @@ -204,10 +204,11 @@ public Status process() throws EventDeliveryException { Status status = Status.READY; Channel channel = getChannel(); + txn = channel.getTransaction(); + txn.begin(); + int i = 0; try { - txn = channel.getTransaction(); - txn.begin(); for (; i < batchSize; i++) { Event event = channel.take(); if (event == null) { From f8abaf78fb98e91b7a228aaa231f4164d8dcfc97 Mon Sep 17 00:00:00 2001 From: Jarek Jarcec Cecho Date: Tue, 29 Mar 2016 09:42:24 -0700 Subject: [PATCH 276/341] FLUME-2821: Flume-Kafka Source with new Consumer (Grigoriy Rozhkov via Jarek Jarcec Cecho) --- flume-ng-doc/sphinx/FlumeUserGuide.rst | 58 ++- flume-ng-sources/flume-kafka-source/pom.xml | 5 + .../flume/source/kafka/KafkaSource.java | 398 ++++++++++++------ .../source/kafka/KafkaSourceConstants.java | 36 +- .../flume/source/kafka/KafkaSourceUtil.java | 112 ----- .../kafka/KafkaSourceEmbeddedKafka.java | 96 +++-- .../kafka/KafkaSourceEmbeddedZookeeper.java | 17 +- .../flume/source/kafka/TestKafkaSource.java | 281 ++++++++++--- .../source/kafka/TestKafkaSourceUtil.java | 92 ---- pom.xml | 5 +- 10 files changed, 649 insertions(+), 451 deletions(-) delete mode 100644 flume-ng-sources/flume-kafka-source/src/main/java/org/apache/flume/source/kafka/KafkaSourceUtil.java delete mode 100644 flume-ng-sources/flume-kafka-source/src/test/java/org/apache/flume/source/kafka/TestKafkaSourceUtil.java diff --git a/flume-ng-doc/sphinx/FlumeUserGuide.rst b/flume-ng-doc/sphinx/FlumeUserGuide.rst index 423e0cf8a3..341ae4229c 100644 --- a/flume-ng-doc/sphinx/FlumeUserGuide.rst +++ b/flume-ng-doc/sphinx/FlumeUserGuide.rst @@ -1188,9 +1188,9 @@ Example for agent named a1: Kafka Source ~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~ -Kafka Source is an Apache Kafka consumer that reads messages from a Kafka topic. +Kafka Source is an Apache Kafka consumer that reads messages from Kafka topics. If you have multiple Kafka sources running, you can configure them with the same Consumer Group -so each will read a unique set of partitions for the topic. +so each will read a unique set of partitions for the topics. @@ -1198,11 +1198,13 @@ so each will read a unique set of partitions for the topic. Property Name Default Description =============================== =========== =================================================== **channels** -- -**type** -- The component type name, needs to be ``org.apache.flume.source.kafka,KafkaSource`` -**zookeeperConnect** -- URI of ZooKeeper used by Kafka cluster -**groupId** flume Unique identified of consumer group. Setting the same id in multiple sources or agents +**type** -- The component type name, needs to be ``org.apache.flume.source.kafka.KafkaSource`` +**kafka.bootstrap.servers** -- List of brokers in the Kafka cluster used by the source +kafka.consumer.group.id flume Unique identified of consumer group. Setting the same id in multiple sources or agents indicates that they are part of the same consumer group -**topic** -- Kafka topic we'll read messages from. At the time, this is a single topic only. +**kafka.topics** -- Comma-separated list of topics the kafka consumer will read messages from. +**kafka.topics.regex** -- Regex that defines set of topics the source is subscribed on. This property has higher priority + than ``kafka.topics`` and overrides ``kafka.topics`` if exists. batchSize 1000 Maximum number of messages written to Channel in one batch batchDurationMillis 1000 Maximum time (in ms) before a batch will be written to Channel The batch will be written whenever the first of size and time will be reached. @@ -1214,31 +1216,49 @@ maxBackoffSleep 5000 Maximum wait time that is triggere ideal for ingestion use cases but a lower value may be required for low latency operations with interceptors. Other Kafka Consumer Properties -- These properties are used to configure the Kafka Consumer. Any producer property supported - by Kafka can be used. The only requirement is to prepend the property name with the prefix ``kafka.``. - For example: kafka.consumer.timeout.ms - Check `Kafka documentation ` for details + by Kafka can be used. The only requirement is to prepend the property name with the prefix ``kafka.consumer``. + For example: kafka.consumer.auto.offset.reset + Check `Kafka documentation ` for details =============================== =========== =================================================== .. note:: The Kafka Source overrides two Kafka consumer parameters: - auto.commit.enable is set to "false" by the source and we commit every batch. For improved performance - this can be set to "true", however, this can lead to loss of data - consumer.timeout.ms is set to 10ms, so when we check Kafka for new data we wait at most 10ms for the data to arrive - setting this to a higher value can reduce CPU utilization (we'll poll Kafka in less of a tight loop), but also means - higher latency in writing batches to channel (since we'll wait longer for data to arrive). + auto.commit.enable is set to "false" by the source and every batch is committed. Kafka source guarantees at least once + strategy of messages retrieval. The duplicates can be present when the source starts. + The Kafka Source also provides defaults for the key.deserializer(org.apache.kafka.common.serialization.StringSerializer) + and value.deserializer(org.apache.kafka.common.serialization.ByteArraySerializer). Modification of these parameters is not recommended. +Deprecated Properties -Example for agent named tier1: +=============================== =================== ============================================================================================= +Property Name Default Description +=============================== =================== ============================================================================================= +topic -- Use kafka.topics +groupId flume Use kafka.consumer.group.id +zookeeperConnect -- Is no longer supported by kafka consumer client since 0.9.x. Use kafka.bootstrap.servers + to establish connection with kafka cluster +=============================== =================== ============================================================================================= + +Example for topic subscription by comma-separated topic list. .. code-block:: properties tier1.sources.source1.type = org.apache.flume.source.kafka.KafkaSource tier1.sources.source1.channels = channel1 - tier1.sources.source1.zookeeperConnect = localhost:2181 - tier1.sources.source1.topic = test1 - tier1.sources.source1.groupId = flume - tier1.sources.source1.kafka.consumer.timeout.ms = 100 + tier1.sources.source1.batchSize = 5000 + tier1.sources.source1.batchDurationMillis = 2000 + tier1.sources.source1.kafka.bootstrap.servers = localhost:9092 + tier1.sources.source1.kafka.topics = test1, test2 + tier1.sources.source1.kafka.consumer.group.id = custom.g.id +Example for topic subscription by regex +.. code-block:: properties + + tier1.sources.source1.type = org.apache.flume.source.kafka.KafkaSource + tier1.sources.source1.channels = channel1 + tier1.sources.source1.kafka.bootstrap.servers = localhost:9092 + tier1.sources.source1.kafka.topics.regex = ^topic[0-9]$ + # the default kafka.consumer.group.id=flume is used NetCat Source diff --git a/flume-ng-sources/flume-kafka-source/pom.xml b/flume-ng-sources/flume-kafka-source/pom.xml index 0f93476c61..5f5c2a8479 100644 --- a/flume-ng-sources/flume-kafka-source/pom.xml +++ b/flume-ng-sources/flume-kafka-source/pom.xml @@ -60,6 +60,11 @@ org.mockito mockito-all
    + + org.apache.kafka + kafka-clients + ${kafka.version} + org.apache.kafka kafka_2.10 diff --git a/flume-ng-sources/flume-kafka-source/src/main/java/org/apache/flume/source/kafka/KafkaSource.java b/flume-ng-sources/flume-kafka-source/src/main/java/org/apache/flume/source/kafka/KafkaSource.java index fd1dd3c17b..db806ccfd9 100644 --- a/flume-ng-sources/flume-kafka-source/src/main/java/org/apache/flume/source/kafka/KafkaSource.java +++ b/flume-ng-sources/flume-kafka-source/src/main/java/org/apache/flume/source/kafka/KafkaSource.java @@ -17,40 +17,49 @@ package org.apache.flume.source.kafka; import java.util.ArrayList; +import java.util.Arrays; +import java.util.Collection; import java.util.HashMap; +import java.util.Iterator; import java.util.List; import java.util.Map; import java.util.Properties; +import java.util.UUID; +import java.util.concurrent.atomic.AtomicBoolean; +import java.util.regex.Pattern; -import kafka.consumer.ConsumerIterator; -import kafka.consumer.ConsumerTimeoutException; -import kafka.consumer.KafkaStream; -import kafka.javaapi.consumer.ConsumerConnector; -import kafka.message.MessageAndMetadata; - -import org.apache.flume.*; +import org.apache.flume.Context; +import org.apache.flume.Event; +import org.apache.flume.EventDeliveryException; +import org.apache.flume.FlumeException; import org.apache.flume.conf.Configurable; import org.apache.flume.conf.ConfigurationException; import org.apache.flume.event.EventBuilder; -import org.apache.flume.instrumentation.SourceCounter; import org.apache.flume.instrumentation.kafka.KafkaSourceCounter; import org.apache.flume.source.AbstractPollableSource; -import org.apache.flume.source.AbstractSource; -import org.apache.flume.source.BasicSourceSemantics; +import org.apache.kafka.clients.consumer.ConsumerConfig; +import org.apache.kafka.clients.consumer.ConsumerRebalanceListener; +import org.apache.kafka.clients.consumer.ConsumerRecord; +import org.apache.kafka.clients.consumer.ConsumerRecords; +import org.apache.kafka.clients.consumer.KafkaConsumer; +import org.apache.kafka.clients.consumer.OffsetAndMetadata; +import org.apache.kafka.common.TopicPartition; import org.slf4j.Logger; import org.slf4j.LoggerFactory; - /** - * A Source for Kafka which reads messages from a kafka topic. + * A Source for Kafka which reads messages from kafka topics. * - * zookeeperConnect: Kafka's zookeeper connection string. - * Required + * kafka.bootstrap.servers: A comma separated list of host:port pairs + * to use for establishing the initial connection to the Kafka cluster. + * For example host1:port1,host2:port2,... + * Required for kafka. *

    - * groupId: the group ID of consumer group. Required + * kafka.consumer.group.id: the group ID of consumer group. Required *

    - * topic: the topic to consume messages from. Required + * kafka.topics: the topic list separated by commas to consume messages from. + * Required *

    * maxBatchSize: Maximum number of messages written to Channel in one * batch. Default: 1000 @@ -58,99 +67,167 @@ * maxBatchDurationMillis: Maximum number of milliseconds before a * batch (of any size) will be written to a channel. Default: 1000 *

    - * kafka.auto.commit.enable: If true, commit automatically every time - * period. if false, commit on each batch. Default: false + * kafka.consumer.*: Any property starting with "kafka.consumer" will be + * passed to the kafka consumer So you can use any configuration supported by Kafka 0.9.0.X *

    - * kafka.consumer.timeout.ms: Polling interval for new data for batch. - * Low value means more CPU usage. High value means the time.upper.limit may be - * missed. Default: 10 - * - * Any property starting with "kafka" will be passed to the kafka consumer So - * you can use any configuration supported by Kafka 0.8.1.1 */ public class KafkaSource extends AbstractPollableSource implements Configurable { private static final Logger log = LoggerFactory.getLogger(KafkaSource.class); - private ConsumerConnector consumer; - private ConsumerIterator it; - private String topic; - private int batchUpperLimit; - private int timeUpperLimit; - private int consumerTimeout; - private boolean kafkaAutoCommitEnabled; + private Context context; private Properties kafkaProps; - private final List eventList = new ArrayList(); private KafkaSourceCounter counter; + private KafkaConsumer consumer; + private Iterator> it; + + private final List eventList = new ArrayList(); + private Map tpAndOffsetMetadata; + private AtomicBoolean rebalanceFlag; + + private Map headers; + + private int batchUpperLimit; + private int maxBatchDurationMillis; + + private Subscriber subscriber; + + + /** + * This class is a helper to subscribe for topics by using + * different strategies + */ + public abstract class Subscriber { + public abstract void subscribe(KafkaConsumer consumer, SourceRebalanceListener listener); + public T get() {return null;} + } + + private class TopicListSubscriber extends Subscriber> { + private List topicList; + public TopicListSubscriber(String commaSeparatedTopics) { + this.topicList = Arrays.asList(commaSeparatedTopics.split("^\\s+|\\s*,\\s*|\\s+$")); + } + @Override + public void subscribe(KafkaConsumer consumer, SourceRebalanceListener listener) { + consumer.subscribe(topicList, listener); + } + @Override + public List get() { + return topicList; + } + } + + private class PatternSubscriber extends Subscriber { + private Pattern pattern; + public PatternSubscriber(String regex) { + this.pattern = Pattern.compile(regex); + } + @Override + public void subscribe(KafkaConsumer consumer, SourceRebalanceListener listener) { + consumer.subscribe(pattern, listener); + } + @Override + public Pattern get() { + return pattern; + } + } + @Override protected Status doProcess() throws EventDeliveryException { + final String batchUUID = UUID.randomUUID().toString(); byte[] kafkaMessage; - byte[] kafkaKey; + String kafkaKey; Event event; - Map headers; - long batchStartTime = System.currentTimeMillis(); - long batchEndTime = System.currentTimeMillis() + timeUpperLimit; + try { - boolean iterStatus = false; - long startTime = System.nanoTime(); + // prepare time variables for new batch + final long nanoBatchStartTime = System.nanoTime(); + final long batchStartTime = System.currentTimeMillis(); + final long maxBatchEndTime = System.currentTimeMillis() + maxBatchDurationMillis; + while (eventList.size() < batchUpperLimit && - System.currentTimeMillis() < batchEndTime) { - iterStatus = hasNext(); - if (iterStatus) { - // get next message - MessageAndMetadata messageAndMetadata = it.next(); - kafkaMessage = messageAndMetadata.message(); - kafkaKey = messageAndMetadata.key(); - - // Add headers to event (topic, timestamp, and key) - headers = new HashMap(); - headers.put(KafkaSourceConstants.TIMESTAMP, - String.valueOf(System.currentTimeMillis())); - headers.put(KafkaSourceConstants.TOPIC, topic); - if (kafkaKey != null) { - headers.put(KafkaSourceConstants.KEY, new String(kafkaKey)); + System.currentTimeMillis() < maxBatchEndTime) { + + if (it == null || !it.hasNext()) { + // Obtaining new records + // Poll time is remainder time for current batch. + ConsumerRecords records = consumer.poll( + Math.max(0, maxBatchEndTime - System.currentTimeMillis())); + it = records.iterator(); + + // this flag is set to true in a callback when some partitions are revoked. + // If there are any records we commit them. + if (rebalanceFlag.get()) { + rebalanceFlag.set(false); + break; } - if (log.isDebugEnabled()) { - log.debug("Message: {}", new String(kafkaMessage)); + // check records after poll + if (!it.hasNext()) { + if (log.isDebugEnabled()) { + counter.incrementKafkaEmptyCount(); + log.debug("Returning with backoff. No more data to read"); + } + // batch time exceeded + break; } - event = EventBuilder.withBody(kafkaMessage, headers); - eventList.add(event); } + + // get next message + ConsumerRecord message = it.next(); + kafkaKey = message.key(); + kafkaMessage = message.value(); + + headers.clear(); + // Add headers to event (timestamp, topic, partition, key) + headers.put(KafkaSourceConstants.TIMESTAMP_HEADER, String.valueOf(System.currentTimeMillis())); + headers.put(KafkaSourceConstants.TOPIC_HEADER, message.topic()); + headers.put(KafkaSourceConstants.PARTITION_HEADER, String.valueOf(message.partition())); + if (kafkaKey != null) { + headers.put(KafkaSourceConstants.KEY_HEADER, kafkaKey); + } + + if (log.isDebugEnabled()) { + log.debug("Topic: {} Partition: {} Message: {}", new String[]{ + message.topic(), + String.valueOf(message.partition()), + new String(kafkaMessage)}); + } + + event = EventBuilder.withBody(kafkaMessage, headers); + eventList.add(event); + if (log.isDebugEnabled()) { log.debug("Waited: {} ", System.currentTimeMillis() - batchStartTime); log.debug("Event #: {}", eventList.size()); } + + // For each partition store next offset that is going to be read. + tpAndOffsetMetadata.put(new TopicPartition(message.topic(), message.partition()), + new OffsetAndMetadata(message.offset() + 1, batchUUID)); } - long endTime = System.nanoTime(); - counter.addToKafkaEventGetTimer((endTime-startTime)/(1000*1000)); - counter.addToEventReceivedCount(Long.valueOf(eventList.size())); - // If we have events, send events to channel - // clear the event list - // and commit if Kafka doesn't auto-commit + if (eventList.size() > 0) { + counter.addToKafkaEventGetTimer((System.nanoTime() - nanoBatchStartTime) / (1000 * 1000)); + counter.addToEventReceivedCount((long) eventList.size()); getChannelProcessor().processEventBatch(eventList); counter.addToEventAcceptedCount(eventList.size()); - eventList.clear(); if (log.isDebugEnabled()) { log.debug("Wrote {} events to channel", eventList.size()); } - if (!kafkaAutoCommitEnabled) { - // commit the read transactions to Kafka to avoid duplicates + eventList.clear(); + + if (!tpAndOffsetMetadata.isEmpty()) { long commitStartTime = System.nanoTime(); - consumer.commitOffsets(); + consumer.commitSync(tpAndOffsetMetadata); long commitEndTime = System.nanoTime(); - counter.addToKafkaCommitTimer((commitEndTime-commitStartTime)/(1000*1000)); - } - } - if (!iterStatus) { - if (log.isDebugEnabled()) { - counter.incrementKafkaEmptyCount(); - log.debug("Returning with backoff. No more data to read"); + counter.addToKafkaCommitTimer((commitEndTime - commitStartTime) / (1000 * 1000)); + tpAndOffsetMetadata.clear(); } - return Status.BACKOFF; + return Status.READY; } - return Status.READY; + + return Status.BACKOFF; } catch (Exception e) { log.error("KafkaSource EXCEPTION, {}", e); return Status.BACKOFF; @@ -161,96 +238,153 @@ protected Status doProcess() throws EventDeliveryException { * We configure the source and generate properties for the Kafka Consumer * * Kafka Consumer properties are generated as follows: - * * 1. Generate a properties object with some static defaults that can be - * overridden by Source configuration 2. We add the configuration users added - * for Kafka (parameters starting with kafka. and must be valid Kafka Consumer - * properties 3. We add the source documented parameters which can override - * other properties - * + * overridden if corresponding properties are specified + * 2. We add the configuration users added for Kafka (parameters starting + * with kafka.consumer and must be valid Kafka Consumer properties + * 3. Add source level properties (with no prefix) * @param context */ @Override protected void doConfigure(Context context) throws FlumeException { this.context = context; + headers = new HashMap(4); + tpAndOffsetMetadata = new HashMap(); + rebalanceFlag = new AtomicBoolean(false); + kafkaProps = new Properties(); + + // can be removed in the next release + // See https://issues.apache.org/jira/browse/FLUME-2896 + translateOldProperties(context); + + String topicProperty = context.getString(KafkaSourceConstants.TOPICS_REGEX); + if (topicProperty != null && !topicProperty.isEmpty()) { + // create subscriber that uses pattern-based subscription + subscriber = new PatternSubscriber(topicProperty); + } else + if((topicProperty = context.getString(KafkaSourceConstants.TOPICS)) != null && !topicProperty.isEmpty()) { + // create subscriber that uses topic list subscription + subscriber = new TopicListSubscriber(topicProperty); + } else + if (subscriber == null) { + throw new ConfigurationException("At least one Kafka topic must be specified."); + } + batchUpperLimit = context.getInteger(KafkaSourceConstants.BATCH_SIZE, KafkaSourceConstants.DEFAULT_BATCH_SIZE); - timeUpperLimit = context.getInteger(KafkaSourceConstants.BATCH_DURATION_MS, + maxBatchDurationMillis = context.getInteger(KafkaSourceConstants.BATCH_DURATION_MS, KafkaSourceConstants.DEFAULT_BATCH_DURATION); - topic = context.getString(KafkaSourceConstants.TOPIC); - if(topic == null) { - throw new ConfigurationException("Kafka topic must be specified."); + String bootstrapServers = context.getString(KafkaSourceConstants.BOOTSTRAP_SERVERS); + if (bootstrapServers == null || bootstrapServers.isEmpty()) { + throw new ConfigurationException("Bootstrap Servers must be specified"); } - kafkaProps = KafkaSourceUtil.getKafkaProperties(context); - consumerTimeout = Integer.parseInt(kafkaProps.getProperty( - KafkaSourceConstants.CONSUMER_TIMEOUT)); - kafkaAutoCommitEnabled = Boolean.parseBoolean(kafkaProps.getProperty( - KafkaSourceConstants.AUTO_COMMIT_ENABLED)); + setConsumerProps(context, bootstrapServers); if (counter == null) { counter = new KafkaSourceCounter(getName()); } } + + // We can remove this once the properties are officially deprecated + private void translateOldProperties(Context ctx) { + // topic + String topic = context.getString(KafkaSourceConstants.TOPIC); + if (topic != null && !topic.isEmpty()) { + subscriber = new TopicListSubscriber(topic); + log.warn("{} is deprecated. Please use the parameter {}", + KafkaSourceConstants.TOPIC, KafkaSourceConstants.TOPICS); + } + + // old groupId + String groupId = ctx.getString(KafkaSourceConstants.OLD_GROUP_ID); + if (groupId != null && !groupId.isEmpty()) { + kafkaProps.put(ConsumerConfig.GROUP_ID_CONFIG, groupId); + log.warn("{} is deprecated. Please use the parameter {}", + KafkaSourceConstants.OLD_GROUP_ID, + KafkaSourceConstants.KAFKA_CONSUMER_PREFIX + ConsumerConfig.GROUP_ID_CONFIG); + } + } + + + private void setConsumerProps(Context ctx, String bootStrapServers) { + String groupId = ctx.getString(KafkaSourceConstants.KAFKA_CONSUMER_PREFIX + ConsumerConfig.GROUP_ID_CONFIG); + if ((groupId == null || groupId.isEmpty()) && + kafkaProps.getProperty(ConsumerConfig.GROUP_ID_CONFIG) == null) { + groupId = KafkaSourceConstants.DEFAULT_GROUP_ID; + log.info("Group ID was not specified. Using " + groupId + " as the group id."); + } + kafkaProps.put(ConsumerConfig.KEY_DESERIALIZER_CLASS_CONFIG, KafkaSourceConstants.DEFAULT_KEY_DESERIALIZER); + kafkaProps.put(ConsumerConfig.VALUE_DESERIALIZER_CLASS_CONFIG, KafkaSourceConstants.DEFAULT_VALUE_DESERIALIZER); + //Defaults overridden based on config + kafkaProps.putAll(ctx.getSubProperties(KafkaSourceConstants.KAFKA_CONSUMER_PREFIX)); + //These always take precedence over config + kafkaProps.put(ConsumerConfig.BOOTSTRAP_SERVERS_CONFIG, bootStrapServers); + if (groupId != null) { + kafkaProps.put(ConsumerConfig.GROUP_ID_CONFIG, groupId); + } + kafkaProps.put(ConsumerConfig.ENABLE_AUTO_COMMIT_CONFIG, KafkaSourceConstants.DEFAULT_AUTO_COMMIT); + + log.info(kafkaProps.toString()); + } + + Properties getConsumerProps() { + return kafkaProps; + } + + Subscriber getSubscriber() { + return subscriber; + } + @Override protected void doStart() throws FlumeException { log.info("Starting {}...", this); - try { - //initialize a consumer. This creates the connection to ZooKeeper - consumer = KafkaSourceUtil.getConsumer(kafkaProps); - } catch (Exception e) { - throw new FlumeException("Unable to create consumer. " + - "Check whether the ZooKeeper server is up and that the " + - "Flume agent can connect to it.", e); - } + //initialize a consumer. + consumer = new KafkaConsumer(kafkaProps); - Map topicCountMap = new HashMap(); - // We always have just one topic being read by one thread - topicCountMap.put(topic, 1); + // Subscribe for topics by already specified strategy + subscriber.subscribe(consumer, new SourceRebalanceListener(rebalanceFlag)); - // Get the message iterator for our topic - // Note that this succeeds even if the topic doesn't exist - // in that case we simply get no messages for the topic - // Also note that currently we only support a single topic - try { - Map>> consumerMap = - consumer.createMessageStreams(topicCountMap); - List> topicList = consumerMap.get(topic); - KafkaStream stream = topicList.get(0); - it = stream.iterator(); - } catch (Exception e) { - throw new FlumeException("Unable to get message iterator from Kafka", e); - } - log.info("Kafka source {} do started.", getName()); + // Connect to kafka. 1 second is optimal time. + it = consumer.poll(1000).iterator(); + log.info("Kafka source {} started.", getName()); counter.start(); } @Override protected void doStop() throws FlumeException { if (consumer != null) { - // exit cleanly. This syncs offsets of messages read to ZooKeeper - // to avoid reading the same messages again - consumer.shutdown(); + consumer.wakeup(); + consumer.close(); } counter.stop(); - log.info("Kafka Source {} do stopped. Metrics: {}", getName(), counter); + log.info("Kafka Source {} stopped. Metrics: {}", getName(), counter); } +} - /** - * Check if there are messages waiting in Kafka, - * waiting until timeout (10ms by default) for messages to arrive. - * and catching the timeout exception to return a boolean - */ - boolean hasNext() { - try { - it.hasNext(); - return true; - } catch (ConsumerTimeoutException e) { - return false; + +class SourceRebalanceListener implements ConsumerRebalanceListener { + private static final Logger log = LoggerFactory.getLogger(SourceRebalanceListener.class); + private AtomicBoolean rebalanceFlag; + + public SourceRebalanceListener(AtomicBoolean rebalanceFlag) { + this.rebalanceFlag = rebalanceFlag; + } + + // Set a flag that a rebalance has occurred. Then commit already read events to kafka. + public void onPartitionsRevoked(Collection partitions) { + for (TopicPartition partition : partitions) { + log.info("topic {} - partition {} revoked.", partition.topic(), partition.partition()); + rebalanceFlag.set(true); } } -} \ No newline at end of file + public void onPartitionsAssigned(Collection partitions) { + for (TopicPartition partition : partitions) { + log.info("topic {} - partition {} assigned.", partition.topic(), partition.partition()); + } + } +} diff --git a/flume-ng-sources/flume-kafka-source/src/main/java/org/apache/flume/source/kafka/KafkaSourceConstants.java b/flume-ng-sources/flume-kafka-source/src/main/java/org/apache/flume/source/kafka/KafkaSourceConstants.java index 911012cefc..2999cf272b 100644 --- a/flume-ng-sources/flume-kafka-source/src/main/java/org/apache/flume/source/kafka/KafkaSourceConstants.java +++ b/flume-ng-sources/flume-kafka-source/src/main/java/org/apache/flume/source/kafka/KafkaSourceConstants.java @@ -16,25 +16,33 @@ */ package org.apache.flume.source.kafka; +import org.apache.kafka.clients.CommonClientConfigs; + public class KafkaSourceConstants { - public static final String TOPIC = "topic"; - public static final String KEY = "key"; - public static final String TIMESTAMP = "timestamp"; + + public static final String KAFKA_PREFIX = "kafka."; + public static final String KAFKA_CONSUMER_PREFIX = KAFKA_PREFIX + "consumer."; + public static final String DEFAULT_KEY_DESERIALIZER = "org.apache.kafka.common.serialization.StringDeserializer"; + public static final String DEFAULT_VALUE_DESERIALIZER = "org.apache.kafka.common.serialization.ByteArrayDeserializer"; + public static final String BOOTSTRAP_SERVERS = KAFKA_PREFIX + CommonClientConfigs.BOOTSTRAP_SERVERS_CONFIG; + public static final String TOPICS = KAFKA_PREFIX + "topics"; + public static final String TOPICS_REGEX = TOPICS + "." + "regex"; + public static final String DEFAULT_AUTO_COMMIT = "false"; public static final String BATCH_SIZE = "batchSize"; public static final String BATCH_DURATION_MS = "batchDurationMillis"; - public static final String CONSUMER_TIMEOUT = "consumer.timeout.ms"; - public static final String AUTO_COMMIT_ENABLED = "auto.commit.enable"; - public static final String ZOOKEEPER_CONNECT = "zookeeper.connect"; - public static final String ZOOKEEPER_CONNECT_FLUME = "zookeeperConnect"; - public static final String GROUP_ID = "group.id"; - public static final String GROUP_ID_FLUME = "groupId"; - public static final String PROPERTY_PREFIX = "kafka."; - - public static final int DEFAULT_BATCH_SIZE = 1000; public static final int DEFAULT_BATCH_DURATION = 1000; - public static final String DEFAULT_CONSUMER_TIMEOUT = "10"; - public static final String DEFAULT_AUTO_COMMIT = "false"; public static final String DEFAULT_GROUP_ID = "flume"; + /* Old Properties */ + + public static final String TOPIC = "topic"; + public static final String OLD_GROUP_ID = "groupId"; + + // flume event headers + public static final String TOPIC_HEADER = "topic"; + public static final String KEY_HEADER = "key"; + public static final String TIMESTAMP_HEADER = "timestamp"; + public static final String PARTITION_HEADER = "partition"; + } diff --git a/flume-ng-sources/flume-kafka-source/src/main/java/org/apache/flume/source/kafka/KafkaSourceUtil.java b/flume-ng-sources/flume-kafka-source/src/main/java/org/apache/flume/source/kafka/KafkaSourceUtil.java deleted file mode 100644 index 4a4034bd82..0000000000 --- a/flume-ng-sources/flume-kafka-source/src/main/java/org/apache/flume/source/kafka/KafkaSourceUtil.java +++ /dev/null @@ -1,112 +0,0 @@ -/* - * Licensed to the Apache Software Foundation (ASF) under one or more - * contributor license agreements. See the NOTICE file distributed with - * this work for additional information regarding copyright ownership. - * The ASF licenses this file to You under the Apache License, Version 2.0 - * (the "License"); you may not use this file except in compliance with - * the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. - */ -package org.apache.flume.source.kafka; - -import java.util.Map; -import java.util.Properties; - -import kafka.common.KafkaException; -import kafka.consumer.Consumer; -import kafka.consumer.ConsumerConfig; -import kafka.javaapi.consumer.ConsumerConnector; - -import org.apache.flume.Context; -import org.apache.flume.conf.ConfigurationException; -import org.slf4j.Logger; -import org.slf4j.LoggerFactory; - -public class KafkaSourceUtil { - private static final Logger log = - LoggerFactory.getLogger(KafkaSourceUtil.class); - - public static Properties getKafkaProperties(Context context) { - log.info("context={}",context.toString()); - Properties props = generateDefaultKafkaProps(); - setKafkaProps(context,props); - addDocumentedKafkaProps(context,props); - return props; - } - - public static ConsumerConnector getConsumer(Properties kafkaProps) { - ConsumerConfig consumerConfig = - new ConsumerConfig(kafkaProps); - ConsumerConnector consumer = - Consumer.createJavaConsumerConnector(consumerConfig); - return consumer; - } - - /** - * Generate consumer properties object with some defaults - * @return - */ - private static Properties generateDefaultKafkaProps() { - Properties props = new Properties(); - props.put(KafkaSourceConstants.AUTO_COMMIT_ENABLED, - KafkaSourceConstants.DEFAULT_AUTO_COMMIT); - props.put(KafkaSourceConstants.CONSUMER_TIMEOUT, - KafkaSourceConstants.DEFAULT_CONSUMER_TIMEOUT); - props.put(KafkaSourceConstants.GROUP_ID, - KafkaSourceConstants.DEFAULT_GROUP_ID); - return props; - } - - /** - * Add all configuration parameters starting with "kafka" - * to consumer properties - */ - private static void setKafkaProps(Context context,Properties kafkaProps) { - - Map kafkaProperties = - context.getSubProperties(KafkaSourceConstants.PROPERTY_PREFIX); - - for (Map.Entry prop : kafkaProperties.entrySet()) { - - kafkaProps.put(prop.getKey(), prop.getValue()); - if (log.isDebugEnabled()) { - log.debug("Reading a Kafka Producer Property: key: " - + prop.getKey() + ", value: " + prop.getValue()); - } - } - } - - /** - * Some of the producer properties are especially important - * We documented them and gave them a camel-case name to match Flume config - * If user set these, we will override any existing parameters with these - * settings. - * Knowledge of which properties are documented is maintained here for now. - * If this will become a maintenance issue we'll set a proper data structure. - */ - private static void addDocumentedKafkaProps(Context context, - Properties kafkaProps) - throws ConfigurationException { - String zookeeperConnect = context.getString( - KafkaSourceConstants.ZOOKEEPER_CONNECT_FLUME); - if (zookeeperConnect == null) { - throw new ConfigurationException("ZookeeperConnect must contain " + - "at least one ZooKeeper server"); - } - kafkaProps.put(KafkaSourceConstants.ZOOKEEPER_CONNECT, zookeeperConnect); - - String groupID = context.getString(KafkaSourceConstants.GROUP_ID_FLUME); - - if (groupID != null ) { - kafkaProps.put(KafkaSourceConstants.GROUP_ID, groupID); - } - } - -} \ No newline at end of file diff --git a/flume-ng-sources/flume-kafka-source/src/test/java/org/apache/flume/source/kafka/KafkaSourceEmbeddedKafka.java b/flume-ng-sources/flume-kafka-source/src/test/java/org/apache/flume/source/kafka/KafkaSourceEmbeddedKafka.java index 26c5c9d0aa..46d545f90a 100644 --- a/flume-ng-sources/flume-kafka-source/src/test/java/org/apache/flume/source/kafka/KafkaSourceEmbeddedKafka.java +++ b/flume-ng-sources/flume-kafka-source/src/test/java/org/apache/flume/source/kafka/KafkaSourceEmbeddedKafka.java @@ -18,27 +18,59 @@ import kafka.server.KafkaConfig; import kafka.server.KafkaServerStartable; -import kafka.javaapi.producer.Producer; -import kafka.producer.KeyedMessage; -import kafka.producer.ProducerConfig; import kafka.admin.AdminUtils; +import kafka.utils.ZkUtils; import org.I0Itec.zkclient.ZkClient; -import kafka.utils.ZKStringSerializer$; +import org.apache.commons.io.FileUtils; +import org.apache.kafka.clients.producer.KafkaProducer; +import org.apache.kafka.clients.producer.ProducerRecord; +import org.apache.kafka.common.serialization.StringSerializer; +import java.io.File; import java.io.IOException; +import java.net.InetAddress; +import java.net.UnknownHostException; import java.util.Properties; +import java.util.UUID; +import java.util.concurrent.ExecutionException; public class KafkaSourceEmbeddedKafka { + + public static String HOST; + + static { + try { + HOST = InetAddress.getLocalHost().getHostAddress(); + } catch (UnknownHostException e) { + throw new RuntimeException("Host address can not be obtained", e); + } + } + KafkaServerStartable kafkaServer; KafkaSourceEmbeddedZookeeper zookeeper; + int zkPort = 21818; // none-standard - Producer producer; + int serverPort = 18922; + + KafkaProducer producer; + File dir; - public KafkaSourceEmbeddedKafka() { + public KafkaSourceEmbeddedKafka(Properties properties) { zookeeper = new KafkaSourceEmbeddedZookeeper(zkPort); + dir = new File(System.getProperty("java.io.tmpdir"), "kafka_log-" + UUID.randomUUID()); + try { + FileUtils.deleteDirectory(dir); + } catch (IOException e) { + e.printStackTrace(); + } Properties props = new Properties(); props.put("zookeeper.connect",zookeeper.getConnectString()); props.put("broker.id","1"); + props.put("host.name", "localhost"); + props.put("port", String.valueOf(serverPort)); + props.put("log.dir", dir.getAbsolutePath()); + if (properties != null) + props.putAll(props); KafkaConfig config = new KafkaConfig(props); kafkaServer = new KafkaServerStartable(config); kafkaServer.startup(); @@ -55,37 +87,49 @@ public String getZkConnectString() { return zookeeper.getConnectString(); } - private void initProducer() - { - Properties props = new Properties(); - props.put("metadata.broker.list","127.0.0.1:" + - kafkaServer.serverConfig().port()); - props.put("serializer.class","kafka.serializer.StringEncoder"); - props.put("request.required.acks", "1"); - - ProducerConfig config = new ProducerConfig(props); - - producer = new Producer(config); + public String getBrockers() { + return HOST + ":" + serverPort; + } + private void initProducer() { + Properties props = new Properties(); + props.put("bootstrap.servers", HOST + ":" + serverPort); + props.put("acks", "1"); + producer = new KafkaProducer(props, + new StringSerializer(), new StringSerializer()); } public void produce(String topic, String k, String v) { - KeyedMessage message = new KeyedMessage(topic,k,v); - producer.send(message); + ProducerRecord rec = new ProducerRecord(topic, k, v); + try { + producer.send(rec).get(); + } catch (InterruptedException e) { + e.printStackTrace(); + } catch (ExecutionException e) { + e.printStackTrace(); + } } - public void createTopic(String topicName) { + public void produce(String topic, int partition, String k, String v) { + ProducerRecord rec = new ProducerRecord(topic, partition, k, v); + try { + producer.send(rec).get(); + } catch (InterruptedException e) { + e.printStackTrace(); + } catch (ExecutionException e) { + e.printStackTrace(); + } + } + + public void createTopic(String topicName, int numPartitions) { // Create a ZooKeeper client int sessionTimeoutMs = 10000; int connectionTimeoutMs = 10000; - ZkClient zkClient = new ZkClient(zookeeper.getConnectString(), - sessionTimeoutMs, connectionTimeoutMs, - ZKStringSerializer$.MODULE$); - - int numPartitions = 1; + ZkClient zkClient = ZkUtils.createZkClient(HOST + ":" + zkPort, sessionTimeoutMs, connectionTimeoutMs); + ZkUtils zkUtils = ZkUtils.apply(zkClient, false); int replicationFactor = 1; Properties topicConfig = new Properties(); - AdminUtils.createTopic(zkClient, topicName, numPartitions, + AdminUtils.createTopic(zkUtils, topicName, numPartitions, replicationFactor, topicConfig); } diff --git a/flume-ng-sources/flume-kafka-source/src/test/java/org/apache/flume/source/kafka/KafkaSourceEmbeddedZookeeper.java b/flume-ng-sources/flume-kafka-source/src/test/java/org/apache/flume/source/kafka/KafkaSourceEmbeddedZookeeper.java index 1b8a27106e..db144c2b8e 100644 --- a/flume-ng-sources/flume-kafka-source/src/test/java/org/apache/flume/source/kafka/KafkaSourceEmbeddedZookeeper.java +++ b/flume-ng-sources/flume-kafka-source/src/test/java/org/apache/flume/source/kafka/KafkaSourceEmbeddedZookeeper.java @@ -23,6 +23,7 @@ import java.io.File; import java.io.IOException; import java.net.InetSocketAddress; +import java.util.UUID; public class KafkaSourceEmbeddedZookeeper { private int zkPort; @@ -31,19 +32,25 @@ public class KafkaSourceEmbeddedZookeeper { File dir; - public KafkaSourceEmbeddedZookeeper(int zkPort){ - int numConnections = 5000; + public KafkaSourceEmbeddedZookeeper(int zkPort) { int tickTime = 2000; this.zkPort = zkPort; String dataDirectory = System.getProperty("java.io.tmpdir"); - dir = new File(dataDirectory, "zookeeper").getAbsoluteFile(); + dir = new File(dataDirectory, "zookeeper" + UUID.randomUUID()).getAbsoluteFile(); + + try { + FileUtils.deleteDirectory(dir); + } catch (IOException e) { + e.printStackTrace(); + System.exit(1); + } try { this.zookeeper = new ZooKeeperServer(dir,dir,tickTime); this.factory = new NIOServerCnxnFactory(); - factory.configure(new InetSocketAddress("127.0.0.1",zkPort),0); + factory.configure(new InetSocketAddress(KafkaSourceEmbeddedKafka.HOST, zkPort),0); factory.startup(zookeeper); } catch (IOException e) { e.printStackTrace(); @@ -59,6 +66,6 @@ public void stopZookeeper() throws IOException { } public String getConnectString() { - return "127.0.0.1:"+zkPort; + return KafkaSourceEmbeddedKafka.HOST + ":" + zkPort; } } diff --git a/flume-ng-sources/flume-kafka-source/src/test/java/org/apache/flume/source/kafka/TestKafkaSource.java b/flume-ng-sources/flume-kafka-source/src/test/java/org/apache/flume/source/kafka/TestKafkaSource.java index 8ec14cccf5..8e04da8387 100644 --- a/flume-ng-sources/flume-kafka-source/src/test/java/org/apache/flume/source/kafka/TestKafkaSource.java +++ b/flume-ng-sources/flume-kafka-source/src/test/java/org/apache/flume/source/kafka/TestKafkaSource.java @@ -23,17 +23,18 @@ import static org.mockito.Mockito.*; import java.util.List; +import java.util.Properties; +import java.util.regex.Pattern; import com.google.common.base.Charsets; import com.google.common.collect.Lists; import junit.framework.Assert; import kafka.common.TopicExistsException; -import kafka.consumer.ConsumerIterator; -import kafka.message.Message; import org.apache.flume.*; import org.apache.flume.PollableSource.Status; import org.apache.flume.channel.ChannelProcessor; +import org.apache.kafka.clients.consumer.ConsumerConfig; import org.junit.After; import org.junit.Before; import org.junit.Test; @@ -42,63 +43,133 @@ import org.slf4j.Logger; import org.slf4j.LoggerFactory; +import static org.apache.flume.source.kafka.KafkaSourceConstants.*; + public class TestKafkaSource { private static final Logger log = LoggerFactory.getLogger(TestKafkaSource.class); private KafkaSource kafkaSource; private KafkaSourceEmbeddedKafka kafkaServer; - private ConsumerIterator mockIt; - private Message message; private Context context; private List events; - private String topicName = "test1"; - + private String topic0 = "test1"; + private String topic1 = "topic1"; @SuppressWarnings("unchecked") @Before public void setup() throws Exception { - kafkaSource = new KafkaSource(); - kafkaServer = new KafkaSourceEmbeddedKafka(); + kafkaServer = new KafkaSourceEmbeddedKafka(null); try { - kafkaServer.createTopic(topicName); + kafkaServer.createTopic(topic0, 1); + kafkaServer.createTopic(topic1, 3); } catch (TopicExistsException e) { //do nothing + e.printStackTrace(); } - - - context = new Context(); - context.put(KafkaSourceConstants.ZOOKEEPER_CONNECT_FLUME, - kafkaServer.getZkConnectString()); - context.put(KafkaSourceConstants.GROUP_ID_FLUME,"flume"); - context.put(KafkaSourceConstants.TOPIC,topicName); - context.put("kafka.consumer.timeout.ms","100"); - + context = prepareDefaultContext(); kafkaSource.setChannelProcessor(createGoodChannel()); } + private Context prepareDefaultContext() { + Context context = new Context(); + context.put(BOOTSTRAP_SERVERS, kafkaServer.getBrockers()); + context.put(KAFKA_CONSUMER_PREFIX + "group.id", "flume-group"); + return context; + } + @After public void tearDown() throws Exception { kafkaSource.stop(); kafkaServer.stop(); } + @SuppressWarnings("unchecked") + @Test + public void testOffsets() throws InterruptedException, EventDeliveryException { + long batchDuration = 2000; + context.put(TOPICS, topic1); + context.put(BATCH_DURATION_MS, + String.valueOf(batchDuration)); + context.put(BATCH_SIZE, "3"); + kafkaSource.configure(context); + kafkaSource.start(); + Thread.sleep(500L); + Status status = kafkaSource.process(); + assertEquals(Status.BACKOFF, status); + assertEquals(0, events.size()); + kafkaServer.produce(topic1, "", "record1"); + kafkaServer.produce(topic1, "", "record2"); + Thread.sleep(500L); + status = kafkaSource.process(); + assertEquals(Status.READY, status); + assertEquals(2, events.size()); + events.clear(); + kafkaServer.produce(topic1, "", "record3"); + kafkaServer.produce(topic1, "", "record4"); + kafkaServer.produce(topic1, "", "record5"); + Thread.sleep(500L); + assertEquals(Status.READY, kafkaSource.process()); + assertEquals(3, events.size()); + assertEquals("record3", new String(events.get(0).getBody(), Charsets.UTF_8)); + assertEquals("record4", new String(events.get(1).getBody(), Charsets.UTF_8)); + assertEquals("record5", new String(events.get(2).getBody(), Charsets.UTF_8)); + events.clear(); + kafkaServer.produce(topic1, "", "record6"); + kafkaServer.produce(topic1, "", "record7"); + kafkaServer.produce(topic1, "", "record8"); + kafkaServer.produce(topic1, "", "record9"); + kafkaServer.produce(topic1, "", "record10"); + Thread.sleep(500L); + assertEquals(Status.READY, kafkaSource.process()); + assertEquals(3, events.size()); + assertEquals("record6", new String(events.get(0).getBody(), Charsets.UTF_8)); + assertEquals("record7", new String(events.get(1).getBody(), Charsets.UTF_8)); + assertEquals("record8", new String(events.get(2).getBody(), Charsets.UTF_8)); + events.clear(); + kafkaServer.produce(topic1, "", "record11"); + // status must be READY due to time out exceed. + assertEquals(Status.READY, kafkaSource.process()); + assertEquals(3, events.size()); + assertEquals("record9", new String(events.get(0).getBody(), Charsets.UTF_8)); + assertEquals("record10", new String(events.get(1).getBody(), Charsets.UTF_8)); + assertEquals("record11", new String(events.get(2).getBody(), Charsets.UTF_8)); + events.clear(); + kafkaServer.produce(topic1, "", "record12"); + kafkaServer.produce(topic1, "", "record13"); + // stop kafka source + kafkaSource.stop(); + // start again + kafkaSource = new KafkaSource(); + kafkaSource.setChannelProcessor(createGoodChannel()); + kafkaSource.configure(context); + kafkaSource.start(); + kafkaServer.produce(topic1, "", "record14"); + Thread.sleep(1000L); + assertEquals(Status.READY, kafkaSource.process()); + assertEquals(3, events.size()); + assertEquals("record12", new String(events.get(0).getBody(), Charsets.UTF_8)); + assertEquals("record13", new String(events.get(1).getBody(), Charsets.UTF_8)); + assertEquals("record14", new String(events.get(2).getBody(), Charsets.UTF_8)); + events.clear(); + } + @SuppressWarnings("unchecked") @Test public void testProcessItNotEmpty() throws EventDeliveryException, SecurityException, NoSuchFieldException, IllegalArgumentException, IllegalAccessException, InterruptedException { - context.put(KafkaSourceConstants.BATCH_SIZE,"1"); + context.put(TOPICS, topic0); + context.put(BATCH_SIZE, "1"); kafkaSource.configure(context); kafkaSource.start(); Thread.sleep(500L); - kafkaServer.produce(topicName, "", "hello, world"); + kafkaServer.produce(topic0, "", "hello, world"); Thread.sleep(500L); - Assert.assertEquals(Status.READY, kafkaSource.process()); Assert.assertEquals(Status.BACKOFF, kafkaSource.process()); Assert.assertEquals(1, events.size()); @@ -112,14 +183,15 @@ public void testProcessItNotEmpty() throws EventDeliveryException, public void testProcessItNotEmptyBatch() throws EventDeliveryException, SecurityException, NoSuchFieldException, IllegalArgumentException, IllegalAccessException, InterruptedException { - context.put(KafkaSourceConstants.BATCH_SIZE,"2"); + context.put(TOPICS, topic0); + context.put(BATCH_SIZE,"2"); kafkaSource.configure(context); kafkaSource.start(); Thread.sleep(500L); - kafkaServer.produce(topicName, "", "hello, world"); - kafkaServer.produce(topicName, "", "foo, bar"); + kafkaServer.produce(topic0, "", "hello, world"); + kafkaServer.produce(topic0, "", "foo, bar"); Thread.sleep(500L); @@ -138,6 +210,7 @@ public void testProcessItNotEmptyBatch() throws EventDeliveryException, public void testProcessItEmpty() throws EventDeliveryException, SecurityException, NoSuchFieldException, IllegalArgumentException, IllegalAccessException, InterruptedException { + context.put(TOPICS, topic0); kafkaSource.configure(context); kafkaSource.start(); Thread.sleep(500L); @@ -151,7 +224,7 @@ public void testProcessItEmpty() throws EventDeliveryException, public void testNonExistingTopic() throws EventDeliveryException, SecurityException, NoSuchFieldException, IllegalArgumentException, IllegalAccessException, InterruptedException { - context.put(KafkaSourceConstants.TOPIC,"faketopic"); + context.put(TOPICS,"faketopic"); kafkaSource.configure(context); kafkaSource.start(); Thread.sleep(500L); @@ -162,10 +235,11 @@ public void testNonExistingTopic() throws EventDeliveryException, @SuppressWarnings("unchecked") @Test(expected= FlumeException.class) - public void testNonExistingZk() throws EventDeliveryException, + public void testNonExistingKafkaServer() throws EventDeliveryException, SecurityException, NoSuchFieldException, IllegalArgumentException, IllegalAccessException, InterruptedException { - context.put(KafkaSourceConstants.ZOOKEEPER_CONNECT_FLUME,"blabla:666"); + context.put(TOPICS, topic0); + context.put(BOOTSTRAP_SERVERS,"blabla:666"); kafkaSource.configure(context); kafkaSource.start(); Thread.sleep(500L); @@ -177,37 +251,39 @@ public void testNonExistingZk() throws EventDeliveryException, @Test public void testBatchTime() throws InterruptedException, EventDeliveryException { - context.put(KafkaSourceConstants.BATCH_DURATION_MS,"250"); + context.put(TOPICS, topic0); + context.put(BATCH_DURATION_MS, "250"); kafkaSource.configure(context); kafkaSource.start(); Thread.sleep(500L); for (int i=1; i<5000; i++) { - kafkaServer.produce(topicName, "", "hello, world " + i); + kafkaServer.produce(topic0, "", "hello, world " + i); } Thread.sleep(500L); + long error = 50; long startTime = System.currentTimeMillis(); Status status = kafkaSource.process(); long endTime = System.currentTimeMillis(); assertEquals(Status.READY, status); assertTrue(endTime - startTime < - ( context.getLong(KafkaSourceConstants.BATCH_DURATION_MS) + - context.getLong("kafka.consumer.timeout.ms")) ); + (context.getLong(BATCH_DURATION_MS) + error)); } // Consume event, stop source, start again and make sure we are not // consuming same event again @Test public void testCommit() throws InterruptedException, EventDeliveryException { - context.put(KafkaSourceConstants.BATCH_SIZE,"1"); + context.put(TOPICS, topic0); + context.put(BATCH_SIZE, "1"); kafkaSource.configure(context); kafkaSource.start(); Thread.sleep(500L); - kafkaServer.produce(topicName, "", "hello, world"); + kafkaServer.produce(topic0, "", "hello, world"); Thread.sleep(500L); @@ -224,14 +300,14 @@ public void testCommit() throws InterruptedException, EventDeliveryException { @Test public void testNonCommit() throws EventDeliveryException, InterruptedException { - - context.put(KafkaSourceConstants.BATCH_SIZE,"1"); - context.put(KafkaSourceConstants.BATCH_DURATION_MS,"30000"); + context.put(TOPICS, topic0); + context.put(BATCH_SIZE,"1"); + context.put(BATCH_DURATION_MS,"30000"); kafkaSource.configure(context); kafkaSource.start(); Thread.sleep(500L); - kafkaServer.produce(topicName, "", "hello, world"); + kafkaServer.produce(topic0, "", "hello, world"); Thread.sleep(500L); kafkaSource.setChannelProcessor(createBadChannel()); @@ -252,13 +328,14 @@ public void testNonCommit() throws EventDeliveryException, @Test public void testTwoBatches() throws InterruptedException, EventDeliveryException { - context.put(KafkaSourceConstants.BATCH_SIZE,"1"); - context.put(KafkaSourceConstants.BATCH_DURATION_MS,"30000"); + context.put(TOPICS, topic0); + context.put(BATCH_SIZE,"1"); + context.put(BATCH_DURATION_MS, "30000"); kafkaSource.configure(context); kafkaSource.start(); Thread.sleep(500L); - kafkaServer.produce(topicName, "", "event 1"); + kafkaServer.produce(topic0, "", "event 1"); Thread.sleep(500L); kafkaSource.process(); @@ -266,7 +343,7 @@ public void testTwoBatches() throws InterruptedException, Charsets.UTF_8)); events.clear(); - kafkaServer.produce(topicName, "", "event 2"); + kafkaServer.produce(topic0, "", "event 2"); Thread.sleep(500L); kafkaSource.process(); Assert.assertEquals("event 2", new String(events.get(0).getBody(), @@ -276,14 +353,15 @@ public void testTwoBatches() throws InterruptedException, @Test public void testTwoBatchesWithAutocommit() throws InterruptedException, EventDeliveryException { - context.put(KafkaSourceConstants.BATCH_SIZE,"1"); - context.put(KafkaSourceConstants.BATCH_DURATION_MS,"30000"); - context.put("kafka.auto.commit.enable","true"); + context.put(TOPICS, topic0); + context.put(BATCH_SIZE,"1"); + context.put(BATCH_DURATION_MS,"30000"); + context.put(KAFKA_CONSUMER_PREFIX + "enable.auto.commit", "true"); kafkaSource.configure(context); kafkaSource.start(); Thread.sleep(500L); - kafkaServer.produce(topicName, "", "event 1"); + kafkaServer.produce(topic0, "", "event 1"); Thread.sleep(500L); kafkaSource.process(); @@ -291,7 +369,7 @@ public void testTwoBatchesWithAutocommit() throws InterruptedException, Charsets.UTF_8)); events.clear(); - kafkaServer.produce(topicName, "", "event 2"); + kafkaServer.produce(topic0, "", "event 2"); Thread.sleep(500L); kafkaSource.process(); Assert.assertEquals("event 2", new String(events.get(0).getBody(), @@ -304,13 +382,14 @@ public void testTwoBatchesWithAutocommit() throws InterruptedException, public void testNullKey() throws EventDeliveryException, SecurityException, NoSuchFieldException, IllegalArgumentException, IllegalAccessException, InterruptedException { - context.put(KafkaSourceConstants.BATCH_SIZE,"1"); + context.put(TOPICS, topic0); + context.put(BATCH_SIZE, "1"); kafkaSource.configure(context); kafkaSource.start(); Thread.sleep(500L); - kafkaServer.produce(topicName, null , "hello, world"); + kafkaServer.produce(topic0, null, "hello, world"); Thread.sleep(500L); @@ -322,6 +401,110 @@ public void testNullKey() throws EventDeliveryException, Charsets.UTF_8)); } + @Test + public void testSourceProperties() { + Context context = new Context(); + context.put(TOPICS, "test1, test2"); + context.put(TOPICS_REGEX, "^stream[0-9]$"); + context.put(BOOTSTRAP_SERVERS, "bootstrap-servers-list"); + KafkaSource source = new KafkaSource(); + source.doConfigure(context); + + //check that kafka.topics.regex has higher priority than topics + //type of subscriber should be PatternSubscriber + KafkaSource.Subscriber subscriber = source.getSubscriber(); + Pattern pattern = subscriber.get(); + Assert.assertTrue(pattern.matcher("stream1").find()); + } + + @Test + public void testKafkaProperties() { + Context context = new Context(); + context.put(TOPICS, "test1, test2"); + context.put(KAFKA_CONSUMER_PREFIX + ConsumerConfig.GROUP_ID_CONFIG, "override.default.group.id"); + context.put(KAFKA_CONSUMER_PREFIX + "fake.property", "kafka.property.value"); + context.put(BOOTSTRAP_SERVERS, "real-bootstrap-servers-list"); + context.put(KAFKA_CONSUMER_PREFIX + "bootstrap.servers", "bad-bootstrap-servers-list"); + KafkaSource source = new KafkaSource(); + source.doConfigure(context); + Properties kafkaProps = source.getConsumerProps(); + + //check that we have defaults set + assertEquals( + String.valueOf(DEFAULT_AUTO_COMMIT), + kafkaProps.getProperty(ConsumerConfig.ENABLE_AUTO_COMMIT_CONFIG)); + //check that kafka properties override the default and get correct name + assertEquals( + "override.default.group.id", + kafkaProps.getProperty(ConsumerConfig.GROUP_ID_CONFIG)); + //check that any kafka property gets in + assertEquals( + "kafka.property.value", + kafkaProps.getProperty("fake.property")); + //check that documented property overrides defaults + assertEquals( + "real-bootstrap-servers-list", + kafkaProps.getProperty("bootstrap.servers")); + } + + @Test + public void testOldProperties() { + Context context = new Context(); + + context.put(TOPIC, "old.topic"); + context.put(OLD_GROUP_ID, "old.groupId"); + context.put(BOOTSTRAP_SERVERS, "real-bootstrap-servers-list"); + KafkaSource source = new KafkaSource(); + source.doConfigure(context); + Properties kafkaProps = source.getConsumerProps(); + + KafkaSource.Subscriber> subscriber = source.getSubscriber(); + //check topic was set + assertEquals( + "old.topic", + subscriber.get().get(0)); + //check that kafka old properties override the default and get correct name + assertEquals( + "old.groupId", + kafkaProps.getProperty(ConsumerConfig.GROUP_ID_CONFIG)); + + source = new KafkaSource(); + context.put(KAFKA_CONSUMER_PREFIX + ConsumerConfig.GROUP_ID_CONFIG, "override.old.group.id"); + source.doConfigure(context); + kafkaProps = source.getConsumerProps(); + //check that kafka new properties override old + assertEquals( + "override.old.group.id", + kafkaProps.getProperty(ConsumerConfig.GROUP_ID_CONFIG)); + + context.clear(); + context.put(BOOTSTRAP_SERVERS, "real-bootstrap-servers-list"); + context.put(TOPIC, "old.topic"); + source = new KafkaSource(); + source.doConfigure(context); + kafkaProps = source.getConsumerProps(); + //check defaults set + assertEquals( + KafkaSourceConstants.DEFAULT_GROUP_ID, + kafkaProps.getProperty(ConsumerConfig.GROUP_ID_CONFIG)); + } + + @Test + public void testPatternBasedSubscription() { + Context context = new Context(); + + context.put(TOPICS_REGEX, "^topic[0-9]$"); + context.put(OLD_GROUP_ID, "old.groupId"); + context.put(BOOTSTRAP_SERVERS, "real-bootstrap-servers-list"); + KafkaSource source = new KafkaSource(); + source.doConfigure(context); + KafkaSource.Subscriber subscriber = source.getSubscriber(); + for (int i = 0; i < 10; i++) { + Assert.assertTrue(subscriber.get().matcher("topic" + i).find()); + } + Assert.assertFalse(subscriber.get().matcher("topic").find()); + } + ChannelProcessor createGoodChannel() { ChannelProcessor channelProcessor = mock(ChannelProcessor.class); @@ -352,4 +535,4 @@ public Void answer(InvocationOnMock invocation) throws Throwable { return channelProcessor; } -} \ No newline at end of file +} diff --git a/flume-ng-sources/flume-kafka-source/src/test/java/org/apache/flume/source/kafka/TestKafkaSourceUtil.java b/flume-ng-sources/flume-kafka-source/src/test/java/org/apache/flume/source/kafka/TestKafkaSourceUtil.java deleted file mode 100644 index 0cbb4b69ec..0000000000 --- a/flume-ng-sources/flume-kafka-source/src/test/java/org/apache/flume/source/kafka/TestKafkaSourceUtil.java +++ /dev/null @@ -1,92 +0,0 @@ -/* - * Licensed to the Apache Software Foundation (ASF) under one or more - * contributor license agreements. See the NOTICE file distributed with - * this work for additional information regarding copyright ownership. - * The ASF licenses this file to You under the Apache License, Version 2.0 - * (the "License"); you may not use this file except in compliance with - * the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. - */ - -package org.apache.flume.source.kafka; - -import static org.junit.Assert.assertEquals; -import static org.junit.Assert.assertNotNull; -import static org.junit.Assert.assertNull; - -import java.io.IOException; -import java.util.Properties; - -import kafka.javaapi.consumer.ConsumerConnector; -import org.apache.flume.Context; -import org.apache.zookeeper.server.*; -import org.junit.After; -import org.junit.Before; -import org.junit.Test; - -public class TestKafkaSourceUtil { - private Properties props = new Properties(); - private Context context = new Context(); - private int zkPort = 21818; // none-standard - private KafkaSourceEmbeddedZookeeper zookeeper; - - @Before - public void setUp() throws Exception { - context.put("kafka.consumer.timeout", "10"); - context.put("type", "KafkaSource"); - context.put("topic", "test"); - context.put("zookeeperConnect", "127.0.0.1:"+zkPort); - context.put("groupId","test"); - props = KafkaSourceUtil.getKafkaProperties(context); - zookeeper = new KafkaSourceEmbeddedZookeeper(zkPort); - - - } - - @After - public void tearDown() throws Exception { - zookeeper.stopZookeeper(); - } - - - @Test - public void testGetConsumer() { - ConsumerConnector cc = KafkaSourceUtil.getConsumer(props); - assertNotNull(cc); - - } - - @Test - public void testKafkaConsumerProperties() { - Context context = new Context(); - context.put("kafka.auto.commit.enable", "override.default.autocommit"); - context.put("kafka.fake.property", "kafka.property.value"); - context.put("kafka.zookeeper.connect","bad-zookeeper-list"); - context.put("zookeeperConnect","real-zookeeper-list"); - Properties kafkaProps = KafkaSourceUtil.getKafkaProperties(context); - - //check that we have defaults set - assertEquals( - kafkaProps.getProperty(KafkaSourceConstants.GROUP_ID), - KafkaSourceConstants.DEFAULT_GROUP_ID); - //check that kafka properties override the default and get correct name - assertEquals( - kafkaProps.getProperty(KafkaSourceConstants.AUTO_COMMIT_ENABLED), - "override.default.autocommit"); - //check that any kafka property gets in - assertEquals(kafkaProps.getProperty("fake.property"), - "kafka.property.value"); - //check that documented property overrides defaults - assertEquals(kafkaProps.getProperty("zookeeper.connect") - ,"real-zookeeper-list"); - } - - -} diff --git a/pom.xml b/pom.xml index 15c086b509..3b2c97ce52 100644 --- a/pom.xml +++ b/pom.xml @@ -50,6 +50,7 @@ limitations under the License. 0.90.1 2.4.0 0.7.0 + 0.9.0.1 1.0.0 1.0.0 2.7.1 @@ -1337,12 +1338,12 @@ limitations under the License. org.apache.kafka kafka_2.10 - 0.8.1.1 + ${kafka.version} org.apache.kafka kafka_2.10 - 0.8.1.1 + ${kafka.version} test test From 7f588e6a158f5d108e39f50a92f8d1d108b12c24 Mon Sep 17 00:00:00 2001 From: Jarek Jarcec Cecho Date: Tue, 29 Mar 2016 09:43:40 -0700 Subject: [PATCH 277/341] FLUME-2822: Flume-Kafka-Sink with new Producer (Jeff Holoman via Jarek Jarcec Cecho) --- flume-ng-doc/sphinx/FlumeUserGuide.rst | 49 ++-- flume-ng-sinks/flume-ng-kafka-sink/pom.xml | 7 + .../apache/flume/sink/kafka/KafkaSink.java | 224 ++++++++++++++---- .../flume/sink/kafka/KafkaSinkConstants.java | 38 ++- .../flume/sink/kafka/KafkaSinkUtil.java | 103 -------- .../flume/sink/kafka/KafkaSinkUtilTest.java | 55 ----- .../flume/sink/kafka/TestKafkaSink.java | 74 +++++- 7 files changed, 310 insertions(+), 240 deletions(-) delete mode 100644 flume-ng-sinks/flume-ng-kafka-sink/src/main/java/org/apache/flume/sink/kafka/KafkaSinkUtil.java delete mode 100644 flume-ng-sinks/flume-ng-kafka-sink/src/test/java/org/apache/flume/sink/kafka/KafkaSinkUtilTest.java diff --git a/flume-ng-doc/sphinx/FlumeUserGuide.rst b/flume-ng-doc/sphinx/FlumeUserGuide.rst index 341ae4229c..15f27c3b5b 100644 --- a/flume-ng-doc/sphinx/FlumeUserGuide.rst +++ b/flume-ng-doc/sphinx/FlumeUserGuide.rst @@ -2493,7 +2493,9 @@ Kafka Sink This is a Flume Sink implementation that can publish data to a `Kafka `_ topic. One of the objective is to integrate Flume with Kafka so that pull based processing systems can process the data coming -through various Flume sources. This currently supports Kafka 0.8.x series of releases. +through various Flume sources. This currently supports Kafka 0.9.x series of releases. + +This version of Flume no longer supports Older Versions (0.8.x) of Kafka. Required properties are marked in bold font. @@ -2502,20 +2504,21 @@ Required properties are marked in bold font. Property Name Default Description =============================== =================== ============================================================================================= **type** -- Must be set to ``org.apache.flume.sink.kafka.KafkaSink`` -**brokerList** -- List of brokers Kafka-Sink will connect to, to get the list of topic partitions +**kafka.bootstrap.servers** -- List of brokers Kafka-Sink will connect to, to get the list of topic partitions This can be a partial list of brokers, but we recommend at least two for HA. The format is comma separated list of hostname:port -topic default-flume-topic The topic in Kafka to which the messages will be published. If this parameter is configured, +kafka.topic default-flume-topic The topic in Kafka to which the messages will be published. If this parameter is configured, messages will be published to this topic. If the event header contains a "topic" field, the event will be published to that topic overriding the topic configured here. -batchSize 100 How many messages to process in one batch. Larger batches improve throughput while adding latency. -requiredAcks 1 How many replicas must acknowledge a message before its considered successfully written. +flumeBatchSize 100 How many messages to process in one batch. Larger batches improve throughput while adding latency. +kafka.producer.acks 1 How many replicas must acknowledge a message before its considered successfully written. Accepted values are 0 (Never wait for acknowledgement), 1 (wait for leader only), -1 (wait for all replicas) Set this to -1 to avoid data loss in some cases of leader failure. Other Kafka Producer Properties -- These properties are used to configure the Kafka Producer. Any producer property supported - by Kafka can be used. The only requirement is to prepend the property name with the prefix ``kafka.``. - For example: kafka.producer.type + by Kafka can be used. The only requirement is to prepend the property name with the prefix + ``kafka.producer``. + For example: kafka.producer.linger.ms =============================== =================== ============================================================================================= .. note:: Kafka Sink uses the ``topic`` and ``key`` properties from the FlumeEvent headers to send events to Kafka. @@ -2523,22 +2526,38 @@ Other Kafka Producer Properties -- These properties are used If ``key`` exists in the headers, the key will used by Kafka to partition the data between the topic partitions. Events with same key will be sent to the same partition. If the key is null, events will be sent to random partitions. +The Kafka sink also provides defaults for the key.serializer(org.apache.kafka.common.serialization.StringSerializer) +and value.serializer(org.apache.kafka.common.serialization.ByteArraySerializer). Modification of these parameters is not recommended. + +Deprecated Properties + +=============================== =================== ============================================================================================= +Property Name Default Description +=============================== =================== ============================================================================================= +brokerList -- Use kafka.bootstrap.servers +topic default-flume-topic Use kafka.topic +batchSize 100 Use kafka.flumeBatchSize +requiredAcks 1 Use kafka.producer.acks + +=============================== =================== ============================================================================================= + An example configuration of a Kafka sink is given below. Properties starting -with the prefix ``kafka`` (the last 3 properties) are used when instantiating -the Kafka producer. The properties that are passed when creating the Kafka +with the prefix ``kafka.producer`` the Kafka producer. The properties that are passed when creating the Kafka producer are not limited to the properties given in this example. -Also it's possible include your custom properties here and access them inside +Also it is possible to include your custom properties here and access them inside the preprocessor through the Flume Context object passed in as a method argument. .. code-block:: properties - a1.sinks.k1.type = org.apache.flume.sink.kafka.KafkaSink - a1.sinks.k1.topic = mytopic - a1.sinks.k1.brokerList = localhost:9092 - a1.sinks.k1.requiredAcks = 1 - a1.sinks.k1.batchSize = 20 a1.sinks.k1.channel = c1 + a1.sinks.k1.type = org.apache.flume.sink.kafka.KafkaSink + a1.sinks.k1.kafka.topic = mytopic + a1.sinks.k1.kafka.bootstrap.servers = localhost:9092 + a1.sinks.k1.kafka.flumeBatchSize = 20 + a1.sinks.k1.kafka.producer.acks = 1 + a1.sinks.k1.kafka.producer.linger.ms = 1 + a1.sinks.ki.kafka.producer.compression.type = snappy Custom Sink ~~~~~~~~~~~ diff --git a/flume-ng-sinks/flume-ng-kafka-sink/pom.xml b/flume-ng-sinks/flume-ng-kafka-sink/pom.xml index 8475aa17eb..195c921434 100644 --- a/flume-ng-sinks/flume-ng-kafka-sink/pom.xml +++ b/flume-ng-sinks/flume-ng-kafka-sink/pom.xml @@ -72,7 +72,14 @@ org.apache.kafka kafka_2.10 + test + + org.apache.kafka + kafka-clients + ${kafka.version} + + diff --git a/flume-ng-sinks/flume-ng-kafka-sink/src/main/java/org/apache/flume/sink/kafka/KafkaSink.java b/flume-ng-sinks/flume-ng-kafka-sink/src/main/java/org/apache/flume/sink/kafka/KafkaSink.java index 38b854b55a..2e2140e33b 100644 --- a/flume-ng-sinks/flume-ng-kafka-sink/src/main/java/org/apache/flume/sink/kafka/KafkaSink.java +++ b/flume-ng-sinks/flume-ng-kafka-sink/src/main/java/org/apache/flume/sink/kafka/KafkaSink.java @@ -19,20 +19,46 @@ Licensed to the Apache Software Foundation (ASF) under one or more package org.apache.flume.sink.kafka; import com.google.common.base.Throwables; -import kafka.javaapi.producer.Producer; -import kafka.producer.KeyedMessage; -import kafka.producer.ProducerConfig; -import org.apache.flume.*; +import org.apache.flume.Channel; +import org.apache.flume.Context; +import org.apache.flume.Event; +import org.apache.flume.EventDeliveryException; +import org.apache.flume.Transaction; import org.apache.flume.conf.Configurable; +import org.apache.flume.conf.ConfigurationException; import org.apache.flume.instrumentation.kafka.KafkaSinkCounter; import org.apache.flume.sink.AbstractSink; +import org.apache.kafka.clients.producer.Callback; +import org.apache.kafka.clients.producer.KafkaProducer; +import org.apache.kafka.clients.producer.ProducerConfig; +import org.apache.kafka.clients.producer.ProducerRecord; +import org.apache.kafka.clients.producer.RecordMetadata; import org.slf4j.Logger; import org.slf4j.LoggerFactory; -import java.util.Properties; +import java.util.LinkedList; import java.util.List; import java.util.Map; -import java.util.ArrayList; +import java.util.Properties; +import java.util.concurrent.Future; + +import static org.apache.flume.sink.kafka.KafkaSinkConstants.BOOTSTRAP_SERVERS_CONFIG; +import static org.apache.flume.sink.kafka.KafkaSinkConstants.BATCH_SIZE; +import static org.apache.flume.sink.kafka.KafkaSinkConstants.DEFAULT_BATCH_SIZE; +import static org.apache.flume.sink.kafka.KafkaSinkConstants.BROKER_LIST_FLUME_KEY; +import static org.apache.flume.sink.kafka.KafkaSinkConstants.DEFAULT_ACKS; +import static org.apache.flume.sink.kafka.KafkaSinkConstants.DEFAULT_KEY_SERIALIZER; +import static org.apache.flume.sink.kafka.KafkaSinkConstants.DEFAULT_TOPIC; +import static org.apache.flume.sink.kafka.KafkaSinkConstants.DEFAULT_VALUE_SERIAIZER; +import static org.apache.flume.sink.kafka.KafkaSinkConstants.KAFKA_PRODUCER_PREFIX; +import static org.apache.flume.sink.kafka.KafkaSinkConstants.KEY_HEADER; +import static org.apache.flume.sink.kafka.KafkaSinkConstants.OLD_BATCH_SIZE; +import static org.apache.flume.sink.kafka.KafkaSinkConstants.REQUIRED_ACKS_FLUME_KEY; +import static org.apache.flume.sink.kafka.KafkaSinkConstants.TOPIC_CONFIG; +import static org.apache.flume.sink.kafka.KafkaSinkConstants.TOPIC_HEADER; +import static org.apache.flume.sink.kafka.KafkaSinkConstants.KEY_SERIALIZER_KEY; +import static org.apache.flume.sink.kafka.KafkaSinkConstants.MESSAGE_SERIALIZER_KEY; + /** * A Flume Sink that can publish messages to Kafka. @@ -67,16 +93,25 @@ Licensed to the Apache Software Foundation (ASF) under one or more public class KafkaSink extends AbstractSink implements Configurable { private static final Logger logger = LoggerFactory.getLogger(KafkaSink.class); - public static final String KEY_HDR = "key"; - public static final String TOPIC_HDR = "topic"; - private Properties kafkaProps; - private Producer producer; + + private final Properties kafkaProps = new Properties(); + private KafkaProducer producer; + private String topic; private int batchSize; - private List> messageList; + private List> kafkaFutures; private KafkaSinkCounter counter; + //For testing + public String getTopic() { + return topic; + } + + public int getBatchSize() { + return batchSize; + } + @Override public Status process() throws EventDeliveryException { Status result = Status.READY; @@ -92,7 +127,8 @@ public Status process() throws EventDeliveryException { transaction = channel.getTransaction(); transaction.begin(); - messageList.clear(); + kafkaFutures.clear(); + long batchStartTime = System.nanoTime(); for (; processedEvents < batchSize; processedEvents += 1) { event = channel.take(); @@ -110,11 +146,11 @@ public Status process() throws EventDeliveryException { byte[] eventBody = event.getBody(); Map headers = event.getHeaders(); - if ((eventTopic = headers.get(TOPIC_HDR)) == null) { + eventTopic = headers.get(TOPIC_HEADER); + if (eventTopic == null) { eventTopic = topic; } - - eventKey = headers.get(KEY_HDR); + eventKey = headers.get(KEY_HEADER); if (logger.isDebugEnabled()) { logger.debug("{Event} " + eventTopic + " : " + eventKey + " : " @@ -123,19 +159,22 @@ public Status process() throws EventDeliveryException { } // create a message and add to buffer - KeyedMessage data = new KeyedMessage - (eventTopic, eventKey, eventBody); - messageList.add(data); - + long startTime = System.currentTimeMillis(); + kafkaFutures.add(producer.send(new ProducerRecord (eventTopic, eventKey, eventBody), + new SinkCallback(startTime))); } + //Prevent linger.ms from holding the batch + producer.flush(); + // publish batch and commit. if (processedEvents > 0) { - long startTime = System.nanoTime(); - producer.send(messageList); + for (Future future : kafkaFutures) { + future.get(); + } long endTime = System.nanoTime(); - counter.addToKafkaEventSendTimer((endTime-startTime)/(1000*1000)); - counter.addToEventDrainSuccessCount(Long.valueOf(messageList.size())); + counter.addToKafkaEventSendTimer((endTime-batchStartTime)/(1000*1000)); + counter.addToEventDrainSuccessCount(Long.valueOf(kafkaFutures.size())); } transaction.commit(); @@ -146,6 +185,7 @@ public Status process() throws EventDeliveryException { result = Status.BACKOFF; if (transaction != null) { try { + kafkaFutures.clear(); transaction.rollback(); counter.incrementRollbackCount(); } catch (Exception e) { @@ -166,8 +206,7 @@ public Status process() throws EventDeliveryException { @Override public synchronized void start() { // instantiate the producer - ProducerConfig config = new ProducerConfig(kafkaProps); - producer = new Producer(config); + producer = new KafkaProducer(kafkaProps); counter.start(); super.start(); } @@ -197,31 +236,132 @@ public synchronized void stop() { @Override public void configure(Context context) { - batchSize = context.getInteger(KafkaSinkConstants.BATCH_SIZE, - KafkaSinkConstants.DEFAULT_BATCH_SIZE); - messageList = - new ArrayList>(batchSize); - logger.debug("Using batch size: {}", batchSize); - - topic = context.getString(KafkaSinkConstants.TOPIC, - KafkaSinkConstants.DEFAULT_TOPIC); - if (topic.equals(KafkaSinkConstants.DEFAULT_TOPIC)) { - logger.warn("The Property 'topic' is not set. " + - "Using the default topic name: " + - KafkaSinkConstants.DEFAULT_TOPIC); - } else { - logger.info("Using the static topic: " + topic + - " this may be over-ridden by event headers"); + translateOldProps(context); + + String topicStr = context.getString(TOPIC_CONFIG); + if (topicStr == null || topicStr.isEmpty()) { + topicStr = DEFAULT_TOPIC; + logger.warn("Topic was not specified. Using {} as the topic.", topicStr); + } + else { + logger.info("Using the static topic {}. This may be overridden by event headers", topicStr); + } + + topic = topicStr; + + batchSize = context.getInteger(BATCH_SIZE, DEFAULT_BATCH_SIZE); + + if (logger.isDebugEnabled()) { + logger.debug("Using batch size: {}", batchSize); + } + + kafkaFutures = new LinkedList>(); + + String bootStrapServers = context.getString(BOOTSTRAP_SERVERS_CONFIG); + if (bootStrapServers == null || bootStrapServers.isEmpty()) { + throw new ConfigurationException("Bootstrap Servers must be specified"); } - kafkaProps = KafkaSinkUtil.getKafkaProperties(context); + setProducerProps(context, bootStrapServers); if (logger.isDebugEnabled()) { - logger.debug("Kafka producer properties: " + kafkaProps); + logger.debug("Kafka producer properties: {}" , kafkaProps); } if (counter == null) { counter = new KafkaSinkCounter(getName()); } } + + private void translateOldProps(Context ctx) { + + if (!(ctx.containsKey(TOPIC_CONFIG))) { + ctx.put(TOPIC_CONFIG, ctx.getString("topic")); + logger.warn("{} is deprecated. Please use the parameter {}", "topic", TOPIC_CONFIG); + } + + //Broker List + // If there is no value we need to check and set the old param and log a warning message + if (!(ctx.containsKey(BOOTSTRAP_SERVERS_CONFIG))) { + String brokerList = ctx.getString(BROKER_LIST_FLUME_KEY); + if (brokerList == null || brokerList.isEmpty()) { + throw new ConfigurationException("Bootstrap Servers must be specified"); + } else { + ctx.put(BOOTSTRAP_SERVERS_CONFIG, brokerList); + logger.warn("{} is deprecated. Please use the parameter {}", BROKER_LIST_FLUME_KEY, BOOTSTRAP_SERVERS_CONFIG); + } + } + + //batch Size + if (!(ctx.containsKey(BATCH_SIZE))) { + String oldBatchSize = ctx.getString(OLD_BATCH_SIZE); + if ( oldBatchSize != null && !oldBatchSize.isEmpty()) { + ctx.put(BATCH_SIZE, oldBatchSize); + logger.warn("{} is deprecated. Please use the parameter {}", OLD_BATCH_SIZE, BATCH_SIZE); + } + } + + // Acks + if (!(ctx.containsKey(KAFKA_PRODUCER_PREFIX + ProducerConfig.ACKS_CONFIG))) { + String requiredKey = ctx.getString( + KafkaSinkConstants.REQUIRED_ACKS_FLUME_KEY); + if (!(requiredKey == null) && !(requiredKey.isEmpty())) { + ctx.put(KAFKA_PRODUCER_PREFIX + ProducerConfig.ACKS_CONFIG, requiredKey); + logger.warn("{} is deprecated. Please use the parameter {}", REQUIRED_ACKS_FLUME_KEY, + KAFKA_PRODUCER_PREFIX + ProducerConfig.ACKS_CONFIG); + } + } + + if (ctx.containsKey(KEY_SERIALIZER_KEY )) { + logger.warn("{} is deprecated. Flume now uses the latest Kafka producer which implements " + + "a different interface for serializers. Please use the parameter {}", + KEY_SERIALIZER_KEY,KAFKA_PRODUCER_PREFIX + ProducerConfig.KEY_SERIALIZER_CLASS_CONFIG); + } + + if (ctx.containsKey(MESSAGE_SERIALIZER_KEY)) { + logger.warn("{} is deprecated. Flume now uses the latest Kafka producer which implements " + + "a different interface for serializers. Please use the parameter {}", + MESSAGE_SERIALIZER_KEY,KAFKA_PRODUCER_PREFIX + ProducerConfig.VALUE_SERIALIZER_CLASS_CONFIG); + } + + + + + + } + private void setProducerProps(Context context, String bootStrapServers) { + kafkaProps.put(ProducerConfig.ACKS_CONFIG, DEFAULT_ACKS); + //Defaults overridden based on config + kafkaProps.put(ProducerConfig.KEY_SERIALIZER_CLASS_CONFIG, DEFAULT_KEY_SERIALIZER); + kafkaProps.put(ProducerConfig.VALUE_SERIALIZER_CLASS_CONFIG, DEFAULT_VALUE_SERIAIZER); + kafkaProps.putAll(context.getSubProperties(KAFKA_PRODUCER_PREFIX)); + kafkaProps.put(ProducerConfig.BOOTSTRAP_SERVERS_CONFIG, bootStrapServers); + logger.info("Producer properties: {}" , kafkaProps.toString()); + } + + protected Properties getKafkaProps() { + return kafkaProps; + } +} + +class SinkCallback implements Callback { + private static final Logger logger = LoggerFactory.getLogger(SinkCallback.class); + private long startTime; + + public SinkCallback(long startTime) { + this.startTime = startTime; + } + + public void onCompletion(RecordMetadata metadata, Exception exception) { + if (exception != null) { + logger.debug("Error sending message to Kafka {} ", exception.getMessage()); + } + + if (logger.isDebugEnabled()) { + long eventElapsedTime = System.currentTimeMillis() - startTime; + logger.debug("Acked message partition:{} ofset:{}", metadata.partition(), metadata.offset()); + logger.debug("Elapsed time for send: {}", eventElapsedTime); + } + } } + diff --git a/flume-ng-sinks/flume-ng-kafka-sink/src/main/java/org/apache/flume/sink/kafka/KafkaSinkConstants.java b/flume-ng-sinks/flume-ng-kafka-sink/src/main/java/org/apache/flume/sink/kafka/KafkaSinkConstants.java index 3ee12de636..c84dec0d85 100644 --- a/flume-ng-sinks/flume-ng-kafka-sink/src/main/java/org/apache/flume/sink/kafka/KafkaSinkConstants.java +++ b/flume-ng-sinks/flume-ng-kafka-sink/src/main/java/org/apache/flume/sink/kafka/KafkaSinkConstants.java @@ -18,30 +18,42 @@ Licensed to the Apache Software Foundation (ASF) under one or more package org.apache.flume.sink.kafka; -import kafka.serializer.StringDecoder; +import org.apache.kafka.clients.CommonClientConfigs; public class KafkaSinkConstants { - public static final String PROPERTY_PREFIX = "kafka."; + public static final String KAFKA_PREFIX = "kafka."; + public static final String KAFKA_PRODUCER_PREFIX = KAFKA_PREFIX + "producer."; /* Properties */ - public static final String TOPIC = "topic"; - public static final String BATCH_SIZE = "batchSize"; + public static final String TOPIC_CONFIG = KAFKA_PREFIX + "topic"; + public static final String BATCH_SIZE = "flumeBatchSize"; + public static final String BOOTSTRAP_SERVERS_CONFIG = KAFKA_PREFIX + CommonClientConfigs.BOOTSTRAP_SERVERS_CONFIG; + + public static final String KEY_HEADER = "key"; + public static final String TOPIC_HEADER = "topic"; + + public static final String DEFAULT_KEY_SERIALIZER = "org.apache.kafka.common.serialization.StringSerializer"; + public static final String DEFAULT_VALUE_SERIAIZER = "org.apache.kafka.common.serialization.ByteArraySerializer"; + + public static final int DEFAULT_BATCH_SIZE = 100; + public static final String DEFAULT_TOPIC = "default-flume-topic"; + public static final String DEFAULT_ACKS = "1"; + + + + /* Old Properties */ + + /* Properties */ + + public static final String OLD_BATCH_SIZE = "batchSize"; public static final String MESSAGE_SERIALIZER_KEY = "serializer.class"; public static final String KEY_SERIALIZER_KEY = "key.serializer.class"; - public static final String BROKER_LIST_KEY = "metadata.broker.list"; - public static final String REQUIRED_ACKS_KEY = "request.required.acks"; public static final String BROKER_LIST_FLUME_KEY = "brokerList"; public static final String REQUIRED_ACKS_FLUME_KEY = "requiredAcks"; - public static final int DEFAULT_BATCH_SIZE = 100; - public static final String DEFAULT_TOPIC = "default-flume-topic"; - public static final String DEFAULT_MESSAGE_SERIALIZER = - "kafka.serializer.DefaultEncoder"; - public static final String DEFAULT_KEY_SERIALIZER = - "kafka.serializer.StringEncoder"; - public static final String DEFAULT_REQUIRED_ACKS = "1"; } + diff --git a/flume-ng-sinks/flume-ng-kafka-sink/src/main/java/org/apache/flume/sink/kafka/KafkaSinkUtil.java b/flume-ng-sinks/flume-ng-kafka-sink/src/main/java/org/apache/flume/sink/kafka/KafkaSinkUtil.java deleted file mode 100644 index 66bde85bb0..0000000000 --- a/flume-ng-sinks/flume-ng-kafka-sink/src/main/java/org/apache/flume/sink/kafka/KafkaSinkUtil.java +++ /dev/null @@ -1,103 +0,0 @@ -/* - * Licensed to the Apache Software Foundation (ASF) under one or more - * contributor license agreements. See the NOTICE file distributed with - * this work for additional information regarding copyright ownership. - * The ASF licenses this file to You under the Apache License, Version 2.0 - * (the "License"); you may not use this file except in compliance with - * the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. - */ -package org.apache.flume.sink.kafka; - -import org.apache.flume.Context; -import org.apache.flume.conf.ConfigurationException; -import org.slf4j.Logger; -import org.slf4j.LoggerFactory; -import scala.util.PropertiesTrait; - -import java.util.Map; -import java.util.Properties; - -public class KafkaSinkUtil { - - private static final Logger log = - LoggerFactory.getLogger(KafkaSinkUtil.class); - - public static Properties getKafkaProperties(Context context) { - log.info("context={}",context.toString()); - Properties props = generateDefaultKafkaProps(); - setKafkaProps(context, props); - addDocumentedKafkaProps(context, props); - return props; - } - - /** - * Some of the producer properties are especially important - * We documented them and gave them a camel-case name to match Flume config - * If user set these, we will override any existing parameters with these - * settings. - * Knowledge of which properties are documented is maintained here for now. - * If this will become a maintenance issue we'll set a proper data structure. - */ - private static void addDocumentedKafkaProps(Context context, - Properties kafkaProps) - throws ConfigurationException { - String brokerList = context.getString(KafkaSinkConstants - .BROKER_LIST_FLUME_KEY); - if (brokerList == null) { - throw new ConfigurationException("brokerList must contain at least " + - "one Kafka broker"); - } - kafkaProps.put(KafkaSinkConstants.BROKER_LIST_KEY, brokerList); - - String requiredKey = context.getString( - KafkaSinkConstants.REQUIRED_ACKS_FLUME_KEY); - - if (requiredKey != null ) { - kafkaProps.put(KafkaSinkConstants.REQUIRED_ACKS_KEY, requiredKey); - } - } - - - /** - * Generate producer properties object with some defaults - * @return - */ - private static Properties generateDefaultKafkaProps() { - Properties props = new Properties(); - props.put(KafkaSinkConstants.MESSAGE_SERIALIZER_KEY, - KafkaSinkConstants.DEFAULT_MESSAGE_SERIALIZER); - props.put(KafkaSinkConstants.KEY_SERIALIZER_KEY, - KafkaSinkConstants.DEFAULT_KEY_SERIALIZER); - props.put(KafkaSinkConstants.REQUIRED_ACKS_KEY, - KafkaSinkConstants.DEFAULT_REQUIRED_ACKS); - return props; - } - - - /** - * Add all configuration parameters starting with "kafka" - * to producer properties - */ - private static void setKafkaProps(Context context, Properties kafkaProps) { - - Map kafkaProperties = - context.getSubProperties(KafkaSinkConstants.PROPERTY_PREFIX); - - for (Map.Entry prop : kafkaProperties.entrySet()) { - - kafkaProps.put(prop.getKey(), prop.getValue()); - if (log.isDebugEnabled()) { - log.debug("Reading a Kafka Producer Property: key: " - + prop.getKey() + ", value: " + prop.getValue()); - } - } - } -} diff --git a/flume-ng-sinks/flume-ng-kafka-sink/src/test/java/org/apache/flume/sink/kafka/KafkaSinkUtilTest.java b/flume-ng-sinks/flume-ng-kafka-sink/src/test/java/org/apache/flume/sink/kafka/KafkaSinkUtilTest.java deleted file mode 100644 index 84d213ccf0..0000000000 --- a/flume-ng-sinks/flume-ng-kafka-sink/src/test/java/org/apache/flume/sink/kafka/KafkaSinkUtilTest.java +++ /dev/null @@ -1,55 +0,0 @@ -/* - * Licensed to the Apache Software Foundation (ASF) under one or more - * contributor license agreements. See the NOTICE file distributed with - * this work for additional information regarding copyright ownership. - * The ASF licenses this file to You under the Apache License, Version 2.0 - * (the "License"); you may not use this file except in compliance with - * the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. - */ - -package org.apache.flume.sink.kafka; - -import junit.framework.TestCase; -import org.apache.flume.Context; -import org.apache.flume.conf.Configurables; -import org.junit.Test; - -import java.util.Properties; - -import static org.junit.Assert.assertEquals; - -public class KafkaSinkUtilTest extends TestCase { - - @Test - public void testGetKafkaProperties() { - Context context = new Context(); - context.put("kafka.serializer.class", "override.default.serializer"); - context.put("kafka.fake.property", "kafka.property.value"); - context.put("kafka.metadata.broker.list","bad-broker-list"); - context.put("brokerList","real-broker-list"); - Properties kafkaProps = KafkaSinkUtil.getKafkaProperties(context); - - //check that we have defaults set - assertEquals( - kafkaProps.getProperty(KafkaSinkConstants.KEY_SERIALIZER_KEY), - KafkaSinkConstants.DEFAULT_KEY_SERIALIZER); - //check that kafka properties override the default and get correct name - assertEquals( - kafkaProps.getProperty(KafkaSinkConstants.MESSAGE_SERIALIZER_KEY), - "override.default.serializer"); - //check that any kafka property gets in - assertEquals(kafkaProps.getProperty("fake.property"), - "kafka.property.value"); - //check that documented property overrides defaults - assertEquals(kafkaProps.getProperty("metadata.broker.list") - ,"real-broker-list"); - } -} \ No newline at end of file diff --git a/flume-ng-sinks/flume-ng-kafka-sink/src/test/java/org/apache/flume/sink/kafka/TestKafkaSink.java b/flume-ng-sinks/flume-ng-kafka-sink/src/test/java/org/apache/flume/sink/kafka/TestKafkaSink.java index 72117b197f..1852099965 100644 --- a/flume-ng-sinks/flume-ng-kafka-sink/src/test/java/org/apache/flume/sink/kafka/TestKafkaSink.java +++ b/flume-ng-sinks/flume-ng-kafka-sink/src/test/java/org/apache/flume/sink/kafka/TestKafkaSink.java @@ -19,11 +19,18 @@ Licensed to the Apache Software Foundation (ASF) under one or more package org.apache.flume.sink.kafka; import kafka.message.MessageAndMetadata; -import org.apache.flume.*; +import org.apache.flume.Channel; +import org.apache.flume.Context; +import org.apache.flume.Event; +import org.apache.flume.EventDeliveryException; +import org.apache.flume.Sink; +import org.apache.flume.Transaction; import org.apache.flume.channel.MemoryChannel; import org.apache.flume.conf.Configurables; import org.apache.flume.event.EventBuilder; import org.apache.flume.sink.kafka.util.TestUtil; +import org.apache.kafka.clients.CommonClientConfigs; +import org.apache.kafka.clients.producer.ProducerConfig; import org.junit.AfterClass; import org.junit.BeforeClass; import org.junit.Test; @@ -33,12 +40,14 @@ Licensed to the Apache Software Foundation (ASF) under one or more import java.util.HashMap; import java.util.List; import java.util.Map; - +import java.util.Properties; import static org.junit.Assert.assertEquals; import static org.junit.Assert.assertNull; import static org.junit.Assert.fail; +import static org.apache.flume.sink.kafka.KafkaSinkConstants.*; + /** * Unit tests for Kafka Sink */ @@ -50,7 +59,7 @@ public class TestKafkaSink { public static void setup() { testUtil.prepare(); List topics = new ArrayList(3); - topics.add(KafkaSinkConstants.DEFAULT_TOPIC); + topics.add(DEFAULT_TOPIC); topics.add(TestConstants.STATIC_TOPIC); topics.add(TestConstants.CUSTOM_TOPIC); testUtil.initTopicList(topics); @@ -61,6 +70,50 @@ public static void tearDown() { testUtil.tearDown(); } + @Test + public void testKafkaProperties() { + + KafkaSink kafkaSink = new KafkaSink(); + Context context = new Context(); + context.put(KAFKA_PREFIX + TOPIC_CONFIG, ""); + context.put(KAFKA_PRODUCER_PREFIX + ProducerConfig.VALUE_SERIALIZER_CLASS_CONFIG, "override.default.serializer"); + context.put("kafka.producer.fake.property", "kafka.property.value"); + context.put("kafka.bootstrap.servers", "localhost:9092,localhost:9092"); + context.put("brokerList","real-broker-list"); + Configurables.configure(kafkaSink,context); + + Properties kafkaProps = kafkaSink.getKafkaProps(); + + //check that we have defaults set + assertEquals( + kafkaProps.getProperty(ProducerConfig.KEY_SERIALIZER_CLASS_CONFIG), DEFAULT_KEY_SERIALIZER); + //check that kafka properties override the default and get correct name + assertEquals(kafkaProps.getProperty(ProducerConfig.VALUE_SERIALIZER_CLASS_CONFIG), "override.default.serializer"); + //check that any kafka-producer property gets in + assertEquals(kafkaProps.getProperty("fake.property"), "kafka.property.value"); + //check that documented property overrides defaults + assertEquals(kafkaProps.getProperty("bootstrap.servers") ,"localhost:9092,localhost:9092"); + } + + @Test + public void testOldProperties() { + KafkaSink kafkaSink = new KafkaSink(); + Context context = new Context(); + context.put("topic","test-topic"); + context.put(OLD_BATCH_SIZE, "300"); + context.put(BROKER_LIST_FLUME_KEY,"localhost:9092,localhost:9092"); + context.put(REQUIRED_ACKS_FLUME_KEY, "all"); + Configurables.configure(kafkaSink,context); + + Properties kafkaProps = kafkaSink.getKafkaProps(); + + assertEquals(kafkaSink.getTopic(), "test-topic"); + assertEquals(kafkaSink.getBatchSize(),300); + assertEquals(kafkaProps.getProperty(CommonClientConfigs.BOOTSTRAP_SERVERS_CONFIG),"localhost:9092,localhost:9092"); + assertEquals(kafkaProps.getProperty(ProducerConfig.ACKS_CONFIG), "all"); + + } + @Test public void testDefaultTopic() { Sink kafkaSink = new KafkaSink(); @@ -89,7 +142,7 @@ public void testDefaultTopic() { } String fetchedMsg = new String((byte[]) - testUtil.getNextMessageFromConsumer(KafkaSinkConstants.DEFAULT_TOPIC) + testUtil.getNextMessageFromConsumer(DEFAULT_TOPIC) .message()); assertEquals(msg, fetchedMsg); } @@ -98,7 +151,7 @@ public void testDefaultTopic() { public void testStaticTopic() { Context context = prepareDefaultContext(); // add the static topic - context.put(KafkaSinkConstants.TOPIC, TestConstants.STATIC_TOPIC); + context.put(TOPIC_CONFIG, TestConstants.STATIC_TOPIC); String msg = "static-topic-test"; try { @@ -110,8 +163,7 @@ public void testStaticTopic() { // ignore } - String fetchedMsg = new String((byte[]) testUtil.getNextMessageFromConsumer( - TestConstants.STATIC_TOPIC).message()); + String fetchedMsg = new String((byte[]) testUtil.getNextMessageFromConsumer(TestConstants.STATIC_TOPIC).message()); assertEquals(msg, fetchedMsg); } @@ -172,16 +224,14 @@ public void testEmptyChannel() throws UnsupportedEncodingException, fail("Error Occurred"); } assertNull( - testUtil.getNextMessageFromConsumer(KafkaSinkConstants.DEFAULT_TOPIC)); + testUtil.getNextMessageFromConsumer(DEFAULT_TOPIC)); } private Context prepareDefaultContext() { // Prepares a default context with Kafka Server Properties Context context = new Context(); - context.put("brokerList", testUtil.getKafkaServerUrl()); - context.put("kafka.request.required.acks", "1"); - context.put("kafka.producer.type","sync"); - context.put("batchSize", "1"); + context.put(BOOTSTRAP_SERVERS_CONFIG, testUtil.getKafkaServerUrl()); + context.put(BATCH_SIZE, "1"); return context; } From e8c4a7bffc74f6ea10ae6cc45adbaf4919f45186 Mon Sep 17 00:00:00 2001 From: Jarek Jarcec Cecho Date: Tue, 29 Mar 2016 09:45:40 -0700 Subject: [PATCH 278/341] FLUME-2823: Flume-Kafka-Channel with new APIs (Jeff Holoman via Jarek Jarcec Cecho) --- flume-ng-channels/flume-kafka-channel/pom.xml | 6 + .../flume/channel/kafka/KafkaChannel.java | 641 ++++++++++++------ .../kafka/KafkaChannelConfiguration.java | 32 +- .../flume/channel/kafka/TestKafkaChannel.java | 219 +++--- .../test/resources/kafka-server.properties | 2 +- flume-ng-doc/sphinx/FlumeUserGuide.rst | 88 ++- 6 files changed, 634 insertions(+), 354 deletions(-) diff --git a/flume-ng-channels/flume-kafka-channel/pom.xml b/flume-ng-channels/flume-kafka-channel/pom.xml index fa1bd420de..587b4b4f37 100644 --- a/flume-ng-channels/flume-kafka-channel/pom.xml +++ b/flume-ng-channels/flume-kafka-channel/pom.xml @@ -40,6 +40,12 @@ limitations under the License. org.apache.kafka kafka_2.10 + test + + + org.apache.kafka + kafka-clients + ${kafka.version} org.apache.flume.flume-ng-sinks diff --git a/flume-ng-channels/flume-kafka-channel/src/main/java/org/apache/flume/channel/kafka/KafkaChannel.java b/flume-ng-channels/flume-kafka-channel/src/main/java/org/apache/flume/channel/kafka/KafkaChannel.java index c0c1c6678f..2d9b0c6c18 100644 --- a/flume-ng-channels/flume-kafka-channel/src/main/java/org/apache/flume/channel/kafka/KafkaChannel.java +++ b/flume-ng-channels/flume-kafka-channel/src/main/java/org/apache/flume/channel/kafka/KafkaChannel.java @@ -20,108 +20,120 @@ import com.google.common.annotations.VisibleForTesting; import com.google.common.base.Optional; -import kafka.consumer.*; -import kafka.javaapi.consumer.ConsumerConnector; -import kafka.javaapi.producer.Producer; -import kafka.producer.KeyedMessage; -import kafka.producer.ProducerConfig; -import org.apache.avro.io.*; +import org.apache.avro.io.BinaryDecoder; +import org.apache.avro.io.BinaryEncoder; +import org.apache.avro.io.DecoderFactory; +import org.apache.avro.io.EncoderFactory; import org.apache.avro.specific.SpecificDatumReader; import org.apache.avro.specific.SpecificDatumWriter; -import org.apache.flume.*; +import org.apache.flume.ChannelException; +import org.apache.flume.Context; +import org.apache.flume.Event; +import org.apache.flume.FlumeException; import org.apache.flume.channel.BasicChannelSemantics; import org.apache.flume.channel.BasicTransactionSemantics; import org.apache.flume.conf.ConfigurationException; - -import static org.apache.flume.channel.kafka.KafkaChannelConfiguration.*; - import org.apache.flume.event.EventBuilder; import org.apache.flume.instrumentation.kafka.KafkaChannelCounter; import org.apache.flume.source.avro.AvroFlumeEvent; +import org.apache.kafka.clients.consumer.ConsumerConfig; +import org.apache.kafka.clients.consumer.ConsumerRebalanceListener; +import org.apache.kafka.clients.consumer.ConsumerRecord; +import org.apache.kafka.clients.consumer.ConsumerRecords; +import org.apache.kafka.clients.consumer.KafkaConsumer; +import org.apache.kafka.clients.consumer.OffsetAndMetadata; +import org.apache.kafka.clients.producer.Callback; +import org.apache.kafka.clients.producer.KafkaProducer; +import org.apache.kafka.clients.producer.ProducerConfig; +import org.apache.kafka.clients.producer.ProducerRecord; +import org.apache.kafka.clients.producer.RecordMetadata; +import org.apache.kafka.common.TopicPartition; import org.slf4j.Logger; import org.slf4j.LoggerFactory; -import java.io.*; +import java.io.ByteArrayInputStream; +import java.io.ByteArrayOutputStream; +import java.io.IOException; import java.nio.ByteBuffer; -import java.util.*; +import java.util.Arrays; +import java.util.Collection; +import java.util.Collections; +import java.util.HashMap; +import java.util.Iterator; +import java.util.LinkedList; +import java.util.List; +import java.util.Map; +import java.util.Properties; +import java.util.UUID; +import java.util.concurrent.Future; +import java.util.concurrent.atomic.AtomicBoolean; import java.util.concurrent.atomic.AtomicReference; +import static org.apache.flume.channel.kafka.KafkaChannelConfiguration.*; + public class KafkaChannel extends BasicChannelSemantics { - private final static Logger LOGGER = - LoggerFactory.getLogger(KafkaChannel.class); + private final static Logger logger = + LoggerFactory.getLogger(KafkaChannel.class); + private final Properties consumerProps = new Properties(); + private final Properties producerProps = new Properties(); - private final Properties kafkaConf = new Properties(); - private Producer producer; + private KafkaProducer producer; private final String channelUUID = UUID.randomUUID().toString(); private AtomicReference topic = new AtomicReference(); private boolean parseAsFlumeEvent = DEFAULT_PARSE_AS_FLUME_EVENT; - private final Map topicCountMap = - Collections.synchronizedMap(new HashMap()); + + //used to indicate if a rebalance has occurred during the current transaction + AtomicBoolean rebalanceFlag = new AtomicBoolean(); + //This isn't a Kafka property per se, but we allow it to be configurable + private long pollTimeout = DEFAULT_POLL_TIMEOUT; + // Track all consumers to close them eventually. - private final List consumers = - Collections.synchronizedList(new LinkedList()); + private final List consumers = + Collections.synchronizedList(new LinkedList()); private KafkaChannelCounter counter; - /* Each ConsumerConnector commit will commit all partitions owned by it. To + /* Each Consumer commit will commit all partitions owned by it. To * ensure that each partition is only committed when all events are - * actually done, we will need to keep a ConsumerConnector per thread. - * See Neha's answer here: - * http://grokbase.com/t/kafka/users/13b4gmk2jk/commit-offset-per-topic - * Since only one consumer connector will a partition at any point in time, - * when we commit the partition we would have committed all events to the - * final destination from that partition. - * - * If a new partition gets assigned to this connector, - * my understanding is that all message from the last partition commit will - * get replayed which may cause duplicates -- which is fine as this - * happens only on partition rebalancing which is on failure or new nodes - * coming up, which is rare. + * actually done, we will need to keep a Consumer per thread. */ - private final ThreadLocal consumerAndIter = new - ThreadLocal() { - @Override - public ConsumerAndIterator initialValue() { - return createConsumerAndIter(); - } - }; + + private final ThreadLocal consumerAndRecords = new + ThreadLocal() { + @Override + public ConsumerAndRecords initialValue() { + return createConsumerAndRecords(); + } + }; @Override public void start() { - try { - LOGGER.info("Starting Kafka Channel: " + getName()); - producer = new Producer(new ProducerConfig(kafkaConf)); + logger.info("Starting Kafka Channel: {}", getName()); + producer = new KafkaProducer(producerProps); // We always have just one topic being read by one thread - LOGGER.info("Topic = " + topic.get()); - topicCountMap.put(topic.get(), 1); + logger.info("Topic = {}", topic.get()); counter.start(); super.start(); - } catch (Exception e) { - LOGGER.error("Could not start producer"); - throw new FlumeException("Unable to create Kafka Connections. " + - "Check whether Kafka Brokers are up and that the " + - "Flume agent can connect to it.", e); - } } @Override public void stop() { - for (ConsumerAndIterator c : consumers) { + for (ConsumerAndRecords c : consumers) { try { - decommissionConsumerAndIterator(c); + decommissionConsumerAndRecords(c); } catch (Exception ex) { - LOGGER.warn("Error while shutting down consumer.", ex); + logger.warn("Error while shutting down consumer.", ex); } } producer.close(); counter.stop(); super.stop(); - LOGGER.info("Kafka channel {} stopped. Metrics: {}", getName(), - counter); + logger.info("Kafka channel {} stopped. Metrics: {}", getName(), + counter); } @Override @@ -129,98 +141,147 @@ protected BasicTransactionSemantics createTransaction() { return new KafkaTransaction(); } - private synchronized ConsumerAndIterator createConsumerAndIter() { - try { - ConsumerConfig consumerConfig = new ConsumerConfig(kafkaConf); - ConsumerConnector consumer = - Consumer.createJavaConsumerConnector(consumerConfig); - Map>> consumerMap = - consumer.createMessageStreams(topicCountMap); - final List> streamList = consumerMap - .get(topic.get()); - KafkaStream stream = streamList.remove(0); - ConsumerAndIterator ret = - new ConsumerAndIterator(consumer, stream.iterator(), channelUUID); - consumers.add(ret); - LOGGER.info("Created new consumer to connect to Kafka"); - return ret; - } catch (Exception e) { - throw new FlumeException("Unable to connect to Kafka", e); - } - } - - Properties getKafkaConf() { - return kafkaConf; - } - @Override public void configure(Context ctx) { - String topicStr = ctx.getString(TOPIC); + + //Can remove in the next release + translateOldProps(ctx); + + String topicStr = ctx.getString(TOPIC_CONFIG); if (topicStr == null || topicStr.isEmpty()) { topicStr = DEFAULT_TOPIC; - LOGGER - .info("Topic was not specified. Using " + topicStr + " as the topic."); + logger.info("Topic was not specified. Using {} as the topic.", topicStr); } topic.set(topicStr); - String groupId = ctx.getString(GROUP_ID_FLUME); - if (groupId == null || groupId.isEmpty()) { - groupId = DEFAULT_GROUP_ID; - LOGGER.info( - "Group ID was not specified. Using " + groupId + " as the group id."); + String bootStrapServers = ctx.getString(BOOTSTRAP_SERVERS_CONFIG); + if (bootStrapServers == null || bootStrapServers.isEmpty()) { + throw new ConfigurationException("Bootstrap Servers must be specified"); } - String brokerList = ctx.getString(BROKER_LIST_FLUME_KEY); - if (brokerList == null || brokerList.isEmpty()) { - throw new ConfigurationException("Broker List must be specified"); + + setProducerProps(ctx, bootStrapServers); + setConsumerProps(ctx, bootStrapServers); + + parseAsFlumeEvent = ctx.getBoolean(PARSE_AS_FLUME_EVENT, DEFAULT_PARSE_AS_FLUME_EVENT); + pollTimeout = ctx.getLong(POLL_TIMEOUT, DEFAULT_POLL_TIMEOUT); + + if (counter == null) { + counter = new KafkaChannelCounter(getName()); } - String zkConnect = ctx.getString(ZOOKEEPER_CONNECT_FLUME_KEY); - if (zkConnect == null || zkConnect.isEmpty()) { - throw new ConfigurationException( - "Zookeeper Connection must be specified"); + } + + // We can remove this once the properties are officially deprecated + private void translateOldProps(Context ctx) { + + if (!(ctx.containsKey(TOPIC_CONFIG))) { + ctx.put(TOPIC_CONFIG, ctx.getString("topic")); + logger.warn("{} is deprecated. Please use the parameter {}", "topic", TOPIC_CONFIG); } - kafkaConf.putAll(ctx.getSubProperties(KAFKA_PREFIX)); - kafkaConf.put(GROUP_ID, groupId); - kafkaConf.put(BROKER_LIST_KEY, brokerList); - kafkaConf.put(ZOOKEEPER_CONNECT, zkConnect); - kafkaConf.put(AUTO_COMMIT_ENABLED, String.valueOf(false)); - if(kafkaConf.get(CONSUMER_TIMEOUT) == null) { - kafkaConf.put(CONSUMER_TIMEOUT, DEFAULT_TIMEOUT); + + //Broker List + // If there is no value we need to check and set the old param and log a warning message + if (!(ctx.containsKey(BOOTSTRAP_SERVERS_CONFIG))) { + String brokerList = ctx.getString(BROKER_LIST_FLUME_KEY); + if (brokerList == null || brokerList.isEmpty()) { + throw new ConfigurationException("Bootstrap Servers must be specified"); + } else { + ctx.put(BOOTSTRAP_SERVERS_CONFIG, brokerList); + logger.warn("{} is deprecated. Please use the parameter {}", BROKER_LIST_FLUME_KEY, BOOTSTRAP_SERVERS_CONFIG); + } } - kafkaConf.put(REQUIRED_ACKS_KEY, "-1"); - LOGGER.info(kafkaConf.toString()); - parseAsFlumeEvent = - ctx.getBoolean(PARSE_AS_FLUME_EVENT, DEFAULT_PARSE_AS_FLUME_EVENT); - - boolean readSmallest = ctx.getBoolean(READ_SMALLEST_OFFSET, - DEFAULT_READ_SMALLEST_OFFSET); - // If the data is to be parsed as Flume events, we always read the smallest. - // Else, we read the configuration, which by default reads the largest. - if (parseAsFlumeEvent || readSmallest) { - // readSmallest is eval-ed only if parseAsFlumeEvent is false. - // The default is largest, so we don't need to set it explicitly. - kafkaConf.put("auto.offset.reset", "smallest"); + + //GroupId + // If there is an old Group Id set, then use that if no groupId is set. + if (!(ctx.containsKey(KAFKA_CONSUMER_PREFIX + ConsumerConfig.GROUP_ID_CONFIG))) { + String oldGroupId = ctx.getString(GROUP_ID_FLUME); + if ( oldGroupId != null && !oldGroupId.isEmpty()) { + ctx.put(KAFKA_CONSUMER_PREFIX + ConsumerConfig.GROUP_ID_CONFIG, oldGroupId); + logger.warn("{} is deprecated. Please use the parameter {}", GROUP_ID_FLUME, KAFKA_CONSUMER_PREFIX + ConsumerConfig.GROUP_ID_CONFIG); + } } - if (counter == null) { - counter = new KafkaChannelCounter(getName()); + if (!(ctx.containsKey((KAFKA_CONSUMER_PREFIX + ConsumerConfig.AUTO_OFFSET_RESET_CONFIG)))) { + Boolean oldReadSmallest = ctx.getBoolean(READ_SMALLEST_OFFSET); + String auto; + if (oldReadSmallest != null) { + if (oldReadSmallest) { + auto = "earliest"; + } else { + auto = "latest"; + } + ctx.put(KAFKA_CONSUMER_PREFIX + ConsumerConfig.AUTO_OFFSET_RESET_CONFIG,auto); + logger.warn("{} is deprecated. Please use the parameter {}", READ_SMALLEST_OFFSET,KAFKA_CONSUMER_PREFIX + ConsumerConfig.AUTO_OFFSET_RESET_CONFIG); + } + + } + } + + + private void setProducerProps(Context ctx, String bootStrapServers) { + producerProps.put(ProducerConfig.ACKS_CONFIG, DEFAULT_ACKS); + producerProps.put(ProducerConfig.KEY_SERIALIZER_CLASS_CONFIG, DEFAULT_KEY_SERIALIZER); + producerProps.put(ProducerConfig.VALUE_SERIALIZER_CLASS_CONFIG, DEFAULT_VALUE_SERIAIZER); + //Defaults overridden based on config + producerProps.putAll(ctx.getSubProperties(KAFKA_PRODUCER_PREFIX)); + producerProps.put(ProducerConfig.BOOTSTRAP_SERVERS_CONFIG, bootStrapServers); + logger.info("Producer properties: " + producerProps.toString()); + } + + protected Properties getProducerProps() { + return producerProps; + } + + private void setConsumerProps(Context ctx, String bootStrapServers) { + String groupId = ctx.getString(KAFKA_CONSUMER_PREFIX + ConsumerConfig.GROUP_ID_CONFIG); + if (groupId == null || groupId.isEmpty()) { + groupId = DEFAULT_GROUP_ID; + logger.info("Group ID was not specified. Using {} as the group id.", groupId); } + consumerProps.put(ConsumerConfig.KEY_DESERIALIZER_CLASS_CONFIG, DEFAULT_KEY_DESERIALIZER); + consumerProps.put(ConsumerConfig.VALUE_DESERIALIZER_CLASS_CONFIG, DEFAULT_VALUE_DESERIAIZER); + consumerProps.put(ConsumerConfig.AUTO_OFFSET_RESET_CONFIG, DEFAULT_AUTO_OFFSET_RESET); + //Defaults overridden based on config + consumerProps.putAll(ctx.getSubProperties(KAFKA_CONSUMER_PREFIX)); + //These always take precedence over config + consumerProps.put(ConsumerConfig.BOOTSTRAP_SERVERS_CONFIG, bootStrapServers); + consumerProps.put(ConsumerConfig.GROUP_ID_CONFIG, groupId); + consumerProps.put(ConsumerConfig.ENABLE_AUTO_COMMIT_CONFIG, false); + + logger.info(consumerProps.toString()); + } + protected Properties getConsumerProps() { return consumerProps; } + + + private synchronized ConsumerAndRecords createConsumerAndRecords() { + try { + KafkaConsumer consumer = new KafkaConsumer(consumerProps); + ConsumerAndRecords car = new ConsumerAndRecords(consumer, channelUUID); + logger.info("Created new consumer to connect to Kafka"); + car.consumer.subscribe(Arrays.asList(topic.get()), new ChannelRebalanceListener(rebalanceFlag)); + car.offsets = new HashMap(); + consumers.add(car); + return car; + } catch (Exception e) { + throw new FlumeException("Unable to connect to Kafka", e); + } } - private void decommissionConsumerAndIterator(ConsumerAndIterator c) { + private void decommissionConsumerAndRecords(ConsumerAndRecords c) { if (c.failedEvents.isEmpty()) { - c.consumer.commitOffsets(); + c.commitOffsets(); } c.failedEvents.clear(); - c.consumer.shutdown(); + c.consumer.close(); } - // Force a consumer to be initialized. There are many duplicates in - // tests due to rebalancing - making testing tricky. In production, - // this is less of an issue as - // rebalancing would happen only on startup. @VisibleForTesting void registerThread() { - consumerAndIter.get(); + try { + consumerAndRecords.get(); + } catch (Exception e) { + logger.error(e.getMessage()); + e.printStackTrace(); + } } private enum TransactionType { @@ -233,54 +294,41 @@ private enum TransactionType { private class KafkaTransaction extends BasicTransactionSemantics { private TransactionType type = TransactionType.NONE; - // For Puts private Optional tempOutStream = Optional - .absent(); - - // For put transactions, serialize the events and batch them and send it. - private Optional> serializedEvents = Optional.absent(); + .absent(); + // For put transactions, serialize the events and hold them until the commit goes is requested. + private Optional>> producerRecords = Optional.absent(); // For take transactions, deserialize and hold them till commit goes through private Optional> events = Optional.absent(); private Optional> writer = - Optional.absent(); + Optional.absent(); private Optional> reader = - Optional.absent(); + Optional.absent(); + private Optional>> kafkaFutures = + Optional.absent(); + private final String batchUUID = UUID.randomUUID().toString(); // Fine to use null for initial value, Avro will create new ones if this // is null private BinaryEncoder encoder = null; private BinaryDecoder decoder = null; - private final String batchUUID = UUID.randomUUID().toString(); private boolean eventTaken = false; + @Override + protected void doBegin() throws InterruptedException { + rebalanceFlag.set(false); + } + @Override protected void doPut(Event event) throws InterruptedException { type = TransactionType.PUT; - if (!serializedEvents.isPresent()) { - serializedEvents = Optional.of(new LinkedList()); + if (!producerRecords.isPresent()) { + producerRecords = Optional.of(new LinkedList>()); } - + String key = event.getHeaders().get(KEY_HEADER); try { - if (parseAsFlumeEvent) { - if (!tempOutStream.isPresent()) { - tempOutStream = Optional.of(new ByteArrayOutputStream()); - } - if (!writer.isPresent()) { - writer = Optional.of(new - SpecificDatumWriter(AvroFlumeEvent.class)); - } - tempOutStream.get().reset(); - AvroFlumeEvent e = new AvroFlumeEvent( - toCharSeqMap(event.getHeaders()), - ByteBuffer.wrap(event.getBody())); - encoder = EncoderFactory.get() - .directBinaryEncoder(tempOutStream.get(), encoder); - writer.get().write(e, encoder); - // Not really possible to avoid this copy :( - serializedEvents.get().add(tempOutStream.get().toByteArray()); - } else { - serializedEvents.get().add(event.getBody()); - } + producerRecords.get().add(new ProducerRecord + (topic.get(), key, serializeValue(event, parseAsFlumeEvent))); } catch (Exception e) { throw new ChannelException("Error while serializing event", e); } @@ -291,53 +339,64 @@ protected void doPut(Event event) throws InterruptedException { protected Event doTake() throws InterruptedException { type = TransactionType.TAKE; try { - if (!(consumerAndIter.get().uuid.equals(channelUUID))) { - LOGGER.info("UUID mismatch, creating new consumer"); - decommissionConsumerAndIterator(consumerAndIter.get()); - consumerAndIter.remove(); + if (!(consumerAndRecords.get().uuid.equals(channelUUID))) { + logger.info("UUID mismatch, creating new consumer"); + decommissionConsumerAndRecords(consumerAndRecords.get()); + consumerAndRecords.remove(); } } catch (Exception ex) { - LOGGER.warn("Error while shutting down consumer", ex); + logger.warn("Error while shutting down consumer", ex); } if (!events.isPresent()) { events = Optional.of(new LinkedList()); } Event e; - if (!consumerAndIter.get().failedEvents.isEmpty()) { - e = consumerAndIter.get().failedEvents.removeFirst(); + // Give the channel a chance to commit if there has been a rebalance + if (rebalanceFlag.get()) { + logger.debug("Returning null event after Consumer rebalance."); + return null; + } + if (!consumerAndRecords.get().failedEvents.isEmpty()) { + e = consumerAndRecords.get().failedEvents.removeFirst(); } else { + + if (logger.isDebugEnabled()) { + logger.debug("Assigment: {}", consumerAndRecords.get().consumer.assignment().toString()); + } + try { - ConsumerIterator it = consumerAndIter.get().iterator; long startTime = System.nanoTime(); - it.hasNext(); - long endTime = System.nanoTime(); - counter.addToKafkaEventGetTimer((endTime-startTime)/(1000*1000)); - if (parseAsFlumeEvent) { - ByteArrayInputStream in = - new ByteArrayInputStream(it.next().message()); - decoder = DecoderFactory.get().directBinaryDecoder(in, decoder); - if (!reader.isPresent()) { - reader = Optional.of( - new SpecificDatumReader(AvroFlumeEvent.class)); - } - AvroFlumeEvent event = reader.get().read(null, decoder); - e = EventBuilder.withBody(event.getBody().array(), - toStringMap(event.getHeaders())); - } else { - e = EventBuilder.withBody(it.next().message(), - Collections.EMPTY_MAP); + if (!consumerAndRecords.get().recordIterator.hasNext()) { + consumerAndRecords.get().poll(); } + if (consumerAndRecords.get().recordIterator.hasNext()) { + ConsumerRecord record = consumerAndRecords.get().recordIterator.next(); + e = deserializeValue(record.value(), parseAsFlumeEvent); + TopicPartition tp = new TopicPartition(record.topic(), record.partition()); + OffsetAndMetadata oam = new OffsetAndMetadata(record.offset() + 1, batchUUID); + consumerAndRecords.get().offsets.put(tp, oam); + + if (logger.isTraceEnabled()) { + logger.trace("Took offset: {}", consumerAndRecords.get().offsets.toString()); + } - } catch (ConsumerTimeoutException ex) { - if (LOGGER.isDebugEnabled()) { - LOGGER.debug("Timed out while waiting for data to come from Kafka", - ex); + //Add the key to the header + e.getHeaders().put(KEY_HEADER, record.key()); + + if (logger.isDebugEnabled()) { + logger.debug("Processed output from partition {} offset {}", record.partition(), record.offset()); + } + + long endTime = System.nanoTime(); + counter.addToKafkaEventGetTimer((endTime - startTime) / (1000 * 1000)); + } else { + return null; } - return null; } catch (Exception ex) { - LOGGER.warn("Error while getting events from Kafka", ex); + logger.warn("Error while getting events from Kafka. This is usually caused by trying to read " + + "a non-flume event. Ensure the setting for parseAsFlumeEvent is correct", ex); throw new ChannelException("Error while getting events from Kafka", - ex); + ex); } } eventTaken = true; @@ -351,32 +410,41 @@ protected void doCommit() throws InterruptedException { return; } if (type.equals(TransactionType.PUT)) { + if (!kafkaFutures.isPresent()) { + kafkaFutures = Optional.of(new LinkedList>()); + } try { - List> messages = new - ArrayList>(serializedEvents.get() - .size()); - for (byte[] event : serializedEvents.get()) { - messages.add(new KeyedMessage(topic.get(), null, - batchUUID, event)); - } + long batchSize = producerRecords.get().size(); long startTime = System.nanoTime(); - producer.send(messages); + int index = 0; + for (ProducerRecord record : producerRecords.get()) { + index++; + kafkaFutures.get().add(producer.send(record, new ChannelCallback(index, startTime))); + } + //prevents linger.ms from being a problem + producer.flush(); + + for (Future future : kafkaFutures.get()) { + future.get(); + } long endTime = System.nanoTime(); - counter.addToKafkaEventSendTimer((endTime-startTime)/(1000*1000)); - counter.addToEventPutSuccessCount(Long.valueOf(messages.size())); - serializedEvents.get().clear(); + counter.addToKafkaEventSendTimer((endTime - startTime) / (1000 * 1000)); + counter.addToEventPutSuccessCount(batchSize); + producerRecords.get().clear(); + kafkaFutures.get().clear(); } catch (Exception ex) { - LOGGER.warn("Sending events to Kafka failed", ex); + logger.warn("Sending events to Kafka failed", ex); throw new ChannelException("Commit failed as send to Kafka failed", - ex); + ex); } } else { - if (consumerAndIter.get().failedEvents.isEmpty() && eventTaken) { + if (consumerAndRecords.get().failedEvents.isEmpty() && eventTaken) { long startTime = System.nanoTime(); - consumerAndIter.get().consumer.commitOffsets(); + consumerAndRecords.get().commitOffsets(); long endTime = System.nanoTime(); - counter.addToKafkaCommitTimer((endTime-startTime)/(1000*1000)); - } + counter.addToKafkaCommitTimer((endTime - startTime) / (1000 * 1000)); + consumerAndRecords.get().printCurrentAssignment(); + } counter.addToEventTakeSuccessCount(Long.valueOf(events.get().size())); events.get().clear(); } @@ -388,37 +456,66 @@ protected void doRollback() throws InterruptedException { return; } if (type.equals(TransactionType.PUT)) { - serializedEvents.get().clear(); + producerRecords.get().clear(); + kafkaFutures.get().clear(); } else { counter.addToRollbackCounter(Long.valueOf(events.get().size())); - consumerAndIter.get().failedEvents.addAll(events.get()); + consumerAndRecords.get().failedEvents.addAll(events.get()); events.get().clear(); } } - } - - private class ConsumerAndIterator { - final ConsumerConnector consumer; - final ConsumerIterator iterator; - final String uuid; - final LinkedList failedEvents = new LinkedList(); + private byte[] serializeValue(Event event, boolean parseAsFlumeEvent) throws IOException { + byte[] bytes; + if (parseAsFlumeEvent) { + if (!tempOutStream.isPresent()) { + tempOutStream = Optional.of(new ByteArrayOutputStream()); + } + if (!writer.isPresent()) { + writer = Optional.of(new + SpecificDatumWriter(AvroFlumeEvent.class)); + } + tempOutStream.get().reset(); + AvroFlumeEvent e = new AvroFlumeEvent( + toCharSeqMap(event.getHeaders()), + ByteBuffer.wrap(event.getBody())); + encoder = EncoderFactory.get() + .directBinaryEncoder(tempOutStream.get(), encoder); + writer.get().write(e, encoder); + encoder.flush(); + bytes = tempOutStream.get().toByteArray(); + } else { + bytes = event.getBody(); + } + return bytes; + } - ConsumerAndIterator(ConsumerConnector consumerConnector, - ConsumerIterator iterator, String uuid) { - this.consumer = consumerConnector; - this.iterator = iterator; - this.uuid = uuid; + private Event deserializeValue(byte[] value, boolean parseAsFlumeEvent) throws IOException { + Event e; + if (parseAsFlumeEvent) { + ByteArrayInputStream in = + new ByteArrayInputStream(value); + decoder = DecoderFactory.get().directBinaryDecoder(in, decoder); + if (!reader.isPresent()) { + reader = Optional.of( + new SpecificDatumReader(AvroFlumeEvent.class)); + } + AvroFlumeEvent event = reader.get().read(null, decoder); + e = EventBuilder.withBody(event.getBody().array(), + toStringMap(event.getHeaders())); + } else { + e = EventBuilder.withBody(value, Collections.EMPTY_MAP); + } + return e; } } /** * Helper function to convert a map of String to a map of CharSequence. */ - private static Map toCharSeqMap( - Map stringMap) { + private static Map toCharSeqMap(Map stringMap) { Map charSeqMap = - new HashMap(); + new HashMap(); for (Map.Entry entry : stringMap.entrySet()) { charSeqMap.put(entry.getKey(), entry.getValue()); } @@ -428,13 +525,105 @@ private static Map toCharSeqMap( /** * Helper function to convert a map of CharSequence to a map of String. */ - private static Map toStringMap( - Map charSeqMap) { - Map stringMap = - new HashMap(); + private static Map toStringMap(Map charSeqMap) { + Map stringMap = new HashMap(); for (Map.Entry entry : charSeqMap.entrySet()) { stringMap.put(entry.getKey().toString(), entry.getValue().toString()); } return stringMap; } + + /* Object to store our consumer */ + private class ConsumerAndRecords { + final KafkaConsumer consumer; + final String uuid; + final LinkedList failedEvents = new LinkedList(); + + ConsumerRecords records; + Iterator> recordIterator; + Map offsets; + + ConsumerAndRecords(KafkaConsumer consumer, String uuid) { + this.consumer = consumer; + this.uuid = uuid; + this.records = ConsumerRecords.empty(); + this.recordIterator = records.iterator(); + } + + void poll() { + this.records = consumer.poll(pollTimeout); + this.recordIterator = records.iterator(); + logger.trace("polling"); + } + + void commitOffsets() { + this.consumer.commitSync(offsets); + } + + // This will reset the latest assigned partitions to the last committed offsets; + + public void printCurrentAssignment() { + StringBuilder sb = new StringBuilder(); + for (TopicPartition tp : this.consumer.assignment()) { + try { + sb.append("Committed: [").append(tp).append(",").append(this.consumer.committed(tp).offset()) + .append(",").append(this.consumer.committed(tp).metadata()).append("]"); + if (logger.isDebugEnabled()) { + logger.debug(sb.toString()); + } + } catch (NullPointerException npe) { + if (logger.isDebugEnabled()) { + logger.debug("Committed {}", tp); + } + } + } + } + } +} + +// Throw exception if there is an error +class ChannelCallback implements Callback { + private static final Logger log = LoggerFactory.getLogger(ChannelCallback.class); + private int index; + private long startTime; + + public ChannelCallback(int index, long startTime) { + this.index = index; + this.startTime = startTime; + } + + public void onCompletion(RecordMetadata metadata, Exception exception) { + if (exception != null) { + log.trace("Error sending message to Kafka due to " + exception.getMessage()); + } + if (log.isDebugEnabled()) { + long batchElapsedTime = System.currentTimeMillis() - startTime; + log.debug("Acked message_no " + index + ": " + metadata.topic() + "-" + metadata.partition() + "-" + + metadata.offset() + "-" + batchElapsedTime); + } + } +} + +class ChannelRebalanceListener implements ConsumerRebalanceListener { + private static final Logger log = LoggerFactory.getLogger(ChannelRebalanceListener.class); + private AtomicBoolean rebalanceFlag; + + public ChannelRebalanceListener(AtomicBoolean rebalanceFlag) { + this.rebalanceFlag = rebalanceFlag; + } + + // Set a flag that a rebalance has occurred. Then we can commit the currently written transactions + // on the next doTake() pass. + public void onPartitionsRevoked(Collection partitions) { + for (TopicPartition partition : partitions) { + log.info("topic {} - partition {} revoked.", partition.topic(), partition.partition()); + rebalanceFlag.set(true); + } + } + + public void onPartitionsAssigned(Collection partitions) { + for (TopicPartition partition : partitions) { + log.info("topic {} - partition {} assigned.", partition.topic(), partition.partition()); + } + } } diff --git a/flume-ng-channels/flume-kafka-channel/src/main/java/org/apache/flume/channel/kafka/KafkaChannelConfiguration.java b/flume-ng-channels/flume-kafka-channel/src/main/java/org/apache/flume/channel/kafka/KafkaChannelConfiguration.java index 9a342efb06..faf46b6232 100644 --- a/flume-ng-channels/flume-kafka-channel/src/main/java/org/apache/flume/channel/kafka/KafkaChannelConfiguration.java +++ b/flume-ng-channels/flume-kafka-channel/src/main/java/org/apache/flume/channel/kafka/KafkaChannelConfiguration.java @@ -18,27 +18,45 @@ */ package org.apache.flume.channel.kafka; +import org.apache.kafka.clients.CommonClientConfigs; + public class KafkaChannelConfiguration { public static final String KAFKA_PREFIX = "kafka."; + public static final String KAFKA_CONSUMER_PREFIX = KAFKA_PREFIX + "consumer."; + public static final String KAFKA_PRODUCER_PREFIX = KAFKA_PREFIX + "producer."; + public static final String DEFAULT_ACKS = "all"; + public static final String DEFAULT_KEY_SERIALIZER = "org.apache.kafka.common.serialization.StringSerializer"; + public static final String DEFAULT_VALUE_SERIAIZER = "org.apache.kafka.common.serialization.ByteArraySerializer"; + public static final String DEFAULT_KEY_DESERIALIZER = "org.apache.kafka.common.serialization.StringDeserializer"; + public static final String DEFAULT_VALUE_DESERIAIZER = "org.apache.kafka.common.serialization.ByteArrayDeserializer"; + public static final String TOPIC_CONFIG = KAFKA_PREFIX + "topic"; + public static final String BOOTSTRAP_SERVERS_CONFIG = KAFKA_PREFIX + CommonClientConfigs.BOOTSTRAP_SERVERS_CONFIG; + public static final String DEFAULT_TOPIC = "flume-channel"; + public static final String DEFAULT_GROUP_ID = "flume"; + public static final String POLL_TIMEOUT = KAFKA_PREFIX + "pollTimeout"; + public static final long DEFAULT_POLL_TIMEOUT = 500; + + public static final String KEY_HEADER = "key"; + + public static final String DEFAULT_AUTO_OFFSET_RESET = "earliest"; + + public static final String PARSE_AS_FLUME_EVENT = "parseAsFlumeEvent"; + public static final boolean DEFAULT_PARSE_AS_FLUME_EVENT = true; + + /*** Old Configuration Parameters ****/ public static final String BROKER_LIST_KEY = "metadata.broker.list"; public static final String REQUIRED_ACKS_KEY = "request.required.acks"; public static final String BROKER_LIST_FLUME_KEY = "brokerList"; - public static final String TOPIC = "topic"; - public static final String GROUP_ID = "group.id"; + //public static final String TOPIC = "topic"; public static final String GROUP_ID_FLUME = "groupId"; public static final String AUTO_COMMIT_ENABLED = "auto.commit.enable"; public static final String ZOOKEEPER_CONNECT = "zookeeper.connect"; public static final String ZOOKEEPER_CONNECT_FLUME_KEY = "zookeeperConnect"; - public static final String DEFAULT_GROUP_ID = "flume"; - public static final String DEFAULT_TOPIC = "flume-channel"; public static final String TIMEOUT = "timeout"; public static final String DEFAULT_TIMEOUT = "100"; public static final String CONSUMER_TIMEOUT = "consumer.timeout.ms"; - public static final String PARSE_AS_FLUME_EVENT = "parseAsFlumeEvent"; - public static final boolean DEFAULT_PARSE_AS_FLUME_EVENT = true; - public static final String READ_SMALLEST_OFFSET = "readSmallestOffset"; public static final boolean DEFAULT_READ_SMALLEST_OFFSET = false; } diff --git a/flume-ng-channels/flume-kafka-channel/src/test/java/org/apache/flume/channel/kafka/TestKafkaChannel.java b/flume-ng-channels/flume-kafka-channel/src/test/java/org/apache/flume/channel/kafka/TestKafkaChannel.java index 319e779abe..637428d120 100644 --- a/flume-ng-channels/flume-kafka-channel/src/test/java/org/apache/flume/channel/kafka/TestKafkaChannel.java +++ b/flume-ng-channels/flume-kafka-channel/src/test/java/org/apache/flume/channel/kafka/TestKafkaChannel.java @@ -19,13 +19,8 @@ package org.apache.flume.channel.kafka; import com.google.common.collect.Lists; -import com.google.common.collect.Sets; import kafka.admin.AdminUtils; -import kafka.javaapi.producer.Producer; -import kafka.producer.KeyedMessage; -import kafka.producer.ProducerConfig; -import kafka.utils.ZKStringSerializer$; -import org.I0Itec.zkclient.ZkClient; +import kafka.utils.ZkUtils; import org.apache.commons.lang.RandomStringUtils; import org.apache.flume.Context; import org.apache.flume.Event; @@ -33,17 +28,26 @@ import org.apache.flume.conf.Configurables; import org.apache.flume.event.EventBuilder; import org.apache.flume.sink.kafka.util.TestUtil; +import org.apache.kafka.clients.CommonClientConfigs; +import org.apache.kafka.clients.consumer.ConsumerConfig; +import org.apache.kafka.clients.producer.KafkaProducer; +import org.apache.kafka.clients.producer.ProducerRecord; import org.junit.*; - -import static org.apache.flume.channel.kafka.KafkaChannelConfiguration.*; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; import java.util.*; import java.util.concurrent.*; import java.util.concurrent.atomic.AtomicBoolean; import java.util.concurrent.atomic.AtomicInteger; +import static org.apache.flume.channel.kafka.KafkaChannelConfiguration.*; + public class TestKafkaChannel { + private final static Logger LOGGER = + LoggerFactory.getLogger(TestKafkaChannel.class); + private static TestUtil testUtil = TestUtil.getInstance(); private String topic = null; private final Set usedTopics = new HashSet(); @@ -78,11 +82,52 @@ public static void tearDown() { testUtil.tearDown(); } + //Make sure the props are picked up correctly. + @Test + public void testProps() throws Exception { + Context context = new Context(); + context.put("kafka.producer.some-parameter", "1"); + context.put("kafka.consumer.another-parameter", "1"); + context.put(BOOTSTRAP_SERVERS_CONFIG, testUtil.getKafkaServerUrl()); + context.put(TOPIC_CONFIG, topic); + + final KafkaChannel channel = new KafkaChannel(); + Configurables.configure(channel, context); + + Properties consumerProps = channel.getConsumerProps(); + Properties producerProps = channel.getProducerProps(); + + Assert.assertEquals(producerProps.getProperty("some-parameter"), "1"); + Assert.assertEquals(consumerProps.getProperty("another-parameter"), "1"); + } + + @Test + public void testOldConfig() throws Exception { + Context context = new Context(); + context.put(BROKER_LIST_FLUME_KEY,testUtil.getKafkaServerUrl()); + context.put(GROUP_ID_FLUME,"flume-something"); + context.put(READ_SMALLEST_OFFSET,"true"); + context.put("topic",topic); + + final KafkaChannel channel = new KafkaChannel(); + Configurables.configure(channel, context); + + Properties consumerProps = channel.getConsumerProps(); + Properties producerProps = channel.getProducerProps(); + + Assert.assertEquals(producerProps.getProperty(CommonClientConfigs.BOOTSTRAP_SERVERS_CONFIG),testUtil.getKafkaServerUrl()); + Assert.assertEquals(consumerProps.getProperty(ConsumerConfig.GROUP_ID_CONFIG), "flume-something"); + Assert.assertEquals(consumerProps.getProperty(ConsumerConfig.AUTO_OFFSET_RESET_CONFIG), "earliest"); + + } + + @Test public void testSuccess() throws Exception { doTestSuccessRollback(false, false); } + @Test public void testSuccessInterleave() throws Exception { doTestSuccessRollback(false, true); @@ -99,7 +144,7 @@ public void testRollbacksInterleave() throws Exception { } private void doTestSuccessRollback(final boolean rollback, - final boolean interleave) throws Exception { + final boolean interleave) throws Exception { final KafkaChannel channel = startChannel(true); writeAndVerify(rollback, channel, interleave); channel.stop(); @@ -122,82 +167,89 @@ public void testStopAndStartWithRollbackAndNoRetry() throws Exception { } @Test - public void testNoParsingAsFlumeAgent() throws Exception { + public void testParseAsFlumeEventFalse() throws Exception { + doParseAsFlumeEventFalse(false); + } + + @Test + public void testParseAsFlumeEventFalseCheckHeader() throws Exception { + doParseAsFlumeEventFalse(true); + } + + @Test + public void testParseAsFlumeEventFalseAsSource() throws Exception { + doParseAsFlumeEventFalseAsSource(false); + } + + @Test + public void testParseAsFlumeEventFalseAsSourceCheckHeader() throws Exception { + doParseAsFlumeEventFalseAsSource(true); + } + + private void doParseAsFlumeEventFalse(Boolean checkHeaders) throws Exception { final KafkaChannel channel = startChannel(false); - Producer producer = new Producer( - new ProducerConfig(channel.getKafkaConf())); - List> original = Lists.newArrayList(); + Properties props = channel.getProducerProps(); + KafkaProducer producer = new KafkaProducer(props); + for (int i = 0; i < 50; i++) { - KeyedMessage data = new KeyedMessage(topic, null, RandomStringUtils.randomAlphabetic(6), - String.valueOf(i).getBytes()); - original.add(data); + ProducerRecord data = new ProducerRecord(topic, String.valueOf(i) + "-header", String.valueOf(i).getBytes()); + producer.send(data).get(); } - producer.send(original); ExecutorCompletionService submitterSvc = new - ExecutorCompletionService(Executors.newCachedThreadPool()); + ExecutorCompletionService(Executors.newCachedThreadPool()); List events = pullEvents(channel, submitterSvc, - 50, false, false); + 50, false, false); wait(submitterSvc, 5); - Set finals = Sets.newHashSet(); + Map finals = new HashMap(); for (int i = 0; i < 50; i++) { - finals.add(Integer.parseInt(new String(events.get(i).getBody()))); + finals.put(Integer.parseInt(new String(events.get(i).getBody())), events.get(i).getHeaders().get(KEY_HEADER)); } for (int i = 0; i < 50; i++) { - Assert.assertTrue(finals.contains(i)); + Assert.assertTrue(finals.keySet().contains(i)); + if (checkHeaders) { + Assert.assertTrue(finals.containsValue(String.valueOf(i) + "-header")); + } finals.remove(i); } Assert.assertTrue(finals.isEmpty()); channel.stop(); } - @Test - public void testTimeoutConfig() throws Exception { - Context context = prepareDefaultContext(true); - KafkaChannel channel = new KafkaChannel(); - Configurables.configure(channel, context); - Assert.assertTrue(channel.getKafkaConf().get(CONSUMER_TIMEOUT) - .equals(DEFAULT_TIMEOUT)); - - String timeout = "1000"; - context.put("kafka."+CONSUMER_TIMEOUT, timeout); - channel = new KafkaChannel(); - Configurables.configure(channel, context); - Assert.assertTrue(channel.getKafkaConf().get(CONSUMER_TIMEOUT) - .equals(timeout)); - } - /** * Like the previous test but here we write to the channel like a Flume source would do * to verify that the events are written as text and not as an Avro object * * @throws Exception */ - @Test - public void testWritingToNoParsingAsFlumeAgent() throws Exception { + public void doParseAsFlumeEventFalseAsSource(Boolean checkHeaders) throws Exception { final KafkaChannel channel = startChannel(false); List msgs = new ArrayList(); - for (int i = 0; i < 50; i++){ + Map headers = new HashMap(); + for (int i = 0; i < 50; i++) { msgs.add(String.valueOf(i)); } Transaction tx = channel.getTransaction(); tx.begin(); - for (int i = 0; i < msgs.size(); i++){ - channel.put(EventBuilder.withBody(msgs.get(i).getBytes())); + for (int i = 0; i < msgs.size(); i++) { + headers.put(KEY_HEADER, String.valueOf(i) + "-header"); + channel.put(EventBuilder.withBody(msgs.get(i).getBytes(), headers)); } tx.commit(); ExecutorCompletionService submitterSvc = new ExecutorCompletionService(Executors.newCachedThreadPool()); List events = pullEvents(channel, submitterSvc, - 50, false, false); + 50, false, false); wait(submitterSvc, 5); - Set finals = Sets.newHashSet(); + Map finals = new HashMap(); for (int i = 0; i < 50; i++) { - finals.add(Integer.parseInt(new String(events.get(i).getBody()))); + finals.put(Integer.parseInt(new String(events.get(i).getBody())), events.get(i).getHeaders().get(KEY_HEADER)); } for (int i = 0; i < 50; i++) { - Assert.assertTrue(finals.contains(i)); + Assert.assertTrue(finals.keySet().contains(i)); + if (checkHeaders) { + Assert.assertTrue(finals.containsValue(String.valueOf(i) + "-header")); + } finals.remove(i); } Assert.assertTrue(finals.isEmpty()); @@ -216,12 +268,12 @@ public void testWritingToNoParsingAsFlumeAgent() throws Exception { * @throws Exception */ private void doTestStopAndStart(boolean rollback, - boolean retryAfterRollback) throws Exception { + boolean retryAfterRollback) throws Exception { final KafkaChannel channel = startChannel(true); ExecutorService underlying = Executors - .newCachedThreadPool(); + .newCachedThreadPool(); ExecutorCompletionService submitterSvc = - new ExecutorCompletionService(underlying); + new ExecutorCompletionService(underlying); final List> events = createBaseList(); putEvents(channel, events, submitterSvc); int completed = 0; @@ -233,14 +285,14 @@ private void doTestStopAndStart(boolean rollback, total = 40; } final List eventsPulled = - pullEvents(channel2, submitterSvc, total, rollback, retryAfterRollback); + pullEvents(channel2, submitterSvc, total, rollback, retryAfterRollback); wait(submitterSvc, 5); channel2.stop(); if (!retryAfterRollback && rollback) { final KafkaChannel channel3 = startChannel(true); int expectedRemaining = 50 - eventsPulled.size(); final List eventsPulled2 = - pullEvents(channel3, submitterSvc, expectedRemaining, false, false); + pullEvents(channel3, submitterSvc, expectedRemaining, false, false); wait(submitterSvc, 5); Assert.assertEquals(expectedRemaining, eventsPulled2.size()); eventsPulled.addAll(eventsPulled2); @@ -259,18 +311,18 @@ private KafkaChannel startChannel(boolean parseAsFlume) throws Exception { } private void writeAndVerify(final boolean testRollbacks, - final KafkaChannel channel) throws Exception { + final KafkaChannel channel) throws Exception { writeAndVerify(testRollbacks, channel, false); } private void writeAndVerify(final boolean testRollbacks, - final KafkaChannel channel, final boolean interleave) throws Exception { + final KafkaChannel channel, final boolean interleave) throws Exception { final List> events = createBaseList(); ExecutorCompletionService submitterSvc = - new ExecutorCompletionService(Executors - .newCachedThreadPool()); + new ExecutorCompletionService(Executors + .newCachedThreadPool()); putEvents(channel, events, submitterSvc); @@ -279,11 +331,11 @@ private void writeAndVerify(final boolean testRollbacks, } ExecutorCompletionService submitterSvc2 = - new ExecutorCompletionService(Executors - .newCachedThreadPool()); + new ExecutorCompletionService(Executors + .newCachedThreadPool()); final List eventsPulled = - pullEvents(channel, submitterSvc2, 50, testRollbacks, true); + pullEvents(channel, submitterSvc2, 50, testRollbacks, true); if (!interleave) { wait(submitterSvc, 5); @@ -301,7 +353,7 @@ private List> createBaseList() { for (int j = 0; j < 10; j++) { Map hdrs = new HashMap(); String v = (String.valueOf(i) + " - " + String - .valueOf(j)); + .valueOf(j)); hdrs.put("header", v); eventList.add(EventBuilder.withBody(v.getBytes(), hdrs)); } @@ -310,7 +362,7 @@ private List> createBaseList() { } private void putEvents(final KafkaChannel channel, final List> - events, ExecutorCompletionService submitterSvc) { + events, ExecutorCompletionService submitterSvc) { for (int i = 0; i < 5; i++) { final int index = i; submitterSvc.submit(new Callable() { @@ -334,10 +386,10 @@ public Void call() { } private List pullEvents(final KafkaChannel channel, - ExecutorCompletionService submitterSvc, final int total, - final boolean testRollbacks, final boolean retryAfterRollback) { + ExecutorCompletionService submitterSvc, final int total, + final boolean testRollbacks, final boolean retryAfterRollback) { final List eventsPulled = Collections.synchronizedList(new - ArrayList(50)); + ArrayList(50)); final CyclicBarrier barrier = new CyclicBarrier(5); final AtomicInteger counter = new AtomicInteger(0); final AtomicInteger rolledBackCount = new AtomicInteger(0); @@ -366,9 +418,9 @@ public Void call() throws Exception { eventsLocal.add(e); } else { if (testRollbacks && - index == 4 && - (!rolledBack.get()) && - startedGettingEvents.get()) { + index == 4 && + (!rolledBack.get()) && + startedGettingEvents.get()) { tx.rollback(); tx.close(); tx = null; @@ -407,7 +459,7 @@ public Void call() throws Exception { } private void wait(ExecutorCompletionService submitterSvc, int max) - throws Exception { + throws Exception { int completed = 0; while (completed < max) { submitterSvc.take(); @@ -420,8 +472,7 @@ private void verify(List eventsPulled) { Assert.assertEquals(50, eventsPulled.size()); Set eventStrings = new HashSet(); for (Event e : eventsPulled) { - Assert - .assertEquals(e.getHeaders().get("header"), new String(e.getBody())); + Assert.assertEquals(e.getHeaders().get("header"), new String(e.getBody())); eventStrings.add(e.getHeaders().get("header")); } for (int i = 0; i < 5; i++) { @@ -437,14 +488,10 @@ private void verify(List eventsPulled) { private Context prepareDefaultContext(boolean parseAsFlume) { // Prepares a default context with Kafka Server Properties Context context = new Context(); - context.put(KafkaChannelConfiguration.BROKER_LIST_FLUME_KEY, - testUtil.getKafkaServerUrl()); - context.put(KafkaChannelConfiguration.ZOOKEEPER_CONNECT_FLUME_KEY, - testUtil.getZkUrl()); - context.put(KafkaChannelConfiguration.PARSE_AS_FLUME_EVENT, - String.valueOf(parseAsFlume)); - context.put(KafkaChannelConfiguration.READ_SMALLEST_OFFSET, "true"); - context.put(KafkaChannelConfiguration.TOPIC, topic); + context.put(BOOTSTRAP_SERVERS_CONFIG, testUtil.getKafkaServerUrl()); + context.put(PARSE_AS_FLUME_EVENT, String.valueOf(parseAsFlume)); + context.put(TOPIC_CONFIG, topic); + return context; } @@ -452,22 +499,18 @@ public static void createTopic(String topicName) { int numPartitions = 5; int sessionTimeoutMs = 10000; int connectionTimeoutMs = 10000; - ZkClient zkClient = new ZkClient(testUtil.getZkUrl(), - sessionTimeoutMs, connectionTimeoutMs, - ZKStringSerializer$.MODULE$); + ZkUtils zkUtils = ZkUtils.apply(testUtil.getZkUrl(), sessionTimeoutMs, connectionTimeoutMs, false); int replicationFactor = 1; Properties topicConfig = new Properties(); - AdminUtils.createTopic(zkClient, topicName, numPartitions, - replicationFactor, topicConfig); + AdminUtils.createTopic(zkUtils, topicName, numPartitions, + replicationFactor, topicConfig); } public static void deleteTopic(String topicName) { int sessionTimeoutMs = 10000; int connectionTimeoutMs = 10000; - ZkClient zkClient = new ZkClient(testUtil.getZkUrl(), - sessionTimeoutMs, connectionTimeoutMs, - ZKStringSerializer$.MODULE$); - AdminUtils.deleteTopic(zkClient, topicName); + ZkUtils zkUtils = ZkUtils.apply(testUtil.getZkUrl(), sessionTimeoutMs, connectionTimeoutMs, false); + AdminUtils.deleteTopic(zkUtils, topicName); } } diff --git a/flume-ng-channels/flume-kafka-channel/src/test/resources/kafka-server.properties b/flume-ng-channels/flume-kafka-channel/src/test/resources/kafka-server.properties index c10c89de41..216bfd8813 100644 --- a/flume-ng-channels/flume-kafka-channel/src/test/resources/kafka-server.properties +++ b/flume-ng-channels/flume-kafka-channel/src/test/resources/kafka-server.properties @@ -38,7 +38,7 @@ port=9092 #advertised.port= # The number of threads handling network requests -num.network.threads=2 +num.network.threads=4 # The number of threads doing disk I/O num.io.threads=8 diff --git a/flume-ng-doc/sphinx/FlumeUserGuide.rst b/flume-ng-doc/sphinx/FlumeUserGuide.rst index 15f27c3b5b..5149ab5ab9 100644 --- a/flume-ng-doc/sphinx/FlumeUserGuide.rst +++ b/flume-ng-doc/sphinx/FlumeUserGuide.rst @@ -2679,36 +2679,60 @@ The Kafka channel can be used for multiple scenarios: * With Flume source and interceptor but no sink - it allows writing Flume events into a Kafka topic, for use by other apps * With Flume sink, but no source - it is a low-latency, fault tolerant way to send events from Kafka to Flume sources such as HDFS, HBase or Solr + +This version of Flume requires Kafka version 0.9 or greater due to the reliance on the Kafka clients shipped with that version. The configuration of +the channel has changed compared to previous flume versions. + +The configuration parameters are organized as such: +1) Configuration values related to the channel generically are applied at the channel config level, eg: a1.channel.k1.type = +2) Configuration values related to Kafka or how the Channel operates are prefixed with "kafka.", (this are analgous to CommonClient Configs)eg: a1.channels.k1.kafka.topica1.channels.k1.kafka.bootstrap.serversThis is not dissimilar to how the hdfs sink operates +3) Properties specific to the producer/consumer are prefixed by kafka.producer or kafka.consumer +4) Where possible, the Kafka paramter names are used, eg: bootstrap.servers and acks + +This version of flume is backwards-compatible with previous versions, however deprecated properties are indicated in the table below and a warning message +is logged on startup when they are present in the configuration file. + Required properties are in **bold**. -====================== ========================== =============================================================================================================== -Property Name Default Description -====================== ========================== =============================================================================================================== -**type** -- The component type name, needs to be ``org.apache.flume.channel.kafka.KafkaChannel`` -**brokerList** -- List of brokers in the Kafka cluster used by the channel - This can be a partial list of brokers, but we recommend at least two for HA. - The format is comma separated list of hostname:port -**zookeeperConnect** -- URI of ZooKeeper used by Kafka cluster - The format is comma separated list of hostname:port. If chroot is used, it is added once at the end. - For example: zookeeper-1:2181,zookeeper-2:2182,zookeeper-3:2181/kafka -topic flume-channel Kafka topic which the channel will use -groupId flume Consumer group ID the channel uses to register with Kafka. - Multiple channels must use the same topic and group to ensure that when one agent fails another can get the data - Note that having non-channel consumers with the same ID can lead to data loss. -parseAsFlumeEvent true Expecting Avro datums with FlumeEvent schema in the channel. - This should be true if Flume source is writing to the channel - And false if other producers are writing into the topic that the channel is using - Flume source messages to Kafka can be parsed outside of Flume by using - org.apache.flume.source.avro.AvroFlumeEvent provided by the flume-ng-sdk artifact -readSmallestOffset false When set to true, the channel will read all data in the topic, starting from the oldest event - when false, it will read only events written after the channel started - When "parseAsFlumeEvent" is true, this will be false. Flume source will start prior to the sinks and this - guarantees that events sent by source before sinks start will not be lost. -Other Kafka Properties -- These properties are used to configure the Kafka Producer and Consumer used by the channel. - Any property supported by Kafka can be used. - The only requirement is to prepend the property name with the prefix ``kafka.``. - For example: kafka.producer.type -====================== ========================== =============================================================================================================== +================================ ========================== =============================================================================================================== +Property Name Default Description +================================ ========================== =============================================================================================================== +**type** -- The component type name, needs to be ``org.apache.flume.channel.kafka.KafkaChannel`` +**kafka.bootstrap.servers** -- List of brokers in the Kafka cluster used by the channel + This can be a partial list of brokers, but we recommend at least two for HA. + The format is comma separated list of hostname:port +kafka.topic flume-channel Kafka topic which the channel will use +kafka.consumer.group.id flume Consumer group ID the channel uses to register with Kafka. + Multiple channels must use the same topic and group to ensure that when one agent fails another can get the data + Note that having non-channel consumers with the same ID can lead to data loss. + +parseAsFlumeEvent true Expecting Avro datums with FlumeEvent schema in the channel. + This should be true if Flume source is writing to the channel and false if other producers are + writing into the topic that the channel is using. Flume source messages to Kafka can be parsed outside of Flume by using + org.apache.flume.source.avro.AvroFlumeEvent provided by the flume-ng-sdk artifact +pollTimeout 500 The amount of time(in milliseconds) to wait in the "poll()" call of the conumer. + https://kafka.apache.org/090/javadoc/org/apache/kafka/clients/consumer/KafkaConsumer.html#poll(long) +kafka.consumer.auto.offset.reset latest What to do when there is no initial offset in Kafka or if the current offset does not exist any more on the server + (e.g. because that data has been deleted): + earliest: automatically reset the offset to the earliest offset + latest: automatically reset the offset to the latest offset + none: throw exception to the consumer if no previous offset is found for the consumer\'s group + anything else: throw exception to the consumer. +================================ ========================== =============================================================================================================== + +Deprecated Properties + +================================ ========================== =============================================================================================================== +Property Name Default Description +================================ ========================== =============================================================================================================== +brokerList -- List of brokers in the Kafka cluster used by the channel + This can be a partial list of brokers, but we recommend at least two for HA. + The format is comma separated list of hostname:port +topic flume-channel Use kafka.topic +groupId flume Use kafka.consumer.group.id +readSmallestOffset false Use kafka.consumer.auto.offset.reset + +================================ ========================== =============================================================================================================== .. note:: Due to the way the channel is load balanced, there may be duplicate events when the agent first starts up @@ -2716,12 +2740,12 @@ Example for agent named a1: .. code-block:: properties - a1.channels.channel1.type = org.apache.flume.channel.kafka.KafkaChannel + a1.channels.channel1.type = org.apache.flume.channel.kafka.KafkaChannel a1.channels.channel1.capacity = 10000 a1.channels.channel1.transactionCapacity = 1000 - a1.channels.channel1.brokerList=kafka-2:9092,kafka-3:9092 - a1.channels.channel1.topic=channel1 - a1.channels.channel1.zookeeperConnect=kafka-1:2181 + a1.channels.channel1.kafka.bootstrap.servers = kafka-1:9092,kafka-2:9092,kafka-3:9092 + a1.channels.channel1.kafka.topic = channel1 + a1.channels.channel1.kafka.consumer.group.id = flume-consumer File Channel ~~~~~~~~~~~~ From 4eb2a3bb510671fe92cec0ebb61d7e78adc8f526 Mon Sep 17 00:00:00 2001 From: Jarek Jarcec Cecho Date: Thu, 21 Apr 2016 13:37:26 -0700 Subject: [PATCH 279/341] FLUME-2852: Kafka Source/Sink should optionally read/write Flume records (Tristan Stevens via Jarek Jarcec Cecho) --- flume-ng-doc/sphinx/FlumeUserGuide.rst | 9 ++ .../apache/flume/sink/kafka/KafkaSink.java | 66 ++++++++++++++- .../flume/sink/kafka/KafkaSinkConstants.java | 4 +- .../flume/sink/kafka/TestConstants.java | 2 + .../flume/sink/kafka/TestKafkaSink.java | 68 +++++++++++++++ .../flume/source/kafka/KafkaSource.java | 77 +++++++++++++++-- .../source/kafka/KafkaSourceConstants.java | 3 + .../kafka/KafkaSourceEmbeddedKafka.java | 19 +++-- .../flume/source/kafka/TestKafkaSource.java | 83 +++++++++++++++++++ 9 files changed, 316 insertions(+), 15 deletions(-) diff --git a/flume-ng-doc/sphinx/FlumeUserGuide.rst b/flume-ng-doc/sphinx/FlumeUserGuide.rst index 5149ab5ab9..9c11fe6235 100644 --- a/flume-ng-doc/sphinx/FlumeUserGuide.rst +++ b/flume-ng-doc/sphinx/FlumeUserGuide.rst @@ -1215,6 +1215,11 @@ backoffSleepIncrement 1000 Initial and incremental wait time maxBackoffSleep 5000 Maximum wait time that is triggered when a Kafka Topic appears to be empty. Five seconds is ideal for ingestion use cases but a lower value may be required for low latency operations with interceptors. +useFlumeEventFormat false By default events are taken as bytes from the Kafka topic directly into the event body. Set to + true to read events as the Flume Avro binary format. Used in conjunction with the same property + on the KafkaSink or with the parseAsFlumeEvent property on the Kafka Channel this will preserve + any Flume headers sent on the producing side. + Other Kafka Consumer Properties -- These properties are used to configure the Kafka Consumer. Any producer property supported by Kafka can be used. The only requirement is to prepend the property name with the prefix ``kafka.consumer``. For example: kafka.consumer.auto.offset.reset @@ -2515,6 +2520,10 @@ flumeBatchSize 100 How many messages to proce kafka.producer.acks 1 How many replicas must acknowledge a message before its considered successfully written. Accepted values are 0 (Never wait for acknowledgement), 1 (wait for leader only), -1 (wait for all replicas) Set this to -1 to avoid data loss in some cases of leader failure. +useFlumeEventFormat false By default events are put as bytes onto the Kafka topic directly from the event body. Set to + true to store events as the Flume Avro binary format. Used in conjunction with the same property + on the KafkaSource or with the parseAsFlumeEvent property on the Kafka Channel this will preserve + any Flume headers for the producing side. Other Kafka Producer Properties -- These properties are used to configure the Kafka Producer. Any producer property supported by Kafka can be used. The only requirement is to prepend the property name with the prefix ``kafka.producer``. diff --git a/flume-ng-sinks/flume-ng-kafka-sink/src/main/java/org/apache/flume/sink/kafka/KafkaSink.java b/flume-ng-sinks/flume-ng-kafka-sink/src/main/java/org/apache/flume/sink/kafka/KafkaSink.java index 2e2140e33b..7bef7f3bcb 100644 --- a/flume-ng-sinks/flume-ng-kafka-sink/src/main/java/org/apache/flume/sink/kafka/KafkaSink.java +++ b/flume-ng-sinks/flume-ng-kafka-sink/src/main/java/org/apache/flume/sink/kafka/KafkaSink.java @@ -18,7 +18,13 @@ Licensed to the Apache Software Foundation (ASF) under one or more package org.apache.flume.sink.kafka; +import com.google.common.base.Optional; import com.google.common.base.Throwables; + +import org.apache.avro.io.BinaryEncoder; +import org.apache.avro.io.EncoderFactory; +import org.apache.avro.specific.SpecificDatumReader; +import org.apache.avro.specific.SpecificDatumWriter; import org.apache.flume.Channel; import org.apache.flume.Context; import org.apache.flume.Event; @@ -28,6 +34,7 @@ Licensed to the Apache Software Foundation (ASF) under one or more import org.apache.flume.conf.ConfigurationException; import org.apache.flume.instrumentation.kafka.KafkaSinkCounter; import org.apache.flume.sink.AbstractSink; +import org.apache.flume.source.avro.AvroFlumeEvent; import org.apache.kafka.clients.producer.Callback; import org.apache.kafka.clients.producer.KafkaProducer; import org.apache.kafka.clients.producer.ProducerConfig; @@ -36,6 +43,10 @@ Licensed to the Apache Software Foundation (ASF) under one or more import org.slf4j.Logger; import org.slf4j.LoggerFactory; +import java.io.ByteArrayOutputStream; +import java.io.IOException; +import java.nio.ByteBuffer; +import java.util.HashMap; import java.util.LinkedList; import java.util.List; import java.util.Map; @@ -85,6 +96,7 @@ Licensed to the Apache Software Foundation (ASF) under one or more * improve throughput while adding latency. * requiredAcks -- 0 (unsafe), 1 (accepted by at least one broker, default), * -1 (accepted by all brokers) + * useFlumeEventFormat - preserves event headers when serializing onto Kafka *

    * header properties (per event): * topic @@ -101,6 +113,17 @@ public class KafkaSink extends AbstractSink implements Configurable { private int batchSize; private List> kafkaFutures; private KafkaSinkCounter counter; + private boolean useAvroEventFormat; + private Optional> writer = + Optional.absent(); + private Optional> reader = + Optional.absent(); + private Optional tempOutStream = Optional + .absent(); + + //Fine to use null for initial value, Avro will create new ones if this + // is null + private BinaryEncoder encoder = null; //For testing @@ -160,8 +183,13 @@ public Status process() throws EventDeliveryException { // create a message and add to buffer long startTime = System.currentTimeMillis(); - kafkaFutures.add(producer.send(new ProducerRecord (eventTopic, eventKey, eventBody), + + try { + kafkaFutures.add(producer.send(new ProducerRecord (eventTopic, eventKey, serializeEvent(event, useAvroEventFormat)), new SinkCallback(startTime))); + } catch (IOException ex) { + throw new EventDeliveryException("Could not serialize event", ex); + } } //Prevent linger.ms from holding the batch @@ -255,6 +283,12 @@ public void configure(Context context) { logger.debug("Using batch size: {}", batchSize); } + useAvroEventFormat = context.getBoolean(KafkaSinkConstants.AVRO_EVENT, KafkaSinkConstants.DEFAULT_AVRO_EVENT); + + if (logger.isDebugEnabled()) { + logger.debug(KafkaSinkConstants.AVRO_EVENT + " set to: {}", useAvroEventFormat); + } + kafkaFutures = new LinkedList>(); String bootStrapServers = context.getString(BOOTSTRAP_SERVERS_CONFIG); @@ -342,6 +376,36 @@ private void setProducerProps(Context context, String bootStrapServers) { protected Properties getKafkaProps() { return kafkaProps; } + + private byte[] serializeEvent(Event event, boolean useAvroEventFormat) throws IOException { + byte[] bytes; + if (useAvroEventFormat) { + if (!tempOutStream.isPresent()) { + tempOutStream = Optional.of(new ByteArrayOutputStream()); + } + if (!writer.isPresent()) { + writer = Optional.of(new SpecificDatumWriter(AvroFlumeEvent.class)); + } + tempOutStream.get().reset(); + AvroFlumeEvent e = new AvroFlumeEvent(toCharSeqMap(event.getHeaders()), ByteBuffer.wrap(event.getBody())); + encoder = EncoderFactory.get().directBinaryEncoder(tempOutStream.get(), encoder); + writer.get().write(e, encoder); + encoder.flush(); + bytes = tempOutStream.get().toByteArray(); + } else { + bytes = event.getBody(); + } + return bytes; + } + + private static Map toCharSeqMap(Map stringMap) { + Map charSeqMap = new HashMap(); + for (Map.Entry entry : stringMap.entrySet()) { + charSeqMap.put(entry.getKey(), entry.getValue()); + } + return charSeqMap; + } + } class SinkCallback implements Callback { diff --git a/flume-ng-sinks/flume-ng-kafka-sink/src/main/java/org/apache/flume/sink/kafka/KafkaSinkConstants.java b/flume-ng-sinks/flume-ng-kafka-sink/src/main/java/org/apache/flume/sink/kafka/KafkaSinkConstants.java index c84dec0d85..6b64bc14e2 100644 --- a/flume-ng-sinks/flume-ng-kafka-sink/src/main/java/org/apache/flume/sink/kafka/KafkaSinkConstants.java +++ b/flume-ng-sinks/flume-ng-kafka-sink/src/main/java/org/apache/flume/sink/kafka/KafkaSinkConstants.java @@ -34,6 +34,9 @@ public class KafkaSinkConstants { public static final String KEY_HEADER = "key"; public static final String TOPIC_HEADER = "topic"; + public static final String AVRO_EVENT = "useFlumeEventFormat"; + public static final boolean DEFAULT_AVRO_EVENT = false; + public static final String DEFAULT_KEY_SERIALIZER = "org.apache.kafka.common.serialization.StringSerializer"; public static final String DEFAULT_VALUE_SERIAIZER = "org.apache.kafka.common.serialization.ByteArraySerializer"; @@ -42,7 +45,6 @@ public class KafkaSinkConstants { public static final String DEFAULT_ACKS = "1"; - /* Old Properties */ /* Properties */ diff --git a/flume-ng-sinks/flume-ng-kafka-sink/src/test/java/org/apache/flume/sink/kafka/TestConstants.java b/flume-ng-sinks/flume-ng-kafka-sink/src/test/java/org/apache/flume/sink/kafka/TestConstants.java index f99be539e3..6d8570051d 100644 --- a/flume-ng-sinks/flume-ng-kafka-sink/src/test/java/org/apache/flume/sink/kafka/TestConstants.java +++ b/flume-ng-sinks/flume-ng-kafka-sink/src/test/java/org/apache/flume/sink/kafka/TestConstants.java @@ -22,4 +22,6 @@ public class TestConstants { public static final String STATIC_TOPIC = "static-topic"; public static final String CUSTOM_KEY = "custom-key"; public static final String CUSTOM_TOPIC = "custom-topic"; + public static final String HEADER_1_VALUE = "test-avro-header"; + public static final String HEADER_1_KEY = "header1"; } diff --git a/flume-ng-sinks/flume-ng-kafka-sink/src/test/java/org/apache/flume/sink/kafka/TestKafkaSink.java b/flume-ng-sinks/flume-ng-kafka-sink/src/test/java/org/apache/flume/sink/kafka/TestKafkaSink.java index 1852099965..f577e98d9f 100644 --- a/flume-ng-sinks/flume-ng-kafka-sink/src/test/java/org/apache/flume/sink/kafka/TestKafkaSink.java +++ b/flume-ng-sinks/flume-ng-kafka-sink/src/test/java/org/apache/flume/sink/kafka/TestKafkaSink.java @@ -19,6 +19,11 @@ Licensed to the Apache Software Foundation (ASF) under one or more package org.apache.flume.sink.kafka; import kafka.message.MessageAndMetadata; + +import org.apache.avro.io.BinaryDecoder; +import org.apache.avro.io.DecoderFactory; +import org.apache.avro.specific.SpecificDatumReader; +import org.apache.avro.util.Utf8; import org.apache.flume.Channel; import org.apache.flume.Context; import org.apache.flume.Event; @@ -29,12 +34,17 @@ Licensed to the Apache Software Foundation (ASF) under one or more import org.apache.flume.conf.Configurables; import org.apache.flume.event.EventBuilder; import org.apache.flume.sink.kafka.util.TestUtil; +import org.apache.flume.source.avro.AvroFlumeEvent; import org.apache.kafka.clients.CommonClientConfigs; import org.apache.kafka.clients.producer.ProducerConfig; import org.junit.AfterClass; import org.junit.BeforeClass; import org.junit.Test; +import com.google.common.base.Charsets; + +import java.io.ByteArrayInputStream; +import java.io.IOException; import java.io.UnsupportedEncodingException; import java.util.ArrayList; import java.util.HashMap; @@ -208,6 +218,64 @@ public void testTopicAndKeyFromHeader() throws UnsupportedEncodingException { } + @SuppressWarnings("rawtypes") + @Test + public void testAvroEvent() throws IOException { + + + Sink kafkaSink = new KafkaSink(); + Context context = prepareDefaultContext(); + context.put(AVRO_EVENT, "true"); + Configurables.configure(kafkaSink, context); + Channel memoryChannel = new MemoryChannel(); + Configurables.configure(memoryChannel, context); + kafkaSink.setChannel(memoryChannel); + kafkaSink.start(); + + String msg = "test-avro-event"; + + Map headers = new HashMap(); + headers.put("topic", TestConstants.CUSTOM_TOPIC); + headers.put("key", TestConstants.CUSTOM_KEY); + headers.put(TestConstants.HEADER_1_KEY, TestConstants.HEADER_1_VALUE); + Transaction tx = memoryChannel.getTransaction(); + tx.begin(); + Event event = EventBuilder.withBody(msg.getBytes(), headers); + memoryChannel.put(event); + tx.commit(); + tx.close(); + + try { + Sink.Status status = kafkaSink.process(); + if (status == Sink.Status.BACKOFF) { + fail("Error Occurred"); + } + } catch (EventDeliveryException ex) { + // ignore + } + + MessageAndMetadata fetchedMsg = + testUtil.getNextMessageFromConsumer(TestConstants.CUSTOM_TOPIC); + + ByteArrayInputStream in = + new ByteArrayInputStream((byte[])fetchedMsg.message()); + BinaryDecoder decoder = DecoderFactory.get().directBinaryDecoder(in, null); + SpecificDatumReader reader = new SpecificDatumReader(AvroFlumeEvent.class); + + AvroFlumeEvent avroevent = reader.read(null, decoder); + + String eventBody = new String(avroevent.getBody().array(), Charsets.UTF_8); + Map eventHeaders = avroevent.getHeaders(); + + assertEquals(msg, eventBody); + assertEquals(TestConstants.CUSTOM_KEY, + new String((byte[]) fetchedMsg.key(), "UTF-8")); + + assertEquals(TestConstants.HEADER_1_VALUE, eventHeaders.get(new Utf8(TestConstants.HEADER_1_KEY)).toString()); + assertEquals(TestConstants.CUSTOM_KEY, eventHeaders.get(new Utf8("key")).toString()); + + } + @Test public void testEmptyChannel() throws UnsupportedEncodingException, EventDeliveryException { diff --git a/flume-ng-sources/flume-kafka-source/src/main/java/org/apache/flume/source/kafka/KafkaSource.java b/flume-ng-sources/flume-kafka-source/src/main/java/org/apache/flume/source/kafka/KafkaSource.java index db806ccfd9..84fef524e4 100644 --- a/flume-ng-sources/flume-kafka-source/src/main/java/org/apache/flume/source/kafka/KafkaSource.java +++ b/flume-ng-sources/flume-kafka-source/src/main/java/org/apache/flume/source/kafka/KafkaSource.java @@ -16,6 +16,7 @@ */ package org.apache.flume.source.kafka; +import java.io.ByteArrayInputStream; import java.util.ArrayList; import java.util.Arrays; import java.util.Collection; @@ -28,6 +29,9 @@ import java.util.concurrent.atomic.AtomicBoolean; import java.util.regex.Pattern; +import org.apache.avro.io.BinaryDecoder; +import org.apache.avro.io.DecoderFactory; +import org.apache.avro.specific.SpecificDatumReader; import org.apache.flume.Context; import org.apache.flume.Event; import org.apache.flume.EventDeliveryException; @@ -37,7 +41,7 @@ import org.apache.flume.event.EventBuilder; import org.apache.flume.instrumentation.kafka.KafkaSourceCounter; import org.apache.flume.source.AbstractPollableSource; - +import org.apache.flume.source.avro.AvroFlumeEvent; import org.apache.kafka.clients.consumer.ConsumerConfig; import org.apache.kafka.clients.consumer.ConsumerRebalanceListener; import org.apache.kafka.clients.consumer.ConsumerRecord; @@ -48,6 +52,8 @@ import org.slf4j.Logger; import org.slf4j.LoggerFactory; +import com.google.common.base.Optional; + /** * A Source for Kafka which reads messages from kafka topics. * @@ -69,6 +75,8 @@ *

    * kafka.consumer.*: Any property starting with "kafka.consumer" will be * passed to the kafka consumer So you can use any configuration supported by Kafka 0.9.0.X + * useFlumeEventFormat: Reads events from Kafka Topic as an Avro FlumeEvent. Used + * in conjunction with useFlumeEventFormat (Kafka Sink) or parseAsFlumeEvent (Kafka Channel) *

    */ public class KafkaSource extends AbstractPollableSource @@ -87,6 +95,11 @@ public class KafkaSource extends AbstractPollableSource private Map headers; + private Optional> reader = Optional.absent(); + private BinaryDecoder decoder = null; + + private boolean useAvroEventFormat; + private int batchUpperLimit; private int maxBatchDurationMillis; @@ -139,6 +152,7 @@ protected Status doProcess() throws EventDeliveryException { byte[] kafkaMessage; String kafkaKey; Event event; + byte[] eventBody; try { // prepare time variables for new batch @@ -178,11 +192,41 @@ protected Status doProcess() throws EventDeliveryException { kafkaKey = message.key(); kafkaMessage = message.value(); - headers.clear(); - // Add headers to event (timestamp, topic, partition, key) - headers.put(KafkaSourceConstants.TIMESTAMP_HEADER, String.valueOf(System.currentTimeMillis())); - headers.put(KafkaSourceConstants.TOPIC_HEADER, message.topic()); - headers.put(KafkaSourceConstants.PARTITION_HEADER, String.valueOf(message.partition())); + if (useAvroEventFormat) { + //Assume the event is in Avro format using the AvroFlumeEvent schema + //Will need to catch the exception if it is not + ByteArrayInputStream in = + new ByteArrayInputStream(message.value()); + decoder = DecoderFactory.get().directBinaryDecoder(in, decoder); + if (!reader.isPresent()) { + reader = Optional.of( + new SpecificDatumReader(AvroFlumeEvent.class)); + } + //This may throw an exception but it will be caught by the + //exception handler below and logged at error + AvroFlumeEvent avroevent = reader.get().read(null, decoder); + + eventBody = avroevent.getBody().array(); + headers = toStringMap(avroevent.getHeaders()); + } else { + eventBody = message.value(); + headers.clear(); + headers = new HashMap(4); + } + + // Add headers to event (timestamp, topic, partition, key) only if they don't exist + if (!headers.containsKey(KafkaSourceConstants.TIMESTAMP_HEADER)) { + headers.put(KafkaSourceConstants.TIMESTAMP_HEADER, + String.valueOf(System.currentTimeMillis())); + } + if (!headers.containsKey(KafkaSourceConstants.TOPIC_HEADER)) { + headers.put(KafkaSourceConstants.TOPIC_HEADER, message.topic()); + } + if (!headers.containsKey(KafkaSourceConstants.PARTITION_HEADER)) { + headers.put(KafkaSourceConstants.PARTITION_HEADER, + String.valueOf(message.partition())); + } + if (kafkaKey != null) { headers.put(KafkaSourceConstants.KEY_HEADER, kafkaKey); } @@ -191,10 +235,10 @@ protected Status doProcess() throws EventDeliveryException { log.debug("Topic: {} Partition: {} Message: {}", new String[]{ message.topic(), String.valueOf(message.partition()), - new String(kafkaMessage)}); + new String(eventBody)}); } - event = EventBuilder.withBody(kafkaMessage, headers); + event = EventBuilder.withBody(eventBody, headers); eventList.add(event); if (log.isDebugEnabled()) { @@ -275,6 +319,12 @@ protected void doConfigure(Context context) throws FlumeException { maxBatchDurationMillis = context.getInteger(KafkaSourceConstants.BATCH_DURATION_MS, KafkaSourceConstants.DEFAULT_BATCH_DURATION); + useAvroEventFormat = context.getBoolean(KafkaSourceConstants.AVRO_EVENT, KafkaSourceConstants.DEFAULT_AVRO_EVENT); + + if (log.isDebugEnabled()) { + log.debug(KafkaSourceConstants.AVRO_EVENT + " set to: {}", useAvroEventFormat); + } + String bootstrapServers = context.getString(KafkaSourceConstants.BOOTSTRAP_SERVERS); if (bootstrapServers == null || bootstrapServers.isEmpty()) { throw new ConfigurationException("Bootstrap Servers must be specified"); @@ -334,6 +384,17 @@ Properties getConsumerProps() { return kafkaProps; } + /** + * Helper function to convert a map of CharSequence to a map of String. + */ + private static Map toStringMap(Map charSeqMap) { + Map stringMap = new HashMap(); + for (Map.Entry entry : charSeqMap.entrySet()) { + stringMap.put(entry.getKey().toString(), entry.getValue().toString()); + } + return stringMap; + } + Subscriber getSubscriber() { return subscriber; } diff --git a/flume-ng-sources/flume-kafka-source/src/main/java/org/apache/flume/source/kafka/KafkaSourceConstants.java b/flume-ng-sources/flume-kafka-source/src/main/java/org/apache/flume/source/kafka/KafkaSourceConstants.java index 2999cf272b..9f20f614c8 100644 --- a/flume-ng-sources/flume-kafka-source/src/main/java/org/apache/flume/source/kafka/KafkaSourceConstants.java +++ b/flume-ng-sources/flume-kafka-source/src/main/java/org/apache/flume/source/kafka/KafkaSourceConstants.java @@ -34,6 +34,9 @@ public class KafkaSourceConstants { public static final int DEFAULT_BATCH_DURATION = 1000; public static final String DEFAULT_GROUP_ID = "flume"; + public static final String AVRO_EVENT = "useFlumeEventFormat"; + public static final boolean DEFAULT_AVRO_EVENT = false; + /* Old Properties */ public static final String TOPIC = "topic"; diff --git a/flume-ng-sources/flume-kafka-source/src/test/java/org/apache/flume/source/kafka/KafkaSourceEmbeddedKafka.java b/flume-ng-sources/flume-kafka-source/src/test/java/org/apache/flume/source/kafka/KafkaSourceEmbeddedKafka.java index 46d545f90a..affac03b6c 100644 --- a/flume-ng-sources/flume-kafka-source/src/test/java/org/apache/flume/source/kafka/KafkaSourceEmbeddedKafka.java +++ b/flume-ng-sources/flume-kafka-source/src/test/java/org/apache/flume/source/kafka/KafkaSourceEmbeddedKafka.java @@ -24,6 +24,7 @@ import org.apache.commons.io.FileUtils; import org.apache.kafka.clients.producer.KafkaProducer; import org.apache.kafka.clients.producer.ProducerRecord; +import org.apache.kafka.common.serialization.ByteArraySerializer; import org.apache.kafka.common.serialization.StringSerializer; import java.io.File; @@ -52,7 +53,7 @@ public class KafkaSourceEmbeddedKafka { int zkPort = 21818; // none-standard int serverPort = 18922; - KafkaProducer producer; + KafkaProducer producer; File dir; public KafkaSourceEmbeddedKafka(Properties properties) { @@ -95,12 +96,16 @@ private void initProducer() { Properties props = new Properties(); props.put("bootstrap.servers", HOST + ":" + serverPort); props.put("acks", "1"); - producer = new KafkaProducer(props, - new StringSerializer(), new StringSerializer()); + producer = new KafkaProducer(props, + new StringSerializer(), new ByteArraySerializer()); } public void produce(String topic, String k, String v) { - ProducerRecord rec = new ProducerRecord(topic, k, v); + produce(topic, k, v.getBytes()); + } + + public void produce(String topic, String k, byte[] v) { + ProducerRecord rec = new ProducerRecord(topic, k, v); try { producer.send(rec).get(); } catch (InterruptedException e) { @@ -111,7 +116,11 @@ public void produce(String topic, String k, String v) { } public void produce(String topic, int partition, String k, String v) { - ProducerRecord rec = new ProducerRecord(topic, partition, k, v); + produce(topic,partition,k,v.getBytes()); + } + + public void produce(String topic, int partition, String k, byte[] v) { + ProducerRecord rec = new ProducerRecord(topic, partition, k, v); try { producer.send(rec).get(); } catch (InterruptedException e) { diff --git a/flume-ng-sources/flume-kafka-source/src/test/java/org/apache/flume/source/kafka/TestKafkaSource.java b/flume-ng-sources/flume-kafka-source/src/test/java/org/apache/flume/source/kafka/TestKafkaSource.java index 8e04da8387..b4250debca 100644 --- a/flume-ng-sources/flume-kafka-source/src/test/java/org/apache/flume/source/kafka/TestKafkaSource.java +++ b/flume-ng-sources/flume-kafka-source/src/test/java/org/apache/flume/source/kafka/TestKafkaSource.java @@ -22,7 +22,12 @@ import static org.mockito.Matchers.any; import static org.mockito.Mockito.*; +import java.io.ByteArrayOutputStream; +import java.io.IOException; +import java.nio.ByteBuffer; +import java.util.HashMap; import java.util.List; +import java.util.Map; import java.util.Properties; import java.util.regex.Pattern; @@ -31,9 +36,13 @@ import junit.framework.Assert; import kafka.common.TopicExistsException; +import org.apache.avro.io.BinaryEncoder; +import org.apache.avro.io.EncoderFactory; +import org.apache.avro.specific.SpecificDatumWriter; import org.apache.flume.*; import org.apache.flume.PollableSource.Status; import org.apache.flume.channel.ChannelProcessor; +import org.apache.flume.source.avro.AvroFlumeEvent; import org.apache.kafka.clients.consumer.ConsumerConfig; import org.junit.After; import org.junit.Before; @@ -505,6 +514,80 @@ public void testPatternBasedSubscription() { Assert.assertFalse(subscriber.get().matcher("topic").find()); } + @Test + public void testAvroEvent() throws InterruptedException, EventDeliveryException, IOException { + SpecificDatumWriter writer; + ByteArrayOutputStream tempOutStream; + BinaryEncoder encoder; + byte[] bytes; + + context.put(TOPICS, topic0); + context.put(BATCH_SIZE, "1"); + context.put(AVRO_EVENT, "true"); + kafkaSource.configure(context); + kafkaSource.start(); + + Thread.sleep(500L); + + tempOutStream = new ByteArrayOutputStream(); + writer = new SpecificDatumWriter(AvroFlumeEvent.class); + + Map headers = new HashMap(); + headers.put("header1", "value1"); + headers.put("header2", "value2"); + + AvroFlumeEvent e = new AvroFlumeEvent(headers, ByteBuffer.wrap("hello, world".getBytes())); + encoder = EncoderFactory.get().directBinaryEncoder(tempOutStream, null); + writer.write(e, encoder); + encoder.flush(); + bytes = tempOutStream.toByteArray(); + + kafkaServer.produce(topic0, "", bytes); + + String currentTimestamp = Long.toString(System.currentTimeMillis()); + + headers.put(TIMESTAMP_HEADER, currentTimestamp); + headers.put(PARTITION_HEADER, "1"); + headers.put(TOPIC_HEADER, "topic0"); + + e = new AvroFlumeEvent(headers, ByteBuffer.wrap("hello, world2".getBytes())); + tempOutStream.reset(); + encoder = EncoderFactory.get().directBinaryEncoder(tempOutStream, null); + writer.write(e, encoder); + encoder.flush(); + bytes = tempOutStream.toByteArray(); + + kafkaServer.produce(topic0, "", bytes); + + Thread.sleep(500L); + Assert.assertEquals(Status.READY, kafkaSource.process()); + Assert.assertEquals(Status.READY, kafkaSource.process()); + Assert.assertEquals(Status.BACKOFF, kafkaSource.process()); + + Assert.assertEquals(2, events.size()); + + Event event = events.get(0); + + Assert.assertEquals("hello, world", new String(event.getBody(), + Charsets.UTF_8)); + + Assert.assertEquals("value1", e.getHeaders().get("header1")); + Assert.assertEquals("value2", e.getHeaders().get("header2")); + + + event = events.get(1); + + Assert.assertEquals("hello, world2", new String(event.getBody(), + Charsets.UTF_8)); + + Assert.assertEquals("value1", e.getHeaders().get("header1")); + Assert.assertEquals("value2", e.getHeaders().get("header2")); + Assert.assertEquals(currentTimestamp, e.getHeaders().get(TIMESTAMP_HEADER)); + Assert.assertEquals(e.getHeaders().get(PARTITION_HEADER), "1"); + Assert.assertEquals(e.getHeaders().get(TOPIC_HEADER),"topic0"); + + } + ChannelProcessor createGoodChannel() { ChannelProcessor channelProcessor = mock(ChannelProcessor.class); From e6df16d782ae8917b443be81d6a5ad755e02f5c3 Mon Sep 17 00:00:00 2001 From: Roshan Naik Date: Mon, 25 Apr 2016 12:43:31 -0700 Subject: [PATCH 280/341] FLUME-2889: Fixes to DateTime computations (Tristan Stevens via Roshan Naik) --- .../org/apache/flume/source/SyslogParser.java | 11 +++- .../org/apache/flume/source/SyslogUtils.java | 40 +++++++++++- .../apache/flume/source/TestSyslogUtils.java | 62 +++++++++++++++++-- 3 files changed, 104 insertions(+), 9 deletions(-) diff --git a/flume-ng-core/src/main/java/org/apache/flume/source/SyslogParser.java b/flume-ng-core/src/main/java/org/apache/flume/source/SyslogParser.java index c8245ff536..b57ffac73b 100644 --- a/flume-ng-core/src/main/java/org/apache/flume/source/SyslogParser.java +++ b/flume-ng-core/src/main/java/org/apache/flume/source/SyslogParser.java @@ -317,9 +317,16 @@ protected long parseRfc3164Time(String ts) { return 0; } - // try to deal with boundary cases, i.e. new year's eve. // rfc3164 dates are really dumb. - // NB: cannot handle replaying of old logs or going back to the future + /* + * Some code to try and add some smarts to the year insertion as without a year in the message we + * need to make some educated guessing. + * First set the "fixed" to be the timestamp with the current year. + * If the "fixed" time is more than one month in the future then roll it back a year. + * If the "fixed" time is more than eleven months in the past then roll it forward a year. + * This gives us a 12 month rolling window (11 months in the past, 1 month in the future) of timestamps. + */ + if (date != null) { DateTime fixed = date.withYear(year); diff --git a/flume-ng-core/src/main/java/org/apache/flume/source/SyslogUtils.java b/flume-ng-core/src/main/java/org/apache/flume/source/SyslogUtils.java index 5a9f4c85e1..4866183fdb 100644 --- a/flume-ng-core/src/main/java/org/apache/flume/source/SyslogUtils.java +++ b/flume-ng-core/src/main/java/org/apache/flume/source/SyslogUtils.java @@ -33,6 +33,7 @@ import java.util.ArrayList; import java.util.Arrays; import java.util.Calendar; +import java.util.Date; import java.util.HashMap; import java.util.HashSet; import java.util.List; @@ -369,7 +370,44 @@ private void formatHeaders() { // try the available time formats to timestamp for (int dt = 0; dt < fmt.dateFormat.size(); dt++) { try { - timeStamp = String.valueOf(fmt.dateFormat.get(dt).parse(value).getTime()); + Date parsedDate = fmt.dateFormat.get(dt).parse(value); + /* + * Some code to try and add some smarts to the year insertion. + * Original code just added the current year which was okay-ish, but around January 1st becomes + * pretty naïve. + * The current year is added above. This code, if the year has been added does the following: + * 1. Compute what the computed time, but one month in the past would be. + * 2. Compute what the computed time, but eleven months in the future would be. + * If the computed time is more than one month in the future then roll it back a year. + * If the computed time is more than eleven months in the past then roll it forward a year. + * This gives us a 12 month rolling window (11 months in the past, 1 month in the future) of timestamps. + */ + if (fmt.addYear) { + Calendar cal = Calendar.getInstance(); + cal.setTime(parsedDate); + Calendar calMinusOneMonth = Calendar.getInstance(); + calMinusOneMonth.setTime(parsedDate); + calMinusOneMonth.add(Calendar.MONTH, -1); + + Calendar calPlusElevenMonths = Calendar.getInstance(); + calPlusElevenMonths.setTime(parsedDate); + calPlusElevenMonths.add(Calendar.MONTH, +11); + + if (cal.getTimeInMillis() > System.currentTimeMillis() && calMinusOneMonth.getTimeInMillis() > System.currentTimeMillis()) { + //Need to roll back a year + Calendar c1 = Calendar.getInstance(); + c1.setTime(parsedDate); + c1.add(Calendar.YEAR, -1); + parsedDate = c1.getTime(); + } else if (cal.getTimeInMillis() < System.currentTimeMillis() && calPlusElevenMonths.getTimeInMillis() < System.currentTimeMillis() ) { + //Need to roll forward a year + Calendar c1 = Calendar.getInstance(); + c1.setTime(parsedDate); + c1.add(Calendar.YEAR, -1); + parsedDate = c1.getTime(); + } + } + timeStamp = String.valueOf(parsedDate.getTime()); break; // done. formatted the time } catch (ParseException e) { // Error formatting the timeStamp, try next format diff --git a/flume-ng-core/src/test/java/org/apache/flume/source/TestSyslogUtils.java b/flume-ng-core/src/test/java/org/apache/flume/source/TestSyslogUtils.java index be4598eb5c..1c005ffffa 100644 --- a/flume-ng-core/src/test/java/org/apache/flume/source/TestSyslogUtils.java +++ b/flume-ng-core/src/test/java/org/apache/flume/source/TestSyslogUtils.java @@ -28,6 +28,7 @@ import java.text.ParseException; import java.text.SimpleDateFormat; import java.util.Calendar; +import java.util.Date; import java.util.HashSet; import java.util.Locale; import java.util.Map; @@ -130,25 +131,33 @@ public void TestHeader8() throws ParseException { @Test public void TestHeader9() throws ParseException { - String stamp1 = "Apr 11 13:14:04"; + SimpleDateFormat sdf = new SimpleDateFormat("MMM d hh:MM:ss"); + Calendar cal = Calendar.getInstance(); + + String year = String.valueOf(cal.get(Calendar.YEAR)); + String stamp1 = sdf.format(cal.getTime()); String format1 = "yyyyMMM d HH:mm:ss"; String host1 = "ubuntu-11.cloudera.com"; String data1 = "some msg"; // timestamp with 'Z' appended, translates to UTC String msg1 = "<10>" + stamp1 + " " + host1 + " " + data1 + "\n"; - checkHeader(msg1, String.valueOf(Calendar.getInstance().get(Calendar.YEAR)) + stamp1, + checkHeader(msg1, year + stamp1, format1, host1, data1); } @Test public void TestHeader10() throws ParseException { - String stamp1 = "Apr 1 13:14:04"; + SimpleDateFormat sdf = new SimpleDateFormat("MMM d hh:MM:ss"); + Calendar cal = Calendar.getInstance(); + + String year = String.valueOf(cal.get(Calendar.YEAR)); + String stamp1 = sdf.format(cal.getTime()); String format1 = "yyyyMMM d HH:mm:ss"; String host1 = "ubuntu-11.cloudera.com"; String data1 = "some msg"; // timestamp with 'Z' appended, translates to UTC String msg1 = "<10>" + stamp1 + " " + host1 + " " + data1 + "\n"; - checkHeader(msg1, String.valueOf(Calendar.getInstance().get(Calendar.YEAR)) + stamp1, + checkHeader(msg1, year + stamp1, format1, host1, data1); } @@ -169,15 +178,56 @@ public void TestHeader11() throws ParseException { @Test public void TestRfc3164HeaderApacheLogWithNulls() throws ParseException { - String stamp1 = "Apr 1 13:14:04"; + SimpleDateFormat sdf = new SimpleDateFormat("MMM d hh:MM:ss"); + Calendar cal = Calendar.getInstance(); + + String year = String.valueOf(cal.get(Calendar.YEAR)); + String stamp1 = sdf.format(cal.getTime()); String format1 = "yyyyMMM d HH:mm:ss"; String host1 = "ubuntu-11.cloudera.com"; String data1 = "- hyphen_null_breaks_5424_pattern [07/Jun/2012:14:46:44 -0600]"; String msg1 = "<10>" + stamp1 + " " + host1 + " " + data1 + "\n"; - checkHeader(msg1, String.valueOf(Calendar.getInstance().get(Calendar.YEAR)) + stamp1, + checkHeader(msg1, year + stamp1, format1, host1, data1); } + @Test + public void TestRfc3164Dates() throws ParseException { + /* + * This test creates a series of dates that range from 10 months in the past to (5 days short of) + * one month in the future. This tests that the year addition code is clever enough to handle scenarios + * where the event received was generated in a different year to what flume considers to be "current" + * (e.g. where there has been some lag somewhere, especially when flicking over on New Year's eve, or + * when you are about to flick over and the flume's system clock is slightly slower than the Syslog + * source's clock). + */ + for (int i=-10; i<=1; i++) { + SimpleDateFormat sdf = new SimpleDateFormat("MMM d hh:MM:ss"); + Date date = new Date(System.currentTimeMillis()); + Calendar cal = Calendar.getInstance(); + cal.setTime(date); + cal.add(Calendar.MONTH, i); + + //Small tweak to avoid the 1 month in the future ticking over by a few seconds between now + //and when the checkHeader actually runs + if (i==1) { + cal.add(Calendar.DAY_OF_MONTH, -1); + } + + String stamp1 = sdf.format(cal.getTime()); + + String year = String.valueOf(cal.get(Calendar.YEAR)); + String format1 = "yyyyMMM d HH:mm:ss"; + String host1 = "ubuntu-11.cloudera.com"; + String data1 = "some msg"; + + // timestamp with 'Z' appended, translates to UTC + String msg1 = "<10>" + stamp1 + " " + host1 + " " + data1 + "\n"; + checkHeader(msg1, year + stamp1, + format1, host1, data1); + } + } + public static void checkHeader(String keepFields, String msg1, String stamp1, String format1, String host1, String data1) throws ParseException { SyslogUtils util; From 46ae5accd03e5b8b18d3e1fb52761c1df4e51e5a Mon Sep 17 00:00:00 2001 From: Jarek Jarcec Cecho Date: Thu, 5 May 2016 09:20:41 -0700 Subject: [PATCH 281/341] FLUME-2908: NetcatSource - SocketChannel not closed when session is broken (Grant Henke via Jarek Jarcec Cecho) --- .../src/main/java/org/apache/flume/source/NetcatSource.java | 5 +++++ 1 file changed, 5 insertions(+) diff --git a/flume-ng-core/src/main/java/org/apache/flume/source/NetcatSource.java b/flume-ng-core/src/main/java/org/apache/flume/source/NetcatSource.java index 61e3f90c01..9513902966 100644 --- a/flume-ng-core/src/main/java/org/apache/flume/source/NetcatSource.java +++ b/flume-ng-core/src/main/java/org/apache/flume/source/NetcatSource.java @@ -357,6 +357,11 @@ public void run() { counterGroup.incrementAndGet("sessions.completed"); } catch (IOException e) { counterGroup.incrementAndGet("sessions.broken"); + try { + socketChannel.close(); + } catch (IOException ex) { + logger.error("Unable to close socket channel. Exception follows.", ex); + } } logger.debug("Connection handler exiting"); From 37ddd1b27258f2c3095094b79cbd8852478de33d Mon Sep 17 00:00:00 2001 From: Mike Percy Date: Wed, 18 May 2016 10:38:29 -0700 Subject: [PATCH 282/341] FLUME-2909. Upgrade RAT to 0.11 MIME-Version: 1.0 Content-Type: text/plain; charset=UTF-8 Content-Transfer-Encoding: 8bit This allows command-line override of RAT using the command mvn -Drat.skip=true which is useful in a development environment. (Bessenyei Balázs Donát via Mike Percy) --- pom.xml | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/pom.xml b/pom.xml index 3b2c97ce52..09290956bb 100644 --- a/pom.xml +++ b/pom.xml @@ -729,7 +729,7 @@ limitations under the License. org.apache.rat apache-rat-plugin - 0.7 + 0.11 verify.rat From f4f15bee1e4ba109c6a21f639668911a13241014 Mon Sep 17 00:00:00 2001 From: Mike Percy Date: Fri, 27 May 2016 12:10:41 -0700 Subject: [PATCH 283/341] FLUME-2913. Don't strip SLF4J from imported classpaths MIME-Version: 1.0 Content-Type: text/plain; charset=UTF-8 Content-Transfer-Encoding: 8bit It is no longer necessary to strip other versions of SLF4J from classpaths imported from hadoop or hbase. With recent versions of SLF4J, when multiple connectors are found, one of them "wins" and the logging does not break. A warning is printed once, however. This is acceptable because having too long of a classpath can cause major problems on systems with a low ARG_MAX. (Bessenyei Balázs Donát via Mike Percy) --- bin/flume-ng | 31 ++----------------------------- 1 file changed, 2 insertions(+), 29 deletions(-) diff --git a/bin/flume-ng b/bin/flume-ng index e28723d2a9..9bea01f5f8 100755 --- a/bin/flume-ng +++ b/bin/flume-ng @@ -93,21 +93,7 @@ add_hadoop_paths() { # determine hadoop classpath HADOOP_CLASSPATH=$($HADOOP_IN_PATH classpath) - # hack up and filter hadoop classpath - local ELEMENTS=$(sed -e 's/:/ /g' <<<${HADOOP_CLASSPATH}) - local ELEMENT - for ELEMENT in $ELEMENTS; do - local PIECE - for PIECE in $(echo $ELEMENT); do - if [[ $PIECE =~ slf4j-(api|log4j12).*\.jar ]]; then - info "Excluding $PIECE from classpath" - continue - else - FLUME_CLASSPATH="$FLUME_CLASSPATH:$PIECE" - fi - done - done - + FLUME_CLASSPATH="$FLUME_CLASSPATH:$HADOOP_CLASSPATH" fi } add_HBASE_paths() { @@ -142,20 +128,7 @@ add_HBASE_paths() { # determine HBASE classpath HBASE_CLASSPATH=$($HBASE_IN_PATH classpath) - # hack up and filter HBASE classpath - local ELEMENTS=$(sed -e 's/:/ /g' <<<${HBASE_CLASSPATH}) - local ELEMENT - for ELEMENT in $ELEMENTS; do - local PIECE - for PIECE in $(echo $ELEMENT); do - if [[ $PIECE =~ slf4j-(api|log4j12).*\.jar ]]; then - info "Excluding $PIECE from classpath" - continue - else - FLUME_CLASSPATH="$FLUME_CLASSPATH:$PIECE" - fi - done - done + FLUME_CLASSPATH="$FLUME_CLASSPATH:$HBASE_CLASSPATH" FLUME_CLASSPATH="$FLUME_CLASSPATH:$HBASE_HOME/conf" fi From a619cc15748d76357f84bd3d0fa0a3e496ee61a1 Mon Sep 17 00:00:00 2001 From: Jarek Jarcec Cecho Date: Fri, 10 Jun 2016 15:34:24 +0200 Subject: [PATCH 284/341] FLUME-2915: The kafka channel using new APIs will be stuck when the sink is avro sink (Jeff Holoman via Jarek Jarcec Cecho) --- .../flume/channel/kafka/KafkaChannel.java | 4 ++- .../flume/channel/kafka/TestKafkaChannel.java | 29 +++++++++++++++++++ 2 files changed, 32 insertions(+), 1 deletion(-) diff --git a/flume-ng-channels/flume-kafka-channel/src/main/java/org/apache/flume/channel/kafka/KafkaChannel.java b/flume-ng-channels/flume-kafka-channel/src/main/java/org/apache/flume/channel/kafka/KafkaChannel.java index 2d9b0c6c18..09d3f9d376 100644 --- a/flume-ng-channels/flume-kafka-channel/src/main/java/org/apache/flume/channel/kafka/KafkaChannel.java +++ b/flume-ng-channels/flume-kafka-channel/src/main/java/org/apache/flume/channel/kafka/KafkaChannel.java @@ -381,7 +381,9 @@ protected Event doTake() throws InterruptedException { } //Add the key to the header - e.getHeaders().put(KEY_HEADER, record.key()); + if (record.key() != null) { + e.getHeaders().put(KEY_HEADER, record.key()); + } if (logger.isDebugEnabled()) { logger.debug("Processed output from partition {} offset {}", record.partition(), record.offset()); diff --git a/flume-ng-channels/flume-kafka-channel/src/test/java/org/apache/flume/channel/kafka/TestKafkaChannel.java b/flume-ng-channels/flume-kafka-channel/src/test/java/org/apache/flume/channel/kafka/TestKafkaChannel.java index 637428d120..13e073b17a 100644 --- a/flume-ng-channels/flume-kafka-channel/src/test/java/org/apache/flume/channel/kafka/TestKafkaChannel.java +++ b/flume-ng-channels/flume-kafka-channel/src/test/java/org/apache/flume/channel/kafka/TestKafkaChannel.java @@ -186,6 +186,11 @@ public void testParseAsFlumeEventFalseAsSourceCheckHeader() throws Exception { doParseAsFlumeEventFalseAsSource(true); } + @Test + public void testNullKeyNoHeader() throws Exception { + doTestNullKeyNoHeader(); + } + private void doParseAsFlumeEventFalse(Boolean checkHeaders) throws Exception { final KafkaChannel channel = startChannel(false); Properties props = channel.getProducerProps(); @@ -215,6 +220,30 @@ private void doParseAsFlumeEventFalse(Boolean checkHeaders) throws Exception { channel.stop(); } + private void doTestNullKeyNoHeader() throws Exception { + final KafkaChannel channel = startChannel(false); + Properties props = channel.getProducerProps(); + KafkaProducer producer = new KafkaProducer(props); + + for (int i = 0; i < 50; i++) { + ProducerRecord data = new ProducerRecord(topic, null, String.valueOf(i).getBytes()); + producer.send(data).get(); + } + ExecutorCompletionService submitterSvc = new + ExecutorCompletionService(Executors.newCachedThreadPool()); + List events = pullEvents(channel, submitterSvc, + 50, false, false); + wait(submitterSvc, 5); + List finals = new ArrayList(50); + for (int i = 0; i < 50; i++) { + finals.add(i, events.get(i).getHeaders().get(KEY_HEADER)); + } + for (int i = 0; i < 50; i++) { + Assert.assertTrue( finals.get(i) == null); + } + channel.stop(); + } + /** * Like the previous test but here we write to the channel like a Flume source would do * to verify that the events are written as text and not as an Avro object From f46bee03eb2c9094aa192dd6b82a494cb26eef35 Mon Sep 17 00:00:00 2001 From: Jarek Jarcec Cecho Date: Fri, 10 Jun 2016 15:52:20 +0200 Subject: [PATCH 285/341] FLUME-2920: Kafka Channel Should Not Commit Offsets When Stopping (Kevin Conaway via Jarek Jarcec Cecho) --- .../flume/channel/kafka/KafkaChannel.java | 4 - .../flume/channel/kafka/TestKafkaChannel.java | 82 ++++++++++++++++--- 2 files changed, 70 insertions(+), 16 deletions(-) diff --git a/flume-ng-channels/flume-kafka-channel/src/main/java/org/apache/flume/channel/kafka/KafkaChannel.java b/flume-ng-channels/flume-kafka-channel/src/main/java/org/apache/flume/channel/kafka/KafkaChannel.java index 09d3f9d376..dfc95bc25e 100644 --- a/flume-ng-channels/flume-kafka-channel/src/main/java/org/apache/flume/channel/kafka/KafkaChannel.java +++ b/flume-ng-channels/flume-kafka-channel/src/main/java/org/apache/flume/channel/kafka/KafkaChannel.java @@ -267,10 +267,6 @@ private synchronized ConsumerAndRecords createConsumerAndRecords() { } private void decommissionConsumerAndRecords(ConsumerAndRecords c) { - if (c.failedEvents.isEmpty()) { - c.commitOffsets(); - } - c.failedEvents.clear(); c.consumer.close(); } diff --git a/flume-ng-channels/flume-kafka-channel/src/test/java/org/apache/flume/channel/kafka/TestKafkaChannel.java b/flume-ng-channels/flume-kafka-channel/src/test/java/org/apache/flume/channel/kafka/TestKafkaChannel.java index 13e073b17a..d01346afd1 100644 --- a/flume-ng-channels/flume-kafka-channel/src/test/java/org/apache/flume/channel/kafka/TestKafkaChannel.java +++ b/flume-ng-channels/flume-kafka-channel/src/test/java/org/apache/flume/channel/kafka/TestKafkaChannel.java @@ -32,26 +32,42 @@ import org.apache.kafka.clients.consumer.ConsumerConfig; import org.apache.kafka.clients.producer.KafkaProducer; import org.apache.kafka.clients.producer.ProducerRecord; -import org.junit.*; -import org.slf4j.Logger; -import org.slf4j.LoggerFactory; - -import java.util.*; -import java.util.concurrent.*; +import org.junit.AfterClass; +import org.junit.Assert; +import org.junit.Before; +import org.junit.BeforeClass; +import org.junit.Test; + +import java.util.ArrayList; +import java.util.Arrays; +import java.util.Collections; +import java.util.HashMap; +import java.util.HashSet; +import java.util.List; +import java.util.Map; +import java.util.Properties; +import java.util.Set; +import java.util.concurrent.Callable; +import java.util.concurrent.CyclicBarrier; +import java.util.concurrent.ExecutorCompletionService; +import java.util.concurrent.ExecutorService; +import java.util.concurrent.Executors; import java.util.concurrent.atomic.AtomicBoolean; import java.util.concurrent.atomic.AtomicInteger; -import static org.apache.flume.channel.kafka.KafkaChannelConfiguration.*; +import static org.apache.flume.channel.kafka.KafkaChannelConfiguration.BOOTSTRAP_SERVERS_CONFIG; +import static org.apache.flume.channel.kafka.KafkaChannelConfiguration.BROKER_LIST_FLUME_KEY; +import static org.apache.flume.channel.kafka.KafkaChannelConfiguration.GROUP_ID_FLUME; +import static org.apache.flume.channel.kafka.KafkaChannelConfiguration.KEY_HEADER; +import static org.apache.flume.channel.kafka.KafkaChannelConfiguration.PARSE_AS_FLUME_EVENT; +import static org.apache.flume.channel.kafka.KafkaChannelConfiguration.READ_SMALLEST_OFFSET; +import static org.apache.flume.channel.kafka.KafkaChannelConfiguration.TOPIC_CONFIG; public class TestKafkaChannel { - private final static Logger LOGGER = - LoggerFactory.getLogger(TestKafkaChannel.class); - private static TestUtil testUtil = TestUtil.getInstance(); private String topic = null; private final Set usedTopics = new HashSet(); - private CountDownLatch latch = null; @BeforeClass public static void setupClass() throws Exception { @@ -74,7 +90,6 @@ public void setup() throws Exception { } catch (Exception e) { } Thread.sleep(2500); - latch = new CountDownLatch(5); } @AfterClass @@ -191,6 +206,49 @@ public void testNullKeyNoHeader() throws Exception { doTestNullKeyNoHeader(); } + @Test + public void testOffsetsNotCommittedOnStop() throws Exception { + String message = "testOffsetsNotCommittedOnStop-" + System.nanoTime(); + + KafkaChannel channel = startChannel(false); + + KafkaProducer producer = new KafkaProducer(channel.getProducerProps()); + ProducerRecord data = new ProducerRecord(topic, "header-" + message, message.getBytes()); + producer.send(data).get(); + producer.flush(); + producer.close(); + + Event event = takeEventWithoutCommittingTxn(channel); + Assert.assertNotNull(event); + Assert.assertTrue(Arrays.equals(message.getBytes(), event.getBody())); + + // Stop the channel without committing the transaction + channel.stop(); + + channel = startChannel(false); + + // Message should still be available + event = takeEventWithoutCommittingTxn(channel); + Assert.assertNotNull(event); + Assert.assertTrue(Arrays.equals(message.getBytes(), event.getBody())); + } + + private Event takeEventWithoutCommittingTxn(KafkaChannel channel) { + for (int i=0; i < 5; i++) { + Transaction txn = channel.getTransaction(); + txn.begin(); + + Event event = channel.take(); + if (event != null) { + return event; + } else { + txn.commit(); + txn.close(); + } + } + return null; + } + private void doParseAsFlumeEventFalse(Boolean checkHeaders) throws Exception { final KafkaChannel channel = startChannel(false); Properties props = channel.getProducerProps(); From ee4999bc23f42bc300ed87b0d46fd96418d6a185 Mon Sep 17 00:00:00 2001 From: Mike Percy Date: Fri, 10 Jun 2016 12:52:29 -0700 Subject: [PATCH 286/341] FLUME-2910. AsyncHBaseSink: Failure callbacks should log the exception that caused them (Abraham Fine via Mike Percy) --- .../org/apache/flume/sink/hbase/AsyncHBaseSink.java | 11 ++++++----- 1 file changed, 6 insertions(+), 5 deletions(-) diff --git a/flume-ng-sinks/flume-ng-hbase-sink/src/main/java/org/apache/flume/sink/hbase/AsyncHBaseSink.java b/flume-ng-sinks/flume-ng-hbase-sink/src/main/java/org/apache/flume/sink/hbase/AsyncHBaseSink.java index c1ff0c41eb..28f0de1744 100644 --- a/flume-ng-sinks/flume-ng-hbase-sink/src/main/java/org/apache/flume/sink/hbase/AsyncHBaseSink.java +++ b/flume-ng-sinks/flume-ng-hbase-sink/src/main/java/org/apache/flume/sink/hbase/AsyncHBaseSink.java @@ -191,15 +191,15 @@ public Status process() throws EventDeliveryException { Callback putSuccessCallback = new SuccessCallback( lock, callbacksReceived, condition); - Callback putFailureCallback = - new FailureCallback( + Callback putFailureCallback = + new FailureCallback( lock, callbacksReceived, txnFail, condition); Callback incrementSuccessCallback = new SuccessCallback( lock, callbacksReceived, condition); - Callback incrementFailureCallback = - new FailureCallback( + Callback incrementFailureCallback = + new FailureCallback( lock, callbacksReceived, txnFail, condition); Status status = Status.READY; @@ -622,7 +622,7 @@ private void doCall() throws Exception { } } - private class FailureCallback implements Callback { + private class FailureCallback implements Callback { private Lock lock; private AtomicInteger callbacksReceived; private AtomicBoolean txnFail; @@ -639,6 +639,7 @@ public FailureCallback(Lock lck, AtomicInteger callbacksReceived, @Override public R call(T arg) throws Exception { + logger.error("failure callback:", arg); if (isTimeoutTesting) { //tests set timeout to 10 seconds, so sleep for 4 seconds try { From 9601f5bf0a5294d5ffd324010768fe0299044d6d Mon Sep 17 00:00:00 2001 From: Dylan Jones Date: Tue, 14 Jun 2016 21:35:47 +0100 Subject: [PATCH 287/341] User guide: fix mistake and formatting change source to sink and fix formatting in Kafka Channel documentation (Dylan Jones via Mike Percy) --- flume-ng-doc/sphinx/FlumeUserGuide.rst | 16 +++++++++------- 1 file changed, 9 insertions(+), 7 deletions(-) diff --git a/flume-ng-doc/sphinx/FlumeUserGuide.rst b/flume-ng-doc/sphinx/FlumeUserGuide.rst index 9c11fe6235..74d2887c72 100644 --- a/flume-ng-doc/sphinx/FlumeUserGuide.rst +++ b/flume-ng-doc/sphinx/FlumeUserGuide.rst @@ -2684,19 +2684,21 @@ The events are stored in a Kafka cluster (must be installed separately). Kafka p replication, so in case an agent or a kafka broker crashes, the events are immediately available to other sinks The Kafka channel can be used for multiple scenarios: -* With Flume source and sink - it provides a reliable and highly available channel for events -* With Flume source and interceptor but no sink - it allows writing Flume events into a Kafka topic, for use by other apps -* With Flume sink, but no source - it is a low-latency, fault tolerant way to send events from Kafka to Flume sources such as HDFS, HBase or Solr + +#. With Flume source and sink - it provides a reliable and highly available channel for events +#. With Flume source and interceptor but no sink - it allows writing Flume events into a Kafka topic, for use by other apps +#. With Flume sink, but no source - it is a low-latency, fault tolerant way to send events from Kafka to Flume sinks such as HDFS, HBase or Solr This version of Flume requires Kafka version 0.9 or greater due to the reliance on the Kafka clients shipped with that version. The configuration of the channel has changed compared to previous flume versions. The configuration parameters are organized as such: -1) Configuration values related to the channel generically are applied at the channel config level, eg: a1.channel.k1.type = -2) Configuration values related to Kafka or how the Channel operates are prefixed with "kafka.", (this are analgous to CommonClient Configs)eg: a1.channels.k1.kafka.topica1.channels.k1.kafka.bootstrap.serversThis is not dissimilar to how the hdfs sink operates -3) Properties specific to the producer/consumer are prefixed by kafka.producer or kafka.consumer -4) Where possible, the Kafka paramter names are used, eg: bootstrap.servers and acks + +#. Configuration values related to the channel generically are applied at the channel config level, eg: a1.channel.k1.type = +#. Configuration values related to Kafka or how the Channel operates are prefixed with "kafka.", (this are analgous to CommonClient Configs) eg: a1.channels.k1.kafka.topic and a1.channels.k1.kafka.bootstrap.servers. This is not dissimilar to how the hdfs sink operates +#. Properties specific to the producer/consumer are prefixed by kafka.producer or kafka.consumer +#. Where possible, the Kafka paramter names are used, eg: bootstrap.servers and acks This version of flume is backwards-compatible with previous versions, however deprecated properties are indicated in the table below and a warning message is logged on startup when they are present in the configuration file. From 1b9e589152f0093b6ef7cea580c00ff52944d2cf Mon Sep 17 00:00:00 2001 From: Mike Percy Date: Fri, 17 Jun 2016 15:32:09 -0700 Subject: [PATCH 288/341] FLUME-2923. Bump asynchbase version to 1.7.0 MIME-Version: 1.0 Content-Type: text/plain; charset=UTF-8 Content-Transfer-Encoding: 8bit Newer versions of HBase and the asynchbase library support error types such as RegionMovedException. Also, asynchbase has incorporated several fixes since version 1.5.0. See https://github.com/OpenTSDB/asynchbase/blob/v1.7.0/NEWS (Bessenyei Balázs Donát via Mike Percy) --- pom.xml | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/pom.xml b/pom.xml index 09290956bb..694c073d6f 100644 --- a/pom.xml +++ b/pom.xml @@ -1292,7 +1292,7 @@ limitations under the License. org.hbase asynchbase - 1.5.0 + 1.7.0 org.slf4j @@ -1308,7 +1308,7 @@ limitations under the License. io.netty netty - 3.5.12.Final + 3.9.4.Final From 7d1e683fbd7d261fff9fcf17ad78fd8469c64905 Mon Sep 17 00:00:00 2001 From: Mike Percy Date: Mon, 20 Jun 2016 01:09:07 -0700 Subject: [PATCH 289/341] FLUME-2918. Speed up TaildirSource on directories with many files This patch greatly improves the performance of TaildirSource on directories that contain a large number of files. (Attila Simon via Mike Percy) --- .../taildir/ReliableTaildirEventReader.java | 56 ++-- .../apache/flume/source/taildir/TailFile.java | 7 - .../flume/source/taildir/TaildirMatcher.java | 278 ++++++++++++++++++ .../flume/source/taildir/TaildirSource.java | 3 + .../TaildirSourceConfigurationConstants.java | 4 + .../source/taildir/TestTaildirMatcher.java | 227 ++++++++++++++ .../source/taildir/TestTaildirSource.java | 2 +- 7 files changed, 531 insertions(+), 46 deletions(-) create mode 100644 flume-ng-sources/flume-taildir-source/src/main/java/org/apache/flume/source/taildir/TaildirMatcher.java create mode 100644 flume-ng-sources/flume-taildir-source/src/test/java/org/apache/flume/source/taildir/TestTaildirMatcher.java diff --git a/flume-ng-sources/flume-taildir-source/src/main/java/org/apache/flume/source/taildir/ReliableTaildirEventReader.java b/flume-ng-sources/flume-taildir-source/src/main/java/org/apache/flume/source/taildir/ReliableTaildirEventReader.java index 5b6d465701..8128df4c82 100644 --- a/flume-ng-sources/flume-taildir-source/src/main/java/org/apache/flume/source/taildir/ReliableTaildirEventReader.java +++ b/flume-ng-sources/flume-taildir-source/src/main/java/org/apache/flume/source/taildir/ReliableTaildirEventReader.java @@ -20,18 +20,14 @@ package org.apache.flume.source.taildir; import java.io.File; -import java.io.FileFilter; import java.io.FileNotFoundException; import java.io.FileReader; import java.io.IOException; import java.nio.file.Files; -import java.util.ArrayList; import java.util.Arrays; -import java.util.Collections; import java.util.List; import java.util.Map; import java.util.Map.Entry; -import java.util.regex.Pattern; import org.apache.flume.Event; import org.apache.flume.FlumeException; @@ -43,11 +39,9 @@ import com.google.common.annotations.VisibleForTesting; import com.google.common.base.Preconditions; -import com.google.common.collect.HashBasedTable; import com.google.common.collect.Lists; import com.google.common.collect.Maps; import com.google.common.collect.Table; -import com.google.common.collect.Table.Cell; import com.google.gson.stream.JsonReader; @InterfaceAudience.Private @@ -55,13 +49,14 @@ public class ReliableTaildirEventReader implements ReliableEventReader { private static final Logger logger = LoggerFactory.getLogger(ReliableTaildirEventReader.class); - private final Table tailFileTable; + private final List taildirCache; private final Table headerTable; private TailFile currentFile = null; private Map tailFiles = Maps.newHashMap(); private long updateTime; private boolean addByteOffset; + private boolean cachePatternMatching; private boolean committed = true; /** @@ -69,7 +64,7 @@ public class ReliableTaildirEventReader implements ReliableEventReader { */ private ReliableTaildirEventReader(Map filePaths, Table headerTable, String positionFilePath, - boolean skipToEnd, boolean addByteOffset) throws IOException { + boolean skipToEnd, boolean addByteOffset, boolean cachePatternMatching) throws IOException { // Sanity checks Preconditions.checkNotNull(filePaths); Preconditions.checkNotNull(positionFilePath); @@ -79,21 +74,17 @@ private ReliableTaildirEventReader(Map filePaths, new Object[] { ReliableTaildirEventReader.class.getSimpleName(), filePaths }); } - Table tailFileTable = HashBasedTable.create(); + List taildirCache = Lists.newArrayList(); for (Entry e : filePaths.entrySet()) { - File f = new File(e.getValue()); - File parentDir = f.getParentFile(); - Preconditions.checkState(parentDir.exists(), - "Directory does not exist: " + parentDir.getAbsolutePath()); - Pattern fileNamePattern = Pattern.compile(f.getName()); - tailFileTable.put(e.getKey(), parentDir, fileNamePattern); + taildirCache.add(new TaildirMatcher(e.getKey(), e.getValue(), cachePatternMatching)); } - logger.info("tailFileTable: " + tailFileTable.toString()); + logger.info("taildirCache: " + taildirCache.toString()); logger.info("headerTable: " + headerTable.toString()); - this.tailFileTable = tailFileTable; + this.taildirCache = taildirCache; this.headerTable = headerTable; this.addByteOffset = addByteOffset; + this.cachePatternMatching = cachePatternMatching; updateTailFiles(skipToEnd); logger.info("Updating position from position file: " + positionFilePath); @@ -238,12 +229,10 @@ public List updateTailFiles(boolean skipToEnd) throws IOException { updateTime = System.currentTimeMillis(); List updatedInodes = Lists.newArrayList(); - for (Cell cell : tailFileTable.cellSet()) { - Map headers = headerTable.row(cell.getRowKey()); - File parentDir = cell.getColumnKey(); - Pattern fileNamePattern = cell.getValue(); + for (TaildirMatcher taildir : taildirCache) { + Map headers = headerTable.row(taildir.getFileGroup()); - for (File f : getMatchFiles(parentDir, fileNamePattern)) { + for (File f : taildir.getMatchingFiles()) { long inode = getInode(f); TailFile tf = tailFiles.get(inode); if (tf == null || !tf.getPath().equals(f.getAbsolutePath())) { @@ -274,21 +263,6 @@ public List updateTailFiles() throws IOException { return updateTailFiles(false); } - private List getMatchFiles(File parentDir, final Pattern fileNamePattern) { - FileFilter filter = new FileFilter() { - public boolean accept(File f) { - String fileName = f.getName(); - if (f.isDirectory() || !fileNamePattern.matcher(fileName).matches()) { - return false; - } - return true; - } - }; - File[] files = parentDir.listFiles(filter); - ArrayList result = Lists.newArrayList(files); - Collections.sort(result, new TailFile.CompareByLastModifiedTime()); - return result; - } private long getInode(File file) throws IOException { long inode = (long) Files.getAttribute(file.toPath(), "unix:ino"); @@ -313,6 +287,7 @@ public static class Builder { private String positionFilePath; private boolean skipToEnd; private boolean addByteOffset; + private boolean cachePatternMatching; public Builder filePaths(Map filePaths) { this.filePaths = filePaths; @@ -339,8 +314,13 @@ public Builder addByteOffset(boolean addByteOffset) { return this; } + public Builder cachePatternMatching(boolean cachePatternMatching) { + this.cachePatternMatching = cachePatternMatching; + return this; + } + public ReliableTaildirEventReader build() throws IOException { - return new ReliableTaildirEventReader(filePaths, headerTable, positionFilePath, skipToEnd, addByteOffset); + return new ReliableTaildirEventReader(filePaths, headerTable, positionFilePath, skipToEnd, addByteOffset, cachePatternMatching); } } diff --git a/flume-ng-sources/flume-taildir-source/src/main/java/org/apache/flume/source/taildir/TailFile.java b/flume-ng-sources/flume-taildir-source/src/main/java/org/apache/flume/source/taildir/TailFile.java index eabd357b5e..cb36e411a7 100644 --- a/flume-ng-sources/flume-taildir-source/src/main/java/org/apache/flume/source/taildir/TailFile.java +++ b/flume-ng-sources/flume-taildir-source/src/main/java/org/apache/flume/source/taildir/TailFile.java @@ -212,13 +212,6 @@ public void close() { } } - public static class CompareByLastModifiedTime implements Comparator { - @Override - public int compare(File f1, File f2) { - return Long.valueOf(f1.lastModified()).compareTo(f2.lastModified()); - } - } - private class LineResult { final boolean lineSepInclude; final byte[] line; diff --git a/flume-ng-sources/flume-taildir-source/src/main/java/org/apache/flume/source/taildir/TaildirMatcher.java b/flume-ng-sources/flume-taildir-source/src/main/java/org/apache/flume/source/taildir/TaildirMatcher.java new file mode 100644 index 0000000000..245aef5156 --- /dev/null +++ b/flume-ng-sources/flume-taildir-source/src/main/java/org/apache/flume/source/taildir/TaildirMatcher.java @@ -0,0 +1,278 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, + * software distributed under the License is distributed on an + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + * KIND, either express or implied. See the License for the + * specific language governing permissions and limitations + * under the License. + */ + +package org.apache.flume.source.taildir; + +import com.google.common.base.Preconditions; +import com.google.common.collect.Lists; +import org.apache.flume.annotations.InterfaceAudience; +import org.apache.flume.annotations.InterfaceStability; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +import java.io.File; +import java.io.FileFilter; +import java.io.IOException; +import java.nio.file.DirectoryStream; +import java.nio.file.FileSystem; +import java.nio.file.FileSystems; +import java.nio.file.Files; +import java.nio.file.Path; +import java.nio.file.PathMatcher; +import java.util.Collections; +import java.util.Comparator; +import java.util.HashMap; +import java.util.List; +import java.util.concurrent.TimeUnit; +import java.util.regex.Pattern; + +/** + * Identifies and caches the files matched by single file pattern for TAILDIR source. + *

    + * Since file patterns only apply to the fileNames and not the parent dictionaries, this implementation + * checks the parent directory for modification (additional or removed files update modification time of parent dir) + * If no modification happened to the parent dir that means the underlying files could only be written to but no need + * to rerun the pattern matching on fileNames. + *

    + * This implementation provides lazy caching or no caching. Instances of this class keep the result + * file list from the last successful execution + * of {@linkplain #getMatchingFiles()} function invocation, + * and may serve the content without hitting the FileSystem for performance optimization. + *

    + * IMPORTANT: It is assumed that the hosting system provides at least second granularity for both + * System.currentTimeMillis() and File.lastModified(). Also that system clock is used + * for file system timestamps. If it is not the case then configure it as uncached. + * Class is solely for package only usage. Member functions are not thread safe. + * + * @see TaildirSource + * @see ReliableTaildirEventReader + * @see TaildirSourceConfigurationConstants + */ +@InterfaceAudience.Private +@InterfaceStability.Evolving +public class TaildirMatcher { + private static final Logger logger = LoggerFactory.getLogger(TaildirMatcher.class); + private static final FileSystem FS = FileSystems.getDefault(); + + // flag from configuration to switch off caching completely + private final boolean cachePatternMatching; + // id from configuration + private final String fileGroup; + // plain string of the desired files from configuration + private final String filePattern; + + // directory monitored for changes + private final File parentDir; + // cached instance for filtering files based on filePattern + private final DirectoryStream.Filter fileFilter; + + // system time in milliseconds, stores the last modification time of the + // parent directory seen by the last check, rounded to seconds + // initial value is used in first check only when it will be replaced instantly (system time is positive) + private long lastSeenParentDirMTime = -1; + // system time in milliseconds, time of the last check, rounded to seconds + // initial value is used in first check only when it will be replaced instantly (system time is positive) + private long lastCheckedTime = -1; + // cached content, files which matched the pattern within the parent directory + private List lastMatchedFiles = Lists.newArrayList(); + + /** + * Package accessible constructor. From configuration context it represents a single filegroup + * and encapsulates the corresponding filePattern. + * filePattern consists of two parts: first part has to be a valid path to + * an existing parent directory, second part has to be a + * valid regex {@link java.util.regex.Pattern} that match any non-hidden file names within parent directory. + * A valid example for filePattern is /dir0/dir1/.* given /dir0/dir1 + * is an existing directory structure readable by the running user. + *

    + * An instance of this class is created for each fileGroup + * + * @param fileGroup arbitrary name of the group given by the config + * @param filePattern parent directory plus regex pattern. No wildcards are allowed in directory name + * @param cachePatternMatching default true, recommended in every setup especially with huge parent directories. + * Don't set when local system clock is not used for stamping mtime (eg: remote filesystems) + * @see TaildirSourceConfigurationConstants + */ + TaildirMatcher(String fileGroup, String filePattern, boolean cachePatternMatching) { + // store whatever came from configuration + this.fileGroup = fileGroup; + this.filePattern = filePattern; + this.cachePatternMatching = cachePatternMatching; + + // calculate final members + File f = new File(filePattern); + this.parentDir = f.getParentFile(); + String regex = f.getName(); + final PathMatcher matcher = FS.getPathMatcher("regex:" + regex); + this.fileFilter = new DirectoryStream.Filter() { + @Override + public boolean accept(Path entry) throws IOException { + return matcher.matches(entry.getFileName()) && !Files.isDirectory(entry); + } + }; + + // sanity check + Preconditions.checkState(parentDir.exists(), + "Directory does not exist: " + parentDir.getAbsolutePath()); + } + + /** + * Lists those files within the parentDir that match regex pattern passed in during object instantiation. + * Designed for frequent periodic invocation {@link org.apache.flume.source.PollableSourceRunner}. + *

    + * Based on the modification of the parentDir this function may trigger cache recalculation by calling + * {@linkplain #getMatchingFilesNoCache()} or + * return the value stored in {@linkplain #lastMatchedFiles}. + * Parentdir is allowed to be a symbolic link. + *

    + * Files returned by this call are weakly consistent (see {@link DirectoryStream}). + * It does not freeze the directory while iterating, + * so it may (or may not) reflect updates to the directory that occur during the call, + * In which case next call + * will return those files (as mtime is increasing it won't hit cache but trigger recalculation). + * It is guaranteed that invocation reflects every change which was observable at the time of invocation. + *

    + * Matching file list recalculation is triggered when caching was turned off or + * if mtime is greater than the previously seen mtime + * (including the case of cache hasn't been calculated before). + * Additionally if a constantly updated directory was configured as parentDir + * then multiple changes to the parentDir may happen + * within the same second so in such case (assuming at least second granularity of reported mtime) + * it is impossible to tell whether a change of the dir happened before the check or after + * (unless the check happened after that second). + * Having said that implementation also stores system time of the previous invocation and previous invocation has to + * happen strictly after the current mtime to avoid further cache refresh + * (because then it is guaranteed that previous invocation resulted in valid cache content). + * If system clock hasn't passed the second of + * the current mtime then logic expects more changes as well + * (since it cannot be sure that there won't be any further changes still in that second + * and it would like to avoid data loss in first place) + * hence it recalculates matching files. If system clock finally + * passed actual mtime then a subsequent invocation guarantees that it picked up every + * change from the passed second so + * any further invocations can be served from cache associated with that second (given mtime is not updated again). + * + * @return List of files matching the pattern sorted by last modification time. No recursion. No directories. + * If nothing matches then returns an empty list. If I/O issue occurred then returns the list collected to the point + * when exception was thrown. + * + * @see #getMatchingFilesNoCache() + */ + List getMatchingFiles() { + long now = TimeUnit.SECONDS.toMillis(TimeUnit.MILLISECONDS.toSeconds(System.currentTimeMillis())); + long currentParentDirMTime = parentDir.lastModified(); + List result; + + // calculate matched files if + // - we don't want to use cache (recalculate every time) OR + // - directory was clearly updated after the last check OR + // - last mtime change wasn't already checked for sure (system clock hasn't passed that second yet) + if (!cachePatternMatching || + lastSeenParentDirMTime < currentParentDirMTime || + !(currentParentDirMTime < lastCheckedTime)) { + lastMatchedFiles = sortByLastModifiedTime(getMatchingFilesNoCache()); + lastSeenParentDirMTime = currentParentDirMTime; + lastCheckedTime = now; + } + + return lastMatchedFiles; + } + + /** + * Provides the actual files within the parentDir which + * files are matching the regex pattern. Each invocation uses {@link DirectoryStream} + * to identify matching files. + * + * Files returned by this call are weakly consistent (see {@link DirectoryStream}). It does not freeze the directory while iterating, + * so it may (or may not) reflect updates to the directory that occur during the call. In which case next call + * will return those files. + * + * @return List of files matching the pattern unsorted. No recursion. No directories. + * If nothing matches then returns an empty list. If I/O issue occurred then returns the list collected to the point + * when exception was thrown. + * + * @see DirectoryStream + * @see DirectoryStream.Filter + */ + private List getMatchingFilesNoCache() { + List result = Lists.newArrayList(); + try (DirectoryStream stream = Files.newDirectoryStream(parentDir.toPath(), fileFilter)) { + for (Path entry : stream) { + result.add(entry.toFile()); + } + } catch (IOException e) { + logger.error("I/O exception occurred while listing parent directory. Files already matched will be returned. " + + parentDir.toPath(), e); + } + return result; + } + + /** + * Utility function to sort matched files based on last modification time. + * Sorting itself use only a snapshot of last modification times captured before the sorting to keep the + * number of stat system calls to the required minimum. + * + * @param files list of files in any order + * @return sorted list + */ + private static List sortByLastModifiedTime(List files) { + final HashMap lastModificationTimes = new HashMap(files.size()); + for (File f: files) { + lastModificationTimes.put(f, f.lastModified()); + } + Collections.sort(files, new Comparator() { + @Override + public int compare(File o1, File o2) { + return lastModificationTimes.get(o1).compareTo(lastModificationTimes.get(o2)); + } + }); + + return files; + } + + @Override + public String toString() { + return "{" + + "filegroup='" + fileGroup + '\'' + + ", filePattern='" + filePattern + '\'' + + ", cached=" + cachePatternMatching + + '}'; + } + + @Override + public boolean equals(Object o) { + if (this == o) return true; + if (o == null || getClass() != o.getClass()) return false; + + TaildirMatcher that = (TaildirMatcher) o; + + return fileGroup.equals(that.fileGroup); + + } + + @Override + public int hashCode() { + return fileGroup.hashCode(); + } + + public String getFileGroup() { + return fileGroup; + } + +} diff --git a/flume-ng-sources/flume-taildir-source/src/main/java/org/apache/flume/source/taildir/TaildirSource.java b/flume-ng-sources/flume-taildir-source/src/main/java/org/apache/flume/source/taildir/TaildirSource.java index 8816327abc..dfb5b2901f 100644 --- a/flume-ng-sources/flume-taildir-source/src/main/java/org/apache/flume/source/taildir/TaildirSource.java +++ b/flume-ng-sources/flume-taildir-source/src/main/java/org/apache/flume/source/taildir/TaildirSource.java @@ -78,6 +78,7 @@ public class TaildirSource extends AbstractSource implements private int checkIdleInterval = 5000; private int writePosInitDelay = 5000; private int writePosInterval; + private boolean cachePatternMatching; private List existingInodes = new CopyOnWriteArrayList(); private List idleInodes = new CopyOnWriteArrayList(); @@ -94,6 +95,7 @@ public synchronized void start() { .positionFilePath(positionFilePath) .skipToEnd(skipToEnd) .addByteOffset(byteOffsetHeader) + .cachePatternMatching(cachePatternMatching) .build(); } catch (IOException e) { throw new FlumeException("Error instantiating ReliableTaildirEventReader", e); @@ -166,6 +168,7 @@ public synchronized void configure(Context context) { byteOffsetHeader = context.getBoolean(BYTE_OFFSET_HEADER, DEFAULT_BYTE_OFFSET_HEADER); idleTimeout = context.getInteger(IDLE_TIMEOUT, DEFAULT_IDLE_TIMEOUT); writePosInterval = context.getInteger(WRITE_POS_INTERVAL, DEFAULT_WRITE_POS_INTERVAL); + cachePatternMatching = context.getBoolean(CACHE_PATTERN_MATCHING, DEFAULT_CACHE_PATTERN_MATCHING); backoffSleepIncrement = context.getLong(PollableSourceConstants.BACKOFF_SLEEP_INCREMENT , PollableSourceConstants.DEFAULT_BACKOFF_SLEEP_INCREMENT); diff --git a/flume-ng-sources/flume-taildir-source/src/main/java/org/apache/flume/source/taildir/TaildirSourceConfigurationConstants.java b/flume-ng-sources/flume-taildir-source/src/main/java/org/apache/flume/source/taildir/TaildirSourceConfigurationConstants.java index 616527606b..b0c934dd37 100644 --- a/flume-ng-sources/flume-taildir-source/src/main/java/org/apache/flume/source/taildir/TaildirSourceConfigurationConstants.java +++ b/flume-ng-sources/flume-taildir-source/src/main/java/org/apache/flume/source/taildir/TaildirSourceConfigurationConstants.java @@ -49,4 +49,8 @@ public class TaildirSourceConfigurationConstants { public static final String BYTE_OFFSET_HEADER = "byteOffsetHeader"; public static final String BYTE_OFFSET_HEADER_KEY = "byteoffset"; public static final boolean DEFAULT_BYTE_OFFSET_HEADER = false; + + /** Whether to cache the list of files matching the specified file patterns till parent directory is modified. */ + public static final String CACHE_PATTERN_MATCHING = "cachePatternMatching"; + public static final boolean DEFAULT_CACHE_PATTERN_MATCHING = true; } diff --git a/flume-ng-sources/flume-taildir-source/src/test/java/org/apache/flume/source/taildir/TestTaildirMatcher.java b/flume-ng-sources/flume-taildir-source/src/test/java/org/apache/flume/source/taildir/TestTaildirMatcher.java new file mode 100644 index 0000000000..4bff8418ce --- /dev/null +++ b/flume-ng-sources/flume-taildir-source/src/test/java/org/apache/flume/source/taildir/TestTaildirMatcher.java @@ -0,0 +1,227 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, + * software distributed under the License is distributed on an + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + * KIND, either express or implied. See the License for the + * specific language governing permissions and limitations + * under the License. + */ + +package org.apache.flume.source.taildir; + +import com.google.common.base.Charsets; +import com.google.common.base.Function; +import com.google.common.collect.Lists; +import com.google.common.collect.Maps; +import com.google.common.io.Files; +import org.junit.After; +import org.junit.Before; +import org.junit.Test; + +import java.io.File; +import java.io.IOException; +import java.util.List; +import java.util.Map; + +import static org.junit.Assert.*; + +public class TestTaildirMatcher { + private File tmpDir; + private Map files; + private boolean isCachingNeeded = true; + + final String msgAlreadyExistingFile = "a file was not found but it was created before matcher"; + final String msgAfterNewFileCreated = "files which were created after last check are not found"; + final String msgAfterAppend = "a file was not found although it was just appended within the dir"; + final String msgEmptyDir = "empty dir should return an empty list"; + final String msgNoMatch = "no match should return an empty list"; + final String msgSubDirs = "only files on the same level as the pattern should be returned"; + final String msgNoChange = "file wasn't touched after last check cannot be found"; + final String msgAfterDelete = "file was returned even after it was deleted"; + + /** + * Append a line to the specified file within tmpDir. + * If file doesn't exist it will be created. + */ + private void append(String fileName) throws IOException { + File f; + if(!files.containsKey(fileName)){ + f = new File(tmpDir, fileName); + files.put(fileName, f); + }else{ + f = files.get(fileName); + } + Files.append(fileName + "line\n", f, Charsets.UTF_8); + } + + /** + * Translate a list of files to list of filename strings. + */ + private static List filesToNames(List origList){ + Function file2nameFn = new Function() { + @Override + public String apply(File input) { + return input.getName(); + } + }; + return Lists.transform(origList, file2nameFn); + } + + @Before + public void setUp() throws Exception { + files = Maps.newHashMap(); + tmpDir = Files.createTempDir(); + } + + @After + public void tearDown() throws Exception { + for (File f : tmpDir.listFiles()) { + if (f.isDirectory()) { + for (File sdf : f.listFiles()) { + sdf.delete(); + } + } + f.delete(); + } + tmpDir.delete(); + files = null; + } + + @Test + public void getMatchingFiles() throws Exception { + append("file0"); + append("file1"); + + TaildirMatcher tm = new TaildirMatcher("f1", tmpDir.getAbsolutePath() + File.separator + "file.*", isCachingNeeded); + List files = filesToNames(tm.getMatchingFiles()); + assertEquals(msgAlreadyExistingFile, 2, files.size()); + assertTrue(msgAlreadyExistingFile, files.contains("file1")); + + append("file1"); + files = filesToNames(tm.getMatchingFiles()); + assertEquals(msgAfterNewFileCreated, 2, files.size()); + assertTrue(msgAfterNewFileCreated, files.contains("file0")); + assertTrue(msgAfterNewFileCreated, files.contains("file1")); + + append("file2"); + append("file3"); + files = filesToNames(tm.getMatchingFiles()); + assertEquals(msgAfterAppend, 4, files.size()); + assertTrue(msgAfterAppend, files.contains("file0")); + assertTrue(msgAfterAppend, files.contains("file1")); + assertTrue(msgAfterAppend, files.contains("file2")); + assertTrue(msgAfterAppend, files.contains("file3")); + + this.files.get("file0").delete(); + files = filesToNames(tm.getMatchingFiles()); + assertEquals(msgAfterDelete, 3, files.size()); + assertFalse(msgAfterDelete, files.contains("file0")); + assertTrue(msgNoChange, files.contains("file1")); + assertTrue(msgNoChange, files.contains("file2")); + assertTrue(msgNoChange, files.contains("file3")); + } + + @Test + public void getMatchingFilesNoCache() throws Exception { + append("file0"); + append("file1"); + + TaildirMatcher tm = new TaildirMatcher("f1", tmpDir.getAbsolutePath() + File.separator + "file.*", false); + List files = filesToNames(tm.getMatchingFiles()); + assertEquals(msgAlreadyExistingFile, 2, files.size()); + assertTrue(msgAlreadyExistingFile, files.contains("file1")); + + append("file1"); + files = filesToNames(tm.getMatchingFiles()); + assertEquals(msgAfterAppend, 2, files.size()); + assertTrue(msgAfterAppend, files.contains("file0")); + assertTrue(msgAfterAppend, files.contains("file1")); + + append("file2"); + append("file3"); + files = filesToNames(tm.getMatchingFiles()); + assertEquals(msgAfterNewFileCreated, 4, files.size()); + assertTrue(msgAfterNewFileCreated, files.contains("file0")); + assertTrue(msgAfterNewFileCreated, files.contains("file1")); + assertTrue(msgAfterNewFileCreated, files.contains("file2")); + assertTrue(msgAfterNewFileCreated, files.contains("file3")); + + this.files.get("file0").delete(); + files = filesToNames(tm.getMatchingFiles()); + assertEquals(msgAfterDelete, 3, files.size()); + assertFalse(msgAfterDelete, files.contains("file0")); + assertTrue(msgNoChange, files.contains("file1")); + assertTrue(msgNoChange, files.contains("file2")); + assertTrue(msgNoChange, files.contains("file3")); + } + + @Test + public void testEmtpyDirMatching() throws Exception { + TaildirMatcher tm = new TaildirMatcher("empty", tmpDir.getAbsolutePath() + File.separator + ".*", isCachingNeeded); + List files = tm.getMatchingFiles(); + assertNotNull(msgEmptyDir, files); + assertTrue(msgEmptyDir, files.isEmpty()); + } + + @Test + public void testNoMatching() throws Exception { + TaildirMatcher tm = new TaildirMatcher("nomatch", tmpDir.getAbsolutePath() + File.separator + "abracadabra_nonexisting", isCachingNeeded); + List files = tm.getMatchingFiles(); + assertNotNull(msgNoMatch, files); + assertTrue(msgNoMatch, files.isEmpty()); + } + + @Test(expected = IllegalStateException.class) + public void testNonExistingDir() { + TaildirMatcher tm = new TaildirMatcher("exception", "/abracadabra/doesntexist/.*", isCachingNeeded); + } + + @Test + public void testDirectoriesAreNotListed() throws Exception { + new File(tmpDir, "outerFile").createNewFile(); + new File(tmpDir, "recursiveDir").mkdir(); + new File(tmpDir + File.separator + "recursiveDir", "innerFile").createNewFile(); + TaildirMatcher tm = new TaildirMatcher("f1", tmpDir.getAbsolutePath() + File.separator + ".*", isCachingNeeded); + List files = filesToNames(tm.getMatchingFiles()); + + assertEquals(msgSubDirs, 1, files.size()); + assertTrue(msgSubDirs, files.contains("outerFile")); + } + + @Test + public void testRegexFileNameFiltering() throws IOException { + append("a.log"); + append("a.log.1"); + append("b.log"); + append("c.log.yyyy.MM-01"); + append("c.log.yyyy.MM-02"); + + // Tail a.log and b.log + TaildirMatcher tm1 = new TaildirMatcher("ab", tmpDir.getAbsolutePath() + File.separator + "[ab].log", isCachingNeeded); + // Tail files that starts with c.log + TaildirMatcher tm2 = new TaildirMatcher("c", tmpDir.getAbsolutePath() + File.separator + "c.log.*", isCachingNeeded); + + List files1 = filesToNames(tm1.getMatchingFiles()); + List files2 = filesToNames(tm2.getMatchingFiles()); + + assertEquals(2, files1.size()); + assertEquals(2, files2.size()); + // Make sure we got every file + assertTrue("Regex pattern for ab should have matched a.log file", files1.contains("a.log")); + assertFalse("Regex pattern for ab should NOT have matched a.log.1 file", files1.contains("a.log.1")); + assertTrue("Regex pattern for ab should have matched b.log file", files1.contains("b.log")); + assertTrue("Regex pattern for c should have matched c.log.yyyy-MM-01 file", files2.contains("c.log.yyyy.MM-01")); + assertTrue("Regex pattern for c should have matched c.log.yyyy-MM-02 file", files2.contains("c.log.yyyy.MM-02")); + } + +} \ No newline at end of file diff --git a/flume-ng-sources/flume-taildir-source/src/test/java/org/apache/flume/source/taildir/TestTaildirSource.java b/flume-ng-sources/flume-taildir-source/src/test/java/org/apache/flume/source/taildir/TestTaildirSource.java index f9e614c332..f6289cd15b 100644 --- a/flume-ng-sources/flume-taildir-source/src/test/java/org/apache/flume/source/taildir/TestTaildirSource.java +++ b/flume-ng-sources/flume-taildir-source/src/test/java/org/apache/flume/source/taildir/TestTaildirSource.java @@ -77,7 +77,7 @@ public void tearDown() { } @Test - public void testRegexFileNameFiltering() throws IOException { + public void testRegexFileNameFilteringEndToEnd() throws IOException { File f1 = new File(tmpDir, "a.log"); File f2 = new File(tmpDir, "a.log.1"); File f3 = new File(tmpDir, "b.log"); From 4a7db4a363a7ed7391e11685bb6a0f47cbfa2328 Mon Sep 17 00:00:00 2001 From: Mike Percy Date: Fri, 24 Jun 2016 16:39:35 -0700 Subject: [PATCH 290/341] FLUME-2934. Document new cachePatternMatching option for TaildirSource (Attila Simon via Mike Percy) --- flume-ng-doc/sphinx/FlumeUserGuide.rst | 4 ++++ 1 file changed, 4 insertions(+) diff --git a/flume-ng-doc/sphinx/FlumeUserGuide.rst b/flume-ng-doc/sphinx/FlumeUserGuide.rst index 74d2887c72..affdf66797 100644 --- a/flume-ng-doc/sphinx/FlumeUserGuide.rst +++ b/flume-ng-doc/sphinx/FlumeUserGuide.rst @@ -1126,6 +1126,10 @@ writePosInterval 3000 Interval time batchSize 100 Max number of lines to read and send to the channel at a time. Using the default is usually fine. backoffSleepIncrement 1000 The increment for time delay before reattempting to poll for new data, when the last attempt did not find any new data. maxBackoffSleep 5000 The max time delay between each reattempt to poll for new data, when the last attempt did not find any new data. +cachePatternMatching true Listing directories and applying the filename regex pattern may be time consuming for directories + containing thousands of files. Caching the list of matching files can improve performance. + The order in which files are consumed will also be cached. + Requires that the file system keeps track of modification times with at least a 1-second granularity. =================================== ============================== =================================================== Example for agent named a1: From 2f731f0aaeb490ab72400d8c83074daac840ac44 Mon Sep 17 00:00:00 2001 From: Mike Percy Date: Sun, 26 Jun 2016 14:17:37 -0700 Subject: [PATCH 291/341] FLUME-2935. Bump java target version to 1.7 * Renamed README to README.textile in order to allow GitHub to properly render the headings. Also changed the min. requirement to JDK 1.7 * pom.xml: * Changed the sourceJavaVersion and targetJavaVersion to 1.7 * Removed the test dependency hadoop-minicluster since it is already required as a "normal" dependency. This was causing Maven warnings * flume-ng-elasticsearch-sink/pom.xml: pom file had two identical dependencies, causing Maven warnings * FlumeUserGuide.rst: changed the system requirements to Java 1.7 * KafkaLocal.java: The tests threw a MethodNotFoundException, since there isn't a constructor for KafkaConfig with a java.util.properties parameter. Instead, I used the static method KafkaConfig.fromProps. (Lior Zeno via Mike Percy) --- README | 50 --------- README.textile | 100 ++++++++++++++++++ flume-ng-doc/sphinx/FlumeUserGuide.rst | 4 +- .../flume-ng-elasticsearch-sink/pom.xml | 5 - .../flume/sink/kafka/util/KafkaLocal.java | 2 +- pom.xml | 10 +- 6 files changed, 105 insertions(+), 66 deletions(-) delete mode 100644 README create mode 100644 README.textile diff --git a/README b/README deleted file mode 100644 index a86e09840c..0000000000 --- a/README +++ /dev/null @@ -1,50 +0,0 @@ -h1. Welcome to Apache Flume! - -Apache Flume is a distributed, reliable, and available service for efficiently -collecting, aggregating, and moving large amounts of log data. It has a simple -and flexible architecture based on streaming data flows. It is robust and fault -tolerant with tunable reliability mechanisms and many failover and recovery -mechanisms. The system is centrally managed and allows for intelligent dynamic -management. It uses a simple extensible data model that allows for online -analytic application. - -The Apache Flume 1.x (NG) code line is a refactoring of the first generation -Flume to solve certain known issues and limitations of the original design. - -Apache Flume is open-sourced under the Apache Software Foundation License v2.0. - -h2. Documentation - -Documentation is included in the binary distribution under the docs directory. -In source form, it can be found in the flume-ng-doc directory. - -The Flume 1.x guide and FAQ are available here: - -* https://cwiki.apache.org/FLUME/flume-ng.html -* https://cwiki.apache.org/confluence/display/FLUME/Getting+Started - -h2. Contact us! - -* Mailing lists: https://cwiki.apache.org/confluence/display/FLUME/Mailing+Lists -* IRC channel #flume on irc.freenode.net - -Bug and Issue tracker. - -* https://issues.apache.org/jira/browse/FLUME - -h2. Compiling Flume - -Compiling Flume requires the following tools: - -* Oracle Java JDK 1.6 -* Apache Maven 3.x - -Note: The Apache Flume build requires more memory than the default configuration. -We recommend you set the following Maven options: - -export MAVEN_OPTS="-Xms512m -Xmx1024m -XX:PermSize=256m -XX:MaxPermSize=512m" - -To compile Flume, run `mvn compile`. -To build a distribution, run `mvn install`. - -The final Flume distribution artifacts will be in $project/flume-ng-dist/target/. diff --git a/README.textile b/README.textile new file mode 100644 index 0000000000..2b884fa16e --- /dev/null +++ b/README.textile @@ -0,0 +1,100 @@ +h1. Welcome to Apache Flume! + +Apache Flume is a distributed, reliable, and available service for efficiently +collecting, aggregating, and moving large amounts of log data. It has a simple +and flexible architecture based on streaming data flows. It is robust and fault +tolerant with tunable reliability mechanisms and many failover and recovery +mechanisms. The system is centrally managed and allows for intelligent dynamic +management. It uses a simple extensible data model that allows for online +analytic application. + +The Apache Flume 1.x (NG) code line is a refactoring of the first generation +Flume to solve certain known issues and limitations of the original design. + +Apache Flume is open-sourced under the Apache Software Foundation License v2.0. + +h2. Documentation + +Documentation is included in the binary distribution under the docs directory. +In source form, it can be found in the flume-ng-doc directory. + +The Flume 1.x guide and FAQ are available here: + +* https://cwiki.apache.org/FLUME/flume-ng.html +* https://cwiki.apache.org/confluence/display/FLUME/Getting+Started + +h2. Contact us! + +* Mailing lists: https://cwiki.apache.org/confluence/display/FLUME/Mailing+Lists +* IRC channel #flume on irc.freenode.net + +Bug and Issue tracker. + +* https://issues.apache.org/jira/browse/FLUME + +h2. Compiling Flume + +Compiling Flume requires the following tools: + +* Oracle Java JDK 1.7 +* Apache Maven 3.x + +Note: The Apache Flume build requires more memory than the default configuration. +We recommend you set the following Maven options: + +export MAVEN_OPTS="-Xms512m -Xmx1024m -XX:PermSize=256m -XX:MaxPermSize=512m" + +To compile Flume, run `mvn compile`. +To build a distribution, run `mvn install`. + +The final Flume distribution artifacts will be in $project/flume-ng-dist/target/. +h1. Welcome to Apache Flume! + +Apache Flume is a distributed, reliable, and available service for efficiently +collecting, aggregating, and moving large amounts of log data. It has a simple +and flexible architecture based on streaming data flows. It is robust and fault +tolerant with tunable reliability mechanisms and many failover and recovery +mechanisms. The system is centrally managed and allows for intelligent dynamic +management. It uses a simple extensible data model that allows for online +analytic application. + +The Apache Flume 1.x (NG) code line is a refactoring of the first generation +Flume to solve certain known issues and limitations of the original design. + +Apache Flume is open-sourced under the Apache Software Foundation License v2.0. + +h2. Documentation + +Documentation is included in the binary distribution under the docs directory. +In source form, it can be found in the flume-ng-doc directory. + +The Flume 1.x guide and FAQ are available here: + +* https://cwiki.apache.org/FLUME/flume-ng.html +* https://cwiki.apache.org/confluence/display/FLUME/Getting+Started + +h2. Contact us! + +* Mailing lists: https://cwiki.apache.org/confluence/display/FLUME/Mailing+Lists +* IRC channel #flume on irc.freenode.net + +Bug and Issue tracker. + +* https://issues.apache.org/jira/browse/FLUME + +h2. Compiling Flume + +Compiling Flume requires the following tools: + +* Oracle Java JDK 1.7 +* Apache Maven 3.x + +Note: The Apache Flume build requires more memory than the default configuration. +We recommend you set the following Maven options: + +export MAVEN_OPTS="-Xms512m -Xmx1024m -XX:PermSize=256m -XX:MaxPermSize=512m" + +To compile Flume, run `mvn compile`. +To build a distribution, run `mvn install`. + +The final Flume distribution artifacts will be in $project/flume-ng-dist/target/. diff --git a/flume-ng-doc/sphinx/FlumeUserGuide.rst b/flume-ng-doc/sphinx/FlumeUserGuide.rst index affdf66797..287d066e0f 100644 --- a/flume-ng-doc/sphinx/FlumeUserGuide.rst +++ b/flume-ng-doc/sphinx/FlumeUserGuide.rst @@ -50,7 +50,7 @@ in the latest architecture. System Requirements ------------------- -#. Java Runtime Environment - Java 1.6 or later (Java 1.7 Recommended) +#. Java Runtime Environment - Java 1.7 or later #. Memory - Sufficient memory for configurations used by sources, channels or sinks #. Disk Space - Sufficient disk space for configurations used by channels or sinks #. Directory Permissions - Read/Write permissions for directories used by agent @@ -1093,7 +1093,7 @@ deserializer.maxBlobLength 100000000 The maximum number of bytes to r Taildir Source ~~~~~~~~~~~~~~~~~~~~~~~~~ -.. note:: **This source is provided as a preview feature. It does not work on Windows.** This source requires Java version 1.7 or later. +.. note:: **This source is provided as a preview feature. It does not work on Windows.** Watch the specified files, and tail them in nearly real-time once detected new lines appended to the each files. If the new lines are being written, this source will retry reading them in wait for the completion of the write. diff --git a/flume-ng-sinks/flume-ng-elasticsearch-sink/pom.xml b/flume-ng-sinks/flume-ng-elasticsearch-sink/pom.xml index 0e2b751d91..c372c0b734 100644 --- a/flume-ng-sinks/flume-ng-elasticsearch-sink/pom.xml +++ b/flume-ng-sinks/flume-ng-elasticsearch-sink/pom.xml @@ -83,11 +83,6 @@ guava

    bq?@Q1)c3-n}MBP}+m(xz1P%~lL)U>0H?@L`P zoJ8-R<@_6w>`27KzrDAe+L7YPw_BsPrf%u|vswRZPtJzI_fP^q@Tm7heBkw;9}elw zx4&QFy^0wmE~31uL@6mNszY^Dw#A_OL`Wa3$^TLiB43Rp{4Z~TQ@bsx&vd*6=}(2} z4sr$prJw|2?q9A7Nx?AN-8)SZ3&QNQ)|gvUx6J(??EkffIm7I6N`R?XUo^Lj^Z#)D z6O8lq;c+gd@qeZRh~<3be1Ah_g;g56uaV2QXSMlJ*hp#oAI^U)jrE%c2&6CmZU}Mx zOL1yg&hvgmmH-jJ5aRw<64LJo2(MWS0EaO8wJl(nM@Mtb>TA0M3+hlijOv9VN zy#&GRe;Cb2WPrd!Dg5$7L2V_FjXvqcLfXSGya*vS`}J>tqRmphGvL?jM2lqa4A>Tg zsKxWbN}cbY!G9a&xSR^xW_L?^Li}&8j{!A+!IQ-N{9>!|+b%vN)KIi*2#(8A5XzEwNnh(4fvE^UChV7Xic=zp?*KDv>uV7%Hg$64rwsFFfZwp(1|&?Ojlz z-}TD`@S|K-0R2w=89;le09qdoAmR4=K%1MA`H9LOUjj8t=HK@enIrE6FjVGSRC4-K zmVwu7DikUr^Y80KX8qy-0_lstORE3-*9}ew=M`_JQ`_Cee_{DFQ{|-M z=Hg~los}(SJS?B$i3>u~w3bi*W&bbW#u9ya94{F3o+LXg2!~3EPCTnb|F2p9YbjZx zkBtGQJ<)hEtJLCyxcx8J{{atb^r|<@Z+v*O($@%Q+EAnLig1pwk~P$Se+;psO1s5T z=6X+yO2b7Z;#n&FF*t-rmG+1Mrh~20zFMU}%>KW?Nw0bP%3IS$I^MqNtAyh_s8X;b z#6}wRb`4SIkHi_)nXE1!Sb0~lK z@?5?D3D(xj^E9m;=twI<$IRg&>MWa{pdGSoYwO(KF>@|pIn9hy)6B+XWm8Ao4c4<> zi_GJbcR@R5VgH{`rQL6&vdcv;nU$l+oW=Z4nI|TZSsw_|F^lyN z=ZJ|8XPON-y{uwq*+03Pv$G1QvchI89qXH03|BVTFrddwoH|Xlq&NqW2U7{$ibBce z!}R0Dabl(SF#U@8u2j+w@vQWd&G|o&kCVgr7+|vPaF{MUQ2$!6)TMHO?5P~)3`k!W zJlsTGq`AiS9XS|HSpNhPQIqO84vF&a7fptUCZ@e8ft}MMghx###0X<+Qm-}1q5lIO z)TC-hr&dk!yE{9kuL&9>)MQ-s)M>S2YS7P()l_T$usKa`qE(LB|EuGGTE*fxR?gcf zS`~{{rm1bT`e#T|YBezi7{AyhoSwWp=58H=rB*|CaB9`W=s7#2uNC|4&}!;5C{&|6 zptFRHsFQnCh!xd|$MJ$x??F+AZ?Y#HGUOkW*TBBIVTee;0!A`?-j&BPERDl;mMBfGo@M43WShIp1TZA|^AeM)5} z#{lEn%19n)6ZSuq8NQQK+mWEVIXk7VjN}3J<^UR)HLsNi+V+m#k_Wc4qynoa?eH`fqQc-0eYE zbYarITt%z8qGTC#MLV{8J)}&#Zs7N-Vsdq}riuF-@zV_hCqs#W2aKc;$Q_3R30dn z*pK6lJ9QHllI*1#&OHto(P)HIM^>=%(IJN2)4vP9$($Br=Tf3V_ zDXgNtV>tgaxUTo`K6}%8u5XRCHFe9{ow|f{WNba9B(SY5qr1~?fDK0Xn(*kB0=X0F zU)9Q*n4&=Lg#AB$fvk)AwugMieWzs89UqM1PT2pT3S|9KLj0p#{N0Gd^)JPlq>B1= z$y!+*J8ibP6_CoH-v`sgq#`nR-b!SSyf08#b*`|fPBQp)b0a{6E!c!oO_52CYxLgXKLhXoVcChCxwPX&LS)*SDCjQBO@^K{cUf zhOTwzf*{#QJ=NtBh{HsxS+SlP9p4R!^cfdGSc~i?H3;#nda8?=|1=(Jkf|}is3HlV zDz}I_T{A;G*b2G4i&NW$96YBBx^qr@%BQ*@SZJg_bThqEK!a=~J#{tr|Fl?4ls|g! z2I=WtHFVRm9J?Q+ZVg>5O>JW_QQl9OOlrc*N|aqq|3^#0Id^;S94WkzD9=%nR24N> z;-*nUi$3H8^*N|h5=oTZ%>1XRGg02-y(Vi_B+A#)kC61Ot);LMWj9m*X(=W{iX!hO z*){|nZ)%OT107{r>Td3TXzEOqPty*e<*tnK?#=KFGt*Qx%3nyDD7&8oAP(TEDF$%& z>#YH-6XogmcF2d>$wrb+_ZLBh16aS+6FV)d8Qh~B?>!Y#Y30MR!4+jH(+n>Fz*dXJ;NOD zjr4WGQBv3!IVz@B%JV^8pZ=ETD?tomvJ))(U&;TS7w7FJt3lq0H-|%ve!| zc(yWMgz%`$^cY}z5sepzHCIsZOHb4P36$B5X<8k%hXvca=_}LC=>lbvG<6mWJMmG^ z6GGxqtv%wSq1G#+)@V_Sc$Qi{H-_-2*4{C|sCNtiRH-!^-&E_X5G=!O1bfozwi$+y z?n$>|2#uQ<-?Nr8qqN`Vfe(3!PTR2l35G0n+A}@^Iz1~ojS!uPXX&(!x&NWfPn~AO z0Hb>60JU}6Cagbo8p)YP?HGnmdZuZuPGcwdI^kxxty8bUh)(0;BcaoCqSHvxiFlSy zz1aT_Y(bs&i2+8XFQXG1s_;{RD^JxAxEaXsqo2@%lb#jsbF2G~ z;-=oF{t42<`1mNO^pdDFN>n1ArBZM1{|&1&GX@xCdVnf7?T>G$)O$_{mMT^AwvP%^ z+_8NpeU)&>)~GaXpGmq4w+LLw$!PRyh#LcMLc9hVy(}8lh(^S-H0skKghwas8v~3o zJV0%Y`c#ErsZq>d^FpJyTB}j`hV24yvoz{+Rfrokni!8mqZdV^xM)N?Tch_wc+_ZC z3@~cQ15|0m&Nh71w|xkf8pXXgWp#^9uWzQW5ju~8kUj3%; zQgE`S*ZyYzYmf}4#K%CH*F>2yq73mYW%`Hbe=1WO1B_~o7-gi{+JBj?xXY&RJ#1ir zvT9E121a3daZ_n=W6QfBaaL*>kVv>UK8UUUtbhHi1ZUzJ@3HiAMS6 z{k*gkg_dRlx&Nbw5BDjK@y>8+hcW!1(HX6AfTnIaVB2v1l8b3C6yQI9&n{Cg`&DjF;5@IY6~ zk0Y?F@efEr%F@-?J{8-JLprcrOKsf(1#Ga)c{EeN2lQBMEcxa4&5FX=Q0Ep-qPzNA(2&Sj#Y7D^r z-?4zz=LzvI=aVt}{0m=X!wvwcaQftTCd zl*~@W&zyj%l#%%>D`}tB2Vhl zJ1sq1lP4*Kl_z(4D-I@14nJ4#6GJcIX z{}yrf6>*Z~$z6Kd+zP3{8Sap5L8XBXc8qs;rJka?H=Eh-(3AeHy``Qa6zA~NrC zw8*?)5}Ee{EKInIS^w-OIj4j3hPS`ma;kQ`{Z-B>F0E`v&T09U6Yl23Cx@Q^=A-^( z#2f-=oY=s=O(x>?x`c&r)TO+5e^S zs7iedFlyTXK$R-e+Zkl)zrZ(dd9&r9Sad>Wr>_ck<4;c5y1o+JtWL-v)Bg=LnjW73 zjouNBCWuDFvo$jP-$0{=7+_RVG8*w#9h@`R^nU}5-uAF%M3tPl!HF#+NK9!(8@H}c zuAQ>igxczTPpHpH{EDKzt65Ngdaz~m?KO<~yo%meRvwQ@%o+SImIo1p1-NP;$~{}Qs=%s3wW z_1+e3#)~$@v$QGNHiSpt%!vU;RV@HjzG2mdfm0OrKec(EogMXzJxas+^tC~0h)DB0 zyoZ4SnNTn?x-%pcU9)d|5|nvQl$j*T5YJZTn-CtAnHvL4f$bV*8}KOZ9fGAYA8-?w zn&Oey8q!xr=e4~jPT8|&ipkW6g9|x{M#Y@}!;{Xe_+)7GzGyUAG$Nj*QSn0|JZdyA z2AFMajim8b{4X2UW#jE$YBZRz*cI5IN}`6c=0=>|((*<-G~UXAWHsJO_6QMYO3UOao4k#=sE`|Z?1$H95DBPcR)9fJS_*5JsC0)H^tpR2N;c7>{ZjMJw@xlhrGm> z1E4%ioOy_P^*>i=`=wZ+RiXdkcZD*Z2bS~F&UPyAIvM2|>clic8t9D?=v!U4 zOhK0g(EWw>*AeL39Jdrd=46*T8l0fLH2BcIKKMd3C=2ng+ac`mr7l8kg%5-tYVv=r zvsgi9IDgkb*0bRqhaLoa^hRx>H^e?e>3)YkoS7au^F0Yu})lsgw6Xfq-^ z#Di1z)t%&7-SJK*^)R$*0w>PftmV_6OfqqsM)Q=(WQ4i@sb%EU{U7h6U=!{~YQjv* zifjmzIw{L!GQ!;d)YNBV>VAkYp}U15$;jN01&RPW&^u3dykS~9Pf3D@CQ1v7o3{Mi z#MdIp!T5%KM>d71bLzgIJ4@6O5>vO{S(28X>07&-M=2TT>@xv9`zgk6BDxn^X4-lpu7LfFvM~>mQEzCN<){q*J$dK7g2jnob?Toq(E?3}z9hI!nZ97IBhI-PJn@p*cyWrX9c#ey$!57}{Wt znwmJ~4jkaBk=X%XZS}EhJb4YTIFS$0j)|bn-`-_63kOTYR;5IP{t4U;+RkO$h-gQo zMEfp+wqRSAOJ<@So(e5S{lmH~EO6O7AlhLm(Pl@`7UjF_1`+MhlxXKf&=&W1*{>wp z%9LoEB4|tcx$M>u?T}Pxag+Z7zb)iHUY_O0=aBwB@~B z-k>Ae@(7wlY23d5;a&=8XXWpA9E~pmNCd`t|068Y!M$AW^Cj(N5!!+_Dkp-rVjGwH zmx#7BB^vL4gt;Bk(~T|BXiHL}EsLP7+}h=?U2S%ytlQz;UG5zu+JXq0lub30fX1xVoFZ%0EVi;%yLUm(Ilg{;9lk6# z)|?$-endByTm8uVpa^quV-4${u*|Evy4&f=B7kr{tuhvXu!|WXme7b?QZV> z1R3F&JU7;?(VA1DO^9$iw!O^iWF;^*n*UC7Ty-u-$83Cs@Q5Sp%dsE(rexAP4WGiGEVI{_dvx$2nT%P#tiv}i zYkj##380wb<|L*#sk!!G|C5Wpc}YqoSzGU6?*GnHr4ozthjU$3!nehju9U!+;0DF$ zAD}#v6K5P(=$D7cb_yYJtPAHd+x?R};fvy%WKD3Educ&Vlu>6jgkRQQ%10(2jjrDmdtmzS~H!+Al(^#LS!de@_dJE&|Q;qd$DlF6g4=n#S zhSDb*>yuPiKSy}IgVFS{#`-uBOA4%sgCkh)VnBTatPa`uveAel8F->u{~xG2eEImL zJa}ls&cG8-^%*b&Gtl3|H!mv#PrMya98(9T7*i8p0|3X=A~mKo0scQTG%GC(---VL zjH6(_!>+7JXxrikd^ub#;Ofl(BSId2Y4bT=e4gMJP#dlVgCSGZrU01JBfZI&dlO@F|V=^Mgy32lQk zp6Nd0ouOLzj!~DOqV5#a|5~CVbT)q30c4V>JEcg}U6QEo2|xnpDN}5973XL12!Pgi zeir`B&PRsczO1OYtf8#s^AYmlA$2-SJ}A4>6ka;#q$}_#;7OULiCo72iyYSDi$U?U zeB7bN-%gdA|GymsQ>S@MHTC}@hif(RSVtbT<7l40p9%H(|0n$~C(n=>0}{rb`e4Y| z?4bUNB&JVfDRsP$Gg@J5EuN+=p*F2Q=rh!oCJD6yEyJ|9$gos}8mq?BV{2T`*@)l$ zU?Z>u+cZ=EE>*#H8h$B4l1UP5(=G;Sj;*DM!Nx8Z^q&^ipS;hAt-)^`Ty_K$yME{a-cEaZ{j^v)eAx zMjzqqlKE#P$x|AU$xgzE`Jc&2GvYmg{o^?0$NSg?K%NudE2DraXhb53x%T=xgh+Fp z89Nf160gpQ9oZUd*=NCNgCW?qhQYAoBtE8^!_U85TFaP=>m!(J`e7kqXs!{v`diO; zZH>7SMaiR5YKyyAY`VGs7g($|J{=bOgqp)*(~ZT5NF;tk$h}SdqY-Jbvtmcdww@*@ zc2sLD7CsD6+YD-MImjr?{vyyV5vz?HBvy+%gIZ&?s5p64O>q*t&EWVCN80Q-9w_oY z6}#_$W)ouI{>F(ZUXyPX|78YU&)d?$8vYwVWvXh?0otaQvrnV9q8&l!0B#Zrht z=M5I~?coe=jrmeal}AMtL$TsMrv3|}uP#0VR{Tt?IKx$pR0JIuqqFkH8Ba;I10A=mJTU1LTpMPX|K;>;Wr5{JgC9wIl% zJZDI2jF%ux9<@?z#AGvB{{@;e#ns36fyq7>lkH{sqo%E#AGv# z$%sgk$s|9E{a;S<%TiDBvvw5X%hV*#g~|KgH?9}1C<{PTCiz+W_z3JH_wS1FrEcOjH5HG2$Dzx{>Ts=27NkfIiq)(@&EE^Hpu z+*E>RpLx}nHx-o>7b?l$PatmgKL(Qjf(LLt-ot}gR%qgt6=kJO6?GQz#;xv?ISep!p3r}PVIdj$>n|} zEO*`bMB1)1*u`!KMSHo>u5WH=D%-LzA55Ta?tl5dblsg?Z>6TaQqgW~sB5SyF1ELp zCYPIiTFWN&*%7%jc67Z%HSI$+?fP=qq{Mz`JUQ)W6KI?JKY`qt16-UftZcGM(=IKq zuPkh^GfMJ7&s=HCeSbvmtbVS?<_~;%u%=yDTG7x{Za;^cT<(X$v>VL*-$3r{zAlc( zP_*^Y7!6HzjmW2VnoTbE+X=L}|Aoa(KsyJGPHt0x+=nQ+>+2g!8;grewm9aW657&^ zYGD1t*>rj0Y$`2-2DAS|&!(6_++6S2pk;7uM%uF%YocyyCMIo$B9diexc|e=P7m;4 zAa(G}emoec74vDD+noYID8ytD)aG3iprLN+4r=pEADSIh(xw@b z6oAdg0FohlaAL@=1rqdazCuVTPl6N(l*`ogw=D zV@7}cKp83PG_?^E{~`Tf{IU`Moh9q-#VYD}cp+H7?JTbGZRNOrs<;bO*QAXoN_92OiQcNF=go=`zswu$daTkw{3Rh?*jQ9tY(Hd_e13W+ zf%x4K;+M0UepM6y6E8iqBtKxQ2=ObtnCsoHiQldzFRd%CEN(W{MryOL6KE4R``-ea zUx|a^Jr)iy%d?tzNnJ@pW%HKHB#K1hEDJEw4tOCV`Bfub98#`mzoKXtH&>QdVi{yj zt;whDcZIIdKG5X9K=7-v40vACeqLx7HIz1$*;~GoyK`d#?Gqw$U$dKw(|VQM`lR0C zrutGm-fs?qN^Nk?4V2l){YixOwL@L+VNLsCEq7I2d1G;fDK}EnJ|Rqdz89f=U8U>Y zt!dw_X~U1Db(Ln>m6|rQr6u?L3nR3zFLgZ@uZWYUHSOYxrpn4D`?a3Q?M+MCw11A! zzM;hRp3=0R(zJ_;@g~|PySz(I`^GTsgSh{NZel1los3v=>EaKiJLn?$EUF(6k%M>l(_MwrrEV5@(S$o`%X=}zOft! z9Beth_YTuuwqu0$L!DiZjXTi3O=ve17vi0>Ta51s3ADNX3(uDicXBIMvolnjU(_TJ2Li^DUu6K{7eUGMH zSyf+HS7plk)MI{L0&R2uD;V>SVHe_~n)aibHX2(+#Z_Cj_o)fAzmLfMc#ezn50t(C z#MuXyy$_xgq5VX*>#;(DFK^UxhtE|=Z4<7hZQ9pIXg`Tvg`8DE`*BUXxT>hIu+-Q) zwJ$Fb+FkICb;F7_pv@k{io_noBmpqbKbf{!hWnKN3?G_FZ|Y`JI?Ghut-|tO?c!nk zh60=@%4z!5I5gs2Z=M_;u&3kA%S^e7Z&j0wY{_!6W{b3{$6nARyP0QIV6_JT=ry;s zO)@edaan~Q!u~H;;fEw?LluY+$BGb#sGEyC-{V^@Hv1=)#U8Z+Os}1oq9KD_ zZj)Nk3IceIzA+799cm65lZ>tUThpV7K9 zNwiAS^e{gE!0BowqZK#6Go0Hi+j*bak9{_>h>1e?`ohjm>1wd1pd6 zI&Zn-otdq|ufD3WuCT1-u(_SU?&koV&Ca&{h z$026v>3S!2T-#Q7np@4{e0km2QF^X)Q|uWeaerSgCD{)j82ZTVL(*)(e$gYnxV+=K z+VSF;>)^-qv`vsEk7_AKI|;LOA!gjg)fnFwGxrx14?9)&%_yK68j(mx3F6ZsMB429 z*zt1QMY9t-zBM-E*3h_rNUSy+e455~K}s`QOLUZ3|LgHeyB+1>#X9QLuu+aTsx@|t zN|Z;q_RtOFug z^Ktejj^I>S_UW58*7^w6K{$sP#}O+m`y^r;>$M2h0v!2^lYbSKedw=+bu7lUnk55k zEW}ysIC5QK*{7}BSbYA6b8CRL2rpxckJVWA>)33pLn2s<@s64J7>#AWWyZ$3F@m)O zFXW5wuCaEPqkz!VQ;|zd#Qd?JM6j0Py=w8%8q0pOn#Jq5UJz6jO|yh14+(^&RPlx!@s|0R&{ z5WER6Uahh0cL7>h$D96FfVC1YIEs53%M;6UjpMW(g!s}@7Q8!+$vC0LjE&=P% z2-Y#)BW}D*W0eWk>;$ZvBUs0J54-VFja4dGwFy|J{txsz&U?s>muRdK!D4Y~dHHXB zBD{|GHoEa*ja4jItdeXjv;QmLb%OVx8!ysWMS?Xe0qcYauM@oo+<2kJDikbscWqt| zM6mwm-S5VSXsjWEH8TP0`v}%a-hFO-u*MoJSo?&rPTDzwwbr}WjStdTg9K|v0@myZ z*2&&IZhTjbwX0z5oq%QP-@t~acz3(;T{PA%f;Bw>i}g=9Z=CAg<;Hi`SUU?AmsOSx z*JebpPV??`<18%^^E(L^3nUxM^nU^g*Lioi@f|hRj)KJk$;Mg`;dQ!qyBps@W9=YV zERbxh%OhB4cz<`}1c8JD1&alejm7n!h9H@>~b z+Fr0&AlX=>BUop9x47}`G}d;4#RAF3Iy{1Pj(4*g-&SL7D_AU$Y^+-&Sm$~-x$y#x zRUlX_kZi1fM6k~DZgk`M8Y^G0SRh$gr}T?po$uY?#`|lm{({8<$;O%%!Meb^-i`Ou zSp5Wx1(J>Rw+PmS-gR!At7mw*uVAr2vavQsurBhhb>n?BRv*D)fn;O-7{R*OyT*<8 z)>yp-Yjhav)LkQ3mv~pZad!G3VK2d|Nx-U)V6FGAa^u@*tZf8~1(GGGVgLXzLmz>O0ZZU*;qp(SeJX3 zx$zzvtA}8D30R9FSXX$Lx^XViV8iZ$#RAFZb!7zWN^gT3@20W32^I?^8|$42)>Ynm zH{MlabrmcYNEX(*t`V%Oy-VD97md|Luvj43SYsks*LWAZaV|okS7*Uufn;N?ieO#q zUF60)X{=6y#RAF3`g;WHI`2X^-ce(96f71riv<#~=tiuGPX8HL>|7p^*tz7q2Y@ro{(s%PV58uc_$>5cE_MNcKFq9) z0^Nr>9{)|G)%Xz6)IS=LeV7YkD6`c&An*X<32pNsNb9;q)v<9kx(Sn+stH4>))ep( zQCOByoDao3K@;X^5AOn2=W?QXRRfBDR-8QQWjC+RoDgD0qum!jv5u@hTkG`wvB%`5+X!rE|IEf_B24`VorQ>R5b%dCF` z!yV({jpWL3HI6sBHHJ$fQXchF%xDML;4I$%VZjna+--5Z;|VDX05IPE#&|>|(i?*K zXb6$J%Rllq%QNctIo{?r%EjsQnOhMHZNMB???tV^u^*2tEh%j(#ba$P?!whNvPOe} z&b~gRXAy2;%c! zj^(VSF5dU~3m;rL7xn*G5Vjs%Sr)cNhprLN++h6B(^Pv}N>%a&}Z-1S~^Zz)BtV*0j)@oJGGyNasNEjM~a_|$) z{Z~G?qVxyN4E3=eTv;UCGB)Cm(1R=I{m1e}cb1>d&l~1kgfIBQJh*bcIsZXNB2&x9 z9%3dF8n}3H<-7mW)N*7#Vj}Uu6>RUx|NqZ}EA(Z{53W>|Hy7ji5wpuS`4i3B;4=I# zF#SJm^deY$2b_joyDh^8dG35Y{L>+<(+eD?MmM{77+@Ee{)eW|tw!&82gutmv3KJD zr+^8Lm!pPmAkr z`12w&|F7mfEti{p=6hq+iz*7;k!o99b5l`yXY?E)f%XG52yd+hje6d;o2b2%O z<565A6uOm~a%EFdWl^Eopq`R4ml-zYL&RAD^GCVTU^7cd6nXN{WHKdP=kY0?ZXvS#LV>D^LAE3&Y zI{D5Tj@QBn)%Umv} z3*Fre;lk3gW|R9;1_xb8qK2@ke*&486}wcd&@Ix08%oM?n1gwQD`jkSO(6VrMB?Rz z>h&!6e@(cwq`0!KA|RaXSwEP9y%9<)4&1PNgz&+5yrmituyVB~U0hUA+0?kjk;3Mh z6&oAO{g1#HEARvgBOavpP^25`n~F-C%3D5Y0aBL=X|eN$-y>2Vvh)Ar?K|M4Dw6-- zY+hK{Wm%QHvnncjz(&{=(P5WG36fD!aalG1ie&WEI|TzKR6IO$01RZ*-)z#JE#aO63R;>xw4joe7JjGrlm+!jAMOkfbj2jj$tmTT5yymjmP%Z zmRH-`OM_=D(1HVmtRekSfb{Qi6o=6qjANuGUC~yBqfdr(q&jvM(xQ&5%>38W^YLXd zCSS^9HHz>o&}}WRFm5B#IG7(LQA60A|MUo-5RWlMSsvR#6K<(%YbkHpnuCsYN+SGR zKpD$RV@$b~$A)Rb&DG5L$sph4gCXzSM-S`Xq3?MI8C~8Xe}_CgB77+ z1-eEyb9q+p-mpAC`b3;#Y1E_}6={rfl`VMgj5SjZwT)eqNSpf)JUyRO97`lL>4YL( z*NTC#x!LRA5$=x3Y76z%LjtHz#>oe&3oNmmXTij%Ng^+{DiTCC%0v;WCMeM&)$ zS}Tw3sR>uqRaaNGZ8er*64h$Nt4;pXBfJtfG}mj)^@?+YspFcT0)$V`iX|u&Fpt%w>)Kme+J>4A9vO1bDX^kozSgY&c$m+~jIo`| zV^x}PZF_n7&|1&z!XvOOW(7#J8{QaJuEqXeHQzoVdA==EjBCyMhnfPYNchYI9*e6U zLy9TJL_NhwZ3@6$?`%TS<}?NXt{Wsu)AYIP{YPAm)wDZjW|%oG-6k_9&6X@htm`)T z*p>mx#sqe~s|QO$EZOy*YCgyli-Q})x<1>bKM#N5%;|ea^^Bd=0f_$UrSqsU^%uE0 zJ`1J!A|1wXq%{e&IAK{HoxQn!nebcA>`eyijVg(ilhQK&gBRBdIc|{Rdbc$4F({3P zmgDd0i3!`R{SIN3^Pcse$n>10w~5*Fe&n)5H3c$p9%N{e=h zr;6VtF2-;p^ItTE6(PnD2M0)Z$A6D-CnVa9&iUQ&Sa6g9l+q zGu&kMzi24UfLB}XH1*fbP@P^I=THc%jc@wIHlWxdxUH%Rr)dvOyh0OC4LGjJHpHdV z-CQqPJ}z0y^xn|&%`H}ma_ttL)<3|<{OSzfy(^5_fv6Q(&i1~k}txr;?HdZtD zrhVj6eBxEae@Ji}pW3*kza>GZL&&A&(tjU>4@-hu#)t$bhe$wo236cLA7wb2pRCf> zkf5Wq^(~hsm4>*{Y3o~_aZ598P1+G@>sy%rW4|~#xy3kl2h3Zy6H+H}-~%a1TeDw; zp4uR@z^RQ(gxD;b7`k<4?9W)tz_nob{RAiRN9}ZxoT@-t&W<$#IS4M{_27w(ag9?O zM{-O5HaNdQm~%AL+q$?=+wd1Y2lKklc(yUVJNw5vtLI=wu~(s(N==pXW+1F<&8n(x zA6VVeihEDW%bUxat1A%2^YY3}^_&;-T!uIFecLH6-y`uC^1YsX>C+D8u$u{m>3Z83 z1OZIcApX?J1;Nl`1rmw>#$N)dtI|=PF}Za2Vi1ei+?_}I z@X+Qa;*nSS39PWWKcu6%WIP3w%(l5Z&HUd}+g%OKP}_P@TeDFcD^hJI;|;alRpNt5 zwXLl`iw`Hm?{l2`v(iypq0@yn_p5ee5YH^Zbgd5)RW}LG($K0KbpwRH_NDx6HArwrt0^=Mo=%ZpZjGf#(&+NgvxW7NyhVX)Rn5h_xN#-kp6wxNtcxae>pM zc633|3$&oAwUaHVM9}x14El`6PD+l)7+#`7-n+qyKXn9M{nUwwq3rncUdDfnpjU>( zpL2x}=f8}gSE>kl?f${gUnRU*NuAreRgeV;l1 zaH@)+*P=G;LcNaCANDh!?t2du8E>1?FrDZKdZXF@bE-P`ycYB#BIu3Gf3n3+O>Qx+ zGor;d9w>}gst9^52@xiHX&-&=c{|^^=e78O`!?_*iL>!0nagc5&P>G4FJQ?WANYMS zmdvwN7b|aTuS4H87wm=Rp5qQqP^yQxGi>HjSR@*zs_LnEBFRe-8D~I1cfVQxTj`i^ z`G#&z6Rp}w#JXiBaZD>hHf-hyt7}|Em|NQw2#U|nPosem5sYW! zvXymg7mGOQ4C3BNX9mTG;*A!4u&WQwI?SlccEUu2Xht|YX@{Gdu*qNwD3<+5o(GTh z@nTeWPeTjF*A3)`khmqIj8f=|*7*A1?Y@c}Uq4CwU7qlP@%8U%j<2c7!=8&*@3m<( zh&{dzsU6ae`K^6n*jC2Zhs^k=ZLb2lI4yyv#^h2zu)Tw{?WICkkFU^?J-$BlFP|Pq zVEle4@wU^X4%uUTeLJ1zPHTK65NmvWnByP4v=(#{fK(=Kb()Gr&|A~%WZh{DN>CYJ zAO0Bh>8)2Ldut9pF#Ao}*68FMk3Tx6O?dNmd|0E?rrkk>CqV{tVA;D6xjb<8LbH=QaFPmh9#_}3jk=Wyn2`MdxX@v#n2p0~PP{U&nx;j1*e_`V6W^!U!Fpf?y6H4K~=Q#g0 zTe4)g0NclFz?N2emaYC-5I${{1?B#vT}bn{NA%p+l?7 zw=(G>{!>wlgbne#l8I+ve`5^ZTM>U-EwZ+>45_QFY%%+dLK|Nxhtq!&9)~xid!l=Q zcq|@EY{stuzFE;8QdeGqtEfB7)+*Yog|4BE`;S!raqXT_+96gZsxkA}fjhp8nZak@{O-j}BYJdeGnMU#UMlD%I5=nt1Dvx3u*4%xWJd z>hIhH9%5G|`aAen>d(VeKoM>IJ)7xcN&Vf|&<6coBl>GI`eQ}vj}ao;`dKso)%H#O zT~&_`*p2mYOVzpQ=nwZo!TzjUsvu~n=14s3ma1o&|DhYvXsEeABye$5H7nqt-y!K} zE(uKm#X`XHSN37xHrU0)?2kL_@Bk++rFaPy_ ziWQrPY}j0ArkBn5=UKyLzloLO$Gu|Z!l-s=ZFzIY-N$Yu94^{J7zmSB1_$K1d=GVM zX|Gsi+Dudnp$p{-6DkjqymDTE$rZb+{j-T!Rq9NVn+J^Yz<^A|;$AWP-#nGA9iYzH z>=Ua+*6pVE8pE>3YECWfL#wNOQ{mtycQnrUhDmNu9kD5i z5A$Ds-*e@zG1Z^JXPWRyVODG#0W;7u{d`{I{NK-K{VuTt4HA3~Oq-ACsr9pjkMz{n zE(`Fv3J-CiX3sP$3`bsjue;cx}KT75XUj)B{rIm1GTM)hukp>|S3Q zV0HCQvBX4_g8C+ERzvU{;9B2vfJ4oJ>mAdkG)`6?;R#S659vp*zZ2kd&5ru$8y*R^ zf9ia&mO(yKrjD4ZJ?f*vM0EF;#sH&h%k*(4=x%)KjHo-!XfigqDVZJvE3%wpj(JWFIgZwHJ%ajX03;W)|J9T0hJLZcB$R?pOiG;#yDqq>_=nOw2@13c zi5cGbHXzN7ef3!^#0;Z^PfKO(kn)b76_pe-ydo^b#NRaQzn(ZZ;lvQ*B#1Lvv#F`B zEU&Aule3{F?(=w``Netjp@29y<54wJPzvHq(R?a~w%4`|F~c77X+b`Iaq2Tg9O+4K z?Gj*f%l5GZ155OzslujwR$XP4oo)=Wu~!9YZ1sIWf=n1i!VKOz2aK3gIy*V1L?6Ne z#{Ruo?yU)Ir0W~Q@7R3Sl+Fw!B(^W$M8P35nYz7QCZc<@kOv#^Tn=1KhC;}$%*`m% z*%ek~r4@Js>D#{izgpV8w*HS;?O?q&r~Z%W%tEkZ2|+y*^x6yQ8E61#B%EK|mcTYTr8;b(nwgI3!T=ReJjG3P_s$F-N6z3LYQRMU zxC{;jp}WJ4?pTqPmg5cGy>q^=B4=>d)t~P;+p``v?wy~G?sO|F-A$c7u~DloZ|kZf zmS&CTU1R^A)%_`ft#ti}2Ugddj_N}26i_(Z>fXJ;rvPg1;f5oixtm0DM;OhqA~km^ z-q75;FZwD{bJy2j;5ceq>k&@<1?gz6yVH$U=Nlrl`amT#IRy}2a*B}JHLUkI|Me%Q zZckt{U2lpr#!1XdM}bkW3Mi=JDE9ZB+5hSF-AxVipv9X-i}Q>YS&@}a#v59Ee|sNX zYVn5p3mr#oZ9``C!gRD4n96Duc8X9Tp}F}z;%N~~*ZVNB``(ejehKwlZDf{P(otb3 zrUHs+XGS;o^|7QCKGJX`^mmKs??|IRR;2zUGrIWzUq$Ng#`=pKrzh(nGkQ@v`qK#w z26ouiU3pzUR?h(yBZ&0PQY8{eILY{||>yr->i1 zyw7^r2=qf<8r$1cdM%aJ@;ng7tM{#ncxnqd_C}!A=2_LaM8F(h4qx8%ZM6O9Dqmw% zIN7nh#7a3FhIO=+85&9oglP`lsUv$M(8pOm`rK%{C9zoOVZMqq3yjjJrRd*u5Aeu!Ou=Eorv5;Za@Hz9r3IEJ{sW zk6I)w#6L0n-#p^u6C-2TKA?!7qlu@sXy09}vf9Lt2q2%pMSCu*0{PkL5q~EM`BMSn z6LFu8T5AUJv(h8}wx9T?{R6}&;YnhN6&m>pO&p8JxY5sES`XLx{Yk`63=p5p$AK@` z#FwW>e4`MTb^A}v`nT7hQ!ry;s1B{1r)gKW;K7ou<}r%l9g@cKC2Enb(Eg0;-yBvv z+EX#P<6;D8uSt*g7fG~D{O8f0h6&vtH0?j6NBi?6+9v<&(LMk(d4?LmeX^!qRa=cm z>~?(R1ak(5}Lz6D=K|G$3udc}w4)-w>dE5FTTGhNgW+ zdbI!Ir~O54fcEsdSYoNBy;RX|!wVTo+i`~*_>D=lTLQEX#^p{+H0>qn(Y_&xwweEX zZF$JhSmGp2`y@?!Ru!I9TVv80;q!m$`$0eLFM9`YABvl!PSmtdOpo>hNwkj+(4J8f z!|4uXCyO-g>gHxV`r2H@6pj1pB-*b8Xdj04n%`;Kzf-hZYbz^iYuh`{iBKnZ`DuS; z^8cQ`aSKD@I8FPw^l0CiMBA+Yc(i9?UFKL#``Gko-;qT7vw%*Tu@1u^8U6BjO}nDK zytb}<>-xUePy4S80ot>0LDlay?cb+I`<^7)R|IIc>=8>W)3lc<+AXQ99I!2KN}~O* z0PWV@V`>K#baJlHPGPlxv_J9F{`$ZG?Y03ioG(@SUY#E8kCSNM9-!Th)s&+(?V~mA zRMO|feT$#=-wFe?XJZBBC{6pQ^l0ClL^~M&%)zP&cRE1dCurKK4$z*5M@aux)BbIGv~Ts({&r}9_K~8lH9gvY7TS^*-*OFTGcSHla$cOPW|%r{G4a2i3v*hu zC~=WwC^1L8NalzsZ3w3^nK|NIP5-;y0KfqNeQaQp<8YEq=8TarCoaN=dSRRUw!AX> zyLHyFff&&^pFNj53Lo3yU4l2H|LzH(JLOl zWdq&c5&w+FS9OlF=vn^!P4T{E_&|Xe!%i0R_W#euUznHqEb$NhB+A%qC-ING|7Big zQyf+ixHP(|JdPdsac5)fq*xt5UUQdBn?HBavbF=~E}b)S*}{2qTbh@) zjbAoz-jdj0)MaDj&z;}e_S>n)ENsL7WY595e2GA)jbG%Xo_+!WyWAtK~9&9N<%&J1uIQd5bG zjMCETD}IjuH20r)96!f?rY6m?DTHGM_PeIVkujMi<3COQ&*S(7b~PQQIUW|uG0on= zLD1s(FSGyE?0z!*XVXavYMLj87+Vd^BLIU+ox69I81U8su1Af#X|gY4r&| z$DhsqH&4dDVz$L?N|5o8P>yM~<0DBN8UHfZ>T&!!9>c*9#qr=!j%jL@*)OXd|J^gd z@o(7nG+lF?9?G#MJ*_ekZgVvGA5X?_`sv+%&?-}cq} z7{L+y7(+Owsnt9E9Di95;J5|*q7Kkn#V*4Tj%jL@vtUcBznJsCo>sr>t@ngNM(ha< z;h3f;zs1k-KT`rSe&0)PmI6m?mI`pJtw__8GsA4j_@BoE9RH4!k5jZ(u?sAOW161) zS|7(kTvC$UAAjhfx9dShY}X5NEKkz|T$RM}x&X(2V5b}7TX4jt(GZSldh#oUqn@f2 z#+d)+RPEg4sTu>wOdK=ZOCaf>ngQ*P32ZM>yBu&E@Vv~(sYr^Ui5F{=rHs8 zleE%O6N)!OLm`T`*h1!hKKi^F`u~uo)_1JJ7M5)jThMGRTgWu)Kgt$jyczm_%&3}b zMXWjL*@7q5LWlaJ>`2oyr zn(7Fk3j0vfL4Z4>fnAZ7E}r!Z$g_)|ncT&7zN0W3^1JyCeeFiw?;ECp3Jd7TZ(;UG zP~qfvO-S}>F3LfX*>g~a`Li|Z+Krc(L(m4a6rcaQ{8?c(#y?jAWMA;Qy5hUr@L@@i zlOqzG7b1bqpB3ipjxr3v&Qtj_NYFw4tT5-Gq|y*KGJjT>v(hcS0Qoc0X8I(9Jn-`J zXN5W3|H~FTFS*5-dxoNOUJ_DkaMX{ato+%c|2ls**v+3Q`N5JHb^grGp0RN=9>zcR zBi9OZG5>MfB6Apj9kZnTHGH5gV$Jx30J(p#1#p->`#kX@Yy>o8*lYS@UK;)ooh~)^ zD>=g6FU66`#l zk_B-7%k5vR_=e> zUXoPU`5a4v94^&4e?05eG5DZ}B0=XnM1l)KBoJ@i`3;nzx4uAm>&>VQyH>BG^oMm% zVqpROzg-&QMtkdmYPU4K^=8nE@YV%~p(I=Eg5(w(ib`m)f)j+)`N~^wCLzM4I|{ug zaWj6f?nyic|4TAY*kqiUNN>NCxKOTyTjC@x%vQdkrnS5V*CE+YXA9)b<2X=JpzcXj zCkB4~MTvyQ91P%fX>;Mx1Mr%nTZ-`DLwv{Kx+3Jp{Vdby|Cwq}0r`mOb}{|`e24v? zyly2g_W$wh|Cvw>|6T2D1IA=3y}#=iS4j5$6}assFY$%UhTulO&~7xfHqbCNZEhri z61$rH@5)rvd77K+FM<8BYc@Oem*l6>#@=x!JzI!HY{my^Z!LNou>VAeT> zPD6!kkKV_r$o$n%3Mi6wbwpty)<0F()SS4X{PBi`*d6^R%8fHL3p2_ngRVFLe|rOO zLVU|tk#WQqiT{&FYGZfw|K%x=sa^hwfi<;;GjYV`G&ErEj&2?@v>m&nJG{zNWE8tA zdc-Fum$n8ZE|sU5!tO5Bc9#laJv>53%BGLedhBM-Kk5eHHHi=7;wOFf2fL%+NvE-) z$J7+g;g3KFLImx0gHO;&X!J+9O*@N*06`b0*TFP>(w5=1xHIU}S)ZTmtYvsD{*5&_ zsiPjwzxFs~AShUaQ!(@399}O98JuXIFwNp+BFROr*U!j>U#B1)U6WW?e6v8jC<@{W zHpI(7&#BnA`3HR15dSU^FA0H2j|Pa{x&F@x;1U%9e9KWsw*=>Fcop#<4#?X$0_c9O zCBb$-lgk6^Paro-g6?;T1ebwB5|7f zKm3^_x&4}CP5h(CIy2!FUdDbHeKzE1?1zzvpnXVXc};tVd%xpOu~WpyevD|CI(owR zrb*Kpr!~gGy#PBHm%6uTo9UxH{$@PWV^3i zekL-&Ge4TL7#U2~k-#|izXeDryI>X!OT5k#`vHaYc|N_3DCeOzyYTP7I#u`yz)#0G?M5Z=)94krJv zMP+O-vQlP2!<|YQF46{4Kyefg@uE9y^oh*K;E9GshzxG0%!mvYWt8d2fE8KkM7#-* z{?BU*>gv{d_)2vUxZSD0EFE3p$^z)>*E~{$Xk?1q!{mQFl{Hq0${L)ibW|3^Q$X1i zCsAAvzx-6(lMNV16L*N>78}K}A}i_JxC(m?3@Gll`pdDyjP-CE*X8Lb4i^4>C~j7wh%xClnx$8YXR(f&+@UO>Yi@EGb0msiRzXb)v+Q~_dC2< z@e*G}cGKJI*E&v|^>9_*+H_Q>8(7_7qOC1^>j&V8LrkrYzx}DR{f0PAI}u948Iq39 zl8_WoB-_;bnEa=Pv=zKFeBsMU zcuU9Zx5_6AjciP<)3giAk92C&(OL+g0!n9h%zpp!v7_pqYd9LJyO#_xejaU9$BI6)cx{_EM=*gCgs z)}^DhBrF9K%I=zR|M*9ZJ>T$KXzV`G*l&%-SdkiAjyE(GZ}+L1#&&o8RgR-h;Nw1z ztJ2XJ?vO#(oIGuO(}*crWvHEkr4h@ag6AJTJhZfNdy39RID4j}vt%>{luPlU{E)pY z=>PxRa14~TQIvL!Q5q|<(i>9I@Bf*=`7-g7JTMOloS$GRDd!WIz)30O%mZ`0dVi}E zI4RY!6F6g8M6 zb)8~^D4g#Je}m!70ES1fkMA!U!(SAJl3_BMFFg%K#eQ#%p3aUXUeImvg2GVVT0R61 ze(f-ql&Zs4{^6}83^xSy@JuY0cv)k3S!2M>v#`mISHIKI!~23ka>Tn01_q|{E=x}5 z9fN{!q`R5>Z*(-mZ27Z^_hc0a+4A?&8Cz2X>Un*FkWpo~H$__-2(#sn;mm!LdT=U6 zl_h#qNo`QHN0qr+&;fOzk96ZMDnZON6Nf7@r#r_e>W(u=XnM!B{v$7x+3M^4_TO^t znlCbC{ude^fajl&##eO&D>fV7oc(XRhJD6L5D)kZ&~ePWJUT%<2EX-Y%f=mhcZ2fC$^AXc=gh76*`s~6s4QM)t$ON+WZ*`j1a&+bb?1E*En;o>M1$>N_J3$g zWE}TZ;-C5vzOm6x;-CM^IId4Dj#Y}lak!fDc-otKsFeT7rXCWG+SCL1@B-y6Lz=Ov z$G(mvIMix|OZWecKmT8${16^)W;XS_t-F6JgtdD?gQ?!l4u?h^SZiA)w%O!kzbCFn zQ`dE2zvq$ga}jz^%d3$sS&CQ#Z}G9ES9=_{lr*U)8pEqCP1ma#u^!iA4KnecwpH%^ z{ec&%XR^0p?{6a}^}s*0vZlGZrro{@CD0zKU~29)LQ&gl@9!XU|CGu{+65gS2x=Tc!>uS zc*cmzwqx&adpZcP_ZQghy}#xEgm@fyE=eAEWT35p{O&-jcaxpjIzpVgTKa{ODiWZ= zfp%@kK+92^Wm-{&p~70OCHE=PV;6fgLj}C>m;cFHas|%6x)S89C6|+b*|9f1tWcpc zLnK%iB0)ctg#?wmpbRa1oeC8cUXHa)#+eE4_X-YJ;Q5XC z{RAAa$X33sxvjOlwx->_S1H5^_Huft-DJrN$SO_x>(-D7bB;iL$c5U5zi_wL?S)QL zA-+5N#|kqMX+^T9BTr>B3fx(yx_OK|m*EW+Rh#`!Yu#ODcaSez?O+a=nNXOnw~av% z0J{!BnDsB!+-k>Jr^Hv@;fJp>(r70QS<{w0Z%rRpxpceCTHaH?!EyT24v^>Fke>#9 zb2veUGzApP zA)^7B*$|We(9D?WeWBr4BrWf!?8uEDn^8tFbVXK@q~#DZ|I-y2?EW?JFH~VYBrX4y zr%j4vtr!qBQE*Nz2yS>ed#-D%KS{fvybtR?>2)iT|{mOj>rrH8Ofd3^B!N z+V)Z*tm8}QNZGZ_qlaGTvpwF)`5mXPWz-|)(M!^4?G!t@2gI63*OvK!a8vhEZm(09 ztU^$;y)GqZ$a%C8ROWxRX8yM}*{RDgT#NaiJBaCBXCLMFTnjZ=gIMj8K*?cveaIjt zHeGA>zpQshhtzhR)$I)8j1<;KK&)%DAQn#tH0RIHtfEm`5bF*Xh#Ny74n|q@@w&4` z*&7|WbDa2yIrs3UV)aC_?zpKqLycu*5J&k84$rK*8{VvL#=nh}P*uM-!6(@@(P}#L zxKx>2k9GFc_4dqm+0MgIoxtF`E=JlrX7zLO`M>Eh=3OXjxCU)nUUdCAg|t@EbNJ+h5*6@wZna&IBz+H-@C z>TNdFVy7%gDXh$jy`}PpzRC<^_hb>OgfA8oKEt~BimpSMSy+HnJB*8Q+Ra5j#DHYQ z{&)lAFth(zDLdytgbw0~|m6?7D@Tmez7}#)=|5ZxLS!DC+;a3vl3)WF8VRcc-@W0vI^2%%yTZOv! z9(LuNnPE_c&)#PLgN8aOH9os4%!PZK^&eGPd9nLEFV=}(jK_DCY^-9cJ!8XDX53*l zwji{+Pt2~SeBb)yhL#4{2m7B#a2BM1 z!#ocYHxgPlxew<*9{-7GfF_!)ZIcEn^S@qELY=*r(=>!ddpPLv5-~X%s?ujEezR;P z(aT4IP+bzm&R$7SS=oY`F!^7tDngx!B4OPJ7s}dD%fQDSGNPTk7+S*2e>9&Jsa4(| zVA1r6pM9LqmCD_zq)HA}{hU563+6AKJ0C;T@DWSqHVBsv!z5C zvq*SnO?GNYicYlJ6x8>ym22+?sClVC?xEs7HInP@%pKW-PvrRSZ(BIM5(+oT$ANZruiSCmi>!A489L7^u$T3UvX?PMOg zWHzyy`eLaKb8Q+5t4pKUsZPSm$`;m!tL@4P>r@m8>rM^k{Fmwx*)IX>puSF%dj3nX z(>I9^vn>{%k>>t;UHM_?q*^CrP!2Bx3pyFe^)F9-qISU^om5hg}p)wfklTUxV zS@|fpGR!ZRm8YAw?}UaciCtEg!P zi`ae2@HxgVs-mVe_}t~I%)T4oBW`BY2fm_=SaY9eII&FsthpKIzWYH~?z zRVDj2+m&->hCvmT>}&Qvs`kv8WJ8VSvn#-&OYQr#%}06k2Rx6CsHt5KIh2i6q}qOt zuUa?o?64X=0J!Zp)UKwYrmK@1TBd~i&9aN;vH@I&w1?Bzvr_J`jh+(HgIH6-{ruzK zCYx%|%^CM&(Es3r#5NhqBDR^z)Od8iMl0=m%x-4_QFQjyjj z&H2A)MHM!sVrQ_|v6Sy)WvgS2zQ(ScvmA&6gH?=ydQ_7vD-6>pOr`$%2 zy~PsH5HFQO8iIF>FaFCTUYAa>Q=WvEl`XtueDR;c8#Q2!@x^~RfxVd zjR-%92p=SiAguQ{_P^O#pa|k2dV28_L^SK|@d?)Zj}(xbu>hl4@1@G%fra=KhWoSs zvmG}lGh{Q7%$Zg@c6yo=&K-I1!T7M+aev-_L&bg+qBKszNa$6;^N~^oCtuAm>JPcw zQqcteFRqJ^H~Am0_$)^ya&oi#*_so@jIVOZQ!!kOiBQb=8E#1`hCo_SnlDQ+<4-~< zD(0hP#fYJdzX}B?^CuxP?*w9ydAvFQ_lc63Q>XoO^LIk?tNfV?09o9~@h?I5iE3cl zUxXYJO#J_eYG4v)Mw%~6jtOH$j!z=wm@rS|_$)+@G7y6t6IKhT&nQQ(16I>wF?x-> zSZ^H<;yVlPC>1sZDjeX%C}MtgFUPIH(jxZ+0rfRy&Y`(>B=h9?f#My?Jo$3BJY)v>NSP;_{{J;)RzPO5I9`^_ zlbQdd%wI>yJcaw8DDyYbGEXu7?;9oa%&o{g#l*i}@eXC4V$Q#PL)*rLAyVckX8!k$ zl9@nQnlDS{DIbc=-$cla8%edyTS8>s1;o%jr<(oGTa?TRu)0p$b{C8h?o5^4lr>B( zcqA%G{~M zmnFosYek6fql94mL(BakLgl=pfoZJ?+oQqDr)T5b{6n zvVJD#ole!Q#qZ<8igpj8|3~BQ3U`BF{8NOT@E3A=M1w*ZMh*lud@GUPK{EeG6?9Hi zVT&sr;G1rJwi<{}FLjBdT@~@YRT*EqPH)u0f?m6WT`~GaO-KYgyY-J2H5HYW6*aXLb%Vt^2erPV);Z1CRuD%Fki}<410E$9 zj5nna``=Yd!Ol}&DyLIQoYD+Sh&ak;P(oIf?sLc>R|)KU>^_a`J}BXIk41JL`g1Tp zgz-NHHZVU@YL7CVub;33&*JcYNFvgphujH5=v*L_WN*^B4td?`T+Zo)cj3alcgKg- zxenp{hn=e=q}8iY7LX4$_Fv*4Titsz1kQ}*;;;n8eq+%Zd{}4?y-v{X5Q6q(lqK5dQHBcMfi~LvRv>YA_{?>33p~8B z(68uTp8qQ=W0S;=R{t(mfwNG;cvm zGxVe$Bflp;foX>1e+3g0mB7>#qmoLU+{{oQF~yX6CKa*W8M6K%lwPtal{mSU?LrP9 zXLVMcfj5DC=_-d!N#*FKY(!HQ=yeBGuerY`EX)$Es^sq1zduwc6G|#ZE%0fT#MIoH zjHJyEWB!L0xpANE_T&5IsFP3O(;TVTz0n;&+Ww0UVro5xWWVjU*)-(5@THgg;} zz-BzBf$G(}RrwjmTioVhTSN_|YE0*^J;L2)6X!pap)^?r@#;-`3B2AR@VE;M@S0`{ zyWTWO1#ea?m3jLoIwz*DzePX z6Ipf)ktL3DkY(m?M3xfE(kQ}xV4-SP`m`AAjI`sSTOnnO}1%^L;!&MIJd7|k)XTPzL?YL_-Yt&~ag7j9`ZM)t$_HNx&^ zbw^1y#?Hx&!2tw~F{@fgDV>27=0e@@6E#(c#n+gpv@-mOwL~kkWd5(Ul1&%p#;zrZ zU9F02EVXRRs$!(GZUSR!WtR}G#8HmmWc?duXr%N92leT|AGE|om|f@(=rg1A(z1u> zWtU{VjL@uG#@TvNtf%+EPt?mUS$&*rOT02}yuq%9I8In)&Yi#1Y#DMK5*vayWND%Q z<9McxlYRP7+*C9;hM#+@n~IEadgP!r@+1XOdJE^jE)})&z@<{+)C82y>a02vZ-8js z(XOtfIv80hHGmCp`X&jb_dih3oFz*aV;W&L7-@R41k-wyk0u41f$N0IDOewMoscD% zAqmNt9U}>mtX1a!(#aT(xnxS5Apyx)omIu0T7L!#n$xb)=F}D!q`Ru&B>{7~v7_Oo zP3FI#xd8q?0d5i9tc~+ucC%fR%qg}TDnapWm)J@IsSM{P89HKJ;Pdi5bAn5^6jtF3 zPGt}oJ%8b{rK8mD$vF7M#+!fVI!rz z(f*20Z4;=Okta%!-Y|%7(xc9|0P7Bmr+MW@WL< z+2?@(?Q%efU5fLceUC61pzLxcj00g$97MvL|LjlQ(zME%C>3FqbBa)sRyiQaD%H|3 z)HA0-Na;9(jNOsAZ(t-AAD2a3cyXF9zHN@||ImF`?Phgbsdw_gtLoD{)i7)MwmA>k zXw=pWmr4r5?3`avotoY~MALDAg}`%-|Jz+@Iu3zH6?ZV8%$+Qdc2^#NR3dae_jf2s zUGJW(>r;fa&;RF;wb~Cz!#^C`!aMuAeb&@BFT2CmFHBzygMAjnIK|SQ+<(LJ@+M4=RP=EyDn?Aej&9C4Og9^Dk>BTeUsyHS#+Gceh7rl1n6_J}uxlnx!pdKeWz zmDIv?tiZM`j;v<*#+x{fdUdL(uB@u59$H&fg}Nj!G5NeKG)_fBqU$-*tb{Fi2}ZfbGX;+yl{4R?Hi4=b;=U~F;x+YS+u96 zn3(EVt1mf+Xf4GoY7)iND8={}Fb1G39KxcL1XK+dFjQhLC#H}W0i4^2Y)n*OC26}w z_Y1_DWQgJk7riUohlMmtjpn}C%zuZu8D4E$ef=5UO#ZK{+zeDE0YS2u?N2KjmW+hD zfQ*Z0+sb+qe{`nbe9tZTu#~lUmB8IQ1n${F$c+DcyD8pW=w7uT_UlT z`Y6%NT}mZbAn-{f)#=t|;0=8*`HNfKBzyA|i$+k1O7>^(mDD6Qzi%} z>!J_ZZJRU^oPg4&G|6wx{7+l@B)h3QvcL-6B*$?6=QfE-6FtNyuPO^hnut$cwOT?< zn3WBIV|@hJ10o4{eSnq486LCICXh4J?gM?2`#@46h9o2rb8Lo>%pzM4y^~Eo3-vIB ziZoUt=GgsgDk>3^ih-My*RiLfI(_Mw5Z5FruVc;n=NQh})LHp4-D7bgTE>6yV#!{3 zqstuorT`kl85>GR&e)F2L`jYYW0IXKDw5Q3JKM4;#$&tVCpPcYq^%DyRh@N}} zsb?d0#|2%o7DGL5y^nnv8mMMRdDd8ovtz){2#r-u^$&g3SEO2fB1k~^-y#0@>^%Qm zsjd<(HO`J05KbUHbUrg~bHbGCUyA8jr<+Bdkjwf1_KP?Ls4pXqV%$|3$USgyeRS zK>c`=|Itb#<2JITs!Fq9r(VY_i(}b7i>JT!FoY5+P#5=k!UzGSDK$e&bA~%jLm9~Q zc82>2MY_)u=>NQ4u*s%U;tUV;@i?fn1Bo~%JmITOaoWoyYAeM_cAqqOx{*4*uEilv zQW0^M`{F-!$HCma`}5;h{J#l@e#~fHzggfeE8KO~;*ZYx|DKe znBug-=&0>%GwbYwS*II^c*zmuUi6fcBqXzPVrL(jmuxa>Pnz)~YyzbuE3z|(e2nJBmz7yLafv8qN`zugME=*!n<+-ljQ;=G3skJuJ@H;q%rvE#BE^OO(EUhe zPW9IgIctct5tmL!(`lmc(wZ3+QMdv)R z5pb=path-gI>JREtuKynCHH?(zl&T)*j0&D<%wlE!hU_}2qAE?Bkac`XJBuom{+!l z5Q`#&Sb4H7gmQ%M&By5@pAaj}{?8>LLWm<=Y1TiMxQ;N_7)c)(iR%cxZ^{wULM%sk zs)_$J8Cr--rWh9Dl5vcESdQ>iuK!Sxx-uPMq3K3oV>`lA&HC?>2*sR=_21E`qZSbI zh|n3Z-|P3_!)o1A8UL|$mnp@BI>J-g{|RcD>j;Zts(FjDtwszw!qZBn?Up4&6i0Yk zLb$I8X_jgrg^N0^Rk*KEwzU>23Ef28IcyX0T%lq5{|e=uCW3~$wRgOei?wkxI>l74 zUEuY&vwUCiWGMeW@v!~7WXe7d@TZ&kk7nGg8SDMK{$014F&6+>RdAm^)0KTT8lSi` zfm9tf81Lj+xZ!xgJ9@u+){FtlhxeT<14jGGkW5~q_j ztZ2pnW#c~cG+%Y%&gqPOnp)qTE!^4L7Wb-{k1=tBag$tCL}^Se5{!HfN}tA#sn+u}ZZKma!w zHz~#?PFD-JW(-g^?z89ks#D*c3NeY-xC<@Z`P&%x2R_EsHyAf5#wAXng!@V zabI=f?u;R}lg3?S;qJ6;ai4RMk1=tBv3`CbZV|YrE4XaX0tI&$ z;Ny*}UbJvAPrUHz2sd~?OU1UNB)HGrM{u7Wf%{yH|1R!~n^Du5`_6ms;={sy?kR$M zRS528lm+f{8UNDXtx^HdLg01-pixdP1{IFre8nOrVq4ABqAr=yiooy>y?hK;Xog!4 zd~TQO(|y%%MfGkN^J45c7doA+>7*a}e=S8E-~-dmejCa{_Cbb)euOVswKaWwB5Z;P zd1gp2#WT`9mHR~ie~#{?BD!4D?xVyRg-gjF9K;lS6CB8#bB)>mvr5@4by=)fG-h>* zoO9md8S*VK0;6~jaeje68+#Z;99BMi@Z&~*v zJ}kI@%oVs{_lKX2vVi+Xp8p`+^=^WsJE9>K$}e#`TNaH-$ueW})rshrWuX>}v~5}m zl0QD}Lq}uHH*s94)7dgsKROsILpsY?&+qP|L}Mi-iDd1F5E4ms+BuyY)O7x2Uv(O5 z34$**b6DdPgsaIK&Z5+`RreLH#*7ryNaWJzFEIIU-N;m9ClgkdItAfs1bXO!5vHZa z3z+})%!O(sB?+wU09C8e1gO&IFEHc(dbiJSm%)~5%<3G~=SvZK!P=N-D$PTm?;P6a zOL3hJI}sVp#v5&RVUCM+9%iX*v+WhL2^}!&>@mabn=Gi4LDWUFJ7$*Vd5{!avJ@FF z{C$A!4w=+LBew5E>{2T2>apz#84E0~-jUSOjG&C>{Hv$X9*Wt74ww~t%!(l+P=(6a z)s?ZUE2H?Qi<%)L-D%Xr11>tz@=wmcAiZ z%^O^c>8TpA%4Yy?-C&0OX;r4(_MhHt1OFju}50; zbMXq%&kYg7&Ba$*`q9_ht$V$l;jX2hi=Pzz+!Uf8$qrupl?ZW@(*`4F*Y0dQWFL4) z%CBDDHY}ALWT&&TgO^nL$gp3NjB<}Sl97}hWaWW)GnaCQWGx8C^uA$d9q7}_*7*J?xl?X>BDqo!tDe4 z%9n$Uoy@$H`48IpO%d&QDfa)mG$-NG)-?SmU8JnT#pF17>)o7d8wpODk-+-0`8Km31-ch*GDEHQxEr!B!xI zJ>FeD#z%>UMoKCZm@3TuuMGuwkDR)M$kbSM2LU zPc?Qm*^yGGYq%P9*Z?7IHC|!zpW0lgMpBaekez4IYGie)QHInjO#bt3cMvGeqN*G5 zZ`6=F(i`PQVo2>`4RY3yIy^9>t{v`5ZNi4QVm6@zX8U@~_Qe1MRG~xaNOw3I=?4>}qkfWZWkp8G$NP#xbsp zV_X@$hDZA zR3lb-jcAYUQy4b5t!9nw`+8%$Aw45NdS|7(Do=L-(yp>B(&IhSqMs}85&b+Ap`RAH-N)SUCk46CBfcc*r?_@Caz>dH5U*p39zTpG`{zM4yW|Rf+4QBu6 z6DojM4dDNy%paQ&46iop-%luuUjc@h=A@SIh?TPQTvW2!{_07Ph#{k{yiiy6Zv+xk zz1rk|bWxfrQiM8^?t}cAYhzjNH^q_JG0Rl1e$>a2rb;quWuh1>Z^6nI+-pqyqcPA_ zcLn`e3_SQ^2NbnjWAZ<`Xfuiyq^y=}%=(9}OzZ0k&H4C4BTRp7Ei|zy|KWu`O=bIq0ACrI}xxS|$ zc`+FYjSO6_-_x=}XAS=79ClE=6(5!rUf(2eUkQOLD>B!gByeAG9dls@=i<9yr=@R2 zrgRA+w>+O;4y?#r&+(5Md?gC^cNT8FBJ=j38)<(Q?i+Rx+^>h=7QcMMFv0!0^2_W! z*j1>z^e87UOT~a7i|KvD6mF|2P1Geb#`n{))_v=KP-)fg?t-S3L;(crmMh=CjEMZN#FFiWzsse^2nd2uLE4mq;F#UL(dWR1c(}xg^AqU zRm6JV^&FkTJV(CIbI?{uK;YgyMc@V=4nZ{JvebgBQ!6j7ej*D$T5xZ^NZ@`L(gV*H zLJtVs58c$tU<`D!B2#2pbe?C%G4HDr(J#xwE%r+MJ&i4^x0wCE8Xb+byIH|6b&4!w z^`nEaaas{3`I(vZsBUwot6e^sqt2xe`Tlj)JRIQ z6{MSAVyKbTtv-L7iT^%SorC_fB7-f}m{k?TZMfQq3B;wT7Xt;k%@mQ`-(dKF`lrTDc!ne(rnj8&P`Lt|FA zNEvVQ*xm*iyIEW^?U?_JMp?br+nN8MeR!mYDBV?gx(kqYm1U8>-6Jjfxud)2=gSEF z+)-`mN2gZi{Ih6`Uq5$D7ybM-L_abCyyN#G#9y5@Ott7Cw=*8H4?Lvnfz%oz=%Y<& z%j|U41n`c>d}QbzNk%4sMcxFE&95pC1OZ8{+!^y#X0IL!`e7?FcmBp#lyJ8TA^MoxOXn`ks(}?QEsT%>6QeSm2J3Y|DRSF;qvayuoam*-v%Gr`ClX2@y?&E zcGO|R?{4^F6c}1|ei!qfwDa#n+OY;@(WCBa6p6lf!-mdbVM8Zh*pQFPBp_yY*BJu$ z`v|yqVgBcKOC2_RvGW&iB*A6=n{fXT0$0L@yZ$b4|KWxWb-02`Wz&{8#nyP3ubd4y z!0^1UQR87*oc5J`(xvnp{5_4Z;qE{9(9z0vD0Z3(^mtcnjdy-@V9k59tnu#dcYKs+ zXr!d}E|xgOf$@&jt+3&q-oENIwLOt!P+7b+PWNy%`H4boxv!ntbMIx_>WSenHEu~k zjaJxj&znBUC*NX7aqJU(8^aGLR99R82iR}87e)*xpMsow;`hI?Odv6`@9FU4#^2h4u* znEiwS2&h7b)E_-`?vN^B!+oOyY=>u44~;nE93|t=9^0QGV|R8{GtU4XQ! zEQ|DiJkp|{`^SiWeu>b}{YO~((P6{RujF0q*U$Z{wSIPULi8hH!~J(?AyCGI4LyyA z?Bf)B<8VG(Q@vE!z)ok4!}tHoM~3~HWR!a>hI@1?bBt5I)f$H%sP$E*bA)l|F@zi$ zT?pI^~OV2d?*#AzYGCZm1Y;$Pbs5ZMdKLDidx!iiEAm zJXnmPuybg)9Rh(4UuCaB3~Z4Ck_mqBAW(sw@1~#Jf*<9@-(^MS zL5crShR$jtKozOP->%7tzj$|8I0g_J9hy|lU6!a0YOTmT_^bdAJcY#o$NbL?AoPmN z?>iZ-KY7n2a(s#+fY zr%x@kzHU&9FYp)Fwy8ggvP)z~bcszE|2)g0l{hDKeq#eZEZ4sY{!euY{gfR}V)Tn$ zB91p2dK2^iw5)u0MJ89DSkH~+g{;VIdPtB3o&+QPaMNaC+F6;ipXmV2^pPxK+Sy%^ z8JVrZm?De~GEmBJsaIo6#o>X9jakZ57U7|r9aTIkP;peI%C{6{Ta#4s1VnN(9B&C! z+&4o_>5A}}$c{LE9;moqOl53~vXQaX()nbRwttlKAGUqxhE z_|bOZUg%Dc^r4fas{lItCP-Z8Bmslu20>Doj70p~qt97Z=&V6ar|^o{ZTPUP@X;*- zw>Si@tjIjJoxm-29dmJJKIrU%qnN%GnfL%iZuz!f?iW~*d5q&9HCP;h`>~TP+Ob1V|UdI>6at!5&%O45!4~8-Y(T8{~^`s>PEw=6y6K5Yu>}#$@?1fzW!<{6qw|R zYP_l}{S}!f5lJwesCnTlxa7)fJy){6o*tFT|wT@nK<4t`)KRxt>GsT1oO81wPNw z8I?)E!2Lwv_KSe~RE`B#r&gxF{)Zp^aG%;k;Fg8JmDI{p69jIVn_3wQiDgA*JIkUw zduAN-zG55<%^8%1TkOI3dm5>gr@r=~qp|kg&S@&t#=4zltbTMb2R55!tWV4O-)>q? z8Y?NOBu$C4on@?$NUGCVnV>+cK5fQ7Pfg>t$1eo3#@RkxO@5-GYTBy%3RmNdvaPGp zO07JD{IBY%8a>t6Z+oYyNUL%Ca5Vxw^Z+4kH9m8^j}p~LN-9t+!4VXnVMukW&p%`0 zzcST1=s)+*U`sV-Z5P?+TNP2&=UdU|w+rv{2YY>9Rx6(6{Ga~GV>Vtfo6rHX*&egm zn8E;+#2669Tam%837OTzTUEqkt;jr!{clNZCuCBMjal1A%6PcPmN)rsZ*jH!+KhmV z&zbn&lX0SAHlYJ%^E_trAR|zP%6PadBksxqe#$8R>A6!NBi(6PgnxPt^PkZuZ>`8& zIAFJ=)WLI{|IwX>JvL=5YNO*lC;1PQF{8vJ)reJIBii|o1x&5Dtp?nton}KukR(|l z{XFv@v=5K;WTm?*Pj>;*uCgrB^E}d`pXWD-egY3=p|L}(=by0jqf;x_{V<^sKeB)X zs?UEV`q?AIe@m|7&xIny9!?u(hV0s<#zXdTw(}-{o!vQ{ngEiGH39sy?0?!#lVQIm z8RZ^};U3+}M6Z17KoF1>nLn@bVPLPG0{Xs*nSuMDuPEWhjVJ2kY##IVJ$Xb@D=+*-Kn-*w`0gr#-!%}yp9l*_R8ONb#yvpUwv}3WaTfs|c-#na z!2n;x`3JXy>eR}-<7=z^z+apvz{^7nuvrK(|3__>yQvkR)7sp&Av~K9$X}B6KQN@l zuR!yKU-pzqQ)K{ewf##b{@0ZkqH;cp#5fZ2Wu;bLlKtPpho)L)0_8pkl-(}T$^E8! z7CWzHsxRH@>W6S?swAVv-eN?_Ze<0Rm2J2t{?Tx0s?$)!2Upbcat|M$W)$seuHo_B zffc5fmredh^O=asC8iOLTEw-zyc~SkCCVeZ#LF1}Mx&B){jqOYy=lN({II&j%kY1y zOH?b%@@F~XD2s-E`Ad62f$Vl;QqI zn)47hwQ@kFih_!=E#J=p6BSs=RJ<@y@xTn_DT}hL@hGGs_dohM9u!j(p`vU|gjm76 zgqyU)tDR7hEioiQ-d9chzh_AItrZr?hd8Q} z?InYWj@QNs?%@%*UxWX5ZCNKeoO{OA`EkF-{3mfILU4d z;!OyoE51<*X6%ZI2$OyT`5)J$b)uvHsGKE$Y<0yq+CYUS9rmOQc6?aa8>fp{>PZ>B zH$y9hVVopH0yuk z+(gGAJ={b`56cF+cxD{)zB&>8vMk(UAH(0%NOZjUx+qhlqp|MP!)Yqk#@fR&RzEtJ z1B9@R^{u#%5{;FVRDQ3->0ucwB$DbhR!MZcWyU{GO^4!+DM@tn3|Et%=(bFBJduJL ztwhJ$eSN}HjXU>rn!0N>_6%1e&_fTrNNZ2L#ecp%+ee9NBqf!3Dsg)HC6?+|pMU#m zx4P-`GcwpxjafY+`~0(BpMMs8zDIbUf6VLilIVD+dw|(tirIt?m_6??dma-Cpz=J7 zmFRfZont(!A|5Nz@y;0mwoRE-V`Em&NEu)B*uDrEds#vcRBW-4YAI$-v) z$LwXu2vngmzUa#MqAR2Lr+4RrE#2w32zUDKIhKFYdq_r~RkRFRu-xgp-2Xs#IyJ;U zNjBo$H$=dxZla@EHDZ<5h;}yObtXF8Rs(J`7knNvg1wdKc&{ozdX~~%m8ZJ^X;)bm z>6bmyqM!FL|5f^#8lj)}Zn5;E6CHCedn;)P|Gig5KL>>9hjS?mH}CzQ2yuX#_OWaC zHy*N&)5DvNb#Z4@YC6VFM-l@ynfLeckzv0k8RZ^};U3+}WU72CD@vl{eY5{TV_;O& z0{UTzj`y$e@gdw@j3?^j^bCWm5*_S6Hr&7Y$Pg~cC^u9LH{^%Q$`;(sjQ=^a@Zh$h zNLZp{GxI;R^8+H#`>L~Nw4>9=*3;hMYVs4^hG9clmlV`!g$*A~^`WO4 zckAsmZKuPA-r;K0VFNvb6*hcyxsMXnNJ?tbQR4LWON>@vbt`Q6=v}wE2^(f-QPqw3 zH)2h+th+nPm0{PF)5{vqrQh8^a9_&Bs+xzkTf{y*%Q6%salS||eQ zXI8M|9ibYr%4B9r0=PBJ)dAbXbc9mt3?(LBl{e1d3 zNYj5uJtX7PPc8lEu;K8j^{4pt^I2z5;c#k$PU}X#L=d%CZf;%5Y!nU4%eirz!9S@JN z^UrU#+woyAa$dXf+ZFh*?ELdrK!xqNAfz2x4q?ORjQ<@~<~x-}#}kwEiv`B(LSOU| zu@|th@!Y-~=io%W>=3SW67?C*F3!%?<#j__YTAYjEU&1ZHE?KUdHcZjwyL&)<$TQ= z(pFJbS&nVA-4j0w<->&fu|Db}oLuK9+_0iXi3VQOg4=2L8`m`Tpz)(eHjN%Xvhg5t zt5syn%R#=ia^S40+V+9fEv?Pu)?D6PT~XOm?r|e0Ba7Mpre&Fg8`_s8E^(U5I4D7u zOO!0F6;&EOBf{{+jZtBUmWnu$*+(FBhXM8+k!QBY(NZYGn05 zGUw`@KD!JQtVaIw9#CN;FN|*FFW*5KHu6G8H*(DEW+oxAgs-svW!AqIhhUcB>?^bWwOHBQI;X%nB{5RA zd5>@sBQt3Qv+7!#%WGOYZl^9z=cx@%BS%afQ9oizBTTTM{!{=WquuaEFZk-;5R0Bf zGrJADm|tu-)M-jwPOIqbG#rY#h^faq(%gTo@zq<6dr3>ZtY=B{xAneyl;-|=lu=h_p{Qyb2{QiX zYyVg?+N-YGY3ff7DEj6uG9Xh#8C#O5=SZR-McDG@~7D2@73a4ogO}avn zZm%w%Ra?{2nks1ym9VI9PVteRh=Hbo+)0`*7TXJN7B%a?iy37*-#XdOKQI#E>rPqt z$Tb@sm8VmGkzkX5AU)C%WqtFr%{HSOejkTN7Y@=A4Npf|nRIcMvc5I*Kh;Mv%7nCf zDq=aFXGkZOqOx4lqO5Pt`iCY>t?Wncwz6c<`_{~VRM8kCv9l5r<>8_j^d_b*VwRJQ zUZICmrYX=ATe$w~dQa?L&vO2mSn4$GP9*y|iKUs?c~d-P(zuG6^2+kM%8Hty6;%}& zo>L#jHBa&riMHJ0<5>nA4<=?iP37bXiDoN_QW_p?iNvpeXXgLfv(dPIq}0%yezBa4 zo8R&LI|tXrYH(eFypJbWVKiH0?=tko)6w*ovN+0!^5cpLh_YzVE*CX5O7=SOu<~Br zF3l4>X*Zs2T%cu<~VItLX(j z5R7Z?{+F{58D+QhxUfKA9N)=oJK8<7y|!v$WR(4+2PnVJI97RVR{N z@R5r+sv@E~fWPpfgv?p$w#-?yc+LbxYEq^nRszLQD8iJb=EPKtD3E}w(Jz_9J~cS( zBb8q%O1m(A^x}n)_T?o3!{|N_J-j?H(i&2{^|4|Ya>vr(|3&?k?^Wm% zv`|1MBhB?sE#=;Xn8x=X1ugtdO`nQYi58i$G@gGl0TeIMU%BoOKLaRk0w}Dk01~0h zVXrj?feK7MekRt$=gb-LIbH^Nrq`oocO$jbU2T~cE#hs)mpXZQ%o`pc#R)@xn62JGAHz0nSr_6B0Na=Y{jb8b>&9Iv`_?Wkb{p zCtGjH63!((tb#s)GrV&+`M&TzOnKmB?^&h@=d(WuI8y?L|KX4U$8jDw+1J)M!g)&> z^8mjK=Q@5t1Z2NivJup}m+reCF7n+EK2=YmbRp0k=;WT;|I(&$*rQD;a_oOuW)h~%;ApBTwQ!XsMk+|| x4un|s5@jk#4*NevRqx6BFV97bRFK>|Xd!JHEKxyd4Yw!vRoodAcdV+X{s12SxTpXC diff --git a/flume-ng-sinks/flume-hive-sink/metastore_db/log/log4.dat b/flume-ng-sinks/flume-hive-sink/metastore_db/log/log4.dat deleted file mode 100644 index 9efa133a9c5c1e9a4c9a54faaa9b21d3ad3d8a1d..0000000000000000000000000000000000000000 GIT binary patch literal 0 HcmV?d00001 literal 1048576 zcmd442b>f|`aWL0d0~NtB}tqG0TF?X>@LaP8CKxPNyRiQ8(@_jjHiM*dw$h3qj>6_ zdY*cQGYgpW>3Qayb2#G}|IbtPR!{eYWk$W<=ihvKXW4mkz4g{xZ&g<-r8X#~3UTww z!JVy&RY!dGkpK5q8>iw|=ROBNOL42uzxnxcd=Q{3ZX!)k8|%BNGj}>nwVFTRZ~9A1 zWj`O4j-;f1_w=_z@Iip?xJl(`P;o%zaCIgA7rXfVp129~XTWUGFfI7sX4OT_C{xic zBsoEqWviiT$LjKm>ZaGgDRN6=gjDo zS@Y(DZc+GH5Fn48aD$K2CBWx;m0j>$WG@wUFC?dqDzaCWeJ zT&=Uf?Jlru1F+O(<_HxnCg?adB7I#7R33GiI!Q~k0qEplHF-Y3g_=AcSpZGW5lt3| zCdBh;lGQbUM@_bc3FFsfG5!y0WwHH(tukwrigqRFWHl;%O}eXY)MR@7y!p{74fS*9 zha2im`y35Y8J`to%s#g<2DKtQ6^_vfDx5> zDY841Iaid~U6di7PnjI{|Ckq1nft}S{uuya%o==zDSphuia(@^wIeuMc00#||dzc^A zsYegH+ih0SI>MGKw>gVZt+lFbacy-|bGuP3TV>S%iG=$K@#g}>%T$)Sl^|9Qg3UCb zdD#I_2SdSW?kp7@O4thJ&Prd79F^;ag>th{ZwyfH3eCTF*Q)4nQlFySwb>X4y3Lz2 zy}Y`tqO7i>yt<~mvaGI*y4UK^T~>w=-dr(qab<1m$f~C1Mn?3;vc{_Nil(wM&@(*A z&uaAV0MAk=f04UHMQcT)C0e7j#tTcGlqJ^ZSNf@|&s47ZR%>l-OE?(-^P_-OdffUC z+NKd*6kqZpfQXxaOn_iHy76P~^C}t=ai7=XRJS%zrU*7yejQ=x2?r*aV1=4Ie-+3uM@CVgqZ!iE5kjmpXD zpq{NmFB8 zEn-U3c3tUHp?n}d(X1Ut2B_D=Q6F_*knp>ga$iV4{-)o%OI%7p3BZp zZWl?U4p!yz2I9k)C3n0MRG1|nkt9pD;yfne9HC6KanNY`B7g<@kDps9zutdB=ZOL! zG8b|Dqs&L7kh!SVPesf8UhAQ62o+}TU|qDY$b4jKnTs}x%tvaOXRBl~7hU1!re&UZ z%o;Wbm9jEFA~GMDO6HGzRI<+prdqcA=!=Ak$lQtfFYUZBiOiBxb*csoGnS321=yQ0 zbs98SMRl@2Q4Q9;psb~}uBw`YRXej0q*ODRQ_XDH)1S0YVt z^~l%e3`j8tQdGLv$wUS^UYCirV>*hpi|Yc^879A=9Rf362|2|r zfMuB6sQdUONyB9ENdRKV*_fjH7GLe_zDkFjWhXy!DikIg>AuDP1QmAQqmp!AE6$f9 z&QT&xvXIkt2S2w`a8PI8dl7?%FXVJ({HM%ErI5KR@;@)MclnW2_WpXMQ$niK|!qVWF-3#-bT0X9_SwFvWt zD@Bzhq6+bRs+9B%;8B%_NLv>I0IF4y5uwD^e`3vZtJ@-J<^<)oq_0Y$%7-dmfr=jI z8xcyb3kXGJUXQGVGS`bTD@7UN`IUJufJbE>4g*H}k5#4#zo1>ZcMrg_U2bz1t7w0+ zn5^8z=_^yBxZDR z0jO@HMfjvz9|mBl*6l9l^aIEOU1(wYYMCxHeO6>zlp~{5;7$h}@)DhT>>c1oo!*RK ze(zo(IyH+<#PjLYsQ0sI!> z7M*&I4#-BG-ipA_-9L*?Eus_gd^+`962PNQkA(rFi=F`0Iz{n`cIp|_pE}*?E|TeR zM7fL7*Qtx@ME7Gmp$aEutFys|tVE?=HvchQmnyv-*&izXRaDwvR3e^FrCwtbRC+uN zm>xvq#Yo3U50!eg24JbuU2da_mJ&9s+{W}(>a2>aO7j=aHibB*S>$9jdOE<38od)a z02*B*8XX`S5znVl@1g)68|jHKVDvN=pni>dBmeh}57g*xH>#q8M5Ab?8qG$Lbn5)6 zvu1))QTQxy^J&x@`M-}FHF`IK8isqdXw)Ve5znvDD*-%e^iR^(^+K!BUibxVv_ppg zEH&EV?yI7M$zq~%_f20T6c^D(^XtRYP5ZE1KrYfgJFE+EqBieESRJ`av{@?J5YMO0 z4u1{cQJW{jfYGH~fa>9}9m$z0`G&NPXRJNPSq3&A?BwyXPFY{k7Pjp!Ky$v7^Y4h^cefxeN0A=0xba#%5 z4kIk)2XoSwgH_T=z~aoK88^C9zl8znto#1pu2<0!q>i%V`ebECRDLsTv+SrV>d?qk9n(sZ{Tj)Pr~e2)bzS%UFSUk8n2T_n96!X*NJiBWZvE!T851=as=tTPn8e zP+vg`Qa%tj6b%LazaEwIuiQ049aW7r=?CFVsv4An5{QnW>jUOJ2S{UOEnv#QOMk!9FfU$Z z{zsXQO(F9j)IZ~7-sy#BJ`pM+^Ps&%=HpV!Jm?6K`8X}}bxDiAgRuS)cB0g);%tS0{N3r%$yZ&Li0HfsBv_l~+$cu0imWDP$5=O}rbQgauX_Op#9Do>c zPDs&vhyK~udrg#dx1Pw^o-axceH2s}B~MJ!d#yO1i8v>UILV^qupWMHrQo8Ls>hxe zZW1MjasE%4PfQ{6Fw}p%xYg+ZQn~Q%W_DGd%)>T_%qJ(2d2hg?n-04SFpM}Sdl6@T zr6g6Zsx%JOI;Ntmx)oa`+F5SGN={@>bW%NLruFm8)Re1VWFt{$Cp-QbR*X6yMc_B? zjci&(oi!45i06wsJK6b~wYAAApD6(1;l%-4Q;}!FfYJK}0O}8)!^a0;ITGw% zEhUh8Rh_;f2(wTmSOK{hy!H{OFO^=Yeoh_%bjc6KR~C*9U~6mwg&J z1j^hl${Zrf5YMN~&esI+sLXSut&17})ynLNPb%|#0G5rjXPwkPCaAjfl|eBF%IHme zT9-e8K6!{PBkBWusLN-OgQ3f9qRYXe3-Nrqj5s8KM_rx|17&!^8QJO7Wl*~?+T3?dpYb{Q$MjS9qnhefunE~=8c8Bt_w z)d^5dMRj9Ub!+=mVHDZwgm7Pc!dRni{-XuaMYh`k0;$bKFiAE?bN=Uz-6+FlsZZPu zvi%Xs=7#j;U`eV$C^zGig4`9L&LZ27E<9rwN`e~Cfk|sT)tMF9$dmj;qcS`HG0L;Z zw#7v)-lmpqCbdi_AMtC%wu44xfADLhi)`Of>uhUV!bvVyb_ZaYTAs{Yjw5syzT`y! zEAHz7g5^3N$6kyMB!~uhQHxvFG^P>NRy-`wPWr@b|2v^94N$&ofI5q8Ke$*X*UPGC zfaSlw0Sr%H1C-wo;K|(n0T-!&u5Cf1b(u6kOe1N4ieUU>ZvPFnHcxLrd%`WkjmD}7 z#y`#bf!dEqQdtv#Wu>i?d%TK{7Oy&9OVU!?+=8UM{p}d^ zs{cX?QkFSn<=Fx1oI^h0z9MsF%pqS%KOAQ=hols~Ib>DefE4tz2f1@wdjhPzHWO>d zbQEh>1@j;J*|)TV?w1fmw~BQ1D&{|4NAHVrmA|9Qe5{J~FV4qKPB9;=!ur2AAItg| zd3@35_|Ets8>w1VGymrt@{}a=F%hS_O2j!u;!a==X$dV9LZ@m%#bl>1em?*=`cL)Y zfZ^b9s@flGG+eB?Bg?77m$^<|R&K69TIj12pzm~?S`A$rgB~cfZT=IZeV1}-@MT_Z zjWXZ_?PI{l*zx~VwNi_s)R5SKrlEQYq>cZls?{R&-S~zVA*40N>*HNiug=|4~rOL>6$ejD_9yzVg3@UQ~51j zbD;+bOKbIi#wRbIrE9hUik3b--qPZ(HQ4{;`MoM7`dROPx&|KvkoKx&`;*EkDX7#A z_fsJ~b=$M&Qv+Y`spa|)8}*FTy{DG?vooKeD=?oWt@+lT6*&~S@f{8TP=Gm9a%1B8 za^t!O0}^uD@=6#mLx{$UVRtER^7t|U%VGPrF7hzt4AmtYKG<8AedEXOOt=4@FFa+? z9J?&Qm&12A_rus@IUni)B`qtm1Wf9rECb3|#{XCwarl0bFoD%UE6LXZSPtKv-4mqL zfuZdLElG22RdZ8y>zH=ilf(D-A_)gi*mvCg0Cg^uea(Thlx>0G8;?9;SWZjNOonet zK?y|1ad!RJcoG-N4r7^Hr!&w|%G~MrtY1g5_W0U>jvT)KOFQT!6hioi@A1umWsg5o zkCzwG7uKVxXG?da$l z8PG0Ei3a_(f5*^XAMCJXL~egdiDu_NF|;=ZIh-~V?b1|e6RAIYQw;6RQis(8qHRuz zHY)+`t&qc75YaA4iN^YWklWh>9o8v{c5zCyI}^~}8Q`#5L$pmP(QN#S>Gp1ahvjOb zU6cxKSG)clLwm2E!{RW}E=-BGI6=tw`#M~bBiaQi(auXi`=F1*ExSZJKPB2f6VN`~ z!QoaUqMert&9(VYOt+7EJKVTUv~yFUO-$hSaW97(aEW$KN;F&ljdA;=r^Bs=MEhe( zG+Y0Pp?%uJ;hsyPot+ZR?*ERVeb(L4TMU7ARw}gmJ_)*gUgB`S9l4#EfF{wU9_xQz zww1jXpw*J+PdWo11dtF=e_#Ub|8#To#&yvCzXaMA&CY*gntajK;ifr?bVf?Fw-UI0 zS?qB8Ez$mv3N38se=%-fb#b^8nP{h{M4Owy?d#4CpFSemX(`bd|AQgm8^H7N4t}IW z+me9xZIKf`#h{&%678D=wEuQ=!Y3QFlT)EZMkSzq*TD&&WYA7ZiN@z&g1UWQ=!8!+ zXeXvbv-y9l0e&cO!Y3HC6H=o6D}mdO`A+zFgLZr>v`HNj(0W8 zPu$W{yA1*qTk7oimXewLZV>=5=Q<}zsx%mcpv-Pdgw;7ZRbq1fQof^8_(FVm^LT<= z6svy><>fp_mjr=wUQ(3D61cpQ>*&2AKsh%V%4C&*@@kHwH{k*0oTMm~2`I0jDnTCu z${&-Wv?iduj+(^T2IcIeDAxaD>c4@C#90RAtN_Z6_#=!w`OyR}Z=xQ-5vxNkK0F-4 zi?>XP=}&K2gnK764nFinDEb1ZPuba{PH$lozNKLT-ZEu2py(~qoDoR`E~ zSSBGWTmL*ydkZHQ-xWD}n#GR-qlnI{iD9uG5dmXZ-=lc*9Uy@9T?(wJP6E~s9UNVe z1=fF4VcGg`jMtB7{BI4fZ&P6{PvG@af#ZH-u)ax!^```^pV4=|HdtS$!g@X)3r8I| zdcPy|`YHw1@9h2$Pp@nTJ?Klr>&sMFcKtVom4#09g~9qF71m)15@w?xVJV00^`BH& z*C$}*peucDus%7>}M4pcMk2fNB z-pX%x>9CNkOaL;3bkmO)4K7Zifz-dJ+xqv#3aw2IP|M~IQ}*B|AuY7^e+vzD^Nf<~ z35tu5(quF2{O4jenWpW)47N@|sIIsn>5Ot|jEmzN1J>05HviqEmS|ESL@^6+56yx9 zgs}PMU-*w)npqx>6Pby7KK3vz;1?t&wwb~H&(^&piEUWIr&JFBhK_UzZKH$p+c>MJ z48JR@E~Q{ckmE{(RW!6 ze$5TB+*LQGtg5N8y4@Al5L!qd#u9=s&U}rR&KW=&d@guWruib5ZU0TG1%Fu$ihty} zFZf;6IiX*7?n|TLF>8WYeUoCQNgn5r2kp392-eAl`uut3uQ_Oj%-Bz1>{);F7{a#{ zE2t&zM=DzB;D1%h{U|E~TO08VZE>~PcK`F{B(7F!WSCtcGHlkahN@RVUib>fRa@}A z=WGNPXPa&F-_2e^T8SW&#Mx$_D$ce!-q{WZIp{w-s6Tn|BjG__Z(t19<1> zv8fObLuv=?c8lN_pW|3bHqDAj}yTXpGG`Tkvpqn(Ud`Rk_o$ zf2k}*S|_p@F08Hyu;Qw~*O9}pDsU$ozDPw5%POT329Zc&usN>=5NWW2@SmY8@fub5 z&zacEe*l~@8QiTGOon|Ykty};fZpZMUe;jy;mF&F0$rx8X*ceG~aTY<8E}?Dy7YL?qHiBj&u41e+Cx|03IaT2%Ninb<6N z2B5JT)ZD(KQJ6&{klp_kv)hsG46z#?T%VDN-4X>VqH>y-SZ;pzfB|T^ZzG4pa(9d6 z4!4#gB9Trog3g~2K%}>I2w$npd}5gjUzv&JlAZ}^449LS0VxxMBph%4+X1=gcpKcA zV!&z2otcRNQ_59DRW(bo5&uy!OOk&ZO-F0k_-)8MpY$M989 zg?LRWd{riP#DXJF?%@-!#(KFSMCQ(L!TjCp=f?Nj4YuD>lzojjyYCp#gXY`l;($QC z%d8#;1ZHBsIB|-omu4hJ+x?6HGaBu?$dNGGRx#R<)@VfZ8EyB61Bf(QQTS?k1O!IA z`WG0Dr*IgfzyjVz?o+aHDVg2E69S5IMY@Z7qMW&c%?yjQL9Lm(-yIYA!cwrZ-ve ztJ!AM+>7hKu~}}pLW>yY>+p$YEyjnwIKBHtv)or(59V41FEZ_c8}{9Mif@+daNJRU z-0`lv1weY+-mL$yxBV$eZEQkdCeXl{s@xy{r-7wBgxP)Eskc>_>W08Sq%LxfcIqHzJ`^r&ThZKd@ccto zwctOw;rVkL8t_Zb;-*z)&S=2UD1`^K95TIS#gcVPy+7r(t*u|%6a(qgwsygaw$Bz%d}@)emzHLI4yjp*f2i*e(! zBypn@JmAmKheY+u^eBx}i5=zc=D1qMWYhBKcOC+Qf!>EW1V^bR2?mRp|1lU`79R}m zH@p`G^(Sw~-3{NdOJ9|PUl9*r;FiUeRb`E1+I>hdgMlbTOQLi%Tl~|UAKl*Z(Ykch zKxZ-+HIi0swCmrxRS_4m)gRos@(3APb*^dE^pxVG4zvhqi?cS``nRFahc|w5@feKu z8aV4gzlpONwmxTV{ExJ|Vaw+?wz@d)YPdM-{+T&zOf7NN#jJnQSuaoGtTM7lNE0^q02MajpOZA9WLnL2fMKTfXT6KgJMG&42f4(BU|8sV+bFMSS=*cvdv{OA-yl37fwM3~T0p(V3P@N_qT| zbUmSEcpNm;O`U1A?BPMfW&63H$$>hF%eHI;BwOq+@hvt1NNCcQONG>*88|=+geWFm zc92<5=o?s1a5bJ;Nmzj2aX^+*XXrnw7VH0-m|BDZ_+4xYHTA&t*c6(pBWHDWU3G2a zn5y=6zJ}1kompAnF5y`-8)nX*r-SMI%wjLAh7@i;BVg+vJZ){+{@hYbgYd12IHi0` zO&n8`iEKa8!0%ZBYhea0&7w_& z2S87qM61AGQdHnb+^dyS5~-ChT@sLnkC*N{P9A7-)i{iFh^Fc38YfE;mC|g6;}fzi zeJa3;wU8eo8&M0nhr*#&zA>wmN*F{U9c2*PItCDFvCiRZWCMAt3SX0n#aL#IctUFLSS?YWBI@QZnC~CdpN9J}auf`AuNdwqYd9j( za8j<^e^Nk98m>$DZ*Tt|~=iW4;=EwlUoVrE-3(K}#zVkTxwB2W>v)2yVM zE(^v#n(pVwF)-czV!C6j>4-?A=R{77|9aRqa&qF{(fvu5XYNq$PX!qaKxs#v0x-wb z&&?oG`744=Wfd(LQQLnu$ml2o#B#g;&FIOBV6l6Wit01`F+iM@rA0}HFg>D#Hqryc zA^3y~F1PbPL!S?s{?mO`URtuRa$n7)#dR*c{V_Nu2!)tq1X*!(ph4$g5_*Hn!(AuS z6QF6E(KNl`!D>q4ZNlI!Fo>n9_=bdOW~QoW*FC_byb!J%~X}NX@)HFz~!sf zuj2k!rm9!Qr>asETxHk)v`c{$2r*Tqxqr)|pw=HWDPtC>$BKe3{z7h9Tks=L6kLV< zPgC(H{X)aO|K_)sJT-X)KF}7nD7d<_*di8Nirk;g0;Yqa;Lrcl*fJ++ajLv5^#7lt zAg$YeQLwCROjT9mnD&p<2owd^1p7ZvXn2Arrs?A}(EVRB-9H_|CcdD-bnlir%bGU> zjmo0nPcDwk*GD>`&ax7LE=ehgcF!oRku6!uBzo;$V9S1emG0MGjM3lE032fQTD$-6 zD!tK_q3=e#BMWENV$(K0%oFzMyw{Rp?=xVLuqFPY=p7?|6W)=8YJ(=eBs1cl28pk; z_5T?0n{nc)+hmA08RBKFwN(|1+g}EwG*_M*4g#%81aAX>fjXwTJXi0(S8G1 zq-$w^o?w+*COR%NZdiOjO}nb4xv{*W(qE^nY;LJBZxU^8s%x%nu7#W>?xR84>uvmt z$$jet2gjmo+EGIrrS0a5D*w68$!R|sNBhbIxo@j-+(m}=B15~Xs-n80%5U%Fw4V;r zJ}@_d_U)q_cMn5*4@0}Wy0*5g(x1R2r+r==?ZyP!cZ_sUN7Xi&YiO6XG}bm(S=!}` zYev@8weYonxE7CT#;aS6y_q%m+Va3V6KLN#)WI3nnzlK=y0WEdOj&cKJ%b}P?e#(0 z2knqR`>r95JKNBnt>sReHuDUh+y|{spndm_4tAtyxo2iZdu<%;ClhFI8R)pX8``@Y z+Rcm0$}4KEf2X#|$wAr&k4&Jwb%5hG8rqG9c4Kp0Q(4RQ-AH!#X9}6>0l^N}=akSeKXg{3ixUGh^IgYHNsGNIPX6Ed5!h zO*`*1^bcqL$K?exm*ApkmOB{(%oxD&H+z$XpW`_R z9G#sVobskQu1l&T-i4gzd<7QE)n^;|IBqCR;FvYs!LcQpBaSRd@T_0-mS#UkyZ-I9 zqm#XpgHx(C$2CbfmepnWevo?5@^kz+LC2h7j=S1$T%D9-hE{cjVQp>o)JSXU-~pW%GQ5l3=0N3*hHu`T%LqsZCwVh6NgF12N@o1_rJy%77lXo zDizIeSyF~o8EH5!j^SAeZP=mIak+SiHas9H!-|X;a@T;*q{rC#UrfV}AqRt@X1IR> z!!gwrn3!earFC%(?f#D#!y-&lSg(bKZAloG*Jai%YU3Ebn_$vTn7Odd0)|VIGR&-7 z)C3tGyJrH!&izg81q_!YWjH1yla7gF_)r4FF8xe?35KnS46Dk=WY#UJgA9)wo4~NR zuSq(=uq7$O%-XOjj-h@2Gv+j1`PP5n8De) zh%^<#P;%DePX$BftXIe9tV;lh;pF(gi9|Z5Wa{ITxHvdnA2*9q<{{a?MJdx{Wn?K? z-w8JVF>Fhb^=(4miH>ssgH(B`$@CMpdTD+1R{@UkB^WgG2Nc zA}CG^Bo4sz_9!CjpR2d+MDxle;G$CAreo4+GBzXp8%jT&5+ zL<5F1YVZkQ*psi*>&T1nSNNR1IDO_tNP#}hj(-*!>ZV>tKJ9lNZLT9P0-Y4=$fvCX zB>Tm6@h!$wA7Y+n*MF|nTOJpY5XEG_NV?^5k;aSL@_2#%qdM&kkxLUpx6XFn!!rk1 zWyJSK;+cb7-EV3e>#DIg-Cyf$ZK=d@1!fHbOAr-h+)~q<+a0Gj%*UJ4f1AyZ6q9)< z^gf;AKbz-T)q=n9VAk6^s%S2LS2LWB`e4>n?9dxZdz4TBlH-&hC=zz9kqPV$6c>ah_UB73->sAxIc2}#B$CQPK)PDW%y`$(bx zL4ZDYsdWusCkKvWm)doO8TbL;aPtdxL{gOfj5uf5{ZGa%`F!ZnSh*aL- z*r;}WCRXFSL4M(`NQ%=aBH8+1%yP%@xeRT&20WLMiRF@rR7CwWGui<*IMe1oG2=NA zyjIhFP>gq+H69U(v_TN}O)y?b`0ubc@tRfm@0l3Sx?H9^BPomc+?lri6|>y2SgDHY zx3I(K=4E2JczKGbncwHm+B={bEF^ zu;t$%*<4-GihV=vzi%gqY|qsyU;OI?DCs54MS%0|_#YGChytfv3t--L zTUS|8Tiduewt44Q3c%h)=HdkBZ%$}}Bl8^=KNmXY&3APb<#=+s$u>bnS?kDz%sr!y z@NeNRCb+=Xe_|%skgcmldpc#h4Qgv^>#Az1ZS4BBZP2Z2&;%D;mSBR7Sx&V!fqBbh zZFNOmRf{J8O^~<%pGiNsTTDPn&<`#g3JEx4yFPx#c8~x}77@*u!#H>p-nOXcSo`2@ zix~_GX^I(wTqa~w1errz_;^5!IVjUz&8?n2IrL#jSZRiY^hPvGpR~me7mX4-TpMqP z9>xwA&GOCEO|7BmU*~Z3jR16QSo0#ze>t$+kYuLbCNSp$hP8$pbgg04|Fc@drmj8> z^0?tn^{oH-WqkNF*fd5oxG{+aQa#vY_dndIYYot#y;{Sj6F`wuUki;EOI>T&bh8J| zS_5gPs5NYQ4UqKQ8{=Dy3pQwsiya|#gRV8K^4A*rna3-81Rk$6%;r-ak~mGsO!1RZ ztTMdsZbD_5p0`H1n{ss*sIF>l#bY4Nz9BOqR)rKE*3$1^VS#~9+@UZ#-}E3otdwzQ zWOK#H#g(VzMjEOn+>z@=O5Lt=S!Zmg|A)z((S0r2R(5_tM@$R zr>5(t*8oP!K>1+va8ECmCpanT5xx;;akGI@KDg0LO46f`xzkKGE?b#4f%~R-pb}rj z9;jsbKr+G2w*-bnaHJf{%cIjnMh6qXA`-PPsRe+?pMBApTVkf;L>GZCjC1Fy!oke)tvCbBAn9_OElRJPVNVn>$E zsFLRsftkpk8Ul?`j>xqS_SZ!7sV9b=i1EF22-A)ww2}5}hF^HcpHTnR!O_@}k7_;X zz7~5_>$OapJEQTHf=~$QWS4RLW0W?6aL?Qm*q7iPbaFsuoh+uPjIWoS2KtQFH^fJ4 z4nDBNW!L&fC;h0iTKL%m)%cPCGQM8+G^lWVy*bI~BtCuFx1!0-Uf9pdM>nLxo$=17 zkS;dkKNt_p>{>*;yj(=QIfaOq&-IIlLNne=e#Epn`0$B%IiLTeh_@sWu??`$_3{e< z!xZ2aodRqyqOc{_;l@?WAJq9bUT(OeyHA6iipZtQxld%{!>7R&b)vzoNi>iY;0oLS zZ`CQl22x7B+;GLoph)Mk(6D^!6yS&WQ^}s>772yEjCq&x%Z&_e=8v3dSwX)chw;IGY#T-5GC^#kXo#RulpZ zGfE525iT4o!RaqcJx}R)WFV#H; z-e>$*8c)Ac@ zlt8@qM8|#0(0)tPUR>AQSXS-T5K_}VOXynKcO=l>VZ4Jk>S?(@)U?xj%N*rCKS=wk z-U+n(jC0(#4ehrLZOpuzs+-&0x~$vu>^Rz-|FEOS4Bi)eqdqXSf5A(9X%DfQlFOl+@Rc7bNx3ccmJM_`=+7&rlFnQ!+P2JtsFkN zueS4_nA`)f&Hf!j`yEZYtfjTCs+#G3yDzCWDLHA&kahK2kej3GZSet+qY`?+U+wyz z4uCp$9_W6K1kYuVLIU%-8Q0SbQl>AP$GeXa2CZT_!+^vDqosW<(^??{Pz|>umi$X0si_ zwGs3UfUn<~r4yi-<(xq#WkDerf=bU9?a5R%PD%g8IO%nV3r&HMZ27 zo62I8YC|b65k(T8>z{)-Onh!nlKAW)BL5&F->wrMTJ?8$H1}Wl7EdkzOP97=vBFYS z){JML+Zl5TRX$pmCyc`qMPa5tWU;4V^Y^9%Hq-FD&Lj91nb0HYv+=pQ=F=cNET2ab z_)H(+xQ`h=k7dk<*4KO{3m+-H+)$aoX9k|lc^pt-e5B6@dk z`bFGK?xDFwgj*v%q1#RK6Lg!4XKK10)ydCa_KGgDYBz$f&&+2+47E6yM6=tB@NHFz|A zHZ}rlCcA++8Awyzd`5yU3-GkfQ-FYDJe58Z;~4dkISZ!F#b!OR9MfWQl9+Mx&k5Xi z$D=lP0RpkQiReO7p~kUAy$c|E7H<~Bnt`!zm z9lQTEX0(0r*vzwlf;!Kp&nABOJpdHQL-^cg_y5ND?1!gio;Q4+&zO%l{4N(BG9=z+ zpZ|;T!6J|Q0-!MbzK}kjWW(Cwf#LVL^w|W4-(|u;n(8h)|A{eK zQtY^|01A%rO8QKa55L^hLryX*-gQKR<(77F+frHCAKCj>zw3^j8VI$47h3kK;u*O&|@8Gyw0EK4SqFLcNp{ClFYP;Vo z`Cg~EW~%=L6nxFJOT`c(o9?TOW?Dzz;W*Z6q@Ot^!X%d zruxr8#n((*A5P%4D&KLp0t(HvHGN)*%~by>h|n9l;zs7Tts1OYP`*9Bpe$qAw%tXR zScPjf)()b&c7zJo+nLmBB=4Y&L)vMW?~&1Pk>;KNGgbu)Bd4Kc^G^o=sH&c3OEyF# z($Ruw>mRWSSKlyFa{XWT0a zG4b%0#M?75UVO@8te3z2w@XqOP7o-N_5YaZ*1GSC>99lp-AqiEOrRpFr@4vk?i(L4 z0d3bY@(0-NDY4xjtnG+Mqz$+s>V4}2h_qe*@EzD_M?5?iaz`e%Gp+2Umd11i+czC$ zX*BZOA6)-oA#k1ho{E-G9(dh*nV2q#JVjK_?{)Vd7%%`WR}?t|miw1j?hI==BGPil zk9WJ3xi+%*1lZ@;hTbP)S2@r223SZ5f#-O#s40#>%Ye1X~j;F|AQ5u7AyXrwIUITbc~2=pMQ@9zJcMp zl+y1Wz;fJOnOHF)(K9COkivu%&07?qJWWQX2Tx2e;el8ZHcxe7Iq8E;Oqf)rBC4p_ ziTxf7#(&zcbL33e@84p-Gp+rINTef-tPgSg3&y@s81;c2h==8*yECz0Z1!#p*D=+Y z0ot)rKPBPd59fO*IR4T34#FmYsFuc6?uVIJE+|bAwbFdVWDk!HXik%LiJS$KJtHPN z%bJXcK9fBh+yAiy-!SBrhPNoCPp*K;wq$NH`!-CB@tK1BASwN6E zXvyE>!D=H2c3E6x6iu(4{ED`L2kAcw`bsB{|3M3$fjtqzoRtJLY%xCc#hr4$;W_t@ zH~2OTI+R9zwD%R98u!30E0T}z59zofc}MCEgOA(!-yM3xAT(%i!{Fm~{Ik$dH}!_W z$3GD3-@$V`XdM_K={fgLVE?0@-rNyCy*Uj?F!U3F_zzMbl&9(8Nbg_$^2>7X<(FE2 z(9Oo31t#2YxEFrM)o%lS(7&=@!w>Tq0*y`SQ2&~WU-e1%xvcZ5&}(O0f)4;VN8^(~ zcK@r1fDA6h_29!T%gz#2pZz}#E@y<`6$!7uR!>n{tTarcAzC(|k)l!53{Nz9BR2)tl(S)BsVUkq=}v)@<(_d^U%~!6)th zSJNcrD%%-_XCOW2&6HV0fZ`I8??2Qc(`4*t(Li7?uq^l^C9 zD#XDQt&&mHBU`c*wVn!&e~fyEQ!V?xK~XFHsAtu>j(wt!kZJSxYXBIkSHoYg6q zMneAbsv10wS<~)jMGcsqASn!mM{j0pDtIDMU)z0Br-5x{E-EBO` zcQJJkUpUJ6LV8N^#4Q-a?+gF3^?yU32Z?oe4~@Mj_0ZqM7Ytka!W`UC^XV1-wsLrk z);2yr;5LX}2N=E5YXhqniLVj{PXrk7{f^tlp`Vld0(*>6dR9xbs|oK}hX2231^Cw^ zg52Zcq%{2`2wv~FG?RA28Do&$|9Z!N{|)3>c)KNjWRlqbZ|LvEe)Ifc!NH-cLB%%` z|NBc&;YfT}l98CBH(Kf$?*C;jahHDhU^8Q-@u2^4F7XW3|2+*b65%>w=_%*q!>7SB zH;V>$C(&R4V5z|ifMNQ4x6UP?L3_ExvjuU`P&aih@$486nz;mNr^qFq<^FHB*xm6h zc8U@6*^NT#F8%PqW)iZw#Bce|9IZcCGG{Qkgg=vjwkm52egxi`^IU&lTV&nCZ|g^v z?}`s*5{>wTou3Q)6;RA1y4K_GeEXZzpz3q}rpoI zfAbP;`bDx7#(|756=KZds_L4u#=sf-N#;x1*fKUee?!3F%o9fA;fd%37@T>+#U{q2 zLpX#6EMa5mv7r&4`27XD|3P~sHl*gOqsK}iW;Zn!BSR94XOtn3A(@FnFB}t4h#A5W zR7jr>7gH$RK_RA4S5e6Bf4?if4@%U05%o_m=9sAY)|sDP4s2h{d2we*$*8$CNz9R` z`Ql8#FluhqQS&-Be=KU^*J9jyLi8Iyvi0HvJ`M5?C71rc%jM#+UxSZCgKbGPkf`~R zUH{#tqb4+HFKWItQCiMIL*3L-^Cdg~-KJ+?q@5yazVt`XWz^gj-(odDg8RH=*FU%F zsCgX;F>11Uk@Tr;!)!jj)=zDpj!%;LN#xgL)FO1H58X@T0oAq2y(Cxn(dH_ganV@g zKj!x+5ZfH9fQ#*H?3!A1l$Aq)=+Q5iuXdf>0-w4oW<#AAWt5SVQzfpy^7G}`KD38vZp6O%T6hvC|)s+EZ zSgSq6#an-Mg6iU}zlg4A#PpO63R6U-{DsF?UkvbKt+s3Ak4Uzkqvlu_{bN=s8`dBW z$KRyz_!{dU%$!WG;E3L>K385&io#=Vffh0yPs>0Xc3oWkuFC+&UwEu(X)Z@6ZTDye z(DxM{U$gapqh~p6aS~o^;*C2CjpwC9n073ojkH&@<_FJyeV>3GS$O=`U9Y0UXnzzQ z*Jsk$X?E?HAil!m>u(1H;Q)IidUJFT1wj|PB(p9SQ&a}nH&Fl6;d*O)xRwF-jaJ|A zWFFi8{E=SAgS&5ddV}>34zTwo8J_5#FzFlDiYE7ZaX-Dsw%>S9M7%eJh;JhPO$8HO zH8VNDhJh!X6%_H!Q6l2~NknWjBEE^|zw4O;+^=(hE&tDQfHxoUX;8cw(x@l%U;aBj zd`5orGtuCIBpOH#@RsfW59k~K8nl-Kyk+x$3k|1AoddkJ3j7&c9?&@eX{X2m-Z~$U z%mE&VZ?VZhf-k&vyO6qH`@$A~4zP=PYMkQ7wG@t&m?&!ZjgPu zo1d&=PVkWnYwBf}gOs~DR|lvuEp=s0jkW&w_$A2+x~XCw*jgW%KXum3=$r)&3mQTw zdi;{(Tqxcm5%DHZ#1{O8WA!~9R20Hvxt<4;C1IOUZg3t5EyB%A094-@;_wc#l*%Db~P2IJO2(F*p^6o!>n*Aj)P{=nQpn(28Ek z#ba(KngJtoQHrRL|1r0BZT(O89mU+BTjU(%2G3J#1ZS62bTn-q|t_k?^9|Z@vuklzD$gU85)e{J#c3n zE+>`Yz#-`#y$>q`BGGWexn8Xehuu=AWMa4^A{9|TT^|wSeRx@bAw8}~Pf z5)XsH1DRM1^E38QJZ@kFBGP!=yah=={e1wD#@jjk5MFviJnV^jC==s(U1IKh<81{gk`EI1HsDbb z>7bu=2uMS_jdZb_)ppQ7WMa2CX^NR&`ez3Pn9*iCM6f9Az9u%?WNk)7B5e@F z>jH?h*@!Tn8s3q3*gE@gCN{%UBj})W7tD;-&of2?cJh-6NPa^pFaCmiKip8{b2lIl ztv1TVPF8I-9I1XrCRU4=r-+(qPGYvt{}$jzv-OFfQtrMkX1my$jfg~&lFsM$`4MF*UcK!E3UB!R~?Nu?pvg^MV8tSI5Vti%Se;?FU4AM?f#rSFw+KE++2ji<4 zvi|#(UH{cRv)_{>#I~lnf6Mh>tv_gHiL9}mWLxxKlCA%K?O*@Ry#;XUnx7B+BRMk~_ONs|o+WM* zOSJ~<=w4cdR}Ymn*0z6wxiPpDMZU4?U&iR=@YQ|YNiysp&P_7iKOMp*&Y{6{?_OdW z_1j*7@KTRqO}VQj`NUqa)rfORwm{MD8AUa+B}-B3+tUJU8Py)fi=DlS{DGN9HLKQr zqShw?`rIp)>rTbwnY|5r#ikT6#)xPZbQ}_l55U`;DewI@!ZLOneQ#aM*9+3x=)F7rqP3e;n== zy9Y1bH4lWq#g}E~3o*4;8DID=Q2%haRqUSeh=R1*0yRDpqrqj;>1iNL-m?tpvD69+Ns% z__5KWy%MSLYoNzrg~KqfKDfK$rW9QNVv9Wz-(s@q;m5xUsfYD~$7>`+F_~s3-SqGp zzTmELTqJP;Zu0rUPeyTp<74+yv-O)V#JeAQX}e5n>_#H3A>!M$Xmx_J50})JOPCq5j^|)c8pL=?I?SjYs zlph``%)k#Y;h%}QbcP4oQ)>*{~2vJBWeuZ(R+w($LwD& zOA(dQY=#Rf?tf)v&9LGux^D!NME4D9j>V5lvP#*=29Zc+7VS7A1BjeO_ozqj)9-V` zEV`gTOPTh@i1NyGnhp$j^?NM?6ZmJ*%`Me+=*aCKck7=;PnZKi zs@&^UR6kCP9OL@Tx>!t66VjdR{h%mA`or-dog-9^aWLJK-k!Rifc@)KH^uWxnJ# z(JMPLVJhX?I-1PjjKwGM-th2n)=i#^q4!#cFoSYJs&GiDdAaxy6)Mb)@i*@m&jl2z z?1meZa(D44L4iVMY-0vV9y~$~%8sF{UaH5sHH|CQwzaHS7hTr4cHN}rW%JvXw@|(k zp*T{g9u=T^y`O3~H7L#~L?&WiLB2bH%p`VKg3u(q2xj@%=ijvX3pWF%72A4sp|YDA z8{<=Jg(Tt-+)zDln8ydk*T_rodT9Mm>0UOI%o(b%5tYH4&55U~V(C3;6Q!R9>S64+*MxP2OuBI=zPI?R(8Q;6vej z*1tGLXxX*bT<4JX;2e65T&4Vv_=-;jfRRiCZjx&9NVOo#VgonB@Y?Nn#(wIU>Z+&e zzP4peTjR2}LmSt%tz0pxb>*7njq9velc2O_zODb6&h)yUXku%I1W~*u|EqvRm!M&q z0pcX_8jNVx4iG+%_Va|_E}w#Z1IUOFBBy#UusT z|5R_945Aj4`blh8)IwYT)Ar0;luT6NQ)u&l13D)?KEog=^enXdpG>wP1yf;IRJ$InBGbnh^8SQmDZz zu>m?v_j4=U>L<}nO^8JpFTB&DB5DVl{~7+X(Ev+A_}cH2R@55G;YLwC65Z6qI8ll06SZSifImfD3Xo*or(-k#S_(eikdZpLi=%bC zFo3)(e>7LL}D^M9{56rLf--D%GX4pswHzowNd*0rs`5H+EGZCm5$rj_egtXp%) zgsCf<+Zw&7q#LUQSR}loAzozb-^P5KL42tl4hk_GXp<9%@Y29V`v(N#XfT5y6o?nW zB<2^L?FZFE{XOmyIS~Mv!jic587Qa{^NXJL&>8a`u(;RV-4AO1^pY5h*N{-tePz}ELjbW_!FCLl5q#n_#m4e)2=-J2kOV^`t-5Vdnw0F;{*58$oty1?|_CY)wvRph_I_5#l}DSUI@ZX!8T3|06E=MN0V;o zRF(fVF6a4B4gW8sqz82gw7*V!QuQFMLR=3jj_J=HM4w2j2NlosRJT3o244^IvL^N* zxnf#C1NNW>+^}Ntc^;`2$(ma5Mp1nz$j*49s1L85tciM&ptNhTt^XT8eI0svM7ycF zxOOG7&wsj(2uO4Zny7^!N&Khl?tW06HD$o(%m6a`LjoTOGhOcrfUfrI(>rJ+Z_uX5 zzA%F$`F!Jq&YIE}wcA8La^9k3qKfIe+5KPI^m%jqqNXA#a1Q3@o%f2YGRnP;m1mE_dl7~#WopF5Wkzs)VSnFzw~)$0A}}c6&*p)fvVgu zJ+`KJ*@7LcNJS*>&iTKWZ4{LIDRoojv5uv)iMYY4`!RmxybX{zVX&$NFE~PX`}~LT zB1*hRg^KP%iTkSxzr@(`CJBulut>wo8_55(ydIrysv=I_xB;t2Apg_yrW&w%1oFQ; z+Qsva)GaAlR*yjbrw0-*11Q8mqO&YEE#b`?fz#ukep}`8a*GG{obN}@3!y!y#(SU?$a)?LP!`CZOHqC5d5N$3q+Wr9 z>bT+sZ{ovOAnSP_sIWlxW0LAqAHc$MdcH40{pg@C>rQ=>6Vhuh0PwBDIK4_lrXS;F z;u;o6^UFmeK1t{!k?AMC@1?K|?cyq@1YhRomZ)r#L=aX?*1Mkv zzf!|9A%xe-MW~**LAkfh|9-+6jiFpGP=pXodoL4Be~Q;s4BMN}KftiVog_*y0+ZGC zMkv5g2~NJ+9H$gt<~gOJ13^OqF7+QD`2Nb_m-)HK!xD7fq1Yo&ow*c~LgyXEdXUta zf*b&7UOt_7SOh4vF8H@nesj?vqVo&%Y{|8)|peMnWo`7py71= zOecT@m$uvCIS++;&>vFh(MR%s2tYkZd={X2`Sj?sljxD1LXSREMUUJhda!w*N1x>) zR4(<%S75cT<1o4wIQ>mNh~Ei3NDww9And0c4j+ZN{gj76Xwkcm#J}OW+?fRysrAPC zH-{`hLT@chl;F60diQ1hi|v=2Lhruy0ATy&C(+xiFFSo(gjGKE&fNmN$0~#6Q5G+4AVF?djZqn&=P%Z&&C3hk5YO8RS#y+~4;9 zBI>LKNuqOq)<3CpQ3{>=GyYTO&Z%`CVB=qBt@EPo=sZC3Uy!7U2;Q#F16GR8o$1?v zPpR_&ng4+@btdH`IuE#8bncu&=K+lW)VVl`&O^Z-VQC=!pT1qJm2ko8AI7&kVT|x* zs=0tBhQ+e+KZbRSaq&)Y@dPaC+yhs6yl4Oy^inwbK&k&gDjI-N-3n-4K1Uz8MGR1! z!TViQwlErUA*uw7_bPY?$;FM@T&4Mv0M|BSSK;#HNDeK6`j zQvs{@)@5_vsYdi!ekMMADf?i?e>QF?N#in=Bd3c+s1O@BmqLZNllcvn`9H|QNr8f` zBgwP&!9RG!(XN_!1kk*E?YiS2Q8pB>tYok|MnsRDlC*0G#GuEHON7-<>T_W=32y*0 zwp~w-xI(zaId?qAgSSfbhwfPf!$l}*iygO$csr#M?+p=eSQ7DW1|f(yBv054*_=mNjcfAF^Wf zlA6_(2dybvF{ZYzd|A^02OYSgtbA=$>^@W2eE@;Cn5OJLb@&?0hcf@?AUcev z259W5w(|`TcHo(!ACibP=+OTIA$G1|@eL}SYv>KW&V_AbxTa_Ir|0@Q*HF%X*tsf_ zw0ad_DYEtd3hqzn$3vvDoqT-PFDG9)F>7V$$v^B+pXX5XGTEF6e6)O)~p-+07z4nl%rOO*vj9Rv`sS$S4FA*4vPsp^B z)PF^$GmK2;G^2qkKRc-)j%G?blL{S>lw|!wC~fvr>aOyAz6(7dInjx_ub@BR8HY_t zB-0-rI7qvzISByB#N^&fg@b&570zQEV! z#diZ!{o{>%gc0);sSY;?*S)Z=t|U5O#HRNiE*~rTB_{WnvEMirT>Vk5YA|AvHn` zU%6(`s|(F91E+Ko3gQv|Y>`nS+mkeeL7}-@wY282yb0N-z=wMy2 z`0B)bdJz6ZEV0T+JO4MBPr@o*?%Ex>s}E5-DX9`woOI+#U`(q_N@A4|;3%A|e*=b2 zsvz8=0qw+tk*|m{lNb*eGgH}RRDsxKQoLR24eL?E{dUo;8`%Q1%g~$wD%Yo8X$Zn% zS3^i)$3WYPb^NcnNNfyl=rW4&&+F6q?9+>CRJ7c|*L_uu&p8Kq`#>(*czVH6`U>B$nz z=y?H}RIIU9MJou_Pt^t$vm~Jz_-YcO*=RfeH=T^>m`A3&s!h<0=tLEF8vQ(^pgT3B zcBj!lilhy6r#u(#bX$AFOPQVjHt6BS1-BI4tc>$tcC&^!cXD%#j5q|cGJ5jb!41rf3~S5|Lfd+v{_&?&e0*H zDpXlIBF|dC&L^`~8JV%RnRrF{bpgB0fme7Ty}gl;BLmD{{;3~bFQnhk1K9OO-X6;3 z`RkT5j4eYck%d4-S%3hYJ0*eGSWIN`%Zk+?K);-v#4p)9A!Wtc!ep}c%SG@@jKv{* znJoBM+~Yy7gcmOYR0^-G_!yA%%E@tFscL})#+99glQdi#e?q>*yg%-XVqe`pu}HSw9&sgx6?mgHX1Jyu1 za(Jp(1x+_4CEs+an(H6lu$cdOx_XG~jN$$-y3VxJt}|w=u$V^I$!A~LwLRAvv%d$A zZVLF6-E_=ZfTZh8i+7!QK!WRxxk*UrZmL zwUyPtC3(roYkCJ*um^jNFiM&N(70g#)Ep2%DHw^0i_|dwc`Z}0)lWECmPmA>O3TzR z{bMa5 zPDc?eq^g2C`pHWANHU^f9rM4KpOyPbbyrmh8rt-+tC3_Z{huve7vIt>VL;TenE%qP z&Ew;j;5XId%=d2M2X67!vHO7vy>(U+Zxxprdz1(|OIP7GK+qgr-l>e0ZiWoekyF=W z)}fliZ%nbqJUOJlSb)EPHOX2fIRB**vr;MXlV6E1fvLW{_*U*O@F_8FuqZJni4v0Y zkCXjx!{?|LWch5k?0h-ayB`W+UQ*>RB%;p5DVcBE)i~S!hRkL^nQo|pd1REWfpPgm z7BI$f{g*9nkZH(V(0AxOynG87LG@x6S8ar96#wNE{n`3y6DyclMlHcW8O z(Qv@RO~x-0q2_S`qaTIJbF|vU5YB5vHYN~=ByBhTJP~nTyolmk<8Kr0%ab%qwQ#rd z-{oF~*Z5ZdV1+lE|MiTU1!NKsNhYxU=|#)qC7~^#QstrIcMTB0=?0q_>_tPl$79=nBKAKSM-L(|V*!rtdwEnJPU+Z^x9Ac`{ zdEo=`;cNX}x&FmKxIRhi?+IA6{;nsBTI;=P>H&kK^(WwootRa|mS$W`B@Y=HcKss2 zrz@<#WIUevV4Uc#4?fZT9se7vXC;t9qfWsm#B?kC==qmmi1)-C85FbKqM}m{dZcOT zvi?Q)((3DCr6<-%HQuj&yL7WCFKLRpF4zA&y1H2D(QQXvXA($2-Fmjaho0v`-8;a6 zebO1O1{L~RCHCu}Utgy0I;!*Cl00)3QwGRLvn@e|_hHV)d+aSlu3I3aatRs}1P5u8d_a0zU71{gny)z8VzyL!K1E4bm z1u;x!hUA@YMnG~DP#A^@g5)HMiDbYC<^Tfbj2Ktfyyl!;S6y>nbK3o_;s3r>=XO{1 z?H+C``#oR#v^{j!iFNAKsZ-}xRlszhzF4m}wS;dXNibxfMWV{=-vd+!g`$8VO~#BQ z!H^3q#M@28P1&9F%#c?B&Yn3wNzcfT9`YX{alGo8F;LIkZf-vKNH(O0?h|XxexVVi z=$D~m(USdgd}6;y7&&yYpwj(9!t>~#sj9tj*ofUPD0Jbo$r#)CnJ;1y_c`=l3sWt$ zH1{BBn|m9M201-G;qvar6+>632t- z(F=$~1O#q}HyAmjBM`6)mnC)~BU&~nGeTZx3%+#5&Z;;cA4bS~$W&mV5uTXD2xaYm z2>m~Ge`0LypQ9rAl6WLaqP6S8Z z03#jY`~M4KYyT{jct}|LH?!9MnE)mNQPxQIKZn8viJVXupfVEmUv%6m4d$w~|K*iy zf5kV3vPL!tyB8;6SJwU`*P#uE!o{(*e{PQS;1VatSo<4pz>UvgPQFpz#jj)v0O(yLN#qI`6#-HF0q#=a4_8;ZvzcnVBc??rWC&%zj zkVwGkZe{I1%HRLIIOZ{0D-zYsa#F4RdmO3M#3W&lBeC|+NxJs$(GXbskM1hHe{qW5 zA3ezEeeDSUwcBz!H>3AQ*GcbhPSX402uGhHy}vmoRvRT&Nn9+`5w4~qgusc8u*Z=B zM>v}6Uz+gd6he&2G=)%(aF;_$TG~R4;qyQA=3A2pA&zj&2oYjy%n`Qmjid{VB<8{5 zeu3o#twm~$;Ow2&B@_OOr`8s!KHY&gO(FId2Igl+v{gqEfw9K-s5DrRd6#f2=iRky?I$iCk7qi*vo{)D-u~0 zM>zHYp}s9ix0HfYxTvuo2=#5swiZGqNy}VYB`b5Ct#OR=$Nx6vo$`SrR(v|jX@T8a ze|B=!s9g{rabH<~;zA9(k{Ndi2P#ZSlw%&ukJ}Ps{s>I2MVhd#FtzJKHDM;y3F`Cx zjs;G>Gr_5PBw&=2ZSW1p18=j__#QvmB22zPSU-zkr#LlXs?Xqi{Dl@c`NnWZzoFSE zC&%C${&r{bJt5B`Ouj)_J9^taYrg@{};EDEO-(eGDyB#8hkg~$$UQ%;2VT% z6yc6eOJiEmgz4T4eNSv*fm7ejTcMIr^WDneJ8S3iJ#nQ)nED1`ea34?+zuNbSeh`H zviRosr)P=lyG1UFU^L&k2H*H|XWGdm#(#0bY#?63q ze!A(%0r)WZK8*D*y6b;%V)43ixk zxQcNq{Vf4{lCB2nF}WvVr2Pq+m!+nz09$SsquIeTUX)|}HefEYc;Jt<6j z+O393JLf@4=b)44^3yQsX|DqdP5OZ(CVddv!osF;{X?xh5F3iE)ll3gaVYj|uS#yL zvG`v#7Kuv6;`Bp=@dr{cK7EeCcm}5B$< z9RQ;bVxEpSv9A>cXzW#ZP-!g=$ZkvIQJ(#biDKx8nhBFf4I5lDY}DZD$)pzMmIYwf z)JVyfh52u~Jxy&a7E3EBOCL*?#>Uc;irT(SUp?q!CuSB6LYBHQ7It0{(TVb^Kpjm~ zJF%X+djm=tb=hI(Pi~0h7#zUDrJfEVS$Al6Xw%QUP1aw6HteStl+A5L?fy*cPOGQcJrqaK2Fl7R%PMNio9-pt={+GDyJZ}xb0ar_ z1j+Q$q#Cj;QoInPOLVnU<2^!~Xyrs#BVpvj5k@+@VW0)a8&viuZnc_3Gu7^Jd_tTC z*8kb!)Zx%-xosPVR)-Y_88nC!yTn@YwGA{#$Crouzj0cWr)zXAi5d+Pc1VwdLhaUG0QOra4C|(F*EW`&pKo3Sh*tRp@ez+N(Qh$a4Vd`a;rCG(lP;ssp)Gn9WZNm%T&L;pv1UR0qxauJ-@_+kSe(QAU} zC56adb{3!=`gsBBpVa7iiOWuxQ|>Svu#F$HvpPEdVwa8g&;Sej!~hm30L-}Y5Ja`= zBrPY#S}qdQhu{~!*g|x+@L05>&R$B?nMkp4BN|X0uOyM88xTQ?g%1j;S15&=3vo0q zRgsgmOVSf!7y9eJS6Jx_2%a_@b1$YCi@L;cLkxv`G@IjPh_NUlV!VSdcwW4#Nlo$m%s#6h6KcZq;sL_*YbkhMJk7w;oY9L@{MhwVZJrme7M_z`9VecC z@zrQUPp^)BceXk$om0GVk|KyW-ZgfqqY%7UXfB@wjAO(M>+J*>7A=QMAuQ-cdLlsIG47bmtZyu}u0*+~{Diqk!M z62y62#5qaCA)F!3vimGJic=av8?{S`rpn-E;S;^H>@%w^uhM( zJ%NWK{PD196X=nqOyoz^T67sU_3*xu`-L!(f2Ah!P(zn@*b=)DHXZeVg`Hv3*K7lQ z=@*Bb&&d~7TZT=$c&9rxJ=qqVoUS;jFDq_DC|Y7zdR`(4=@EeUWhavlb6y5fIiR3EUl|8YiQD3 znPS6$9y)sb1YpjM+yH!}LRSJxE4cqnZ>lNL{q%^gg!_5eMFY%tE5-c~&Tu~~IRD4t z=zhwGJBwg^(QqWiwgU05(U#Jj;SHDL8wNPu@bJ>0w3*UKitTEl;mh)ZMTfHVj2;VF z?h{##6J!c{Qzb!@5Cma?4bjgqwqR^6k*%YyU>Wf@U5enQRA z>Z)-QBGpy8kFEnP(i6Fkz1E^hx%Q5p0J$C%xlRzd2xrRmt_4TA`bN+u7VxrF59S0A&6>TS@KfmhEIpT{#_`FbsH8&AUZ*7$EGf^!L9t z!Xo&lV#IJd3lDZT>8Pc!zPzHgtggw4j_IslMG?{gE8jDf;V?y-;mwzgH*~=KKi>hG zrqKbb{QQTOXe>J5CerkiSM!xbAv+*0k#xYS+boF~hwVYBEtfGtch)LD|F1b&1Wqo^ zm;7+0laD2;SC+id^^@WvnrL|>^JG7)B7BcLkiJN`W}BsJTGq|pJPD&9;kqR@>cQu^j)vE5^6rrj|tCjr{Wp;pBR?R zO91U$cXmy0!6H1L%=nLbekTdfQ_vQ2oy_$QC)RhI27JS<^2UQ?Iva#%t~m(GN*f!o z^HAU907F1EiV@YQ>d89by&dRCL;|XlFSZD6M+mZS6l2+Y+C?KQO4dmTLO3G?Ir&8k zjv+{Y92nxL%@H(Je&sNHQkY+@w#g6`9=+zxELlDaXF{a|C8r;Q}BG+R0B)f--u=ASuaBYCeNqA|0|x~Pr~yI zw1v7)y8><4neU6sX_c|U`&?4xgB|a4Wm2IE_H-d(RdnpcVdIeHl%@9lphH^Xc}~ak z?*=`3o<7lY;dx%A^}zF-E1rjNhUYnbxCKYgGa!OCyAg~pn$MwlL#ET0TWz=ENv~tQ z55@KLcf1e7%Y-u}P#ifp0<^i2zX%Cmlz&=eD9V1(Ga<_JBFdQ}3gHY<&hXcNTICdF zAaSeQH<~I@WJWlH`(JkV9LMlawH=7?FT5ytARVH_(o_$EkTD~iaf3yO!t5VCAHuvY z!kjO{5Y80l0}GDA42qzQsw6~HB}^SY(JyEE^&fFtJKp=usp&uxLmcn3@WOO(+Ci8k znVtK9isYn^&gA}ATz?enfap08>lG2}91)9fhFE9bW5H3ZgCc07W)w74V$HxO#bW+5 zPJ`xlg7=;*u`q<*3on))LL){;hu3gslq}pGphH?B(^-=&dX(wFC@K`ZXGEscL?*%+ zGM#mn1xJ~@2->JpU^F#lI?I-yGM$LC-t_TEkm>zDB-5}_F`4cJy%~6mOlSM`AKEl1 zQ(+X_2i~(H)9E4;;S8D1o*9s-DuOm@rw>h)OtLF@wkmkxhBGP&hiExHU>&pTnMIvaUPNwojnvOqUdh7lD zpZJJc<$Wpv0k&>G4KEV5Zhes^Oc|roaQA`=DM=@-f8OGZom3p%0Eu1}i8hEtgfk@C z(87YF;YA~8qjpZvRCSU_v;pyd!eCnMeJlkskm%!dC9-#J?*TPKq7DA~H{M4j(X%1Z ziz3n4A`#(CiQcfdq(p-wXrpQyeTk;x5A@MFe*J5_kJfk}#iDxeqwo?Tv4}p3Z`|Gu z3Zx?abIt;bEA~%m^ejm8f=F|gNJBV7nsctP;3!RX1Z|XGMpM;4zBDgeZ8?b6dLKwt zBGRNEq%BQs-}Ww0GNd_ophbz<s^gG|!1NXNWX}Go(3pnFU8_4vwIW+9~&?;mn7= zIoIxgN^_F;h2*a=y?zm18a=(na=s6M0EtM~oY&G~io%pf&x0_pi7@AhFoZLNInSQ| zDa?=v+NeYC(NuMfoPo2n=qIp0|-E-xzWTVGe(cSq&P8U2CW$oZbf z_%}AmshsblXyTH6zB=PA>x1)tvYN6i_InQn|Ef|0YoOKP`Cx{0s9=_qRTrB>v0)PW#dcgBuA!6yPP5{RZFm9)n|Bu-OE%D z)YcZ(me%&&VO?xhCP)*r^uPuF`cGqL(l^svD5arD`Y!zQJ!hg zY*Mv~4$@2gZU#rBB35_NAd4$zHTLnoh1EsH>b_Oo7*gm=JcgxFi_jNsvd}ZF@g3V( z+exDSi=MRFvbtke55BlmI5|Uc(ooq@h{w8iSQJ~uxdw$ zo7j=oSQ_37hacn}6c^+`WW4w$OGcX5YVMRME-+~HPDw~@YwU)JUDC?JP7~Xc9H<00 zm{6uDWBii7Xd55nEA25ZDS}H5MP-cOMtxa4>vMWE9frOQtzG|5O-@{-Geg=vis zlB5WHc9{PnJ0GaD1|FPk1_XSuf#2v0lUf0P-F>?Iu{02#FU=F4KS;szr9~zdrb(Pr z`agcMU=f}#<@$#_f0&x*OS%6^ooX9-3&9)LD0RhtGtGZ;CUHSn&Be<2w8QVoOp(vo5fvZlA{ z5R8l(5zqcU1O!GfvhhNT5QCAv(F@>M{!U8B!CWB02;q!iWTPMdf9#M!1|vfwXrmsp z!xzei5636Q^hW>rcion5@+lngq1I(pPF|*i^@hs+szp?q>EA7&K|10nHyvTgLr2-m z`wsGWlm|oiJLR52X^#0{qBo}JO@97Yi$X{FJ=<6rNpT{Z{QR$OTLe{{?oEl$Q&SWt zeH-eF(1%Uu@R|S30w;!}yv)!4YV35B|K}}pYWAU=aFol!TSz*Nl3W;$@-lz^*Idw1 zu7N&kx{0>8q@(S)j4~~4_Vd3QJ00a-)Pwbyq&S_;e*X7kHOf((sVdIFE*pr`+3e?k zKTa`#HwW{-bDw;AfM5})vzhTP2k<9J2C&c0*Jwkh^NFyN%u#L$=6}aOdvlS^^AINjpse?L=WdMosNARl*83k;Mc zou*8xyrHtNuC}72$xSEh=S0Waj{^S1Vstze(Tgxp|3TA%KuG>VAW6bK4D=-7 zi}I#LhN4tPFN7#>h$t6|D1X_C5q%iEm(W#VZcm!?sA{bvB z*oWbb&bgwU)s_+!d&nRu=}|MbA-qJaVjo*GZsMq#s_{Ub8+l5|NC#cP{a@SG`bTjR zf%l$>vsuI;oFUE?w_9)&X9RJpiX}8v;>cRyig&HH6sN>Pici@kk|FEEi-Xh-#8D-E zN}4A@$w-D=>92p{oiiYc^^f!KHwUCC^{}6z z3OpdqnSVeUNsT^cOLNt379|dzfzeAL&D$c)r6LXC3~8?N=fAi#qatXNpG2Cg?EYup zl(8UPnI)t-JzZ(yX}3obr1A6r@xB=py%^HGDbidl(h$y&=IT8yEzr_NN6<#;8%>p^ zNo;tv-T#!P+(T+lNsid?tnktxHiV_ca=ed&kP#bReVfG;yXK(iMhNpy5oV(ZLpW0y z#Q(-{qcCG4Xj5Qzjrg@|cD344n7$seuBt8k+S%cS(SGgF(c^|!kMlkC6X4A7Yu9l8 zw;h2O#k$UWMP%m^Y$ws(bD(cO!7u<5hw z$Nw53>pyC|!J;CtJ6N$>ibv@RYb%PH45t|Xw-7~02V8IazbXuzX5J$4(CC0gf4&1W zO``*@=lVB3e8xg;PkTqjS2RbZ(*bdbqyui~W?^TvpHHc+r}}2-&bncg)s{=jr(4P? zwADPotG$!Qhpeii6NZf*RWo*C^~CBlvc8$#R?s0W>B}4Lx9D+M+1vX*Uhw#RI+APb z#4M8E*u}!mRpk%lz)A=SCpX&hZxPy-dRTG!={LRNs;w-+s1aDHqiFl4(=65Xl(fseiMMPhG=ezvjyspv10 z=%*(Tktsj^{gV1o+c2#%T{Q`|!bb~z(oGhAB7hI%*T2Mpax+}DX$7F6$$@V1n!bhmztr@XiJBI3z2yhP?wtaH|8!k`5kA;J`fEG;pIAOmfn|F? z6ANPk=l+|XxLvSFr*5wS7Ix~_sYlOt%Fn_4wHiGSCQSry$Nmp#7-hg2zyC;nO`bXb zljpBf@%)v6CG%=PJHwW5c|ouU&$n{@PoBR`!t-RbMNi+#{67ckw@w4T+3MsuJ$gt= zW}wqUMaM-YGjSYXMMaY#1QwN}5bMXPfS(37(qNQJI>>(;bQtSbMKP^-@4IM#bj%is zj0tB%#<#w1!7(x(7eSkz1mlZNcNX5Daa*R9kc(Myot<;>^uxKKyi& z>0K@~Wy-vb=U>Ei$jN(O?*}>i9f8;nYB~v}MW%#_-I%;@^Xor!4{`GTk!`FEL*eB1 zEQ=HFo$bbDwO=lKrsAZ&p{x@9+4S)WoV>Gz6Pdhkud}do@_wC#S}Kgdmy2eCJ&_9RE`R?w&g7v5F0%&&dpsrB z^Z>QV`7ZN81bZwc*!cmlDSNrhSQ6~f0GRaTUC$UMk)Z;tj(co66;Ny->b~pS0Pd-K zx=iX2_agz^0``|)0kFfmy3BAB>~AT-IRDu;ad;P(*<^w}oD%GU0NAw7E;sTB_E1VN z&VM$w>3g`52Q}D(sle{`pMQ^cE;bif!$fKiqy+09pf+PSmlfXxyFVq^;s99ft}ZKg z33guqOyZHduLBqz!gq-dL3-%!rvi9qc5+$7NWAw3@QTsg;~GZOYB*Xudu*)z2_J^h z+_QH8cik>7OFoJFuL0b`(mm4xVD%kcRv{AXo|Itc1;83QxGeW0*xf0?xc_DMeq(!A zmBa(=FR8%(O8;kr9nsEZIUcFql@hEn05&VnWnnMD?o0``FaS2Yt;-r`g58l4?3w`B zk!^SpvI4t3B^dp`E!!NB;UPc#NCkH9E&;H)xo(7q=>Y83lwczQVDnnJJOPPd+f#zA z34qOS=|*nRV7H_M^XtFkqjo`#%j0TEZCgsPp#NK#?eb(;g58`7?7pD?Thzjh+@!&7 zN(mP9e~X*DJiC?DZcGUl^nXjTTy<^-z-~wh7W98ho4M-Yet=z{3he%%|67*n@{|ZN zd0k4dp#M86!{u2F1iLmRSkV71cU_)&K(K29V7>4IqKo_K|7=q}8drgH4bvtI9~K;8 zWOaG}doeh!F4yB-8V}3N)#U@NVl5Y`mbux-;FGUTc#9u?pa`w#hrdts!xD2nFb)kE zD*c!wRO${y5axlSgw&5JRAO-cQwvvx@HzO4x zL9qT+-5&?}1^I}L8f2pd(gR;Y$qzFBMT?1p{EBSErTDdF7CvGyY%kXGP+Wdm%M85t zMd#o{z1WJj0PGL>{{Kgni(nW2rW={o@({;At>woQ*7DFwG@!NooWxr87E)J=AU`W> z;o_ouuB(<={K(dVsJNOCKGZ(~P(50?%v{6#t6HRhdN?lt)w89`JS3r-r-JhLf8reO zo#V15hEQ3lppFaB+9%tsYNnx@rGmOO0M)C7Ta~GyGE+f)8i4BE+*JjZ;4mWv)Zg|D zK<%65qQ8_}ZYrn=0jNIBTrBw%6pq)F7yfPZ*WZ}`qq)X)xF5#R?+VKMJr&f00jT{k zlxT{%_(%oyQvm7!jHX{T)USz963#v1=fC3|9*6<;3!qwO;ll(Yh>WR6{Pq9OJaE&= z$Gczr!41;`KXPV_EqY)YdLcgfGCc4j-2dg6`Z>jzdgK)};F$U)$*OlYkYV^f@^7?p zR2vqle}l{#ei(!IQoQlXyD%r_##n>w;#p+=P>+JCGvKi zz+b!l^X4*XAals0oc}4ouSq1>6Kx^EqkjDJtMjdv0GVu}(|Z>DWJ|xbZ~uO4h}&EX zxr|NmVoT|=$2x&NZShxj8MOl?aI03o*h;3u$2_!TXZ)Jj8PG1-e5_Vb{h}`G`G$zd zW;Wh?H*4;=`6$Ctdk^>mFGZ|5&KDVl_XKAm{=R0Rzk64xB);Q|YvlSL_w!%BDGQwekUkD}x$()DA@1V`inzZeihC*=Lfpsw{ZAz> zDQ7s|75I(v&TWQ2v+^z_>9Vo4)wlsdV%I+(9t4n#L1;{QrwHPiSl+4eIgWd~lHo@%jHt|IEoV zB*t+Hp`P&TAGETWLG9!H=+u?#cSd}nu*K=0}_j|>}v$71nS2fWoQyMQnJNk$IC8_Ya) zh$upL22Us3;7xRDirE<`oST^WM|sIrGLcMgA$5mEin{`l=vM3sd_Y>cPIPNV0fo?j z1QLZkh5ZlZ*EEk**m03(AuHi(oyfE4=*v73POA*&wkuSIs*~u@Dh|Nd%cZ`2g-yrT zP}tLc{y(m;bG%7TO$nJnjnSlZ6qd+LF2zzcMxwK){rtD4Mx8~YmqTYCiq02@KGQ9rv*RPrNpVku6L~Hjo!RFCYOO)c9UB^%kx}^!=YLE9`{E26e5)pEeCpq2 zf9e_8|3MpWl+qZ_Epoll_6lt&t@M}u;nJxD9ZpmScKi!%mc^7sf( z$Lv34xz`908C}oH`o|!2{(&w|XT;oykMM--9#EvN1rm~(i8o|=j{AR@{0&kU9N>D~ zD;kQA@T8hP+>+EL2}P-YGbmoTzl-BRaajvK!c&a7FeTU3LP2cexi128U9_L;aV~*P zj0sOgY$A3}#SOqf8bas!@c}v)_tBg5pfgOd@bIHDX zw;i^3aCkZa+baVh5)s>bp+|twrM-160BmnactS~RuUyE8?Y(ezfYiobx>fUUhhiw(XSyp4Kfk@!}ByDwpl;dWWC|Y~qmcR1$5k zQfP=xy!d>8&gS+mjxAUIXsn_W>J)0&ggSve_UQX9KtUox=%tDPp)KuP9QUpW{n?}5 zNn{${@J26f4G`Lz=XwlBqWB0eQOaZTN$7A*?BzBAVwbmdaT+KlWPF4t#-iV3w+xVu zXz=9~0a{nIaXk+$Fnm2ttE{-Pw4uyAjJ%}2q3`~MeAU-g)|b>*pbEn7swymK&8~X+ zg8-o`Tf1IV6N;u!s46l-9llZZgAkIgdPU-Y!;N2sB*b8}LRSqAPbgVeRs9GwMpwNe z`@aUUt5KO0K`V4sBs{SsT~+lxu&}FSk^5>-Xb_8B&}yVn>KA{Ufj5lwRX_g~59O}$ zCQBv%V8@%BnMUkud&zt?l_B0zv?Q2&^-kTT$Z#BiUl_~{j$VOa?js52uJD670uty{ zyvgX-79a!Q6C%$$SRdo}xlZKybb`60K9>A~RG65KJdu6y_1Y;GUyRqT_3%)ldK6-~ z<4s9NfhqaQrKpOe=<&7REV9&Nb@WQ;@ng~BmA)PcNT3rnzpwlCPr3uB#}gwjKor8& zIguCA(W5cXYwcy_W3kujD& zbG8=qE#$weVu^P?hDiO9_)@<$FxRFJ_$oP~!ca#-q;?93;96-sr!#J~tBYF6SL* z#9cp3l?F1}-fj!-INCCjj5bN7zb*a`Z7|yO1HQ~?<8_q&Fca@@*J^#nT0`8_iTAht z^-l(qg9R#kDZ;=nIC*{XAobpeR z$w348)6I7dbYIi|(DLJN@<<{R)I*{8hin_*mk04L$01+R7ymrU=!@n*^VfIA*DxcG z25tjBA;3Q`)tZw5N6%7H)e^r$0(Y2G)$)JoSh~9T@W2UomzLq~(tOa*cQ(0e+)A+7 z0s`mEo?qW^^n_y;HsBvwkqP6ftMNy(nRN>b-2>5vO39hiaLkB?`A00B75l3B>?Ku8 z>f%lIn7w4;{Mn0^H5@v7>8!!a7S5erSG%-f)UvsAmnaw_%J$pDu(VmQxV~X=Lw$6? z+-39T>z3e;Ji-fi zI!ujj3H09e>z{ltm*Zx6i{pXbV)(ffdQZ!%kt|92Vtr!S6200j9v(|8ge}o|FG|;| z`C^IRcy9>s({E%Z@f!u80egOLj^+VvI7{=tUjfp5lPhvM~ zjKUHwC_wXS8k2reBI(ult^E)9g-qw1*o|}O)$zJl!wJP_FAoviVyqu<|A*V4%2-`K zO}dANOI?)D!>Z)Conow-tYNGl`1OyPEFUO+oQM4P5o4W~p0UQo5@Y?~2Z%*uZI;AX zWs>_)p8r7`7%KwF4Cm>AZa(xAvs7ch_(NI$qBW*16?XOlL=sE=kmH}0+APsh7XdX! z$%h*(os;l01Mw3LbyBl|PC8v0+5=)jj2+eUQG1JlD!APzydx#fE8*=aX^a$XB9r;i zc(=nF@_oek2MT~qHS!Dn;*T;nesqp8H#(d6d%@`yk4OXQ!jJ9(7ItCtBwZ+x)<>vM*Yh$pj9Ofd;Ei>p6*`9yemZ%2|hkXBxsRD z0x`mmH=zw3U<(y#K?253>XjtY`uORD)(|&!r1kNCVyziz5jQ95BqsYwceJF*wn*$T zb}hQ}lfi#hWMcp5HM)cuC?IgX zE5RH)Z<*s=nWfCHsIsA?uD-CzGcxlLOSj=23-JrERcXM4^n5R~pJZ-2-eB<4;+V7y z)E2=zT~+{}h`*8ME7)1LZqE^FTVmAa3AN3n#(o-;4}LPC9R7df&bjcLWMy{wtaFSs zdtw!;3-tf*|8RWrbAXmUV_cCr)N5mmvYkRX(nxO zVJg%0)C%k+MFx(@W83%SWE$S!=JOnjo2{_kk75^9AsezBa~=F?IB65O$)#9kHso`} z|7u`sBj1i}$k6CD$cB7I_OO9}O-2F5(15-1-D);L``9~L9)(2e_vm!=7UU?G(kWWvH^0~p4bbyv zE5E6HB3fv_x;H+dr7vC;EoH0SZ>o0~o|ueo!OyMmVLUTQ2!Z-ns{qs?IEn}-5-BM2 zFd`Gmmp@KxIF9_m$v+CpJoLweS{|S^0;iqe&=Un^o_AtG-4uWtiBn&q8#R=9?u!ZK z?|;NO9ED?iaGH;zWgg^XKz+G;fYxZ7Glk=(6qI?=lnFID05t}O?BJ{&1!W$yV?z1+ z-*FDd;=nPSJEow_BgagrzXoWH!`WUq;!8o9r+b-D-v*$@<5b1y1scjcSJ8m_YM%hq z1l+q2JzqnaH!qk_(*jTvah!DYJPlj153ILjPInky*t zG;Q#jf->)f zGNC>S(5l7F=ePh}L78`-8&Kc+^M9PyOk6M?Jw-#ASB#rbV*|A6aB+O}WDR9r9dAON z5`e15?ccZrTybdL|7}9a`q$85gZG#lU8|wiil5(xf0|I=2WT~VkGjz{8fuL|-E2dB z*E;}pg!hOWiht$*|>~2x>7@}6e!l+ z7;=5*$N%yEIuaL_;@(olp?PJg0rkBbpfv}V>P8tFK`!%JT@y<7{|ve2;^tahV5^|a zyK7CTc>!AUa6xSJI1Oc95ox^-$v7%*OXlz2g5rA6Y-RnkIXs8vU!>jO5 z1L}vi0jP!EU)|_28tNENszk8|4}hwB`vE10)mb*#Ok>-py`wu7;W`Pz;a^sGluEygjx`QTIt>3Mwzk!t=R&_0Lg^9 zDgd?0yWWk?(onMmiUE=d^=1HSwRfEx{Pz;bv zs38HUwca&uv_V5P2owV(6YBT?)JfjeZnR!Q)e95@Bopek0MyCeRc^FSL)8fs10)mb zvjEg7-j!~2riPj+Pz;a^sQ>I0fI1Zql0<7YRINZUKr*2w2B6k?m%Gs!8fu0>F+eh* zP7gqx=52MOH5#f$pco*TP!9#5PWQIB(dim$xi_l+K%MDb=0;g61*pRXiUE=dRTqFd%iH8e57SVG3Dn66P!|WF&h|FC(Wx40sz5P7 zGHE>@fLiZe>PDw%s3`))0Lg&*Z&ms{nVCuyij0>uEygt{gGb)I*j8=a`3CJNL_fog|OQvW2$3nmKAk6w#B%w;YbAP;kGMgj98ItxJ{22%k1#gBh<0*85+HIY}4St1-t zd0t5;4MVS|WWBPhMRRy$(v-EK0l1R~iASG#U=L{8j3ogi9)IZRAxsH~t7G@@hItL)jvd zSFx+ZwBnjIp}36SeZ`TTT4Bjw|7uy7z50|_6{(}hs~vAeI*LoeQZD6F#Khu$ll@=A z;x3F{59MtUnyFa8lbeQl{|)WuT#}3TXk``9mUYLkPYH^bg}w~C;lHadL##64RHn&Ghi#ewtF+g}BaL&R*b6mN8F z9zN8I`#w9b&?v)2{67qc)%KSzG#d2Yky-d=7rZ;--3+oG4epr6%}KfamErpP|2b;= z3lcQ3{e{a*q+|Wo5I5!aSBA^ypSg_6QH4gtO*LeY23~vMttXL;3_kzGNhBw65@FN> zQ8N7ekLt*-1w_;wKT+K~v(QM%51iyN1K4AQMzVRAq|hiM;~k?faykLhIpp0#55x!h zLJEyCGUfae@kw+oWmUPlD4K;vx&KqwQjomhr6~XZKZQo@%ccvB3QNk1D(VZHKG@eR zG|Fh^=l}HZr=NSt`=^|Kh~fWFJ^aHU)_x8hrbaiHH?YcPe*Qz_=RwrXy(MyFFlslJ zz|SQsPl4Xk@@gbYl5{K}n>9wpRotuf^=^}=U?J9R>3TI^tVad^Ef#(j^0eoJa%y|{ zI>#H{Lh;|Xu)HtITbn$4GYvnVLQyEpIXDe<*>yY=NR-z6BvUAelHvU5t(Ak7p|rJH zY2k!o9S;;@+CuY>Ec~qF>FgaZmEN$0<9CWJXtIVav^d-%TL7`PcsO!j-HZ*fW~XNh zaj^zyTWE1_4F6nwLtE(Js2Zm3s-ml>dU15XKGZjIzi+E%{%FYHZ&PRB1K>g^AX%|rZc9KD)vcI6O@rXHP#v#B06oSJ0R2__}iFave%*|XPB0W zGtAXO_$ooA29+W_mcE?P+3C$XIa%r1d^n#1FA%P93}!t})m4lGJLGsI-cVQ0VHVfz z;6z6sBzHL%I|rp>)0s{t}a;1Eey9(C1M=;Jphx$99o`>mNbV}iRQqJ zH&NYg%C1O{U}!~in9*z(K754E3+F8|oEKELU&>$Pq4XqBcSg8ejg9gYoA;r#s z^!m6tg)sWKl^_2pvv(H(dEXMLEGTwLH1cqa<9rVvxrBAC{v%p$o~UIugqCynFhVC? zhxXJd%ih6{Y#^4BOZjOjt&>=a1md~O|1c15t?JMU8BJ*SycC~!!j*FyqXe1V>Lu1w-$lB~~BEzuG z%*gPrf+J_=o#=R1X{TFWTwISjw3$sUaG8|YjWuW^swY$rW|7iPs7NE5vxL=0Vyrgc zn?>O9*;a@o@wYS5ZKV$IPDMU!um_c7ssrjsa~a-HQQHAAy7TZ2={`rg^l4M`6~t8n zx09J517m~=@BJ*-xgf~+S-XjF{X`+r#|pR3^kw7~#DrvM!=%+ok=Nu} z3S|0T%TfL_9DegrWhw}^9;Qr0W0W!18+`aIl=k67rjQvotl83q@kr{Y{Q2UkvLPwUm^ngR$J;M{i zQh@_}MAi5SHAAbb#!ZM+S4~j#IPs9y1iVRaMS$L%61OXU?C*9BPtQJfp^e}dOK30W z-@;nBFSpfc>&z{7yWr1(ZkO;h#1di$esll=QV=Q~{QBQGm3c*OXGNv6rczj3R9{?d zddcMWCMBFoU%o~ARIU%`Ykr~2=xU(5N7_{UXZRzXM2@NWM6Yzr3s6~bpu0PMta5h` zPbDE0tmN4hNJvEtyJN8b7arj5h6bLyTX;%#$)qC0*_}jukv<5Bv}k{~0u8F%3Pq@_ zvAnpjtg*?%*zi0y@29lyal2foOvERYx68-?rN#TXeevS}w{Lh#3D@6k1w09e9qe*z zfXI?QZYdfJbW6h%flc#{y_g+bCKQB9C;$16xJ@tJ*DY66%G0HSJ0sX{6(IB^9cq*K{tm{0l1O}22JKJ@5fXwoJ z+%hz%a?8S#(RLQw*ija}S*VQ1C+uw3I|7uB-rFt4kNw=@@RY2}V>bet=w)>77VQ6H zdb%ZO;JGE?NnmtBXM?L?XI10Xp&^k?LPW;QZhrh5H?$Q6ZV?&{aEmmd`oglpvWBMb z9Z9%OoYA2%W_J5Hps-`Rs+*z)y8ShgvdXfG+VZ9on6gnBI}_(dP~muyF|+%K0F~pq zxD3t*x&zau5*wK;#3KbUv)zN^|M<@CAT+3Q2Zg7S%tLPzDq;hBbP33F!XD~2F3%kh zo>CGIeG>?ghz!s@{QO_s22R}FJqQg3xd&-NrHzf~o~ExO#{eD0*dunC(CtD(`e%=S z1>{-TQ9VgJ&^=I-DJm|jE2}f->EL!^8HE@u~47}lNy2$x|Mhvy4oqGU&9O51jo>1zzaE;KC zxUkDP0m0VhxqZ;6zuPA~Jv%O332@>UOX%XCe-SshliI3G=0LYFJdMP-@FbxiRJtA> zkmTexZf`Ui==RoB3L7g*i)S`AwR*;}B-ym;iQ}ussmWlvP!W6U`fz~CDXrbT(P)sn zcX%oRdpismNJ8vw&)xwtr{=o*pn>P^6P`>Gdplf+%)lq?ZO>r;uWRM@!jD7TUf~I) zwzp|OMOtESdu0R!JFTVL6Ak*iJ;T$p?TxUMsCIfWlDFu3l)iW3jF%dxbD_Bb9k8C37tcNNM_dmrlYw;esU?9X>&bZx1^`dT@t+sYP&+zc9X9* z0#a>j@rK%Z`uUHz&7B%~gICkw_gPNljdawO=zFy4nl+)ic#$6YnJGM7X$eE4JC8SZ zE7cv08@tm{T{50>DWRf9O+X8Kcd{r_eV0aWhWf4)_1)~NkAMWC8VNYs0%R#oc@<_L zbuJ#N^jc>t&l0X^jEUM%+Y8wsj#t;hG~AY1rB`3yw-D7Ub%xIkhW%!hUdBFt{i`+; z-k7<&cUJs}jKB32bF0}N>D!n(n52a!OReB$$)8)`(IEDL+ zkO|pd=#_6VG8W7|%g4g~8iwUs&~W|a<0o#Jt9q>je$G{`6X&X3v?h8X|2+(Us%kpD z4cTVX6TjmdHs~mgSpUmifp$qY@n%aKoc~qDZOC?N8`1+x{6JQg;*A&c;s1+m$ljR$ zViGhzmQ?=p%H=HMHzeqNsz}g2i3EG1EsUb~jcCJd$o6U*(!;mdSX8g0^oMcqSVr%E zB(#ROskb3fBJShnHlzo9DYhZ^Ek;ZBSo_2t>kA;Txv#(f*-mXkdPGDvIZC_e<1luy zj>GWq1IzjFA__jZK{S5+r_eaufXwu+cAQLh+hWJNI!g_)+QyPvRM`+(u zav{YN5>T@-L4k)G7%cG+&LleRQz80l=QQ9K?)^RA&Z(hQV?op|Gl3$*)i9(Wc&55K zi)3cu4Mp_1B1UQ+z9Fd>NGb#W7@4mit`@jIPnR@h=@5CU5B+c*K)KL)3ipZlgk1X_YvE^vvM%xuT<1po*jfBXP8uDIh#;2xp-x~?DRvg| zQ~Ur7vLXS>{{H@#wzsV~hhwsLz7z~1KsjH#+;HM1vXe`}%m8Koqb=b%!)}aj!wh>B zrAJV_Eu(;PXu#h1PG;Esw_AXmVf$8L+bfH3m|@#wr(xHy%5d^#=`E_@?24MqzMGwn%*#r3o0}0f$sx9{dK!Ue=RdC z5?=B7$vJ`jytaGxxA-vTr@~&q!Wp(>lKF||i2+^c=f69~9Dha&3`F)=8ywM+uciR} z1KwUojlyCf^S@-VV=5M}Hd*8jg7e8KEfRu&!l#ABPDxmt4G^;UHQF!;=%j)GcsjE3 z|3MH?G|P}+S59%xS1*2l5k3qh7o8~*?3zRZ2?C1z@xQAI0?2w3K|m4e-&M%qw}!Z> zgMi}JvDOR%h&x3PP|Wz3J+^CNkBtNn6kR-7P<2waFouZyAfQ+W0e!3>fW&eU1o)SJ zO~eIY))inxe5N4x_rDZ!hoM0;?;6l$XD)KQYjkK(*HBbhS6{S)GqQ4>mP3b4s;(I^ zIy!v(AB_yim?V=*{QQ5XSY+^G8>fc(F&ymQCKFN`1k(x(HVYL_$Ix-fy)hln!#4~J zULqw11|#y(A`?pC|2I2-nk-35uwBahA6>z2%1ZO_i+=IryBXE8hKGk9d-?yKH!@CnY#+66c-DtEnqY25!(xI^6>MBJAGigV z`^fRSoi^nkl1dO&(I7*gMp34uQVj|2Z`S>7+2P|DT@M!#ODm2M4ES zDrSa={~ilJ!{^f?Z#s_33L#VVrVgLO;m1LNLq{kVSW`WDf?6Vlj-kmJoP zKMRohJ3aCi(g1{m{@(h7`omRuG5w*7cYL*(p})$rES{*pN4z7Pn$F}2`a9we>Mzby zE=4r;*Uz;`Qh%34w?lu|i2kKagQNlkTaiQ<_!97ckMP}gm~ zx(G-h$q)?KWC2oLXGPv|^yR_G5WJI)x};O%TNqiXG)Asg7VsskuPy&X!sUX*sbuQUQuX={Ybmn<%+w6i1c zB6=encFo>RM`_rif{n$B(X^>GZ$feL4GgkVt63Vks}#bMWMPCAN9!cs2f zQ^dsL2EAqxr1CD0-T~#^Aj-SLR~`Wgvw(U!JY>TW~ieO%p5IP5sSpN{TyFLO;+Ghxb@YOT8L9jlJK z)Z$Hk-n96k>Ym~;>U)wOSlzsIRF{OGTncAeU6r5zh!5E-qjy1bH<2PVcbBg@0#b9Q z;0;cznEzm$8rR%8kq;baSHjJ4A|IrqxeiV{TAfvYuhj=EN#|4$iO;EIsS$6m74hr; zG+R{Q)82fEMhpm5erMmPaWP628TD9JW++u zcnjhI@q%^Ke6l4chvN;+4Yub$YVKLyL)3%Zc({is9nIO?>gib1T<7uJ;(y2%Fd5`>^Lfhg33NlUuFIcVsIGgs;v4M ziyNx!IS-edbP<&;N?&C@H@TF`RN2A){#V@BT^s!?RJL7I_E%qJ1fv9!6#g8aj3HAjIk|r^(;FhQ z50?K!G88MNArNx}b)b&S{fi-YTll$u@e4QR_n{uKe|YFxSd4K0 z;v^3ji|3OO2zt2|G#n$QptAuFM=uAb>_O4~ z?&=&L;-(rhNCU4u@Ro$tFxEdXVYPc=!iu{QQ1l^||L*tyD&`2 z51g5JX8=1h{#=vdrM4W%d*-|Rrz!=YvdP_bMsH-XQqbv?HE0b!(7@#<8$Bo5&|Zno zrLqe4!M!w`#8j2{KXop(`ACrmt_&+F%vUG*{8g?D%Yn$r4%4A3!rTFt{JJrHn^=*0 znt=w8c)J*{Bh$Z&h^KU01oQnSCo7E9#4yeoRJphonJA)LLD_N~t}( zvmKshsocZaKo0j425~;RxC6UWjs7F>i7p>u=l^)lPYZ9U9KnFIU6x|fOVKN7c|DRP zNfB$LJ^#_`o$TQT-U1Qp3N2Q6ujh;Ps1|GFMV8ulwo6A3X>+AEobxlHg;HB_X>A$y z)tk&-tA^niNuel|zvziGObLtHLaSz9=@$hOrH%6UKh*HVzHo-~lXp_QnCv92v~WW4 zoFDEZ5VzREsAo;3sr$ORcqha!mpfsn*n%c&*uv<+7Fpic^#s>R{lymMr)LXsu@*u! znD}VF{!^=sS7JRo2o8@v9P{7ou`Y={c7|qi+$3SMGlyo^ zMMUI>5{v$HC=tJVRLKvL-mExypInC61Dh* zp2zQM3g9r1=vUPa;U;jp)lOBr|7jqxC?7#BH{{fYNX@WOgR3X!t0_30dq?55|2OxJ z(k_xkFfMq6s^DWP%lr04w6}g~+dDG8{2G!;%YlX7de0=jPQ3M`OVNhjdQau8SJLd_-Wq?Z&STNZPChs4 z(2j-#x?ua?k4$X^LWZ|Kv{EG4D~SZ+tq+}sHuTneDQ~?J7~p@^D@nojq4fV|YlxfL zTOaDL|MybfdL^)?@YaWZ0KDw6y%Kv&3brRV6T*8cZ@rR;$R?8}Nei|$u?hHD6>Oi4 zPgus};U*eqhLf3*CgkPTI%II@c>nBUS=Z6G{(QKP$C^M-6{U~-_zmvC#_dVi{pP3$A2ai!~bt_ zb^>7%mC--t`IwL#{i{%PQsMpAsVNmV`d>Zz!}Wn?Q{z4!*wDvQ547+z;dw#i6UR{n z3bjt;lbkgAIQ|qPRRa4>kzB`5@#H!gksh7u*S{&dby%@I#k&9na1_>aybCh_Ff+Ln z%FK5j*2@x{DdOv*_afhUD~Vxm=iZD0N}&OJ<2%WB9_FuqHIs}a`eQU=nTj^Zcec+~ zOoo|h(vXHj7q6}>(||Gaoed?GrO0*=W|*#Se$;G zu-H8bixdYePUrqNgMjWT2v`s9@B?j*OE$dn_`m-_5KyzbAwk|&QWt5vpXLK-eq9_1o? zGc=2}c>Y0kQ1@71@M=q^hBJ~=&FBABRvKOYPU2K z3gX}~a?se;dEa1ElO5Csev^*g64=S5T#Aq=t-*i(Maxd5-57lUO1ndp z_JFT60upEq-q3c#R~DC4+9i=M9A_`Wp*HY~bd-iA8kCk$8`vzB;y@v3ZD8XnizO=V zH1Atc9ICXwO-FG_Sjwe*PA%S|yhh~z#NE>aQh7H;AB6Jm6y-hWE02H#Iu36r?}$DY zAeDD%INhoG^$L}L&68Y3VzCX>M|KmQRQqZ=b% zI!*!MFd2NAj>fQB?CXnw#_*hdrUgg~+Z4gmLEQ+4;rUfM`qE<|c3!FWYb}}> zp7Hr$4an8v4WmCY-(rb+I};Dk)ToQzFg$-qM{h}3%B6hf@H}#ZMUcw7CHglg?`~1v z-+bi}kjmo%h@3uU0aAIFMZU(R6NE!8*4ODMFE%8`A+}`jk)0y=Ao5t@A32c$I>&$h zF>Y^Xc|XPn=#S~>ErFX{ilqoqe26~hQA>2{Y8Z9`qmM#ue-*Vo>Z^@_1d^Q2JpcI*%_eg?1FAZpI+Sq8>2%0W zV?lykM7R(%8_4nMx?Uj<2V>@RDvB!_>*|Y}d>j_|jhxQ>u9kv|oJ_YbiyTxnJQfR5 zF694At4J1Op`ZWksj?W5ps6gzLO=iQw}!Z>vlt84iuQY| ztqkI(8Zt-&FQbNS;mv5tEJn}7EQV}6FZA=@YGiiZN<_?JP~AIo<5|fMoaB+b#w*y? z3$=^~@rSkXyl9Tm7kP|JoQGbj~AU)U|9) zS}Rprn4k9MGnJB1MaHiZ%hJLWyamhCQf)rFYLu1~&MbwWYx*f?t#FP&D1C`v|E-l? zgzcJ_y?3P69K-(|J^aHU)_#ryhq$@GApQn1aE+feYT4d0*^@<$+A>UQNlLG%_q4nk z$&#cm)>{@?dbM>pgr}w#M+3yVI$f{ki}k1$Yv}+BKWo%>@kTnBCGb0H)J7uO1pj^O z>nls^i<_*SRt>}X`4o!USv6|Q+L=Nj6`tYz?49h?l!?+#)=CQ}6l>I=5YrZxEwb?Q zF1oJXiBc{NTR3s2*n%cYTVS8T1ebCBLm$7l^6`22R=@Z=Swx$oa)q_Md1!;nI<_ah zp)N|j)S(S?)bsL}bsBj|NquQy-42z~U-0_4 z{}EvPIi8YnsK$M$!rg!uHu{QyQaq$Dbr)k_XOX?~b z%FLX8Qrsm8xX%gTe!;79J?2D!dz{8yQQKHoXvMNgaqpdg+s}W+`F;`mb{;dvz&%yr zuCFLAs;Fo*&zet)yHIcs$0tV5F^2^Be(50Bqb~;PF&cGoLt$lcUtc9fGyCq}x3U3d zqpyRm$9>QhwC`>JB3d8yRRPp54|2W3H0r}N>Y~QN3LNjTqn<1hv|=SIum_OPldlYP z@#MU+?pcC5)KMx~IFnRcwy?e8@Br>t2e{r?&GuM@yRM9i3Z~g|l-lWgq)wfbBPcbc$wsipE`9 z+E`K1utUo}P}mk_ALr-);%xr|TCUZoYZdC+iiVkpx%__QMdwL1QjviA!~oZC?&o^r zHR|ySb!A0ET}5H>4lR2hK`qKYF6jT>+SkSN`bybHYSiV0rFFGsP1Yh|Y-i!%IZN5c z2mRmMy{`t~W!Yo}p2P zxp53!vjmVFi?Ua?4RHNYGuNAq7H}rB73xs8WpU6{3*L74#00q#>tD5a-#c;fF6)(* zX9#h%Oj7aZ$9&{>KMB1TW~8x9AOnoFh2(&*vzfgY(ZI`iZd%dBd(&!mI{` z_GnFi)m%%yDy)Z};#m$o$v_Fta?tXHTUy{b3Qk-^E&5&cwCJ~cqJEj{K?kgY{2Tzr zQOez)JzzI{!v?ZYSk3hh7YeRTJw>K#$gdo3-5aUSS0q-z3a2oz-nlm*%xbpS|fx^QiOxMTvcUmVkOR>I9Hi$ zGU5cQYyJFJFNeV#D`#JCi=*!GH|}z6tL}15MLuk@1avaJ%Yl$Im*EY%CuPS}G!Ng9 z?%zq5y*4#pL0m6zJDCYGz>t_s`1^mVyH#;cKkrv;NMJy!jLomwLxk%nN{LQZ$WIUg z{`yA~;C|1R$hVH8JP`KCzRgL)5U?vX3L8G`4Jrs$o??#-KPBBGJNdYn&KUx8Z*e`Z zu2m<9VH5l3%sWqkv*)z~1b(-ZIvIf`)-5$=F|~c|{5E;SeW|T7e@BS5t(0JqCw`LURwZ?|m+DcWc(o$>~|pQ@@;U zIazV~S^6bdxa0$-)BXIP5+H{`TSa#{f2)tvJu?oD`kh^tF0P%wWOl>+r8RSFmnz5@6#A?pCAcmv?-QEtAQ(28-Ji%(tcr^x2 zt08#PR?}beerD?!oNqPvE34_QtVWEcfd3F%F+>0gqNx^Ce*K41d<%B)EGS-?{k{%v z&lCp_gYD#WT+*<3cJ18RD{7a{UNHZF#s!P#)h_j|nh2#g&%7zdAJy`ni8R=oREl9W zXMS!1x7dP?nTsZF6013D7qe+|>MzFp4(XgV&I0BhX@Cw>zA&(}{QZBWq~?oEI=k6Q zVtj#}CG8*xE-E?eL6cf`fr(`) zM5n!2OL{TamyIAk*hUMC-OwbMxQDa3{^8n2vr6ChKw?Kr2b}HaKUCA4T2PpOkW*7h zo7m04)vLahc7qsv1v_AUl}Rn5{uU2pcOjZ2 z8$uhb@t;!wO0j7pwSyO9+i<=`XH_`hkzf?Yw&5iUc!L0|nw?@!SbI8M>K4pjI(t5* zsex5XX4f8Aw_w@)rHhXlIPCv1_a0zU71{gny%QN2Bnpaz9)ekzfguUV?QTb5MUsPA zhhb(2iez+MSH;!c)!**wnq4yz6tkE!m;*?VAc~4vG3Tt``&OO0UDaXeR{THT^||)Y zT_?P!PF0;cRk!Z6`YE+;g|T-h{a^_);gN8sGDsom?zjQ{f|VgT`+XnUt>8f%VYDz! z#w^m2h8;HOE-Vb`OuxU(W0`IRWmdGneNs_}y2W>fTJH*Q%XFnFPZ6bIsHu+Dj66K8V z7s~0EkTa0RXd`lZ;1dSDctj`%?pI9WT2uE;QE_D|QY8@yn7t#>86gpUF9PL8A~41v zP|(I}rb7-DuJWJS?dHPOYe1x&a{oRlWU}K;HJtXH>#TRzZEJKIc2ZvW&Iub`m}5<|AZR!_Ncqhzc>SoMF2_#e`W0-$;*G8v>Z0Ods@PF?3x7_BM+Rc&Ja?}4Y{~yME#jKMki`Z@| z)8qef{vQ@CIv-_@>?0YFE?ys7d+eePqXjcqk&D#s<$=wqNrla+wbv)nu+;26X|TH7 zjsNFC1iu;lzJM)HlFg%kg&fh3FoN-&V9D2r9g^0HKLg<1dC zozxp5P=$RRwL=!wPHmVqXLi_uswyEHvJm@R+0wR8@#op;E$!#ZgKgrX2<$T|*#jj> zcIBT@lu5FR3>l~^k^dSFz4T($Yxv9CdBpC2!H1QBy7CclVFv2hG#RK4C<|j=`KBm! ztdHtcjrwuFt6DT8Qp$bUv9F{Y^-|^e zEKpq7*2~vH1j~xat~T@k@!XJ&@*Lyfb#k#(JKW%WwTb`6W64BwW_QMmCB(2-bNPD1dpEW^&;N)wj?{kJ$rXs8pUKZwSxc*Pf6Ea}F<}WBfXR1$wnXrKQ8ngaauP}4! zRIMwnDJo9o&{P1(;!=0ltP$<%Ro&4aGT|`sf4!-z>}uBEh981QR`Hj7YH$-H_eeIVaZ`7YsW)h3zvT)De8KsDOVg}sv_eBBL-{ds7sVQL0KUPFdCho#z zz&+IS*%uTRw)Xr&{K1j}VZMp~CKCcmXM`~So{%pfD82zo^Rgjq76_9wL16xi5ROX& zL7LrltbbPBajNRB1%zq7Ps11aT~nLgb)!>?PxpM*Y;j>(nr3(1ys$j0?OGJf;I7x1 z_J17R6(x~;L1|tV=Ic2BVYMBX0rPdAqX02a&5U`0>Hnq@a}nLOG0Y407YvEwjbdJa z^&e%@sjO$P&xm<}Y5!A+St*hR^MWS?^VAHO7qI^)=4qKRFEst%G=+KmHZU(V@o%Dd zqnH;?7R=KW=JA;^FEsPNX$mtHNrQReeS&#f2FwfD{}c0!G?;gVcr?$2CjUD_VGbbb zN$uOsF-C+lRW?&n2-PCf|0k%P(k|W{E}lXqjeF4)F~AIEfB^h5IQk-!|Cyl-K&_re zX0U}@q11$PQl$xa)H*FpBe1>erg>BNb*2O);@@%c+ zQ7{9D#U}oprCKXF$5EP>1!D1O0%BGM5Q{ngBZxU^K*->-_;FEcj;h6Z5Y_nLviPeM zf^7^g*Y6rqu*J>;iVV1~H~rrnRe4k<&ERtViQvaxYEA~+*K_!fRGpc`Zc0VCQN%scu-WV}o{^?%c|EKcM-5>B!i`OV2v9G3yyQr)N4e_nDzg2xY16D z8jkJE+O~61hF1i)IOmQ371hoOL39E3JQl;nQc@Q;J|OCylS#c-M7^`qsCN$-5yGz` z?rb(wrQV-dQ_L}5QLv*|6bWa`-O3`~1aJJT+LQL}HDlJ46Km%*^g6M2R@CE4H#H4weI+nPMlU=F?(RS+8Q)zTCeI}$IYEKWzK+#-u-)* z_wL_kVBdcDEbmj%rtX67vr~%1;w79uvQ*l96?v$MVBSLXnV^*(e}a& z+TL=crLDh;EDqhjh@p(7?JcK^wwI*Q_AHczwzn)m85;N!wabUE(hhg|O!10K6Z-mT z|3vdj^nYb#tP=ku_5WF_ynoQFmb;yf?@?;*wBx5uKM8(@e|7x9TwN5oYjnu^}&QGg|uY}UhDl~t(9QoZh==xTc;z@B6M*`o(cC=*H=MlJAZllW?GO+ixU zw=w@i54|Kqo!>Uns`HkQg3@a@@shLgVb%Fp5yx z-60LL>d^st6e8YnJL5n4z82I9)q9*-Z*GhA=7gUOVlC%U36dHtuL+jC5gK7G73oO3+316d^kxHy`r}nAzCnZ;sij&@XCIr(e zi_%ynf^t;udQonXa#9b{|&jQod+S8w%))Lqm!JZ`{Sk(-`V6aP@Z>O83uVOv4>Zd1WNCJ zAkd=Ob7o;0p@Fe5`N+PF@jN~L0ce;Bmw4ggJ&g@Qieti*!^=Hp{<~NWF9D=wXlD0t z{>x^zILV#--9ZGy-!s!Tk`n!FH~a*5@|&a2D@j-hT{@kN7d)ZD>y^zQGJM*MxpNLs zJ1AApIMV!oiZ|(xj@^6d1vc#*O(R}!uRI%-N6npM>1-emf*dG*v)}u^O|BNwUkIklX5JmWV0wyMu~JAZ4DQ>*VL<23$RIHmlUV%nzT?1vetAP0 zzm(4ZzH>#$4azUa!!I!wN5CXq=YQWrVd?4c;_)bz!7J~37bWSHHzawbC(FzC7Yi%Z z=D;e?3rFHkDI+oL$v&=tdf_4u>D%r%`#&_FnRYGhl05J#desBa*5n{{egDO_Hfn1| zR&_0wzJDFaY3U_tEFFOu=zqWI|CT6AM*w&%o&{!)maugMe|Nx*@(=7TDlJhx0Ds9~ z>j%c7ByGJU+17s-R!;w4MA2$LARYfWYz=SbH2Z?;O;<}2XK*lSWDG5Lc{Y?BvNpyC=G!)a=g8u9~eau>5pYKBMbA^dFR@>)f2|I)`XGJ#?$E(y<6d zS5b4CWB^>p9bz**j}gJ+UQH`1d-v&E*?&MEwmY(ynEc^ZjtI8k&=Gn{RiHF3*gvxb zsTndA7kQZdUsy9umfMWeMTy2mq-Gvw|IeDaHFM28yi!Emsv7k?n5WSw@nN(6ajR<7 z^Pp^oM*YZc;LMu2HMwTwarZ}t+31QWdP)TRSTp4XDN&-mt4Os|JFEhUdM14cKG8TH zG5HVOr=AbP9o^Tf>BYv;)$8llxyx9>MlzCGc;qu%PqqIiWY*T}Yc*6s{_!8qW*mqjaViB*Oi_vBa70ZN;AbgbZ zU&7BSY*uZ(eklk^PSRGmp~*7(KWqB(>Je7~R0ofe>qLVfH=^^#QIrs+R$<~12 z{FfkZ&jjLI8^l+j^roEq>tXn?KrHVlAeN>9Au<1Qv;S?W*ML+a6Jhzqo=${yL?TQz z0J66eiO4f?N@i`lT5jrJvth3jvT2PhSjfhMQz~Mvo5%vja<2cf#x*f>V1?lC(|LGV z3mD5k6~ipeV3@~RT88o2dcBf+PM?Gi%P^1i6~o-64AYXO)WyDP58QB*$Bq!C?&1Ol zUhIp*|9qu+ODGpsA}dpfr0yQOKvcXdSw-myAG=eeuSioZeMP#N|E>r#yxO-qxf$Mk z{xdXg9*D_6GGrHa?ANnWyjJt|RntHTTy~0A^)-4XQpP?kHQBor-)}3~OU)!Yd7?WI(7&{H z^zj0d;qT%#u{tMLJBX$Ok&E?t%00Vb%VVT>oN6 zxGqiYAFg3Nd6vLh7iLpucbD29;&r#|JGd<8WGZEd^`HF6k<$w1JAOZ}W&pjct$L5F zYVY{hU^VtZ2SqvzpRoQ@d)eg0*J6kd)!fHbv(coYUjcsPY4|e#MfXzh)uWQ3TKjnW z+3*|Xn@4@gQ{XN>+w^8-eTZ4;Fy?H0wVkFA4znIY?le^!M<(mn3`@)@lM8 zUfq6wp5h~%@4r)~Plqz|~w14gCW3u%?dJL-M8Pon1 zxk_TJ;sHU*gOSAG0YO$xAT~mU*-4CNKC$amJr|IXL81*%x(xTvn)pux7;m?Mj>&BR z8rx|CvXC^vvnSems0Uv|*W4x9Ow%<< zLgu+HqUc7>DAd#3hwkV{sL@>h50{mVf9~M0^mJ7^<9y<|lTebK>&E2H6@;wn=gj^G zg-r=QjNK}$w>oKI9&5B?`$AW*N) zN!K$PbmBNGDuIrU^Ku%;iD(Eb4i^i&tQ@Bt4duYb_I<24J1jpPu2T*!8K$BumZBtG z=jCMAktD&27loBZN72KK>kNSFSOW^IlX(i?NRnV>KSxB=uHT+cH_)gAsTnItu<|^I z@f|kf#{5odX5|YYXU%L*Q!~<}SAHQPHmjN$4fD+D?z7s`HR%^3VQJP2ohU=SyznQK zWW8)ot``X-UpQV^>3X5yx$K|Wq`MS!PpTJebm6nfXxsRi7r98b_JzA0PPMSoj3DaO z3}PhH(F>w(6-&1u9VmIoc=CeDe`t-CJA5#nL{r*1^(C=DTD6~}2qB%=Ln5~>UJq9d zyvyoRHZHNU@=l>sHOTzG(9DfGVr~@1BZ(50fyuL_uem{fG>yiJarYh;#^apa8 z``Ql|2p4h8^<4j_&Rf&yd<@FM=hn|a8G6N5<#Y4Vus`5yjMYg7QNG5Qrx;cYZ0J*0 zU*32<;q8jm@tXakSkH}DHlp_&IwHC*hBE3c+{8pgN8&~mZEWL|C87`cRwDI|bG$e7 zROMvDYL9;~Akj_H0(ZoO6iHK%F3}u|=Xwbm_ zlN|1&;3PTL4`@!x8O`A%hwrb@Fy%!lg65@z97#lTZye|91y>20>je$zEHp2f{7(YS z!Whb^R|KP|svC(JUUKT+uZ12%ajaGuC6x0#Grn)6ykGzF`tkvd9b<}7w1^!9*$V4T zX8)H`!=DM(1Mp0|YMAAGk{zsT>l(@{>&xxeC8VCJXt9hR->@jR-140&?Y!Vm__JF9K8nqbR@Kv~Z*({WTFsAq45G~;uLcZ&tsrkS zPIpq%$Z{Udl)bU?+X-7?{c?YYb(yg4s_Q|s&eB*RYcbZ#9REXOVc5?1#!|X%EE)7( ze%LOGcqh;ME;!4p*^^~4=$(~4L>?DSY~);z8&&bjE)I9Tsikvps!WrIdBLgqh;~~a zGG;_YU+m%^*t?={|BAi?Dk^B!Y)%7(3@wxL1ko#I{->R=42b3jQ@onif=Hjhoz?JQ zxRWz`HzNAjMx>mpJyFAQ`eHdcCdzK!8D%-RZdHS8S=tC{vzhFSaqOA(pe5Ipq6uosxFx#t%P+e5O?B8DjTBcP2uOj|U zcpn~j%0oU_HbWMF;uaz~SyXb0Egp$_$G z=(tyw&n^1ZhmWk`In1L*R*kGaBnQXKM{MZCkFPaxq@CgUd4u@oMKz=V_cyuV*4!<} zdu^h_eLN1JZbA(l$CD=Tn$!LfmFM~EgIB$pwzLs6c~xuDw|8Ik#P%CF()(01@u?H> z3AcXjGe?ssxW62{<<&GL_anXFt?Y-x?3;O;D7Bylg7I~e|J8m#FB=#vm1DwyacM5r z*4!;?zs~(%VGruj%c9f(!>_)qX1>Kk*+etPRL-6#NqD@a2MRHy-CBnN>>r9B4WjnyFq zV5qu8X3YnMx(L?5_#rF_7>akbl(JyF$^0K-e3A@9YV1wK|KZHYqd~n%-47g#4;Bz( zyvg=YE+1y%g8WBL5CQmOST^nj`)7{` z8#jr_ofUGc1w{#@x6Sj92}t+HZv~`<0_j$PL^>Oi$^U9PLRuU{ndUa6qd)^l@0k71 z2}qX)amnWl^@4czkoaO^XIl6YL{N;tdFLX>(yXBe;_C6or$q-Q!|M}5B8d=*G4c<8c|iv9l*amY+3|)lqm!G zdqW+$44Ve?6$Wb7h=KfDHIQf3bosgla<>NZ_pWldGi)kh8Q4o-Y;s1EXPvSw}edNEaPJ?07E5X|`7ed(dcJ}U)OV~s$ zsDWU7->m;@PZ|TwUSKZ6%#KdV#oBG62BQS$A2f3?GHi;mOo44}89_ea{FlSb7iyR( zm!TG_TBF_vN23r&y)QD1ARi$AlQim`vGm?Eg^P@OA6y46j1#_0GlCd(o)UGw6m`;# zdLMjfODlsE@22_x{Ls~TE9XDd`O6GCZ>_MnG%F#OX_YUXAzVb~t?d7)^Vex~mc-On z#y=c~zLrt1sdrOwl(-ZUQ%5P68rZL2Ig&Yzzgd%MHZ@^fJkR`Kp%E==-2@*A@4$x@uK4#jvEc+!(mR@`8Xi|X1wgH zrVGms_JVQQqrp=PL?cPI6(WOrQO@m-9AbGmJ|9>v6)f`w3+Zeu?>ltFa$}5TY%H7y zL&isKowCGo3l{jodG`_7V?lz1SVlw-sjeAT6&+h0tBUG6S_N5@C%8U3+mT6JkHoQl z5nL{~778xXS-3u0=Fkz>O)-=SLtfUAp8#4QfJkQn`uO}5KsU!w2Bq~yKoXODZ1P_TK)2y_ zel=ZL7K8Yq*#kmMB7my(Qo>3RBXB<1-BF8hmc_AF7F;H9t`j(OBx%!NmJB?ofLXu40<=u40>0q-e^>!XlID9$gPHQU3J}fA zr2-O&qyoOU(BaNFEKbx;m%&s88rc`-`G0NOnb2fDj*aTbx*F(3uoQ#6frpRGEj79T z#N;pa_hmOnAIA7Yg0B$(oxvM9&%q5_er4ic&7B(-{)L0r)uQhaJX0KzdsdAqmk6Ia zP`)7i%B+8A9q3tia)+;a*b4|}y!e;b`m()W#s|5y#SLfqW`OAY zZDyUn;rxd>e@mTV!gM-+1OHD*Q##-F$!$-FG|~B+#iH}KnRLedKje~sC8)heufOhh5wEUIVD-BL#e9aIuUn2&pbs&7gK-pIWRwOl)+C86nk3 z##DSkB1~2qqX{WSLaJ}g{x>a(LC9nA+Yp2-@=*xWlG`K*A)OV3e0!&(B7>0I$Xlfg zQB?IT2F~kFS>{DvZY5)X;I+#45Xh?6X;M+lII?fL`N3j{q8Mqq-*NwE!gA~;_Y1z0 z69&+9ztf#lHkM;5gxtwln(lY>{{(k-lg(HrpSI$~az>MNPFW7P^>~AK7#|#`C4 z_11}@AUkU(Fp9N*XyT}`~&8H zVFT73W%)I$uY>8WPUi=Xe{8@%q-nrLoi(D)528-GPUi>2{|RYjkm7Y&dChE*CY{cY zT>q!eKV;DP$H5kt7C({8s6$@mq}tN?$6C?(r!+c`M_F{HKmHwM=-fX^r&Hjq3UD}w z+C#NJ4(CuVmDT1wED8@BQ4@`i88>_^GMzHnej9QqOM0FkvHoGnW6$$={C@O2x3PZF z^V}~z59zF)=cj!fI`%xb$53V`+bs?Q4KV#==D!I{s{`z0P`juGdBK$IF=690Fzp{x zo#<;|PZ0v;r)7>xgz`ij>-E7BfpVunA)O7y=f=kAV- z`S@-P!(A0+4neq^Jrq232`FKr>Rkw+5CQY^e;h#s^JM%%fVo4!JSbpDXMy?ItbZiH zERCUz@*fmcVCwKmFrPaz2xcwcsG{~29EcaHWDll|*9u_L*&rH%CxuLZ9HfI>z8MovV`?wFj2CJXhIXShozg)h*9x_ zYA|w}JlrdgLs^39SH?eyCQD3D$8QIw#e(T}!9+R>)2}leI%2vjhB9hp3`JE{jmIZ2 z{p#W;rggz2nc4x?V9LiCST-Fo z-5o<2wbdC#6(+qhnD4pxiD`X+_rXRe0xN@4v&V#$LAoF733)h4TfG4`iIu^8&%{5v zWw4r_jo$-EHw&bD1QO{iNPhbiNcY50M!il2MHM72sv=bJ<4##Z+7PBAuzY($_K>iA zYamr0FoNl1lRLaHJ8@bMbqD=oUeAuOK3wjpzf1j+Jz83}w`AKoqqR;dwE; zLWyW2U&GgxB7o@lToJjew_Cu)LX>x=BaMijkKYYMHwmJ<1rg~iM0pQ7bVPJt3}w_t zArw`Jj>ISGDDOR|ED^oLWL!6jKx*;0>=7Zgh&oE_>3tm>C`IZge~crEXja7U0-75I z&0T_qbQYTYa~wLNxj%+7ZAiweBW^*Hf45VXXf_3t<-rA@nVd74aP9V0ks@dcx;T=E zW@Q{3m4X`t%~C-_Itxv~VGbS9JP<<}wY-d?3XL9K^9$ViCz_Wz=kGxg7+z;&k46u# zVb=E@Fh~rq`2{BbpXdx;h(7=@w+omD1PtjcFoo{;PcRS0P)05CqNoC+v%dL-qix_m zv%V|XbkyVwSzo-t5>bEe%G$n|du(fqvU>g|k@d|lWc(WrIzF?$tM{`KW4k~XJqOA! ze90-wt+C$+e@`rx{ylp=m?Iq_oa^z4=2X(y>o&SzpSeJV8`+ zqa%}9-}f1~?8VB%jAxoclvR*yBVw}xA{@72nO0?ecP46hguYl#Pcia~D^Zp+o}bhd zqt>1Ah`PmY{LA(D&cS(dC_U6YPpLbh|A2`D>MORrHMS+hle1JnvC02yS7z3?SujVY zxXAj>`TZ5pCQ6xB1vJ^qF%hSn|3RiQoK`Q$r2;f}B9aPd;`V=>avnepoM%p?|7+s* zf6QvUvV&Kn_PGr8c3>dWblC7it5k51TkiLIXhbPeizROV$E3#4;0Lc}KXJMra>1M* zkh_Js1nd8a_KZ+0Kd+Q!oaH2=$r7h5lNt@dffC@tpAJ--R1WMrv9i8m+uLGWh$cFz zk>9k~;m)Ln&z%*!6DN`y`YDmD8dok!4Qi1{YUDR{=RfqYbNN7?3J!tsx?HT?CTcK> zhc(6i2g};@umf2p+~jFB@n($w5>0%x+r)K(Aio*r|0+T7Q~Cr!ezRk&1c6R!EPwjc zsQ{fw5ac)e8@Mp3@pGC4f%G`dnE#|YKTD63CaIC%?0H*S8KiipeE5G?h%}?~XQK1Z z8FVgfWpiPY#9Q#(uu;NAXHxS^+5c1LUoz`l%Jom`{EN~VPo)-+87_hnV5O(q(v;34 z_gXnw3W(073q|K&GU<%@Zydk|T(5)vEYI2t}6qD2NQq zLlTOR&I(1E%kxj7AVZOdVko1sDk!RY)1&c;A+7l?nuP5YEGr(tukA zFbG;X%6@_8-{+<6C|fq55HhJKG?4)G#nx@O{KIO%x}%)_c*`DAKpJq%o4^H~4g$U< z!a{%Wcf)9vsN;(|={lX3U)j>i$mE4D2h0>M(&@C~`X6=nGw9rEUyDogQ_1D!e?HPd zxQNcJYDDM!G&&!HvZ&2gGXL2tUpk#;dFKb8N~DNwNuR1tr@nt>zXA2V`!>4dgw>q* z826Fj4LTbS{c2D3aT0Bk`YAHx~w>om! zl?iDh@sEXQZG0IJ-6M#W2_n*2h}ua0SLlf7(HP38&4&ijQTPK6(B^XoJuBxTKIo(OlRYBtV;@s9cHGFC=pkTT8v6_)D2A#*RNJE+QS`MJ<4*$4g97JCfkQeAPTPwd zI>K2--rY&Yi#{n+fwq$Ww<_pjZq86a9a14*XAcL79l%jr`V^W^MT(%=QJ#OWDra5% z37~mE&^#e%NN1thaY723Azb7}CIJW~iXpXG`sk#yVYqG`wd z&u*`_A^s@P+%0Gx6*Q!?(fs1j5zXT_JsyO;7y^G_(4(ojCrx)$>yPaUi;15Iru4NM|A1=_-eg zh@On0OjjEb^JD02c6!z>>of7Tk_!txSQd%5sVXGuUtd44vc7lY^J5$oH55c-%#0eC z@4TzSoQbzJC<3kZ#bOeF?kxFVU3)$gZ$AY8@M`+9EJCw?WUmL)R8)eTCGWGdiT~B` z?KAOK7Mw0KWaNELPn-9t%q{UonO5GXLpKLJ6K@|dKfAXe(tEP9icf4r5?FM={vXRX zc;G#K4}iq%J$k^RFV0)5>V@lUFlTT<@n`kmm>FPR<7TPY!ZUD|cWS zCm|%7baLb03i#)~On#~bLDYdxeh%n8U?RM#@pF6VRc%BQnO1hf{Ku+cPAl_y0SPF?)HGj6n>F4ItkExC}-bE^fMwRt`Vn?iqRUSXk> zG^MXhDgC1F@25)dmnhvuct4j?dY$JF!H-3SLp)uaML9#ID&L32rz=%>a$7i{SbMiDq`Z_V%kTo!t9do zt3j|CK8p2OH#8Fo4AJ9)Btg_M&sR%UAR3W@$i%-1qE1M89|X1_Iw*-~S9}fi%JxZ- z)Y?dK6Tw~2wxubZANc8-w?vxg+?C@$buP)I^9vT2e0KNV1805mOOno<|5N8? zX>^upWjE&k*-)E#4WMRxd^w-%RqNN7x_qnRnn#1SgN(xo!!*5%MT=l$P5a>mQaCIef>0pMeeLSHbYlYDCG( zjDQi4Jz2|$()}TaJ;(1BEaNO0%J|*=eWxtb!ja&7*~5qN`+TKI-GuTAjNgr)vB2@0 z4T+p({O)mp!=2-I0T01bAu`5qeF$DwICB}lsf9Iu_gLX*!SVZiEGO3NBi6n?7i%Xl zinVv$)8WqX`ykput-3-HRV&m8HMQ%ZD9a|_Of~u2(~jS}o`OPb&CN2j=3O!VvENY- zA{4OS^bT3Q@t^ph80mp_T@Eg6&82Btvr&isPjyN~opj^(ZcT+z^EBhrZV(7uyP5d6 zxf-9?=$`K3v)SRyh}h>PGCrw+oFse}e4UZa#BIl=f*x zvKIk_eOH3)xvoCv%_RFI6WQ*>?~*-V<}-mnvX3*8MN`ODM0}=%NcK@ivOlMgt?c46 zX-TpVGm@G4Zvxv3oqeWkNVYX2*}EyyR(0}YA84`fUn$bocJR56mt^l|Bzr7{?8Tk^*gKl+os48A{!1viZYQ6+xhd`K zOk{hS`EP=3eS4p~a!K}9Ml$aIcb#iPJ3q!fiXeM4Bbk~1C8TZK(T}~M$==9FwmwDL zOKttw7EQJ#6B)XPWFBC3T7l^C42EcR6hUxE)gwHC`@C~ippokG&8$PYM;1Id!KB#)+$v@Z13{t=Me zkd}ncztRsAinS^5x$TZpu1`l&(IbVVEz%N;HOb<%B!{Pv?1-eqB2BU=Er}Wb6Ny-y*2t#5O38w%O$Mqyd!$f}D)9lFGoWbeOjI=~RHK{t zQ8qMCwaP?waSGL#VqfjY1y##TRA&7zfop7$uXaI#sznB>NEe&ux_X{-9Ie2MLS-oGhxt!96z%Xhf1PyeR}V=68AJM=EC|}CK_Ch6 zerEl*JzrHs1W0D{e>&~KPnI;LjsKg{fLo+N&Sgr&F$&~@?ssbkdZKvMexeRQ4s@Vkht_4M&AHhx>wfulDa4BI$IAUZZ?rLuw z{=-FPVw?VK|E%4e(%44Q{r&Gp8TMj3(Kf1c-Wg9!m6Mf$Yo+`9oBY>K%0tJa)IJ^- zx>+0-jUsRdv=+EKCBr=%gV&dO#Qd>8t~0baJ-IZ!QgLipda~tXTj8J83TKT z*iR4{3n+tjJRm|X0H6O~@OuuQp)j^nFdBGmC~y+KVG1=K{307pE4|>Cyc}$;)ziGi z)%FEXpakVn7!@k``hn3rI_JtpK%kX>XY%DBoYu|Apfr(oCqmxPFZ2O*$ zlI-YqPIk6a!4CTF>*A;EPh;rLl(W?q;LnmA3=kfz$Ereq81Z(-F6j1NQEd2ucrF%S zwZW~fCS|+7-WAEGr`M*_{QiX|P_7G8*Iik2M z@N)cVq|P6vtWq!jbY2-jXhIT+!3LTA&ss+s?6Vls1U~6%z1Wwz*vmc@PMZww)*dFq z#*_H4C>vmS`LvfY*s=H;1{-{=gMtPd7+feT%GhRfVJ-$sR;HA&)HWlr*i^EsG3sSdWJUy_UYlGG_BE+t5e7M$q_ zqtRZAKM$iV7o$CIj7CDsXu(4cA)W85*than2#ogaZ!j8<=Fmnf%GGGmgB*2ev?0Nz zGCII$m;MH$8FfmD%Qjl{K1Udh_Ii8;jP{rqZG|x!32C&pxY07A-2ctkvP0$pKenSV z@1O>JPEwAHw(|owx-0`j8SrCf{?j3x{|v!Unuij+tmrbv`40#D4#@*P^S;n8cD@6Q zvX6>#YSaYx!v*-WiE6spMTuF5q2!LM)3O};1>Y-+c)q77$}4MoS2j#&{LN6FN01WP z#ZCTSH+k+;+!$<;o$H9~w&Z2am=4=U8Hv14#tlQpUv>-`1^>;#JF*Y*7%zAy7kA8Z zRXU*vl@Or&Gyjtq%*H^uReWY@u#YkT`ezL^yXk@rbczPL|Ks4FnCGtXP$ydW9DJf; zTjE1qoZ$VYdG6~I|9r0SpwJz0!@^aAt$D7`k;m)s=sEON7LfK?w*E^3|&1dYvetL{uH4Y---xm9n3r8&$OVO0sG0lH4@^4IUd!|=Z90AwAM-mp!_$SJ%3kBpaKjQV$?~M^1`8{={uGMP44E`arbo{am z|MEQQC3SJYGge&`{m5VE^mvcjp@7f%AOU=A1Mt|l935?s-+{m%?nT@Gp5CT2gSjGQ}l>TE?r zMomnakjoC!XVo{%YN(G-pE~!1X}TozqmJkW`r_4Bl?}CxKRBrkF3o?KdH%)pb4A|7 zV3>qrX#T@=^UsE{?p{z~s%w!@Z zm~LV|+^qj}RO9q#V6%r;n65MU(@qf2&Y)v*XYiDk{U;YcWtRoq62&HcV*&m|e*gp5 zmRB~^HB4x9qrEDaUH_kKXa+otUE!E;>E8HHv3<+ zUq#XLzQ8lW)Yd#yzzMl%fTQ7*`sCdTHy|GTwAi3svJGT>JQ(@k;V7#Q^}Pyzc{{KD z-#hrQVwI8H|G{ePlBOCZ+8SAbGK{vmsJ(Psz!FTT3ZpGtE%`tC#dcm*7@FP>D{`0CvH zU*-;}oMZ||pB*aLfNy7_hgw*MQz`Mc7x&Ay*V`_S2Rqny^f#e2lg3{}X{@Kw5te{) z`2U65=S7b#l$_vsz6uptL_T2s|u~DW`3FoSnCL2R^;p0 zcW_$rpWwy5E6Jg%nY+OYta&^f%>swXkccDmFlA0nj=>G>#*TNi;YRu^cq+8oNROn* zclp0vn^MBElOkhRI>ML~*%Ds~+dfY9kThSJS4J3`a1VU?EJ)#lFwQX3V!H-pj7dhC0n614nJfi$*5@uQj6Gt_;2xyCDPoeJ*xO)u@!{lq^~S z7N~BLV!AitFTiw9i0NK1rXwMVBy(9EbqMKh-^PA`Gms8RxF2#cow!>xHlj+~4Jgyw zjojyz@e)O$T>dt_GWw)jP|4$ZIv+^ z32C%5al_i>2++gNaunfnD;Ho3Ul<(Upn+yrPd3mg8tCDS{}UnW=N>|-M97MYRUs?B!uQ)X zL3qS5R>-Oogu}l*a}WesA?p!;gLVwM%QA|9d#TQWKyOwJGG%zw~_;-=Ho`g7ByvYycj+mSdwk>*Y)) z2zUNNyOz(>CaU_ENB;klAf$C0PY_o2uN^R;VM62QR5=O4qs{uC?$0aWtCND^@}>av zbHjD>&xWz?=TKp)YmuDG(Qi4G$^_vKJlnE69R)d;lhDtlNh{IrIrVB3Yvo*yInoi! zUhTqQiM*@{u$JW7s~K2NLJnHvF%LNWnSkpWj6www))FrmRjl-{=+}25#+`wUJ)^1$ zX4k!DFccDM2jpNbH~&UJG87zpo(&36^1L5|f64)jFxo%0(Xs=@{2L5n`@*rGI{bNl zO1Iz)IlcqFaK?7=1ufR{g_?q+Az#0sC_(IK7A^)lPhQ83v zQ@O`(Dihm7U2Glj5A}`g4CfcT$VYN{J9$^*>tJ{Q?>MNAsLzy^dboQ@;_FiFb zKA&Bmf?goIgfHBBtjdKjxj~KTeK0_6gQImTbfRN|K$d#@ctMbhSLb{AL`>F({SyxR zGQ++ry(_^!TMQw4afk`sz@84#EsaA&AOZ*{94IPuQ_0=Y=)*^2!+RXRIUp)^#W8d4E;PWBdT>p`Upy^Q%=zUf?`5Sy#2pU=mf?d-fpt3MU zLtm6(PIy;(JHKo!9B-&gDh;@q!|?{Q|7BOMdnn!+!tsU&!Iw3*YjTYZ5j8i9;2w-i zz)He#c9!Wf$VylNqG|3(A4gDRbOml;o@kzbQOwY=^Yc0ex1c*`<(=*Yx9G07&%l1Q z2>%*=!8}tFJMpNA5mk73GP8>{{MV?FRU@kpQT=DNYK2*^<*~b^nv+FUGymz~HQ*Z_ zcky~NoN$T1y@UN`>bQ$c;!AUs7Y%=B^ZF5f&6k6T3d~J!BC|a^>jCpT|E&b>R9oXeBa~>hd)zS-^YHEW1Nuh z`>90LU3UC=g6MC^FH@CcaYMby%N%v)>b8adOFgW zvwAze205#z*$R+gUXxeGiq?dC;JY(%L(M5>{ZF?ZTJ49}&vM8b5`I7DVl|$)_!~3K zRHs%{`oY0M%e@&)63ZdsHz^m(rBSJr`0Rw=amPCXX}ovhYhk=+#CU6s@kmJH&BG1j z9k?&w6*y&d4KfXZ|S@U4J3G}m-Gl@g(nMoYj7 zQ{QkT(tPj5*THY(a zK2ZVFCOG^t0Qs*5@5+FLH1oT;R5|RJT~uHcp%Mn@hmIm+u<_|do)9Y|BTk6T!RCfP zc}oO6o%_ETkM&57$9B|~m_F5tKXn9M_Q;)F8)gBSZB1wV#|V1&H1X#YVRSdjFoND) zMbIt(pGD9!F#ZjP4l0S-T()A_(?KgP^lUFvHCM_D~Tt5HuD+&sYSG z?8!`NxMt=Edd7=kX-3fGogspr@iR)Y#`Z|AvHn`knVp5z?ka+AX-Ci<^&X5J?L8PG zteSAhg1c1E1ZJA~zhdJtcJ371>UsHWVza#9)Z%JKudDOoo*rtm zk6`HeFB^2~cLQchs4W)br7N5=8|}2vFonl;5UKDS_Dm%AB|( zjU(*)@%0#CpQn2xC|;jeMjV>Z8e!-B)gj~vJ3hLLJimhxc9$GS*zE9_5w>LxZO$HH zC-kXjD{b_#X5i)sI|etrW3I`6>gHd8c0UL^i{uzY46zHf`(?wKhHOJ4DdsX}%zYAu zTy+XiV}KEVN!)kVEs$=p?j(mHn`eb)XT>^<1jbHhdR%4;>5~618Vy>{x-TS zhhU{A+chPK4kw>gJ-{DFr(+d6!oqC?X?{eYPS)CG<)M?Ke9_UR3OXf&CV zOI<_@-MaYmsVO=%+r!7gh|&S$5_PEPQ&--%q4DOHgHPh+AdXL-?5F5Zx;x)}r*!xO z-+O1(#mRq9(V_WnK3;CAbeN{>P*>M);DFwB+pTQ|Z@YH*3-iB>u@e2m4&8m6^RIN! z?_ixUu)emuZ~gYGi_epE_)m%sExP%5m_zAsqOwEn#6Go^_2rE}fRckhsFQB2x;Uj( ziViKi`T_THqg~9_IyBUm59m9w@ip!obih6<$0tv@I7Nq6WqvSI>o8O6FtB$;UElh~ zkDAX#hc8_n{;i)<6-|u5R6>UMQ_cF9R%8s8H}8he)aW;|VThlt zhxqI?v4?mu#i=)lDRxgbg-k9_eb$gbDOQloxklr>k@(O23qO+ae{hre zDdK2~xyM;?=ioN{M8BHp1-I#0aqoc>>+uNLHl9gv-uU}lC&g6kNUhu%eEyBCpr_ZM z)XDST;wy&nw>Q+^VSBt{m@`qB;f%k9l1>mwZ&MOu=MfQ>fKmAW>wo{7jVY4Nf?^r} zu`4J~GiNwhm-$;KQLS9f8Mu~oU~r%xe2$-c<3pV%p4Fs*`d3x=2|4HaQvWbtE7AoG~ zDBgH1xes_Z9u86Og`&3%T7Sn9sR}X5le<6;Cm{HHnE=zh6x+b4XuemYR#C8?ny&}; z?0P9GfF4ezrOp!mXE^+sI{GD!9#CbaC*#>CojS^nKc37QHbTAiyr$~#YP`LH{Rg`L zZ1P9crw%uC{Kt-tPF$sVCpgZlY0IAZKrc8>hgH8rr&2;x;w0+$hk5=>o0&;;JUa`+TxkwN<>S=7d};Lw0P}5u= z#W%rRtHfNJjJZfiBFT>bpUHnFTr4kM=;;GeksUA0#avRUiRJskOctY7^U8RMDy(|c z-v52+;Go4mK(3{xJqh4r({iy`k~*ctW&7B_#yi4jw2yHzPff5|jP|lI8VPB%GjT)9 zfBnxPq>tst5qqn8o#e%faxogVXTZmj7VnE1FhHm>HL@F~uEA$_;|3^0gn(jm7*0kxc?_owRrc0+# zDd8z;V!Qura!f$meHz~k+pQJbZ8o+eA&KVUhV9NZ^Z!I^EsQt8@x-L7_u@@*v7N5v z5u@S2+)>8MMs=wtT3Buw`aQeibaQ6G5S%v?;DMxRoGv%T)THYt|-EdDAi z_o7(tRbx34(sKXA4a=Q3!Xc#B6~#+DPjxKEd-0N7EZ4?sMX!q3}A+1;(Z|Zrfz7fzh&Bcm3Adbezj2k`{TTP7#n`ST}RSUxs z>Qkq6I)7J3Cz|jRruR%hJUbT?rq!vG5bc2Oe7FCn{l19fHMYTevES>)ek7#*B%nKg zt)naL*CgJ|^L8d30=j0o*bi^2h8K=Iqzcb9u-9vrY0SWHiNThd9E}@paDhAi(Q=<+ z(XmFoVLR}!@Tityvztn|+LRKNk|Qp6!3qaCP4;CRC+Y&V(yZw$u7Lm(S|1b4C`__28PMz=58{hO({{?CcE%gM;1-?Rs0Q@ zY@?X$4P!DA(qvrMM!jA%z#-(?eQmTO(vqaZ+I`0+s$R3(tcA>bb`H=4%yD&&+ z+WmmOwRjk=E{BJ(FOE1kE4+MvF!p`xfg5@KMR^)$7L3_~+R#S!?zoHRIsCah?zP}O zIhhG781Lm$**UFXP!X*VY&Erxc?hE`@PLrpD7KvnRE7+{!`qbLgnnqn`X$P3|!n&*$Ox8cp34WAxEZP5#Kvr6Q8< zao_>v5%n1MgK|z|2@IO<5P+cdZ8{bPtQAAIySr1XbLI)v3CkNxI4} zck_RINV^3eol~!~#e=zjMn9P5*<;p9PQ4z*Qlx?9j(_a+)-xAi&ik9DVP)Uz8CXwh zSXb0IX5)jo-2=$ZV{61i+M_Xzg#LXi`r&Ypx@|wCot+{X428#2N9SO!j1OsFY1Thf z^Tfh%p7&F5cH*5BXKSNn2Z|4Avod&#FI;(%Z8V<`X?G9)mN?P#Z`;Kev{=g*u4?9p zEd#9UnDs7WUBShV%gq-Ou(+QIz1mgBJN$VJ;Y&djy&m}=>IKnU%toirqOE~5rHi*; z^|7Nt6kW(Fdb^rh0ik$%wiTC0?G^7<_yoeM`TQ%#rk=@Tll1;qWB+g1`|BO>(J$vd z0csYImC36a|8gwulV&WQB9iV!8AcO*)D8ez5kC;-9~@0wQ*I&9N4mUn+t4xywh&yy z{67&?ra>TU`q!L{GR$pNs%QcT8jB{bxjv~h;AV~{u2~-Vj?T03KtBx+v;^=aczWUbOQc8cReYh-hOpf zg>yOY89nhIbKW!kB3%&ULr0hjoy^3_{y5I1uJJ9P+Sp^X!LJ+Z7@W@&b>q8Bf&dn0 zKYEew=d$4(K@GNHZK-`$$BKYELjeu=Ch8S?|K8WDX^FqFT)MA1D$=!hwjqL&#=eY_ zj>irESx^>AI|1KN+J7jGj*8J#>&O4M^0td;ik0qq!IhzC4pR(HHZb3=w(-M|({me; zv?0k`y6Xj8|6=3slic{FyIzR>A4%QyrhEUo0UWH(Zea{u*xB_<)7eRPy>KGRu)FT3 zx@+h_U!W@Nu5q>G|JeU;=M7s6U$+nx{e@DhPJM;j=Pd+_3Isv_GzjQ?aDzoA{_U^2 zYpRY8SY2s0Y%Mwz9ND>>(tw+}yIyowSeo56d1vUZ7u|}Itg-&dH6{-%Em|e4`l;@E zG#OD%CQi~mu%v}m69`$HhZ|@Yx3k%J+?v%TxLuyinC%6(7pl&ouKz&1^rEiOy|@u< z8Y~^j-3V$GfLjvKomJ-+hmIB1ju!Rq4%K51z$cd8X(>HG)Nh%eEHa1opRe8>OF81Y z*PHg=&tv4n<2rDpArdn zDk3#{gUNqrW#)s)D_o;c8Q36Pqxdb#loFJk^St3p$FR(KejR@k1Jp}Ij;$td=9LkJ zCfozxNzQYLS^v~ZGL)!~c9By%k@M`*BnPj?*qR+0lk>!2lQoT<^X%8B60c$0#sf9Z z6TvsObqrJiTl_mX(yPg%eK77EsXZ?n#0aOOqn~nj0cGvxqyK9a< zu*6NLS;Ldww0+lk8K+sqWaLbY4Nar%8;Z&JpfLl!dYp_ZHlXr#~ z;FhJ}OQ#x~Tw@1<2)=O3iz0ZC@`dqaWMY85^``9}&Zcb&E5Rmhee4f6Y>!j_!q;SZ zA#Sj}73V)GJc_Nym>?3|0ZeQxCwajgIwoisIH01gq4DQZB3^5+B~Rdq#;b>q8d)=D zT-CU$-x?QCz44;r%1}iU7rfWhtKop;RryWx(}~R~Hdq2i<8d<*fJtr}63R2N!TXfQ z*g!3q=EFcJ7cB9%lW>X|Bf=jDg`4w%ok{BF(6MqX<-9*#~4&KTui$En!xCg%b8*Zq0$8d*`)+&v+z=67? zo9M+`CalhZ;7RMYBL@;H8SVF=mtxpU5nlI+GzWG z!OyuEEmb3hPRXJbAaChGjsA+y=uF(y@_vVqM%y8dxT6c{us^U>E=I!?4Mt1aA6Ssd z;UFRH{=mD;{-=b)UB_2jD8pf=)~~r3E{#g1#D}SgwgYT%*TaqtXuR*^I9evyEXI4+ z7>|TB-d}N(@EeDa#%mF8?RgQ>VFPdLT#Toy*xDLcxb6LI)I;3uZu9(4!fFem=wQ{} zq9`0L7po=dQc6&^yWRb%1DqE7A&%_^!K-4i_l(6zNQ=o}aF5AwRM{5?amk-oqV2dRVJC5?ozLhvwNf7klZpaM-A^2UU#?$55;srXLR|VO|+8 z{Bb7kTHJ8@dp~qkV!yU1nip>0jppTIw=^o15}(~X?>pKNNN4*g{sD~lI#tA2`hhVX z32D63a6{Ak%=&-A*;>aDtal?Fwy^D(i}AuHF}6Aa=Ey%}2tLR>-26vPq=DYw&4EF? zE#?zIrh(??Vz(r1N(oB|B78u<-#q`Nq0?qR$G5^}Tf}Btjm=0%qIp7y=YOs6oDSC} z-p=#7lMdT<+vQ?29!fW+X6(3;HPJ(~(LkMrw^5g4wLTA+{V$1DdOg!u%4#t$T9Av? zlJzMiGW!gg2h998VYXl5AHr;Jh}k|gW+NfZCOMr4IsfG>O%pPw(-4iwahAyGMBpu1 zrx;K$yAZSq6u{Mad=3%1lhf(jr+;NbeWTCBf}^a%;C?R&W^$9iMI`(o=bISZE&NAmD71>jiU&2Iu{2wC5~FZ zNdwKUo(z8)aSCoU(1$C)Kas|m>3Osu3(Ucdie;RpF3#V#oJSDrunzT;Mi zVo$&xl-e=>&qU0S%xR2=P5yI;N@D;)V`+>>O#a)H2HebPj7LU^^@pf526@vAd6a>x z1MXJ1q3I)B|7IFvNOBrOR-PX*`ES)SJI^E|rZH&l?YZ)-@Pj9Hq^^lwY~$58=~kW} z?PsNw3eksPd-}tV*Wv?pVO}Qk=uu*eMAtGgTBgr@_3<*1vOlzInVELK)WGsc&a2Nf zO43d;ewDm6Et^emsaw=Ctv>szJF4$p(SJZgedCATXoIr_!mXCAc8t!|=kCG9(hs4Z zyI42>Y#2wtuMI0qwQlBEz<)XRe^}!M_o=nS!A&*0(k-!1?PT{Al|&g>_Bm!8Vq9l z!sCM-{@kb5BRDhhVwp3yi!W%gmM=W+%zu2oOynazjAC9UB1g?+R|Ph#S%3#$c*4nl z_}r(qDTtvPq%UBfS}Yf{!D+H-Yv4@jQgKf>@xLEMZ?ihU#s~EX8oXULK4e>QdDLF< zZiP=G{2CR!-GAaLPWJ2cH$E78J}H7LVq zVW?X4n8zVTtHNjjS4;kngZFmc`0(UQ7J{Prl=9;-)v$fCznX8aFS zv@nmHI1J-Tv+?06Gya>>fSWm5cx2SW3ut# zsb_@MP-g(Zq94swV9n*x$ z@`+gFX!IltPMCI58y}KaI@A>Yw{3!;nv+FUlm8#;HQ*a=O!%M$PBX^ec*{Wx^_GK7 z;(dthFN}iejnIZD%E@fbQ9evN;Gsq~w~j;zf`d1b6- zO}GcXld1G`CjYOsQ{oNI*K6m?hBFP>hDK5gZ!(vD-t2$ZhU8qjT`*5J zqGNx=yj-d~r>K*PSaazWogAc$IxofhKx)*xeK8K-l3Od&K-q$29A3fxpS}9fzqkododx9hu4iDMwk60NI6=NAxTp6`g zY+Va+-thfyVs~PxxRU!Hsp66}r zyieX;3%YwHJ<9Z}<9c}AO9~V6_1ohLh06iL$+Uk3poCG|zTM@ikiER_`3Z8by|?3> zS+&z^b|_PpzJELId@q*~oDp;W|Im4`wO3!{*xB>z-s35|eNjA%4vv zW+*$HMQ9RUEGBYRl{iJ`qs+AZMElm>ehE2)jglnnfg8qO#r-cFj+F6BNIZS0S2K`u zntDUU@{~-1zCv_LNCeB5w1jepE2rsHu%lN*yO;IgUo>`TyTOIRK)hzvvXDsws=XB? zXYv}Qc#_ww`ZXk{*O0t%uNkCuU)?{XGhVYoc}+LvHR3d7{D=681pzF`rb<|u^)KZ% z#jN0|FuZd6J-z)BZZp_x=e3>PFl$Qf)G2?doike}6G;-c~CR24aE&HjI_>X_{0>;^^A zkk+1O%Tc}9+C(qbl)ad5u#v=&J>`(G8X6^&?qMzGKf2k%ON!X{txT?HX{;~qZWk@7 zg@jo_)TVk{DSj?#F_`gzjHfF&!gGJp~vbxHuJ%oX%yRM@nkLo6vK;2l9x+Ev5sJqVOKb0et zP~AVHNV;aU?jMdM`erzEWbWj};ns2d4{NdHjO>UHhN5t|^=AD?D|x;Ut4u?04qBbP zj&;+g&6zR{L)DPz>?yUq>ZZ?~HfPq!LxxYQpHk~q*p6UI#;g@GKgQ8)KA6ui!$Jv$ z1!Z*7u)J7pJQpy8-+1dG^}EcLVgA-kw*EH5%mq9#G8pXo{U(+zA$DiOo_A478X z{XRsj3$W)pj2LFgxJ5eB48jdm8%+G0h!|dgzalIt=w?L>{70hG@0JJ|$VR%g4cPyh z09(vw2nm47MZ7L$qe*jD_uh#t-XDd zbS1H+>qe~qShk|BQ&A*cbK1DxE?T$-x>iPHGkt3>k|c*a7c4n1nfae;5rxYjN9~Fr zP+l~h;CX3?EvJ}r>IZ~!1|;MRqA{4|B4H2QFz8Ev35nr;inzhleN$9inTl3PM1rA0 zbX!QoxFZ7Ps#W@Iw}Q4_GaYuQkd^<`E*DnJApToeNeg<(secvzWNJYQ0=?X7L7Niz z*@Ea3S+$@|a~%``P^_|AP#8V21<92_0TtMSs&K=Ko1FMhMNcOTkuG3QZ%D!|2Q?Nw z5f>Rsy}rC-s9%FbPa)IR-jJkvC9&l{FFWIZ@wEUqmqpU}&&#LUa#Zw`gB+9p(s72D zPL6alFTdl+nF(D&IXk&V3Wsj0Y(-D6IOD&Ho|V)lP%RAX#a zGt1cx%EB35G5entvFg#+Cwg>5PqXxpLu@1wvdw({EgW)`8#hWOo#$rezeBPjdhzPy zik7M2|3}?hz)5v1Z=mO_vatA~L1xhqY+a&fIgmgECqiUdb{9zq4hc6#-2KMoLfjML z9^yCdx$zKpAs%Ay`?~AQ>7FyQ%q0BZ@4d-yGQ&<+)mPQk-PP4{-s1c>UNoN-5buzN zI+dTdmQM+@xpV&F(FQoC>C{CjAJR;4DVPEKXmu+$8n#74t<`&yGYZzT{%2CEv7 zz!Bbl)hbHF^;J%N36<;ZRGEme0ZsxM;w?r2@({f5^fCD;ym3CAoT>!8BsTEAW5s_8 zZ>j<79V`CJqg~?uk-8=&%6i9&|I|Pdj{piVkf_U;uZ%0$_O6j`Wu zA2+g4?xjHJ?4PxD2|gG*k}JI8-7mp~S*SyjWTATMGV3}Bs6*VoG9`M!9iS3+C}1wh ztlLsB9g>Ji#I0lfQ{16SoLcE34+SSEbC{GlED59-$^!PfO9j%Q&c4W#l)z3qIxh9f z&+X!5>p`7m0~KD9@`uG$coCy(CzRlerNpf3{y<5Z^{_-0gw=b2{~yW)Du`zDfkVlQ zrMRCAAm1As2f0W%EqZ?t^nXWFZ-_{#_lL!C-q2ft0fl-QdVjcs=zVkwy+52IdLNraZ&_;o@FWp+ zEcMP^3%y4>E~h8Cxt$$17hhOXRQNE6|Bj^!D4kk`5955Hg5o;{rTH>c_^7q0aBK<{ zJ{l@29G^r5X?7p6{u$^Tuc~epR9Nn+c9iE1h&H>A4viE)z;WrH`MDTxB3;`@JpZrR zv)WdnUU?4foj>9J7wUX+YMnm``oEKv z&NDWm^Cv<48y0U-=TFWSolmB3=gmm1^C!Xl?_{Mjfh5uSlaEE`lT+yYDf@rwd}J^mc%W(8hKM@0*n!*5|wGsoIp2Pq#FoOX;8z!JmSB;yt-obG> zK;`EaC26gn?G-Q2);a?PQz-FS5dWU8S}Qq^MQOeaB|f`OlsG+=5}g0B7SBwggbXg9 ztrt*ds#;tLQ7sQHpZALpY-Dixd}^G6mA4XFq|p2Gp#M8lRUW~BLcI*VKfet8*kaF2 zq4(z;|ETv_N%WR%=jWdZsI!#bF^Jk#S6+Kmp5%(uPzmv*5l{Rg82`>vO)-XFQYiC9 zu>N_LQic+CLutMYWxiM>%AA!#nJ>-|WzJX1Xa@X09ZlvPU)(OrobNPaM4NP%Z@WBHk)L5W}s(scPn19tj&Ya-`nKK~g z%c$3{#)-145|tG_zS>vxxH3t-%Cz*WQ$*C2T#PUH7FMmzVq&>(PAeyCU@Sfv9m0Qy z+4TV^LKb59ubBU(;jc`@B8LC!C&Tbr3qk4pHhJ1d_%ICr^=1P2+9ZHuC=1(s9jt#| z%iV#>Hsiaxo;Kq$l;JCex3s3O7l?w_#;slqduFj1UMwX#e0`qiaBZRvV$ZJ`|Ish5 zNdoV7Fe1EnMBFvp6rtcvV4cyV7vywx@aO>6c|Fp>pQbkfhSyzxg0e-V7tKy^m*mM+rQFC1rcB z#m^A_ZP5Q+!&?s&^@O0e=p*8gIO>wT;~*2M&~NGgbjoWID^#5F+Y=3^bdD#yUE432 zf)B$fzr6`uSnIbXsr3q!CFGYy;oI1XaGw@3KyE&M>y?x5xS_SOoP3 zAznA>S2VA%!!9mnmAD6}|1VbM-JKS9*rF%?j#A6!?>B$J0q`?iN;q-JipDwnH_RU~ zZ$V=N?4+K*z`hrlzUwL$)R>r$1EzYHM(XS2XD0>XDxtJ9xrpz6H^bsoXL2fX@(tgG z9zafVlFHf3@BR}fXI0WTx+=$@Dhu^OiK1)Joug8=f2eo=fdgSenE}%%MqDzhy2T zO3*i_O+r79f2Zu{zrRla-hgd{FesYwDY!aBmfTbUk|nbIGF!F7zey{t9A-- zy9B`acS^uNa{fz~crXRvA3eju*~dVDv*n?kifzC@a{Y%c@o*BrF_cA<`|)_O@WUKQ z^BjzcD+abOlKyyCy!--Hp2?Ay)Ia?ox;#cx=daqBsec|2FAr0rd`eUQJRT)!>cjua7($D8Ul`E3EAZ%$jQsSe@4!={=L^#1ZB*wFTmC)!@(i(kGlnrhoe zA;7u)uKS1K!|1(#<@yH!Kam8mI21d>@{tquBPxxMAaRI6eKwy<9d9oU9{#mmDk z9QRXoU>kz@|5J%}sT0vb{#!dgMK9`ypJw)WdM7EdZRcWbN- zvsf=@Yg@5^B{YLv#X>auy~<>xG>fw-a<-0WMskvhJN?f6FLbA;Q@hjeod3~dpH}0c z2kl>NX*~S>iFo-17@#~zOEDh)&in^Eil-CY$=w`8F#I1y!b;T}SY^@dZyPbw;K$>W!PfaM{xB)z)Rzm5&epKi8T4e?9*ksaLhNbF_{*Wll*t zkS8o#YG4jjMs%laCSLL9&X(OKnPX5(LpO&qnEg-8|5T0Z=Hi0=T6BW3?4`)~j7%sxXF1)y>L;AMA zxc(6z3e;Be_=vt)Vx*wx6yCi{>H#Ikc}gHX7thotw{?#5BTCZN&n4RWXf4{c`u{wNR-16?e*5A~MjOi$ zbEdw2(Y^LxP|it`-Vt*KUaWc#*=G`S@L11X>c-dYfaesd>+6b(!GwTx^U86rHBFvZ z3iobt3};kB(t!MPlU!|+;pS77GcLmB(@+83HUe^;63EFKVv)Zm($%vq`#QyHft;)*7N_;;>LIR^bu-A>NMB6tI$5uZh!^QP`D`oWT5_H2ym+q7k7jaQWYqT7wn%}0yC0czLa5AKd0KRbDOb*jbtRXkb9=dOl`W0Gb?hc zjfSe5S3e;gD_CAM$(5S%uyJ@KY(!E}w*N(RBzW@NO7qRz!s1IE8?f2CjKyM3jjf}} zR>DWJ5e@Sg|HU0xX&7f!@TDp1quD1ZSda2nx;RaI`XjoLATb|H>|=pXa3ix zUWpGDkbWd@1GvyzUrpkz;xhSt1n8@rnyU0;j!IFMhv~T<}^bI zjvLuQGPf6!x$&UNOok&@$|)IH496QrZ57nNX2VQwoJ}WWa^q|qgl3AEgj^0J@oX`q zCXRtP7yMm14__=MhG}(%80NJUhG}(+VHj5pHTj3WX3yhuS~35}+I>SAh8w%|i?u%l zZ-V+=K)u0LLsh$Zu0q}c;Nq3Y$`m4jSWqbt-$;ZgzE!ZRNdGQLwNz^91;PCHJLOwk zrO>|B+g_!}=0D@c%>pqQ1V~u_Yzp5cBB7H&$AWdHv0ld?opJt;hvCEUqSmx6h`xPRnq_iyv$_^)z|2mNHIf%=z_&D`1{5dEQm~^ zlh%(z1NJBy9eo@tr22^keVtZD=VYUUXgZJ)owVs_aiW{YITc}%%hE{5*{$u3H|*VJ zVw^naFw z$p8PIr1qr`DGc)ezsGZQ+ZMD3pT0Tm6LNI;cd8s+PGOM$|2+lZLNov0dL0Bf&iAoy z_`eC5{Xg6Gh9rP@M_JTEVUYjd5YJa1G)UZa8)uN=5Ze}KoUW#fQZ1*gC-tr6bcDMs z@I0q}OZIG-w0efaY^$PSpismdS^>1};U&l1z{6|#n?Rg8W4m0 zf0kZ3#yRT$rSuBUh^9U=8bh_?T7Z9;d)XGwOnXYDv=8$C+7BmL)!m{W3WE*>W~HdJ zL;v&ep!UF#ml}?YxHi_v-4#fFemYDW_wQ|NoO24wOzg8S3~y@MT=^XJSVSLRzWgFQ#@%@ZMeV6Pw%A zgq<;PSY$Zv(|!wEZRALF*k;Y<3%(RR5^sp_#QYD(YgI`FH11XdoqBb7eUURT;%kIP z(t!MSi~R6($tQ|*=6^!zrwh5tWmGx?(KDv4qHYR=N8aK(ou^s+SA+i|4hl-Wq^?5- zBo^1{d^Q+hgd652ah({-!GxXf)?X5=`E)?19%n2cNUJgJ#3%>S@vvXj(|H0mx>wFs19%}j)O4j$yjfFoI>?jrkt zP@45ZH%d`2U9JOP)=PF`y+{hvXHY`DV6PePGmvJDBIF{SXxDBQ zrv_MQspreX=g^7zb`_`WM7sw0Z{5xxwD_PU_>q<#j4Gm1*C79+8BnD|2TL5+*BNZX zz(Z!#r3_jnMqs8&;{Vd+P^D$wXs3Q#mV-*8ZC+H#RpSE0DYLMBqTO{XvrI#kLOxJw zbVMaGATe7I88H9JzIj7pHOic=+rN!Do3jd3PWjNUO7LL>!QHt3gN~S+#1SQZ(d}&w z6=j0SNP(J+l!ucMGktL{(-)YgMA8?hM$#AEYqSzzozO>!65aPOmH2^5?6c;G+ifMf zA1X@ZCs9Jeg6`b^i_T_PzSE4RQ3`wK22;CUnA*i7#4rPa%!HoTk%go$x_@S|sWaJh z;^xJmKXC$LBYn|>>)#=U7>HY=yi3>N%Sd1JVE&hlG(Ux5dMq{!qth3I58A2?xEQtD z;~a2d?G`A*bWl?GKXyZszUXm}fGUWmFWR_@VFwr}UWw84MUNi@VnHHA>EwEL5$U~> zR7)igz>9mPJK50tFKdFKr+4p9YjM3Gkvky^A;z%d7M*`Q(*K>&`8J` zR7v`xSNk}5&||b!BF5wtq)J~5?V`-YJm=6ZNM95rO25^nq@AVM4u&E48Qv1>o_WD@V8W`^g+bOlmL$!>KaOg;OgituKBOKZ# z(h>Hq6D0=vsw1R_7#(5n@5C0G4Lu~zrV~6Q&PH{FBsMz2J|it+c7*LwBw0t;XSx_>a0566Gj?hs{|q#a=&wtx1{L*gA_2Uk^Zhg_o)qa9(NKLp~C zM2ON6_GSK`(kqixO9hakBkVg$q*p56>Hw1@-EH0`*>3Z>!Xs$^mF%5*uLY0z9`Nx_ zeqPuQxvJGJ2wU7G)>k~1$39SyY@;`Y%0#N~Uvcmh{DKy&H+tLKv$f!SAr;)QZ7p)* z&PL~~j?s*FS{b;}@u0WaWVmB@Sb~Wgf_1*v&eCeZ95JYJfcqzloVd~4QEw7*_<8E!>&e^oM`$_&64PmzU2?WwzbCH!NA>m)8Za*J7Uap*>KKvS{u_z8~Q)iLN&t!)c+TUb$MrTlFf$6LO;Tn%-WC* zmI|d%p;a)X_r>$7LzCMoV3_0RPA+20Ek;|MIHQPjvV#Lx0Vg?0WzM;Uoc};o1ar<# zO2n9))+y(l4~5D<1V?V2e9n1iWX`$8x4_D#7J(~zvYsQ)>N#>jOa{^dhyF+4rnvYkXXsi3ZWduW zvpZJ}!G{U=BY`_PNk?}I$^tGjO&acGwm{|4oe>SGP=1Ni#_(uFN`@N;U!4@~vN);7 z4$H&eGf0pOyV62OXDtZgxV}ys!&z-~a8`zNhO-Wn{QoemC!Lj?B$72FLTDu9hO-W5 z{-3ovIqs}QSrjn_CpS~k*rid@(}b4iLFLz_Zm>&Za6#A$09eoq z42VJg|KE3MjBj4pFfRrp{_|ZL+aV?x9ac&y_dl`s(EdIteLs|1ExplXpmu4z<}H(J z=uxZ73`a&?7>*n;;P@9upzeB2$-r(2l7%cQH5ug1Fk3>e#FmhHmqyuv7Mn$uC9oQo z;P|Wy)gLQ2H+E^1Vf-`nDYp2uA=wGm>UZW75&ak}M&zC92wk>)-K z@t~kpm<vkQB&ZnjCvS1KioGrVtHEDAG(oACv~mcL#Jm9o04pxG z7;vM(_@HvCoo+_w2=k=g1(3MUfZRELK>}he(*z6$;uaiXZ8~aEf>aHeO4U?i{nN1R z8k2ELHOqwo;*$Ljx~EWk2oBp^M#etQ=CR)NYXx#YE%@L|N~)klL1L!~{Egi1T1EU;EzElTeZ4?Mf7Sg<^d z1&!(bure-T#e=_amOrc=r~LZ#iYQT&BTDR%LW!DzrV>AZ+W9ip`5k;1O4M-wAC=fM zi4xT)3nglT@o!J3S=RDf2Vs0Kgz@n>BHTf&marILlf`ghbDP*gv#CS9k&TMsJ0YtZ zX9M@r695u&k)9ZW^)F2v193O-k6-=8Vq%!u-eQziDxAgZ|GDOxzv1;E|`=Pr4YmwO}@W z!@sv_vdHP*ovC`L1MNUrxw-yXg3OC0e47wiI ze;B;YOfkIIVf-J5vU*wO1blP44}Iiee6WSHcLJ9>?*CxiGduP0UKgx?&sM`b&-%4W zmJ80ehIb~`D)AP$bzJ`;+}Ww%V*C%`o(W2)vTVjee3)>V|0LXflfc~;WueljE-1r} zc3;9(fkZc)i{oWF9qm=5VOJoZ+0k4@8g_o-D$*#%Kg7Ln3f!YG{>O3aRip!tj;*qB zk7E2w-1{fN-5|JS|1-+W2JUq*b9_6>_D-vua4wHHBpjJ-Py1RW&c%VO3dFHhP}7KkqeDfxsjN^K1|F@QyjzfAG-AZDO`HYZiY)c{{{?a z>jPV}Ng7RY%)#J7mtLI2rI9(LhmBb!YAud8#qO#p4vsWM?B3?zj1}EsSE#l~RsryD z1o+|u3*?ArjU9)gY!b^-c-C0t zf8xEoZW8}`<+j(s#_+7Mr-BQc#EK-IB~4;1&i^arSHzn{Pc@hhPTb=#>=`r(?0Ybp z#L$}(o5Z**0lp#y@NxYO;JQida&*sMY~bU@3h;xI0GB2)Zk_-?INl^u3{URg7@qXw zNUbI@E{OjRjyDNhY@jzeSXVYE2hE2@M&0iitp8|YHi^iU4bmib4Ceo;sA>{zgRKO; zu$3V0V>$6}N@8nBG@8VYLHw)Put|`O+N{?Jn+&7GCfuO^*TifRn}NUULIz)K=+Y#1 z4CcRDGd78CX`5xIVEscAucaJV&XOZdVyC^tvkp$-Svz6;k2i_zT?zBEhweK8A4Zed z=@Rj*mCCa^u#|c+{*2*`?rW#V#Isg%2eb-&^0HJPm+!X9;rg$X4a)x%DW9)zzQ7g* zv%F}s^B~b=Wuhh$1n<0qfIK1z$RQf!&ijebBjVe%du6LAphGT(Xrc^WARtjdcv*z7 zca~b$?|@5_TOthnKa|+VRa;9sVE1Y=LeUqe8FgoNL<)U(;r<`?Q^zK0?yNji$}T+r z>!>o{VFbk2Ed0P13rxV9g;t7g$FhkcU!Lm@^!|-pHn=m>?EO0nPf{5$3)eu^G&eR5 zFDb2oCwN*82$icP;jHxM%YeT>7Ol0KS8# z)?$;;*q!UA?mm9p==yQvM~~T^($tOiU<~$6Wy5Ed*UTDT(b&{LX$>U}6{Tg3C80D* z8n2~|59YsGnMt@}^M4*5NURcS&;<`92KTbsSxptqjn!%CLe9N}dvN|ibB{Tlv*%D> zwL26=R5_`@@!0>TMx$dBD^MzDJobN$1C5^A@tbquzQqZn(S+XM!dgBqbuCZe`486e zagMHKS1qA{A0>&j37G$d(uyf<&iI2*lLCTy!ij?UxD=QtTxW36WANk^rH=_0X^9hF z5X>he!7Q=zgdb6c;ll~a=~g3UewFvDJh->c@qW!_JvGlNDX(ar*>XL#b=pkz>qpn& z#=5#GW8i{?qkaV-3fdfRc=*JbmasKWc5BCf79RcxJ-iLRBlRB8kw&U};`tWcX-;85 z7xGqniBU|I6E~k2^nam3S9`z7<#wYS@3-_7Lhc(1O{|+db==f(6UHlKzd;%$iAIwu zEM(Nk^`D1EkBLUli$N*N#)NWlART@plC#*IpSpN$(y2jfe4}I9y z@iwHdQER6aG*VXzDs47E0;PyHlNtYn&OmLl{O6#}qoU1oq7CT`Z6gJQpY<`b#tXvm9t8kXV$jZ&1E}z zOuCY>tn*Dd-2~z;g1{{+ocbPwU*c3K)(s`-KbvQkr^A|T8T%0Tmlo@Z*l;zO(#e`H z7TX$c6rI{FUUVM*@-2p?L5|1HxPJ~l^0ByxR}4j2o`2CpxB3J5k&l>bD)T>a2g~XR zd6PZdhM-Ifb9tGVE4!O84w$dCm=_1k`w>K(d71``GZ%wRWB(r_#{|pSjq*){$Xnvb9q;gLu2)T& zv~y`?Nm)s4S!rc8+^@PMgTY+Oq&(4R*C75^J&;=r<-g*?q(@SF4CafKMrjQarbg1c z?;7m?(fu18Y%sNk<@Af?Wc1vX@t>oHD-5xi*;|2B05^#LHN#Rk_Z)AW3;}?PX&%P3 zis}Z$Q8O!BY&ndhdO*BU0lNkJ|1|eH^w}M~1DyH+go_F|;Gb83mT6SLZWox9?o2{o zbgp-`+_(shuFj+aLXD&XcKg-h&S+{bwazvxK!(@dgY_@1{UW%+6?oj3>b-qjILasz zA!NNjVV8*$#*Z04bxH<~m+f5#8&Z;l#JitqAv@f0v!-}|p$i{HXi)sGOcdAb$yyZO zJy`$O%FJ-Gv-%N&<$2;%dj$E9z^Tv;dS}Qe3?yeLB#jL-OVQKUY}Bc`0|hxR#~ZDB z59a^keTGwq3hw3|<5ihhyK`Nj zqgZ?T3`oIN)aI43f)XF^epc2YnX?db`llT^5E&(4cc7WKSm z6UwljPvXSa<+wY=!4=bpm2$h{@>sm^orQY}L9qUL60d{-f#VDgBN0`K)MWiYob@7M zJr%$3#S&t(JwHMz+U%r6n~4^C<%t%jB+;TD7(vutTZpJrs0F67+0O2c%i*vfw_CD@ z4Cm~{{0H?ug=(nDFYcfTD&ACMuVdmRp@!nk3ggRAV=v5qlp3cbsv$MD7uJ6grdfl2 ztbYJpSU`-i7u!FL5qbWraM`=F$wf^K_q@mbO1Ox{_O1aJI?!oJrda_a>Q8?@O*zmy z+kt{Yhnv#q1kSZZV;OWAnD^%RPt2#K(0uR54K8`T$z}U1*E}U$0?ds6i1~~pn5C=V z2l;mWtuKzMrdQwz+SyUpOp;GAg!c1hZC zs7}8B0@Qh2)OkVFA)TR4LlFOk4%cfG%BTaYfjV>W1+~*K#gefIW8D|QCdXa4$+4%? zosrVcW2W+||GytMT@QJbDFb=KQX4$cv=*I6(?-K4j*8^G9B(2C z)_-(eKri=Q?_)`7B5e9NeRs(uY$6IG5FHz9EghNbe9fCDn|+|;yiBa!xh~LAtlfBy z#hqc(ek`N5xB`T#6&XPqA4XXYGiRs_Bz{ywEk2DAq>=X@aRfOd#R$?=V2mJYcf51h z7F}NuE;52NjQ|&pAo>B*_)%W`UEnlL7C2`L94l-J3LRidD<+p;r>qFp$7Q5$x==7j zp8FhdxzFIzrY*T#^*|p+peztzX8%vjXC)c+rlTBOs+sW*yM?o4)GKhe_0Ew<9Qmnp zluK3D%&J11U)rL>GTVkGJ$AyBso-2V>Ssux6m}+*w9t~o4ycv?G&-P%*fucYJuMv& z>5LAj`6`Q!{b_IVR*PdOsvKf>d{RcR|1H#JJI-1LQ^eMowWO~N?zy5il50a4jF>O3 z%dD*}Ow^^oe;m5pC%QZ?x{%J$W!6%Qj=J<2#WE!0i;X}sAG20jWmjY3?RjT8^_#OS zCf;YIuM09H)Me+oDO2mmjzNNV)R;P5M?Z=@siWCNmQ3o^+J6#ytrfkV6un4i=rwz1 zi;jBr9fdOSke7PxjW_COc98!F>!^o0iwhq=7{)nJuYmJ~<&JP%07Sq7_pM_b%^O6v_(#Whg4y~WC;X}YS+@}z-0 z(P+O>mQ3bJzxEc%u6Ag&NNJST*ugw$ppn>iKfC{9o^*3n#if{b(Du(v zw9WR&ot#D6`Ry(4jPeJlGN!u;lKER%WlJ$EyymTwbsZpCr;wCXG?ZZ#xoo4J)K!oy z$0wY5{&^O6M)~W!kHX~WN9lLKnMCf2hZt(W@JY}}pe0v*NL z3l6ck)3^3lWlU>J<6dwz%7%@5t=+gKVqU=hk5T?vDWd!ZnE!?uGL_q!eATT-L1Bt1 z!W*Vv$oPj5^VvxvW`VPi`7gpbTjjQ}hbjlee6c`KXm3+mF@NoILEbtkAea|&{3GVG zQ(#_rsmVoQer0w;fp7^h|5q@d(*kCW|HOQb!n`&q=0zP%X$tce59hbHF)ykT%;%)Q zyl6LrOZLT}c3zpX>s%Z2qJstVc}Xx!ShXnV|Ic%pLA@Gj;C(nVQF#=EReEO9P*YY@ z(%gs*J1uMvLRdAPan<-SyQ{F*Cn_U1vbYW4uxHVt7>KmCj)6#jmQlyLD1+WE>MUMhm1S<^OYBBfCwn}nV>VJhZHP~$MKJ3~zv*UsH;5Q% zx{KeiM6sjX6<%FmL2Zx@aaH$}mgSfZA$M|?rn`jx9~z4t8<)uHim;Gnb?-He5Z3%xx}Bvj zqi)#F&r=x)tjU;hkaRmsx&Fy+=e!h6cp28e+1VI7#QH7y^i%wWVx-$ywjH>zt2{qR z6Banj8U@bz0w-Blx$HPoS}`Ozzm48C=;Q?Rvg-x&`Kd6Y|BqvC&y=jQ=Y7BJgAWt) z_k#JtB$#KQEL?VZH`|O4{Wqe`D zV4C?SPJ+6y>{gDqfBHJ%dIId496btb3w;iJ6d|e{!2DNeI;zsae;ukkC#t+Is*ui5 z<$&P)OQ^~~@>Zt@QBJznDkY_F({}Kq^tZwFrW}o z=D;BS(SoQS_@bRmwEt6UrQTq*a#n@(VYb zsK>GFfQ=^UE7RHO2xX!f9-j?~;sUjPvP4p?PX0Sk>t#{v9Z`#PhFU9_{|xJvY7HiD z1rB{KA}>U=(Hj@{}NVJ zXa7~`^t9;os_4Xj8an+e%fd;WHYabjY=EMsPXDs?r%oHZ{bY9Fj{&lg1jrCOI{|`DnBRc(0bRwOp)2kL8bsFNMjN12%qDm)OA3Vs`pE~{KEtJi$ zSRY)NzD`&lr2Da+kd4#HE1$b+1@sl=Whu6NKYR#~d_J5v(sSdO z`RQwf1S9GwzJ%-NLXweSJmdpQ61C~>zX5HY7j51UZAfQmb13(}*d8~Wyw$Pr1Z@u8 z!758_{^Dww%1R(zIxk~wf)!jp+t%jLApa41TMz#=X!ERS^O|TwI#Zj^EQG9^5#+6= z9)UJ8ydGxPKeTZjZ>c=L3&ZQu^tI8$Ymoc(+dz^rydDr*Y$ZWP3xr zQ2|HnWN~Niw=;8P0|^(&_XGcV1$4%Z3|0jkVaNYmg@2vStu_M1ojc76(A=q!RKStJ z`j>8=oOCWA0_zBh^yfzg^Iu(-nT@a7#q%`nvK?GPH+I~vV^nyMSqAttC+@ zsj>kFAsRQnJ(gA{63J^Eb%Dj58Q}N4Ph?>Qd5urf?}anTYY+tyh_#On^8eb}^sqNL zYg9Xbprc+oOWVT&9mT_rKFrdQ9=5O!zgfplq=_GWEy{*XJZ3j>Nf8{)^Dj(mT$mz7 zaP)d3MWFY2Jhsb0CqQ8%MQ}`4aA8{Gq9iE-fpg530_P%?)+kMyO+BXGlvWHW&cl;# zT_Vy1^D*52Ma&naz_f{i$$~RLn8#|B3nH7BGJ;m@ihCV@WX|+u4++ zFyA;}H}t5=LW22N=KqNK;uM&V-NWFLeE~6_yL`tpZOq4V{3qsBNia(=a_l82!%<+B z(+uj>80+5R^jJ-$5sc{RabtBYmRTBGUa&(jGJfYU|H}=2vkeqgz3E=~#2|fKkpILl$~;G8&9;H(xn$Qt5NrWL@X|A?q~Qo%Dxtspi_6(#GbdsuowBu%P2l zvJMFQ?E#2_*oUA`IO#}B82g=G{)gyy-jIIhL+N)&XY@NK-DA;50 zCkN*rw7scHnU69my%DG)$(@sLwq#J18+bfQP0NuA`73=@xI=%Ey`9}>`!G@=r^GC2 z)Tob-&9>fKqS42q5$Oz#PTAd}qekWAtsa3LXe5h*rv&@ILvOp09koiEY=@3IeT}%C zeNz48Y2)kbroguf{jQKjG2&~dVEx;0Y^u}O{{rf~C+d75>X6P*=hVR#9o4BI?*Sy^ zi~W?$1Wuh|m1PCp#Q0og>b6CEp1wNB>_8p0vrlQmF2%^~oO+)niQ2^c&!Npa(dKi} zhIFPjKU#FurqV~5{z4^Ul zJguLP{VU#^qRmI54e1PRPCwnEW8GByD5ElID5^Y7Lc`PT`lmLxpljDl4BKFdJAG{s z8p6}!Dc;TyWQ2xi46;D6YWn-1LYa3&nNLL-(izH}v5!SZWomqsDK@J{`n5BH^{>#; zZmr9bJk=asiTH|EIF8i_9Amn#YIdSdUl1S(-?4y-C7ZGH?G5_J3;b%)AZpjtmb+ zADJ}qQ=3`ljWUhQ+rRI$wCAqNZ@56M&Z)t+dbdGZ-4ZizfktB6v+VeXnYS%iCfl^F z#CK}vb~+_O{#G?kTB*EBa(v=~rv22gBQ zz*&!4+?jbR;toi4kQkY_75}^fv`nJ{&JN~3y16pmzW~Rm^%*p1be+;D%?b!Lk_tHc z97`j{`-`cyb--^1tbq9OY`g#GobsF2avS^@-agjJ>A>?=bv`e2sh>1$%(O8XZ7f;? zIg~X)%R8ss!p3>!9(B3sI?(m1okh8ssGjYSJz0zD=bT}&=frYzma$So;#TL}ZFk6(H)hrr$SOySzcYy+|=?#K6I-XP>?gHbujF?y)5pWSoZV|lY}lNmWQR^BWE(P zBnl!B9nTH&AEBdNh%Kq{dnm5W#M+^bV(s%BOGi4|k~$!;4)#e)KW`YyhAn-D-O@V( zGK_ZKBoyMr^3oKOwDT4lleDbYLFN4S_OgJNWob{_qoleEi{Ows~4oc|EcWzz9j z6U%_nI$?CVV#HuWzxaC$Z$Lgj2W2=GUheFRDJVx0EZM5@CJ!H3iMPj{UlA$)j_Z2B z0uRBnwBqH9g@03|{JRcT9`I{^EoW4B38N}4+UuME0b7hAQ-_)7i=M*uB4UJl{iDOeliK)+u%bk zz{$lwNe~br|5-=VhD5_)+KHNIVhW;vM~H^QzU(6)X+i?gApDu|o`?{QfRWjV3VEu) zpdy7>`hvgWRIoIZa-1!~6qae}3x~u@!qQ6pk@)1xu=Ir!P>PnmGSSlFv=`$1M?#|G z-&64X6Dv7$A?u%9E=$D)>z^SPesfO#X~k^eB9(d}^M9<=t5dh33#mWb(ABC9%}AOM zzG#3cjehDZnpfhBG->x2G5atZg1dpAF~7VxSpU+J z7$gt$zr;xQzKep$2!DwoYO8 z=5CmK)ngWSj^ayc2kW*0>48=;|H&qQt!nZslaAu6yP*(U^R+2j^XeL-HRs#~MV$+N zJFgNSMr&TZH@L6|x-Lm;7C5Wv|Acd$z)3cWuf9SUU7uup8VsSR=ha;QVz+R;8lTwc zwi@WFrr8!B1;O2_xhPH&A-lwlkZnD{Wo`?*Ui_{p$)JCNv~BvkOd^r2OG>f@5wdOj zxtu_gtSBWJ^|y7~uGnQJfn=RilD!=v^I|S@LnP~zitJLJf3c<2^>tNd6l5J!k_G!8 z!U`DG$7Sw@(mJFhJ0c>@@9nB2Imp_lB)cm@HoBL~q%oznOG)-^glx=aE|=>_);1N{ zWkLQk)NT8ouG%dOvci;PLH;Ks+o6Z6c0qxxO-iyL{tL;*c6WX3>n6z7DaoFW=r*pK z%l)||D@a9ld3JIm7S_O8z%64Kn1WSsxlwKTb%%h-)% zxVkEM;SWZycqRLPn`}y3m-q32Y}8*V$%6Htu$HD4y1e&>WPhe4J31n5S{qkgdIYjR zQj*;pA=|aJ%WG*U?e~;q-$%%H!(sj3G}&*d$gT?JKcQ~Bw{m&oE2V8nN!AdNwnx6p zi+oA;Yf7?nBV^O_T;9k^vR_h?Jr^O{GuKs@aDeRRRAg6k{%2SBUOBEhsSmQBQj%>G zA=?|l+m9OLk15HPM#%Qba(S@=b^9SD8TbF#AaRvHFEAk4_YpD%l8kx{s(-Yzt0$rCF0?FL*ny~$I?syvBHDSc1=8n>xT!Z}&@$MvV7799N zzIOk;_+SBs8o+tY332r}!DRf>fKR?yB0zPMpt?R0mCX9Cc>x6&bKRIERO$~#6!oKs zx>1EnjLx^ncU25ufDhdzpeTzA$+eqDNQNTWu?8hTvL-3Xo)MB^xvttr0unrjg0BQA zr$k7G=eTN%9!PFXO7dWYWCYR@H)votBqjMFLNXFbiR(4V^~p%C3&#J@VkJmPa6|yG z>nxH1_#=#b-SmhQ>=9Fk$y?{*BTmEe;!@Xz`qQOm;5{Ntx|`{U>w^6MjZFNYWOh~= zKKU}d<+^u(fj)d=3U9fN^M88Fnk3#LnU(ANiy~{3w{UW?tk6}{EPk}oim0k2ML-p- z|Akb`+qf!o531r+RGj~_jfYeRw07%a;0vl)Dyks=A5tAy;HrH>pz527>bHoj6|G$L z1OZU>NkMf(X@u%u`L5cR3##6!sDky65aB_2E_XUouU@IBE{e!HIM-F@UqQ85Dyo+v zREOla>V_>)^-Mu^V?l)KP;6)Ip%L~-MYUao>aZ+VU4jZ(-BVF5i%_j}U3HfRsJbOm zN#Jy2(Eo=vMAkyx7tl5rA0`<|q)puz^#5xdwy7B2+lFU2%rvE)Y=!rroIJ;kVN1?S_=midusmEiG9K@#&NTdo z7nzA|Zf5&u?cSNhHq2%M)y>mThP~LGw2kVV%e`B$uz**fE{lq~Hy;%*9`yPn9XwH; z#ql!Kz4-=F_s&Fhk3&I94&pzhu995g-Gbk!@4Os*&5bc@SU$6SR#RDJi`mgwa9v9b zVBD#SE}bW%sN$0$kC>+74P3Vb^?$q5jK3@b$E#vkf5&&H){XtMn_(J6##3U(+nxPI z#?_QTJ8mBXifpLQ|6lu0uA-qZwo@o|>(+6B@GS?OQP!Sy`Lau}!l}#8%D~ndp5`sC zcI$=UPglDuiK`V0hP8s>F6C;B8fW_5UC&vI-{a24!s2YVu8$MY+1R#faW*4~v)#5C zO0uK7E792w0TJ}S&DNi?8~yJ1jn3APgRcb{7$APt9;*rsKIZhmZs=}K-l%aq;Djvx z#B`T>wJPQdUkOAu(cSih=%qb>HB!xc@{$RaamGax5R0a!QHySWY~Y= zkF8?^jK5skD=-*$0a1$$78n|AFAs-T)bmTQkMiv0VkZg5yK9Msi;lMs*L#!!cfxW{CI(E2tB|TH zkz&WYKDWfujzj!!VaJcfj^74$Bq50o(|GR==0Cc^X~)?|3R(e`licdeV8*lt$_?UqouWgOTaJgoXubc2(RfLRHd<@zhSfhP9gO&jdt%;OBjtd)c+1f`&5kfU0^g4(r7Z^-+M9$IpE)& zdcePTjWEAk4fvd-99h@j_3Y@f7))irzc-lw+#R3)Y=@yV3nlnsDG8?U4){gC z`1?}4VeI?Q7h~U}(k?vy`TigmR{&y*0zTBLby^qiWwL6`c`oG(h6nmYFnln;g(Vb* z_Y{Vz=Gv0RhMJZi#+3;MPR~pX!T1;2;lr5geW)>fs4fcK z2O1dG4vN^}qrR^9j>hng!caM@s;mT$JZo{oNd_43KtdbCH4zLS_jbLvG={e{hNhCr zlFH_mZ!yRO1J@f2J3Kfdg5i_RT<=|t;a!cPvbM6Op{nIgyqRG5GXX;|{)PVVX;0T% zr!lP47%FDX0*02a*3SR~I%Ufa4^>6%@EImuZ)gl}Xbh$3aw;1)nNND(Bw)BZg5mQH zE*=D++UCa!Ls}0z$ixmDEewBnc-shuFWR{{q^>ahA0ALQF+3E(@MUY)`#@v(K-mGe z`Ie)9X!)dGCU)R`BnE~@wvS-=3acaUYYgve4COPcOYroyP1Qftaac4lJQ>08HP%kv z)EM4W7)qLF)mBvUNwzKAlbL}XydMOEL=caT2L?tE_asITD^L)F+M~h#7d^0YQ2WNi zu@m)_m%SbDvrOW!G=YXAFhR%&|L8AbEe(X%8obF{uhiKTjPMI(gik9^W{&W4wW5zP z{tXkns~rR*S~#l=RTm%gnc4hX3Ep>S9e4_O@kOCs@P>sSd%;NXx}0)3(|@}PNoI3n z)?+^b9RtmKlLVSf+k(;KJ;g%zsst}IXeq(__}GNfP&ajg_i@HQoI~HM61?P1GbD*g zA76!%^r(9iJ?bQZ8La=QitMwRj2J=C+>4yN{1~_%9Sbd-@+-=#7YfKXH|C7g4^Tt! zsr1JW#7L}D)kabLL(OCOkQ_y1Kd~2Kh1PBKJtQ7BKUnaKYU9P0a^RU9qqk z|HvIRb@G@o_>wcTabbx&5@l$Vg8iCT?A$zm_R=}=KjqC`Qn#crEYfrCl4(52AAgl#Lt$|8Q{Nx?phxrM)QAPH~(}P>+S^=rn(k3I8^GByIK5sBB_h_ zFL?wBavc9cKbOMp8TD!uOOc}1lh0aW*{i+f;o>`Wyc23I%CuJt)GE?yJ;nHs{l$1j1Tn;FRlJRMNMMWWTN7d&a^|Vnv9_{=@&JU zT7CL5vsTeBWIJbj)8y&5sMTq@R?}08r~POV@)l=(CdcB>^NU+}N69%SIO|cH#96gi z!&#p>$P!zO8o$iLWt!@VDNt*_%$zk;OPuu?=0EAI_a|{yF2A9jKEwSF9Ek3xvm%F* z<-F(lZVbQ1U6n;8m+~zAUl+&XiAuQ)14$BBeU{@tUG@G%S6u>jko@clVWql#CHZ~N z$8U7i-E#0XlUhx?6LctE{OZ}?EeLh!d0%D>v=v4k2NfVOP`ZAVrQnmdM1#-mB{sM{ z(FQU;K6i*QJ~|hH(%I_#3#Z_N1*96ECa>T!#J>2_m&iIO(|Pla}ke&7#cj^BpN)JL<5Poo<9y{=vfb{XbT#& z6m30!dqQcbn>yNh{=;}_cx@l@=3t$~YhP%GlJwdK6KiZ65W#C-7%8mQvhx5dFv`PE zblQG;Iejy0Ii13CWFdVTzT|v_(ZUs12-ZIC#HCOeXGpiaK zOUtTSyca%(8cuiwv+7W%X4e0*Ve zPK-+n)~~OnG}hC^7_68La)9lD!w^m-V%*39Mwd(tS~S zHKBoO-iHeRS#i}k0B|*6t`E;wXL@HkexHI2YMN^ncw#3nNTpTaFDVwF2tK}4OeA~b z4JUpn*#D=sV&J<6^)t|9dP3d`5U! zEdD;Lm}+RkA^6)7cmw6jLH<`0(q{Ym7>m?`_AJNmn~BZ1&gjQ)A<||;ou;w!L^Lxr zqE~qSmwi-dw+8Qgu^U!C&(FkeQACB*PRWv<|CI|Nk3IiGs^9E@U-gT>55gN(dS$I( zdPrr-Y2M*Do`+L4>5-?;A0xg7`N?ScdCkaEpwhGB1!Z zsQwqke<4*l?l;0UM~ce4;mD-gGJ>!Im%HGO7e!@W?_yHn{A+^cD{;{k?z>V{=2cfF z)fEw{D%>gMf1s(%Tcu2@w<1*4xXjOgUsIXa`59ENb&62c;JSbRJxyg^_-|4L`H#?s zwYbIDU#F?oVY>p`IP-=k)u9ntTjG%qc=Cg?hWXG3lj_z8)mFH<9~by5D)a7slPXyM z4iRpRE6QLjFd~!c`G~AB zc=WdavZgX0zHLyw8I1p-tnKlHYyTxpWj^EDq}n~J+<#G1y{LU&9pf>n7DuSY z;_=gX3boS9d;qmcbyb9FocEaPKd-5t7g@})nN-~WZd-mw?@`x(PE$Q6R2yunw~8WE zJ9&?|{p!Ha9ulgr5~yB`Q0?yB;rb71st1LN0g_?EcUwiM_V8|Z{k58Etxz#QGO2>~ zzfiB~-fgb`fTnsts2CubR7)eW_VjLb{rff5{X)e6$)pO_ze8DjVNKkaAemHON2q3c*Sr25n(7XrVt`~& zy*D62)#zR4`nPMU+l7h&l1Vi^Le=D5>-x88s@sHWT>{l<5vpeI8rMf!fbsmTLd5{d zl=WDIYL<7k>))cOZV@U5NG8?q5vtkVRjz-trn*_E7$6x`@0UfW=6F}S{u)iSMyMDd znN)KkRCB#6T>mCbb(2srKr*Q=j!^CEUGDlfYN{KBiUE>I^>T!2KkqWv=b1Rz@CKn` zfMig8&^kgj&%4z1uh&%93l#$-lgf)w?eAUU`qycy>x7B{l1a5ZLN(u8?fTbhs%wRc z0g_2|U4&|Zx61Xe(Nxz66$2!b>ir1SLhnDWf3>E%TBsNx8B`y3k5Dc0E_VH^G}To? z#Q@2q8Xuur>|NygS8A#&g^B@^Np)0&YKeED>tCU%t`I5)NG8?Y5vrx$1+IU&rn+3H z7$BKc-$tmGdFQ+SWt!?Tp<;k!P<=EgLbcpG&-E|WRF?`B10<7buL#uv-np)SiKeFf05YkcYY0 zMFHete#k0jUPNahNOYJc{5Xhz!aU4=ehir<(qSiStRMy;nPrRWM(sRCCtY zWiem)LMW2F`Z(DC875(7dKbuI4U$(EfaBkVQ%Jqchn|1o#jHkMaaRF+k>{Ge;CBlCCA^Rt7YC-Zv` zC&};aB4t*IEgn|+Jzg9-1Q$(u|HjW{_)ym-&8n!b!4rF0-r&nkI?T}#55wz$Mi-s)Vi!TXDj5#GZqT<3XMV z-lu5ZP1PkeO(h%mleRYRF9$|!QZn52j?ui2(YzaGR#a3rx9t9`%O4YXA0FXdx`pc< zr+FWzdDm1lHDZ5pi`(|do9Pe3CSSfC;axVw^-k2hPt?3C%Vt$HRJFV%koE?V$@{C| z{BLOQ@&PXH^i}QhZ`|i=;QlH&{~PkID0XpSv*N8UYi=s3ZLDah+^7%YDF-(0^%0v? zVjK18n)m6-CXF*oN*kNXTHd-#n{=^xf4xhDcU3RfJ45q6L-TH!i3iQiY=frP@L z{>R@r=ED`%2l4+SYJ&wdXlaARciqIEL20O)dV|GxW8$S3Kp=Ui*kJJ;_dnBPA4&As zlQhilg8YZ7NU#Em=v)&}qvex2@wVn0*VMblFXR5+EFg9&5 zK2R6IcGmAFi7m7zGI%|<4imL(SdiOU2mPN0uMZ~OR#{UL`~S{%R{BQE+gWRxY8xxd zHZrtngG&SWf%z{sfJakD(mw?ApGPH<&USrojO+ae^@kwrKe{KbsKgWxQVIiO-B z%{N-tk2^@Mhs;NNqg@=pSIjTd%*&f<@F2vEhP:V(WWxWX(y?Gs_X!?rG_>k9Y?j#VyZ;>so$YTi}Vvk**gG${3cN#Ol#g!fLEyl}XHi=V4` zmsd77mNhnSG?@O(=KaeS5#BqOxZY~bd$s0W+E7x`RI+goFG%2hOoaC?gI(;cQMk`n zylbmwl~q)3gQf#m$W$!CA z?@S&8Z1CPNIl_B#7uUN~^S)Gg*UZ8MW1|zgy3V#uHrx>5J*CL?F4DX&(!5KWs%jcG za@v`G@`ug)w;=x?*5%aBF80MKo17=S%PKGrX!(pQd2=O`HjzO6w^_i=u;IJnpcoHd=a0@wfvFV*0`#-&_27++ir@(fVmY@P(r{i7#lehA;fNuO+q^YJKZ1 zl_Np$g{7JKLa5f(P!BHt=aX^%^Y9n?Lb;=kdh}NZUk0idTL=6@{Y7q|{Ig?U7_GrhZ8LQc#0KfE~g>WMed{S~Z#KE^{nD9!7JliqRm z7uSE-S3H*3S6m^iZiy=j^D4o6GFife;E6fio!rcPKD|B#y+C^BAmG-M)zs);5zW3hU_a7kpRNvZ z_SzwGUC#gM5RWHuh!|i%Lw6^E@VGilGZ7Q!Y294UnSr0Z%8RUeB`>9gEL*?@kEz62n8vc^Pjl(V3p_W+6_O^C1&Jwb8<3pi7e-NRXROW zp)_O=hMo!KW?A!Jw*gMRWu+WyDsw6`tK~c@5l&Wevx4zo)v~(?x^D&M{aMLz=IH^) zm3skmv2u1BvGUqPD>KysD`%VYe|=nM-QWk9Uk!6Hx!Kg8p7KNzPnj!>mZ1!@Nl&Qb zIF&L-vNRHFkI=JJe)`W}Tf?<=uuJn3Z>!^x@bn9)4TQO|!(7(ZtG)G4a>oT}PjdihXxxqv{Ja|G(Eaj4Ds%OYT0-^QuW z#dl{Hw~czJPAc)~27E9{wz@N#GMC{E6XlJM%iSM;q1^8%mwnpwn4-8%=r%D30+=Wu zUW$WY_;EO{rQEx&z^QM?dct9z0^J9tS5H(Db*y;n4H3EdCy$K7hKPOz zs_xR`&j%L&jnh1YvJc*{dHzEdoYlAt-u0HtsgzNUw>dwf5xKzYFZcDNPf;LW<_g45*;ao!T7HUX|?%&e{RUa@7NL1KNG9*0k41aJP+YC z2&7>Dn}$ToWqAjP<;FPP0hw4X38X^mr-X^|3WD|D(0GUVzrc8xi1B{OD%Qp$A&FMv zjXEy4!m48$Z-GAm_9k7E;}6Khc!A4hx(SC^#OGR9S$JrHj5&r%kl86)`7&dJ)hN@L32%psgN2fY2tNlKCmRR$sXo!fcY*H^KA&s zM?w<4hBwSt*vlg1QpRz0gYgsTun%W&s|*GyOd!(h5nF*dUVU^FnjVe052qYQWw8&Z z<)@u$9c931n`7z8%P`(VCcAzx^~B8dgDhPoX)un#Zk=f(HDKsf(%iO97Jpv)^R0)+ zw5zTUtE8W2Qso(?V+bT%CCzR7l?7xHW@T@Ci{;7Mhpxa~o z8wU((J+|XYd~@<2J+c-bh6e3<|2Z{yCW!_TLAMX?KX^t((Cf&Fp($R=^@nkPd~W+I z!IANHP#Uh8I)ZNhO1v~?mVMTNUy2C2J@0>DjXjfCW7Qx+jdh3#tEW{2z0QoFd+7u3 zU91D|TG&)jJ7mHAM`SbGWaBV)uJ&$s)T*en*zs=9RZXm+rfe44Zj0+4F$YIy)e|8Z zHuDTL;@_zm^h6Ra6B%bfK-V!hj;q7yeVp7 zTvQMt|3Fb}1G~gfA{%Dm|8H>qxm?PWKCa`Zak)WIyiXmTl3lI)xVqN5>8Do#8l@E! zcN%N)XN0oIALKYH=tfXHNDsB?@kc~3o|Vg5rcL0!DUP_~r|^io5fpd&p9P0U+?%jW zpzm2jfO3)Ua{nHjLMmnkD4oYx(m2Ad^nb$$dpWI(p!m0}V(Os@jS;r-)BGz3H0ubj`ZFw7H_8A=o~V5_RVU*1`Gj&>;73?RtA?)_Y{cniWZ7 zaKRI{?mi`g`T=arn1f&8-g6XdY|U=0DJe5|B9}KcBWBmOu5H3r8Rg#R2;YGBjS=2! zv8$Xz8hFpnj5oKqkgxFW!Tc{fsn8@3f+-^cerVoR)<%KU&2oMM?;!pOc|Qd2Xwtl! zGULs_!NmP=1oy+(n6XInUL?G$xG`hn?WVQOyJv9!eTe%J>?&WVc`wY2cR>R0VErTH z{pbkSqvc@l`NF%TqOqZJqn$~@Vt#JVVEhkxKZYIav>bTvpBZmv_-Sv({qXP2g8W~| z`|)9JDSj+*OEu@pa_)l(w!x%olX)Bz$W(fg%}$C`sZN1(nfidi@$P_S4vMw4dg^UuL{tv3d9A z{Lk)5p27AS%*BX%f#O}$P+M8jP_^-$h9FY7X%~23?==zL&tiMc?)Vkv-d*#qsj01~ zt_-G8DQlARO`|6J6h>G-_y4r_Cg5>Z_5c5!G)vQ_d!PjhofKMVp>2{TT`8S3DQ!VY z3q_DGU(&Qm7bq=sVMoiZ?29M@LMtkYD2s}Kh}7+?2+AtRVp#dgA*(s0{01?^cd|{`|hMi&o(RmYCkmK zJI;Uy^9|t8L<^DsAP|$J2|kIRg7SNJUMLUJRD^| z%Bmojj`YW!$g<{2BL0E2h)tb36YCClCO%xm#JZDx6M29wIR5CPd8olNv5w;(O?))P zM8+UZ+=4QJDf15YbZ=Qx*b?jU#%mz|L!YNwmT0V zOYy*E!bYBrG8`=*Gnp`W&|M~MWH|nXr9r-UCT!%^NH#NJlwBkfwtF>7GO8cTZ!x#C z+I_4O^{9!3wNYVgH^@HvWy;hBnwrz%qL#dxC?cR$}39xy*P zCHFo2iE(v8ko%s^kJZ;t>S}15><=}|x+Y>Vz}FJ|ZDM167E1ffo%iwC?H@b1^}s{_ z-Sij@<392dh(kNu^8&|ZU|gPEEBy^;z`%8uO%5!Av3JT+YsX*v$Sp{Bo5 z6SHF<%c4XtK;eIH4gO=bROyC-QOy6bpFLiD*FI{DGwpG+Yv)zv_T2uN+(!_DDeNON zq;@Ta|4Zla7S0H>bzMIfeVT8Bf`Z>>EO`X!Pw5zk*g9Qf8 z?Rmn(>p!vO9n3epZ-nRGH^P^8HB7~>ecQrsN(vvHE3kYC@(nHJ*79;IeDc)Brmk%s z-R@1nGgMEq4NTT1X5T%-^3(M7cT8+o$B@h_IWY+wd6fUz5= z$9SP_>ZA^Q_u1cF7Yq|7QoiYAyIE_merQaNfsf~pfeTO&4YJpG*V<+bG)F6zx6}l9 z3_bpiqZKt}CZ*W3VT#USh%AwsyQ|VelPu-{9>DG=%UYWEY^Q!MPb8~{LX6q7x>-e`28#7$BN{~ zH6q%2@BWF3qpxKHnxQuyJ{xQIXRKD}@>l^lhq+^4r<@9PrQklc^(+^_at ztk4qVUM_`Po;5XeD`0AYpKyI2u79`+i`XQKo%`y8hWX1}7pYAG#`+r-J%r>-+ z4FdBWvx?p01b>Tf4nsTKN(EaI-w6`?F`gJ_`|5rHo*Gxe z_~F}9LywFWhilxFgeHdAIjtW7*Vi4ck3w9m$V#8W8=S|5=RaaxC${b#*e9~%LfPJ> z;Bt)`pX8}1MvPW%tmDg>!ls8gTWkPtY$VzA8jV7KD zuK!|qC%2BnHjVZ01mSU|z{827a8f;VQOly)S!lo!h;2W4nZ_)!2Gy@mctUhP%f05% z;Rw`grJzeQsD|ztZH`_2?-Lpsc;~m3f!!^RU0KME6(qpmTV z06=n|ZtDed-*n`9h2&U~#c-YjJ==pES6b~91xww>e*_pm{~P+)M#|^ zH*r})D=B`atsE4;R3QrEM_2bPHfKP8JlWC`US~n@H7Dmk z2Hyj@H{3zMT#$RC6mm&xYUq{G<3g_aw+YXQ?83H6AiK>Ws|=B`qK9nK;6z1++!?Kn zL9jFH0a;_|$igQ@WA9QbvPqi~))3h*b8oseFp#}jIJ%o(>V?0wX zPBeB48vE^NWWh0>l17%2j`7eCrmkI{V=BM@8E=Lb!jNfr_)M7R$bH8;+Ou}@9bE|> z7zn1G2>DUco~M);VN}kfJsNf(KFo{R>g{(sp^B-0_LBA)S`++i^wq8S@RIh^YGEDI zp-&V~+D{AjKTj}eZ_ZhDb9{ALIRAyE;coGy{j|?I`cLo+U`94Elo=oxi10ZGZ*ZK( z?|(C4^+bNc>JnEVG5$jo{>Stoaj*N2pN`oOJ{{Bg51GC2wzm_CZz%MZ`5pfDJbthz zVhJ1OPd~$Jjf&s$udRdo*Wd#MGH1I&Uvf0!!R5jhY$~f5HOqsRvAzF`gUg6a7D@8_ z=VM|Z%bFVseLrKLk68_%Y4`LYik>|1GX2?__R*OQ(L*rvAEn7>Z`!|7Px48eg^$q8 zdp0L}a0bp{l<1s1aS{$4hbO3b{kAY$YMa9i4>Ot*O~Tu%O~*}E?DtSH4qgr6Jrr!m zLFNc3dpX{Siy6uMAMe&MG(+j&64>|+{ifU^_XSGatzi+ik+2{49hubGIC!%)&48;)i-p*&wKf*;Z82|##cdPHf|q!f)nwH_ z9^~3e!PdjR2F=v&q8-=2d<$AIgglYMvtmuI9MOJ4DHfCmW#%Lhw}E6f;j^4(DEtY$8Ws3xP|Lq&c#or%*Ads(ee|N;cDVZei>#2^mCk` zDgWiw#H<;f2bhcSYP;#V3-RH3FpK$rdhk?=2kr>Ktn*NY>#e8EYJvp2SxwC1`47J| z+%3MEnDt^*nyU%QF0z_v<@m=Idn&)hErO{A5q;wL$5LX@|f*pAQIHZ~FgS3Gt# z!TdH}?f?URd%yWN?Q+r7`U|fudQIn_``>-!fwn-V1D|x@Prd^IH!2skAlJ%!xo2hv zEqni0*Aj!WSl9BJoVg~_K5u^8>?5*f7B1yE+Zc)eJI~o>T%;{xGO!V-Fvu)hHn|fs zQ}-9i0|VO)GPBzgz`0V`mycSU2H^Z^;TP;6(-XmVrGUmVJ^N_eptFA%$?w8%T7s(J z(rFmO@N>|bnP#5QnFLNhda@BkxY+xJQ`50{BYgIch0+)=!(!I?J%Lrf{^z%QRjse%Q_jw&uLb$gySi$jAo=Btb5BEY}@-9?(_a*FeK z{*QQ{4mbkv$kg8kx)GoyGq6{_$`bm&J&!{@E_4qF`~TAc`#*L4ox0fn=d=I!0vP`H zwqQFbrc&4Y4+!(WrYX(2lNBxfg7#*19h^JqXM2Co+Q6D=WuJliXyXGbBZwSbnDhK( z>%<_~h4MRs)`>Rf*)#b_%+CK-sdot6M;EhXpj>wi!$4_}qZdlfglxKJ_zQ|VJ_~;_k=GN!g!7KHiVur%B)z6Mwj=6|0z`CrN|vIbZX=6|2hZ?QS3gjiS*=6|=E zSg5C`!MrA9@sw2{0qOP$SbYbh2i~gjM#u{f@!(C;S-zj zq#$>=s@ zOboC8ZH?9i|G@YC%kkG>cBW6SG_@tI4Q_{^#dvci0MdudbG>eBv^IF3`nWbYgd9h%a7i2PqlSl$q9E1=)n#>TSX+^U?^$@m#6^n|6-jGj zYjY6n%6iL!*5*=Z;btG8)z-48Wme0f=ok!OwC2tj%6AH?KHhRj>)aNGe)QT{#~1!d zYZl{8i+-Q5h|recV6}zT7UaHB3baCtYUrKO<))y+KAzA=beFcFH{dC+CKvVt~Z*co~`2D9i*zn}j zpxvCu#_7OYO5v7gP7VDsiX60$UzqTn&@OAk-2?GJT@(bgz9BSLWTmt5rj^Wpatmaw zB($?yu?aL+<#0N1aw*WTLj$zD(}Cs10uGj?o(?>m-~WnDV!(CcW{DOY&S`zK6mTgf z)zCleOe_a5IQ-`cg@ku`8&2lq8OA6Ccr_tBRwTUB@kYfXGKq?WcXsQPAjq;Fj_*z> z1)gnU?_glXwokXw4i|1mT%53l)VlZ-ACrYI(NQ&?!IMjtKMrHfl)hiZ<*i z@vLYR1hIZ0F;*loHyL~)JpUAj+c~XMgP@M}Fd0lOg%~!A?4!|fz{+g{qZw|i_zA9m zl3})t_g6P(ud!9UwG?1UQ)=jvzg0Xk{Qg_a>&iCFv3Mdk3W8VvkQXa@-Sfz$2_qR{ z=eA0C;yhcmqOstVnpDw3hx4um8j?OnB$DP7i|J zSr4bNrk4UQ>Jo=wZOMtWcag;hg-1vKXo>5fM~zQdLvAaupo==_?WJ(bv!;e_8AWs- zjdawt31#e{SG6JCgvZUJ00M4c$cznSV2?|9v!}jo3cd zhFgr~zVEON3bC;wvALAa(c%41Y>YCcb4klMcVAnibjBfCdYoZE!JaKad&2;{M${YO zJ*(rVbnpyB?yq7OyP^{fK|PXZ4z2-~(-u>B1)*SQ6%c;-Tfz7JOl#FJxx$P`vT~)uE*&^|Vw2 zMx0D08)i*az32)Gs+8(B0&kQL73VFQ+CE!Bw*=#HOqC*Z^S9t#?6AKxh zQY*uAknLTk5epaESm>EkXnO+jV`14GA6kZ|)W+di-_i4>&fhMvVAXoDuq?6v$?%lg zS8wq0pRBG)lP4{mTJI0nsb=$tEX9VR!~9>2`Z#M-XOAQb_WW94rW9K#-k_kA^>SMz1zr=*XQyD8|D zsmnT-b|ns~rNZXU#cRh^{R)V@=`A8C;G<)qF>&_-v14vms!z`%^gk6%etMd*^W$Ic|eW} z$N!l1E4$k}^+I$fdMcZuU5-3pes>Z1V5pk_x0gPjs07 zh^>BgcqX?PrC{}9V|8<5X!lva6@U-ZN?T8ti}a%zhnDeOrT4=<4=fk4^Pmo%nzEuxa)z)bh+; z6W;$QHv5g*OzvQmg4qWfvnNhjIl@pu4cchr?TEVPjs07 zi$%ZPCu1JV39ApaR^xV^xczx}^VOnOkIoYvp8t(Szk_>7wxbk8+xuEKOzJ{}bavlq zz`8LhdhN~ytlz~gBv)9`_BPH9O_OnB?8H2i+Kf!ldcn7LLxJe`D(zi0Vf9K2WMXF{ z9BR7Vg=yiJ+SpC;BdrwVRZSSdB zKdG^+sj+jL^Gvf7JqVvzQLZ}=qL~4DF+T(3_HXNMa=PvQjZ6H$F_q_@X_j-Tyq@mg z%D9O!`!{F6`YLC@)_enYuokZ0%S-dvC%3G=^k!yNS-@>p)*l2Z+^oEul5$(=1o8ei zOd7syjy}OZ?xpDFdc5%($p28_c0T^R9_!zE5NpntRc^vh!P<-7*o+U)gAD_m2d|`f zup7#vu{MO)|6VakLz<2>pn08WKm6m*8#*A7(^^;>?iNoPZV1=^ub8AEWi!E4=Hz-w z!wq*sE<@&({1)Rbi)J$aNyRUlqfZMch-Nb3o_hS*D&w4Jjz3?EPl!3O&KDE#sMGY^ zJ@)ITeCpvn6(&E?fX8`HZS20g^18_|bNo5~ypy?-hvdbg(sCV#z(ktKF^Q%=7@69I z-|%?yKXHw10Di_(4+og19u`yI)07nS*jouw=LWo?_oTK+FIV`~`vLW`)sD=X8uz$* z+t~;jz-FTnzle@_F3 zaYb!H?-Js@$L_F$<27PI2?+cvYD=e1=G?oDZ#nr_)J_fe|7NJc<6SB;`{k|;+ReRw zkX^ji#$HbZlj9IFGO@>tS!jS$!|NYbehWI-ySxdAiEUg_b2kC$S!;u1&rYR>hIm)h zP7AMpTSFGYkQum#rG1Ds9Q|Htoh)|LO~a>Mf`H+m{&N1H&N&JsPP^5c#LO9i;E>B| zIjymPn}$#0{*Tk}t0|M1i|NzC_5Z8U>`+#P$%YZ#?IG#Y8=Q%+7BTVkLp&3Az6YGSv{Qp-lT`7`qO+aYVLUY*cT5*Wv_MhSch3Y)H@^0|Gycm!c?8Z;TWNd0sg00%bYIl!Hz=fD}U|DglV z^6{j(6PyDlIR{wJci^!|ZRo%=Wh^tYd{5tjJ&QZAKS~3|CwGk`p#`d2&;m0?xu+`a z9gOM~`yUJqc38i*WA(a~U8~o(uj*L0zO8fBqLs&W5%7+XMv2TeQR=#c)Z2ZjgM(3d zP*|B2cX5?(OH}6f#t&u@BH@e09R5!}lPJ0wW!B|#&V1m{#hUgD&16-gf|;KR_dh0n zs#$$KHt6m9I{LF5eQIZwW>;-tq!sk52g1PifMs?q-!Z6U@YIdsPmGv|-+qi73UH=> z>T{70fbY#O8W#(FV|rfbf9i>-`f^vj`_P|mZQmH?|KiYp)`b40 zf@0=4QAb23z=J9_@(F}*I`7_q9|zBVz*D=fYwgO8RVz>JSif@3>ha6gtUaz{eb_lE z(t+8C@vkNBQZFRLrnG4Y#zoJ@Kl#NV@+K`$hOTri~GNbYvtlJb6pdh z)tD%{u&0{XN6@X#;{GqvWQOJE;$gWDhviJ@AFG7bzLu!Q=6I&NYOcZ03cvqhu$aO8 zx%>uq&2V-fUqf{VG@P7k4cce0zoQvi{b5Fr7DBAaP=CShN1B#NM$*cXe1n&R~@956F*NtzP= zS8@g}(B#I_&s6wERi2RW2}6GZRW$U|GjA;Y%mKcT-gOBf+XnNQ&-m3VIRh7%>%e^G zJBgwTd#Z^BqkDYjUB9}SfqxgrCI+pO?4ZTs;TzR17@MfZknXmcYw!!g{HGZ$%nZC0 zQWFjCn&EJ{5`T_L*)Y0PsCQD%Oh}=< zo)j3of1bgt&$^+(TN3qY@Hr@APOG7mDdtA~3*Sf-WrRj^N%2wNRlX?9|JhOh%%YXe zG#O4A2f+fMx+pyVX)9muDw`vFjhw}b&%yJ%j_+8NoP#Nxf;pp7czFHa3f~Oj@xl$^ z+-k#_wOGS(yXYSY7@UKbph(&*c5#^hGYU+H*~*JAQ`ydFp>=T!8Rtjg*9<8)7hD{E z|0PaDJfAz+OGKRPrZLuIo#}YPuZ!dHFWkpJAF#~s<)gfPJpU1I!}f5~f)m5b_Y@Ug z|27^}vF{!;-Q7N8STH?~Q?6{wR(7&qjYL&eckO>kqWycb{UZ^BgXwt)SeX?a>`M~u zU&XO_9gFy*zMH%+x!i9TW9qT{x(wS+!M?t!6+y4#)-9*4yF(W1hqlK1-!ANbX=@~7 zaIkNlt*q?XdTE^hL1km>dK5_nhSPItV*Ok32=q*<3);1o)x(7lG8IS)-!`Jc5r$aDroylCWuqW#Tks#p&)F3=LM z2VEBX4;M`J9`&Hho=bR;+6jq{PqVulPmHN~-62 zt{9o9%)VQo#|<-A@cM7GHZ<#~NYAh&q-gr4$=)z?#bv$-vxSoHt74lR$RfV=F7y7`#QLxDa+FyzIcRTS(GkI9-+FutF50t)lNF_e7XQZQ(7VBx zG&q!9Og>b^6 z3}^3E=lODsy^);3!IV6E^CqmT66+shZ?OsMszm;?l3b$rDBhBLSZ6;Mo`126h(}O4 zS^-pJA~CC5HZ99fc6Hi)$Esd%3WpNVa2t@79jm(5u3r}oyrxMgh9VrF&wVu^=Ix%@ z?IQ4VFZq=#vuMx%$&WyH{q6JpQIyx;-Yjx)>+=)5i(8rZVPSCX@yo8n2PJa)MpS%$ z5v1Vy+qhj)E^h7Y%A9PC8n;VkguBZ05>oB7gD;i}ZTgb63DTBPynXSHp*prd)eY}P z-K$RB@JQXu$Sb6%yLymQH@t|ttC|0$y4{MayV{+99=BVh?iEbDgPrl7cUNz8>UJxp zZnIOjt5G-5iLl-S9f4o{QwJd5AJ?`w`NDE{fdTwbcG7@dOs z8n^#}cw*-{mwfDw=rv*cXAM!>{TY(GzR4SDGu>Dn3;AkmR2L^Wy6bXBydcBDx1ypp$W-75mUJ$BabZ*%E}(Q#7ruB} zR1z*2xd)&$U!DtJRzVlHo=9BWhp(?6~BIm zg5qapOi!)Mj4tfh*S8fEKP$+Lf*(~C$g$b+H;aEdMAxs6%CnU>;V(s^`}*snl5Az# zeKtz-*eEu|!lEYy;5^QwoZAhvzKXki-TCVeL zemkl~Y&lp_r1O1gm#8GM(U(7cIwOxbJ|h6KIKaA(n~^A{o5-VzEJZQxW5z?KU;vRksoO%9Gwkq!%*{!EtB2{VqOO zz-=7A@`2+qGo^8NM>%NvYStMwlZ{(Jqrik_6=pTg2ih)PMS91__Zrf>{#D5;lz%$6PB%6)!DUn-GozCPgp+r zgo!7utzSK9%G8EcOOHM2#MSi;>!#M5W6#T1PgpeJm<_8}uAe-$p?>0|sZ%CRoY*{d zD!yqtbT17jAfJ7wvL<2qK4U$tf_E(P0X<4(gT zZTg{alX<=$Y_d7l#(;0xY+$hxc)cZnP>Y%l4^r0FLvPvEN69AbzeyN z#@xsdm}ubGrkstYtg%;`Oudd#v9puCA4iP>LboR-7;{P2BFUJ`X;kFe{2SkeGPe1m zB5nSS-*|04@Rz6=Ty*95E%@--{2Tv-6pl+rrL=h#Wx@9r&i`!lqd07u-NiY$BLJJ+ ze*KA`S-jn_c}uwdJ<81K_OGeK<}If{>FHuo#vEAZMsSC_?sxiM?jE+5}ImfcJdjyT&Z+0MO?=a53O&FHUYrnodji~efb*NETgiqfrMf2hR=EE&O=2#dmewTS z3gf>crAb`w=hiU(JIu1-e!x9&hfEe$m@ohF)64zb8qR+mY>)yp6)ISj!D$w?3Mv`3Q(dE7)4BV7-baP$`vF zU7+5^{69y9r72XhCmPX%bB#)1S*6S<8m9%Ka$JRd@e0c z^=-a)Mm4^kXSs|n-y^8-+^fsNSe$0b;uc}OR<2&pf7%y?O~M<#;MvZ8xc@b=Y`i)Z&1$CB zF~O*cq*M0X3x8L+Zv9$pCahz$nDQn&nNRyBRnf8I_6yplu-NcmzocWKNO;C@q6p{N z9d*7K<5?u8HrTJgGgkM)=??aPhEr$paJu6hr=pYLRGC9K{js}A=8o?~<=2?f<-8-`{+j#f*;g z!~d=fmjAa`@DuEXRT~_AvO!}|-xHG$-mu=Ynd1##IYcADxWV^3lM~z)BDi`u_=W;b z_63+D6ffpZao?`OgnCLrV1?R40e2?^FuaQjK+_e@$lrM%0vMUgQX}@xLk}ocw&+>fU=UUa@ORRq?DW-i#{E6Xz z48E?{H_`da7yb5&_a%RWa!`(2fhV`g4k*R&g+Roh9rnusLBE4q=C{n5-DV%sP*;Z{ zG>GFTczo}=MB_D}@w%PKX)vpX)t~XJS3QF2JOyb|S+zI?89M;8&_BZJ+!R6kT{|bL z^XLY(56j@sBZ84;{)k)&9mL4!BvFbp#W4(> zD)yK8-G$Cwe~ZemG3A4xNa)vJx1)-CiXp4PM zHjDcR^l4~noY>qndCElgJc@U#^Si_Rr(GVf7e|3G-ZceE;|2c3HMo|cP}eedGyfae z-g}ELIBl&MhKOsKduE_ITV_@9mbs_HsaRz?^+w?7cCC2NkHS4{+Tk*0d z9+6ye=gy^Mdh^bz?j&CLr7{LIUD1e@^7VEFN`T!;@AaM612>yW}|U7ZrGE@XZ% zz!|!l899^2DL1K}iE$cd6j}kD+vH}BLS~15v56bxX~I8DwpYC=);aNmW1JJKi#YKE z?E~=M!%x8*KPuzd56_7oaQ{Omj!$vIt^9xRxO3w8pbKMl1`XFMOo|OhQq0V1 z6z?Y$k#oYn!#{L)U)cUu%phOPU}VH1F#*PWQ<;}S+{KLhj__?TAS)n0!{Fh|yO?p` z*$&L{MZnzmWe-dSgfTGw$mu2uzc)y`A5D(bN}khy*S0(OHdAi?hohxi=%XJ z6{DLih|`*0y4UNznRrowF(OX)nz6>Nd+NAEC`|m`hTm^1@k`6x>g4{r986=KiShC< z+dm@c?F~*z>!6wey{N7fO_}#+mzij=Bi?sfhUwf_J=Vh1~yf|RmWBBk|f6IH2!eG53rS*?M zS+xF-hB>#ch%&3^p+lI~pAn4l__oxQoLHrf2B&9ZLJ#S0h`!>BePFA{zr`#r*)3Ul zt`o4u7f`hclYC_8xTK-orWZ4z%mb|4AMccqdwaq#BraQh1l>w5qs5f!T>f!*{?ni} ziOZh3oN05Wh4~NLHaWQbhHIP5m|#qG9I-9&kC?Rn)11Y5!8!krau)aoe(^8lXAz&c z{QMs#G+GeM2;@_}Kt@|&jT8oQXhC@X z$%X}cLQw%|S5JVO_J8_I--61;eoq*l-xKmu1{A?f2YUX`-1pxwIxNQm{Def{BV~{m z_-qRd_0Am1aQ^)4NZ%ZjGUy>uH%a~MS_t4UbydnR&4N-Hi{rI3-h=Q4g#Q}~xg2P(${%P^(G-~CduJ3cABI11a#>V9 z%)eki8Ubq)298znFT0u>PQ-n-CH`>~GBaCW;J-t|icC-ym zW7V)a=YQ_|OdD69S-iRp)v!9xU_<-Q!~DOkez~h|QUlGwo){T>>|!jRvR&x>{Hla` zn<3xamZXkr5A>xr&%c()KUQ|3^K)GPG@)~KO6X)!j!rQDM+R4$(5XilEW$mZsUi-Z zEPiHKp!_e!M&;2E_>w_@uul!1Uo1jN2G7;`!Q)Z}zu4eJS$Q;jbop~tdLg#^ogka8#J&HWnG7lV&>g?3lq_m9d z)DL{tskp|p%mU!KU~DD}8`3)U13!vNvt1ZQMcUY8GxJm zD5TK|+LREqB`FAfVUE`qY+t7Y)()M*`46wJ)2WO0b}P9N!QM$b@N8l?2GJtg>N)k` zJ_$*5>Wti6I5o>TH8&~Dh%ns~`h2GzOvXQ*@&v)Dxdl#90-bW>^Mf4!*pye~H>KMK zJ&5>c)Y{GTCj1n9@8EGS;lo=9Klm?5VK82o5{w6;EL!p*=KnZMTo+{os?GX-V!Xch zGXk$MBY>5EK}O)tf{egJr#mOEE8@gMU-X@Lk4}8<`A>74^PG5y@lPkNPjSL64IX0t zn@(IGbh(T`MYvlUj@^>C+YE;Vew)wuNfF%GgT=4LCB$%v5{Vhib5&vGAgt_V1b)T* zM`VN9eD^{53}c5cFC*|Pj{j`k>x+Q-)tw#~V`Wg688-@2JTSj{22u!SlL0f(XyJc2 z6mjuIkHYJJo1%SPygkOIUZN znCmhE4~PB#&L|^LQ|(%CT~O_11iWz|6R%-{Y$+Lm`pI>se78rzyo|uZ55NP)m$A-d z1S;{ga&se;5ka+QlJ_I5Lzu$}KO^u+Z9)>mJd#u!RC{9+JaW}JJGzX(BjNeiJEIY! zpHVTpyt-IMpnkf+#GGRNbYujoQ!@hflM6BekKFHC|IQ+<|Huno>j#@)W-xw0`;GYU zTK~~X*ZTLSw7we%9}U+(_eO5@H*QUg-SP&)`h^?_;c)&ySUhbeXhRyPziLK|sGxZ>B-?W3UE1it6|37F(r^_HCI-WW|sH%*|p^WLZ zYvLaFK;p-Oi@0fLP^r|dh@WVUj9-qd@@A{poy+KujRZ#*?WsP^o}c)LCA z{>`@&ifK0#+krkgJX^&uDq;6KiR!ewq7R~w>yY_DAJ6Xc?Q8dwVgBC+DD8&gcB8m9 z=;KW*Rxyn7?SAs+0=p~w;*_DayRT<=#rC!PsofKbX*U$N8^yIjU(ar<7)JSaKgIrU z`}T5l5Av?6L1M|;UE|q}AGc?mJjL~2teAE~al27m8`OAqTg5QSxBIu@^^bTw>D>=! z1+3luJiDv5uid}BvcPUAZa0c+gMOafRxyl9*d3n#jP0)OAMgg~jPL%Q-MzQ3-A^A_ zU^f)E8^yIjf6s2K7)JSaKYc->I^(3z0Bq>3-2*(ktGBP+&-6+tW}HB=eYikvz~^Rg z%q#rCmw5A^Krvz_h!enK(rhT?XkxHcH**=-eb-shC%_6^T4 z|HX;>!TgEa?Hium#amkh7ocV^aJRRX;lo>7Jez|QF1j8`ncbUF7PI@a;rX9OqHRuP z+BT;mvCU!jfC8Mk&wkyR`$#c!G5^m)dAsAk5~ZiC&?0%-rB3X@70$C)AO7OQaE3!iUWdw+LYqWW8?J_vJOh8^eXppUnmOq%{5dttiaN#?(6 zlhNjEuu(C)yid_R=gYD4FC&oqr0+SOE7)^B-vnDZay(fiq@O>` z3+dpq@G`i5%g{&g;f3_`r$P$5;;&j)D`=guDEAGSNx+nh?D4x zkD0zmQLZmO&+9)l{MjOgztG<^yo{qyFni`LWANb_{=!6O_|_D|U0-}5%>SC_we9xs zf3GicV+m3({M?b+8gyZm#lF}pJoA${tb^Yd9M*ZUB2k$m2P!91b8Z9uV!dC~M0|A^ z@xu}E(HK)6{bV#UUcC+Uizg+-Fx(?CgRvGatV}=M*`gP}<5xBt=po_2fl{WJ!}8+R zL{ZL>H9f}i((pv(%TT$0J{A{QFY){jBdgf+^IlrvMONkvn)$WkH*(48Mb=9fLkhdZ zOD3|C8}%&8qM={9-5K?g*{HLv%Iz7Zs!Ub6+n=*+Xc_W9#?IZ7! zN*Vf4`toR}^pz+(P@6I1%Rp>lq7+YIcHrf9r@i1Qb)oi+PW#&_orxWg+HwBNi~?^* z>j(IXzvZP#D--L73KXCK$KmBaI7{Bn=fDPlmRI`tEo9Q9C;WQMSp3mzp;sn2bKgrb z_Yx=QP-pIYW*&YEEwlsnL}p%F7xeY!HSCwX$t?UaC^9?OrOsOoWAOVD4oGZHoi}CuetJR@`!FTB{j9$)gh#GAXFE4#{hs4L$M*N4DQicg zVs?4oVpCSbxM<317>6mVZ|aoQFs5M2`u(d=#{T(Uk=B0|`LAdd7F+}~gNL8F`4)V5 zt^X>of3SbPpVInnfP8hKbL)M&Rn69)Sy=-)Bg$)VE;erQIsCU-I8s?K3vY^W;ri;; z&fxcp8I1407LeU;(vZL4MTx{H0|{p6HFKOJPuu}zRvtV zd$@@gb2RN`J}lEbDhRE5nwKg-TX=!&1iUdSA%H!hsDNZoaD(C-%X|Tqi~XK3JijLt zKbrQ&Hxpt`NRYsxioC#QTbQZGg!vTwBlPDlz8rHjt!H5#tcDQulQ+ZbKZd9|{j~yD zqs$b4Xs}ad9NT*$Bzk!G%SBMwEBp=ut<>!Hif<pPPmQUHPasG5g zLOz%5<^dkYJw)ql&p#V%G?0}KaRxq?5Wv$_9}gR6apgXV*iC>OAUm@jh;5+%4EA>3PBNB?@tZ z-r6xCr~`Iwd(kV|6oI0SuDN11NK~BPnkif;#--|4k-u}F|SPC|wR`B9qCO?4>Z*}+fy^zA? z=!Yqbr2|oxwm$D{{V-Z(4>haoiSa7i%V;*3tM*ubb1TOeCut@&8^-Y;_B(LSiFdgE zrxPC*apE1Uf8r(Td#D}U^h+GwNjUM&nbwK%L5dS@`gn)m{{nE7>2k-H`i6U!;n=f8 zD~Wh@YLX7zuuuf)gT*^M{}+qlG)6II!!j5bvLZ1EFT2}eMVEAVcZ^@zBprr@yE>He zuiC%62t@$~<}RaR`yw?q9t?SRg9oNJC#Il&%&euZfZG|rdkv(Zb;p+(Fsb|UckgpX zl|}pVoie6*2Ud8EXc{u{g{F7^=1eThH_>T-Z4Zn(irvnD$6W2u+z-{bX< zSn*~k?l&lCH;QY6L0%WOirIb~rk&k?nVP6hy9W+-kM^D)4EF4<*}iuF<&K16+T9-y zBd;N_+F-C}w^adAtV<3EGX~+F*!hw^apyXGk@sN4YJY2Yt!H=t?Q8enMkW-K_rThq-6*aNYCXHHVs;=ePX5-Bs7||w zU>F!+?cTw&d%$+K`{{&Y+C6B8pxr304R-MCwu(76d3OKZegDaeliD3|1<=~Pqi6TP z?Q8en-TKcdrrm>g4BCz2+F(b|ZmSqZCG37A;Wl~iFccH7wR@;%_n_@-_dgCuD5l** zh6e3MacwZvv)d|$QNG>(_)MZY?cNdJ2^ehc9_HCSc>CJ@zRUl3ylaOA?M88JFwC>t zDuz+M-S4*)*gbS7-mekG<|np7L$&$S7~w0p;$gLb31HrUy-+bV`p#%|6(SUddlc2wu$qB?)(W(Q`0 zN4>>`y==eNy_a5yKeB+E-v7z@pNotB#i#cVT>cAXR$>Aez4n(=n# zez4e?+rNmpA7K8E7l~J*W>9uy^+EXX%>4l8KaIJAQq1MWcKH3l11Q4@Vvt#lut9g~ zODjf~4@zFPZ@3w$<3awV!iF32FWdi%>mOnsRK)IoVg8TowwLV>f2-~DNxT1bw6nW5 z#qJJk_rK0^cGsE(&~30AN6&38VgI0ad7MkYIwbWQ)!W>0LlKp6bO)$^x&8lmt8fbx z7Wu-nUHEV>%c|eD)jw=Z*!vc$?_A^Ty*jADq9fe=dVMGp|MuVZ!gRxv%zxL$E5AGH zdbsqX`P>bHz=tmXV+ojRAT{C1un|Fxw+&3XfaNDV^{;_^G}DJ+?iw0NN}e&ZELoi_ z-8GPpJ{eU<7c-lRU5tv^xHwSc%i|4q$Ijo4K(0x@269KiHIR?E|7A$mriXMTZb1`= zw7qPygBc( zePzJwe>8ljVut_JGu&Rb-}v|qv+@l8qcePXis7~|R%TrOd;D7n~@Sw|Gwom^?O<(SP*s)mAZb>UM-2ZdLvz0k=RH8_RQy5>Y5g-CYfCvzQ|3%>H))kk6}78g-h2J;B%RKGJ`g$#(24D;@@FPM4KU4b1j9!`Vl%38xfZNvn$Kt? zHbg90pa==E&2l6*k&sxxf~?q*kr2}CA~a%wK*|n*eeKu%LQ=V6hoUN}Z~I-JYhPcx zT)wA@91=pP{|nayRfiVJO z1fH!3{Pvx_cgfp+OpE;f=OpvLLYn^QwWxnoOD0pFx@Y7TnIAC$Pb?*~<3yj5>-2uq z(nZ^$=RgL0jeIxZ{|*{Lvfuf_f+NUC{q>yekt=t3vLxaCNpR z8pB9VvP(y%d1`n3H%4HLz!-rs0%HWm2#gUJBQQo_jKCOyF#=-*p6v+yn!Mr%cmr~N zU-;Eoa^;cy{{wQ}tKWGn2=Tjs$JAsx^XdOjmec=)$&Zrmi=1?33wrlDyvQ*{^Z+aE8>gRv4Ug%=X0_h@lv;17A4mHIdU@M8=0S(ZDm;?^Oc_`pNM#6`;yBB!P6a2`hr1WEH9mZ zb`oD;d;-cuzc@Ax`I9a90;8(Z@(vyErNkE)-#p_C%m3+m;tNbuR9YC+|J}qFG(yT! za)F$t^WDT3G(}O^LZSJ|X5tH)(kjo7p#Gdod_g0v&p&%KKF=k-pk>^$LSXU9R}){* zGONp?Se%Q8<||)Md_k*(wSr^*WHTE1g4P=6M}Xn!YT^qls~RQ?)c>=IFR+ER+EZ-E ze9^7M7g*2MXvr{sC*MhYf$bWtIeLH5ljIWIk6j^*l?Kbk^2>=|=w#WJl40pyeug|3 z+wY{$S9@&zsuY|N* zWB$;yiI3EsM80dKYPyGe`UcWDLq z))jt&IksO-E>Ic$X>7#{hpmSbQ>p;(S|hs><;PE~P!0HAJFYo9=IhnIGR`m8aJ8~? zyZ~ZngBrlws`q?ocf21C;5X^ZfH&NfrEtgbAHZ+XR{-zX#KwxH_a6iJZF(d5<78PK z5mUS!z@Mg{27Ko~@hqz&UTP4(Lq8MoWg&EE*Y5h4c(4DTdbEv@!wElcDP|~{om~$O)=|x(90Sx>L ztHLZFS_cfS(E2qn@E_8O@TqOu6$gp3tF%6dJ&3N=?C3SZputPD{vsH-);~1#(S0*$ zaE;bqiU!gT+8o_Dg9e<|Uj_psq$*v7@{a|rzY@pLI$k=78kDsDDi~3W>Q^-zzuIu$NGoRmr@&U+Qz#j&={%*)uAJ5femcITT z)b0i3D)h3PvCnS?{RH%@_n_Pdx%Qo4HIbpbu^q}YsGUCm%rwZA`$4V(KIZo8uLtDX zhe5{Neq}dc@f7Y`hxubLc&C0pu{$`uG1S|Nq6CfBVCK0NrQu AjsO4v diff --git a/flume-ng-sinks/flume-hive-sink/metastore_db/seg0/c111.dat b/flume-ng-sinks/flume-hive-sink/metastore_db/seg0/c111.dat deleted file mode 100644 index 6a3ccab035eb83e7d241fe07161dbb2eef5f4c51..0000000000000000000000000000000000000000 GIT binary patch literal 0 HcmV?d00001 literal 20480 zcmeHONsJps6n)+0ZS2|iHDSrZff2T6Nroh(z0Hg>o+V??5D=1;WkOg&IE0`GBoGR4 zAlxH_IKYW;At5Ki5s643mJ25i5#o>oD2P*n1mXYk)}HAJ`sRQ@)sny4Rb8)sy{fM6 zzZho#SoGn;uVp|>x2V*+%jo=0`B+!H>CYvqIZdAs&|bO7qR#2}U5_ulYG3*G!dj`^ zg{2F2-k>jE)YXYcd);T~RgW4|WEe0E7zPXjh5^HXVZbn87%&VN1`Gp+f&Ui+Z*|}K z3NO%@D)#+JDEjrthEZ1P*H!oeWV+jU1EH!X!1Ju!tY+w2h*{s4O0AA^E%$7bCM#IL z5JX!fcg>G~vS5kACLUv)Mm|7%bUTA-}Af(WRj&&$9mq0I? z6e$}>rKQe=E=R^rE{4L}zj@nQB6>UZxvx zWDF0ZugWm*7R7uaQIImM0_fLU1}o!2k*P9>0mTl;JZ%}WrD{ho=xQoiqjocf^c}{% ziuAR5FsxU1KHZ?Bvl}A{t!SIE$$Uxl$x zPkugga3!;l$`one%+a{kdQw7i(h2#2Ct;5Z5$Xq(9`CKI{h(sFiHiqQrq zp_foLeMs8215Q%+pIu024pkO|Z2X;YQknqWziusgfw@h;%-HQtfEr?V)7q)MOv#8srlZ~tdUpt&7 zy{T5)3l*ru&0MeP=F~7?7}$(~z#8@efuGqD4s3@*>Fz%vLGmIVCV2{bNRG1%#>hrS z(k3Q~WEEFr&-19bj2(j;g2N_y#}UhK~q z&3=t#@E5KP-C)e%CGIQq5$hIuPRkg};7PtF{4(~Fyv^E%_j7spD#zy|Q(O``tC43} zf7|B1tgEb#b(VXv_VAUK#kKfM+y7_y8tRWr=$L)?!nr#8KY}0?x9Cc@_&fW*d@Wi1 zwM2k&+2;0t!y$bA%vjAV=gC-9HL>AOevf(#p(0`T{F-yafMLKeU>GnA7zPXjh5^HXVZbn882GO-a7kzH-(>HA z^$>dptmoN}U|lKO|K#1M+5f*?JM?j#{r@9&QvEX<2JPQaQYxs#|I5{OHR{iis?xE! z{r}Ii?{zGrE`whb%dms&4WJNbelGK9Cavx9g~U9Cb!4`*-OpYDio|cgk*O14-@q0O zyI?5HwUR037E{If#FA944)zn=9ZwQ~u{$|aU<5;(>L(hmHOMIg6sa!5;rp-BD%WOZ|Fo&9;GP;w&~5_TVl*C{Gohasb63rE=5C74LM(i()@SO{AyV{w7V z6lupjIvz;AN%6V}<6d`L(ZX)ReIBzp7T+yd6RH*Wd!S70+9g71$2KpNDbe~=ec0}S zG8vWWGKd{sC}VNah(@r}6GfzpFN6oYP$tJDy7<~G=g2yk*u>;`Y;r;ZL&NA zF0v+SP0JoE>6`+})wf^byaW3|ERvo71D9Lpwajqu_=9s3@G0w$GmKNS|KEvyllu#( Cp=vAu diff --git a/flume-ng-sinks/flume-hive-sink/metastore_db/seg0/c121.dat b/flume-ng-sinks/flume-hive-sink/metastore_db/seg0/c121.dat deleted file mode 100644 index 77dd26671e21a09d05b4f32e91b1339278ef40aa..0000000000000000000000000000000000000000 GIT binary patch literal 0 HcmV?d00001 literal 16384 zcmeHNON``38UAhe^lRSl-JRL6Nj9@dHj}L?m+dm-U@prgf`rEeLU{O5<{rU-gjPaG z3|zRuDtDw6r$`8CB_u$cR^m=ZLIQD#BD5Dc?STta{VsRUu;VFP;uKs`f9?ML{q^|m zw*7tGa0~#Bdg(Sm93q~DdjHv&+Wxm`{}pch_=ei^ZS@)htkon(4X!?4+8$rD_wRpfQ{y{Z=j<=J6Gv+ytv{P2+IR^+Ll z#soZYuFXyYSsrc&Xw?WkG0a?f)vg+=cJ1DH9qKIH&3oYQ13$Rv~$#Q-T`d%}xpH zPV)IG!H+qKLSi}R2AfXu1x#Qvpn#{OZ9{Pno@VXUwT=)f^8ZSCYiV2KmK~B^!KGh|)G2taA$*KgNN+H9* zmcSv&t0@JcAF$a?uyKUA;i0$>_Z8?X@V}_Q_%=|0(Gb*sxre8S4-k(L?;`GKT=)p_ zp~lgBnDs5h5b=*17fz6Kgm|EF@@pNRys2^UL`MdX5btXo|3k;eKgGQz#zVxR#>s~| zKKhqN?-V09^!D`68prP-{vIR0)aX6X9Pb$MF5-?x_anrIh$kAS-_?Bg2qOm?ohKOI z*73!uM&}X6?<1CoA>vTu`acYEN@#9VeS0+W)@M|1J6MmeK!V!C4+MIlIAM{GSMK*XaMY{3}NPx8+|o`aj}AJy}ZYdaoP( z-;(cc8~xvwzhm@&lyjw*O6z={G5WtH-`zF(KPuB)1V#O<(e$4bZ_Vicw)}OY|J(96 zjs9=T-!l5YEq~kSe?g;&_*VVttr-2^mcMHBzu-J3jM$EsJ2d*gE&qzq{~{L|Vbt3H z$msvJ{HsR)$FvlJ<@M8T)A``;8U5dq?~RQ9Z_6JW{hx}03ns1exo-4-OTITXdOjeh8qTS3B`1aHkW03IzG7h{8N9 zeM@}?i$IZ|bEP=dH_XOg0@kVjW$-)ouY`GJUldG|m__poT!X%Q&N22yE@Qu7Q7zYK z){yXgZC^-{mN{XyW!9*^4r==%7BUxv)#r3$!mZlAP?l39cv?&U#)Ow^`yymS<~*y9 zLSw@1+P;W*DRQ1$6JDw93#l6CMJ>!5M>wqQ3w07vE@lQvW5TPoeUWiBCowZf8WZl+ z_C+BW6=G(PG$xSRz9?u(L_9M{8WU)3Uz9=4xa6fdxf>H$ZC}XD&xH)?ldmzsukDM7 z<;u~H>dR4MLIB3T2)PVn#;hZRVC;)f-8~9rN!Zp9fw3=AA+v({)&wzo#_l)z3j7~c zVEV3Bs*`&fr@zoR{t0fc;r0P;e;T*n(l~ivTb!754%~?c8 zV7Q|rm0vJ=gjuD=@ozL|;UVrlK|I2}A>vTu@D$?*8lUzH2H4L4`Ra*9aVlTna^*jM zjH`HEqx(>2xhIH68mE84IF4Ak0GBKO#RcqbPvatbA7@8-^Kp{ D)e`)E diff --git a/flume-ng-sinks/flume-hive-sink/metastore_db/seg0/c130.dat b/flume-ng-sinks/flume-hive-sink/metastore_db/seg0/c130.dat deleted file mode 100644 index 462c9af1607bb560599872e7f2cc60c73485b45e..0000000000000000000000000000000000000000 GIT binary patch literal 0 HcmV?d00001 literal 12288 zcmeI0JB%GQ6ozLun`AfHJV<~*AcR05;c=7Ww=D%KL=gnyRX~Mij_nT71R5GdheSaU z2`Zp~id1wINOW{Gh>ng1A&O9eghV)V?%vIky=xaV2zgff?~Z-$XZxQse%$V@uIqaG zovzcY`&hTjqJDcB{eNTgx}|v6+vlm~%DndIB4~peUt8Sl^7q+g<)2>v>Q;29)&4K1XbD%lU9B2+S2bu%Tf#yJSpgGVS_211WB zCq_F%qwEZ(E8?Jr08^Zf%6b?~DXw2*Btg7}F_%Ioq=GymjAJRTUuWcqAt7KYg-%E_ zMx_CVV%z~n3Jnqt#x#&_a|4 zeu$B%5WSF9q*KaiA38F>dIK~hqRp^D6uXkH9usA?cw+GARb`EOfRQ7h*GiRJpHe-`;FXHS$M_&4vq7zumBmm(CR%tv+46dSc0O=B zf>y5DORZ_%P_Ep|hZt$S&1hnA`@}mp;}w`#+2JwLMjMk*o{v&aXM}BY!IHRMGSWl^9`enD+>+1< z8Boz_X4~w)EbivX^56e=_;UAB*B$zu^)uENCf)dp^$XTlSocnP@JrU$ST`m;@D*!j zJwECBH>|I-o}6^;Th=SAXC~eM9qZey=O^9w1M9o27bo5OBkOytcGA_KSU+HWV$wao zuztw;)TArFv3|_@%%uGvte>;KIB9o-^&0D|te3h|b2s0A@@oH&^Z!TP9;y*Lyk2}i za7)GN1v=}~qof21MoPDtMH2E%#YDH$6){8=Ig7?I>L{fu#?4o6zyM3Eidc<$0fuT>JIv-=1WussI20 diff --git a/flume-ng-sinks/flume-hive-sink/metastore_db/seg0/c141.dat b/flume-ng-sinks/flume-hive-sink/metastore_db/seg0/c141.dat deleted file mode 100644 index c56f879992a03fd2bead84dffe3a51c94ec2899c..0000000000000000000000000000000000000000 GIT binary patch literal 0 HcmV?d00001 literal 8192 zcmeI0J8vCD6vxk8$Jf_RoH&W&yyL`=>*pmivpX+|VrkM5%qu_wNV~H$Levn64vM6L z779e02GLNYL!yfK1o;9)h>8v*DwsLzwJhRgItsKa-E+^L`TfuAoSmJ0z3`7X~(d4S$3l z>f{JL_Bs0Wh?YLoSD>#zUxB^?eFgdo^cCnU&{v?ZKwp8r0(}MkUlsV`>rV)L9LChX z-+u>r*n9uEQO-XXX#&HU@X6o-j1E2o9O)g1+xL6|H_6i`N?zJ;jh`Okw}9JlcfTdP zz~g}pehkjs0LHM!a>^<7bV6BPO%$u-#EMK#XxwUBW~HGp1!NA6BX5+Wl+h%A98Msg z`OIm?Y%2c=m`C1b)@sI+{5+gQKG&wyEKBky;Z&2qF12QPl0O9t$P4bJW+KTiz-i>2 z$x6$lOO5w5oIzfCMlDlG{tTQ&-U}`*(@Fj;EF!NeWi2yFei6=SVCTF zr!A|J{1TjR@_X+rvq}CuTtMC^ky}u>{kS8;ALo*eHDSV6v`R_9sfQ{!EMRph;}d7eq0;#XmB#Fh*W^Z>(@heheGPbEzvWvpki512&QOTu3TKa=qPzE#xy}SuRbP%D)BM z$WtwZm0{IStpD4vgS<7|Q0aAw-+^7^rQnJxn&fw3f_%8Xlvcb><)4t^6%=IJW|Z=_ z15W)UtDS0ZV5ZJ$9d2NQ6;kXqJ(yZsnzb{uQ?N>kS5aVjU1^%Phtw%pBgJbdV6||0 zUbb_tQ?O2o*BgNnrOvI`A9SZ+Oo}&9kTaolUbnNTQ?NmbH&NiKFo(1=sU9{-aR&v; zmNeHsIfgA#yw&s|m9*wyVeCAlZBo3Af^ZWksT1wW(ka*>#XBglm1DV+;nd=zN~d6# z6n9aejjyd!?cDAZOrW@j0$O_`I8XNQ0u=9}KyYdd&yrK)CKT_XKsv1rKY0A;Jf!%f z)^B|U`U> z%*KV{LNSV<;;_+$xX>&Df-A)V!3jllC?Z4KL7kv^pyXA%uA17am&?#}JpoaQo7A;`yJ={ z4*WM9`01e!BC=1smtlT?9+?+QgB9gSw=TB?hnD7eqm)EkkyD)WM~C{n&jF;;0CvI(gZ7MFrrOSSeHaspD<39C4Z*H-mr zsXDJ~L9L}asDq08zS`>(C3TjklqFhch_=@!AvKRmpR;6bRc}FRnw5+*uI07%$+uOi zuB$*$YpK>Yc{MQXoJV$L!xueCoiry+F~7KxzN zQmyUv=}65o#yQKiRBMlXXCQT1#^SuLwNxi%l2KOr%AxE`qz;p+ECQM)lnhdIEBXyNlk)YN(ueA+12dQ)Fi>Q@psn(|6hSW(~i$FJ2Q7?&zI|@PP zE#-6)px;W)wUr}+l9t#qvB?*D_q|8Xv(&@FRy`2wo^MO7j&-+NQfuTD7g$njDenWf)aTj3k8^lKueL(LM#7jUe zc?U`a-7(*Np4+&qf4GIc`EXMJZ5zgQW{g$kYkpg$kIU( z7mUYL>w2es<&dR=G7j@FrdnB*_ElPz0WucHVeE_4UY%9NJg$qVR9_oqO3$N;7jYQq zP-|Z~WJ;efzX;<M zW0}(DQ5gzqEw77|r8KV9afnRmGhbx9NubV@gx9*NJ(yJ#Fw3>PE_fn7x4AO<^khoU8-^C}Q$s;{ai9`R&KPk9zcL8WyVc~(Sz zP^l}MOGa4t)H&cBa1J;JoCD4Q=YVtI|K-4F0vUV%Q^W@lw*iiPgt&mX39$DL;!TKs zK<`DwD*(x_hzAjOlIx|5KOg>luq^+NurdHW0Lvd6moE;NhWWom$x+k_T8e5=#FF{H z1+@sU+o?U7|8qv6P^!uzIxm!ZvSwf|r<+2l*VGw|-GLf>i}7}a(jCN;#R&_0x+vT( z5U;6d82h6x+cXujD;gH6MMXoQT2?g7sUd+^P|J#j zIW-d{3I(;SXjo9Ue}dpJHWdxy!^o7QB9Rph3)S6pFv-RSb2;6NibOU;SjcHVrnaLZ zksTBkdb$}EiEO(t*P|X4Nz-;=e$FdLMIzfS%vGyLMbflgn5$Nfilk|~Fs4?IibS?u zm>Z%V6^U%Su%K@L1VOAQ+b%4qm7}63+b)7EsYR@)D#0&_6}uURJlU#YE~lG0eowY{ zSjZ6-^OlRKT@&ttCtE_yE8XG>^-s2)6F%*lMO58dV=fR2FK-Pjj!7}m^O1UUT);%R_W??b*lbsORw;N)wF*8ooZ zj`$qnC4l2^0^9gJ;MhT67@Gtf#hL$6ocZtL%zqze{`)xdKY}y=BRKOvf;0cUXMv%& z9?*Lb7)1ULcufBP^{W>?wk-d@lt8AxeT4(V`vZ>MAcGL|1K7>r*u)^_a=IBDo4Cb7 zPA7w76YH4E>1J?j;v;i89Sx36P-UT~o58V(#LV?{G&suW%|ec-ueDrS7o#_GYM9Yi zP}@dtOj$o5wvkxS9&3Ohwow%{94QR3jgX*KRY7eV?F6EAM?q~HxWEk&3ToRJL?Z)~ z4Y20twMI%Tn^`TW)hV&;OSPa@=dH3Y)tnk{6>Pt~>)6P|EjtIC1I_{GfOFsj208M0_6cGQiQF5T8Wkfc?)9A4Hr79Kou=$OPv9`!WCj4l;l5I?Vq+ zLfnZcT$%mj^~1~Z|8WAH{?jWATmAlj?Eg2vqQ~lRQqKRIQ)^VB<@~<|b$i_jLU%cr zZ%z&E+U=2QGcO;dNyTZk*zTSTlC%Ol#Uqc)$`APPyA;Gth($=t#qQ+Q_FZwn-3sS` zbHF*^9B>Xe2b=@W0q1~oz&YR?`1d%V?*ET2KJ}Vq`F|+B^p24AE8gQO^Z)&I^8SCC z21&};@S9_p_!6|Bte{U;Jjx<1gYsxjD$>z3VvwZct?{exyF!Fp;T&)dI0u{q&H?9u dbHF*^9B>Xe2b=@W0q1~$14r-wAN}6Dm+t diff --git a/flume-ng-sinks/flume-hive-sink/metastore_db/seg0/c161.dat b/flume-ng-sinks/flume-hive-sink/metastore_db/seg0/c161.dat deleted file mode 100644 index 99c062f672cb0e492bc1140e6ad1c5ace2553857..0000000000000000000000000000000000000000 GIT binary patch literal 0 HcmV?d00001 literal 20480 zcmeI3J8T@s8OOibJMP`_Es>&L6!joQy-a&|_HC0yY0-@f zH)i9;6&v^J48uUeDn)_-a$_JJQ>95ExQLq+Mw-B$`DX5TlmT;yPDb{H;FWLKueLYxAY=UanB9-YY_U_e0=5FS0=5FS0=5FS0=5FS0=5FS0=5FS z0{>eT_+Up8R<@sTMEAjjBRv88HmdBT> zEP^8Q*wbk(46QB5@1Qj(l7?ARg|&TdXl=72u#2sLt$?k7t$?k7t$?k7t$?k7t$?k7 zt$?k-e|-hMgcsN`{uA{kW`4H1_xWEwJe2?U;VPT{;HhV``qYe=a62rre>@uhCD9v&mlkmp)QuvDJIl}&d{NPqrjddtXP}3?-z0HL3njk>)5z1HUweV? zsrJ7PGlEaDCQST9!8`LXEBJz@DeWtd=K{%JM1y9hqQwlijSLq_{*sa563JgSGOUyQ z6(hq2$?q5$E|dIKBf}=i@0u9eF3DdrGIU7(x{;wv@;8hOJ(AxuGE9^FS4M^zlD}zW zm?imJMus_(|JukfPxAXlh6R%U#>lWp^1{e)jO2YI!*P^=Pe4PF?hX*m3RpkA{TR5kl};Y(F!RZyW}4)Seb$buxJw33f7 z(J;J4Zo&t-owgOQ6|fbs6|fcf|Exe~9ee=yU#NdZ{S@^+>a$#@{)758>Sw6$qGqUF zuI;~}K16+h`g7DJuC0Hdeu?@K>WiprT%B)G|A_ht>OItJT*<#tze3&R+PQ<{2H3ss zBt~_)!anLluAM#3z&4IQ#_>&fUi^)3-1*`chw^_6TWm`pwvOn`$ar)($p3PVrT&8( z0p#3~`Ct1QJ`wp}mtQ0z|LgL{h{*rC{Ba`kzb?Nd^S>^?40CvTr^}y!c_V%W7L53l zuxP}ef@4PfX*h1gpMfPK{wypT@vCq`@Ik)_eRdXp^gSkI9##Y&Renm7!Rh#vl?Db1ApIao!$@T($Cf>L$-i*QcxG>*gArz*Y&Yl08Ug!TKB$8#ReBj5M@(hI1v z-**`<2tKMp*6&lj9-IZZhtt~zt8EwCFH9(?1!;esGe^Z)&*b1?Df_tpYKK3 zkbIVel&bbm7T~hr>!9(d@X#{vRP`ata25F`p>;}Q zCBFjKBp-TS?5X~~jtkcXpJhcFm#Y0cb8rKB-T%+$;29%+4YrK<^W<6NGd~MMS|7Y$ zCNvye@*MJIRL5a7(tn@8cZi34Mg1hLt8n=GoWM^LA9;2Q9xXjuOMYT|VwO;G{4`F& zIvVuTC-MuUCn0uhD_|>ND_|>ND{xc=I+%@J%*HNeV;57gdy8jdcZKWJW7Iv=9j@I? z9N*>I`3}c8)5dWZ=i86?e0vwQL5;a~cer-MR+o=k`=}35w^47Qu5j(_qBf{8SLZS6 z9_kM2UDQpkBS`PTS+NG{3zA4M#!(m;9sy(sfPJ2lQy8YuI_F`574%@XE%U!FKO^(MEtzb-!~^M6A9GK?c-o}7{SKcR8O@{+Rt zCuII-x2N#$$f)uU*_8QTm%k$Ozb=1O=6_xOn#}*Y{B@cCb@>}I|LgM4$o#L#w_KV3 zb@@U5Pl`TEOo{USdrId2q~A0w4=Y~}CyB`ax_PxrME=+1dqm`aU4EK~{IAQ;5Rw0N z`B@_Jzb-#VME=+1=Vkua{eQkL^S>^?A@jd3e;Gvn*X1`seQeA2hN5GCw>6v;$iKTLD{vpP&LiF3(S4exAboEHeD0 z^v=HB&hoQJ?7e*)KSbR|y@k5MRVStQ&^3+wB#*eOhkGNtIL5v8aBtnme5SjHx`TQb eb(5>;L5QBh)BOU{aE8RFF4v>-fBoS8?|%S}z^?`X diff --git a/flume-ng-sinks/flume-hive-sink/metastore_db/seg0/c171.dat b/flume-ng-sinks/flume-hive-sink/metastore_db/seg0/c171.dat deleted file mode 100644 index e98af2e396f3c894bb3121b70f25a39415188c23..0000000000000000000000000000000000000000 GIT binary patch literal 0 HcmV?d00001 literal 20480 zcmeI3O>Epm6vyA#Uhi(c+a%j0P0}<;x1XW*I^Ok`f_!N!RR~(z1}NoY*WTrTgoFfv zBLWc#Aub#WoVZ1B-~dRGxFAlEkT^$zb5(IbLV(l*mooEab~A3WJGQtKG>-Dt9{-+q z{9kW8|9K9q0|24FY{_A?G0LH{Uv8u8_r>x+bI;8ybk7^~w*WBKfe3wby4`oVv)|rN z{`l-NZDG-sXW7~Rzz)l%*4=;&D0F86_FWTcyGEbDS+c`h$?IaT zL0=8plPBS(*u$m^*gG3P(9oG9Fd)2ULK&6=hb-alZmJ~?pmKr8T2EASgU zO8fUU#`Ea;>E?T{27JBa?>~lfv^m~$CKG#Fc!Qm%mi>L%{tM9TI_F)}zw6`v82oy5$V^GY(8zV4~i64VO$q; z@k20~i64d%^3|Z)j{LZ(xE_om974Vp*BVYGb`?B13R9W*Wth&yAA^}JyfFZWGx39D z7I{8r=PKvy24F4|Un27|&;Ee)of8~`BboSFSde)($=L7sqyR@V@yE$x7TzqvQYL;9 zmStWw@ks{YSSJ27DQDr$5jdWSpMeuH-%{y+f}G64n}cvF6Mu+=Pss_Y3&LC=;T1VS z`3+bm;Z-?7b;{;35_(0f4NYt+h<@S2=ZQ6&tJ@VcC!x{9n32ye&womJcl=gekzmtL`UOnzXhIOXu_~cR!WS{YX~wOH{-_dY zmtNxM5V=jiLQ6570p0%RU=Nb;W!|MA@@N%@UK%(8tT^ozFK*F%w!~3A_F{I@_7Djh znBc~KYjw5JQK@IbFbSJHA!;?8mfyL@_DmRsFv0}-5!C%krE?BF6S|L`_81)NX03o$ z;NK`v_#Rw<{HGXgpigw$I~X@vEc}k`KeAZ7%3|R(i}pG)7crJt%-u!)7V=dVb6eQH zfpHCEnMLa!#%&gh3&{V?V&NlPe}zTsHZpCDRg5-^<_^Y9jO#28Tx5LlT^7xYtflxZ zu9w)laTntj#tn>X7|Se@dl;OVs~2s3`}yN`I5fXmYc=)$#wL!RWevtGm|M|EI3u6Yl@h@&|?g z7rg4GO^U+*r{&9p`~Q@@SrGm|Ek7arzu;FDLo;%caQ~l@H;07(Ps>jU|1Wrzzam3~ z`+vc++j&2~WF8RyU+}7dA{i9^KP`WTaQ~l@H%Ep4Ps<+`{$KEPMArX&^Mw2VwEVd6 z|AJSIDao+#|7rPCg!})Lyg3Y~ai}DC<>xyGXJlSAt~cjlMdp=y9wCp&yz2XI9wna4 ztFC9OKzx~3_1_vKfy}GEzt$)@C-bW7-#S1ZlX;c?)&zN6=2d!HlMse-f=WO~12rtPAN=cU>yeuauH9ZdD6LNx5)RT}f^~?bD{TAo)kTCTXm8`kWgJ|9Rr(V$t zXa%$aS^=$qR^XvgfIinwJ+tiR^XHj!=JN>L|C{KmxvRH-!Mz)pRbp|kx6cnEv&HsW zRb$mBAX-0W^W;QwjWfA=2&1eC`AtNe^Po%H`I-s;r;Gx_`Go)XeAV%j{$I^66JGxp{ZWqY4~qJK zTK*K_^?$*u2GV9Z;r~4qZ_OqAzenry{q-*<{J)w%N_hQW^j}$3Dkc2Cuj-Gvl<@yb z-kKr2{?Gg4(}GX`^BorT|Frxu!t4K>uPV*kILP&XC7-j2sQ))2-}R&l*Z$83856+k z|B)9(fm=~N|1vJ>|7rOdQU8zXmAX@HcdAvro>wwPMg2dD8$sZ%D*JN~7SXDv8PzpO!x?>i=o^S>W~mwEPiK|4+*= ziTSUTyirKZe|f6E54j@Y^IwtQj)O|9EZEv&{wrSfDnX;Rd%g82zZS?@yxv;Ps_z8d z?^n-K=ScUCuUE7JS^=$qRzNH8pI0D%k9plZ`scjO*6kgP=$3muN08gXtZf!^8yMFx zma+Xdi}G^>7H?@|dljS2qCCG~uCtarGJHxH0mfH*nB@RGie` z8Uc-fMnEH=5zq)|1T+E~0gZr0KqH_L_y8jCHv8OX*z>VAPvfZ{b%W1cj6VI8cT(9f8?33)7 z+q>6ST6bEVmwGGR-fsJ~l}>BFwR8LWO8vFH{oc*$?(I&mdv~|D$EKN46<=uF>RZMi zpecjJ%2p8^xJO`MuoO1O+O$K|CdT)Mz!uI6in%XxZ;Z+K|#CvfEqI~F<(H;$^5`0|sw z5>e%a=$0IhYdTBc;{zP4xCV-@Mr0W0a}3WNV-K^lV2zY*rp;U$Jt5<|j>%?uZ*$Fb z8oe#0>}{vm5?)ZPRSMo(v*y(k=x&$(f_Jx2xx`+ACHbBX>0qIdZ{$mrjWXl`e$Lkm zm%Yur|7(`&?H|uns*Pf$T#uw3z3Yu5yMnr%dJ}IX5)6_qf*e_s&2e&=ZrZWTR9Z<6 z@q$Kvwd7T6#jEJv8_Ce*Z#YeenZkvmk@rlPIsTIOJ7rRAo>Qh}rdu?`C@FKAo#Y** zxS6kAyW(9-q^JB9r%Jfs4=zR0Y=Kk6cGKG~lmsP37TBC**CdFMXPg*;G9?{o%!C{m zNn{Q54Dq9JtxBRTaRC-HzH`y$H;K3DK*U9fw+vh4B(uzn1(MlHlAUFzgQHEMQc8q! zCphJXiynu7!7*p9ny=+IlNWPT@95zSw7N(7S!{{Z&v7$32tg?03f+7W12E~orRF9^ z+R1C?`6*7hV4>sCY@pN4i|RkvRI}MhPPLwy*@jY)RZ>m#Iy5^`l_Ede?{$MkYYIY3 z&pHId5o}9eHNMW+;013*XPe zQs&!Jf9sqbu>Tj?0xp~pR|DT-U<;fOhp_)S54K&)hMnOmd2o)~u+6nbB4R@be6pH( ziBw-|gnFP}61GJ(-)MN;)ILQcGs%gAQ?qy!JJC2kuW$Eu$7jQ+b>#0EnvsK3*YJPGD(7IS|~G??C9db_#<_OUWS*u`?i4z3LT; z`6M-i7dewelucMg@W~7YAz#y0`VI7F1sC<;-k|~7P*zmZlESA?R3FwNKeyT4vf(1)AxE9jxdnijPGSgxpof*Qb!4#aL18RWZMu|+Lg&DXsx zxM!X9ZDp&^vvZ== z>ow1N(Mz($CU0>}@gui3!%p+oW;)3Q>XE5)V48oO`Y;}T z)_6;l_MyLS7b5Kcv*K#tTX0+8ggBJ;ziYW2fUk}-_E~K;U%1k&Cdq1-wt~DCE?+jw zZ=~26Y);&_R;)$JF5JfDKHVOr_T@6RZ)UrlmSvfaI_#HWDZ!ed+HK{_Yg=$Z!0^~Y z5Oj-z@x`m&)+S{jgxs%_7*xf8Fh<=Zs-dAafU3bRta&A`;mJm=@fK1>Lv##RK&~7V z)T6NDo3$;g-GQQ}4)FwP0O;Y&Zm7#mE}Z(pldG>~^}) zavgPaZa^cfHK7I|H6_-rnFRB8IC=q%aJUvh==#Gd@@YhuQ)?jcIJv&XXIVX1e2zLK zHh|LsKwH48Sl&pYlMmrET#L!zl=H2oUe|<6!BK~v`Z&S3?~fjnw=mWzOKhDw)umU0 z{1jb_AWla7z#)37!MNoSZ*VBITg&vVv!0!XkIqpCfXGM3turLg6s}Czo(LfJWsX=x z*%Buquw(PaBu=qLMm%=$R}L}Otdwu8Wx5&o#~pQ8V}L7%<;$do=m5s$Fjo%OLZVIk z*tEXa-xO#^7!?-M-ng1=+BPCRjyhzp#1@KWh>%ig>(fXajDi4|9;NC3s~m!XQe~4v z*6MV#M%kLvR56J8HLqH#T#M_cz$2cEVtE!NQ=tT=_(TDk@vStg>mYpVs6Ffew1$f2 z3tm!y?h8PJrAk0U3SAJ;D6jy562`+iqvacTY1>71f}{32BRiqyZC0QkC&YNKa-4#N z#=#^QZnjxgupB*0UXS>EQKf!s*|KoG|9~T~_-n=w!RCL}cm_8A)5deK`5!hOg3bRm z<3-r~KWRLRSoc-qMjjpU$0#EJ_WzSUJTL$8KMdIabI9RF&~OMml4B8@ zwEuJ67BasawS$5NEpkJd#+31X21qVe!2YV2}+$419)BS+IwJF~-NheNoQ zL^vRSE`sc$aB4Mep=6L#d%c}rXTQGR!c39gt$m;|0dh;yO`fQuW^m@Ltn`a?yGVm} z)Q<8#L5rKo_TyfVpaX>qa*D(d91|DQ_vHF9!kD|gJ3FmTFS2`5@zEVO{)*&_n7$ix zdYFr32@3ztz2<}DEZ6t@IQBMXFgWU%Lf_W(X%rRaoPf6{|A(AEd|NYEHbFug&27Ur z0(+dkaci%)yIw%bWWkb<_QhM>-m92D_MU~q1bJwfDvrXT1=Q?T3=*+5PKToj7h3;1@3o|zH z$aelBmk#P`2>J-c0LOho4P+AzM!2iZ+J?7Pt2|Hnpr_Wn^;&*o6CrPTYOw)t4AK)l z1aa2JBw}orQuH>(rM}`5>WC2?&rbPHMzd$NEHkT)VfCpTj$xH6&*zJY^ug^Qm1C+Q zsVvCbr*a@N6|Ztj;T45d+IJha9LMRHS#@;jN0=*mM7WZQc;8lXIQ~Azar`Puj6=?% z0LF)er{ctulCPEC5j!I*^?}?kT`}H7F2IxI7nsi(e}`EBzhV3mOuLUwy#bkieCk8+ z0$9ek(D-MKe-{uGzV^`fj>7&&S|euXXVu|AF{^(a(ooFO4Y`FUFsuL4pK~szS^Wpa zQp>65(_oCf5CVtIGN#9qpDu*D@l;*Es=TgL-^g?W+w!e)%W0V)hC1dba<(NZd=gtC znWi(GB9yZoDjUt>?F-rD%52lw09hXViFi5N_RVmo*Y08fQAa~R!ah9gPD294DzXJD zTiy!|zA{9plTE_x5-AF!REB6wTWJ2qB+c~2o!jlbb?LK<8rP{p_uE#F8|?Z}1-icI zCh=mx@lExf(``dzQ-_7aOvL?T_k)S*TrLbUQFLX9UbKaLwny*0!*Vc(G7reW{jhNa zjCIA{vsoQiLLZ0-7Ku_)v%V$W?rrB_wy9&=kRHC-7I5_m_E@h~Hj|VxBK2al>IeqW z?m&sdA#Ry_b_d$7Z~MdS&w`26*#IA9PYo~;xdZPH4+(W}q}oX6w99bf(I=%vH@3^r z&iS5;&UVJeIH*pr=zA_|Ua68_!yJ}EvsP?e+p1Pd#X=I{sw^J>efLyMEhJOY9vLR2 z1F(n*XF06@BxakWvvvlGq&jD!pO;M&cbagPZsiLFOgc%zW%@M@sg4K=79ab#1?S{c?6+Nuv+LOY-oS}e`-6M zhRLap|M&Tf5+{@3vp|a6%RxR1moJ+obm~|eL12KePoMu{Z^awM%OW%|CoQAr*<1>s zsiWup>_Ts|3UNevxgK{f7xKvDOrVQ8963LQI=T`PKiVtBQpUG$)7xzmVo4olhKHg6 zYqn7H&}3TOuu&OGC5-hM{V-b|S_$&T%{x0QpXs(cD`X_^-`?$Y`^!O!6yJWH?;3A( zMESv{QXXYYqLyHlDB@c(kecn97?sq4RJ?#7z-ZuME@cuJ9AwR060_^0Dj-3iRzfz* zela5}7z+#TkM9|Yk!X_jHQJn+ zf$VVAu{bOv;M1h!l{Xrf!$ZZS$gso75Gq?DPrQB)Jz+dS;>s|bYh#pg)nPa+jOD{r zZ>}aQKfcP58ZKUfCB;Uh=jRhY7a=#er~r3@;!24TBNtR~A1*O+{q@MQP-|RuEXe0h zX#5Fa9vIZValoAiWlO**%n~)8`TgAomM^Q!lNd-o$72>J@0E{4(i*b7AFlETP#Q=h z)C;UME9QP%ZVn2$tBwI7AV<6}6BH&YAp$fIDf5FxOA?T&7ej(EnVP>ZUhT-H{OJYT zW*ermtB$!qCtU?%_fsbwE?;6t`*C#AY?__nb9;Jj7oHMV9WemcQ5Ej1P8v}G34$15 zIK8Qlyf>XRn~|@5AMH<08;{Z-^%)S!3*R)-u=b0_&p?rU$9M%c|5uEs5fV6Wd=Vn! zXUWc=zD@T}?a=+kF8fb3|Fg_(_<>z@FyT@514Oux&c1rB zaSzK|zel@HWa!0LLy}nzz0m_@#K^lKG(^#u5%j|+*m-|h{fi%e<|5X(ySQ3xT;z$e zejQK<JyM1_e+HyX{?D3r`mB!>(Vw@8yWUA14=hadX%BB}Jcqwuw1j_D zO@7gz-zqA8q%fakBhnsqrs@(49DJg5dl_`?@!%IA8>8h6!R?tWYd-FWK-Sk}dIVyhedIEOtDe+-r{uoh?s zjM9UpVdA`evL{FMVyvv|FQkw*F>NBJc60Zkp^#OuY9W2>$T%_(GP@jvCS1ORMlfTn z;1qls1%q6q&TVBdp(m&IFZa77vML^FL2_V@yk#ye#Lm5dja4VFup^60oeav76AVA z*Z=+#1NQ#{dEeOl5WGHfhtD6I73=?p7Klxo+sm154~`_-x}97%({-_`z*4zcD0$8L zR=!>+7OC&zFa7w2JG<%Sb_PR*)#<`2GR>CRvQlX*E65yXQ82fQfAYI^x3kl`ySK7O z;oFtnZu{omN~_ztgZcI=YxIAAlvsZER%idl?OUOkt`POdTYH@wH~Q_McG1lE7)dUg zZ7`#Ndm>XdBRg=)b}}g|CI9v-v^rumI%n7^iOy!ef!IFJzey0EU*LUs=4NXjiy!Ru z5h8vV0ik0JWSry18Tm&EQ8Ev3K6wd94)=d?CDvCf08N|yTImCgfJQ(gpb^jrXaqC@ z8Uc-fMnEH=5l|w4_5ZL^AnbpC{l9mnzVaz<|C5zR8?F8)zxkp6B@E%<8GWn~&!A;x$gLvp)8Uc-fMnEH= z5zq)|1T+E~0gZr0KqH_Lcz_VN{@driukHT_h+w*^8Uc-fMnEH=5zq)|1T+E~0gZr0 zKqH_L7y|)q|BnHhencan5zq)|1T+E~0gZr0KqH_L&~zo1{$#WH*<|y#Nel=Jvp?BPVq&JJr*^ipGt-^!o+O*Bw)>MH zdJ;@Pyr@yhMbMMtEr^0Q5!9nzA_#)wA&5xMA`;9=5JBIop6U7P-qhw~pc>|FO;z{T zuU@^XuJ?KtZU8_GKXfFyOmW#MTJ4>6SpK(p^i)TFwXo%7{HOpUt&?JSbNs&Pu@u&N zXT^EPukT>-ietawpG4kojQfB3B){jwZrpd3tdJIv7LXQ@7LXQ@7LXQ@7LXQ@7LXQ@ z7LXQr{4DUs`#;`>>!^1tzJCrVTt3m)lU93ALo$V-dmP>dHOK^nmYSEf1N|7x#_syH zvG3fJQ>KLAg_k@fuaY$2cDn9Ap`txe6$&z+pg zpSZe?U+I{hox3)hhwZSbT1o>6z0?b3}b6v<4$O!zmHQd&S-Kw3arKw3ar zKw3arKw3arKw3arKw9AOx4;+RVkb_+D}GnpyT|_SivOR7C$VL*ucHz1Kco~4|KI8i z|C^U4g1EXJ^ta=8jPjLaZvbDII=*&%c50@uic!2Dz@NOjQ78b$=r!hQWzE%EaeF)q zsyJW>a?bgxuGN_1Xcg9i#1u%2Lq=hQAZFFFjSEJVRhScih9Dyj%0b2lHEQ)XVi2<8 zh!Ge<#DZxvqf%dD%K;{?Ku#P(nGwV^>bh%~bxX6gT0k05U{IXoTewAPLsBp#4)J?c zVXnJuF{8dJLyH5}K_1AXEd7!#+A!6^31)k;+sG9l8i zLmZWbbPE+I6U)L*aY7mtBs9uyaD6-lyZWx@DZ)ZXiV!Cs<^^4?TJ_FpcFXM&L$F(% zfz#S%HS+@NLX5z;I0DtMnN`*FZpJX|31@_5J9oj3X0}t)ml~Ewt7b`af<-Tpg1zFj zS(px`d0M^fu{w>uS2Zi1X6uUvcD-hn+jml6Li|prps?jTd3E!mX0(&JgRoEhhWlY6 z{DwWy9L)DYBOg^bOZBQ*(ru=>tZM3-YjSs5K%Yvfa6+7f^NUC}t1=wD4iu%WQ)wuO zqxQl$p>ScfOvj+5e!(_tj%yqB3i&!$E(0gUi4(9VgxH}ZM`q!aI1*Q-CbG)v6?ZXE zEC)}CQ*hC4Qk+Jqr-y@ZS{&3=$6;=p!Yw}pXT&M^SR{2^8y~Ug5Y#6Q8HT6DA?Q=3 zNS_*kv*IN5DI!Jq)HWy<#aVk`H?ex0sG= zaBXUCePfmO)T{Z`x#^kpd>+s+^@d($4ad_Q-7s3yO+~|0#4+1oxP_@{uD;kFd^Oz{ zM92zhf&Z5UQs3jdW=LIumjDJHzzZlp!gtRw;6WMX1nBtU`DZF8KY$|23s6Ft1^hvJ z;tnjJd=<{2JWl2KA7G&T7%C{A<#HStzFhx-&WYWkniwM{{6lBLSJVnOKm$nUsjU7^ z9OXsgtGDSK-=PVun#zjx+Ym&^GWn-qI4Wp+-JS;&F zsn$W*@59Q5v-u`n4M`Y8Vkzk3y=9jy({{b{4Q4M>cS*x!SjHd{bp$5+Q5Z{0yq9QE1$KO14soCL=p?YSbG8kd-AR;#EeiA|tU5W7V7m?H06pB%Ysg*s4=?&X( zmp#j@8hWsJ4CJ&GR#F{Z%usJ9W+J_nq(&FBJ#{>sNzK@sGc#Ms zTLd$R`rb2xh&C_?`^BF++;gvm#s(TdhX<5BPs92TbqeWlTD z(j2s0EqL{z48xH};B{!%=5hUk|DNpJ@X4~YfV6I=KYDY{{g!w-+ll9 diff --git a/flume-ng-sinks/flume-hive-sink/metastore_db/seg0/c1a1.dat b/flume-ng-sinks/flume-hive-sink/metastore_db/seg0/c1a1.dat deleted file mode 100644 index 1cd053a6c06b93fae3900e34cfa9f0b31af89bb8..0000000000000000000000000000000000000000 GIT binary patch literal 0 HcmV?d00001 literal 8192 zcmeI1JCoc*5XW09?MPbf?yY$oVDrYDd3f>YF{2R}EMA9=tH4my1}+UXLn4MRCRs5J-?Plt$&ZSTDS%P4*uC~LROGH zY7WPb#?b%wV7n`P-;Y;tPg9pdY58N&uH+o|)2#5HQeNo`dpZL#_M-FmK zJ{^-g@G*H~Xsp0kfw2N(1;z@D6&NcpR$#2aSb?zuV+F^TWZYJ%oK1@DW@+82OeW z_)_9mbo>cy!Rg+_os_X$%Q&mod0gvtqf)(=c59m|tqKV)I6ef#8<`}?#~dF(NW3(O z&Sac$d8s2Jx{aeM#|5uZqvU63Mk;pybJ!gApCv8w?EV+<81YGyWsOuFH=iXu zPQ1=klS$Qc{02NheBR`ll$z_W$qG&qZ!*&$&-Irp;MBl3RhCK3^_T0zlf+k9-852j z{pAMm6!C4-;`P#8f4Lz%O?;Mhs0YpUmpg-Ji0@j7dedBgxe+`&@G3z)Yp%cCIXp*v z+NRhqn(Hrj0nZaJmBN11Tz|Psc!Bt=)!6Tv>o0c$P7|LfgX@Xr`paFxi^Lagh3l2( z^zRASBEC^Iu7{e_zvsgl;;XjB^;UEG_X2o{c&&O|&o!riFNBwg@4Fc97dfYYZw9Xr zZ*+q9qny*f7s0E<%Py-jnREL0=5UtyN^9N7oYTLzfHm>Bx}`?uoc^Z*UL!u!l|-J? z|I~+b#OtnZaXy^>rvXUfJKffJzMTH2A;iSjU0+~5IsH#(kPwfh&ynZ!KaC(IzUfni z^WpSAokK=^lB*QYm(%}r0gCvz&l9XCr+*`@B6x%NLZ&z$!|7kl;XLt`QFy+F)4y22 zo5Xi02qO)rf3bwOh|f%c*UNDF7aMSa_+Hki2gB)Kte`B137HlJXSfCsC!l`BA*fFD5l9wj7&!lpM|6Zj#N?^8l$uYojN z7x*(Mw<)11DqOY<*9LwBAA!d6^R8MB%-u;CjJdIpxPg!rc$A`8qE;oT1%n-F9(m@XJq# zSNh73aOcx#BKvjPcBnycfQc63IgKG~&bBOdv1lXyWq*`?+EJ$lshH6*dTt##^F4CZm%1_KUe=g^&c(_0J{JH diff --git a/flume-ng-sinks/flume-hive-sink/metastore_db/seg0/c1b1.dat b/flume-ng-sinks/flume-hive-sink/metastore_db/seg0/c1b1.dat deleted file mode 100644 index 9d0fe202a0d2eaf1402cce28873e1655454f43ba..0000000000000000000000000000000000000000 GIT binary patch literal 0 HcmV?d00001 literal 16384 zcmeI3&5ImG7{=fJtnT?vj9)PuqlwYWKzGl__9Uye8k4xYin4eS5AN<{1utGSih_us z;K3kX1aDqMyoumR#7ho}2N8)L_2xmue?YXJcV_l$YIJ&E5tM3}XJ)E;eqHla^;AFA zyL6d|9R0xNvxcm@Wm3PpOuznPyzMKV`Swv=^PGM}M6JnAjy|}4|Cif}srKrVYj2Ip zcd@wO&Y$&@my9ai(bJUjG5_wsQ=hU0)&c8)b-+4c9k32q2do3u0qcNuz&cXtl0kJA(5x>vbOhxs4%i%D2_>8wwu>C|SDcJd?{wv%LQqn-7;gLblbX;Z(&mUg;Z zorI3i;kCzB(_WG#gDgo0oqW*gkVkHFn}-A2%{pKmunt%UtOM2o>wtB@I$#~J4p;}Q z1OFojJ|WzKU!oUA-|5>cD{t<}|4&m}*E~G+jf1I@Q9-D5WPg2k{2%Y=ZKvy<(7%)5 zb;?)X{s^D9dmFt@JImK~jvqz%!KGoI8zOw(PjrNL6h$J+B2yxYWt0?ciK!x^WS~^2 zDx+X&!>LkDM(Ki5qq>aJ1Eo$48Kn1$SCM(w@TMbD1#o_5`=VvjDofVif$B9`e;iS(M>W6+7iZevy6f>Ags{Q zDGD!o=JPFjb}?SO2Hhf~z!$Gc$7B@v;hU$DN! z+Jy$cvVI7yyhj(buCdmj@m1DWSQnwuZ>%3-|7gg19IAd`eTQ`w8f>s0gZkH4-)3EA z^`YM9tS_==Q1=(s4_KdKJqmTcW4*$9k=2FLXRMc@wR31&^>@}!S)XOS3tIUJs`LLD zt$KTT;mhy$%)p*9Ou$}b$p57%n18+4kpD|jF#UQJL;f#A>0*ZQs)qbuih{}4 zs~PftDGIJ2y}BX)m!hDj^iF^Bh1@~Ov*f-Vz>wtB@I$#}`(}8G?{6Ei%!@s{qu#^86zWRFk EPj(?>DgXcg diff --git a/flume-ng-sinks/flume-hive-sink/metastore_db/seg0/c1c0.dat b/flume-ng-sinks/flume-hive-sink/metastore_db/seg0/c1c0.dat deleted file mode 100644 index c5b91e2c3fef26113800b2a821fa2bb19fdf1105..0000000000000000000000000000000000000000 GIT binary patch literal 0 HcmV?d00001 literal 8192 zcmeI#u?@m75J1tpC<-o8(9=Lg%K*#}rA3TD3rJK@(l7zbumejVQ7{4=PD~;b3K|OZ zBiT9MS-$vnk+q1Zjpvj2IrDRS3iWo*;(Pr>>5t3OD$A-0<9P3_pmW_xQUDh(^bLUeR^QorsGgwRhxSeZerK=R_$3sS&>;9{Cwm(lE17DT= z%L&7~e#|&&;?Fg_>sjs^8XE{e00Izz00bZa0SG_<0uX?}KN2_`w?@`_v-rTyTvWW> zHRjuiywH)sUdnn9&>7WL@ttpEWNuzHRmj@WgPjG;WDzn~r|3|^1{ee&009U<00Izz X00bZa0SG_<0>2TM%F-89r%Uez@u)7| diff --git a/flume-ng-sinks/flume-hive-sink/metastore_db/seg0/c1e0.dat b/flume-ng-sinks/flume-hive-sink/metastore_db/seg0/c1e0.dat deleted file mode 100644 index 6255f852394ad4f811f21dc9c26e43f74ba1e1b6..0000000000000000000000000000000000000000 GIT binary patch literal 0 HcmV?d00001 literal 8192 zcmeI1&q~}-6vj_XYDdREEu|EpR12kv#etjL%*_P4Q74r^n;DZC#JU(VH`*6y3*w@E zfi7M77Tvh^0bKb4T^9Nf_T-MSR%gP1uH;JrB|iM zUyb4X{9;@Sk6gAf^LdbcR9rNJ#Vf^P#QE{C__y=w?jU~Yb3v0h{u0sTWG<15X!)tiO_q;l@gvn83xJ75)WWXl}I_~l6(Eq5)1Im+K5-9POsJOgsn)qvK9NkrgfNBj*jDdZawo# z9bdwt)ePPRG4|YxgSBq>el!jps>)fidddW6rOspaIx_$)S)(Cbd3#3#W|d>c9bXC` zUwz3=7_-MY<(S&3dq)H2m80;|jCoG^ZXK&f12-!TB0vO)01+SpM1Tko0U|&I vh``+t(5E2ApZ3Q}ca=WLRXb7o2|nWdX+DRaAW?T<3-v854(e}ahX;QF2UN04 diff --git a/flume-ng-sinks/flume-hive-sink/metastore_db/seg0/c1e90.dat b/flume-ng-sinks/flume-hive-sink/metastore_db/seg0/c1e90.dat deleted file mode 100644 index de30561b173c8af6549ade4f3499bfa20ab95f1c..0000000000000000000000000000000000000000 GIT binary patch literal 0 HcmV?d00001 literal 8192 zcmeI#u@QqH6oBEEB3_CVDbhGWT1IggwYA6^Ss^o|&kQM2<=jFa!og8+YATAKJMhUv z9)2Cti^xHHY*h?>l;%Tv@-0lP1dZlfx8`CWSM6slesXjB#?N5h?u%tF8~S73M*Cvy zkZ()PzY6E~B!d6~2q1s}0tg_000IagfIx`?kKgetot`Yc-*3MEm-tVd2>}EUKmY** T5I_I{1Q0*~0R&Qk`gwoNK%x>A diff --git a/flume-ng-sinks/flume-hive-sink/metastore_db/seg0/c1ea1.dat b/flume-ng-sinks/flume-hive-sink/metastore_db/seg0/c1ea1.dat deleted file mode 100644 index 1a4f3aee6ac72569c8300458da83e2a533f54f11..0000000000000000000000000000000000000000 GIT binary patch literal 0 HcmV?d00001 literal 8192 zcmeI$p$@_@5P;$9#5Tzj5FBzS6c0fN3WY`k2@ZoK!J`O{r{F1g26zg%w(HW3L4qZJ zlik|8Zp}WoleLH#z3fO_OkIpasK+^-Kg-cbe_W2WveYXV>4Z&257+hA(R%!4_~`kk zqIbDO|E@PPhz{}lYSEpUZ)-Ug2q1s}0tg_000IagfB*sr{G-7BxG}QQFAG1|iJN6_ z-x})8LYisIU?+Li2xy&EPlf+{BZay7Z1+@FZR^4I0;V!6D+ia+ro`1JhzKBn00Iag YfB*srAbl;%Tv@-0lP1dZlfx8`CWSM6slesXjB#?N5h?u%tF8~S73M*Cvy zkZ()PzY6E~B!d6~2q1s}0tg_000IagfIx`?kKgetot`Yc-*3MEm-tVd2>}EUKmY** T5I_I{1Q0*~0R&Qk`gwoNK%x>A diff --git a/flume-ng-sinks/flume-hive-sink/metastore_db/seg0/c1ec1.dat b/flume-ng-sinks/flume-hive-sink/metastore_db/seg0/c1ec1.dat deleted file mode 100644 index f0b7823eedab1831378c895a40ca1014aea49cfa..0000000000000000000000000000000000000000 GIT binary patch literal 0 HcmV?d00001 literal 8192 zcmeI$Ar8VY6oBE^iEWY-5F8Q&L2(EKif{r921sxiBqzY4P#gg+!5QEb@Y=3RHwFoo z{7v@OzSr$#pIc=uB1SJe5*Je!;}Gg`PUp{J)Y2c9W34Rp%0*gXlhNk7{yG}lpN5Z) ze=2&HOLXsgGySNE-&cvQXTGiESRjA^0tg_000IagfB*srAn=a@`@_b_O1~`pU?*;t zy?tw_*9&Q;M+Q5|%Y%T%S@~4>&o|ODH=k`^i>f_(ul;%Tv@-0lP1dZlfx8`CWSM6slesXjB#?N5h?u%tF8~S73M*Cvy zkZ()PzY6E~B!d6~2q1s}0tg_000IagfIx`?kKgetot`Yc-*3MEm-tVd2>}EUKmY** T5I_I{1Q0*~0R&Qk`gwoNK%x>A diff --git a/flume-ng-sinks/flume-hive-sink/metastore_db/seg0/c1ee0.dat b/flume-ng-sinks/flume-hive-sink/metastore_db/seg0/c1ee0.dat deleted file mode 100644 index 1a75cd6bf01fa524e7291c29880aa4b16ca0207e..0000000000000000000000000000000000000000 GIT binary patch literal 0 HcmV?d00001 literal 8192 zcmZQ5V_;xl1Y!hWLolH%RC&1ce<*+p5MTrVn=YWXeB(Yi_ee-uG zLn+a diff --git a/flume-ng-sinks/flume-hive-sink/metastore_db/seg0/c1ef0.dat b/flume-ng-sinks/flume-hive-sink/metastore_db/seg0/c1ef0.dat deleted file mode 100644 index de30561b173c8af6549ade4f3499bfa20ab95f1c..0000000000000000000000000000000000000000 GIT binary patch literal 0 HcmV?d00001 literal 8192 zcmeI#u@QqH6oBEEB3_CVDbhGWT1IggwYA6^Ss^o|&kQM2<=jFa!og8+YATAKJMhUv z9)2Cti^xHHY*h?>l;%Tv@-0lP1dZlfx8`CWSM6slesXjB#?N5h?u%tF8~S73M*Cvy zkZ()PzY6E~B!d6~2q1s}0tg_000IagfIx`?kKgetot`Yc-*3MEm-tVd2>}EUKmY** T5I_I{1Q0*~0R&Qk`gwoNK%x>A diff --git a/flume-ng-sinks/flume-hive-sink/metastore_db/seg0/c1f01.dat b/flume-ng-sinks/flume-hive-sink/metastore_db/seg0/c1f01.dat deleted file mode 100644 index ade0cfa8c0057ffe866461f0bec65f6da269fcf2..0000000000000000000000000000000000000000 GIT binary patch literal 0 HcmV?d00001 literal 8192 zcmeI0F%H5o3`Lz*YCEzc0b5sCm^cJOC(ghKNNg}LasqC^fjTq7#tpztNR^g?n9*|; zH~w~F#rhy$iFTRuycl&2e4V1 z1a6M&wXxOIJwAH5%k|oKk+AcMA41kPX}F*E^sX8@EHMElzyz286JP>NfC(@GCcp%k z02BCM1a9sx1)SqMmFN2*3~*JQr>nOu`;|}>-5!n%uEDDY0gY94S9#7Cuq#%%D&C52 zHhRcr0sC-RbY4iop~NE=y$;M^TP;m21D*IXh*IAKsrFs0v@f$*IIC-mgfLVg?AFw$w*=AXcn{My)sG+`_MDgI&{On?b60Vco%m;e)C0!)Aj rFaaj;X9UC(;J@5rNx4h;o$_s#VU^|1jOqu{HPw4?l0U-d^T+8oI+Tfu diff --git a/flume-ng-sinks/flume-hive-sink/metastore_db/seg0/c1f11.dat b/flume-ng-sinks/flume-hive-sink/metastore_db/seg0/c1f11.dat deleted file mode 100644 index 15209bc19bc079ca210546093a645c30600a014d..0000000000000000000000000000000000000000 GIT binary patch literal 0 HcmV?d00001 literal 8192 zcmeI$Ar8VY6oBE^iER>shdCrD6o){dI070-a2O;f;0Bxm#Su6IIBo#1?YeYhkYLH* zWN+4G^s*yyF?BHxp&sXS{wzl${c$n=YWXeB(Yi_ee-uG zLn+a diff --git a/flume-ng-sinks/flume-hive-sink/metastore_db/seg0/c1f30.dat b/flume-ng-sinks/flume-hive-sink/metastore_db/seg0/c1f30.dat deleted file mode 100644 index de30561b173c8af6549ade4f3499bfa20ab95f1c..0000000000000000000000000000000000000000 GIT binary patch literal 0 HcmV?d00001 literal 8192 zcmeI#u@QqH6oBEEB3_CVDbhGWT1IggwYA6^Ss^o|&kQM2<=jFa!og8+YATAKJMhUv z9)2Cti^xHHY*h?>l;%Tv@-0lP1dZlfx8`CWSM6slesXjB#?N5h?u%tF8~S73M*Cvy zkZ()PzY6E~B!d6~2q1s}0tg_000IagfIx`?kKgetot`Yc-*3MEm-tVd2>}EUKmY** T5I_I{1Q0*~0R&Qk`gwoNK%x>A diff --git a/flume-ng-sinks/flume-hive-sink/metastore_db/seg0/c1f41.dat b/flume-ng-sinks/flume-hive-sink/metastore_db/seg0/c1f41.dat deleted file mode 100644 index cec402499509f7c5e789f07030cd7bac5e820973..0000000000000000000000000000000000000000 GIT binary patch literal 0 HcmV?d00001 literal 8192 zcmeI$u?@m75J1s$iX;_c1L!EBqN0EeD5w~L8j$E{D4Bo_7=b2C!3;13ob5PrLO`NL z-%>92of9Xovt=tHMlahE7gHDG5bAMG`_Hm9(jSL&t*rIRMY>^<(ZzNAwY4sP9zXj2 zspwrUF}Ul^jG{&SyjpZ?=KETX0RjjhfB*srAb;BF`UDXH1Q0*~ a0R#|0009ILKmY**eh`?*%Fmv5F2xIoAT5~y diff --git a/flume-ng-sinks/flume-hive-sink/metastore_db/seg0/c1f50.dat b/flume-ng-sinks/flume-hive-sink/metastore_db/seg0/c1f50.dat deleted file mode 100644 index 1a75cd6bf01fa524e7291c29880aa4b16ca0207e..0000000000000000000000000000000000000000 GIT binary patch literal 0 HcmV?d00001 literal 8192 zcmZQ5V_;xl1Y!hWLolH%RC&1ce<*+p5MTrVn=YWXeB(Yi_ee-uG zLn+a diff --git a/flume-ng-sinks/flume-hive-sink/metastore_db/seg0/c20.dat b/flume-ng-sinks/flume-hive-sink/metastore_db/seg0/c20.dat deleted file mode 100644 index 6a51e91ebb364f7f4452837a914b8a0130176381..0000000000000000000000000000000000000000 GIT binary patch literal 0 HcmV?d00001 literal 102400 zcmeHQYmgjQb?)9>?asdR@B<7aU>7M!Dw%S>A0X^qX~*)$tCh4`*>XZy^%GQagk(ts z0@yO-0e^y(N-9NRgB1d)A|xOR@*)A7q^N|DV1-bGJStMTl7dPg78ODg1IanJySJxj zdT#e=?|5cJkISu|p1pVOeD~aQ&v(wb_X=+i1VM!V@ujfqB)hg>lF!?p!TtZmua3t( z-yVcd&P1Q?O#WyNzPypV$^TcpDew8!7uokl&o}FeoB8su!t?7TK^TL7x8;?EXK|gm zRre@TKdup|5vUQU5vUQU5vUQU5vUQU5vUQU5vUOuNd)E&ANsIx7Mh;$`yIk|cz0vh zk4Zl7_+VlZO1@P5V%y`J;l3fP!hh?+5n;bLBkYHJVX7g#O!y&7VlTfbNxsw!eba2J zfp5WE$$|e|*_NdA?_`PlcZt(L7G5Ac|M0zs_a0c?w{l>AYi)78b$DZWb*Xh!7DQ24 zcvO75XhXB6ps~}(Px!$b*WYw9fH&Lrtgp3Na5Le0CnfP#)^zxrpAO!%5}bJLnb)yL zBK*PR@zaZ^z35Ny{R@wuK62vtsk6a7$IrZO>Fmk-j(g6TVD;>M_nl^C5Ya~RH=9qk zzy5wdxIggszW%f-d!wi@6h9fGI1LTX`ypc`Vae}-2w=G< z;Mhayr)hspbODWkcJv@Ntqo(1i}7^-Fh0~9W0fqa_@g?0=W{Bl1{cMdDk!2N=&cu$ z@qS9UPS^?KeSLAyN_LxErxE?POVe>F1;4U2H1IHtmhz!8O0;ZMu+{X_R+ z9=detM5AROn$xgU+mmFbIqjxmDR$EjJ+y|<{Ly4t1M_El1^ko(vjGend|it1JG}SK z*8atebW%ol-)5e2qg&z6-#hw;-<3T4=<j5ag9U6t0An_wOmb_qM$CYb!@*F6u9fx;Bn^aY~2sipCrML7#c;pzps&g@{;uk3=rNYzTalAgV3pPi3?#!X` zO_wT82bDipCdz0rwhHcQ-8-^c$mRBL6{kW;@PO0^bI`vGy!?*cdd9AnGN?o+w&|Bz z2V1L4!`HXyNiOzyXj{+8OX7Db4?j95LY?0iPE-;n$xGsQFt(Tz`O6(18h?7D zQC{#ss1S%@K^U{?`!3vB9*&kaFGQmE&d1%mxU#%>IO}cwN*oZF?IEGqrDftIynukx z5fDg20G$yTG#{eBM)wZBi~ct2?kZ(k;2$9q?E)%jy0+cnuA$N8t}1!hC(gtS_Vl4O zFypuD#|BML3(yvB6Eqr=@2ZlAU*ayz;1`)?!{Us^U|L1Pw+o7(scKHgFs6ATx~k-% zdoEU73A!DME2l0~_tACN*7V*LO8 z>^dbp0{s7T;;Z2La^d}u3ICM11XJNdq6@X}7F*GGeBkWT*E0N{9{;bvtTa@ig>gRd zQY>J1FSqU)KD!LHY6-iHPt&U~|B~oEF6lmC-9)`|08mo|IolnpbtKw=L zWrR4y-n&M%Z*{&CxC&n`3E$^(SE&m_y7@O?W~K8rV3k;WNenTM4ZTY380HQYgRcjx1hgeqrQDSD&}=%c ziEq*FffP3#V!X*vSrncp+yG64T$c6L{?_PvxZ8fO*9?3oG4kWzTZKQjRmp?!P!9-) z%fXF=4Vw$`z+PWl-nXx{HhN}w{;y4XH~zJ~$n;1(pMO+bg!sQ8JPOSJGvaL! z`47cc1M~kg;Wr@i??Dtpn7Ucq3-SNABHQ0)BikSU7Q5bq`2u4%VMc)XW~>4DKiuQ% z{fAe-lEeS|7XQ!hu^#eyF;_mXnRz`Q3yg!|@Qx&Q>4+L76K6gaNa)(&rMWortLy>q!$~4_qFp<*e|}|{8P(Q4nhzngGE?%HE z4MSF%x?`h9#tu|Rb#?f3(`dTVW;ZV%USHmu(f`BZA1|_;$_W6V5r!`K$4##T4VoK- z1n`q>c5@7HhX_cPVrUZdk|g!+ugJ;JCM{t4>#nXs`mn=94+KWGVG)(p5wc=`2gK72 z9A+-2f0x+JPseVgr6c?I-z#r)w660jp{g(l81GrF?me*5rKG+cXGqNVKpDHWOrP;GB-D-!30lTP zR60>wQQ0IJv$6`qpM{f=X_=lRA*1EsNb_Bgy2i=vnh{c9p6x)Ujs-?*q`(}d=>#&S zX|h*nn4u-m+!rJOPvrtQLh8$s+Gl_s>Wk)LApy9M#5oh?(4o8x$<$zy2frp~gaA+9 zAqbBF|Nk_40p=dS>&#Q)d*Hgr`2Y8i`A>b4@&Dc8gTVjai=2PrJNW7Nd(b;D_6*h_ zJ|b>Fb-Tr5aD~_MFYi2;;s1Tk|5qd(RDujH8qV)cH0ycTqVIX2`_~Q}Szlg-^vv%& z6HqdAOOn~}?`ba6JOrc;xX`l$Mo5QhQfNw&#W)8TfgYr!dzR1Aq)>^5n(6@OAZ4+T zK)l!!#1Yb?DO2T`fi!0Y3B+@;C7n?P9lBJu^Z-T+IM6t{g~s;-New|g13{Hx)>SP9 zD!2R)oC#qVD5mZzX3AHSbfvWe9Z{%ZrK$-zjqJR{28{MNp;6UTPxXe0mMlR+{z@F9 zYD0dwyu?XHaxZK-UaU47x;M5oFANFWtKHbr^;V9XV{7Ou*21H&hGK^h|AMSj8CKo< zR%kc%&^EIJnC3(wp_e4#MUvT}uz*Y_9-iJ^OUsEro~P?Z*p8kSu%$+A=+WE@ zB=in-qemweXtxZ+HQKIxAkZd**eRfK6WKFeU9%#Vy;axkFUH9K++}vXUAO@J|DTb` zPd_T|fh)@nm}1!hlPo)6;@`!e124dx!f%2XU>tM)#ZR%!|J~x-qBfndZ{3^W|1XA+ zkO0PZ=L>fQ%z07Q|C7YS=@yU3^cYryI_o&i8q{b4W?k<&dKZhJ-ff3TZvtBd^5Hlv zhZIJpY}E7G|TTm`3CBKOS35Hp0Dq8 zR=_=szZasHwkA2qvoyD9%B}&n3tQ_AKiBsh)v-5u*2KxuB@p2VP)&DLh^wO$pywBe z8UDJpZGMP3QV*>_@~-_~{hL&z^o1^@?Y^N&_;HGX# ziofT%p5a)hD}gElfatC5*+XK6dlmvHn}(uEE~Gs4u5z=}KZ>1Rszlo9PG|+539lV z2|66p(|QkBF~h|Q+w6F%1#L!p*He9Ows|M&fKnyWHv5|4!WVFrjF^e8Zf%=y$|4~pNz9Iu`|B;L>-n<@m*)*+rH`Iz(6v&zj*m$1`Il}I~X_KiT3eD2^9ZX~r? zb$r+sHQ*|{%{DJ!n@g2Q+w7_`EQ1L6BO!<04)oFX!@|cQ=Knst&b{<9y#`+~Pu+Wy8*U_5{7gZO0fo9ucUUMIebe?PH|_u~z`A6v(J@%Q8W|2IcJ?Z-a$n$KnU zf3*JphOQqD7IogMgZ~#wXQ_URD%&K<#S(ibtljk9ngTC0$STT<@f&O(suDLc+ zmc|zY2+6!2*aoGB_c1n9%9KclifROkB{82MM}N?UcbOI>ogSqeWjh_$`XOn$2JAY` zG$iOhzDUp<5u`E2mh&B1U51Fy#?m5WRl=rDafy;Z^FpW; z;XIG=o&>gygNtj6kd(fv*<=L!rjjs{+(RWX1P#Y<0YR?CA<%pxLFV%^Dd<=DZsS>G9_X92UBmsP|GO(oQ!#9Tv`M{7$F2VM?_i zR_~L$@;ZH#)Vt|RFf28Gr{~e%=j_@1zaWR7myyHIKFxdqyYYVd8N8p~hxd(1yq{Xb z`^jB+Kk)_33m89-`2u4KdJ4n`@KebDhv)dR#(y@*@c*5-)Bvp9&X?HvAbL;b|FW%t zPSwkeg`3$}C{rRG3!!ZLl8igEai-Nw%6uW^Cbe148w;Z}>q&|Rc8Sg(3$*CI9C%F^F@Njf=jg^(GnN)M8X;_Es?APidLtP zw1nX}fhF5~k)Uy89T*t+dCQ5!<4h#Vlt@Lwkqodw9ll7=I5MftmvTe`k`u3$i31!K zo+Y!C{N58kjX~>y3j`N8lq{hoE;ErRQz8`!#Z-WzyL^$LF=$eoujYxwwUf+*X`Yts z^0fpl&B4&Y2`_u4L}E%nktkIn6$vL$!6(2^&B*7(Tl;kTFCt@~e>t-Exzl(*djsCj z9K-u*5$_xO@qY3fct80Hyr1|7yoa6t;ePC2(HAhL;Jt7a@8SHvPeiSJ!>iY&4F8YT z|6kwr8@7o5m+FLg?ws0{H!7|jb1DpF%aRMyvuLgg0~}efUqHE0aWfkgWlE%@Le*h4 zp^O>)JWWi>4j{Ez&J&3tGpDla?GU@tq{}i4!;*czqNF)8OwEGWA1vJC_ETX_IG%+q;TUpYDQr8O-u*{&b5K$N zrk=pH`;cI`0UIN3;Q1!iGUcuV`>8Le^Z1AX2kM)PA3@$1TzQ}ha} z2L}Nl%b&4zIO9!G_;X4Z(X1czMqvJx$lhn45f^~}vz`2=zlyBB@lIs>Q(waS$v5Er zw$I@GgopRzAHn;vgLp4Kg7@kEe{X*H#CnGR&kJ+#`E^}w*dqQGR>=6qA3l(IOm)n_Q&jGPXo2QaAi|b2k5w4Yg8(qf1NB^0QwowcX$}q& zfR}RYRK3C~$3ZAW2B2%2j-{wu>B69-=QLp82~&rcs!bzkfe&Qd$~LU6g}oBE>W=0_ zA)&Tw6SWH34o_+s2x=N^Q{pMOhZSd%p!r6KsO4z8*g;K6fF1;ZY_M9o<7-Mmwld98 zLjq9D({?c{rvNyJamxVIV6S0I(fHbq=8B=@r8U^1_&rDV-qnIlv-ctUu137kQ8SPs z>?png2iNfxAI&dB2Qa%tO@UPv%#5Z;;2;2G5-8~k5D*=A#oQ{-4$B+#gXgF z>&pjLH`w{lqqRuVgcSiwK@xcvMSiXx%zq2L0COK_zJSwsol)@GV6lBT1;79=z%Sr+ z+Y&yX7{}|_=S36v{~7Tpv~#x@?@YMkZI`qA|M!{y)8(jhAjTQCBQX#?+Xok-&Yucs z5q?VYVPlbP<9A?feFSF##e|zgcl1x5DO#5qnHqGtQ^?8v48PkCHL|mpk!`PSN9bO z&Yj?nYnoqx1m!6{logx>gP>{*NF-jFfA73#7G&_a1+5 zJlhK-#pWw`nh$`4-^CpKD%iIJqY9?#I-X+l?OU1)fQVnt03N1|VT$Zq*U&;uf%C$6 z1cAo;NdR8Rv2Q`AkS&Zsj9_Np!V)J)LM>PyOym8Ol>m(=@cdnt1vQdJhOWT@778Bx z!Xc22K?b1#nf$qi=Ua-ycY@LwK4sZK;}Sc!G@60Y@3gwOf7E=RvTdp$ zgM34m#_@>|=E&eE`94ig_CN;tzE2v*r>t{mTw!OBx7L)f7^h5(tsVEPp4oZTJFy4s~DH z_3&4^eFAg%=l&G={Or$*w*&v*i}#Q+wio#Sr}2LB96sN6JKm3f53ggN!c_yn|KXbQ z|J`SvelWxT`@H_Ylo-T3@%NXbb+CE&`2nmuf-MW|qFzk@R5=6K5>!bWc{mZ0C16jdaU! z(2k`PYzvS8$j)>rRU++l*gzJ{WS_5YX*{=73z9Z3=I~tDwt$WFVOgDv9D2g_PFC3^ z3$~2c6rV2=G#2bwJ_OVYwgpJwJ%}@KsS>G3__7tk7w|=bMm5>~WSY=6bq%DD zuL)@$5H(a_@6v+pg%WSVljs2{RU-9(SY~KyYQgzmG!ICr79?%H(mnmD(lB*8LdA1` z`!)pHRD~ZUqWL%=4Z_#dg5$Xo0H4NAFI6J#bi=n?PgVINofhpb)q=RqGh$w+S8yxz zZKM}p zjsO3;`nDg=@c%xq|0ngagY?_OSM?Q3lg?Zq3#w~IFXmK{B;5M=!? zJbwDfiQ}iv2KO94^SY(8C+|D%IcI{^v-jP1TD+Bk3xwkM2|sw_`kPJ$P+;4h^|e+D zZYGbPUOeq-?5|MjqxWjZd!^&Oy!jq|pHXRSM^rkE9O9=@UCG%yQBAlX9)1?pgBy0G zK2-FG75Y%|%WyQCnh$P3i|+=cc^=Hr@hsiv?}wTU&y%bXWlE$f#j#S33(pDU(^?tiWT$hZB2(n3BBO-2O8DaB-`}#prB78H8t}vc6zB2X{W0W zYyzzZTbW^Bss)LlUdZWmYsgLycovxlx-aRHUU2#YjdzpA)Wvi!(oN01Ff8HwMB7Gs z8C7SYe#Nd}s!h=yO%D}r{Ey}vAmRE1?nS!1bQGggGFd|nBDlg}a{}0=UJvDlC7)Vb+)VkmLGhu))-&LS`G^mqT+! zkYIZvZ*HkT>jyDhnb+3$JqffN-)l?ri;w`kk_TV~59lBykO8Q}se+avZDr~`3BXHv z09IJPH3-cYMNmD@mW+aPX(IFU`{p|TATs&6+weN`UA#_z7WejRybG^WNAU_O1NQlw zxWM-R|2p~t#y-sW|9|5>@c%!K;JWUu55D`k+jJv=lH zPK2*3XsAPb$kt-t_$+7sW`iHr4l9x&%UmOy<`6JoZ)49;_@4fl8_qH#qEv~L&%z#O za7?zspE7A2yi^O4Ht)*O__j@_2z~2xteEG-@KsaM43*EUXrXozfVUukn6HrIa#UlN zvGgLU5Hc{B)STt(y7Za&moB}$KDP}oa(WrDh^2;&>faOLV|=jv<=rY z3gW6XM+B)`da0VVIK?}g+Yb~0$fe2pLOkzkQmDK6!mImuBxHAdO z@1Ri$1V*(i z%`*z}ipYM8?XP0||5am9a^^dX|9_HQr&#>|vv@!GPFxo-@iFnsu>SuPJ{kKH{8W4k z-UI(%h`#pG7f%05hX40@{l6Rfp=H>7hgdQLokJlkRU(}=9oy323-}QR8tE?8f~3vo zyZM))R`B`aFX|Y^sU}Dh3Cd`%tbV~-^0X8@XELi8rY;#ae_bVwbdvzQAnXw4;n2rt zVW$tmTgfKO(APD|aQF@j8u=$-cTu=exSs8HmNQL$KakfXq8rU~*@-uq`lG>vGLjt_ z9P$dWUf(hbb`hibBuM>wAy27{k{jQXLRgbnu!k6p{abEK49A}bJB~U#^ z9t+tF4fq27V5X7oQY}c@oD;h-)KTJJp%%K9QLyh6jdYU$ydu0vcmdOfLqQ!9vJg>+ zZcUXpDQ7iUmO3C6>DFLFLLCfGwxRF&u@9Q_g4Fj{+2QkGMc}53)u}gP+!rehC!4$u zPn3^h!@l;0$KPLo=DQ%`*X{AsDr{{(2mqtNaTtIAdVHcm^IbR=Y~y2u1xryQUImA! z!;Dj^L^|Uba7wdfg#0$s_;#rl#52xZPV5N0KXks0zo;*x1}p9{YFMY6aCRy55I7`1fogM~?Y?%cqB9TxD+rNPve*Slm!_WN>-p~F4-p{-d z`F!Io;&&ka|4Mu^`Ahiewma~C{Dk;-!2cDt^FP}`K-evOBC7Idmp=Qy9sD0e7vk=n zFLXBKye1}v2D7cC=+%@YUun9k6*e``2hWitIg+Evu#K@)8k_`9ImhhPWB<>vAXykKax#{5j150Qp#kttd%+UIC7#`ndcTU%qGis&7G! z>~+&+6oyhIQelu~BZS12!~)O%f`z{zc`ZscA!+p0tW6>7hA!K~ly+ZaQ@F6j5+Ink zym}z`IU0hJB14_Y5Y4YMM(_o(CNEJU?RMXDJdmqouc`bmSoq8SHM+xj8g=_HIZruL zC)50@Dfu#Fe8rl-2ZEm`A*cnuBg0Nc+&YtPzZpk+sS<77?ivPsK@z0x(dfS-HQJLU z*H$*oulzcbMn_)FG9}VRJFqbtd;wnvF8npA(T)|kzOp%Q^BcW@NS7+n)@a>?4La0< z0rrLdHTqDVUXA17!{V~cMraYdLZd(N2AiQ5f+QCVzbE^LuY>O3J6AT=kFG8+ZKS(3 zJQ}ChvMkD;>9r9(8id)(W(z#x3XWajE1HE%B=j`juzW@1&x45`h`b+VN~A-|vf-2~ zMN4>I{+M0(8&acT-cfaBb6G-uok^q5p&lqzB5iaihdQ9bpUD@#*uO@f@74nf*p9=q ztP!!|MJYp1g^{i$POtnfTKHQMf|{l&(Cx{x!~8mvZhwqTv}H=9-5y9jOtd~f#p~3Ea7Vz&A4gWd?bqj z!+YVIcn|Ua-;3JQ_~5-SVf;UOn_MT8o6UcF@c+%v>zf*Z8i5*t8i5*t8i5*t8i5*t z8i5*t8i5*t8iAoj;I$vQ@#{7JAKLDz8&)GwBTyqyBTyqyBTyqyBTyqyBTyqyBTyr- zB?#2~e@kGjE2$Bv5vUQU5vUQU5vUQU5vUQU5vUQU5vUOuY6NQjKh)h*H?BsYMxaKZ zMxaKZMxaKZMxaKZMxaKZMxaJuOAx5}|CYd5S5hNTBTyqyBTyqyBTyqyBTyqyBTyqy NBTyqS)CeH||9?%+8>|2T diff --git a/flume-ng-sinks/flume-hive-sink/metastore_db/seg0/c200.dat b/flume-ng-sinks/flume-hive-sink/metastore_db/seg0/c200.dat deleted file mode 100644 index 7050478db1831b0e708f6220b9b33740c08b2ef6..0000000000000000000000000000000000000000 GIT binary patch literal 0 HcmV?d00001 literal 16384 zcmeI2JBXb{6vw~2uWXV{OrnVzT@xQD2n+LmK(O-xg3+`QWu9VX;iD65)XKub%G%o6 zN>H>?5CYmpun-F?3p+jE%-;Pro838lur@Lq^3T2BJ$LfIzd3WhnR9n|Zy1IJ{2hNT z7u_hjUzV@$zXp&0+5I_i`N*d)!z=I9SI;ejw%|YCTs}GC-)9fI{@m@K9U|vajKY*5;!n!ns0)^ zb?{s;uCch8Bu>|yAjBHRBR06#+Cg0V6N1@fxeHXar)?s!Ri`8IOqf7P80n&50wocm zGlq(`*~!G2O2Sfam_SJgVhqH!2L}>?hhQjc$4w^A785D@V3?HHbP&lj#0e~J=5*Xs zOlYPe433frHY$i~f5Q2z<6JS3HA{vml!QrELfmnaIA2NloMCX31XNK0aVQC%c`}r? z@2~;0DbJ z$2gcZXim&rq8BTPoHz`Ql295d1v4m#nbUEpnn+m^X3?DRN@T(8ap%PS#e{VnmYzjP zWNKh3Ta?7ioOqy^pjkr^a%fI?6`-g&G$&@>${#EyoCr{k9ZEt63gz9QBxdHs;k`PqGjn1PaX5K{~Dkfr7u=FxYf(vlVGD<>6?FfssS^TDo`EW5Ibc%r!bOWn0 zVBiGZ2F}ch?K&OYC&FoaV+xhfxa`0Lx*?vK6OR-V$tDRV&^kdW&4N?3PRyK+E5(E- zaGc-_C1JG`U;-sE)5O(cB11VQil8Q93`EF`nwXgrj}{X_1!fdMm#B<_LL9n8XPS7d zn21Vvq42CXAy^=W9)4yf=5$W|FCLxE!-vCg=GUUHj=J?r(Ho)Rn)BzCG&V&!TURI{a4j z^`h67C;S&*yY<-}=l}D=7R28DwfXwtyx{y_rqwZ}grcljDWTrL&6p8I+Z7e!lufy7J`wZ&kZjYJ{M z5g0s>%pne~&@-2MZ82nPodMR^beNnh#2t6NsVxSkZ5A58w>ap^1WlnI)oxGC>~qu> zgNKp|CeWPFfdnvt=0vazieqk1+q61ri@{N7OlZ7`;tcUY^Y+pkm-4LM*P^x;OHw-++Bw3eKbTBp3~MG~X_2lZo15FtGM5 z>`l;wQ$kXLZ=!tSu;!5jo=nsh12IlXp3ubf!rRCbniIwaNqK7Dwf9UMeoN_BJAqE1 z6X*mwfli_+J>e?Sg)2OSri{2b{<>#WWjJo)t=<}lv-xR$zd{hUG z=KJsNIRD%7n*ev8|ChfXaKOvj`Fj2*mTSw{vh&KIqsU`|9d`Y8A2yHCjNk5aO(LDX z(>^ke#B3#F%PsohaeX6sU^e>GPM{O$1Ui9EpcCi>I)P506X*mwfllBK0(}26oyaF*pg;^8?Sf07w?^AmfNtczBiDvAw{$SnWSz7#IOY!0c=L*ya@(#Y@<07&@6F~XQ(To23n^;*dP zx)}c`_pW{!X1)oZ0${d;NZ5H;|JU*PRkJ^S_uEldU(>?E(?7z3(6yahu|4jy_~D9~ zp)@!!I50RcI50RcI50RcI50RcI50RcI50Ty|8n5xU%w~tZ8)ao@B8Nw)zMNqzIur70=|H+E@t8)Ha7Y_Eb|e};feCj z3tF`cT8et1%UUfO$<>0!;hKt9mclsDP#|x3U5Sz=`2r@$J7=|EtV-!mV2ZqyL7j1v z;-|3H^-rlajJp)S1>0Rep0|{Vr2aPSbo#ce4HHTI9oR+QySgz<%2YqQu!p>6$^|~D zzXvnqTg|*-Dyct%{Z7Ab+rTIF_u*Q{SGA=~C;hnw2gn=gO2c$g{{XHdPbfeN0$d|2FhE>V_Z^3QkE79n{C-rZ` z9pqWXjA3@0<$$w@LjYxQo2d-Zh-osq^zL-0S+`wWqwKDgGYZN8Vd61E2K& zK0H9are#ApOX@#>hsd{DP^xH>e+Z9|Z&bsKqDem=!DHl|617k?dA)iJ?;u|>rGzS3 z!kcL}DSt*&cn^7| zZ7X!CQ~F!*KJvA6BJj!nx8VcitI}0Q^CqRgLq0@aY7z8R(*IrZ5%Sy^E~+}&{~lQ& zZloTI6JX5?e!si~CYK6$+RAzlR>%-|CN3EPMxe52*aYiWewzNVp ze|fw;d`?Q_8|jshYyI@_1z}x3;r1vhlhmIOj(oV3h{mzY^SMWVN(AzbmD+eGR(|&I zTSOwSwJYuV{iTQBCJOoRxWXFB{DnRXPb%E`@O8)G8tb@2KX-^mo(g4*SY2;>`CVd= zm+`OHto-cd_lQN_GNpx=%k!<5pTQ?xKbC9D*z$Vb%P$}D;v?NK1_uTQ1_%DL1KA0L z|M70Vh;f9Gyovb+Msgf;l3}Hdar1SIlS9m3#aLj>k=dN%?8_K8y2^RXa~0$GB*rP$ zix*gVjuH6`V}2Sd*txLKN0f uVdFUFImY5V<{LPhond|&;~4YtVa%sEo4<+q=Lt!GkN^oU!<}SFC!i%ILzRL~x|0fO$UsK{bhI?|H1yO| ze1v=ihEG5XcdIyfNF4cE=Fa2QYWJtr>hH79mf;Wph<=(u(Rygd%_u(})Bf+{@lSZ- z?o*w4p`SiLe*_VIa=m}D%W!0YN;2JgT> zxe;)tcXIIPcRqpp}M9{YCe)u@UW4egL$id=3xQ(Y4urvzE%A`EFvFOQ7JfY?0*rKtm-ep zvQ_n zv~hi*^W<*bk4O`i7ftUMx zs+0QZAk9j>yqZxYs^pQ->u<{=tnNRqln9k-;=6<+A1fBKc&wiZ^?ec|uZB@5^7QX| zF@X5(DwtWc%YkEZ@-NQ`_cc@c+c+QiS2r0z$^h_P@q-$GUmB30_c z%}_E^kwQJ^NQyjHaXB33BmF1x^F$zD7D>#+iJlYr1(MbEWtp=w9?iFjd>5#(29jet=mYl1LNFSkNn44cRA44Y>Ez+QpRe{i*W967MGlo%}T zOAHP#?mIZRcwpbs{-Fc=67;jYR2V9ZCRjJ?JX>6!pU79J$OunMEF;fyV&9qNg9rBw z6_-b-NCE#A2S*bLfbQg|Cu55(Z^)&m(&PE~cq*5V_sz^tO~q5W_=!|@{MJ;qZ!kHs zXAix|pPoxEjHmMblrbGYd24zm-j^G{H9ei0NlmBYnOvO8?uq9qrCMd_DU{brre`MH z2Kej#cxozxCRM9ce}DY8)QQyiR4SL_%^{6HEQrJ;dPn&uQu)*Zfy%+G)}Jf@&mPbD z^W~;fQ&X9lJn!=uddh2yPiAMQ^==XQkuNlwU%==S1FnwX}9>Av{fz_DyLN}0#ATK5)Hg?S1^C;b_DxVF9whAKz;s7PB%~qVK2OvCP5KvUiBvc zic1oL{n0@&YYBi-r4|hJt zi`oi~mqpDaW8&!p9oqPUU;;iI-Y#>1R$zb4}qicP=ZNN z3O3#yn%gGE z#usw)W4F(zv!?|kLb5zDY=ZJ|!9FY(CrMsDNuk1W+T@*hAO3;QmpTKp=*;ZQ!u(7I za0$J^V4Q0Q5^+)r?BOzlA(4Cep%5@Sm6@4HpIW$m9`fyE27)QcJLaqK5(;dQSrA>4 zO5%NH{Upv(r3N_k(qAp);>=8HYGH0Rm&s>lXQ+_>oxjZZ?9}}94A*5)cJ0xhGnE@( zn9VLsq;unxQ+xF198U>AGWpZIPVdp5b1roZvK(sTU;L?)nW=m_yD&aGGvW3p@QzE9 zeQ`P61pb~oF`C3iIh_F3v8mbV+yG%Do0)@DJ+M-lUt4+8MzK#Sqb5_NapDszpT$hW z+~RDzKAEb59GFOMtMW@a3lS`ACb>$LiNbneu}~>a7uO5){m>yiuNL|0o?SmzEEE34 zbP}MrO=ukhIXGUtI9^)aIKMVms;sQ9l-Bw%S#kqn#>~u5kCBQ7RTP!L5MkhJ7z2m9b_(cT^1$_54f(&FwIQV^0DBGje}ir z7;2Wjr~NKNh^!(&EsEB2#YzQoP(bL(jn8s308Km!m2|vxG9Dl1xp3fg?m=cZ=655vb=syR3ZhlT;`5|RfCdr z#^xt+3b!zoo;jAkRY*hNxILy{7ZP=7jb75he%!^kv;vq&k7r;K(H}^FteMVc#)Apc zP#~C~MABeNJ~ML~#exZAnPZ{v(Ffqd4{JU{b|PVKbNSTt9Q5tNHYI$-=gP&U6`D!F zIu{y9ZuAUd6!b`x>I3f$%@2X6g$Y2McdbfXSc2leoV_!p^|?xMW4RQVzjL|t{KPEu zE~FYV3{b>rA!K}RgAkNWQ)Avch=aL&HU%u^;!CBqiYu5NPoK`k?<`*Q-ZUY}q9s>y zJwo++k0kU#SoU6!up1)F%1(Gf>=98+yq5wmui5JgKQ$EsA$0@V57kL6C9F&^cPN-r zrNm`@34tm}$|kPu;Vns&aR~GjF7bNH&&}rPa-KVxn?!Ej{w_3Ng~E*A)8JmQg)6g*BZV_di6rdNs+gx;&8`ALohz4?ipv}2Vv`#hOTVeT zWNZye-4h_PbJ^MP^u&Bt8S5CPI$k25VX!qD5GGMr#9o@v1D_bUUJ_q=zMht-_S_%$ z(uuC?=|Wk6biy__A>*m;6EEv-20YDu;%ReP5R@F$$2`q_;%S1+^8}6JE1NvCQ6lOt z6?);)baXm5o|?iK)FR!S&QeysO8CdqQGy6!OEts(V6J`C%M<%6hr5vbNsvndjK{mL9c9fGIFI zgZ_AC;*@?)w6<}6u~?oxGh4<`7nkM6B$#fnw?Jvmm^yNzJhyxoV3c#m$2|qOD+!J_ zJL=_}6fYwgL&@Qo$T9d6ns%y?&gLB97?s?O;)*CU)s=w&rr0vhknj$yLq4LlFfZ0Z z)nfr^ULJBf0O1O)^m!PfX?z4G&hC_#-uCjAPr~wkOUG_V^?%XPkH3f6`-FtQ z^@|UEGs*ujQWifjp~sg_`Q&}5bngxS2K)aOHv7I8-5{r@apG_pkL1mEHL?-lZiM_} z=64yGfVkqO_|K1BzJ`jp5B^*d5E61VK!nU|f`FV0)2UMnCo>cITNg4j3uCAA;`lmL z3ih~g-+{ghFt=#;@07GBx&ot35Ts*e6F5^0REL>$NQ##LW^kLYR}kj?N}5B-`6she zSW64UfiCdfJG4_~7NvpkS0i?Ovpq}-Z}&F~$jzVfyL^C|L(ZCs^rSXv@T35EKCn6} zSWpvCZ?ehrZDUI0Zl79kb2I?^kpP3or>$X1qica|Fa_(O86?#uNztF?>k-Nq0n|`! z{+ugVa3os=QE38??3oN6-qyP?5s2(R{c3Vui!(x zyy7`LFITB}S@~li=j?oTJnbzAp56D(Uj`jY&F5z`GvlBX=^1fj$@ea#%(3(gXImx& zA)%J&l8o=|@Et0Jej{}=~!%A%~Ppd^9hJ>K@IAo-Jx_!0ah4G)K-|yo)BA<-vB4R+l zuM`(yu+$t7SE=8PA3t)im0zif{ScJrmI_?IS)JyZY+HYv@FIUMb@Gde$t&|?dZL>0 zMoqSm?YFKm53mN8V->m+%imS&q#}kTxUbg9zz)Jrn0RnOg3#rksD2tR|W zh8LgS2nP^rsL1)@k=wE0iOv$Y%Tzjb0a@+rB`Q!++>Y9$6%%N&p>^3|4|Y1@!$A?4 z-_-n6o{R6^b!>joQjrW}qr{FGU#aLs}zekhRcqf-Zyw~ zct3J4%g6o6gTuINN-wAZVz5NPGaGA5WJ4RjyR_E3j=$wQ2jVz6Z}+_V9p3RXLXiLN zCk9?&$p6Crzi#=T;WU2oZ70ix3k1Y@pdns}i}TI63kSyoDAsCxy%ayQau0{sMN@yg zcvo=^?@AkI&+#OMSkaxrDzLSDFP4^K}`b6eFWAwni zXY+e7^cdY<8oYAxJdXRtaMvh{$weh;pOaHZtWO41Bv*}@#Ns}!Qqji z1H%WE4CcNND=?I!3s%NauVKDUdBdZP3T9AG7?#tZ1O*K-q*_l^A(z_>Qc-auTo4o7 zgrgY}ZZBa7DxxJt=GB~WJye*IN?nn zJilvPA9jqjs&);%J$E#;jX}DR;7U|q`oMZAw_Ya0{yhVnu2)Wi^!CMjU7-lhIZmcv zJCMSKZ!(?c+oCq0{KjzA5p}B)D;!XVFmC#6&a91HTrXB!bC@vwpEqfB&jwOQPCg6d znF=38-K~KD0^~-j-Ny}cd+5|%3kT9UD`hl4{@3jAxjwq8w+>ZOJNQv*Y|%n6LE*(n zu^OxMo-Obm_seTtRIop=W__CJ=rO`TgnP!L5Me;Wz~PSX6i=T&_uhLA z`~SxM|8et}LSl3nCX+)|&0}`4n6m%pn)N2G|3i(@+1v$y<&n|lkwkL1Dug$%S1Jet zw*XSBtJ0RHVf&rQiye1|8c1;Ajs#Ht#eb9|RRx?+_B02A`xPRXd4xuysc`H^HR>+^ z+!TKk?s*6S;Xe3tNkBH)j6rGcY@G;GQjlU*sUd74D;F_+l^S;JN-gQvkDFyoOLd^f zd=Ykh=ITv?YSidj2l-m$>X1-_7DKoXzsf+vW1PqZ4g(rj)L)k4gG)IKz|vEG5M0d@ zw>$qDzSaTNM>y1tYCl4C9@VagD0PS+Skga%(tr?5y3YY=SjZI&P<(K`5gwFAgX==1 zrp2>H6efB$m54|+3mIw-k%70u+oeRZ@FPXbE%C8*{$v_{X2n&9r1UYsZ#=-Q<_r{E zVTjcgk%3$PjhPBcH>Jag!U61+`>XDixXq_ufT#>78r~Pj-q!L~W!K7&^eqQNo?$Qd z=p*o}wnZ;dDv7KYAIuFyk)v!HjxJB6CkB#{m-3-LRWy-nIvba=dXo-iC2M#8&$6?3O55!>bP-T0$pzfi}l(NOj#_Bp}Ds&dW zLlbTDP#n%-Pn^t*D>SNS;jTwtdD6Dr)iuiLEtgewTi3cQ#l`e**qJb2G-9wDHVrp= z+u$0}_~__sgn9WAg;XAfD`WF{>Df=L6vdHxIvp}T zk+CV{^?&fshLs2zi{Tub4_M8%_-S`7FO@+UL+6Rv7?uxu#qU}0tE`3L z(S&pb0%@m1+Ogo|t<&t#95=GSkUmTo5EdcKB5`wOqraiLrlPNqpRP^K;wm)Pgu2d0 zeXsl_rZY2{XtN_}Evsn6^cyp%>;@Ys!AFec(-qtmi0kSIU$q(`|GP>-es4CI?ccY*e{i&4K&k+s*}bRkck#h~?x(pwDL=&%Ic)Av;NM_kPk&$U z6dcp4DIA}Ks&_1t=f1fB!f^jk@?av--+QXJx3?d^y`hb}4JG<{<)R(&*X)2~ul;>+ zU;8ZO)q9F-)GL=+95#zSFoysp^p-l9y{pBwv+L)2`(4rR(c3>Lkni?YpAx)- zLz~vY5E!t^>CRU#7T?fd=&CVShFIFY7nIAi{t8{QGUl(ge8txURV`MrJbmo#$6B}y z&B=w?+dqu<>+W4xkG1uwlMNq{QpCKse?R_()l=`g`r6}{6MOp)^zTpB-I_OcA)}`E z%>-4^IA1&N>m(6r(gF1LAM8&K`Lb=wjv#<*0xWUeuD5@r5j>gghz?jrrD?$AUI8M2 zrs^Jk+)ChwQB$ zgF%uI>P%vm^z!eUQzaL!Q14KGa#U0--ZWqPsHCMLLMJFUBB1(4OySD+y6IK;oDJmI z)a)4V!EnEK^{H;C^{rB@K|dnowB92*OLIoKi57Ow2Zsfz>NU;Fan+sudUvp-C0D8g z%yH^pYA#4&B*jhKBzVEBo|*=|#suh zgBQda2g5YnpWok~KhU2)*qzc@ff!*R!a#(92m^Iv;BfbsMBdE4@B6G_|KDW%zoC((A^86v zscJF!QuY!Uz6m3GFS7ZDxFIF^c2f64r5&JnPxft4A_|SNc;N9^ zXW=P}f1xTTv`??~NXwOH71d3ctAPl-5 zMNuGg^Dyf&Cxb`8bclP#Hmh3jE4{r4mF4?v)VX)vt0T%7VIaalgnP!L z5Mdy~K!kx84h9alJR~~vmGAk3dkp!1E9*iEMG7!~EdKvHI@s0@)=uJkBC)g#6~Jg! z6~H#O6)J!xZ~r??#PS;+Sw4tZ9EGYaUBg~XEm0t-OVu@`o^ih1fUe=|QN18_2wY2` zs}SnZ&D4*tOHdkUK{TwY3>WYQF9ddR7Z}MzaS$#r7OT$67~76nncKKfEc2<^cr3}I zzr}t-yEE%LKXff1Ngu^ejwlY1P>Ydrs#Zp++ELZuR4+~zn_f*qcNUhOo1aT(1y=cO zQgwMWGd4YdTb&d0)6=I1<#kc0d*UZE`CH@j*#@yJ;_xgb&K$(mNWIJX0f}hoox@Nf>PD+Lqw~UF!d)%DlGGpfIXdvWn+WwMp;V2^+NN)?x^mII zwH&H?L@NFpb!iWB(Vj?WDI}Zd9n^l8qqmo`k{g^iJ;FiFXd;Q#S*$v5cC%f|{fise zukjT+HrFzvtKI0DzCW_#a&QzW8aLEKpDPpxYO(?(Md|sRH zwKf_)*`J#oNo`mLq2^6-V|F#?HPRhSlx4NJ1^y-Dr{hLnR2!y+IimUTo0eG|9>E?? zY^$=R`CRH4olu4C(KcTYE{;`mWW3@Fui~}{SHy0cn&)A6U2N_)y`4|KGoyorL}DmW zb@KJFEyN5VD&hG;<7D>ehuK5!{(NhCI>mVp&{c!K3q`AU6-Dt#DT)iD_!eSX8gOzU zvhvpbeke3h8eGFyaWpQCjxG-;hN??C*RpGX#(=-F^!W?x7aKlTO?|ubtu9tosBvOx z1pB8g9qk&Av1Xd!dKA+`gB9;1zc-p)lG?dK8JvWhij@#{E{hC_ehP+wiur9B#s9c$ zmHAfkL}R!(mYqc~HbmWlmy~oCx1QX{>(3cF46dlb3B%a7ceV^N)-&xKW32lRTfUjG zj=yO6C&t>|-!j5j>rv+nMEaj@`9b_W%+`c>{~MEUzRR%xZ*MIAALRd+b+jV{;J56! zSK9;h`&tNSSl3k|Z9ZA9h0!fyei2O#x_&t;x)esV3?>klcqz;cHWh~Cx&|`36GrsQ zZceHhk?U5Y(wsE96Xvta1!3obZ)~qE*0$~~r07nVc{NI{YmKet_;nEmZZW#HC2(e6 z#NJJ#25*VU=^S0#;!&*xf<0tL*S4zd9DOIvYUk6v8Y8!`878wHj>B~UE5Nq zu7SDe+E#RJOEP@=Ehw=Dqib7+V}@$H9A`nIYg_Qyl}&T8x$bLQtf%v-mTtEDs-dnw zo@R-CXO<5h+&5HQhCzNJQNX{&!O=v*akz93`F{s#0r2w^|M8Jm8R!2^+W(J^6qj%; zw^UW%CywRD%cV*sTRc-N7uS}G4J)W-KD+%%L(>qV7viTY$A$8ZtDEQKgFNI^k3y$4 zcRZjkww~Ec>b2&Q(l4~qJgQXLOg^cFz=BAql0~y<6cvc3y$V7=8Haa}so7~{o=E4$ zvzfX4Y?dMlLPZ} zbp1FxKb1bxi%4j-f`hJ{zfdY8k|jWa+hIglL;OqbeqRsG>8m!|P_;9qln7?Yt92co z1s&{bpm=3usMI~CMv<@}I=VnT4adYpdMf=KU_!6T;yv*r@m`$phOz~NIMme8 zRIaH;_?#2FyVe~sG5E*~$U(#O95;e<9ydt_C3^^0fh(x6UV}f_1;1ej{eg7htMH{Q zd+twG#ZcuNe@Lb(e20%V@co)Qc-=$Z2Si8<-W7M)_yl$vkze=-AVtn*?)yNJNSz?^ zgWG!8IZ2?rDG$eGY53aZvvM-o)L0Vv(5eZYE^1H8#h03I1UQjfn>S}PN+n4hj>|%5 zU0sw)CxXddL>_n7m!x6wT=SJ1WDvD`mFr{eE#8_fVQJV%u2G{qAV!nk0*$QG+c=P>0L zeb3Pa7wx@UfH#G9#R%hx0|F%YFfi_W!?S6uYgqs7bf~rfMAX@jqfkO5Kn<`iP!|x@gQQdHk+%4L+HBUWIf3D9&HF1O|h6HSXmk|&Kc0o+Nw zj4pF21R|6Hk2C>>uDGU3(G#p9O#oD{ktV>WZi_SlrPYn|YpxU&X#%P#1`h9dt4RJ} z$NXO#@_#F9K}vU^u;u=L8}9$}%m1T8NnBJ()+_&C;@#Zwovk~h{BMS>;+Hh|nVq<; zVWw(KFXho5)fupv-@Wa^Yt*CZx+(t9Q0^@TD&ju)bIF$Lh+JsgrlC5G7yf$9t7(+o zeB;k=UO`57^94UwbL0J7h6^y_7_rZFxTwy3qg&7c=Z82TF(zw+>O;!H`m)d~p2c1KJOi)AeZ@1l>)y-Y zzTz3&n@nB?_Z81Dm7Y14zg4CRK*cLgs8k$Ifa^AV{go7b^HM_Gq0>A{-}SX)5@L9H z))xlSp7)TYzc#DILJNl@K753KyD5fJSQ{b+y;hIabf)>UrTMJfe4w4=X22` zXlyErb2f!To)1H#W7fFvEj>fTQ^6DPew;d=jF-x12MQMoxUD__yROBH151VV!fNU4 zz%e*bPu-Y>5e-;epH$8 zKxNeCr)AgX7~%4IsWjA_Ao+Jt_gUF=%SuoK0 zf4aU24IOU%MbX?Z{qnp2+>rk}SQ}Ed27sv{|L-3iUK~jb4ObN;cCc*_BvR|J6JOj| zFE;M-V|=-Fu`U^cbKCVenjz0!Qif2Qli~S{(gj5{HhY!V6*>t(x zs*7de6=}kP(X6VCx3<|HYC>C5d_XO*bxzIWiNbp~k)HHbr}`BV)Yas#fJvE0i|HB) zcsL8@fu@@D?`1y70zEJ|7Ng}rR<180DVChe)T&ox7Mpy1dAT`~FRRj*aBeF%JD(j- zFSt|BWp_5$;g9smIkc9Rcp!Jsi`W@D{8HbvNNJ(>!!`?6rHP)6ZoB6P==$KnPUsk z1ZaFi6>v~!;aGZx%n~5A8%vAhDCe#$v3T&@r~GR5<`JWj=j!EGY;$`A^B@X zd#f#MOam%b?W&BbQ+8Cpgelgotb@17XQ2Mt128ia@4KZxF8`y8?sDdN7R0O*-{nLQ z4^*dj<2JETJX-HHRtIso2RoSKux@^CRvbJygmbflRadoeWj(&SQe0clteq)A+W`N; z>nlxg&_{op{ie3YUG6HdOuX6b=20xubn4Us^k6>LBTxg_ycBs0<&&^>W^uDspW2(t zrUWFK=tYEquL}%p-Eb~2*7=^+&%zhr&8=@hrw_N$DL;OmKK(skGUWdz&;J)kM+Ra4 zUp@X`oV^^}Nx)HMYIU{QL&b^P+HTdzS-h``u6hv{)U&0#bHz#pcjXNxm_NAg9sM_E zjT1sVsB{vnXp5jbadZ^RqZ%ytaX!Wsx&-0Fb@=ddR7PWoEr1=EFlR?csn zKe@8Jeoizk!S&Pw9#f$xxX*?_@GRQ392>MoFKPGm&IaVsi!?U~9;Uz_ZaSM84<-O6 zatRVlP?AV6C7+o&jbg!svCOg1_ZSU`F7Diruu>rO@V7jQxjOZ+jNDwgxU@nu30T*J zD8(Zx==qrtpAz01jI`e@LkxinVeq6y&fb}lbk-c0zaf?zQgV@O5{^RY1i|AIIWNQA z<>H+ZQN`@x$KtuT6wWvX6w{jK%p1z)+zU~{_Ke<$CIovYTJv62=j%kO1Tl=G6B)dA zi9tDXi{SKbiI1i8C)4Q}dCinO$PyEI%i)_7WH>Q{Ypr3!xKdH)RuGXJ6U$^MlEmQ1 z8PL7zK~G-Ht_F$lpAiiw>SY>PD|2$>x#iaBuHy!|Q?o5VIQ+ilb&R@4X5tHGqnDcx zfO}%)%Ndeiu!%U46Z3|PGWsDdKbOk?8kC$HB{mYvAartgB#(_z}&V&)c>_}n+E|(H)>K2 zVpaU(aXF1dLV8ivQUuzu8pa%;Kj9}AC<$IdLxUPnpM6-6!wA@NCt0(cdc>1MT~}5D@_K|G$b}e8(dvK5EGSZD2MT+Y;zwjr0HHU||ub6V-#C zS>%6&=R030UxW|2h8g12oTEZ5EtS?tFjxLkAcJ&=#q+2t1WAF(s|C(mN^2EYv(A0S zz|TiT3PzqZ!bKhq0+*`r$j_(zGSCsyw`*b&WeNg*gnab*I^*VChsJ=;wp(r;1 zAll$sf?4FL25Evg{2d%!Ea33By7$L@>@~{!LqG*IzqazGjbeseXf>+}n*3~s@(XF$ z#Y6JJO*>7)W2S0!mhxziN^@i8cV&)%0Pd#vFIl+42NiK2{JCVybzBKS&IQuJBRWa` zR=B!Z7&}cauuyqWucH_NTPMOeBjQQb-ftdfc`6L9@$gj&+fAG*FHArQlh3K+W51%q zEMqDyXn)V8=O#0Kx)KdDzo5^4CtW&-h)8fOR$*VJQ18osYx4Iije_z3C> z;mq3DMcCQ7df_m#aS5BGcI!Sfamw8##p0DJMB(7hw^4G?b%kPKZCR-&J%wS|X6dU4 z&AM7hs)L^O`}@#3kHIg~5$P{MTL6^PbEg~hIc27YyS>v!Kr|#Z!P4uUa(XA!kfm~b zt+XCGYPuQ{*Saq4VwVU;yEM2oiXFLnnAEG;RnSX{4cCj2+%OY1@tL+iF|?8v3k}X0 zh1}|*T~T-Er+K{OJ0?#Pat(Ptq@96tGAJjWGy{sh_}qYuCro7ndQmslFl{Do<}~LI z!ZW9FMxfEnxdGb2i2lE#aCsGVKl^cSx?5i^CtJ0+vxKL!FW#HaW{%-B zDJ&OQZBn}Ezo^QM`CR8H`-jMnUq5pEJzMwnjIPI2?Ge996U>30;3ce4`HZF|VyHu~ zFyM-K8vP5BW%mtVby$4v3n?7Mb5g#+%%qyRaHIL_Mcd;8?vR9}Y7!2JxJS|Xp-gql z9{r#~S?x$sMF2xfU!D8(8T3G9r&i`9p~wxaRN##ox0@8@5Ie%Y_@HL)V~`PdMl>i2 z1w`hrtry)-w8Q6L%T1l%54a9HUlq{Z^vWl?4f(&x^Z(@N@(ASr>fx1z{Xe%(HUV239>*uZSJ*X-gWnpH5%fevN^@=WwU?xKrz9O;!=YZL`LT)y-;QWIcHl z57Lb$F~&2x@=`l+l7e=Rq_&2&O~*XFb7IrH2l zFvZ;nd!5--*Y4V0dBnX7DwK)c-Udv5gCV`^J235eW(K*i_~e!9NWL)C3)+~~cPcFs zC2uR0i~OB8H~3CZLrF50oG-F!O|n8735GdsO$WuRkW6(ie9rF$Jc~=PT$z*u>oz05 zDCgp?p|7^{H%Pd@ws!Xjng|0C1|keZ7>F5#psx zr1M_V2-GnDC(amWZ(*fK31Tt+=N98syQkY$>f*4P=v0+z%h2_MuruAuVP$Wza1?H< z{N{I!UlibWmF~H8$RGqQ@-X_-}Uzn`(~kUZaLX@ydFkydLMi zCid-%=g%Q-rQXC|SecP~KVMug>|2H-v-lRM-Qp`};_E0& zFBS`xV*KtxCB9THBHH0{{O*N&lj2FUnY6g(r{ipK9Qt@WA7L279< zqi!{uuobBTYB>*#t;@Bq5AIwcX7$!g;nwtlMd`os7{+pN1N-hct#Wq>@?)Pvhh#?- zeXT`UeS7xacyb^Szd4>996mTaGIU`0pj?_PKi9SttOZ+vp9fI^w3Y0e3`jU*`L=q zsPX$HQknH>U{EhfAjy1&8keKp2zHF>!rF63LpUuk6xLiju4a-*A6PHpNM&X1Y~P*% zPI@aRK^N=-!a^OD)*yOW&LSCOdY2YVu`YFB3Mj0uta-R1@xn~m(lriXlQu*T8{nFlJ zZ~sKS$I9f3;{f-?p~<;lA72k|e_V9*@7Dg`V}|_S(VYGNNHRh4fA#o(!u=Pw@+LPX z^^TLg{_3!aueAN+Wzqm>w8%X}^2`NcR3a?aA=U7OJ((gNm~&?N>^#0nhqrK^voJrC zxqV(3k=eOP9))`0_IX@mn#|B`c(+ykJgrcu8pXY(76!*kB~@7}t}PRZ^>EQR zpo;LOlsn=H?v0pGk;;uP%w`v;PJt0~-$#z|lyG#5K<}0ubE#u#T%M;b*DW2`L?<&- z+~9wE}pDsGwx_w+@RyAjGL8ZG~#*q_} z$O#F!h038rbsADx6b3<~Ms7~34&gPOpFDE%-Vl>wX#EtYvT2)8_(?`JH^X&%*oxE> z0WHcEubzl**gfyo1zkVd{%<7zcNGLE&wk2%lKnA zwTv43u*EjhbB8J7e{qVW4$r+rsj?9foj@(U6mhg}99?Pylym3xJ3P1h956xptqLi$&pJrJKO*+0}E z^$~n$F?nDaN2k@}^K4^V!FP^}aKMev>Z7+Y6~saqIbqTYeD^_}I5iL7elS8NBQlH; z(kPsuazlYfHw4q*kwl|T?4Ngf6af%nAi_X|fd~T;1|keZ7>FkXpkGX>hcf3$pDl8Y(JxTX2hOg53hAM@lRk}%iiDSkG zE`0Pvf92=_RJBNzGdgdLswBQT>zzL96U8&da+z+;1SSsd*QT@C)L3Q;w?Tw<#^MXn z@FCeJiPdCs;7;7DG2G_m;^;zVq8}gmF>%c;MeQoe;1P;3-W@$n^?WLqls6Mq-~E(C z-d#v)?)btz?QtUF*2dlBa*dNmI4IGNYIhXSLvK0?P&KlGPKKmw1-MaMBN)vmT>}7t zvoVfK;6tbdJ$$&x?T@QMMk0UU?fX4`Cc|7&a+jav032FKzIB9owiGZxr5akr8%rx| z)m(D&x@8rOd73nOFRogOdoB`N)K_`Y#l;lA;{(iy2Jys)ohD_xk55hC9F-H6s=}52 zuoJ!JJMB#{&k?ETFLZJ?o6a1YA){Q(r?A1P3$t1^9(MJfxYoX8%`cAKnr}_kP?uNI z=30@?PeUD((f|&~QN*Mj*H#Ub)ctjUDM}i5h%I){I?d6K0?b=g zdaed26K4$kUO+{E+~E^FQX@zm)05;hrxO;;b47{uNCfD(7JR@*SzWs5kwBf1qE(`8 z(3X|b^c?mEDUAFGx)koRxUMCl6Sicp<_g#)Y)wOM&^4oOzOIiqT`?7}dp*}`1#4vn z_fXOkQ?s-1p|2BteHw)~$X6oYPir-E#NJ5ga`t@oVlBHHASdzU!a(iLeBw_Yr>Tv_ zkEtyJBFi^&<`OxOYoj)Ezm{_76`n#fAu)n-@slGW}l}8f2+5z zZba{iEYF_dYN9I{ zClWlzO`MWP;<65p!}4EtBELxF-h}#)(Y&z>-p#zHM0q%QRufW_2`SC%D_=cum?HLD z+(^bnWWEN>Y{p$?5}kRg)sOMb9&rxZaO?p9{FR1ITn@2D;DNJLcq9Ahd27|~oNL>Xp z5dDoX5Mdy~K!kw^0}%#ZkQiwBiS}_Gvxnda&bTG{}-zYW-nzg zQTPAdO90J^V$)8~SEVRs-fj2n*2#>TPW!X7r<278$6V8H*}`MB~79~$<%)wGv%ag!uz4is0#m}r3lN^_C!c_i20 z`bUedpYZsvK)?F#&m*0cQ~SE#a){yT@{00XB793DpPU4)UuiRZya~2D->e?Xn#SCy z&{)~p6At%$(?7)Fb$!qu33%-1Gg0Wz1_Q}9xVWRaGLA>pH)KMg6529B|pq9d%+Vn3c|@vQWd5i%w`%UqgX`QCMB(@_S@dq*znB z`l?fF&|C|_i{joDEZEGD&57!ABaW?AWBvt?{@gk{K^;P0+y{RyqvyzIJadbPzPO?9 z^ivDh_qVqzV%LQdr^b!D0dY%5z;)k zR@6ccrXZxmX%q`4h|mSW_eAD`Z|l6dj!x&sQ&SiWzN1y6MHujqE9r_wxMHzaEr?7I zt(G`RmpBAGs7if9U~(CJ?khUN=eK&`DUx5{2`=w|k#dO&PjR2bO42AQPYx0m^KxfA z5E}!z#uatgPSM|E7BxRbd6_8_)Fq!1@&8M9dyChHw9DbUpAy-FagF0Sup>?#d zGy+E|OH~E6Zq^CLV16Q>qXXHc#NZH|yD#rcB+iWN+rPMfXkT)qfTLdiGdNmY9vN{D zWM9Ru2Q8Q)voP-8PE^x(OMl$^FK(&UQ?280bMzOaDE_>3YTMtkUn2Q`+fR$;-;w&g zM-BPEtvUbyOUdO$Oe@ThK_~$Z4lW+px3qr6giQdZeUWaTS4%w8S+E_CyK%7Q(G7XV1w} zX2MkI?$GzC^&G`ovZSTg5P^@wuu&;irb>uXG`Rvm!d}k7ccuncWxZ4`R^~6jr6#Pi z;b#uKUe2?txiDl^Wb8Zr>>39!AZEs%LYXvt`3sJctP{Eg-hj zvg!+7&TO17;^x4l3;?C2h|nrw2^TBt#q%Kv`IYlUaM;C=q+GF3UOFeLOvC8?f|?`% zA$`w!xj+#Z$IlhYg{5^3*Bck;mP#eHikEzpD8Nfx&T4YDb%RU>3c3FV1vRGHXW~bpvV&%eWQTUX=eE>h` zDnq_pSgQcz)CXT3=zI|q=2*FK{(PbAOT>sFDnxPFnjtll%Zw&7YZ$ik#pM;u31szE zBJxav4g}Mv-fvH(tg%x}&al6_+Qlhpj8RrB!4PyY-x3M zv9NS!NIvYs*b6{0YgOpa=&KV81U~%7R@MsTi&I#c<-+PzaqTRkd5L{oq|*jbB~^r- z1gQ=yDk3jt)=Z)+>s5(-;+RdUB$LAX${A$Xm&RwUugpRyUT5W2>b_HEl&F+<^{*)txN{ zJ~)OfFz__Y{}2q%SBmAZQXowQ?10+lmZ_9NiK&)_ccOQ&KiNOrKhhiL%%fW4M3Oyv zms(r)(`bJ^+HPGf>+IU9+RZ>Zv?j(2#J6g-8tc@|n>o8iL?pAOWv9->yZe?3mGyX^ zjC)DJkz75m%Nf8iB4qOg+{HPGFiQ)wGgGJIh&IZdrw=A5ASRSW>QsVWP-xA*-hMbj zqA&HgkQ>j=PEFxrKrRj5AfGuQLV8mqPtguq)!W&F9`wSPd8i+M4$_~Io5;4=eB2NI z4L;uGf69&r^FyU-V=`7)UpglUGB9?5oI+KDmu&2*!(w9o<+3dJIhYl~IIHMna7e}vP9d?uIAj5n?e*vDSu(*<~y znHm9T2O8~2*H)4t4-6%eFbF#WKCul8ioIIi~rjnW}jee z+y7#}&E@}}745(G?LYTHL;l~!dXTb9q-uZ3D?rFkTLPC+%1| zuRNGFtO6Ke2Ydq|ZYjYz8efaF9>uTL%8|j|Mn^_0k&&Tf0X?c7(NHWAS6{aolgq12 zn_rD~K`f5M;!trovA=rUudS>bi$h?t;UU2a3?C9t+1JF4 zq;GJ{3eDa$H$K~WpB5JvmqCiFJC^||#_Hj8j2O$t2j>tQ@%5+5_TshE#2VVcdzBbk z9zw6GUzrnYh&qNo(aBcBYv_6gs|VviMyrjFQKJ(r#=&1&ERJHLRlkb0o9$AYi$_o! z?gBzn`ntf;@71VHwMIL$lXoULbmj~wKy~v)F&1cb8m23B*|_13;AWbyBU){j(1{Sn zEP54TAi_X|fd~UHI1F_C2YVbY`fg-TFxGj2{VHP}Kg>RZ*nYpk{)Dl%e`CLgzy0j@ z8EbtV`$NWBzK`BGf6P9PzdP8cfRMxN(*iBskN)=ONdAYRh4^XV1{a<3$@@_0-W&d{ zt)sJzwn5m3@RNQV_%A2+omoD3aNiK@Lg@UzfPagFqlpALD7dap2L--nqQ?jW5e6a* zL>P!L5Mdy~K!kw^0}%!y3`7`+Fc4v&aSR-0`$eI*-}QyRG35W&M&*Cj!e-yczU1r) zf#R=GS%mAPdgPoPpeH^@SgZ0*M`3M)jB6ao+(Coj-- zIJ*3{keeTC9;BU{1QJ={5J;&7Is|GSp4eu=Go3n>nZ`|Ze(TSG=S8eb-AcrmwBr}a z&_MCez?kjWG=MTQn*z}GO#*1g)GW?<_;7K@4+f5|VV%<4jL<_rgrc|b&j5mox~pn{(in0&Hwl()SX|gR7Wf^_oB>eyn%~Xm zbYO`=MZPIp*>;4gfY|&ExZFK-dn=IY9^zE(fn+*_d+Fuet6r^Y~Hv4Y(%0DXf zW^zM}2Qf^W0!zI+Jgq7V6GVqMn*s|r%_fHn242Ev@msvOAyF=E+8>@-HP$XcOZ>)3`9s< z@k?w!DbM2kLn+IjHlz9r2MDw6*mI{x2X<-G8 z8ju7R5UodxT)$cBI|<2%0ZBtMWwZ2l5|V=kBn{zYv$S>+k^=@LJ6RiDL{}uTik^3~ zlywpelgic)#%8JLBp4=z4FSW#+HaO(PJ&U3x>i{-o28bMkeI}^p^3U#N;wIMNn3qG zLkPJBzadK8ru1=A6&LXZ^&TIqZKu}EC5mDp_RkGp0QvtFA@cus#dmT4f0ro#t0Vh{ z_L7L}2wSHS*8!~f4xqk%xei*gA;di5I#xH%uPNJ~i0cS0nTYGK?XQtGB8;LuH(W== z7YH*)#TG`|2+TcY`zZyQNE;EvNlk4;#20`u;y+419>OI>d_lw))Neul!sZJg|Hnh* z|G(}0*}vZm`~N15=OV5nj2DZ}KthOlbOsV6=9l{nB;qD0xTPh#*cPZ3MroEcI29ZG=yO7uiPmZ?&r((6r_Wa!ggM z05)420r~$MLgfGd{PKHtZG!x7u8D>^8d+q9sQ@C|h%l@~wh_QeWE%k?oG53cjfk`n zN~8;JIfNUFg!87)pU5IJ(ndtK5wsA3GK^9IM0`QS7u0eHB<0=6Z-k~o zlQ_A8{6<840cQ##z97`2J>kZ}z;5m-1?2x%hRFYa`%S4AZG!y2YD#pGR|=3Uwbe&w zAYog5bOsWJmB=D9xYtJ(nL(UHXCRS9W|(*>*<+KqM%oB(GyWQ~$c*@cupQ8xx(yu- zL^hMyC<4dckX%w-8*7DBA_t#o8mxzRA=0;vD!^mf(jR@IO zB5eewijU4k+K5OS5%C2+zLXFWMe-^#il7`+8CBVgBY#q1n{i|t5r&n>HX^tiN460@ z_T+`(Hv;nit3%}fkKggur#C_VcU{~>XCTp4>%Thw#f}PThtK z5+bf6;yTi)8Ionxk44uWaUCy6t|Q_LZlsAHVgwiQ1;9%bSNw{PE3O={Mtni!#WM0@ zxmp~n)2qkjzQGCkzc)nw|H(Jr^w*mp|NDhIxw~B{mKW``M+a45QT)&#PBzyoMdUXk zXvU=s{vvIJu18Z?iL?=6Sc&{bfKvI){vy8-L7ZG}t|Q_LDy6bhLkCdH@{BYH7R$59 zX40oNi)<#b=&jQwvW)ZG!wyTa;bN z9dR8Ed8>-F5i|MpvAUeRN7@J*XBb8D3ft78n;KzQiEe6yaD~xLjUfLM-PCwKZ)#A) zlaPrTol*oRYQz^rd;#msn0#av#Yn^#gi)4=F9;#GS14Zq`F~G{{Qs9Dt?^Be|7$C` zBd-*`v-XC9T8tpzayF}%x3ay zgD}de|+XuKe`F>|M*6^ z4nw!dA`|?sPYn>+Ml@s_5pf-nHbPvqGxcW;Ictct5n*!o<<>@2*b)O_2ZV_j@o3WVWpmXgMQ`YCO-o8Ck!Db5=R(P}7Vr64>Jy%@6sf%rO zSR$Cy58zm#Qq;lQ?yzm^=gOtKGiSzEOO@iX!>;a|dSl_P!i}qiwX-+o*2^nvXKy07 zV{GRZwu9|**!IeLp}d~veMTpH*lz054t6yvJF~L7>VoC4-Ie0{T%lYzUtBMiPZU-+ zih$N(?R}X$sLq$LYpCE$9d?z3VtS=gL5nDJzS6^9CMv$pVXq4IO<_Ur&nq1Eva`kY zDZsu_I9mi(*NPa8(i*<47w#(&DVqzIcQL!L6ZVfltkQ1J} zhfT5Rt?W3PaoCM@!9P)~ER|O-UrUWjNs(BH|jSU@0Y<0ZP3;;{XFnX0{U z`{6aiX=$F#Z)G`l!eKqYnVZXDudh+z%*Oe}qKDub=<(<7VW-$>LeCvy8qRF2uF`-x z>~%Iu5RT_YN9A?w_0;S)I&7iuj;a_NE0tD@g*6u?HF_AxPF;KhwN(I34Ij*6xm+q^ zXDh5N7s|`&dzOk9#9qCGb>vYVEdiVK7Wc8z#r8YwV0hh?V)?F>B`mGgQUP4XjmL^> z#d2Z2xIA`o9lOzTv6HP}3SrQviy;2kZjXno5F2`}wTsHIeD7qdTiF}g`How@Lh2~e zgNCpzemdnR%AM>@U`qfb=DeHcuph8Z5?UVGc3~OdTF~E-nV7KlTw}6kN^=|D{)0|e z*!oshW*Z$xd6%ivR|EK&Qu%yg72B)Ug*&TtK@nSD0~x?|M>Bvy{_hWw|38^}=-N$? z|JgS);vg5a=vhm5;Yw8|LrIUu=hv`9 zYGd?36JunXa`tJ{>}2~+_FaVa@9sF-C1x9;KiR|HA<+N*4tue}eBT})^52O~d$qWB zcKuuz`@t^u4k|j-!`>~5{xEnSK4O7#KhnkC-ElPLwKv?ueq0oKAGQ#ExVzX-bg>`r zIJ(=*yuXKiP-OmTk~uIZy4Z)h*atg~zC>i|IQlX#_kkYvF_HV{A&AV(Pp7k)@htesr#Xi+>^m;GfXb<~ck?;2?a5E~K;*8Gkcd_5?IQlA;v9&mu=wY80 zIsXJw#pKj%3jEM#Kw({2yV#$0u}^m#eYI8!tK|!#)L#H2*)#$B#V+=Rj-!2AE)445 zh+Kb*ahsT(ADc>dvA^qLf75Yvua*xZ_+^prAJHp0_+9KPUF^#pNBgym7`1;98UHUb zj?KRUa6Iuxj-B)=DGEZ*93Ql)9E$AqH>+N#dy4Y7CRu%^j z^f;X&Ul;P_GtV2$$U@~P>$F6Twq^V~<& z@scj*D&E#ek8`ak@iL%gcI>wFc)rVdd6#o-$I&6J=FuMKl_KA(0Mppi>{yrc>elz9rlg3H7}@0V4JT_bOPCXoITE7P#xzrUCt{zjt+ZZCKEkQLV!64FvmHV zLtRdyZ&dgS4%9$0sbmsi|jrGFfDp|60h?*0Xl8y zIVcy_dz^3N`>nG8N(#w^3y9R8s?~?h3g`fwg0o2D21dE1RPJ$#SX4mQnJ%=>uEkb9 zS1ysbj?El;BmOD$Q;xIC)}XFF^D$l)5Nq*~O%}^ETP}0Q9nCSW&WiI!@Dk3QL|2nT zJ1qT@>te|wy7^+=Fb@D2gLJ_@aN$k=RM;2 z$M`b_?#IPq26Kn{0s2pf=bz-yfbRn$|4;E}!1vSQ`62!c_&zM2e}*ors&BS*sK=*3fS2GSxdaacP@^ zBxYQbE9;k?U!j%t81}-lQj$CD2FrZo>K&C0wDebfB@4W!Y-jjh5m8%M)3|k`y=O%sp3y- zYF0f1szXbm)v03vJv+634b6^qhy!ytFyH`?AX&bvu-fDNsp!U^As4C*&s9}{tt4{l zJD}=x{u~%7lisxdwN=K_R}B6B3+E{s(Jw;T+`h;aN9@)-Yw zh{Zo4J@HS-Mf?-84*!HK!#^Rb@J~o0{1dVT|AhR%FGT+TfA1dpz$VE5kWF|ikUjV( zL<{~2fq_0rAmE>1@A)T~cK!*boPUDr<)7ed`6n0%{)tV9e_}J?pV&nBC&(xN1a0J> zpo#nww2*&-2J%nPKK==s$3H>q_$O!_{{(I0pP*^{6SRzff`;)=&@TQ7n#DgstMIvn z)&&2=6y~3pT=={mf)P&C@V6UJJMl?Jv+a9-fpz^HzK||x3zA#dS$rqAJ;+gk8J$5q2cD<54!=!uBpa%trVAJR5D>mB#DLg7YYwZDpTew?E2G;z9>f z+GGE&QeJDObi}T*q{KSgY$;u>DrM2EE!N$xQhe>iwphz-wWjpA5cu2K=9cldv)!7q zBeo?BI1`cg0bTdC?fP#jr-{h>+GGErQp|?$>)3hUuUL{gcfD1m_^RvL^)^e275n$a zY}mgq)@|wgeX%VTOx_o>VgJ5Zk6W9LeG^^x16}vG?fQZ6Ufkav`%jf(qU(O3>%EpF zE4uEtqU-)x$F6s&>P#HpZ^h*OR!rV+oyYfEF?qigllRw!$p?VR2ikW1Xm~FkXpjAu zN-;6{K*vsIOR{3}0V^gSuw(K8D<&VXV)6kiCLge3@&PL*AE*nH4+4`9w(a^!g)ehH zJlNi%5==xsXkWn(cG}kIgH}X7XvNlpu`bJMd@$DSrucxfV(URGwjQhtTMq$S54G+3 zutJK7t%uqz06k=%b`M#x^^g@?580>PLsmpSWJTmdvF;AlArp}gSrPe=6_F3sg~*42 z$cNi@{eo)OMC8NmmYwn8j@TtKDS#kWB6~~WRar~GS z$B$WY{FoKTk6Cg2m>tKD#p=TG-!HUKw ztZ00~ipD3bXnewo#wV<3e8P&xC#-0EqCPY}2{b<0w(GMBTP7NxY>!#i{F5EAVO7S& z@RL~cf2NXr)md@;q!q_cT5q!%%8K!)tQddFit(qc7=Ox&@u%v- z_|w4n(`~!{J{;pux5q5&`DyC}ecFnXr>!`7+KQ8>tvGqwij$|UIC;7*oIC@ZJkz%8 ze~07bnf6$ZN-<}{Gj_4?87o)$jFqc=#?DnfW5w|^R<80HD_8l9m8*QlisNT2I5yYw zv%uuDZM*(e;mf2~&$h>6D#gU)vmGwu=1Z`4{8{Vdde%C*p0#%TS!>6it*7ITv-js6 zr|l}oRYDj&cbxXvYeRePI5tMaaXPPZToUT5&Nhb}$A&MYSTr$6vFO2&V$sAP#iEHp zO1LHl__1xfNU?A)NU4Vlbh?55ZlHgwLb8dk?)KR0REjabobC<_!E?HS{%xv|uR1IG zyRGQ&wxYk=ivDgZ`n#>@@3x}9+lu~fEBd?ZL;ux4|J6W0_T&(Su5)#J?DZsx=$wdFvR`g$OMgP@S^j~d7|J7FXUu{MI)mHRhZAJgp5Q4f@SAF=Y5C7Kz z|JMQku4rx|?7H@yoqrhKz;#xPU1!DEbykdBXT{icR*YR|!I-hJI5(h&H=u{TsvBkx zZ$JikBN~8g0}TPA$zoCegy<=?CDp&h8A3{@=il!yF=xM`F9w@t9Mhj^bEsK_uTL9-VI&k8d+m zzsF2{hxzy}Gc(uMqhVPUKhyBwcuT1YCb-^H+JkX=L>f{?0o5@ zIw#!9di>{v>@cnm(t%=P@2+i+Fgo0O5q@8e5&z_GGBAtIukT$zC7=8N>!f$5KjHkw zqt0(T>}Tod5w*d7z+Y$khxwr;Yya`Rk207>KFapuCRW>{EP>O-M;NWH=k|B+u5*vF zH5`M-UbLH)wm-`5>gEkE#9rLBo87&=^-*@Oc(Zr!uD8pqFLATRuBJ?nvhV4VrLKu} z>}KC<7W*Mt?4@oo>efQ++E}NTH@2I;yEUP;_sC)|^A!WJ-_3qZHW9muzU#%_FN?k0 zSL_wB?%nJIs@N|2t{3|mSuE}=_6@NuyV*xnv7PkYEw&K5KDKo?`#H0_za(pVrBM^) z{~JT(|LY$A*_UjB{J#);RjkK@Bo?FZE}TH-CnQie7(gw=ULD)EoBf*V-VXY%7yB() ztk+kpFSdO*`)yT>f7grsp)9t?S8Q(#E&P!xhL-=xXzg>d*lT>n`l&ykSH;lJ&l|=5 zQWo3iD>e|raQ&4khJpGkquBqH#cuQ!OT;ia{y`PP=>LOJ?4M+@L0_>XO~rp!#W2_Y z*(mmJve=NX*f34}e^bkKKqFc)HnVlUb4?DT=~Qd!kazN!w#uJJN=(O0)7f*>x7 z9q|>rIrh@s&Ns+5I_axXY@aN4i?7(x*tNTz0a*-i517UF%VH^Cv9Z|8b~^`Tv3B}u zw01-m8}}8Ph+Vhaxmgx#qpzaaRgNH`PAu(obhftg6lWFwY|-OKcxACkr>(tJl%bTp zt+8F$FMdg6{rDESMc@u0J0Q0RGE-S*XSgNHNp7sd*l}nqM!A+zY`D4|3Ri^%Uke>> z@V(fR9GE+WHTWjkU=a=8XQ|@87?(Iv^>?)B7BTn5wqv3aOW=RMBmBJr+ z+$%iaWK4fX~A=Tu^LYurrj`0x2_~G#R{c!mFepprf%QY*0EWF~! z!Yh7ERs5;YiuD!wczDH+hgbZ#Wpp1`4gPM;2A>FT@QLsSpRhFeglh29p$(Sgm@HvT zo(ymB$?yiBv`m^OEtBR+1;`gd0r@rw$TtCyr@{ewDjblfEIoh90^m~?0H0C-{!L8) zpAHA`>2Ls_wgC9F1;D2*06wh%{BkIO`ig$WQuQ;Is-Ll}=x0=u{}S5dkI6wJBerM5 zaq?_9PM)>kg&sqR~)&lsm3h=Ln0=`EAPFgA_92CcqRluq?Z1dEs=(beRt*W@H zW))Xks<>KJacyW7kI0VvG&*t}Kgjj($j)`41LIs5Ixx<47C5d89~k5hQExI zL*F4g^v&o{(gI@A0%FpF^P~mDqy@yJ0&yf1M12k&3ZFxV!iVBe_)r|O%%MY;{v5LO z=a8j8hg5%#hW6*(!qA(2xBt-Z^xn3%N1TaIIFn$h7anzPYh|a~9(Cqg*t^-*N1eRG zKEUp9kIRJK?)2W)N1RiiaPGkPy&h$jq4B~uTLqB+6Cv{dr*8hG|1#u%iebiBCuxj~ zA9@@1)8Vl3M?S!IyU@Xy>`~`zi}<7PaXt(mEsvue)Z6kX057%ejin_L@OYQoqq|P; zWN&G0ebl+PjX|;bEyTo2J9y-)JnH=SHg+0@@=gVhZ*z;iO)KWfDNd;q&2$P}Bmeif z`M>8hG?unjzSC>vJ5lN#ZmD;8rSwL+&`6hPBna|5y~^H+%HHKx_O9^CFlS@wScePq zkGchZ^mJ^u73LrIn)zWg^Io^qd%aRR%rGB8GXga?2w+qn+&p&NdgEkGds3YGm6%*|xjc9(J=m zY-Hm{?tzV}6x$glIyxmWy?5g}$nRq!)1{9XK5416ebjm4LwvpLcK*QmLzzlfLR#>) z{o0Q?PY6}-Zd`+UQYLGK|3p>&8RxV9!ms(5^CznCpN6&kIo0y#LtFlwD9oXY{zMpv zFc4uN!a#(92m=uYA`C?>?LyT)~3 zrmCho--2I~vAUPIYpc0Z#kmo$>I#BTN0@?Z0Pf@hph{=cGvk?=V+-T6QwuYx>2&k3 zZC2od?)>Sw^d-X5g>LLvslX9)X=QDxTs&WF&NuxQ4ohzQ*7S7qh#~dQW&qR0ff-Nb zn+JuwHf#nco!RtsDtkOv0}{+w+f1hAu^EtT$*15XaDh-$0~lIXJ@^e-R^+9kSzu^% zySe|f8TeeMhump6^aD^FJ8R;l1hbA6Rt0B7wHgK!Wwun{(Eofv!q*D;B#jEdz6EJ~ zwx@L9w+bBBvka<0Pde94g`TF6qOjcp#M+Ca3V<{*8U~PN)yuF2&`(7AFOl3vq@Siy z2m=w3{t%=@q@QLBh}KR^=piIUq(7DASdxF6fe757)q3^aI*7Bh*v-XWLYkBP*C%-p7y89Pj|CcyGx#t2^f?xGRmA2xS z_8dB3X{}a}XWWKd?H3&5gTvOU!{U6J;lKCu1}ep7*gSseAG?FncrC5(bsPsk_UDXM`{$NcSm0`FS^ z+dTfs&S<^(5*5Rb0Kn1u@Cm%-AmK~ZIvassG7Bvs6ir4&HN#%+6duF@iu+?>W1Hfz z>|hL{@1C3*0tTTbgqv<^F>MZ3aQ>eRIsgC1{@&HikpDwE8{s@`3S7}xvByC>GkRul zI8i)UD-PQJH@k7r{>B~$t*+W2qJ;(&@rN0U;-E!w(BR59ii1{^u?AQCzJgtMz%p1D z!q|MB#X+O^LYcMY;yvVb8qR$kHe`3QuX~9nXTZE53IT0L-GFEmm(KNHeknoFCRiH2 zui0dc@C5$0Iw@?L3T+{Mi`NXN9CwJlJKvh;0C|u7EZBKnmCH7&%o6Bgmtn#VHzEqY=AKL8jcaz+4EO z@Z1IF`bM^}mA#(5fx_h=&`6?A@73BoUt~*g-OZM%21FXE9v>n&PtLHjR2-iCDdgNG zhqgeFu{&vvtkwu^;RB?TA-F-~k@+r&p%K9#dk}ujPr25l7W?462t=IaO6A7e5tC@;%u2D0!Tve+Q6fkm!#+|jQiS9c9<0yJ zg4I3L*J!mAsP&tmT5LS|+neQJ>6lil+0C%^Y~egE?7h=%sz z*Ztdv{_O8U1)N&!TPOQ95ei{!WA)BF52mtIfc!rcBL6?W_m=;)3G%-dGS9i&Mj(8V z8EZ(VYj-``=sOJRNy0fgcOt3a(i?Dqw`_Xl0SuB9YlI)nMiQvA{anUj-tK zAsXP{ZiQy>X~AGzQlv5V_w0XCUH`y`36%KDVwnDsMiBu8zamEQfAeIF>OYI*|ARAO zx!=x;9T!2;|F7ucvzH^d?%nKPX+!)sz=iO+&wboJ_8;s&2~YoJ3C)WM$bMXeC{nC) zzC=OpAz`>8w!^;CXQzDj5TQ-eyZ~zhBGW7-Y?3fJtxg*Vlhe*;&=58?+XgMTQkTop zct`}5bpC&PR{|JSb+zw(FUbt#0WXUzf*KbTBC@X{nh*j;AS?-LaKR8ph(dx%P_b62 zinUd%b!oM=Rn&@gtD+*Y(qi4JR_j`;-CSE+t+jTqZT-J<&Rb{ROh|~W?f*u@ox7iV z?so2dZ@w={bU}S+RD!0eP!$jxk_B%w?D#9P?K;_u?7k3LnRlg<^oa|upZQeBpvc{pULBi%XIH_)jDbn0t+sf-W(Pv`rpQ49B zjl`n2Cl7JGS5kqWD`wGKIi_)KBVG~8c5@bH*+PHsqxYqe4k#_`cpy*PU-S-k=&^-*w1yG+99IKcwux-g zYS4oRUR_FR*vCcwt4v~hj!)`o1&96ksqKILV>{+3@m0HKW6-c$mu=|Ah8^g}R)WR7 zt+{38^h{IFG3?(wI7odlm$Hl`^Jgf=!r%z_C}&TQUd+u{L7V9nL#8QJR8# z5tMs#B^vi~@Pl%%R{WqTHCAFoE|TQ9MME?g0_!Du2}pxE2qb|=OPvi#^@oCr-4CAn z${~9X)@pY$2XQNqS~0J+b?p&&qcpJ3y`5{gxU6Vt6DaG(lzn5&nx(7L0GZjDeU)vo z4Nl55p+Re^aiR7-jSDp>5UcHS(N+_lih_<9t3I2%V0ZL(HcPMMpS1;dTDuwDZWMJ|DZoku}vvCG`a$yT z?NMb%^uqbS;IL;Fy|Z!8ZFMXPcAtwylrKP(%A^K2X`-&7c0p~!{G`4tsmluvdr^iA z(VEktNPju0Cvs*ia9@$kU&UJ#N&Phr#Ts|5tV-9BpLkBM7yk{Y0^G0Pfc5n=Ck1-&;lS+t`og_73rXD+!uyY8#f@Euw+DlfyMCbHunn`e6yDl^bMAj5!RrqFAT{%Qu&mo$=GL7j zczswuLUf>ydIYbyuo>I6Wfzi3wY4Qc@CtkaI#@l6 z6nG?;o}zo4t}o~psVv<1l#Xz&)q^Chz0Bzfd;_WtM6A8V>G})Iz_uQ6y1u4gr%rss z<8&R4>tvVH6;I3XK9~VCMI!z*6t3>s-QnnO!V!>|$YK7pocEz6^vxE(Qtv37$f!;do7Ir&v?z{Ym;gN&_Kn z7$@=X$dqw>xt(xnQ-MtCv8Dnh8aT#|HwFBTIl%Rp&#=&nNG$`ggykHFCA8)liB9#D zBz`;``%2jRr^I%33c=411L+J1Ah{Tie@4D50Hdn=HR@567_m;`s!*AFe%O0tIPuc3 zcLi`#Rsc!mM9KtU9NkRBt*3OVIwFYGEAmV9$x%@u0oEhsdKB`hsFa98A)k(l3Kg;u zg=|D2pN&e1C=~L!sHjjOm!ObKPzYgjM>OXW6moe~6biBXdJ4UUl6d{CH-){ohrM^g z8_ax9xjk^ZeicyKZ>HgKhejaYc90)S9-5BFt?t>TwxJ!9#d%E7xgTeA?#Be3`!S$% zKiR3DPDkes)nGiMMqs=vFyFTPX0 zn83%or}%zd{D5iNmSGpbu>m2gwwJ2!cB)7BROQ9ba?*n0`|ScdVJAVqP@)YEbJOX9 zVuiqg9EdMK=S6i%eA}D9C6d6Nl_K@<{)&*i7*)JT?bv9Svad+Jw$UDl93wdgz=%rp z1pb<*jXct2iJq!U^ueZSu$Ww+p5Cb|hEd(KBpa~(r`on%WEb1um!Uc-iHLU;CqY`# zhMfeHiakIiqg0SAU+3pZu46Hg8vCOSQ2^*DW3|Xepx4{`cj~n+YI#2bAU*{kz1jgX zC)nY$oW#JXb`k@&8>s+xB)H|22;k;WS{@-(hf1^woA^SfM1XE@ zIPsNkzCQF_@|EbTIg5sh>_PTmLJX!jZhs#8O10B|f)KuoQ`@CFzkncwuS6s#HqAW5 z-pjE?eMSK(+KKD52w)iNp@#au!^s~?hZqixb8HW^t}AsQhT}A1H*)5#vxuo_fWRsQ znWLZTN^P_3N`Zy4-QrvgrlY9I`;Gq6N1 z+o~?urid9}&N8HL?o^ineT|s6o8i53S*N<1kk!K5Lsgr&HlXCdf=0rT{@VhBE}kzF{ZC{^mACJOeB?Q0HB-Z-TG@&T_GBB$7wM~FS&>AXtR`#6y>rFGupG~X%g8Nd4;C)mF9Ft#9Ep45i=sRTFF4pS-I7l)LJ`MpRwOvx3I#hrriUACW5Yk30VG zptXyR#QA?x#QFa(zyH^k(D|R#D`0!(A-o(60NeX+h40roGwgDSmF^mdOmcNimnz zku>EA@#Z<+qf2 zrAb%0nFY8weaWeDmYf;od7`kaTZlfwUvl&uZ{F)UsCd3R@Adrjd2hReI{TXQP}g0|=-0gvbsy=~ePkDPM=d%t%RK=-DY`SCf=d)VlgOAJE$=ev zC2rD*kx377TD!NmUSeo9l!H|&u(|20Om{BhVN!(B^r2!wX`

    - - org.apache.httpcomponents - httpclient - - org.mockito mockito-all diff --git a/flume-ng-sinks/flume-ng-kafka-sink/src/test/java/org/apache/flume/sink/kafka/util/KafkaLocal.java b/flume-ng-sinks/flume-ng-kafka-sink/src/test/java/org/apache/flume/sink/kafka/util/KafkaLocal.java index 3c6e064c4b..d8a45ef73c 100644 --- a/flume-ng-sinks/flume-ng-kafka-sink/src/test/java/org/apache/flume/sink/kafka/util/KafkaLocal.java +++ b/flume-ng-sinks/flume-ng-kafka-sink/src/test/java/org/apache/flume/sink/kafka/util/KafkaLocal.java @@ -35,7 +35,7 @@ public class KafkaLocal { public KafkaLocal(Properties kafkaProperties) throws IOException, InterruptedException{ - KafkaConfig kafkaConfig = new KafkaConfig(kafkaProperties); + KafkaConfig kafkaConfig = KafkaConfig.fromProps(kafkaProperties); //start local kafka broker kafka = new KafkaServerStartable(kafkaConfig); diff --git a/pom.xml b/pom.xml index 694c073d6f..50037c7af6 100644 --- a/pom.xml +++ b/pom.xml @@ -37,8 +37,8 @@ limitations under the License. UTF-8 - 1.6 - 1.6 + 1.7 + 1.7 $ @@ -253,12 +253,6 @@ limitations under the License. hadoop-auth ${hadoop.version} - - org.apache.hadoop - hadoop-minicluster - ${hadoop.version} - test - + +# Welcome to Apache Flume! + +Apache Flume is a distributed, reliable, and available service for efficiently +collecting, aggregating, and moving large amounts of log data. It has a simple +and flexible architecture based on streaming data flows. It is robust and fault +tolerant with tunable reliability mechanisms and many failover and recovery +mechanisms. The system is centrally managed and allows for intelligent dynamic +management. It uses a simple extensible data model that allows for online +analytic application. + +The Apache Flume 1.x (NG) code line is a refactoring of the first generation +Flume to solve certain known issues and limitations of the original design. + +Apache Flume is open-sourced under the Apache Software Foundation License v2.0. + +## Documentation + +Documentation is included in the binary distribution under the docs directory. +In source form, it can be found in the flume-ng-doc directory. + +The Flume 1.x guide and FAQ are available here: + +* https://cwiki.apache.org/FLUME/flume-ng.html +* https://cwiki.apache.org/confluence/display/FLUME/Getting+Started + +## Contact us! + +* Mailing lists: https://cwiki.apache.org/confluence/display/FLUME/Mailing+Lists +* IRC channel #flume on irc.freenode.net + +Bug and Issue tracker. + +* https://issues.apache.org/jira/browse/FLUME + +## Compiling Flume + +Compiling Flume requires the following tools: + +* Oracle Java JDK 1.7 +* Apache Maven 3.x + +Note: The Apache Flume build requires more memory than the default configuration. +We recommend you set the following Maven options: + +export MAVEN_OPTS="-Xms512m -Xmx1024m -XX:PermSize=256m -XX:MaxPermSize=512m" + +To compile Flume, run `mvn compile`. +To build a distribution, run `mvn install`. + +The final Flume distribution artifacts will be in $project/flume-ng-dist/target/. diff --git a/README.textile b/README.textile deleted file mode 100644 index 2b884fa16e..0000000000 --- a/README.textile +++ /dev/null @@ -1,100 +0,0 @@ -h1. Welcome to Apache Flume! - -Apache Flume is a distributed, reliable, and available service for efficiently -collecting, aggregating, and moving large amounts of log data. It has a simple -and flexible architecture based on streaming data flows. It is robust and fault -tolerant with tunable reliability mechanisms and many failover and recovery -mechanisms. The system is centrally managed and allows for intelligent dynamic -management. It uses a simple extensible data model that allows for online -analytic application. - -The Apache Flume 1.x (NG) code line is a refactoring of the first generation -Flume to solve certain known issues and limitations of the original design. - -Apache Flume is open-sourced under the Apache Software Foundation License v2.0. - -h2. Documentation - -Documentation is included in the binary distribution under the docs directory. -In source form, it can be found in the flume-ng-doc directory. - -The Flume 1.x guide and FAQ are available here: - -* https://cwiki.apache.org/FLUME/flume-ng.html -* https://cwiki.apache.org/confluence/display/FLUME/Getting+Started - -h2. Contact us! - -* Mailing lists: https://cwiki.apache.org/confluence/display/FLUME/Mailing+Lists -* IRC channel #flume on irc.freenode.net - -Bug and Issue tracker. - -* https://issues.apache.org/jira/browse/FLUME - -h2. Compiling Flume - -Compiling Flume requires the following tools: - -* Oracle Java JDK 1.7 -* Apache Maven 3.x - -Note: The Apache Flume build requires more memory than the default configuration. -We recommend you set the following Maven options: - -export MAVEN_OPTS="-Xms512m -Xmx1024m -XX:PermSize=256m -XX:MaxPermSize=512m" - -To compile Flume, run `mvn compile`. -To build a distribution, run `mvn install`. - -The final Flume distribution artifacts will be in $project/flume-ng-dist/target/. -h1. Welcome to Apache Flume! - -Apache Flume is a distributed, reliable, and available service for efficiently -collecting, aggregating, and moving large amounts of log data. It has a simple -and flexible architecture based on streaming data flows. It is robust and fault -tolerant with tunable reliability mechanisms and many failover and recovery -mechanisms. The system is centrally managed and allows for intelligent dynamic -management. It uses a simple extensible data model that allows for online -analytic application. - -The Apache Flume 1.x (NG) code line is a refactoring of the first generation -Flume to solve certain known issues and limitations of the original design. - -Apache Flume is open-sourced under the Apache Software Foundation License v2.0. - -h2. Documentation - -Documentation is included in the binary distribution under the docs directory. -In source form, it can be found in the flume-ng-doc directory. - -The Flume 1.x guide and FAQ are available here: - -* https://cwiki.apache.org/FLUME/flume-ng.html -* https://cwiki.apache.org/confluence/display/FLUME/Getting+Started - -h2. Contact us! - -* Mailing lists: https://cwiki.apache.org/confluence/display/FLUME/Mailing+Lists -* IRC channel #flume on irc.freenode.net - -Bug and Issue tracker. - -* https://issues.apache.org/jira/browse/FLUME - -h2. Compiling Flume - -Compiling Flume requires the following tools: - -* Oracle Java JDK 1.7 -* Apache Maven 3.x - -Note: The Apache Flume build requires more memory than the default configuration. -We recommend you set the following Maven options: - -export MAVEN_OPTS="-Xms512m -Xmx1024m -XX:PermSize=256m -XX:MaxPermSize=512m" - -To compile Flume, run `mvn compile`. -To build a distribution, run `mvn install`. - -The final Flume distribution artifacts will be in $project/flume-ng-dist/target/. From 4eacba1930c3abfddf4fb50c5a272798708df811 Mon Sep 17 00:00:00 2001 From: "Santiago M. Mola" Date: Wed, 3 Dec 2014 18:30:56 +0100 Subject: [PATCH 293/341] FLUME-2567. Remove external Maven repositories (Santiago M. Mola via Mike Percy) --- flume-ng-sinks/flume-dataset-sink/pom.xml | 23 --------- .../flume-ng-morphline-solr-sink/pom.xml | 23 --------- pom.xml | 49 ------------------- 3 files changed, 95 deletions(-) diff --git a/flume-ng-sinks/flume-dataset-sink/pom.xml b/flume-ng-sinks/flume-dataset-sink/pom.xml index e59127a9e2..186fdf8434 100644 --- a/flume-ng-sinks/flume-dataset-sink/pom.xml +++ b/flume-ng-sinks/flume-dataset-sink/pom.xml @@ -46,29 +46,6 @@ limitations under the License. - - - cdh.repo - https://repository.cloudera.com/artifactory/cloudera-repos - Cloudera Repositories - - false - - - - - cdh.snapshots.repo - https://repository.cloudera.com/artifactory/libs-snapshot-local - Cloudera Snapshots Repository - - true - - - false - - - - diff --git a/flume-ng-sinks/flume-ng-morphline-solr-sink/pom.xml b/flume-ng-sinks/flume-ng-morphline-solr-sink/pom.xml index 4dfa2920cd..8e0bc5ec04 100644 --- a/flume-ng-sinks/flume-ng-morphline-solr-sink/pom.xml +++ b/flume-ng-sinks/flume-ng-morphline-solr-sink/pom.xml @@ -37,29 +37,6 @@ limitations under the License. 2.12.4 - - - cdh.repo - https://repository.cloudera.com/artifactory/cloudera-repos - Cloudera Repositories - - false - - - - - cdh.snapshots.repo - https://repository.cloudera.com/artifactory/libs-snapshot-local - Cloudera Snapshots Repository - - true - - - false - - - - diff --git a/pom.xml b/pom.xml index 50037c7af6..99543534a9 100644 --- a/pom.xml +++ b/pom.xml @@ -616,55 +616,6 @@ limitations under the License. http://www.apache.org - - - - repo1.maven.org - http://repo1.maven.org/maven2 - - true - - - false - - - - - oss.sonatype.org - https://oss.sonatype.org/content/groups/public - - true - - - false - - - - - - - - repo1.maven.org - http://repo1.maven.org/maven2 - - true - - - false - - - - - repository.jboss.org - http://repository.jboss.org/nexus/content/groups/public/ - - - false - - - - - apache.staging.https From 2fe393898f420d100117ca277cce198858e2c24f Mon Sep 17 00:00:00 2001 From: Mike Percy Date: Tue, 28 Jun 2016 13:44:51 -0700 Subject: [PATCH 294/341] FLUME-1899. Make SpoolDir work with subdirectories MIME-Version: 1.0 Content-Type: text/plain; charset=UTF-8 Content-Transfer-Encoding: 8bit (Phil Scala and Bessenyei Balázs Donát via Mike Percy) --- .../avro/ReliableSpoolingFileEventReader.java | 103 ++++++++---- .../flume/source/SpoolDirectorySource.java | 14 +- ...DirectorySourceConfigurationConstants.java | 12 +- .../source/TestSpoolDirectorySource.java | 155 ++++++++++++++++-- flume-ng-doc/sphinx/FlumeUserGuide.rst | 88 +++++----- 5 files changed, 281 insertions(+), 91 deletions(-) diff --git a/flume-ng-core/src/main/java/org/apache/flume/client/avro/ReliableSpoolingFileEventReader.java b/flume-ng-core/src/main/java/org/apache/flume/client/avro/ReliableSpoolingFileEventReader.java index d54f415d29..36d80f0d00 100644 --- a/flume-ng-core/src/main/java/org/apache/flume/client/avro/ReliableSpoolingFileEventReader.java +++ b/flume-ng-core/src/main/java/org/apache/flume/client/avro/ReliableSpoolingFileEventReader.java @@ -80,7 +80,6 @@ public class ReliableSpoolingFileEventReader implements ReliableEventReader { .getLogger(ReliableSpoolingFileEventReader.class); static final String metaFileName = ".flumespool-main.meta"; - private final File spoolDirectory; private final String completedSuffix; private final String deserializerType; @@ -94,8 +93,9 @@ public class ReliableSpoolingFileEventReader implements ReliableEventReader { private final String deletePolicy; private final Charset inputCharset; private final DecodeErrorPolicy decodeErrorPolicy; - private final ConsumeOrder consumeOrder; - + private final ConsumeOrder consumeOrder; + private final boolean recursiveDirectorySearch; + private Optional currentFile = Optional.absent(); /** Always contains the last file from which lines have been read. **/ private Optional lastFileRead = Optional.absent(); @@ -114,8 +114,9 @@ private ReliableSpoolingFileEventReader(File spoolDirectory, boolean annotateBaseName, String baseNameHeader, String deserializerType, Context deserializerContext, String deletePolicy, String inputCharset, - DecodeErrorPolicy decodeErrorPolicy, - ConsumeOrder consumeOrder) throws IOException { + DecodeErrorPolicy decodeErrorPolicy, + ConsumeOrder consumeOrder, + boolean recursiveDirectorySearch) throws IOException { // Sanity checks Preconditions.checkNotNull(spoolDirectory); @@ -175,7 +176,8 @@ private ReliableSpoolingFileEventReader(File spoolDirectory, this.deletePolicy = deletePolicy; this.inputCharset = Charset.forName(inputCharset); this.decodeErrorPolicy = Preconditions.checkNotNull(decodeErrorPolicy); - this.consumeOrder = Preconditions.checkNotNull(consumeOrder); + this.consumeOrder = Preconditions.checkNotNull(consumeOrder); + this.recursiveDirectorySearch = recursiveDirectorySearch; File trackerDirectory = new File(trackerDirPath); @@ -199,11 +201,60 @@ private ReliableSpoolingFileEventReader(File spoolDirectory, } this.metaFile = new File(trackerDirectory, metaFileName); + if(metaFile.exists() && metaFile.length() == 0) { deleteMetaFile(); } } + /** + * Filter to exclude files/directories either hidden, finished, or names matching the ignore pattern + */ + final FileFilter filter = new FileFilter() { + public boolean accept(File candidate) { + if (candidate.isDirectory()) { + String directoryName = candidate.getName(); + if (!recursiveDirectorySearch || directoryName.startsWith(".") || + ignorePattern.matcher(directoryName).matches()) { + + return false; + } + return true; + } + String fileName = candidate.getName(); + if (fileName.endsWith(completedSuffix) || fileName.startsWith(".") || + ignorePattern.matcher(fileName).matches()) { + return false; + } + + return true; + } + }; + + /** + * Recursively gather candidate files + * @param directory the directory to gather files from + * @return list of files within the passed in directory + */ + private List getCandidateFiles(File directory) { + Preconditions.checkNotNull(directory); + List candidateFiles = new ArrayList(); + if (!directory.isDirectory()) { + return candidateFiles; + } + + for(File file : directory.listFiles(filter)){ + if (file.isDirectory()) { + candidateFiles.addAll(getCandidateFiles(file)); + } + else { + candidateFiles.add(file); + } + } + + return candidateFiles; + } + @VisibleForTesting int getListFilesCount() { return listFilesCount; @@ -432,22 +483,9 @@ private Optional getNextFile() { List candidateFiles = Collections.emptyList(); if (consumeOrder != ConsumeOrder.RANDOM || - candidateFileIter == null || - !candidateFileIter.hasNext()) { - /* Filter to exclude finished or hidden files */ - FileFilter filter = new FileFilter() { - public boolean accept(File candidate) { - String fileName = candidate.getName(); - if ((candidate.isDirectory()) || - (fileName.endsWith(completedSuffix)) || - (fileName.startsWith(".")) || - ignorePattern.matcher(fileName).matches()) { - return false; - } - return true; - } - }; - candidateFiles = Arrays.asList(spoolDirectory.listFiles(filter)); + candidateFileIter == null || + !candidateFileIter.hasNext()) { + candidateFiles = getCandidateFiles(spoolDirectory); listFilesCount++; candidateFileIter = candidateFiles.iterator(); } @@ -594,9 +632,11 @@ public static class Builder { private DecodeErrorPolicy decodeErrorPolicy = DecodeErrorPolicy.valueOf( SpoolDirectorySourceConfigurationConstants.DEFAULT_DECODE_ERROR_POLICY .toUpperCase(Locale.ENGLISH)); - private ConsumeOrder consumeOrder = - SpoolDirectorySourceConfigurationConstants.DEFAULT_CONSUME_ORDER; - + private ConsumeOrder consumeOrder = + SpoolDirectorySourceConfigurationConstants.DEFAULT_CONSUME_ORDER; + private boolean recursiveDirectorySearch = + SpoolDirectorySourceConfigurationConstants.DEFAULT_RECURSIVE_DIRECTORY_SEARCH; + public Builder spoolDirectory(File directory) { this.spoolDirectory = directory; return this; @@ -657,22 +697,27 @@ public Builder inputCharset(String inputCharset) { return this; } + public Builder recursiveDirectorySearch(boolean recursiveDirectorySearch) { + this.recursiveDirectorySearch = recursiveDirectorySearch; + return this; + } + public Builder decodeErrorPolicy(DecodeErrorPolicy decodeErrorPolicy) { this.decodeErrorPolicy = decodeErrorPolicy; return this; } - + public Builder consumeOrder(ConsumeOrder consumeOrder) { this.consumeOrder = consumeOrder; return this; - } - + } + public ReliableSpoolingFileEventReader build() throws IOException { return new ReliableSpoolingFileEventReader(spoolDirectory, completedSuffix, ignorePattern, trackerDirPath, annotateFileName, fileNameHeader, annotateBaseName, baseNameHeader, deserializerType, deserializerContext, deletePolicy, inputCharset, decodeErrorPolicy, - consumeOrder); + consumeOrder, recursiveDirectorySearch); } } diff --git a/flume-ng-core/src/main/java/org/apache/flume/source/SpoolDirectorySource.java b/flume-ng-core/src/main/java/org/apache/flume/source/SpoolDirectorySource.java index 3fe947dfdf..3af3e53f3f 100644 --- a/flume-ng-core/src/main/java/org/apache/flume/source/SpoolDirectorySource.java +++ b/flume-ng-core/src/main/java/org/apache/flume/source/SpoolDirectorySource.java @@ -70,6 +70,7 @@ public class SpoolDirectorySource extends AbstractSource implements private int maxBackoff; private ConsumeOrder consumeOrder; private int pollDelay; + private boolean recursiveDirectorySearch; @Override public synchronized void start() { @@ -95,6 +96,7 @@ public synchronized void start() { .inputCharset(inputCharset) .decodeErrorPolicy(decodeErrorPolicy) .consumeOrder(consumeOrder) + .recursiveDirectorySearch(recursiveDirectorySearch) .build(); } catch (IOException ioe) { throw new FlumeException("Error instantiating spooling event parser", @@ -162,12 +164,15 @@ public synchronized void configure(Context context) { deserializerType = context.getString(DESERIALIZER, DEFAULT_DESERIALIZER); deserializerContext = new Context(context.getSubProperties(DESERIALIZER + ".")); - - consumeOrder = ConsumeOrder.valueOf(context.getString(CONSUME_ORDER, + + consumeOrder = ConsumeOrder.valueOf(context.getString(CONSUME_ORDER, DEFAULT_CONSUME_ORDER.toString()).toUpperCase(Locale.ENGLISH)); pollDelay = context.getInteger(POLL_DELAY, DEFAULT_POLL_DELAY); + recursiveDirectorySearch = context.getBoolean(RECURSIVE_DIRECTORY_SEARCH, + DEFAULT_RECURSIVE_DIRECTORY_SEARCH); + // "Hack" to support backwards compatibility with previous generation of // spooling directory source, which did not support deserializers Integer bufferMaxLineLength = context.getInteger(BUFFER_MAX_LINE_LENGTH); @@ -210,6 +215,11 @@ protected SourceCounter getSourceCounter() { return sourceCounter; } + @VisibleForTesting + protected boolean getRecursiveDirectorySearch() { + return recursiveDirectorySearch; + } + private class SpoolDirectoryRunnable implements Runnable { private ReliableSpoolingFileEventReader reader; private SourceCounter sourceCounter; diff --git a/flume-ng-core/src/main/java/org/apache/flume/source/SpoolDirectorySourceConfigurationConstants.java b/flume-ng-core/src/main/java/org/apache/flume/source/SpoolDirectorySourceConfigurationConstants.java index 505369764f..32b7837df6 100644 --- a/flume-ng-core/src/main/java/org/apache/flume/source/SpoolDirectorySourceConfigurationConstants.java +++ b/flume-ng-core/src/main/java/org/apache/flume/source/SpoolDirectorySourceConfigurationConstants.java @@ -92,8 +92,16 @@ public enum ConsumeOrder { OLDEST, YOUNGEST, RANDOM } public static final String CONSUME_ORDER = "consumeOrder"; - public static final ConsumeOrder DEFAULT_CONSUME_ORDER = ConsumeOrder.OLDEST; - + public static final ConsumeOrder DEFAULT_CONSUME_ORDER = ConsumeOrder.OLDEST; + + /** + * Flag to indicate if we should recursively checking for new files. The + * default is false, so a configuration file entry would be needed to enable + * this setting + */ + public static final String RECURSIVE_DIRECTORY_SEARCH = "recursiveDirectorySearch"; + public static final boolean DEFAULT_RECURSIVE_DIRECTORY_SEARCH = false; + /** Delay(in milliseconds) used when polling for new files. The default is 500ms */ public static final String POLL_DELAY = "pollDelay"; public static final int DEFAULT_POLL_DELAY = 500; diff --git a/flume-ng-core/src/test/java/org/apache/flume/source/TestSpoolDirectorySource.java b/flume-ng-core/src/test/java/org/apache/flume/source/TestSpoolDirectorySource.java index fe530ff4d7..47fdc7aa64 100644 --- a/flume-ng-core/src/test/java/org/apache/flume/source/TestSpoolDirectorySource.java +++ b/flume-ng-core/src/test/java/org/apache/flume/source/TestSpoolDirectorySource.java @@ -17,15 +17,16 @@ package org.apache.flume.source; +import java.io.ByteArrayOutputStream; import java.io.File; import java.io.IOException; import java.util.ArrayList; +import java.util.Arrays; import java.util.List; import java.util.concurrent.TimeUnit; import com.google.common.collect.Lists; import org.apache.flume.Channel; -import org.apache.flume.ChannelException; import org.apache.flume.ChannelSelector; import org.apache.flume.Context; import org.apache.flume.Event; @@ -34,7 +35,6 @@ import org.apache.flume.channel.MemoryChannel; import org.apache.flume.channel.ReplicatingChannelSelector; import org.apache.flume.conf.Configurables; -import org.apache.flume.instrumentation.SourceCounter; import org.apache.flume.lifecycle.LifecycleController; import org.apache.flume.lifecycle.LifecycleState; import org.junit.After; @@ -69,34 +69,47 @@ public void setUp() { @After public void tearDown() { - for (File f : tmpDir.listFiles()) { - f.delete(); - } + deleteFiles(tmpDir); tmpDir.delete(); } + /** + * Helper method to recursively clean up testing directory + * @param directory the directory to clean up + */ + private void deleteFiles(File directory) { + for (File f : directory.listFiles()) { + if (f.isDirectory()) { + deleteFiles(f); + f.delete(); + } else { + f.delete(); + } + } + } + @Test (expected = IllegalArgumentException.class) public void testInvalidSortOrder() { Context context = new Context(); - context.put(SpoolDirectorySourceConfigurationConstants.SPOOL_DIRECTORY, + context.put(SpoolDirectorySourceConfigurationConstants.SPOOL_DIRECTORY, tmpDir.getAbsolutePath()); - context.put(SpoolDirectorySourceConfigurationConstants.CONSUME_ORDER, + context.put(SpoolDirectorySourceConfigurationConstants.CONSUME_ORDER, "undefined"); - Configurables.configure(source, context); + Configurables.configure(source, context); } - + @Test public void testValidSortOrder() { Context context = new Context(); - context.put(SpoolDirectorySourceConfigurationConstants.SPOOL_DIRECTORY, + context.put(SpoolDirectorySourceConfigurationConstants.SPOOL_DIRECTORY, tmpDir.getAbsolutePath()); context.put(SpoolDirectorySourceConfigurationConstants.CONSUME_ORDER, "oLdESt"); Configurables.configure(source, context); - context.put(SpoolDirectorySourceConfigurationConstants.CONSUME_ORDER, + context.put(SpoolDirectorySourceConfigurationConstants.CONSUME_ORDER, "yoUnGest"); Configurables.configure(source, context); - context.put(SpoolDirectorySourceConfigurationConstants.CONSUME_ORDER, + context.put(SpoolDirectorySourceConfigurationConstants.CONSUME_ORDER, "rAnDom"); Configurables.configure(source, context); } @@ -134,6 +147,9 @@ public void testPutFilenameHeader() throws IOException, InterruptedException { txn.close(); } + /** + * Tests if SpoolDirectorySource sets basename headers on events correctly + */ @Test public void testPutBasenameHeader() throws IOException, InterruptedException { @@ -168,14 +184,121 @@ public void testPutBasenameHeader() throws IOException, txn.close(); } + /** + * Tests SpoolDirectorySource with parameter recursion set to true + */ + @Test + public void testRecursion_SetToTrue() throws IOException, InterruptedException { + File subDir = new File(tmpDir, "directorya/directoryb/directoryc"); + boolean directoriesCreated = subDir.mkdirs(); + Assert.assertTrue("source directories must be created", directoriesCreated); + + final String FILE_NAME = "recursion_file.txt"; + File f1 = new File(subDir, FILE_NAME); + String origBody = "file1line1\nfile1line2\nfile1line3\nfile1line4\n" + + "file1line5\nfile1line6\nfile1line7\nfile1line8\n"; + Files.write(origBody, f1, Charsets.UTF_8); + + Context context = new Context(); + context.put(SpoolDirectorySourceConfigurationConstants.RECURSIVE_DIRECTORY_SEARCH, + "true"); // enable recursion, so we should find the file we created above + context.put(SpoolDirectorySourceConfigurationConstants.SPOOL_DIRECTORY, + tmpDir.getAbsolutePath()); // spool set to root dir + context.put(SpoolDirectorySourceConfigurationConstants.FILENAME_HEADER, + "true"); // put the file name in the "file" header + + Configurables.configure(source, context); + source.start(); + Assert.assertTrue("Recursion setting in source is correct", + source.getRecursiveDirectorySearch()); + + + Transaction txn = channel.getTransaction(); + txn.begin(); + long startTime = System.currentTimeMillis(); + Event e = null; + while (System.currentTimeMillis() - startTime < 300 && e == null) { + e = channel.take(); + Thread.sleep(10); + } + + Assert.assertNotNull("Event must not be null", e); + + Assert.assertNotNull("Event headers must not be null", e.getHeaders()); + Assert.assertTrue("File header value did not end with expected filename", + e.getHeaders().get("file").endsWith(FILE_NAME)); + + ByteArrayOutputStream baos = new ByteArrayOutputStream(); + do { // collecting the whole body + baos.write(e.getBody()); + baos.write('\n'); // newline characters are consumed in the process + e = channel.take(); + } while(e != null); + + Assert.assertEquals("Event body is correct", + Arrays.toString(origBody.getBytes()), + Arrays.toString(baos.toByteArray())); + txn.commit(); + txn.close(); + } + + + /** + * This test will place a file into a sub-directory of the spool directory + * since the recursion setting is false there should not be any transactions + * to take from the channel. The 500 ms is arbitrary and simply follows + * what the other tests use to "assume" that since there is no data then this worked. + */ + @Test + public void testRecursion_SetToFalse() throws IOException, InterruptedException { + Context context = new Context(); + + File subDir = new File(tmpDir, "directory"); + boolean directoriesCreated = subDir.mkdirs(); + Assert.assertTrue("source directories must be created", directoriesCreated); + + + + File f1 = new File(subDir.getAbsolutePath() + "/file1.txt"); + + Files.write("file1line1\nfile1line2\nfile1line3\nfile1line4\n" + + "file1line5\nfile1line6\nfile1line7\nfile1line8\n", f1, Charsets.UTF_8); + + context.put(SpoolDirectorySourceConfigurationConstants.RECURSIVE_DIRECTORY_SEARCH, + "false"); + context.put(SpoolDirectorySourceConfigurationConstants.SPOOL_DIRECTORY, + tmpDir.getAbsolutePath()); + context.put(SpoolDirectorySourceConfigurationConstants.FILENAME_HEADER, + "true"); + context.put(SpoolDirectorySourceConfigurationConstants.FILENAME_HEADER_KEY, + "fileHeaderKeyTest"); + + Configurables.configure(source, context); + source.start(); + // check the source to ensure the setting has been set via the context object + Assert.assertFalse("Recursion setting in source is not set to false (this" + + "test does not want recursion enabled)", source.getRecursiveDirectorySearch()); + + Transaction txn = channel.getTransaction(); + txn.begin(); + long startTime = System.currentTimeMillis(); + Event e = null; + while (System.currentTimeMillis() - startTime < 300 && e == null) { + e = channel.take(); + Thread.sleep(10); + } + Assert.assertNull("Event must be null", e); + txn.commit(); + txn.close(); + } + @Test public void testLifecycle() throws IOException, InterruptedException { Context context = new Context(); File f1 = new File(tmpDir.getAbsolutePath() + "/file1"); Files.write("file1line1\nfile1line2\nfile1line3\nfile1line4\n" + - "file1line5\nfile1line6\nfile1line7\nfile1line8\n", - f1, Charsets.UTF_8); + "file1line5\nfile1line6\nfile1line7\nfile1line8\n", f1, Charsets.UTF_8); context.put(SpoolDirectorySourceConfigurationConstants.SPOOL_DIRECTORY, tmpDir.getAbsolutePath()); @@ -243,8 +366,8 @@ public void testSourceDoesNotDieOnFullChannel() throws Exception { File f1 = new File(tmpDir.getAbsolutePath() + "/file1"); Files.write("file1line1\nfile1line2\nfile1line3\nfile1line4\n" + - "file1line5\nfile1line6\nfile1line7\nfile1line8\n", - f1, Charsets.UTF_8); + "file1line5\nfile1line6\nfile1line7\nfile1line8\n", + f1, Charsets.UTF_8); context.put(SpoolDirectorySourceConfigurationConstants.SPOOL_DIRECTORY, diff --git a/flume-ng-doc/sphinx/FlumeUserGuide.rst b/flume-ng-doc/sphinx/FlumeUserGuide.rst index 287d066e0f..c4d7c6c747 100644 --- a/flume-ng-doc/sphinx/FlumeUserGuide.rst +++ b/flume-ng-doc/sphinx/FlumeUserGuide.rst @@ -976,49 +976,53 @@ Despite the reliability guarantees of this source, there are still cases in which events may be duplicated if certain downstream failures occur. This is consistent with the guarantees offered by other Flume components. -==================== ============== ========================================================== -Property Name Default Description -==================== ============== ========================================================== -**channels** -- -**type** -- The component type name, needs to be ``spooldir``. -**spoolDir** -- The directory from which to read files from. -fileSuffix .COMPLETED Suffix to append to completely ingested files -deletePolicy never When to delete completed files: ``never`` or ``immediate`` -fileHeader false Whether to add a header storing the absolute path filename. -fileHeaderKey file Header key to use when appending absolute path filename to event header. -basenameHeader false Whether to add a header storing the basename of the file. -basenameHeaderKey basename Header Key to use when appending basename of file to event header. -ignorePattern ^$ Regular expression specifying which files to ignore (skip) -trackerDir .flumespool Directory to store metadata related to processing of files. - If this path is not an absolute path, then it is interpreted as relative to the spoolDir. -consumeOrder oldest In which order files in the spooling directory will be consumed ``oldest``, - ``youngest`` and ``random``. In case of ``oldest`` and ``youngest``, the last modified - time of the files will be used to compare the files. In case of a tie, the file - with smallest lexicographical order will be consumed first. In case of ``random`` any - file will be picked randomly. When using ``oldest`` and ``youngest`` the whole - directory will be scanned to pick the oldest/youngest file, which might be slow if there - are a large number of files, while using ``random`` may cause old files to be consumed - very late if new files keep coming in the spooling directory. -pollDelay 500 Delay (in milliseconds) used when polling for new files. -maxBackoff 4000 The maximum time (in millis) to wait between consecutive attempts to write to the channel(s) if the channel is full. The source will start at a low backoff and increase it exponentially each time the channel throws a ChannelException, upto the value specified by this parameter. -batchSize 100 Granularity at which to batch transfer to the channel -inputCharset UTF-8 Character set used by deserializers that treat the input file as text. -decodeErrorPolicy ``FAIL`` What to do when we see a non-decodable character in the input file. - ``FAIL``: Throw an exception and fail to parse the file. - ``REPLACE``: Replace the unparseable character with the "replacement character" char, - typically Unicode U+FFFD. - ``IGNORE``: Drop the unparseable character sequence. -deserializer ``LINE`` Specify the deserializer used to parse the file into events. - Defaults to parsing each line as an event. The class specified must implement - ``EventDeserializer.Builder``. -deserializer.* Varies per event deserializer. -bufferMaxLines -- (Obselete) This option is now ignored. -bufferMaxLineLength 5000 (Deprecated) Maximum length of a line in the commit buffer. Use deserializer.maxLineLength instead. -selector.type replicating replicating or multiplexing -selector.* Depends on the selector.type value -interceptors -- Space-separated list of interceptors +====================== ============== ========================================================== +Property Name Default Description +====================== ============== ========================================================== +**channels** -- +**type** -- The component type name, needs to be ``spooldir``. +**spoolDir** -- The directory from which to read files from. +fileSuffix .COMPLETED Suffix to append to completely ingested files +deletePolicy never When to delete completed files: ``never`` or ``immediate`` +fileHeader false Whether to add a header storing the absolute path filename. +fileHeaderKey file Header key to use when appending absolute path filename to event header. +basenameHeader false Whether to add a header storing the basename of the file. +basenameHeaderKey basename Header Key to use when appending basename of file to event header. +ignorePattern ^$ Regular expression specifying which files to ignore (skip) +trackerDir .flumespool Directory to store metadata related to processing of files. + If this path is not an absolute path, then it is interpreted as relative to the spoolDir. +consumeOrder oldest In which order files in the spooling directory will be consumed ``oldest``, + ``youngest`` and ``random``. In case of ``oldest`` and ``youngest``, the last modified + time of the files will be used to compare the files. In case of a tie, the file + with smallest lexicographical order will be consumed first. In case of ``random`` any + file will be picked randomly. When using ``oldest`` and ``youngest`` the whole + directory will be scanned to pick the oldest/youngest file, which might be slow if there + are a large number of files, while using ``random`` may cause old files to be consumed + very late if new files keep coming in the spooling directory. +pollDelay 500 Delay (in milliseconds) used when polling for new files. +recursiveDirectorySearch false Whether to monitor sub directories for new files to read. +maxBackoff 4000 The maximum time (in millis) to wait between consecutive attempts to + write to the channel(s) if the channel is full. The source will start at + a low backoff and increase it exponentially each time the channel throws a + ChannelException, upto the value specified by this parameter. +batchSize 100 Granularity at which to batch transfer to the channel +inputCharset UTF-8 Character set used by deserializers that treat the input file as text. +decodeErrorPolicy ``FAIL`` What to do when we see a non-decodable character in the input file. + ``FAIL``: Throw an exception and fail to parse the file. + ``REPLACE``: Replace the unparseable character with the "replacement character" char, + typically Unicode U+FFFD. + ``IGNORE``: Drop the unparseable character sequence. +deserializer ``LINE`` Specify the deserializer used to parse the file into events. + Defaults to parsing each line as an event. The class specified must implement + ``EventDeserializer.Builder``. +deserializer.* Varies per event deserializer. +bufferMaxLines -- (Obselete) This option is now ignored. +bufferMaxLineLength 5000 (Deprecated) Maximum length of a line in the commit buffer. Use deserializer.maxLineLength instead. +selector.type replicating replicating or multiplexing +selector.* Depends on the selector.type value +interceptors -- Space-separated list of interceptors interceptors.* -==================== ============== ========================================================== +======================= ============== ========================================================== Example for an agent named agent-1: From 2252fb1938a4fd578f88c64eb444c74777c46212 Mon Sep 17 00:00:00 2001 From: Mike Percy Date: Sun, 26 Jun 2016 02:57:37 -0700 Subject: [PATCH 295/341] FLUME-2937. Integrate checkstyle for non-test classes Based on the Google checkstyle file with modifications. The changes here do not change the generated Java bytecode (after stripping line numbers). They are syntax / whitespace ONLY. Code review: https://reviews.apache.org/r/49403/ Reviewed by Hari. --- flume-checkstyle/pom.xml | 36 ++ .../flume/checkstyle-suppressions.xml | 43 +++ .../src/main/resources/flume/checkstyle.xml | 177 ++++++++++ .../flume/api/SecureRpcClientFactory.java | 4 +- .../flume/api/SecureThriftRpcClient.java | 41 ++- .../flume/auth/FlumeAuthenticationUtil.java | 6 +- .../flume/auth/KerberosAuthenticator.java | 29 +- .../org/apache/flume/auth/KerberosUser.java | 4 +- .../flume/auth/SimpleAuthenticator.java | 4 +- .../org/apache/flume/auth/UGIExecutor.java | 8 +- .../channel/file/BadCheckpointException.java | 3 +- .../channel/file/CheckpointRebuilder.java | 110 +++--- .../org/apache/flume/channel/file/Commit.java | 16 +- .../channel/file/CorruptEventException.java | 3 +- .../file/EventQueueBackingStoreFactory.java | 71 ++-- .../file/EventQueueBackingStoreFile.java | 138 ++++---- .../file/EventQueueBackingStoreFileV2.java | 2 +- .../file/EventQueueBackingStoreFileV3.java | 78 ++--- .../apache/flume/channel/file/EventUtils.java | 2 +- .../flume/channel/file/FileChannel.java | 156 +++++---- .../file/FileChannelConfiguration.java | 6 +- .../apache/flume/channel/file/FlumeEvent.java | 26 +- .../flume/channel/file/FlumeEventPointer.java | 10 +- .../flume/channel/file/FlumeEventQueue.java | 145 ++++---- .../org/apache/flume/channel/file/Log.java | 220 ++++++------ .../apache/flume/channel/file/LogFile.java | 177 ++++++---- .../flume/channel/file/LogFileFactory.java | 73 ++-- .../file/LogFileRetryableIOException.java | 3 + .../apache/flume/channel/file/LogFileV2.java | 19 +- .../apache/flume/channel/file/LogFileV3.java | 113 +++--- .../apache/flume/channel/file/LogRecord.java | 22 +- .../apache/flume/channel/file/LogUtils.java | 2 +- .../org/apache/flume/channel/file/Pair.java | 4 +- .../org/apache/flume/channel/file/Put.java | 13 +- .../flume/channel/file/ReplayHandler.java | 41 ++- .../apache/flume/channel/file/Rollback.java | 9 +- .../flume/channel/file/Serialization.java | 86 +++-- .../org/apache/flume/channel/file/Take.java | 10 +- .../channel/file/TransactionEventRecord.java | 27 +- .../channel/file/TransactionIDOracle.java | 4 +- .../flume/channel/file/WritableUtils.java | 8 +- .../flume/channel/file/WriteOrderOracle.java | 4 +- .../encryption/AESCTRNoPaddingProvider.java | 16 +- .../file/encryption/CipherProvider.java | 17 +- .../encryption/CipherProviderFactory.java | 4 +- .../file/encryption/CipherProviderType.java | 1 - .../DecryptionFailureException.java | 1 - .../file/encryption/JCEFileKeyProvider.java | 10 +- .../file/encryption/KeyProviderType.java | 1 - .../channel/file/proto/ProtosFactory.java | 11 +- .../channel/jdbc/ConfigurationConstants.java | 1 - .../flume/channel/jdbc/JdbcChannel.java | 1 + .../channel/jdbc/JdbcChannelProvider.java | 2 +- .../channel/jdbc/impl/DerbySchemaHandler.java | 24 +- .../jdbc/impl/JdbcChannelProviderImpl.java | 22 +- .../jdbc/impl/JdbcTransactionImpl.java | 1 - .../channel/jdbc/impl/SchemaHandler.java | 4 - .../jdbc/impl/SchemaHandlerFactory.java | 22 +- .../flume/channel/kafka/KafkaChannel.java | 87 ++--- .../kafka/KafkaChannelConfiguration.java | 15 +- .../flume/channel/SpillableMemoryChannel.java | 311 +++++++++-------- .../LoadBalancingLog4jAppender.java | 18 +- .../clients/log4jappender/Log4jAppender.java | 26 +- .../log4jappender/Log4jAvroHeaders.java | 13 +- .../main/java/org/apache/flume/Context.java | 10 +- .../conf/BasicConfigurationConstants.java | 1 - .../flume/conf/ComponentConfiguration.java | 11 +- .../conf/ComponentConfigurationFactory.java | 9 +- .../apache/flume/conf/FlumeConfiguration.java | 142 ++++---- .../flume/conf/FlumeConfigurationError.java | 2 +- .../flume/conf/sink/SinkConfiguration.java | 4 +- .../conf/sink/SinkGroupConfiguration.java | 3 +- .../conf/source/SourceConfiguration.java | 4 +- .../java/org/apache/flume/ChannelFactory.java | 9 +- .../org/apache/flume/ChannelSelector.java | 1 - .../src/main/java/org/apache/flume/Clock.java | 4 +- .../java/org/apache/flume/SinkFactory.java | 8 +- .../java/org/apache/flume/SinkRunner.java | 6 +- .../java/org/apache/flume/SourceFactory.java | 7 +- .../java/org/apache/flume/SourceRunner.java | 2 +- .../java/org/apache/flume/SystemClock.java | 6 +- .../java/org/apache/flume/Transaction.java | 11 +- .../apache/flume/annotations/Disposable.java | 2 + .../flume/annotations/InterfaceStability.java | 1 + .../apache/flume/annotations/Recyclable.java | 2 + .../apache/flume/channel/AbstractChannel.java | 6 +- .../channel/AbstractChannelSelector.java | 2 +- .../channel/BasicTransactionSemantics.java | 1 + .../flume/channel/ChannelSelectorFactory.java | 2 +- .../flume/channel/DefaultChannelFactory.java | 3 +- .../apache/flume/channel/MemoryChannel.java | 98 +++--- .../channel/MultiplexingChannelSelector.java | 8 +- .../flume/channel/PseudoTxnMemoryChannel.java | 6 +- .../channel/ReplicatingChannelSelector.java | 6 +- .../flume/client/avro/AvroCLIClient.java | 17 +- .../avro/ReliableSpoolingFileEventReader.java | 62 ++-- .../org/apache/flume/event/EventHelper.java | 10 +- .../flume/formatter/output/BucketPath.java | 327 +++++++++--------- .../formatter/output/DefaultPathManager.java | 2 +- .../flume/formatter/output/PathManager.java | 32 +- .../formatter/output/PathManagerFactory.java | 89 ++--- .../formatter/output/PathManagerType.java | 20 +- .../formatter/output/RollTimePathManager.java | 56 +-- .../flume/instrumentation/ChannelCounter.java | 10 +- .../flume/instrumentation/GangliaServer.java | 6 - .../MonitoredCounterGroup.java | 21 +- .../flume/instrumentation/MonitoringType.java | 2 +- .../flume/instrumentation/SinkCounter.java | 2 - .../flume/instrumentation/SourceCounter.java | 17 +- .../http/HTTPMetricsServer.java | 8 +- .../kafka/KafkaChannelCounter.java | 6 +- .../instrumentation/util/JMXPollUtil.java | 18 +- .../flume/interceptor/HostInterceptor.java | 3 +- .../RegexExtractorInterceptor.java | 11 +- .../RegexFilteringInterceptor.java | 13 +- .../SearchAndReplaceInterceptor.java | 6 +- .../flume/interceptor/StaticInterceptor.java | 13 +- .../interceptor/TimestampInterceptor.java | 1 + .../flume/lifecycle/LifecycleAware.java | 58 ++-- .../flume/lifecycle/LifecycleSupervisor.java | 31 +- .../serialization/AvroEventDeserializer.java | 5 +- .../BodyTextEventSerializer.java | 3 +- .../HeaderAndBodyTextEventSerializer.java | 2 +- .../flume/serialization/LineDeserializer.java | 3 +- .../ResettableFileInputStream.java | 28 +- .../apache/flume/sink/AbstractRpcSink.java | 51 ++- .../org/apache/flume/sink/AbstractSink.java | 5 +- .../flume/sink/AbstractSinkProcessor.java | 4 +- .../flume/sink/AbstractSinkSelector.java | 2 +- .../apache/flume/sink/DefaultSinkFactory.java | 3 +- .../flume/sink/DefaultSinkProcessor.java | 3 +- .../flume/sink/FailoverSinkProcessor.java | 31 +- .../sink/LoadBalancingSinkProcessor.java | 34 +- .../org/apache/flume/sink/LoggerSink.java | 5 +- .../java/org/apache/flume/sink/NullSink.java | 2 +- .../apache/flume/sink/RollingFileSink.java | 12 +- .../flume/sink/SinkProcessorFactory.java | 5 +- .../org/apache/flume/sink/ThriftSink.java | 12 +- .../source/AbstractEventDrivenSource.java | 6 +- .../flume/source/AbstractPollableSource.java | 10 +- .../apache/flume/source/AbstractSource.java | 4 +- .../org/apache/flume/source/AvroSource.java | 102 +++--- .../flume/source/BasicSourceSemantics.java | 9 +- .../flume/source/DefaultSourceFactory.java | 5 +- .../org/apache/flume/source/ExecSource.java | 50 +-- .../ExecSourceConfigurationConstants.java | 3 +- .../source/MultiportSyslogTCPSource.java | 4 +- .../NetcatSourceConfigurationConstants.java | 6 +- .../flume/source/PollableSourceRunner.java | 8 +- .../flume/source/SequenceGeneratorSource.java | 6 +- .../flume/source/SpoolDirectorySource.java | 20 +- ...DirectorySourceConfigurationConstants.java | 1 + .../org/apache/flume/source/StressSource.java | 9 +- .../org/apache/flume/source/SyslogParser.java | 13 +- .../apache/flume/source/SyslogTcpSource.java | 8 +- .../apache/flume/source/SyslogUDPSource.java | 26 +- .../org/apache/flume/source/SyslogUtils.java | 262 +++++++------- .../org/apache/flume/source/ThriftSource.java | 61 ++-- .../apache/flume/source/http/BLOBHandler.java | 8 +- .../apache/flume/source/http/HTTPSource.java | 44 +-- .../apache/flume/source/http/JSONHandler.java | 14 +- .../apache/flume/tools/DirectMemoryUtils.java | 10 +- .../apache/flume/tools/GetJavaProperty.java | 2 +- .../flume/tools/TimestampRoundDownUtil.java | 9 +- .../org/apache/flume/tools/VersionInfo.java | 16 +- .../flume/agent/embedded/EmbeddedAgent.java | 38 +- .../embedded/EmbeddedAgentConfiguration.java | 53 ++- .../flume/agent/embedded/EmbeddedSource.java | 6 +- .../thriftLegacy/ThriftLegacySource.java | 10 +- .../node/AbstractConfigurationProvider.java | 121 ++++--- .../org/apache/flume/node/Application.java | 114 +++--- .../flume/node/ConfigurationProvider.java | 7 +- ...ngPropertiesFileConfigurationProvider.java | 13 +- .../node/SimpleMaterializedConfiguration.java | 2 +- .../apache/flume/api/AbstractRpcClient.java | 2 +- .../apache/flume/api/FailoverRpcClient.java | 6 +- .../java/org/apache/flume/api/HostInfo.java | 2 +- .../flume/api/LoadBalancingRpcClient.java | 9 +- .../apache/flume/api/NettyAvroRpcClient.java | 38 +- .../api/RpcClientConfigurationConstants.java | 6 +- .../apache/flume/api/RpcClientFactory.java | 16 +- .../org/apache/flume/api/ThriftRpcClient.java | 120 +++---- .../org/apache/flume/event/EventBuilder.java | 2 +- .../org/apache/flume/event/JSONEvent.java | 6 +- .../org/apache/flume/event/SimpleEvent.java | 2 +- .../org/apache/flume/util/OrderSelector.java | 5 +- .../apache/flume/sink/kite/DatasetSink.java | 12 +- .../kite/NonRecoverableEventException.java | 1 - .../sink/kite/parser/EntityParserFactory.java | 1 - .../kite/policy/FailurePolicyFactory.java | 1 - .../flume/sink/hdfs/AbstractHDFSWriter.java | 15 +- .../sink/hdfs/BucketClosedException.java | 2 +- .../apache/flume/sink/hdfs/BucketWriter.java | 127 ++++--- .../sink/hdfs/HDFSCompressedDataStream.java | 9 +- .../flume/sink/hdfs/HDFSDataStream.java | 19 +- .../apache/flume/sink/hdfs/HDFSEventSink.java | 76 ++-- .../flume/sink/hdfs/HDFSSequenceFile.java | 7 +- .../apache/flume/sink/hdfs/KerberosUser.java | 4 +- .../sink/hdfs/SequenceFileSerializerType.java | 3 +- .../hive/HiveDelimitedTextSerializer.java | 16 +- .../flume/sink/hive/HiveEventSerializer.java | 1 - .../org/apache/flume/sink/hive/HiveSink.java | 30 +- .../apache/flume/sink/hive/HiveWriter.java | 141 ++++---- .../org/apache/flume/sink/irc/IRCSink.java | 4 +- ...asticSearchIndexRequestBuilderFactory.java | 5 +- ...asticSearchIndexRequestBuilderFactory.java | 2 +- .../sink/elasticsearch/ElasticSearchSink.java | 1 + ...tSerializerIndexRequestBuilderFactory.java | 2 +- .../client/ElasticSearchClientFactory.java | 6 +- .../client/ElasticSearchRestClient.java | 23 +- .../client/ElasticSearchTransportClient.java | 6 +- .../elasticsearch/client/RoundRobinList.java | 2 +- .../flume/sink/hbase/AsyncHBaseSink.java | 254 +++++++------- .../sink/hbase/AsyncHbaseEventSerializer.java | 5 +- .../apache/flume/sink/hbase/HBaseSink.java | 117 +++---- .../sink/hbase/HbaseEventSerializer.java | 10 +- .../sink/hbase/RegexHbaseEventSerializer.java | 50 +-- .../SimpleAsyncHbaseEventSerializer.java | 33 +- .../hbase/SimpleHbaseEventSerializer.java | 73 ++-- .../sink/hbase/SimpleRowKeyGenerator.java | 22 +- .../apache/flume/sink/kafka/KafkaSink.java | 45 +-- .../flume/sink/kafka/KafkaSinkConstants.java | 15 +- .../sink/solr/morphline/BlobDeserializer.java | 3 +- .../sink/solr/morphline/BlobHandler.java | 3 +- .../solr/morphline/MorphlineHandlerImpl.java | 6 +- .../solr/morphline/MorphlineInterceptor.java | 5 +- .../sink/solr/morphline/MorphlineSink.java | 8 +- .../jms/DefaultJMSMessageConverter.java | 28 +- .../source/jms/InitialContextFactory.java | 1 - .../flume/source/jms/JMSMessageConsumer.java | 84 +++-- .../source/jms/JMSMessageConsumerFactory.java | 10 +- .../apache/flume/source/jms/JMSSource.java | 124 +++---- .../flume/source/kafka/KafkaSource.java | 55 +-- .../source/kafka/KafkaSourceConstants.java | 9 +- .../taildir/ReliableTaildirEventReader.java | 52 +-- .../apache/flume/source/taildir/TailFile.java | 88 +++-- .../flume/source/taildir/TaildirMatcher.java | 106 +++--- .../flume/source/taildir/TaildirSource.java | 18 +- .../TaildirSourceConfigurationConstants.java | 4 +- .../TestRpcClientCommunicationFailure.java | 60 ++-- .../flume/tools/FileChannelIntegrityTool.java | 85 +++-- .../org/apache/flume/tools/FlumeToolType.java | 2 +- .../apache/flume/tools/FlumeToolsMain.java | 19 +- pom.xml | 66 ++++ 244 files changed, 3823 insertions(+), 3436 deletions(-) create mode 100644 flume-checkstyle/pom.xml create mode 100644 flume-checkstyle/src/main/resources/flume/checkstyle-suppressions.xml create mode 100644 flume-checkstyle/src/main/resources/flume/checkstyle.xml diff --git a/flume-checkstyle/pom.xml b/flume-checkstyle/pom.xml new file mode 100644 index 0000000000..31db3c0f2b --- /dev/null +++ b/flume-checkstyle/pom.xml @@ -0,0 +1,36 @@ + + + + 4.0.0 + + + + org.apache.flume + flume-checkstyle + Flume checkstyle project + 1.7.0-SNAPSHOT + diff --git a/flume-checkstyle/src/main/resources/flume/checkstyle-suppressions.xml b/flume-checkstyle/src/main/resources/flume/checkstyle-suppressions.xml new file mode 100644 index 0000000000..49c88347fb --- /dev/null +++ b/flume-checkstyle/src/main/resources/flume/checkstyle-suppressions.xml @@ -0,0 +1,43 @@ + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + diff --git a/flume-checkstyle/src/main/resources/flume/checkstyle.xml b/flume-checkstyle/src/main/resources/flume/checkstyle.xml new file mode 100644 index 0000000000..e8913f0d8d --- /dev/null +++ b/flume-checkstyle/src/main/resources/flume/checkstyle.xml @@ -0,0 +1,177 @@ + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + diff --git a/flume-ng-auth/src/main/java/org/apache/flume/api/SecureRpcClientFactory.java b/flume-ng-auth/src/main/java/org/apache/flume/api/SecureRpcClientFactory.java index c976458204..35356cdcf2 100644 --- a/flume-ng-auth/src/main/java/org/apache/flume/api/SecureRpcClientFactory.java +++ b/flume-ng-auth/src/main/java/org/apache/flume/api/SecureRpcClientFactory.java @@ -26,8 +26,8 @@ public class SecureRpcClientFactory { /** - * Return a secure {@linkplain org.apache.flume.api.RpcClient} that uses Thrift for communicating with - * the next hop. + * Return a secure {@linkplain org.apache.flume.api.RpcClient} that uses Thrift for communicating + * with the next hop. * @param props * @return - An {@linkplain org.apache.flume.api.RpcClient} which uses thrift configured with the * given parameters. diff --git a/flume-ng-auth/src/main/java/org/apache/flume/api/SecureThriftRpcClient.java b/flume-ng-auth/src/main/java/org/apache/flume/api/SecureThriftRpcClient.java index f31582c984..395bc1f18d 100644 --- a/flume-ng-auth/src/main/java/org/apache/flume/api/SecureThriftRpcClient.java +++ b/flume-ng-auth/src/main/java/org/apache/flume/api/SecureThriftRpcClient.java @@ -22,7 +22,10 @@ import org.apache.flume.auth.FlumeAuthenticationUtil; import org.apache.flume.auth.FlumeAuthenticator; import org.apache.flume.auth.PrivilegedExecutor; -import org.apache.thrift.transport.*; +import org.apache.thrift.transport.TSaslClientTransport; +import org.apache.thrift.transport.TSocket; +import org.apache.thrift.transport.TTransport; +import org.apache.thrift.transport.TTransportException; import javax.security.auth.callback.CallbackHandler; import javax.security.sasl.Sasl; @@ -52,9 +55,9 @@ protected void configure(Properties properties) throws FlumeException { String clientPrincipal = properties.getProperty(CLIENT_PRINCIPAL); String keytab = properties.getProperty(CLIENT_KEYTAB); this.privilegedExecutor = FlumeAuthenticationUtil.getAuthenticator(clientPrincipal, keytab); - if(!privilegedExecutor.isAuthenticated()) { + if (!privilegedExecutor.isAuthenticated()) { throw new FlumeException("Authentication failed in Kerberos mode for " + - "principal " + clientPrincipal + " keytab " + keytab); + "principal " + clientPrincipal + " keytab " + keytab); } } @@ -78,31 +81,33 @@ protected TTransport getTransport(TSocket tsocket) throws Exception { */ public static class UgiSaslClientTransport extends TSaslClientTransport { PrivilegedExecutor privilegedExecutor; + public UgiSaslClientTransport(String mechanism, String authorizationId, String protocol, String serverName, Map props, - CallbackHandler cbh, TTransport transport, PrivilegedExecutor privilegedExecutor) throws IOException { - super(mechanism, authorizationId, protocol, serverName, props, cbh, - transport); + CallbackHandler cbh, TTransport transport, PrivilegedExecutor privilegedExecutor) + throws IOException { + super(mechanism, authorizationId, protocol, serverName, props, cbh, transport); this.privilegedExecutor = privilegedExecutor; } - // open the SASL transport with using the current UserGroupInformation - // This is needed to get the current login context stored + /** + * Open the SASL transport with using the current UserGroupInformation. + * This is needed to get the current login context stored + */ @Override public void open() throws FlumeException { try { this.privilegedExecutor.execute( - new PrivilegedExceptionAction() { - public Void run() throws FlumeException { - // this is a workaround to using UgiSaslClientTransport.super.open() - // which results in IllegalAccessError - callSuperClassOpen(); - return null; - } - }); + new PrivilegedExceptionAction() { + public Void run() throws FlumeException { + // this is a workaround to using UgiSaslClientTransport.super.open() + // which results in IllegalAccessError + callSuperClassOpen(); + return null; + } + }); } catch (InterruptedException e) { - throw new FlumeException( - "Interrupted while opening underlying transport", e); + throw new FlumeException("Interrupted while opening underlying transport", e); } catch (Exception e) { throw new FlumeException("Failed to open SASL transport", e); } diff --git a/flume-ng-auth/src/main/java/org/apache/flume/auth/FlumeAuthenticationUtil.java b/flume-ng-auth/src/main/java/org/apache/flume/auth/FlumeAuthenticationUtil.java index 562765279e..87cef31820 100644 --- a/flume-ng-auth/src/main/java/org/apache/flume/auth/FlumeAuthenticationUtil.java +++ b/flume-ng-auth/src/main/java/org/apache/flume/auth/FlumeAuthenticationUtil.java @@ -50,10 +50,10 @@ private FlumeAuthenticationUtil() {} * * @throws org.apache.flume.auth.SecurityException */ - public synchronized static FlumeAuthenticator getAuthenticator( + public static synchronized FlumeAuthenticator getAuthenticator( String principal, String keytab) throws SecurityException { - if(principal == null && keytab == null) { + if (principal == null && keytab == null) { return SimpleAuthenticator.getSimpleAuthenticator(); } @@ -62,7 +62,7 @@ public synchronized static FlumeAuthenticator getAuthenticator( Preconditions.checkArgument(keytab != null, "Keytab can not be null when Principal is provided"); - if(kerbAuthenticator == null) { + if (kerbAuthenticator == null) { kerbAuthenticator = new KerberosAuthenticator(); } kerbAuthenticator.authenticate(principal, keytab); diff --git a/flume-ng-auth/src/main/java/org/apache/flume/auth/KerberosAuthenticator.java b/flume-ng-auth/src/main/java/org/apache/flume/auth/KerberosAuthenticator.java index 4a0e0f456c..45091f57d5 100644 --- a/flume-ng-auth/src/main/java/org/apache/flume/auth/KerberosAuthenticator.java +++ b/flume-ng-auth/src/main/java/org/apache/flume/auth/KerberosAuthenticator.java @@ -18,6 +18,7 @@ package org.apache.flume.auth; import com.google.common.annotations.VisibleForTesting; +import com.google.common.base.Preconditions; import org.slf4j.Logger; import org.slf4j.LoggerFactory; @@ -36,7 +37,6 @@ import org.apache.hadoop.security.UserGroupInformation; import org.apache.hadoop.security.UserGroupInformation.AuthenticationMethod; -import com.google.common.base.Preconditions; import static org.apache.hadoop.fs.CommonConfigurationKeys.HADOOP_SECURITY_AUTHENTICATION; /** @@ -66,10 +66,10 @@ public T execute(PrivilegedExceptionAction action) throws Exception { @Override public synchronized PrivilegedExecutor proxyAs(String proxyUserName) { - if(proxyUserName == null || proxyUserName.isEmpty()) { + if (proxyUserName == null || proxyUserName.isEmpty()) { return this; } - if(proxyCache.get(proxyUserName) == null) { + if (proxyCache.get(proxyUserName) == null) { UserGroupInformation proxyUgi; proxyUgi = UserGroupInformation.createProxyUser(proxyUserName, ugi); printUGI(proxyUgi); @@ -131,13 +131,13 @@ public synchronized void authenticate(String principal, String keytab) { KerberosUser newUser = new KerberosUser(resolvedPrincipal, keytab); Preconditions.checkState(prevUser == null || prevUser.equals(newUser), - "Cannot use multiple kerberos principals in the same agent. " + - " Must restart agent to use new principal or keytab. " + - "Previous = %s, New = %s", prevUser, newUser); + "Cannot use multiple kerberos principals in the same agent. " + + " Must restart agent to use new principal or keytab. " + + "Previous = %s, New = %s", prevUser, newUser); // enable the kerberos mode of UGI, before doing anything else - if(!UserGroupInformation.isSecurityEnabled()) { + if (!UserGroupInformation.isSecurityEnabled()) { Configuration conf = new Configuration(false); conf.set(HADOOP_SECURITY_AUTHENTICATION, "kerberos"); UserGroupInformation.setConfiguration(conf); @@ -147,7 +147,7 @@ public synchronized void authenticate(String principal, String keytab) { UserGroupInformation curUser = null; try { curUser = UserGroupInformation.getLoginUser(); - if(curUser != null && !curUser.hasKerberosCredentials()) { + if (curUser != null && !curUser.hasKerberosCredentials()) { curUser = null; } } catch (IOException e) { @@ -166,8 +166,8 @@ public synchronized void authenticate(String principal, String keytab) { if (curUser != null && curUser.getUserName().equals(ugi.getUserName())) { LOG.debug("Using existing principal login: {}", ugi); } else { - LOG.info("Attempting kerberos Re-login as principal ({}) " - , new Object[] { ugi.getUserName() } ); + LOG.info("Attempting kerberos Re-login as principal ({}) ", + new Object[] { ugi.getUserName() } ); ugi.reloginFromKeytab(); } } else { @@ -192,9 +192,10 @@ private void printUGI(UserGroupInformation ugi) { // dump login information AuthenticationMethod authMethod = ugi.getAuthenticationMethod(); LOG.info("\n{} \nUser: {} \nAuth method: {} \nKeytab: {} \n", - new Object[]{ authMethod.equals(AuthenticationMethod.PROXY) ? - "Proxy as: " : "Logged as: ", ugi.getUserName(), authMethod, - ugi.isFromKeytab() } + new Object[] { + authMethod.equals(AuthenticationMethod.PROXY) ? "Proxy as: " : "Logged as: ", + ugi.getUserName(), authMethod, ugi.isFromKeytab() + } ); } } @@ -224,7 +225,7 @@ public void run() { @VisibleForTesting String getUserName() { - if(ugi != null) { + if (ugi != null) { return ugi.getUserName(); } else { return null; diff --git a/flume-ng-auth/src/main/java/org/apache/flume/auth/KerberosUser.java b/flume-ng-auth/src/main/java/org/apache/flume/auth/KerberosUser.java index dd37721fe5..22852de110 100644 --- a/flume-ng-auth/src/main/java/org/apache/flume/auth/KerberosUser.java +++ b/flume-ng-auth/src/main/java/org/apache/flume/auth/KerberosUser.java @@ -46,7 +46,9 @@ public boolean equals(Object obj) { return false; } final KerberosUser other = (KerberosUser) obj; - if ((this.principal == null) ? (other.principal != null) : !this.principal.equals(other.principal)) { + if ((this.principal == null) ? + (other.principal != null) : + !this.principal.equals(other.principal)) { return false; } if ((this.keyTab == null) ? (other.keyTab != null) : !this.keyTab.equals(other.keyTab)) { diff --git a/flume-ng-auth/src/main/java/org/apache/flume/auth/SimpleAuthenticator.java b/flume-ng-auth/src/main/java/org/apache/flume/auth/SimpleAuthenticator.java index f7b5beac06..d2791a1a9b 100644 --- a/flume-ng-auth/src/main/java/org/apache/flume/auth/SimpleAuthenticator.java +++ b/flume-ng-auth/src/main/java/org/apache/flume/auth/SimpleAuthenticator.java @@ -57,10 +57,10 @@ public T execute(PrivilegedAction action) { @Override public synchronized PrivilegedExecutor proxyAs(String proxyUserName) { - if(proxyUserName == null || proxyUserName.isEmpty()) { + if (proxyUserName == null || proxyUserName.isEmpty()) { return this; } - if(proxyCache.get(proxyUserName) == null) { + if (proxyCache.get(proxyUserName) == null) { UserGroupInformation proxyUgi; try { proxyUgi = UserGroupInformation.createProxyUser(proxyUserName, diff --git a/flume-ng-auth/src/main/java/org/apache/flume/auth/UGIExecutor.java b/flume-ng-auth/src/main/java/org/apache/flume/auth/UGIExecutor.java index cd62b91721..a6ebd86e3c 100644 --- a/flume-ng-auth/src/main/java/org/apache/flume/auth/UGIExecutor.java +++ b/flume-ng-auth/src/main/java/org/apache/flume/auth/UGIExecutor.java @@ -55,7 +55,7 @@ public T execute(PrivilegedExceptionAction action) throws Exception { private void ensureValidAuth() { reloginUGI(ugi); - if(ugi.getAuthenticationMethod().equals(AuthenticationMethod.PROXY)) { + if (ugi.getAuthenticationMethod().equals(AuthenticationMethod.PROXY)) { reloginUGI(ugi.getRealUser()); } } @@ -70,9 +70,9 @@ private void ensureValidAuth() { */ private void reloginUGI(UserGroupInformation ugi) { try { - if(ugi.hasKerberosCredentials()) { + if (ugi.hasKerberosCredentials()) { long now = System.currentTimeMillis(); - if(now - lastReloginAttempt < MIN_TIME_BEFORE_RELOGIN) { + if (now - lastReloginAttempt < MIN_TIME_BEFORE_RELOGIN) { return; } lastReloginAttempt = now; @@ -86,7 +86,7 @@ private void reloginUGI(UserGroupInformation ugi) { @VisibleForTesting String getUserName() { - if(ugi != null) { + if (ugi != null) { return ugi.getUserName(); } else { return null; diff --git a/flume-ng-channels/flume-file-channel/src/main/java/org/apache/flume/channel/file/BadCheckpointException.java b/flume-ng-channels/flume-file-channel/src/main/java/org/apache/flume/channel/file/BadCheckpointException.java index 588506a849..b75c29e44d 100644 --- a/flume-ng-channels/flume-file-channel/src/main/java/org/apache/flume/channel/file/BadCheckpointException.java +++ b/flume-ng-channels/flume-file-channel/src/main/java/org/apache/flume/channel/file/BadCheckpointException.java @@ -24,12 +24,13 @@ * Exception thrown when the checkpoint directory contains invalid data, * probably due to the channel stopping while the checkpoint was written. */ -public class BadCheckpointException extends FlumeException{ +public class BadCheckpointException extends FlumeException { private static final long serialVersionUID = -5038652693746472779L; public BadCheckpointException(String msg) { super(msg); } + public BadCheckpointException(String msg, Throwable t) { super(msg, t); } diff --git a/flume-ng-channels/flume-file-channel/src/main/java/org/apache/flume/channel/file/CheckpointRebuilder.java b/flume-ng-channels/flume-file-channel/src/main/java/org/apache/flume/channel/file/CheckpointRebuilder.java index b961ae20e2..a0ecdeb668 100644 --- a/flume-ng-channels/flume-file-channel/src/main/java/org/apache/flume/channel/file/CheckpointRebuilder.java +++ b/flume-ng-channels/flume-file-channel/src/main/java/org/apache/flume/channel/file/CheckpointRebuilder.java @@ -23,12 +23,6 @@ import com.google.common.collect.Lists; import com.google.common.collect.SetMultimap; import com.google.common.collect.Sets; - -import java.io.EOFException; -import java.io.File; -import java.io.IOException; -import java.util.List; -import java.util.Set; import org.apache.commons.cli.CommandLine; import org.apache.commons.cli.CommandLineParser; import org.apache.commons.cli.GnuParser; @@ -37,26 +31,28 @@ import org.slf4j.Logger; import org.slf4j.LoggerFactory; +import java.io.EOFException; +import java.io.File; +import java.io.IOException; +import java.util.List; +import java.util.Set; + public class CheckpointRebuilder { private final List logFiles; private final FlumeEventQueue queue; - private final Set committedPuts = - Sets.newHashSet(); - private final Set pendingTakes = - Sets.newHashSet(); + private final Set committedPuts = Sets.newHashSet(); + private final Set pendingTakes = Sets.newHashSet(); private final SetMultimap uncommittedPuts = - HashMultimap.create(); + HashMultimap.create(); private final SetMultimap - uncommittedTakes = HashMultimap.create(); + uncommittedTakes = HashMultimap.create(); private final boolean fsyncPerTransaction; - private static Logger LOG = - LoggerFactory.getLogger(CheckpointRebuilder.class); + private static Logger LOG = LoggerFactory.getLogger(CheckpointRebuilder.class); - public CheckpointRebuilder(List logFiles, - FlumeEventQueue queue, boolean fsyncPerTransaction) throws - IOException { + public CheckpointRebuilder(List logFiles, FlumeEventQueue queue, + boolean fsyncPerTransaction) throws IOException { this.logFiles = logFiles; this.queue = queue; this.fsyncPerTransaction = fsyncPerTransaction; @@ -68,8 +64,8 @@ public boolean rebuild() throws IOException, Exception { for (File logFile : logFiles) { try { logReaders.add(LogFileFactory.getSequentialReader(logFile, null, - fsyncPerTransaction)); - } catch(EOFException e) { + fsyncPerTransaction)); + } catch (EOFException e) { LOG.warn("Ignoring " + logFile + " due to EOF", e); } } @@ -84,27 +80,24 @@ public boolean rebuild() throws IOException, Exception { TransactionEventRecord record = entry.getEvent(); long trans = record.getTransactionID(); long writeOrderID = record.getLogWriteOrderID(); - transactionIDSeed = Math.max(trans, transactionIDSeed); - writeOrderIDSeed = Math.max(writeOrderID, writeOrderIDSeed); + transactionIDSeed = Math.max(trans, transactionIDSeed); + writeOrderIDSeed = Math.max(writeOrderID, writeOrderIDSeed); if (record.getRecordType() == TransactionEventRecord.Type.PUT.get()) { uncommittedPuts.put(record.getTransactionID(), - new ComparableFlumeEventPointer( + new ComparableFlumeEventPointer( new FlumeEventPointer(fileID, offset), record.getLogWriteOrderID())); - } else if (record.getRecordType() - == TransactionEventRecord.Type.TAKE.get()) { + } else if (record.getRecordType() == TransactionEventRecord.Type.TAKE.get()) { Take take = (Take) record; uncommittedTakes.put(record.getTransactionID(), - new ComparableFlumeEventPointer( + new ComparableFlumeEventPointer( new FlumeEventPointer(take.getFileID(), take.getOffset()), record.getLogWriteOrderID())); - } else if (record.getRecordType() - == TransactionEventRecord.Type.COMMIT.get()) { + } else if (record.getRecordType() == TransactionEventRecord.Type.COMMIT.get()) { Commit commit = (Commit) record; - if (commit.getType() - == TransactionEventRecord.Type.PUT.get()) { + if (commit.getType() == TransactionEventRecord.Type.PUT.get()) { Set puts = - uncommittedPuts.get(record.getTransactionID()); + uncommittedPuts.get(record.getTransactionID()); if (puts != null) { for (ComparableFlumeEventPointer put : puts) { if (!pendingTakes.remove(put)) { @@ -114,7 +107,7 @@ public boolean rebuild() throws IOException, Exception { } } else { Set takes = - uncommittedTakes.get(record.getTransactionID()); + uncommittedTakes.get(record.getTransactionID()); if (takes != null) { for (ComparableFlumeEventPointer take : takes) { if (!committedPuts.remove(take)) { @@ -123,8 +116,7 @@ public boolean rebuild() throws IOException, Exception { } } } - } else if (record.getRecordType() - == TransactionEventRecord.Type.ROLLBACK.get()) { + } else if (record.getRecordType() == TransactionEventRecord.Type.ROLLBACK.get()) { if (uncommittedPuts.containsKey(record.getTransactionID())) { uncommittedPuts.removeAll(record.getTransactionID()); } else { @@ -134,18 +126,16 @@ public boolean rebuild() throws IOException, Exception { } } } catch (Exception e) { - LOG.warn("Error while generating checkpoint " - + "using fast generation logic", e); + LOG.warn("Error while generating checkpoint using fast generation logic", e); return false; } finally { - TransactionIDOracle.setSeed(transactionIDSeed); - WriteOrderOracle.setSeed(writeOrderIDSeed); + TransactionIDOracle.setSeed(transactionIDSeed); + WriteOrderOracle.setSeed(writeOrderIDSeed); for (LogFile.SequentialReader reader : logReaders) { reader.close(); } } - Set sortedPuts = - Sets.newTreeSet(committedPuts); + Set sortedPuts = Sets.newTreeSet(committedPuts); int count = 0; for (ComparableFlumeEventPointer put : sortedPuts) { queue.addTail(put.pointer); @@ -159,9 +149,9 @@ private void writeCheckpoint() throws IOException { long checkpointLogOrderID = 0; List metaDataWriters = Lists.newArrayList(); for (File logFile : logFiles) { - String name = logFile.getName(); - metaDataWriters.add(LogFileFactory.getMetaDataWriter(logFile, - Integer.parseInt(name.substring(name.lastIndexOf('-') + 1)))); + String name = logFile.getName(); + metaDataWriters.add(LogFileFactory.getMetaDataWriter(logFile, + Integer.parseInt(name.substring(name.lastIndexOf('-') + 1)))); } try { if (queue.checkpoint(true)) { @@ -171,8 +161,7 @@ private void writeCheckpoint() throws IOException { } } } catch (Exception e) { - LOG.warn("Error while generating checkpoint " - + "using fast generation logic", e); + LOG.warn("Error while generating checkpoint using fast generation logic", e); } finally { for (LogFile.MetaDataWriter metaDataWriter : metaDataWriters) { metaDataWriter.close(); @@ -181,14 +170,14 @@ private void writeCheckpoint() throws IOException { } private final class ComparableFlumeEventPointer - implements Comparable { + implements Comparable { private final FlumeEventPointer pointer; private final long orderID; - public ComparableFlumeEventPointer(FlumeEventPointer pointer, long orderID){ + public ComparableFlumeEventPointer(FlumeEventPointer pointer, long orderID) { Preconditions.checkNotNull(pointer, "FlumeEventPointer cannot be" - + "null while creating a ComparableFlumeEventPointer"); + + "null while creating a ComparableFlumeEventPointer"); this.pointer = pointer; this.orderID = orderID; } @@ -204,22 +193,22 @@ public int compareTo(ComparableFlumeEventPointer o) { } @Override - public int hashCode(){ + public int hashCode() { return pointer.hashCode(); } @Override - public boolean equals(Object o){ - if(this == o){ + public boolean equals(Object o) { + if (this == o) { return true; } - if(o == null){ + if (o == null) { return false; } - if(o.getClass() != this.getClass()){ + if (o.getClass() != this.getClass()) { return false; } - return pointer.equals(((ComparableFlumeEventPointer)o).pointer); + return pointer.equals(((ComparableFlumeEventPointer) o).pointer); } } @@ -245,20 +234,19 @@ public static void main(String[] args) throws Exception { } int capacity = Integer.parseInt(cli.getOptionValue("t")); File checkpointFile = new File(checkpointDir, "checkpoint"); - if(checkpointFile.exists()) { + if (checkpointFile.exists()) { LOG.error("Cannot execute fast replay", - new IllegalStateException("Checkpoint exists" + checkpointFile)); + new IllegalStateException("Checkpoint exists" + checkpointFile)); } else { EventQueueBackingStore backingStore = EventQueueBackingStoreFactory.get(checkpointFile, capacity, "channel"); FlumeEventQueue queue = new FlumeEventQueue(backingStore, - new File(checkpointDir, "inflighttakes"), - new File(checkpointDir, "inflightputs"), - new File(checkpointDir, Log.QUEUE_SET)); - CheckpointRebuilder rebuilder = new CheckpointRebuilder(logFiles, - queue, true); - if(rebuilder.rebuild()) { + new File(checkpointDir, "inflighttakes"), + new File(checkpointDir, "inflightputs"), + new File(checkpointDir, Log.QUEUE_SET)); + CheckpointRebuilder rebuilder = new CheckpointRebuilder(logFiles, queue, true); + if (rebuilder.rebuild()) { rebuilder.writeCheckpoint(); } else { LOG.error("Could not rebuild the checkpoint due to errors."); diff --git a/flume-ng-channels/flume-file-channel/src/main/java/org/apache/flume/channel/file/Commit.java b/flume-ng-channels/flume-file-channel/src/main/java/org/apache/flume/channel/file/Commit.java index 366324486b..8fd53ccbd0 100644 --- a/flume-ng-channels/flume-file-channel/src/main/java/org/apache/flume/channel/file/Commit.java +++ b/flume-ng-channels/flume-file-channel/src/main/java/org/apache/flume/channel/file/Commit.java @@ -36,44 +36,52 @@ class Commit extends TransactionEventRecord { * Type of Commit Take|Put */ private short type; + Commit(Long transactionID, Long logWriteOrderID) { super(transactionID, logWriteOrderID); } + Commit(Long transactionID, Long logWriteOrderID, short type) { this(transactionID, logWriteOrderID); this.type = type; } + @Override public void readFields(DataInput in) throws IOException { super.readFields(in); type = in.readShort(); } + @Override void writeProtos(OutputStream out) throws IOException { - ProtosFactory.Commit.Builder commitBuilder = - ProtosFactory.Commit.newBuilder(); + ProtosFactory.Commit.Builder commitBuilder = ProtosFactory.Commit.newBuilder(); commitBuilder.setType(type); commitBuilder.build().writeDelimitedTo(out); } + @Override void readProtos(InputStream in) throws IOException { - ProtosFactory.Commit commit = Preconditions.checkNotNull(ProtosFactory. - Commit.parseDelimitedFrom(in), "Commit cannot be null"); + ProtosFactory.Commit commit = + Preconditions.checkNotNull(ProtosFactory.Commit.parseDelimitedFrom(in), + "Commit cannot be null"); type = (short) commit.getType(); } short getType() { return type; } + @Override public void write(DataOutput out) throws IOException { super.write(out); out.writeShort(type); } + @Override short getRecordType() { return Type.COMMIT.get(); } + @Override public String toString() { StringBuilder builder = new StringBuilder(); diff --git a/flume-ng-channels/flume-file-channel/src/main/java/org/apache/flume/channel/file/CorruptEventException.java b/flume-ng-channels/flume-file-channel/src/main/java/org/apache/flume/channel/file/CorruptEventException.java index 691d291795..5438f2edfd 100644 --- a/flume-ng-channels/flume-file-channel/src/main/java/org/apache/flume/channel/file/CorruptEventException.java +++ b/flume-ng-channels/flume-file-channel/src/main/java/org/apache/flume/channel/file/CorruptEventException.java @@ -18,10 +18,9 @@ */ package org.apache.flume.channel.file; - public class CorruptEventException extends Exception { - private static final long serialVersionUID = -2986946303540798416L; + public CorruptEventException() { super(); } diff --git a/flume-ng-channels/flume-file-channel/src/main/java/org/apache/flume/channel/file/EventQueueBackingStoreFactory.java b/flume-ng-channels/flume-file-channel/src/main/java/org/apache/flume/channel/file/EventQueueBackingStoreFactory.java index 456df34e2d..dcd6f987d8 100644 --- a/flume-ng-channels/flume-file-channel/src/main/java/org/apache/flume/channel/file/EventQueueBackingStoreFactory.java +++ b/flume-ng-channels/flume-file-channel/src/main/java/org/apache/flume/channel/file/EventQueueBackingStoreFactory.java @@ -18,92 +18,91 @@ */ package org.apache.flume.channel.file; -import java.io.File; -import java.io.IOException; -import java.io.RandomAccessFile; - +import com.google.common.io.Files; import org.slf4j.Logger; import org.slf4j.LoggerFactory; -import com.google.common.io.Files; +import java.io.File; +import java.io.IOException; +import java.io.RandomAccessFile; class EventQueueBackingStoreFactory { - private static final Logger LOG = LoggerFactory - .getLogger(EventQueueBackingStoreFactory.class); + private static final Logger LOG = LoggerFactory.getLogger(EventQueueBackingStoreFactory.class); + private EventQueueBackingStoreFactory() {} + static EventQueueBackingStore get(File checkpointFile, int capacity, - String name) throws Exception { + String name) throws Exception { return get(checkpointFile, capacity, name, true); } static EventQueueBackingStore get(File checkpointFile, int capacity, - String name, boolean upgrade) throws Exception { + String name, boolean upgrade) throws Exception { return get(checkpointFile, null, capacity, name, upgrade, false, false); } - static EventQueueBackingStore get(File checkpointFile, - File backupCheckpointDir, int capacity,String name, - boolean upgrade, boolean shouldBackup, boolean compressBackup) - throws Exception { + + static EventQueueBackingStore get(File checkpointFile, File backupCheckpointDir, + int capacity, String name, boolean upgrade, + boolean shouldBackup, boolean compressBackup) throws Exception { File metaDataFile = Serialization.getMetaDataFile(checkpointFile); RandomAccessFile checkpointFileHandle = null; try { boolean checkpointExists = checkpointFile.exists(); boolean metaDataExists = metaDataFile.exists(); - if(metaDataExists) { + if (metaDataExists) { // if we have a metadata file but no checkpoint file, we have a problem // delete everything in the checkpoint directory and force // a full replay. - if(!checkpointExists || checkpointFile.length() == 0) { + if (!checkpointExists || checkpointFile.length() == 0) { LOG.warn("MetaData file for checkpoint " - + " exists but checkpoint does not. Checkpoint = " + checkpointFile - + ", metaDataFile = " + metaDataFile); + + " exists but checkpoint does not. Checkpoint = " + checkpointFile + + ", metaDataFile = " + metaDataFile); throw new BadCheckpointException( - "The last checkpoint was not completed correctly, " + - "since Checkpoint file does not exist while metadata " + - "file does."); + "The last checkpoint was not completed correctly, " + + "since Checkpoint file does not exist while metadata " + + "file does."); } } // brand new, use v3 - if(!checkpointExists) { - if(!checkpointFile.createNewFile()) { + if (!checkpointExists) { + if (!checkpointFile.createNewFile()) { throw new IOException("Cannot create " + checkpointFile); } return new EventQueueBackingStoreFileV3(checkpointFile, capacity, name, backupCheckpointDir, shouldBackup, compressBackup); } // v3 due to meta file, version will be checked by backing store - if(metaDataExists) { + if (metaDataExists) { return new EventQueueBackingStoreFileV3(checkpointFile, capacity, - name, backupCheckpointDir, shouldBackup, compressBackup); + name, backupCheckpointDir, shouldBackup, compressBackup); } checkpointFileHandle = new RandomAccessFile(checkpointFile, "r"); - int version = (int)checkpointFileHandle.readLong(); - if(Serialization.VERSION_2 == version) { - if(upgrade) { + int version = (int) checkpointFileHandle.readLong(); + if (Serialization.VERSION_2 == version) { + if (upgrade) { return upgrade(checkpointFile, capacity, name, backupCheckpointDir, - shouldBackup, compressBackup); + shouldBackup, compressBackup); } return new EventQueueBackingStoreFileV2(checkpointFile, capacity, name); } LOG.error("Found version " + Integer.toHexString(version) + " in " + checkpointFile); throw new BadCheckpointException("Checkpoint file exists with " + - Serialization.VERSION_3 + " but no metadata file found."); + Serialization.VERSION_3 + " but no metadata file found."); } finally { - if(checkpointFileHandle != null) { + if (checkpointFileHandle != null) { try { checkpointFileHandle.close(); - } catch(IOException e) { + } catch (IOException e) { LOG.warn("Unable to close " + checkpointFile, e); } } } } - private static EventQueueBackingStore upgrade(File checkpointFile, - int capacity, String name, File backupCheckpointDir, - boolean shouldBackup, boolean compressBackup) - throws Exception { + private static EventQueueBackingStore upgrade(File checkpointFile, int capacity, String name, + File backupCheckpointDir, boolean shouldBackup, + boolean compressBackup) throws Exception { LOG.info("Attempting upgrade of " + checkpointFile + " for " + name); EventQueueBackingStoreFileV2 backingStoreV2 = new EventQueueBackingStoreFileV2(checkpointFile, capacity, name); @@ -115,7 +114,7 @@ private static EventQueueBackingStore upgrade(File checkpointFile, EventQueueBackingStoreFileV3.upgrade(backingStoreV2, checkpointFile, metaDataFile); return new EventQueueBackingStoreFileV3(checkpointFile, capacity, name, - backupCheckpointDir, shouldBackup, compressBackup); + backupCheckpointDir, shouldBackup, compressBackup); } } diff --git a/flume-ng-channels/flume-file-channel/src/main/java/org/apache/flume/channel/file/EventQueueBackingStoreFile.java b/flume-ng-channels/flume-file-channel/src/main/java/org/apache/flume/channel/file/EventQueueBackingStoreFile.java index 2b0987b40a..73f1d4c8a3 100644 --- a/flume-ng-channels/flume-file-channel/src/main/java/org/apache/flume/channel/file/EventQueueBackingStoreFile.java +++ b/flume-ng-channels/flume-file-channel/src/main/java/org/apache/flume/channel/file/EventQueueBackingStoreFile.java @@ -18,6 +18,15 @@ */ package org.apache.flume.channel.file; +import com.google.common.base.Preconditions; +import com.google.common.base.Throwables; +import com.google.common.collect.ImmutableSortedSet; +import com.google.common.collect.Maps; +import com.google.common.collect.SetMultimap; +import com.google.common.util.concurrent.ThreadFactoryBuilder; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + import java.io.File; import java.io.IOException; import java.io.RandomAccessFile; @@ -34,21 +43,9 @@ import java.util.concurrent.TimeUnit; import java.util.concurrent.atomic.AtomicInteger; -import com.google.common.base.Throwables; -import com.google.common.util.concurrent.ThreadFactoryBuilder; -import org.slf4j.Logger; -import org.slf4j.LoggerFactory; - -import com.google.common.base.Preconditions; -import com.google.common.collect.ImmutableSortedSet; -import com.google.common.collect.Maps; -import com.google.common.collect.SetMultimap; - - abstract class EventQueueBackingStoreFile extends EventQueueBackingStore { - private static final Logger LOG = LoggerFactory - .getLogger(EventQueueBackingStoreFile.class); - private static final int MAX_ALLOC_BUFFER_SIZE = 2*1024*1024; // 2MB + private static final Logger LOG = LoggerFactory.getLogger(EventQueueBackingStoreFile.class); + private static final int MAX_ALLOC_BUFFER_SIZE = 2 * 1024 * 1024; // 2MB protected static final int HEADER_SIZE = 1029; protected static final int INDEX_VERSION = 0; protected static final int INDEX_WRITE_ORDER_ID = 1; @@ -71,15 +68,15 @@ abstract class EventQueueBackingStoreFile extends EventQueueBackingStore { private final ExecutorService checkpointBackUpExecutor; protected EventQueueBackingStoreFile(int capacity, String name, - File checkpointFile) throws IOException, + File checkpointFile) throws IOException, BadCheckpointException { this(capacity, name, checkpointFile, null, false, false); } protected EventQueueBackingStoreFile(int capacity, String name, - File checkpointFile, File checkpointBackupDir, - boolean backupCheckpoint, boolean compressBackup) - throws IOException, BadCheckpointException { + File checkpointFile, File checkpointBackupDir, + boolean backupCheckpoint, boolean compressBackup) + throws IOException, BadCheckpointException { super(capacity, name); this.checkpointFile = checkpointFile; this.shouldBackup = backupCheckpoint; @@ -87,7 +84,7 @@ protected EventQueueBackingStoreFile(int capacity, String name, this.backupDir = checkpointBackupDir; checkpointFileHandle = new RandomAccessFile(checkpointFile, "rw"); long totalBytes = (capacity + HEADER_SIZE) * Serialization.SIZE_OF_LONG; - if(checkpointFileHandle.length() == 0) { + if (checkpointFileHandle.length() == 0) { allocate(checkpointFile, totalBytes); checkpointFileHandle.seek(INDEX_VERSION * Serialization.SIZE_OF_LONG); checkpointFileHandle.writeLong(getVersion()); @@ -95,7 +92,7 @@ protected EventQueueBackingStoreFile(int capacity, String name, LOG.info("Preallocated " + checkpointFile + " to " + checkpointFileHandle.length() + " for capacity " + capacity); } - if(checkpointFile.length() != totalBytes) { + if (checkpointFile.length() != totalBytes) { String msg = "Configured capacity is " + capacity + " but the " + " checkpoint file capacity is " + ((checkpointFile.length() / Serialization.SIZE_OF_LONG) - HEADER_SIZE) @@ -108,20 +105,20 @@ protected EventQueueBackingStoreFile(int capacity, String name, elementsBuffer = mappedBuffer.asLongBuffer(); long version = elementsBuffer.get(INDEX_VERSION); - if(version != (long) getVersion()) { + if (version != (long) getVersion()) { throw new BadCheckpointException("Invalid version: " + version + " " + - name + ", expected " + getVersion()); + name + ", expected " + getVersion()); } long checkpointComplete = elementsBuffer.get(INDEX_CHECKPOINT_MARKER); - if(checkpointComplete != (long) CHECKPOINT_COMPLETE) { + if (checkpointComplete != (long) CHECKPOINT_COMPLETE) { throw new BadCheckpointException("Checkpoint was not completed correctly," - + " probably because the agent stopped while the channel was" - + " checkpointing."); + + " probably because the agent stopped while the channel was" + + " checkpointing."); } if (shouldBackup) { checkpointBackUpExecutor = Executors.newSingleThreadExecutor( - new ThreadFactoryBuilder().setNameFormat( - getName() + " - CheckpointBackUpThread").build()); + new ThreadFactoryBuilder().setNameFormat( + getName() + " - CheckpointBackUpThread").build()); } else { checkpointBackUpExecutor = null; } @@ -142,13 +139,13 @@ protected long getCheckpointLogWriteOrderID() { * @param backupDirectory - the directory to which the backup files should be * copied. * @throws IOException - if the copy failed, or if there is not enough disk - * space to copy the checkpoint files over. + * space to copy the checkpoint files over. */ protected void backupCheckpoint(File backupDirectory) throws IOException { int availablePermits = backupCompletedSema.drainPermits(); Preconditions.checkState(availablePermits == 0, - "Expected no permits to be available in the backup semaphore, " + - "but " + availablePermits + " permits were available."); + "Expected no permits to be available in the backup semaphore, " + + "but " + availablePermits + " permits were available."); if (slowdownBackup) { try { TimeUnit.SECONDS.sleep(10); @@ -160,45 +157,45 @@ protected void backupCheckpoint(File backupDirectory) throws IOException { if (backupExists(backupDirectory)) { if (!backupFile.delete()) { throw new IOException("Error while doing backup of checkpoint. Could " + - "not remove" + backupFile.toString() + "."); + "not remove" + backupFile.toString() + "."); } } Serialization.deleteAllFiles(backupDirectory, Log.EXCLUDES); File checkpointDir = checkpointFile.getParentFile(); File[] checkpointFiles = checkpointDir.listFiles(); Preconditions.checkNotNull(checkpointFiles, "Could not retrieve files " + - "from the checkpoint directory. Cannot complete backup of the " + - "checkpoint."); + "from the checkpoint directory. Cannot complete backup of the " + + "checkpoint."); for (File origFile : checkpointFiles) { - if(Log.EXCLUDES.contains(origFile.getName())) { + if (Log.EXCLUDES.contains(origFile.getName())) { continue; } if (compressBackup && origFile.equals(checkpointFile)) { Serialization.compressFile(origFile, new File(backupDirectory, - origFile.getName() + COMPRESSED_FILE_EXTENSION)); + origFile.getName() + COMPRESSED_FILE_EXTENSION)); } else { Serialization.copyFile(origFile, new File(backupDirectory, - origFile.getName())); + origFile.getName())); } } Preconditions.checkState(!backupFile.exists(), "The backup file exists " + - "while it is not supposed to. Are multiple channels configured to use " + - "this directory: " + backupDirectory.toString() + " as backup?"); + "while it is not supposed to. Are multiple channels configured to use " + + "this directory: " + backupDirectory.toString() + " as backup?"); if (!backupFile.createNewFile()) { LOG.error("Could not create backup file. Backup of checkpoint will " + - "not be used during replay even if checkpoint is bad."); + "not be used during replay even if checkpoint is bad."); } } /** * Restore the checkpoint, if it is found to be bad. + * * @return true - if the previous backup was successfully completed and * restore was successfully completed. * @throws IOException - If restore failed due to IOException - * */ public static boolean restoreBackup(File checkpointDir, File backupDir) - throws IOException { + throws IOException { if (!backupExists(backupDir)) { return false; } @@ -210,14 +207,14 @@ public static boolean restoreBackup(File checkpointDir, File backupDir) for (File backupFile : backupFiles) { String fileName = backupFile.getName(); if (!fileName.equals(BACKUP_COMPLETE_FILENAME) && - !fileName.equals(Log.FILE_LOCK)) { - if (fileName.endsWith(COMPRESSED_FILE_EXTENSION)){ + !fileName.equals(Log.FILE_LOCK)) { + if (fileName.endsWith(COMPRESSED_FILE_EXTENSION)) { Serialization.decompressFile( - backupFile, new File(checkpointDir, - fileName.substring(0, fileName.lastIndexOf(".")))); + backupFile, new File(checkpointDir, + fileName.substring(0, fileName.lastIndexOf(".")))); } else { Serialization.copyFile(backupFile, new File(checkpointDir, - fileName)); + fileName)); } } } @@ -233,14 +230,14 @@ void beginCheckpoint() throws IOException { if (shouldBackup) { int permits = backupCompletedSema.drainPermits(); Preconditions.checkState(permits <= 1, "Expected only one or less " + - "permits to checkpoint, but got " + String.valueOf(permits) + - " permits"); - if(permits < 1) { + "permits to checkpoint, but got " + String.valueOf(permits) + + " permits"); + if (permits < 1) { // Force the checkpoint to not happen by throwing an exception. throw new IOException("Previous backup of checkpoint files is still " + - "in progress. Will attempt to checkpoint only at the end of the " + - "next checkpoint interval. Try increasing the checkpoint interval " + - "if this error happens often."); + "in progress. Will attempt to checkpoint only at the end of the " + + "next checkpoint interval. Try increasing the checkpoint interval " + + "if this error happens often."); } } // Start checkpoint @@ -249,12 +246,12 @@ void beginCheckpoint() throws IOException { } @Override - void checkpoint() throws IOException { + void checkpoint() throws IOException { setLogWriteOrderID(WriteOrderOracle.next()); LOG.info("Updating checkpoint metadata: logWriteOrderID: " + getLogWriteOrderID() + ", queueSize: " + getSize() + ", queueHead: " - + getHead()); + + getHead()); elementsBuffer.put(INDEX_WRITE_ORDER_ID, getLogWriteOrderID()); try { writeCheckpointMetaData(); @@ -286,8 +283,8 @@ void checkpoint() throws IOException { */ private void startBackupThread() { Preconditions.checkNotNull(checkpointBackUpExecutor, - "Expected the checkpoint backup exector to be non-null, " + - "but it is null. Checkpoint will not be backed up."); + "Expected the checkpoint backup exector to be non-null, " + + "but it is null. Checkpoint will not be backed up."); LOG.info("Attempting to back up checkpoint."); checkpointBackUpExecutor.submit(new Runnable() { @@ -317,16 +314,14 @@ void close() { } catch (IOException e) { LOG.info("Error closing " + checkpointFile, e); } - if(checkpointBackUpExecutor != null && !checkpointBackUpExecutor - .isShutdown()) { + if (checkpointBackUpExecutor != null && !checkpointBackUpExecutor.isShutdown()) { checkpointBackUpExecutor.shutdown(); try { // Wait till the executor dies. - while (!checkpointBackUpExecutor.awaitTermination(1, - TimeUnit.SECONDS)); + while (!checkpointBackUpExecutor.awaitTermination(1, TimeUnit.SECONDS)) {} } catch (InterruptedException ex) { LOG.warn("Interrupted while waiting for checkpoint backup to " + - "complete"); + "complete"); } } } @@ -362,18 +357,19 @@ boolean syncRequired() { @Override protected void incrementFileID(int fileID) { AtomicInteger counter = logFileIDReferenceCounts.get(fileID); - if(counter == null) { + if (counter == null) { counter = new AtomicInteger(0); logFileIDReferenceCounts.put(fileID, counter); } counter.incrementAndGet(); } + @Override protected void decrementFileID(int fileID) { AtomicInteger counter = logFileIDReferenceCounts.get(fileID); Preconditions.checkState(counter != null, "null counter "); int count = counter.decrementAndGet(); - if(count == 0) { + if (count == 0) { logFileIDReferenceCounts.remove(fileID); } } @@ -391,7 +387,7 @@ protected static void allocate(File file, long totalBytes) throws IOException { * totalBytes <= MAX_ALLOC_BUFFER_SIZE, so this can be cast to int * without a problem. */ - checkpointFile.write(new byte[(int)totalBytes]); + checkpointFile.write(new byte[(int) totalBytes]); } else { byte[] initBuffer = new byte[MAX_ALLOC_BUFFER_SIZE]; long remainingBytes = totalBytes; @@ -404,7 +400,7 @@ protected static void allocate(File file, long totalBytes) throws IOException { * so casting to int is fine. */ if (remainingBytes > 0) { - checkpointFile.write(initBuffer, 0, (int)remainingBytes); + checkpointFile.write(initBuffer, 0, (int) remainingBytes); } } success = true; @@ -412,7 +408,7 @@ protected static void allocate(File file, long totalBytes) throws IOException { try { checkpointFile.close(); } catch (IOException e) { - if(success) { + if (success) { throw e; } } @@ -436,9 +432,9 @@ public static void main(String[] args) throws Exception { } int capacity = (int) ((file.length() - (HEADER_SIZE * 8L)) / 8L); EventQueueBackingStoreFile backingStore = (EventQueueBackingStoreFile) - EventQueueBackingStoreFactory.get(file,capacity, "debug", false); + EventQueueBackingStoreFactory.get(file, capacity, "debug", false); System.out.println("File Reference Counts" - + backingStore.logFileIDReferenceCounts); + + backingStore.logFileIDReferenceCounts); System.out.println("Queue Capacity " + backingStore.getCapacity()); System.out.println("Queue Size " + backingStore.getSize()); System.out.println("Queue Head " + backingStore.getHead()); @@ -447,7 +443,7 @@ public static void main(String[] args) throws Exception { int fileID = (int) (value >>> 32); int offset = (int) value; System.out.println(index + ":" + Long.toHexString(value) + " fileID = " - + fileID + ", offset = " + offset); + + fileID + ", offset = " + offset); } FlumeEventQueue queue = new FlumeEventQueue(backingStore, inflightTakesFile, inflightPutsFile, @@ -462,7 +458,7 @@ public static void main(String[] args) throws Exception { int fileID = (int) (value >>> 32); int offset = (int) value; System.out.println(Long.toHexString(value) + " fileID = " - + fileID + ", offset = " + offset); + + fileID + ", offset = " + offset); } } SetMultimap takeMap = queue.deserializeInflightTakes(); @@ -474,7 +470,7 @@ public static void main(String[] args) throws Exception { int fileID = (int) (value >>> 32); int offset = (int) value; System.out.println(Long.toHexString(value) + " fileID = " - + fileID + ", offset = " + offset); + + fileID + ", offset = " + offset); } } } diff --git a/flume-ng-channels/flume-file-channel/src/main/java/org/apache/flume/channel/file/EventQueueBackingStoreFileV2.java b/flume-ng-channels/flume-file-channel/src/main/java/org/apache/flume/channel/file/EventQueueBackingStoreFileV2.java index abd2ea3701..71183aa93c 100644 --- a/flume-ng-channels/flume-file-channel/src/main/java/org/apache/flume/channel/file/EventQueueBackingStoreFileV2.java +++ b/flume-ng-channels/flume-file-channel/src/main/java/org/apache/flume/channel/file/EventQueueBackingStoreFileV2.java @@ -28,7 +28,6 @@ final class EventQueueBackingStoreFileV2 extends EventQueueBackingStoreFile { - private static final int INDEX_SIZE = 2; private static final int INDEX_HEAD = 3; private static final int INDEX_ACTIVE_LOG = 5; @@ -55,6 +54,7 @@ final class EventQueueBackingStoreFileV2 extends EventQueueBackingStoreFile { } } } + @Override protected int getVersion() { return Serialization.VERSION_2; diff --git a/flume-ng-channels/flume-file-channel/src/main/java/org/apache/flume/channel/file/EventQueueBackingStoreFileV3.java b/flume-ng-channels/flume-file-channel/src/main/java/org/apache/flume/channel/file/EventQueueBackingStoreFileV3.java index 9dfa0d13a1..f1a892a834 100644 --- a/flume-ng-channels/flume-file-channel/src/main/java/org/apache/flume/channel/file/EventQueueBackingStoreFileV3.java +++ b/flume-ng-channels/flume-file-channel/src/main/java/org/apache/flume/channel/file/EventQueueBackingStoreFileV3.java @@ -18,6 +18,12 @@ */ package org.apache.flume.channel.file; +import com.google.common.base.Preconditions; +import com.google.protobuf.InvalidProtocolBufferException; +import org.apache.flume.channel.file.proto.ProtosFactory; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + import java.io.File; import java.io.FileInputStream; import java.io.FileOutputStream; @@ -26,50 +32,42 @@ import java.util.Map; import java.util.concurrent.atomic.AtomicInteger; -import org.apache.flume.channel.file.proto.ProtosFactory; -import org.slf4j.Logger; -import org.slf4j.LoggerFactory; - -import com.google.common.base.Preconditions; -import com.google.protobuf.InvalidProtocolBufferException; - final class EventQueueBackingStoreFileV3 extends EventQueueBackingStoreFile { - private static final Logger LOG = LoggerFactory - .getLogger(EventQueueBackingStoreFileV3.class); + private static final Logger LOG = LoggerFactory.getLogger(EventQueueBackingStoreFileV3.class); private final File metaDataFile; EventQueueBackingStoreFileV3(File checkpointFile, int capacity, - String name) throws IOException, BadCheckpointException { + String name) throws IOException, BadCheckpointException { this(checkpointFile, capacity, name, null, false, false); } EventQueueBackingStoreFileV3(File checkpointFile, int capacity, - String name, File checkpointBackupDir, - boolean backupCheckpoint, boolean compressBackup) + String name, File checkpointBackupDir, + boolean backupCheckpoint, boolean compressBackup) throws IOException, BadCheckpointException { super(capacity, name, checkpointFile, checkpointBackupDir, backupCheckpoint, - compressBackup); + compressBackup); Preconditions.checkArgument(capacity > 0, "capacity must be greater than 0 " + capacity); metaDataFile = Serialization.getMetaDataFile(checkpointFile); LOG.info("Starting up with " + checkpointFile + " and " + metaDataFile); - if(metaDataFile.exists()) { + if (metaDataFile.exists()) { FileInputStream inputStream = new FileInputStream(metaDataFile); try { LOG.info("Reading checkpoint metadata from " + metaDataFile); ProtosFactory.Checkpoint checkpoint = ProtosFactory.Checkpoint.parseDelimitedFrom(inputStream); - if(checkpoint == null) { + if (checkpoint == null) { throw new BadCheckpointException("The checkpoint metadata file does " - + "not exist or has zero length"); + + "not exist or has zero length"); } int version = checkpoint.getVersion(); - if(version != getVersion()) { + if (version != getVersion()) { throw new BadCheckpointException("Invalid version: " + version + - " " + name + ", expected " + getVersion()); + " " + name + ", expected " + getVersion()); } long logWriteOrderID = checkpoint.getWriteOrderID(); - if(logWriteOrderID != getCheckpointLogWriteOrderID()) { + if (logWriteOrderID != getCheckpointLogWriteOrderID()) { String msg = "Checkpoint and Meta files have differing " + "logWriteOrderIDs " + getCheckpointLogWriteOrderID() + ", and " + logWriteOrderID; @@ -80,15 +78,15 @@ final class EventQueueBackingStoreFileV3 extends EventQueueBackingStoreFile { setLogWriteOrderID(logWriteOrderID); setSize(checkpoint.getQueueSize()); setHead(checkpoint.getQueueHead()); - for(ProtosFactory.ActiveLog activeLog : checkpoint.getActiveLogsList()) { + for (ProtosFactory.ActiveLog activeLog : checkpoint.getActiveLogsList()) { Integer logFileID = activeLog.getLogFileID(); Integer count = activeLog.getCount(); logFileIDReferenceCounts.put(logFileID, new AtomicInteger(count)); } } catch (InvalidProtocolBufferException ex) { throw new BadCheckpointException("Checkpoint metadata file is invalid. " - + "The agent might have been stopped while it was being " - + "written", ex); + + "The agent might have been stopped while it was being " + + "written", ex); } finally { try { inputStream.close(); @@ -97,7 +95,7 @@ final class EventQueueBackingStoreFileV3 extends EventQueueBackingStoreFile { } } } else { - if(backupExists(checkpointBackupDir) && shouldBackup) { + if (backupExists(checkpointBackupDir) && shouldBackup) { // If a backup exists, then throw an exception to recover checkpoint throw new BadCheckpointException("The checkpoint metadata file does " + "not exist, but a backup exists"); @@ -121,6 +119,7 @@ final class EventQueueBackingStoreFileV3 extends EventQueueBackingStoreFile { } } } + File getMetaDataFile() { return metaDataFile; } @@ -129,6 +128,7 @@ File getMetaDataFile() { protected int getVersion() { return Serialization.VERSION_3; } + @Override protected void writeCheckpointMetaData() throws IOException { ProtosFactory.Checkpoint.Builder checkpointBuilder = @@ -137,14 +137,14 @@ protected void writeCheckpointMetaData() throws IOException { checkpointBuilder.setQueueHead(getHead()); checkpointBuilder.setQueueSize(getSize()); checkpointBuilder.setWriteOrderID(getLogWriteOrderID()); - for(Integer logFileID : logFileIDReferenceCounts.keySet()) { + for (Integer logFileID : logFileIDReferenceCounts.keySet()) { int count = logFileIDReferenceCounts.get(logFileID).get(); - if(count != 0) { - ProtosFactory.ActiveLog.Builder activeLogBuilder = - ProtosFactory.ActiveLog.newBuilder(); - activeLogBuilder.setLogFileID(logFileID); - activeLogBuilder.setCount(count); - checkpointBuilder.addActiveLogs(activeLogBuilder.build()); + if (count != 0) { + ProtosFactory.ActiveLog.Builder activeLogBuilder = + ProtosFactory.ActiveLog.newBuilder(); + activeLogBuilder.setLogFileID(logFileID); + activeLogBuilder.setCount(count); + checkpointBuilder.addActiveLogs(activeLogBuilder.build()); } } FileOutputStream outputStream = new FileOutputStream(metaDataFile); @@ -161,8 +161,8 @@ protected void writeCheckpointMetaData() throws IOException { } static void upgrade(EventQueueBackingStoreFileV2 backingStoreV2, - File checkpointFile, File metaDataFile) - throws IOException { + File checkpointFile, File metaDataFile) + throws IOException { int head = backingStoreV2.getHead(); int size = backingStoreV2.getSize(); @@ -176,14 +176,14 @@ static void upgrade(EventQueueBackingStoreFileV2 backingStoreV2, checkpointBuilder.setQueueHead(head); checkpointBuilder.setQueueSize(size); checkpointBuilder.setWriteOrderID(writeOrderID); - for(Integer logFileID : referenceCounts.keySet()) { + for (Integer logFileID : referenceCounts.keySet()) { int count = referenceCounts.get(logFileID).get(); - if(count > 0) { - ProtosFactory.ActiveLog.Builder activeLogBuilder = - ProtosFactory.ActiveLog.newBuilder(); - activeLogBuilder.setLogFileID(logFileID); - activeLogBuilder.setCount(count); - checkpointBuilder.addActiveLogs(activeLogBuilder.build()); + if (count > 0) { + ProtosFactory.ActiveLog.Builder activeLogBuilder = + ProtosFactory.ActiveLog.newBuilder(); + activeLogBuilder.setLogFileID(logFileID); + activeLogBuilder.setCount(count); + checkpointBuilder.addActiveLogs(activeLogBuilder.build()); } } FileOutputStream outputStream = new FileOutputStream(metaDataFile); diff --git a/flume-ng-channels/flume-file-channel/src/main/java/org/apache/flume/channel/file/EventUtils.java b/flume-ng-channels/flume-file-channel/src/main/java/org/apache/flume/channel/file/EventUtils.java index ff5242a1fe..4c0c96c77f 100644 --- a/flume-ng-channels/flume-file-channel/src/main/java/org/apache/flume/channel/file/EventUtils.java +++ b/flume-ng-channels/flume-file-channel/src/main/java/org/apache/flume/channel/file/EventUtils.java @@ -33,7 +33,7 @@ public class EventUtils { * @return Event if Put instance is present, null otherwise */ public static Event getEventFromTransactionEvent(TransactionEventRecord transactionEventRecord) { - if(transactionEventRecord instanceof Put) { + if (transactionEventRecord instanceof Put) { return ((Put)transactionEventRecord).getEvent(); } return null; diff --git a/flume-ng-channels/flume-file-channel/src/main/java/org/apache/flume/channel/file/FileChannel.java b/flume-ng-channels/flume-file-channel/src/main/java/org/apache/flume/channel/file/FileChannel.java index ed2b996f88..9d82e435b4 100644 --- a/flume-ng-channels/flume-file-channel/src/main/java/org/apache/flume/channel/file/FileChannel.java +++ b/flume-ng-channels/flume-file-channel/src/main/java/org/apache/flume/channel/file/FileChannel.java @@ -25,7 +25,11 @@ import com.google.common.base.Strings; import com.google.common.base.Throwables; import com.google.common.collect.Iterables; -import org.apache.flume.*; +import org.apache.flume.Channel; +import org.apache.flume.ChannelException; +import org.apache.flume.ChannelFullException; +import org.apache.flume.Context; +import org.apache.flume.Event; import org.apache.flume.annotations.Disposable; import org.apache.flume.annotations.InterfaceAudience; import org.apache.flume.annotations.InterfaceStability; @@ -71,8 +75,7 @@ @Disposable public class FileChannel extends BasicChannelSemantics { - private static final Logger LOG = LoggerFactory - .getLogger(FileChannel.class); + private static final Logger LOG = LoggerFactory.getLogger(FileChannel.class); private Integer capacity = 0; private int keepAlive; @@ -125,8 +128,8 @@ public void configure(Context context) { context.getString(FileChannelConfiguration.CHECKPOINT_DIR, homePath + "/.flume/file-channel/checkpoint").trim(); - String strBackupCheckpointDir = context.getString - (FileChannelConfiguration.BACKUP_CHECKPOINT_DIR, "").trim(); + String strBackupCheckpointDir = + context.getString(FileChannelConfiguration.BACKUP_CHECKPOINT_DIR, "").trim(); String[] strDataDirs = Iterables.toArray( Splitter.on(",").trimResults().omitEmptyStrings().split( @@ -137,9 +140,9 @@ public void configure(Context context) { if (useDualCheckpoints) { Preconditions.checkState(!strBackupCheckpointDir.isEmpty(), - "Dual checkpointing is enabled, but the backup directory is not set. " + - "Please set " + FileChannelConfiguration.BACKUP_CHECKPOINT_DIR + " " + - "to enable dual checkpointing"); + "Dual checkpointing is enabled, but the backup directory is not set. " + + "Please set " + FileChannelConfiguration.BACKUP_CHECKPOINT_DIR + " " + + "to enable dual checkpointing"); backupCheckpointDir = new File(strBackupCheckpointDir); /* * If the backup directory is the same as the checkpoint directory, @@ -147,9 +150,9 @@ public void configure(Context context) { * channel. */ Preconditions.checkState(!backupCheckpointDir.equals(checkpointDir), - "Could not configure " + getName() + ". The checkpoint backup " + - "directory and the checkpoint directory are " + - "configured to be the same."); + "Could not configure " + getName() + ". The checkpoint backup " + + "directory and the checkpoint directory are " + + "configured to be the same."); } dataDirs = new File[strDataDirs.length]; @@ -159,10 +162,10 @@ public void configure(Context context) { capacity = context.getInteger(FileChannelConfiguration.CAPACITY, FileChannelConfiguration.DEFAULT_CAPACITY); - if(capacity <= 0) { + if (capacity <= 0) { capacity = FileChannelConfiguration.DEFAULT_CAPACITY; LOG.warn("Invalid capacity specified, initializing channel to " - + "default capacity of {}", capacity); + + "default capacity of {}", capacity); } keepAlive = @@ -172,48 +175,48 @@ public void configure(Context context) { context.getInteger(FileChannelConfiguration.TRANSACTION_CAPACITY, FileChannelConfiguration.DEFAULT_TRANSACTION_CAPACITY); - if(transactionCapacity <= 0) { + if (transactionCapacity <= 0) { transactionCapacity = - FileChannelConfiguration.DEFAULT_TRANSACTION_CAPACITY; + FileChannelConfiguration.DEFAULT_TRANSACTION_CAPACITY; LOG.warn("Invalid transaction capacity specified, " + "initializing channel to default " + "capacity of {}", transactionCapacity); } Preconditions.checkState(transactionCapacity <= capacity, - "File Channel transaction capacity cannot be greater than the " + - "capacity of the channel."); + "File Channel transaction capacity cannot be greater than the " + + "capacity of the channel."); checkpointInterval = - context.getLong(FileChannelConfiguration.CHECKPOINT_INTERVAL, + context.getLong(FileChannelConfiguration.CHECKPOINT_INTERVAL, FileChannelConfiguration.DEFAULT_CHECKPOINT_INTERVAL); if (checkpointInterval <= 0) { LOG.warn("Checkpoint interval is invalid: " + checkpointInterval - + ", using default: " - + FileChannelConfiguration.DEFAULT_CHECKPOINT_INTERVAL); + + ", using default: " + + FileChannelConfiguration.DEFAULT_CHECKPOINT_INTERVAL); checkpointInterval = - FileChannelConfiguration.DEFAULT_CHECKPOINT_INTERVAL; + FileChannelConfiguration.DEFAULT_CHECKPOINT_INTERVAL; } // cannot be over FileChannelConfiguration.DEFAULT_MAX_FILE_SIZE maxFileSize = Math.min( - context.getLong(FileChannelConfiguration.MAX_FILE_SIZE, - FileChannelConfiguration.DEFAULT_MAX_FILE_SIZE), - FileChannelConfiguration.DEFAULT_MAX_FILE_SIZE); + context.getLong(FileChannelConfiguration.MAX_FILE_SIZE, + FileChannelConfiguration.DEFAULT_MAX_FILE_SIZE), + FileChannelConfiguration.DEFAULT_MAX_FILE_SIZE); minimumRequiredSpace = Math.max( - context.getLong(FileChannelConfiguration.MINIMUM_REQUIRED_SPACE, - FileChannelConfiguration.DEFAULT_MINIMUM_REQUIRED_SPACE), - FileChannelConfiguration.FLOOR_MINIMUM_REQUIRED_SPACE); + context.getLong(FileChannelConfiguration.MINIMUM_REQUIRED_SPACE, + FileChannelConfiguration.DEFAULT_MINIMUM_REQUIRED_SPACE), + FileChannelConfiguration.FLOOR_MINIMUM_REQUIRED_SPACE); useLogReplayV1 = context.getBoolean( FileChannelConfiguration.USE_LOG_REPLAY_V1, - FileChannelConfiguration.DEFAULT_USE_LOG_REPLAY_V1); + FileChannelConfiguration.DEFAULT_USE_LOG_REPLAY_V1); useFastReplay = context.getBoolean( - FileChannelConfiguration.USE_FAST_REPLAY, - FileChannelConfiguration.DEFAULT_USE_FAST_REPLAY); + FileChannelConfiguration.USE_FAST_REPLAY, + FileChannelConfiguration.DEFAULT_USE_FAST_REPLAY); Context encryptionContext = new Context( context.getSubProperties(EncryptionConfiguration.ENCRYPTION_PREFIX + @@ -224,41 +227,41 @@ public void configure(Context context) { EncryptionConfiguration.ACTIVE_KEY); encryptionCipherProvider = encryptionContext.getString( EncryptionConfiguration.CIPHER_PROVIDER); - if(encryptionKeyProviderName != null) { + if (encryptionKeyProviderName != null) { Preconditions.checkState(!Strings.isNullOrEmpty(encryptionActiveKey), "Encryption configuration problem: " + EncryptionConfiguration.ACTIVE_KEY + " is missing"); Preconditions.checkState(!Strings.isNullOrEmpty(encryptionCipherProvider), "Encryption configuration problem: " + EncryptionConfiguration.CIPHER_PROVIDER + " is missing"); - Context keyProviderContext = new Context(encryptionContext. - getSubProperties(EncryptionConfiguration.KEY_PROVIDER + ".")); - encryptionKeyProvider = KeyProviderFactory. - getInstance(encryptionKeyProviderName, keyProviderContext); + Context keyProviderContext = new Context( + encryptionContext.getSubProperties(EncryptionConfiguration.KEY_PROVIDER + ".")); + encryptionKeyProvider = KeyProviderFactory.getInstance( + encryptionKeyProviderName, keyProviderContext); } else { Preconditions.checkState(encryptionActiveKey == null, "Encryption configuration problem: " + EncryptionConfiguration.ACTIVE_KEY + " is present while key " + - "provider name is not."); + "provider name is not."); Preconditions.checkState(encryptionCipherProvider == null, "Encryption configuration problem: " + EncryptionConfiguration.CIPHER_PROVIDER + " is present while " + - "key provider name is not."); + "key provider name is not."); } fsyncPerTransaction = context.getBoolean(FileChannelConfiguration - .FSYNC_PER_TXN, FileChannelConfiguration.DEFAULT_FSYNC_PRE_TXN); + .FSYNC_PER_TXN, FileChannelConfiguration.DEFAULT_FSYNC_PRE_TXN); fsyncInterval = context.getInteger(FileChannelConfiguration - .FSYNC_INTERVAL, FileChannelConfiguration.DEFAULT_FSYNC_INTERVAL); + .FSYNC_INTERVAL, FileChannelConfiguration.DEFAULT_FSYNC_INTERVAL); checkpointOnClose = context.getBoolean(FileChannelConfiguration - .CHKPT_ONCLOSE, FileChannelConfiguration.DEFAULT_CHKPT_ONCLOSE); + .CHKPT_ONCLOSE, FileChannelConfiguration.DEFAULT_CHKPT_ONCLOSE); - if(queueRemaining == null) { + if (queueRemaining == null) { queueRemaining = new Semaphore(capacity, true); } - if(log != null) { + if (log != null) { log.setCheckpointInterval(checkpointInterval); log.setMaxFileSize(maxFileSize); } @@ -299,7 +302,7 @@ public synchronized void start() { Preconditions.checkState(queueRemaining.tryAcquire(depth), "Unable to acquire " + depth + " permits " + channelNameDescriptor); LOG.info("Queue Size after replay: " + depth + " " - + channelNameDescriptor); + + channelNameDescriptor); } catch (Throwable t) { open = false; startupError = t; @@ -337,9 +340,9 @@ public String toString() { @Override protected BasicTransactionSemantics createTransaction() { - if(!open) { + if (!open) { String msg = "Channel closed " + channelNameDescriptor; - if(startupError != null) { + if (startupError != null) { msg += ". Due to " + startupError.getClass().getName() + ": " + startupError.getMessage(); throw new IllegalStateException(msg, startupError); @@ -348,27 +351,28 @@ protected BasicTransactionSemantics createTransaction() { } FileBackedTransaction trans = transactions.get(); - if(trans != null && !trans.isClosed()) { + if (trans != null && !trans.isClosed()) { Preconditions.checkState(false, "Thread has transaction which is still open: " + - trans.getStateAsString() + channelNameDescriptor); + trans.getStateAsString() + channelNameDescriptor); } trans = new FileBackedTransaction(log, TransactionIDOracle.next(), - transactionCapacity, keepAlive, queueRemaining, getName(), - fsyncPerTransaction, channelCounter); + transactionCapacity, keepAlive, queueRemaining, getName(), + fsyncPerTransaction, channelCounter); transactions.set(trans); return trans; } protected int getDepth() { - Preconditions.checkState(open, "Channel closed" + channelNameDescriptor); + Preconditions.checkState(open, "Channel closed" + channelNameDescriptor); Preconditions.checkNotNull(log, "log"); FlumeEventQueue queue = log.getFlumeEventQueue(); Preconditions.checkNotNull(queue, "queue"); return queue.getSize(); } + void close() { - if(open) { + if (open) { open = false; try { log.close(); @@ -398,11 +402,12 @@ public boolean isOpen() { /** * Did this channel recover a backup of the checkpoint to restart? + * * @return true if the channel recovered using a backup. */ @VisibleForTesting boolean checkpointBackupRestored() { - if(log != null) { + if (log != null) { return log.backupRestored(); } return false; @@ -428,10 +433,11 @@ static class FileBackedTransaction extends BasicTransactionSemantics { private final String channelNameDescriptor; private final ChannelCounter channelCounter; private final boolean fsyncPerTransaction; + public FileBackedTransaction(Log log, long transactionID, - int transCapacity, int keepAlive, Semaphore queueRemaining, - String name, boolean fsyncPerTransaction, ChannelCounter - counter) { + int transCapacity, int keepAlive, Semaphore queueRemaining, + String name, boolean fsyncPerTransaction, ChannelCounter + counter) { this.log = log; queue = log.getFlumeEventQueue(); this.transactionID = transactionID; @@ -443,9 +449,11 @@ public FileBackedTransaction(Log log, long transactionID, channelNameDescriptor = "[channel=" + name + "]"; this.channelCounter = counter; } + private boolean isClosed() { return State.CLOSED.equals(getState()); } + private String getStateAsString() { return String.valueOf(getState()); } @@ -453,7 +461,7 @@ private String getStateAsString() { @Override protected void doPut(Event event) throws InterruptedException { channelCounter.incrementEventPutAttemptCount(); - if(putList.remainingCapacity() == 0) { + if (putList.remainingCapacity() == 0) { throw new ChannelException("Put queue for FileBackedTransaction " + "of capacity " + putList.size() + " full, consider " + "committing more frequently, increasing capacity or " + @@ -461,7 +469,7 @@ protected void doPut(Event event) throws InterruptedException { } // this does not need to be in the critical section as it does not // modify the structure of the log or queue. - if(!queueRemaining.tryAcquire(keepAlive, TimeUnit.SECONDS)) { + if (!queueRemaining.tryAcquire(keepAlive, TimeUnit.SECONDS)) { throw new ChannelFullException("The channel has reached it's capacity. " + "This might be the result of a sink on the channel having too " + "low of batch size, a downstream system running slower than " @@ -473,15 +481,15 @@ protected void doPut(Event event) throws InterruptedException { try { FlumeEventPointer ptr = log.put(transactionID, event); Preconditions.checkState(putList.offer(ptr), "putList offer failed " - + channelNameDescriptor); + + channelNameDescriptor); queue.addWithoutCommit(ptr, transactionID); success = true; } catch (IOException e) { throw new ChannelException("Put failed due to IO error " - + channelNameDescriptor, e); + + channelNameDescriptor, e); } finally { log.unlockShared(); - if(!success) { + if (!success) { // release slot obtained in the case // the put fails for any reason queueRemaining.release(); @@ -492,11 +500,11 @@ protected void doPut(Event event) throws InterruptedException { @Override protected Event doTake() throws InterruptedException { channelCounter.incrementEventTakeAttemptCount(); - if(takeList.remainingCapacity() == 0) { + if (takeList.remainingCapacity() == 0) { throw new ChannelException("Take list for FileBackedTransaction, capacity " + takeList.size() + " full, consider committing more frequently, " + "increasing capacity, or increasing thread count. " - + channelNameDescriptor); + + channelNameDescriptor); } log.lockShared(); /* @@ -517,24 +525,24 @@ protected Event doTake() throws InterruptedException { // first add to takeList so that if write to disk // fails rollback actually does it's work Preconditions.checkState(takeList.offer(ptr), - "takeList offer failed " - + channelNameDescriptor); + "takeList offer failed " + + channelNameDescriptor); log.take(transactionID, ptr); // write take to disk Event event = log.get(ptr); return event; } catch (IOException e) { throw new ChannelException("Take failed due to IO error " - + channelNameDescriptor, e); + + channelNameDescriptor, e); } catch (NoopRecordException e) { LOG.warn("Corrupt record replaced by File Channel Integrity " + - "tool found. Will retrieve next event", e); + "tool found. Will retrieve next event", e); takeList.remove(ptr); } catch (CorruptEventException ex) { if (fsyncPerTransaction) { throw new ChannelException(ex); } LOG.warn("Corrupt record found. Event will be " + - "skipped, and next event will be read.", ex); + "skipped, and next event will be read.", ex); takeList.remove(ptr); } } @@ -543,20 +551,21 @@ protected Event doTake() throws InterruptedException { log.unlockShared(); } } + @Override protected void doCommit() throws InterruptedException { int puts = putList.size(); int takes = takeList.size(); - if(puts > 0) { + if (puts > 0) { Preconditions.checkState(takes == 0, "nonzero puts and takes " - + channelNameDescriptor); + + channelNameDescriptor); log.lockShared(); try { log.commitPut(transactionID); channelCounter.addToEventPutSuccessCount(puts); synchronized (queue) { - while(!putList.isEmpty()) { - if(!queue.addTail(putList.removeFirst())) { + while (!putList.isEmpty()) { + if (!queue.addTail(putList.removeFirst())) { StringBuilder msg = new StringBuilder(); msg.append("Queue add failed, this shouldn't be able to "); msg.append("happen. A portion of the transaction has been "); @@ -572,7 +581,7 @@ protected void doCommit() throws InterruptedException { } } catch (IOException e) { throw new ChannelException("Commit failed due to IO error " - + channelNameDescriptor, e); + + channelNameDescriptor, e); } finally { log.unlockShared(); } @@ -595,13 +604,14 @@ protected void doCommit() throws InterruptedException { takeList.clear(); channelCounter.setChannelSize(queue.getSize()); } + @Override protected void doRollback() throws InterruptedException { int puts = putList.size(); int takes = takeList.size(); log.lockShared(); try { - if(takes > 0) { + if (takes > 0) { Preconditions.checkState(puts == 0, "nonzero puts and takes " + channelNameDescriptor); synchronized (queue) { diff --git a/flume-ng-channels/flume-file-channel/src/main/java/org/apache/flume/channel/file/FileChannelConfiguration.java b/flume-ng-channels/flume-file-channel/src/main/java/org/apache/flume/channel/file/FileChannelConfiguration.java index 5c3c48ff7d..c5678d4ee6 100644 --- a/flume-ng-channels/flume-file-channel/src/main/java/org/apache/flume/channel/file/FileChannelConfiguration.java +++ b/flume-ng-channels/flume-file-channel/src/main/java/org/apache/flume/channel/file/FileChannelConfiguration.java @@ -87,10 +87,8 @@ public class FileChannelConfiguration { public static final String USE_DUAL_CHECKPOINTS = "useDualCheckpoints"; public static final boolean DEFAULT_USE_DUAL_CHECKPOINTS = false; - public static final String COMPRESS_BACKUP_CHECKPOINT = - "compressBackupCheckpoint"; - public static final boolean DEFAULT_COMPRESS_BACKUP_CHECKPOINT - = false; + public static final String COMPRESS_BACKUP_CHECKPOINT = "compressBackupCheckpoint"; + public static final boolean DEFAULT_COMPRESS_BACKUP_CHECKPOINT = false; public static final String FSYNC_PER_TXN = "fsyncPerTransaction"; public static final boolean DEFAULT_FSYNC_PRE_TXN = true; diff --git a/flume-ng-channels/flume-file-channel/src/main/java/org/apache/flume/channel/file/FlumeEvent.java b/flume-ng-channels/flume-file-channel/src/main/java/org/apache/flume/channel/file/FlumeEvent.java index 53c1251574..cd1b6d8d31 100644 --- a/flume-ng-channels/flume-file-channel/src/main/java/org/apache/flume/channel/file/FlumeEvent.java +++ b/flume-ng-channels/flume-file-channel/src/main/java/org/apache/flume/channel/file/FlumeEvent.java @@ -37,15 +37,15 @@ */ class FlumeEvent implements Event, Writable { - private static final byte EVENT_MAP_TEXT_WRITABLE_ID = Byte.valueOf(Integer.valueOf(-116).byteValue()); + private static final byte EVENT_MAP_TEXT_WRITABLE_ID = + Byte.valueOf(Integer.valueOf(-116).byteValue()); - private static ThreadLocal ENCODER_FACTORY = - new ThreadLocal() { + private static ThreadLocal ENCODER_FACTORY = new ThreadLocal() { @Override protected CharsetEncoder initialValue() { - return Charset.forName("UTF-8").newEncoder(). - onMalformedInput(CodingErrorAction.REPLACE). - onUnmappableCharacter(CodingErrorAction.REPLACE); + return Charset.forName("UTF-8").newEncoder() + .onMalformedInput(CodingErrorAction.REPLACE) + .onUnmappableCharacter(CodingErrorAction.REPLACE); } }; @@ -53,9 +53,9 @@ protected CharsetEncoder initialValue() { new ThreadLocal() { @Override protected CharsetDecoder initialValue() { - return Charset.forName("UTF-8").newDecoder(). - onMalformedInput(CodingErrorAction.REPLACE). - onUnmappableCharacter(CodingErrorAction.REPLACE); + return Charset.forName("UTF-8").newDecoder() + .onMalformedInput(CodingErrorAction.REPLACE) + .onUnmappableCharacter(CodingErrorAction.REPLACE); } }; @@ -65,6 +65,7 @@ protected CharsetDecoder initialValue() { private FlumeEvent() { this(null, null); } + FlumeEvent(Map headers, byte[] body) { this.headers = headers; this.body = body; @@ -116,13 +117,12 @@ public void write(DataOutput out) throws IOException { WritableUtils.writeVInt(out, valueLength ); out.write(valueBytes.array(), 0, valueLength); } - } - else { + } else { out.writeInt( 0 ); } byte[] body = getBody(); - if(body == null) { + if (body == null) { out.writeInt(-1); } else { out.writeInt(body.length); @@ -174,7 +174,7 @@ public void readFields(DataInput in) throws IOException { byte[] body = null; int bodyLength = in.readInt(); - if(bodyLength != -1) { + if (bodyLength != -1) { body = new byte[bodyLength]; in.readFully(body); } diff --git a/flume-ng-channels/flume-file-channel/src/main/java/org/apache/flume/channel/file/FlumeEventPointer.java b/flume-ng-channels/flume-file-channel/src/main/java/org/apache/flume/channel/file/FlumeEventPointer.java index 5f06ab72a0..ebf784380c 100644 --- a/flume-ng-channels/flume-file-channel/src/main/java/org/apache/flume/channel/file/FlumeEventPointer.java +++ b/flume-ng-channels/flume-file-channel/src/main/java/org/apache/flume/channel/file/FlumeEventPointer.java @@ -27,6 +27,7 @@ class FlumeEventPointer { private final int fileID; private final int offset; + FlumeEventPointer(int fileID, int offset) { this.fileID = fileID; this.offset = offset; @@ -34,24 +35,28 @@ class FlumeEventPointer { * Log files used to have a header, now metadata is in * a separate file so data starts at offset 0. */ - if(offset < 0) { + if (offset < 0) { throw new IllegalArgumentException("offset = " + offset + "(" + Integer.toHexString(offset) + ")" + ", fileID = " + fileID + "(" + Integer.toHexString(fileID) + ")"); } } + int getFileID() { return fileID; } + int getOffset() { return offset; } + public long toLong() { long result = fileID; result = (long)fileID << 32; result += (long)offset; return result; } + @Override public int hashCode() { final int prime = 31; @@ -60,6 +65,7 @@ public int hashCode() { result = prime * result + offset; return result; } + @Override public boolean equals(Object obj) { if (this == obj) { @@ -80,10 +86,12 @@ public boolean equals(Object obj) { } return true; } + @Override public String toString() { return "FlumeEventPointer [fileID=" + fileID + ", offset=" + offset + "]"; } + public static FlumeEventPointer fromLong(long value) { int fileID = (int)(value >>> 32); int offset = (int)value; diff --git a/flume-ng-channels/flume-file-channel/src/main/java/org/apache/flume/channel/file/FlumeEventQueue.java b/flume-ng-channels/flume-file-channel/src/main/java/org/apache/flume/channel/file/FlumeEventQueue.java index d305f4d2a7..4311c7fb73 100644 --- a/flume-ng-channels/flume-file-channel/src/main/java/org/apache/flume/channel/file/FlumeEventQueue.java +++ b/flume-ng-channels/flume-file-channel/src/main/java/org/apache/flume/channel/file/FlumeEventQueue.java @@ -18,6 +18,17 @@ */ package org.apache.flume.channel.file; +import com.google.common.annotations.VisibleForTesting; +import com.google.common.base.Preconditions; +import com.google.common.collect.HashMultimap; +import com.google.common.collect.SetMultimap; +import org.apache.commons.io.FileUtils; +import org.apache.commons.lang.ArrayUtils; +import org.mapdb.DB; +import org.mapdb.DBMaker; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + import java.io.File; import java.io.IOException; import java.io.RandomAccessFile; @@ -31,18 +42,6 @@ import java.util.SortedSet; import java.util.TreeSet; -import org.apache.commons.io.FileUtils; -import org.apache.commons.lang.ArrayUtils; -import org.mapdb.DB; -import org.mapdb.DBMaker; -import org.slf4j.Logger; -import org.slf4j.LoggerFactory; - -import com.google.common.annotations.VisibleForTesting; -import com.google.common.base.Preconditions; -import com.google.common.collect.HashMultimap; -import com.google.common.collect.SetMultimap; - /** * Queue of events in the channel. This queue stores only * {@link FlumeEventPointer} objects which are represented @@ -54,7 +53,7 @@ */ final class FlumeEventQueue { private static final Logger LOG = LoggerFactory - .getLogger(FlumeEventQueue.class); + .getLogger(FlumeEventQueue.class); private static final int EMPTY = 0; private final EventQueueBackingStore backingStore; private final String channelNameDescriptor; @@ -72,7 +71,7 @@ final class FlumeEventQueue { * @throws IOException */ FlumeEventQueue(EventQueueBackingStore backingStore, File inflightTakesFile, - File inflightPutsFile, File queueSetDBDir) throws Exception { + File inflightPutsFile, File queueSetDBDir) throws Exception { Preconditions.checkArgument(backingStore.getCapacity() > 0, "Capacity must be greater than zero"); Preconditions.checkNotNull(backingStore, "backingStore"); @@ -88,13 +87,13 @@ final class FlumeEventQueue { LOG.error("Could not read checkpoint.", e); throw e; } - if(queueSetDBDir.isDirectory()) { + if (queueSetDBDir.isDirectory()) { FileUtils.deleteDirectory(queueSetDBDir); - } else if(queueSetDBDir.isFile() && !queueSetDBDir.delete()) { + } else if (queueSetDBDir.isFile() && !queueSetDBDir.delete()) { throw new IOException("QueueSetDir " + queueSetDBDir + " is a file and" + " could not be deleted"); } - if(!queueSetDBDir.mkdirs()) { + if (!queueSetDBDir.mkdirs()) { throw new IllegalStateException("Could not create QueueSet Dir " + queueSetDBDir); } @@ -108,7 +107,7 @@ final class FlumeEventQueue { .mmapFileEnableIfSupported() .make(); queueSet = - db.createHashSet("QueueSet " + " - " + backingStore.getName()).make(); + db.createHashSet("QueueSet " + " - " + backingStore.getName()).make(); long start = System.currentTimeMillis(); for (int i = 0; i < backingStore.getSize(); i++) { queueSet.add(get(i)); @@ -118,12 +117,12 @@ final class FlumeEventQueue { } SetMultimap deserializeInflightPuts() - throws IOException, BadCheckpointException{ + throws IOException, BadCheckpointException { return inflightPuts.deserialize(); } SetMultimap deserializeInflightTakes() - throws IOException, BadCheckpointException{ + throws IOException, BadCheckpointException { return inflightTakes.deserialize(); } @@ -133,9 +132,9 @@ synchronized long getLogWriteOrderID() { synchronized boolean checkpoint(boolean force) throws Exception { if (!backingStore.syncRequired() - && !inflightTakes.syncRequired() - && !force) { //No need to check inflight puts, since that would - //cause elements.syncRequired() to return true. + && !inflightTakes.syncRequired() + && !force) { //No need to check inflight puts, since that would + //cause elements.syncRequired() to return true. LOG.debug("Checkpoint not required"); return false; } @@ -152,13 +151,13 @@ synchronized boolean checkpoint(boolean force) throws Exception { * @return FlumeEventPointer or null if queue is empty */ synchronized FlumeEventPointer removeHead(long transactionID) { - if(backingStore.getSize() == 0) { + if (backingStore.getSize() == 0) { return null; } long value = remove(0, transactionID); Preconditions.checkState(value != EMPTY, "Empty value " - + channelNameDescriptor); + + channelNameDescriptor); FlumeEventPointer ptr = FlumeEventPointer.fromLong(value); backingStore.decrementFileID(ptr.getFileID()); @@ -168,6 +167,7 @@ synchronized FlumeEventPointer removeHead(long transactionID) { /** * Add a FlumeEventPointer to the head of the queue. * Called during rollbacks. + * * @param FlumeEventPointer to be added * @return true if space was available and pointer was * added to the queue @@ -180,7 +180,7 @@ synchronized boolean addHead(FlumeEventPointer e) { //there is a buuuuuuuug! if (backingStore.getSize() == backingStore.getCapacity()) { LOG.error("Could not reinsert to queue, events which were taken but " - + "not committed. Please report this issue."); + + "not committed. Please report this issue."); return false; } @@ -195,6 +195,7 @@ synchronized boolean addHead(FlumeEventPointer e) { /** * Add a FlumeEventPointer to the tail of the queue. + * * @param FlumeEventPointer to be added * @return true if space was available and pointer * was added to the queue @@ -215,6 +216,7 @@ synchronized boolean addTail(FlumeEventPointer e) { /** * Must be called when a put happens to the log. This ensures that put commits * after checkpoints will retrieve all events committed in that txn. + * * @param e * @param transactionID */ @@ -227,16 +229,19 @@ synchronized void addWithoutCommit(FlumeEventPointer e, long transactionID) { * only be used when recovering from a crash. It is not * legal to call this method after replayComplete has been * called. + * * @param FlumeEventPointer to be removed * @return true if the FlumeEventPointer was found * and removed */ + // remove() overloads should not be split, according to checkstyle. + // CHECKSTYLE:OFF synchronized boolean remove(FlumeEventPointer e) { long value = e.toLong(); Preconditions.checkArgument(value != EMPTY); if (queueSet == null) { - throw new IllegalStateException("QueueSet is null, thus replayComplete" - + " has been called which is illegal"); + throw new IllegalStateException("QueueSet is null, thus replayComplete" + + " has been called which is illegal"); } if (!queueSet.contains(value)) { return false; @@ -244,7 +249,7 @@ synchronized boolean remove(FlumeEventPointer e) { searchCount++; long start = System.currentTimeMillis(); for (int i = 0; i < backingStore.getSize(); i++) { - if(get(i) == value) { + if (get(i) == value) { remove(i, 0); FlumeEventPointer ptr = FlumeEventPointer.fromLong(value); backingStore.decrementFileID(ptr.getFileID()); @@ -255,6 +260,8 @@ synchronized boolean remove(FlumeEventPointer e) { searchTime += System.currentTimeMillis() - start; return false; } + // CHECKSTYLE:ON + /** * @return a copy of the set of fileIDs which are currently on the queue * will be normally be used when deciding which data files can @@ -299,19 +306,19 @@ protected boolean add(int index, long value) { backingStore.setSize(backingStore.getSize() + 1); - if (index <= backingStore.getSize()/2) { + if (index <= backingStore.getSize() / 2) { // Shift left backingStore.setHead(backingStore.getHead() - 1); if (backingStore.getHead() < 0) { backingStore.setHead(backingStore.getCapacity() - 1); } for (int i = 0; i < index; i++) { - set(i, get(i+1)); + set(i, get(i + 1)); } } else { // Sift right for (int i = backingStore.getSize() - 1; i > index; i--) { - set(i, get(i-1)); + set(i, get(i - 1)); } } set(index, value); @@ -323,6 +330,7 @@ protected boolean add(int index, long value) { /** * Must be called when a transaction is being committed or rolled back. + * * @param transactionID */ synchronized void completeTransaction(long transactionID) { @@ -334,7 +342,7 @@ synchronized void completeTransaction(long transactionID) { protected synchronized long remove(int index, long transactionID) { if (index < 0 || index > backingStore.getSize() - 1) { throw new IndexOutOfBoundsException("index = " + index - + ", queueSize " + backingStore.getSize() +" " + channelNameDescriptor); + + ", queueSize " + backingStore.getSize() + " " + channelNameDescriptor); } copyCount++; long start = System.currentTimeMillis(); @@ -343,13 +351,13 @@ protected synchronized long remove(int index, long transactionID) { queueSet.remove(value); } //if txn id = 0, we are recovering from a crash. - if(transactionID != 0) { + if (transactionID != 0) { inflightTakes.addEvent(transactionID, value); } - if (index > backingStore.getSize()/2) { + if (index > backingStore.getSize() / 2) { // Move tail part to left for (int i = index; i < backingStore.getSize() - 1; i++) { - long rightValue = get(i+1); + long rightValue = get(i + 1); set(i, rightValue); } set(backingStore.getSize() - 1, EMPTY); @@ -357,7 +365,7 @@ protected synchronized long remove(int index, long transactionID) { // Move head part to right for (int i = index - 1; i >= 0; i--) { long leftValue = get(i); - set(i+1, leftValue); + set(i + 1, leftValue); } set(0, EMPTY); backingStore.setHead(backingStore.getHead() + 1); @@ -386,7 +394,7 @@ synchronized void close() throws IOException { if (db != null) { db.close(); } - } catch(Exception ex) { + } catch (Exception ex) { LOG.warn("Error closing db", ex); } try { @@ -407,7 +415,7 @@ synchronized void replayComplete() { searchTime + ", Copy Count = " + copyCount + ", Copy Time = " + copyTime; LOG.info(msg); - if(db != null) { + if (db != null) { db.close(); } queueSet = null; @@ -440,11 +448,11 @@ class InflightEventWrapper { private volatile boolean syncRequired = false; private SetMultimap inflightFileIDs = HashMultimap.create(); - public InflightEventWrapper(File inflightEventsFile) throws Exception{ - if(!inflightEventsFile.exists()){ - Preconditions.checkState(inflightEventsFile.createNewFile(),"Could not" - + "create inflight events file: " - + inflightEventsFile.getCanonicalPath()); + public InflightEventWrapper(File inflightEventsFile) throws Exception { + if (!inflightEventsFile.exists()) { + Preconditions.checkState(inflightEventsFile.createNewFile(), "Could not" + + "create inflight events file: " + + inflightEventsFile.getCanonicalPath()); } this.inflightEventsFile = inflightEventsFile; file = new RandomAccessFile(inflightEventsFile, "rw"); @@ -454,10 +462,11 @@ public InflightEventWrapper(File inflightEventsFile) throws Exception{ /** * Complete the transaction, and remove all events from inflight list. + * * @param transactionID */ public boolean completeTransaction(Long transactionID) { - if(!inflightEvents.containsKey(transactionID)) { + if (!inflightEvents.containsKey(transactionID)) { return false; } inflightEvents.removeAll(transactionID); @@ -468,28 +477,30 @@ public boolean completeTransaction(Long transactionID) { /** * Add an event pointer to the inflights list. + * * @param transactionID * @param pointer */ - public void addEvent(Long transactionID, Long pointer){ + public void addEvent(Long transactionID, Long pointer) { inflightEvents.put(transactionID, pointer); inflightFileIDs.put(transactionID, - FlumeEventPointer.fromLong(pointer).getFileID()); + FlumeEventPointer.fromLong(pointer).getFileID()); syncRequired = true; } /** * Serialize the set of in flights into a byte longBuffer. + * * @return Returns the checksum of the buffer that is being * asynchronously written to disk. */ public void serializeAndWrite() throws Exception { Collection values = inflightEvents.values(); - if(!fileChannel.isOpen()){ + if (!fileChannel.isOpen()) { file = new RandomAccessFile(inflightEventsFile, "rw"); fileChannel = file.getChannel(); } - if(values.isEmpty()){ + if (values.isEmpty()) { file.setLength(0L); } //What is written out? @@ -498,14 +509,15 @@ public void serializeAndWrite() throws Exception { //transactionid numberofeventsforthistxn listofeventpointers try { - int expectedFileSize = (((inflightEvents.keySet().size() * 2) //For transactionIDs and events per txn ID - + values.size()) * 8) //Event pointers - + 16; //Checksum + int expectedFileSize = (((inflightEvents.keySet().size() * 2) //for transactionIDs and + //events per txn ID + + values.size()) * 8) //Event pointers + + 16; //Checksum //There is no real need of filling the channel with 0s, since we //will write the exact nummber of bytes as expected file size. file.setLength(expectedFileSize); Preconditions.checkState(file.length() == expectedFileSize, - "Expected File size of inflight events file does not match the " + "Expected File size of inflight events file does not match the " + "current file size. Checkpoint is incomplete."); file.seek(0); final ByteBuffer buffer = ByteBuffer.allocate(expectedFileSize); @@ -515,10 +527,10 @@ public void serializeAndWrite() throws Exception { longBuffer.put(txnID); longBuffer.put((long) pointers.size()); LOG.debug("Number of events inserted into " - + "inflights file: " + String.valueOf(pointers.size()) - + " file: " + inflightEventsFile.getCanonicalPath()); + + "inflights file: " + String.valueOf(pointers.size()) + + " file: " + inflightEventsFile.getCanonicalPath()); long[] written = ArrayUtils.toPrimitive( - pointers.toArray(new Long[0])); + pointers.toArray(new Long[0])); longBuffer.put(written); } byte[] checksum = digest.digest(buffer.array()); @@ -539,56 +551,55 @@ public void serializeAndWrite() throws Exception { * of transactionIDs to events that were inflight. * * @return - map of inflight events per txnID. - * */ public SetMultimap deserialize() - throws IOException, BadCheckpointException { + throws IOException, BadCheckpointException { SetMultimap inflights = HashMultimap.create(); if (!fileChannel.isOpen()) { file = new RandomAccessFile(inflightEventsFile, "rw"); fileChannel = file.getChannel(); } - if(file.length() == 0) { + if (file.length() == 0) { return inflights; } file.seek(0); byte[] checksum = new byte[16]; file.read(checksum); ByteBuffer buffer = ByteBuffer.allocate( - (int)(file.length() - file.getFilePointer())); + (int) (file.length() - file.getFilePointer())); fileChannel.read(buffer); byte[] fileChecksum = digest.digest(buffer.array()); if (!Arrays.equals(checksum, fileChecksum)) { throw new BadCheckpointException("Checksum of inflights file differs" - + " from the checksum expected."); + + " from the checksum expected."); } buffer.position(0); LongBuffer longBuffer = buffer.asLongBuffer(); try { while (true) { long txnID = longBuffer.get(); - int numEvents = (int)(longBuffer.get()); - for(int i = 0; i < numEvents; i++) { + int numEvents = (int) (longBuffer.get()); + for (int i = 0; i < numEvents; i++) { long val = longBuffer.get(); inflights.put(txnID, val); } } } catch (BufferUnderflowException ex) { LOG.debug("Reached end of inflights buffer. Long buffer position =" - + String.valueOf(longBuffer.position())); + + String.valueOf(longBuffer.position())); } - return inflights; + return inflights; } public int getSize() { return inflightEvents.size(); } - public boolean syncRequired(){ + public boolean syncRequired() { return syncRequired; } - public Collection getFileIDs(){ + public Collection getFileIDs() { return inflightFileIDs.values(); } diff --git a/flume-ng-channels/flume-file-channel/src/main/java/org/apache/flume/channel/file/Log.java b/flume-ng-channels/flume-file-channel/src/main/java/org/apache/flume/channel/file/Log.java index 247c287930..02d8e7f6bf 100644 --- a/flume-ng-channels/flume-file-channel/src/main/java/org/apache/flume/channel/file/Log.java +++ b/flume-ng-channels/flume-file-channel/src/main/java/org/apache/flume/channel/file/Log.java @@ -63,7 +63,7 @@ * Stores FlumeEvents on disk and pointers to the events in a in memory queue. * Once a log object is created the replay method should be called to reconcile * the on disk write ahead log with the last checkpoint of the queue. - * + *

    * Before calling any of commitPut/commitTake/get/put/rollback/take * {@linkplain org.apache.flume.channel.file.Log#lockShared()} * should be called. After @@ -217,12 +217,12 @@ Builder setMinimumRequiredSpace(long minimumRequiredSpace) { return this; } - Builder setUseLogReplayV1(boolean useLogReplayV1){ + Builder setUseLogReplayV1(boolean useLogReplayV1) { this.useLogReplayV1 = useLogReplayV1; return this; } - Builder setUseFastReplay(boolean useFastReplay){ + Builder setUseFastReplay(boolean useFastReplay) { this.useFastReplay = useFastReplay; return this; } @@ -264,46 +264,46 @@ Builder setCheckpointOnClose(boolean enableCheckpointOnClose) { Log build() throws IOException { return new Log(bCheckpointInterval, bMaxFileSize, bQueueCapacity, - bUseDualCheckpoints, bCompressBackupCheckpoint,bCheckpointDir, - bBackupCheckpointDir, bName, useLogReplayV1, useFastReplay, - bMinimumRequiredSpace, bEncryptionKeyProvider, bEncryptionKeyAlias, - bEncryptionCipherProvider, bUsableSpaceRefreshInterval, - fsyncPerTransaction, fsyncInterval, checkpointOnClose, bLogDirs); + bUseDualCheckpoints, bCompressBackupCheckpoint, bCheckpointDir, + bBackupCheckpointDir, bName, useLogReplayV1, useFastReplay, + bMinimumRequiredSpace, bEncryptionKeyProvider, bEncryptionKeyAlias, + bEncryptionCipherProvider, bUsableSpaceRefreshInterval, + fsyncPerTransaction, fsyncInterval, checkpointOnClose, bLogDirs); } } private Log(long checkpointInterval, long maxFileSize, int queueCapacity, - boolean useDualCheckpoints, boolean compressBackupCheckpoint, - File checkpointDir, File backupCheckpointDir, - String name, boolean useLogReplayV1, boolean useFastReplay, - long minimumRequiredSpace, @Nullable KeyProvider encryptionKeyProvider, - @Nullable String encryptionKeyAlias, - @Nullable String encryptionCipherProvider, - long usableSpaceRefreshInterval, boolean fsyncPerTransaction, - int fsyncInterval, boolean checkpointOnClose, File... logDirs) - throws IOException { + boolean useDualCheckpoints, boolean compressBackupCheckpoint, + File checkpointDir, File backupCheckpointDir, + String name, boolean useLogReplayV1, boolean useFastReplay, + long minimumRequiredSpace, @Nullable KeyProvider encryptionKeyProvider, + @Nullable String encryptionKeyAlias, + @Nullable String encryptionCipherProvider, + long usableSpaceRefreshInterval, boolean fsyncPerTransaction, + int fsyncInterval, boolean checkpointOnClose, File... logDirs) + throws IOException { Preconditions.checkArgument(checkpointInterval > 0, - "checkpointInterval <= 0"); + "checkpointInterval <= 0"); Preconditions.checkArgument(queueCapacity > 0, "queueCapacity <= 0"); Preconditions.checkArgument(maxFileSize > 0, "maxFileSize <= 0"); Preconditions.checkNotNull(checkpointDir, "checkpointDir"); Preconditions.checkArgument(usableSpaceRefreshInterval > 0, "usableSpaceRefreshInterval <= 0"); Preconditions.checkArgument( - checkpointDir.isDirectory() || checkpointDir.mkdirs(), "CheckpointDir " - + checkpointDir + " could not be created"); + checkpointDir.isDirectory() || checkpointDir.mkdirs(), "CheckpointDir " + + checkpointDir + " could not be created"); if (useDualCheckpoints) { Preconditions.checkNotNull(backupCheckpointDir, "backupCheckpointDir is" + - " null while dual checkpointing is enabled."); + " null while dual checkpointing is enabled."); Preconditions.checkArgument( - backupCheckpointDir.isDirectory() || backupCheckpointDir.mkdirs(), - "Backup CheckpointDir " + backupCheckpointDir + - " could not be created"); + backupCheckpointDir.isDirectory() || backupCheckpointDir.mkdirs(), + "Backup CheckpointDir " + backupCheckpointDir + + " could not be created"); } Preconditions.checkNotNull(logDirs, "logDirs"); Preconditions.checkArgument(logDirs.length > 0, "logDirs empty"); Preconditions.checkArgument(name != null && !name.trim().isEmpty(), - "channel name should be specified"); + "channel name should be specified"); this.channelNameDescriptor = "[channel=" + name + "]"; this.useLogReplayV1 = useLogReplayV1; @@ -317,20 +317,20 @@ private Log(long checkpointInterval, long maxFileSize, int queueCapacity, locks = Maps.newHashMap(); try { lock(checkpointDir); - if(useDualCheckpoints) { + if (useDualCheckpoints) { lock(backupCheckpointDir); } for (File logDir : logDirs) { lock(logDir); } - } catch(IOException e) { + } catch (IOException e) { unlock(checkpointDir); for (File logDir : logDirs) { unlock(logDir); } throw e; } - if(encryptionKeyProvider != null && encryptionKeyAlias != null && + if (encryptionKeyProvider != null && encryptionKeyAlias != null && encryptionCipherProvider != null) { LOGGER.info("Encryption is enabled with encryptionKeyProvider = " + encryptionKeyProvider + ", encryptionKeyAlias = " + encryptionKeyAlias @@ -346,7 +346,7 @@ private Log(long checkpointInterval, long maxFileSize, int queueCapacity, throw new IllegalArgumentException("Encryption configuration must all " + "null or all not null: encryptionKeyProvider = " + encryptionKeyProvider + ", encryptionKeyAlias = " + - encryptionKeyAlias + ", encryptionCipherProvider = " + + encryptionKeyAlias + ", encryptionCipherProvider = " + encryptionCipherProvider); } open = false; @@ -364,7 +364,7 @@ private Log(long checkpointInterval, long maxFileSize, int queueCapacity, logFiles = new AtomicReferenceArray(this.logDirs.length); workerExecutor = Executors.newSingleThreadScheduledExecutor(new - ThreadFactoryBuilder().setNameFormat("Log-BackgroundWorker-" + name) + ThreadFactoryBuilder().setNameFormat("Log-BackgroundWorker-" + name) .build()); workerExecutor.scheduleWithFixedDelay(new BackgroundWorker(this), this.checkpointInterval, this.checkpointInterval, @@ -374,6 +374,7 @@ private Log(long checkpointInterval, long maxFileSize, int queueCapacity, /** * Read checkpoint and data files from disk replaying them to the state * directly before the shutdown or crash. + * * @throws IOException */ void replay() throws IOException { @@ -416,8 +417,8 @@ void replay() throws IOException { * locations. We will read the last one written to disk. */ File checkpointFile = new File(checkpointDir, "checkpoint"); - if(shouldFastReplay) { - if(checkpointFile.exists()) { + if (shouldFastReplay) { + if (checkpointFile.exists()) { LOGGER.debug("Disabling fast full replay because checkpoint " + "exists: " + checkpointFile); shouldFastReplay = false; @@ -434,14 +435,14 @@ void replay() throws IOException { try { backingStore = - EventQueueBackingStoreFactory.get(checkpointFile, - backupCheckpointDir, queueCapacity, channelNameDescriptor, - true, this.useDualCheckpoints, - this.compressBackupCheckpoint); + EventQueueBackingStoreFactory.get(checkpointFile, + backupCheckpointDir, queueCapacity, channelNameDescriptor, + true, this.useDualCheckpoints, + this.compressBackupCheckpoint); queue = new FlumeEventQueue(backingStore, inflightTakesFile, - inflightPutsFile, queueSetDir); + inflightPutsFile, queueSetDir); LOGGER.info("Last Checkpoint " + new Date(checkpointFile.lastModified()) - + ", queue depth = " + queue.getSize()); + + ", queue depth = " + queue.getSize()); /* * We now have everything we need to actually replay the log files @@ -460,7 +461,7 @@ void replay() throws IOException { + "Restoring checkpoint and starting up.", ex); if (EventQueueBackingStoreFile.backupExists(backupCheckpointDir)) { backupRestored = EventQueueBackingStoreFile.restoreBackup( - checkpointDir, backupCheckpointDir); + checkpointDir, backupCheckpointDir); } } if (!backupRestored) { @@ -472,16 +473,16 @@ void replay() throws IOException { } } backingStore = EventQueueBackingStoreFactory.get( - checkpointFile, backupCheckpointDir, queueCapacity, - channelNameDescriptor, true, useDualCheckpoints, - compressBackupCheckpoint); + checkpointFile, backupCheckpointDir, queueCapacity, + channelNameDescriptor, true, useDualCheckpoints, + compressBackupCheckpoint); queue = new FlumeEventQueue(backingStore, inflightTakesFile, - inflightPutsFile, queueSetDir); + inflightPutsFile, queueSetDir); // If the checkpoint was deleted due to BadCheckpointException, then // trigger fast replay if the channel is configured to. shouldFastReplay = this.useFastReplay; doReplay(queue, dataFiles, encryptionKeyProvider, shouldFastReplay); - if(!shouldFastReplay) { + if (!shouldFastReplay) { didFullReplayDueToBadCheckpointException = true; } } @@ -514,13 +515,13 @@ private void doReplay(FlumeEventQueue queue, List dataFiles, KeyProvider encryptionKeyProvider, boolean useFastReplay) throws Exception { CheckpointRebuilder rebuilder = new CheckpointRebuilder(dataFiles, - queue, fsyncPerTransaction); + queue, fsyncPerTransaction); if (useFastReplay && rebuilder.rebuild()) { didFastReplay = true; LOGGER.info("Fast replay successful."); } else { ReplayHandler replayHandler = new ReplayHandler(queue, - encryptionKeyProvider, fsyncPerTransaction); + encryptionKeyProvider, fsyncPerTransaction); if (useLogReplayV1) { LOGGER.info("Replaying logs with v1 replay logic"); replayHandler.replayLogv1(dataFiles); @@ -540,10 +541,12 @@ private void doReplay(FlumeEventQueue queue, List dataFiles, boolean didFastReplay() { return didFastReplay; } + @VisibleForTesting public int getReadCount() { return readCount; } + @VisibleForTesting public int getPutCount() { return putCount; @@ -553,10 +556,12 @@ public int getPutCount() { public int getTakeCount() { return takeCount; } + @VisibleForTesting public int getCommittedCount() { return committedCount; } + @VisibleForTesting public int getRollbackCount() { return rollbackCount; @@ -564,6 +569,7 @@ public int getRollbackCount() { /** * Was a checkpoint backup used to replay? + * * @return true if a checkpoint backup was used to replay. */ @VisibleForTesting @@ -597,7 +603,7 @@ FlumeEventQueue getFlumeEventQueue() { * @throws InterruptedException */ FlumeEvent get(FlumeEventPointer pointer) throws IOException, - InterruptedException, NoopRecordException, CorruptEventException { + InterruptedException, NoopRecordException, CorruptEventException { Preconditions.checkState(open, "Log is closed"); int id = pointer.getFileID(); LogFile.RandomReader logFile = idLogFileMap.get(id); @@ -608,7 +614,7 @@ FlumeEvent get(FlumeEventPointer pointer) throws IOException, if (fsyncPerTransaction) { open = false; throw new IOException("Corrupt event found. Please run File Channel " + - "Integrity tool.", ex); + "Integrity tool.", ex); } throw ex; } @@ -616,8 +622,9 @@ FlumeEvent get(FlumeEventPointer pointer) throws IOException, /** * Log a put of an event - * + *

    * Synchronization not required as this method is atomic + * * @param transactionID * @param event * @return @@ -633,7 +640,7 @@ FlumeEventPointer put(long transactionID, Event event) int logFileIndex = nextLogWriter(transactionID); long usableSpace = logFiles.get(logFileIndex).getUsableSpace(); long requiredSpace = minimumRequiredSpace + buffer.limit(); - if(usableSpace <= requiredSpace) { + if (usableSpace <= requiredSpace) { throw new IOException("Usable space exhausted, only " + usableSpace + " bytes remaining, required " + requiredSpace + " bytes"); } @@ -644,7 +651,7 @@ FlumeEventPointer put(long transactionID, Event event) error = false; return ptr; } catch (LogFileRetryableIOException e) { - if(!open) { + if (!open) { throw e; } roll(logFileIndex, buffer); @@ -653,7 +660,7 @@ FlumeEventPointer put(long transactionID, Event event) return ptr; } } finally { - if(error && open) { + if (error && open) { roll(logFileIndex); } } @@ -661,8 +668,9 @@ FlumeEventPointer put(long transactionID, Event event) /** * Log a take of an event, pointer points at the corresponding put - * + *

    * Synchronization not required as this method is atomic + * * @param transactionID * @param pointer * @throws IOException @@ -676,7 +684,7 @@ void take(long transactionID, FlumeEventPointer pointer) int logFileIndex = nextLogWriter(transactionID); long usableSpace = logFiles.get(logFileIndex).getUsableSpace(); long requiredSpace = minimumRequiredSpace + buffer.limit(); - if(usableSpace <= requiredSpace) { + if (usableSpace <= requiredSpace) { throw new IOException("Usable space exhausted, only " + usableSpace + " bytes remaining, required " + requiredSpace + " bytes"); } @@ -686,7 +694,7 @@ void take(long transactionID, FlumeEventPointer pointer) logFiles.get(logFileIndex).take(buffer); error = false; } catch (LogFileRetryableIOException e) { - if(!open) { + if (!open) { throw e; } roll(logFileIndex, buffer); @@ -694,7 +702,7 @@ void take(long transactionID, FlumeEventPointer pointer) error = false; } } finally { - if(error && open) { + if (error && open) { roll(logFileIndex); } } @@ -702,15 +710,16 @@ void take(long transactionID, FlumeEventPointer pointer) /** * Log a rollback of a transaction - * + *

    * Synchronization not required as this method is atomic + * * @param transactionID * @throws IOException */ void rollback(long transactionID) throws IOException { Preconditions.checkState(open, "Log is closed"); - if(LOGGER.isDebugEnabled()) { + if (LOGGER.isDebugEnabled()) { LOGGER.debug("Rolling back " + transactionID); } Rollback rollback = new Rollback(transactionID, WriteOrderOracle.next()); @@ -718,7 +727,7 @@ void rollback(long transactionID) throws IOException { int logFileIndex = nextLogWriter(transactionID); long usableSpace = logFiles.get(logFileIndex).getUsableSpace(); long requiredSpace = minimumRequiredSpace + buffer.limit(); - if(usableSpace <= requiredSpace) { + if (usableSpace <= requiredSpace) { throw new IOException("Usable space exhausted, only " + usableSpace + " bytes remaining, required " + requiredSpace + " bytes"); } @@ -728,7 +737,7 @@ void rollback(long transactionID) throws IOException { logFiles.get(logFileIndex).rollback(buffer); error = false; } catch (LogFileRetryableIOException e) { - if(!open) { + if (!open) { throw e; } roll(logFileIndex, buffer); @@ -736,7 +745,7 @@ void rollback(long transactionID) throws IOException { error = false; } } finally { - if(error && open) { + if (error && open) { roll(logFileIndex); } } @@ -747,14 +756,15 @@ void rollback(long transactionID) throws IOException { * so we know if the pointers corresponding to the events * should be added or removed from the flume queue. We * could infer but it's best to be explicit. - * + *

    * Synchronization not required as this method is atomic + * * @param transactionID * @throws IOException * @throws InterruptedException */ void commitPut(long transactionID) throws IOException, - InterruptedException { + InterruptedException { Preconditions.checkState(open, "Log is closed"); commit(transactionID, TransactionEventRecord.Type.PUT.get()); } @@ -764,20 +774,21 @@ void commitPut(long transactionID) throws IOException, * so we know if the pointers corresponding to the events * should be added or removed from the flume queue. We * could infer but it's best to be explicit. - * + *

    * Synchronization not required as this method is atomic + * * @param transactionID * @throws IOException * @throws InterruptedException */ void commitTake(long transactionID) throws IOException, - InterruptedException { + InterruptedException { Preconditions.checkState(open, "Log is closed"); commit(transactionID, TransactionEventRecord.Type.TAKE.get()); } - private void unlockExclusive() { + private void unlockExclusive() { checkpointWriterLock.unlock(); } @@ -785,11 +796,11 @@ void lockShared() { checkpointReadLock.lock(); } - void unlockShared() { + void unlockShared() { checkpointReadLock.unlock(); } - private void lockExclusive(){ + private void lockExclusive() { checkpointWriterLock.lock(); } @@ -797,23 +808,23 @@ private void lockExclusive(){ * Synchronization not required since this method gets the write lock, * so checkpoint and this method cannot run at the same time. */ - void close() throws IOException{ + void close() throws IOException { lockExclusive(); try { open = false; try { - if(checkpointOnClose) { + if (checkpointOnClose) { writeCheckpoint(true); // do this before acquiring exclusive lock } } catch (Exception err) { LOGGER.warn("Failed creating checkpoint on close of channel " + channelNameDescriptor + - "Replay will take longer next time channel is started.", err); + "Replay will take longer next time channel is started.", err); } shutdownWorker(); if (logFiles != null) { for (int index = 0; index < logFiles.length(); index++) { LogFile.Writer writer = logFiles.get(index); - if(writer != null) { + if (writer != null) { writer.close(); } } @@ -862,9 +873,11 @@ void shutdownWorker() { LOGGER.error("Interrupted while waiting for worker to die."); } } + void setCheckpointInterval(long checkpointInterval) { this.checkpointInterval = checkpointInterval; } + void setMaxFileSize(long maxFileSize) { this.maxFileSize = maxFileSize; } @@ -883,7 +896,7 @@ private void commit(long transactionID, short type) throws IOException { int logFileIndex = nextLogWriter(transactionID); long usableSpace = logFiles.get(logFileIndex).getUsableSpace(); long requiredSpace = minimumRequiredSpace + buffer.limit(); - if(usableSpace <= requiredSpace) { + if (usableSpace <= requiredSpace) { throw new IOException("Usable space exhausted, only " + usableSpace + " bytes remaining, required " + requiredSpace + " bytes"); } @@ -898,7 +911,7 @@ private void commit(long transactionID, short type) throws IOException { logFileWriter.sync(); error = false; } catch (LogFileRetryableIOException e) { - if(!open) { + if (!open) { throw e; } roll(logFileIndex, buffer); @@ -908,7 +921,7 @@ private void commit(long transactionID, short type) throws IOException { error = false; } } finally { - if(error && open) { + if (error && open) { roll(logFileIndex); } } @@ -917,11 +930,13 @@ private void commit(long transactionID, short type) throws IOException { /** * Atomic so not synchronization required. + * * @return */ private int nextLogWriter(long transactionID) { - return (int)Math.abs(transactionID % (long)logFiles.length()); + return (int) Math.abs(transactionID % (long) logFiles.length()); } + /** * Unconditionally roll * Synchronization done internally @@ -932,13 +947,14 @@ private int nextLogWriter(long transactionID) { private void roll(int index) throws IOException { roll(index, null); } + /** * Roll a log if needed. Roll always occurs if the log at the index * does not exist (typically on startup), or buffer is null. Otherwise * LogFile.Writer.isRollRequired is checked again to ensure we don't * have threads pile up on this log resulting in multiple successive * rolls - * + *

    * Synchronization required since both synchronized and unsynchronized * methods call this method, and this method acquires only a * read lock. The synchronization guarantees that multiple threads don't @@ -948,7 +964,7 @@ private void roll(int index) throws IOException { * @throws IOException */ private synchronized void roll(int index, ByteBuffer buffer) - throws IOException { + throws IOException { lockShared(); try { @@ -956,17 +972,17 @@ private synchronized void roll(int index, ByteBuffer buffer) // check to make sure a roll is actually required due to // the possibility of multiple writes waiting on lock if (oldLogFile == null || buffer == null || - oldLogFile.isRollRequired(buffer)) { + oldLogFile.isRollRequired(buffer)) { try { LOGGER.info("Roll start " + logDirs[index]); int fileID = nextFileID.incrementAndGet(); File file = new File(logDirs[index], PREFIX + fileID); LogFile.Writer writer = LogFileFactory.getWriter(file, fileID, - maxFileSize, encryptionKey, encryptionKeyAlias, - encryptionCipherProvider, usableSpaceRefreshInterval, - fsyncPerTransaction, fsyncInterval); + maxFileSize, encryptionKey, encryptionKeyAlias, + encryptionCipherProvider, usableSpaceRefreshInterval, + fsyncPerTransaction, fsyncInterval); idLogFileMap.put(fileID, LogFileFactory.getRandomReader(file, - encryptionKeyProvider, fsyncPerTransaction)); + encryptionKeyProvider, fsyncPerTransaction)); // writer from this point on will get new reference logFiles.set(index, writer); // close out old log @@ -989,22 +1005,24 @@ private boolean writeCheckpoint() throws Exception { /** * Write the current checkpoint object and then swap objects so that * the next checkpoint occurs on the other checkpoint directory. - * + *

    * Synchronization is not required because this method acquires a * write lock. So this method gets exclusive access to all the * data structures this method accesses. - * @param force a flag to force the writing of checkpoint + * + * @param force a flag to force the writing of checkpoint * @throws IOException if we are unable to write the checkpoint out to disk */ private Boolean writeCheckpoint(Boolean force) throws Exception { boolean checkpointCompleted = false; long usableSpace = checkpointDir.getUsableSpace(); - if(usableSpace <= minimumRequiredSpace) { + if (usableSpace <= minimumRequiredSpace) { throw new IOException("Usable space exhausted, only " + usableSpace + " bytes remaining, required " + minimumRequiredSpace + " bytes"); } lockExclusive(); - SortedSet logFileRefCountsAll = null, logFileRefCountsActive = null; + SortedSet logFileRefCountsAll = null; + SortedSet logFileRefCountsActive = null; try { if (queue.checkpoint(force)) { long logWriteOrderID = queue.getLogWriteOrderID(); @@ -1048,7 +1066,7 @@ private Boolean writeCheckpoint(Boolean force) throws Exception { writer.markCheckpoint(logWriteOrderID); } finally { reader = LogFileFactory.getRandomReader(file, - encryptionKeyProvider, fsyncPerTransaction); + encryptionKeyProvider, fsyncPerTransaction); idLogFileMap.put(id, reader); writer.close(); } @@ -1058,7 +1076,7 @@ private Boolean writeCheckpoint(Boolean force) throws Exception { idIterator.remove(); } Preconditions.checkState(logFileRefCountsAll.size() == 0, - "Could not update all data file timestamps: " + logFileRefCountsAll); + "Could not update all data file timestamps: " + logFileRefCountsAll); //Add files from all log directories for (int index = 0; index < logDirs.length; index++) { logFileRefCountsActive.add(logFiles.get(index).getLogFileID()); @@ -1086,7 +1104,7 @@ private void removeOldLogs(SortedSet fileIDs) { // these files) and delete them only after the next (since the current // checkpoint will become the backup at that time, // and thus these files are no longer needed). - for(File fileToDelete : pendingDeletes) { + for (File fileToDelete : pendingDeletes) { LOGGER.info("Removing old file: " + fileToDelete); FileUtils.deleteQuietly(fileToDelete); } @@ -1095,7 +1113,7 @@ private void removeOldLogs(SortedSet fileIDs) { // won't delete any files with an id larger than the min int minFileID = fileIDs.first(); LOGGER.debug("Files currently in use: " + fileIDs); - for(File logDir : logDirs) { + for (File logDir : logDirs) { List logs = LogUtils.getLogs(logDir); // sort oldset to newest LogUtils.sort(logs); @@ -1104,9 +1122,9 @@ private void removeOldLogs(SortedSet fileIDs) { for (int index = 0; index < size; index++) { File logFile = logs.get(index); int logFileID = LogUtils.getIDForFile(logFile); - if(logFileID < minFileID) { + if (logFileID < minFileID) { LogFile.RandomReader reader = idLogFileMap.remove(logFileID); - if(reader != null) { + if (reader != null) { reader.close(); } File metaDataFile = Serialization.getMetaDataFile(logFile); @@ -1116,12 +1134,13 @@ private void removeOldLogs(SortedSet fileIDs) { } } } + /** * Lock storage to provide exclusive access. - * + *

    *

    Locking is not supported by all file systems. * E.g., NFS does not consistently support exclusive locks. - * + *

    *

    If locking is supported we guarantee exculsive access to the * storage directory. Otherwise, no guarantee is given. * @@ -1137,8 +1156,8 @@ private void lock(File dir) throws IOException { throw new IOException(msg); } FileLock secondLock = tryLock(dir); - if(secondLock != null) { - LOGGER.warn("Directory "+dir+" does not support locking"); + if (secondLock != null) { + LOGGER.warn("Directory " + dir + " does not support locking"); secondLock.release(); secondLock.channel().close(); } @@ -1160,10 +1179,10 @@ private FileLock tryLock(File dir) throws IOException { FileLock res = null; try { res = file.getChannel().tryLock(); - } catch(OverlappingFileLockException oe) { + } catch (OverlappingFileLockException oe) { file.close(); return null; - } catch(IOException e) { + } catch (IOException e) { LOGGER.error("Cannot create lock on " + lockF, e); file.close(); throw e; @@ -1178,13 +1197,14 @@ private FileLock tryLock(File dir) throws IOException { */ private void unlock(File dir) throws IOException { FileLock lock = locks.remove(dir.getAbsolutePath()); - if(lock == null) { + if (lock == null) { return; } lock.release(); lock.channel().close(); lock = null; } + static class BackgroundWorker implements Runnable { private static final Logger LOG = LoggerFactory .getLogger(BackgroundWorker.class); diff --git a/flume-ng-channels/flume-file-channel/src/main/java/org/apache/flume/channel/file/LogFile.java b/flume-ng-channels/flume-file-channel/src/main/java/org/apache/flume/channel/file/LogFile.java index 488dcf4a75..336aa2c186 100644 --- a/flume-ng-channels/flume-file-channel/src/main/java/org/apache/flume/channel/file/LogFile.java +++ b/flume-ng-channels/flume-file-channel/src/main/java/org/apache/flume/channel/file/LogFile.java @@ -50,20 +50,17 @@ @InterfaceStability.Unstable public abstract class LogFile { - private static final Logger LOG = LoggerFactory - .getLogger(LogFile.class); - + private static final Logger LOG = LoggerFactory.getLogger(LogFile.class); /** * This class preallocates the data files 1MB at time to avoid * the updating of the inode on each write and to avoid the disk * filling up during a write. It's also faster, so there. */ - private static final ByteBuffer FILL = DirectMemoryUtils. - allocate(1024 * 1024); // preallocation, 1MB + private static final ByteBuffer FILL = DirectMemoryUtils.allocate(1024 * 1024); public static final byte OP_RECORD = Byte.MAX_VALUE; - public static final byte OP_NOOP = (Byte.MAX_VALUE + Byte.MIN_VALUE)/2; + public static final byte OP_NOOP = (Byte.MAX_VALUE + Byte.MIN_VALUE) / 2; public static final byte OP_EOF = Byte.MIN_VALUE; static { @@ -73,7 +70,7 @@ public abstract class LogFile { } protected static void skipRecord(RandomAccessFile fileHandle, - int offset) throws IOException { + int offset) throws IOException { fileHandle.seek(offset); int length = fileHandle.readInt(); fileHandle.skipBytes(length); @@ -93,31 +90,40 @@ protected MetaDataWriter(File file, int logFileID) throws IOException { writeFileHandle = new RandomAccessFile(file, "rw"); } + protected RandomAccessFile getFileHandle() { return writeFileHandle; } + protected void setLastCheckpointOffset(long lastCheckpointOffset) { this.lastCheckpointOffset = lastCheckpointOffset; } + protected void setLastCheckpointWriteOrderID(long lastCheckpointWriteOrderID) { this.lastCheckpointWriteOrderID = lastCheckpointWriteOrderID; } + protected long getLastCheckpointOffset() { return lastCheckpointOffset; } + protected long getLastCheckpointWriteOrderID() { return lastCheckpointWriteOrderID; } + protected File getFile() { return file; } + protected int getLogFileID() { return logFileID; } + void markCheckpoint(long logWriteOrderID) throws IOException { markCheckpoint(lastCheckpointOffset, logWriteOrderID); } + abstract void markCheckpoint(long currentPosition, long logWriteOrderID) throws IOException; @@ -150,9 +156,10 @@ void decrement(long numBytes) { Preconditions.checkArgument(numBytes >= 0, "numBytes less than zero"); value.addAndGet(-numBytes); } + long getUsableSpace() { long now = System.currentTimeMillis(); - if(now - interval > lastRefresh.get()) { + if (now - interval > lastRefresh.get()) { value.set(fs.getUsableSpace()); lastRefresh.set(now); } @@ -160,7 +167,7 @@ long getUsableSpace() { } } - static abstract class Writer { + abstract static class Writer { private final int logFileID; private final File file; private final long maxFileSize; @@ -180,10 +187,9 @@ static abstract class Writer { // To ensure we can count the number of fsyncs. private long syncCount; - Writer(File file, int logFileID, long maxFileSize, - CipherProvider.Encryptor encryptor, long usableSpaceRefreshInterval, - boolean fsyncPerTransaction, int fsyncInterval) throws IOException { + CipherProvider.Encryptor encryptor, long usableSpaceRefreshInterval, + boolean fsyncPerTransaction, int fsyncInterval) throws IOException { this.file = file; this.logFileID = logFileID; this.maxFileSize = Math.min(maxFileSize, @@ -193,7 +199,7 @@ static abstract class Writer { writeFileChannel = writeFileHandle.getChannel(); this.fsyncPerTransaction = fsyncPerTransaction; this.fsyncInterval = fsyncInterval; - if(!fsyncPerTransaction) { + if (!fsyncPerTransaction) { LOG.info("Sync interval = " + fsyncInterval); syncExecutor = Executors.newSingleThreadScheduledExecutor(); syncExecutor.scheduleWithFixedDelay(new Runnable() { @@ -203,7 +209,7 @@ public void run() { sync(); } catch (Throwable ex) { LOG.error("Data file, " + getFile().toString() + " could not " + - "be synced to disk due to an error.", ex); + "be synced to disk due to an error.", ex); } } }, fsyncInterval, fsyncInterval, TimeUnit.SECONDS); @@ -220,6 +226,7 @@ public void run() { protected CipherProvider.Encryptor getEncryptor() { return encryptor; } + int getLogFileID() { return logFileID; } @@ -227,6 +234,7 @@ int getLogFileID() { File getFile() { return file; } + String getParent() { return file.getParent(); } @@ -240,7 +248,7 @@ long getMaxSize() { } @VisibleForTesting - long getLastCommitPosition(){ + long getLastCommitPosition() { return lastCommitPosition; } @@ -253,6 +261,7 @@ long getLastSyncPosition() { long getSyncCount() { return syncCount; } + synchronized long position() throws IOException { return getFileChannel().position(); } @@ -261,20 +270,22 @@ synchronized long position() throws IOException { // methods, so all methods need to be synchronized. synchronized FlumeEventPointer put(ByteBuffer buffer) throws IOException { - if(encryptor != null) { + if (encryptor != null) { buffer = ByteBuffer.wrap(encryptor.encrypt(buffer.array())); } Pair pair = write(buffer); return new FlumeEventPointer(pair.getLeft(), pair.getRight()); } + synchronized void take(ByteBuffer buffer) throws IOException { - if(encryptor != null) { + if (encryptor != null) { buffer = ByteBuffer.wrap(encryptor.encrypt(buffer.array())); } write(buffer); } + synchronized void rollback(ByteBuffer buffer) throws IOException { - if(encryptor != null) { + if (encryptor != null) { buffer = ByteBuffer.wrap(encryptor.encrypt(buffer.array())); } write(buffer); @@ -290,20 +301,20 @@ synchronized void commit(ByteBuffer buffer) throws IOException { } private Pair write(ByteBuffer buffer) - throws IOException { - if(!isOpen()) { + throws IOException { + if (!isOpen()) { throw new LogFileRetryableIOException("File closed " + file); } long length = position(); long expectedLength = length + (long) buffer.limit(); - if(expectedLength > maxFileSize) { + if (expectedLength > maxFileSize) { throw new LogFileRetryableIOException(expectedLength + " > " + maxFileSize); } - int offset = (int)length; + int offset = (int) length; Preconditions.checkState(offset >= 0, String.valueOf(offset)); // OP_RECORD + size + buffer - int recordLength = 1 + (int)Serialization.SIZE_OF_INT + buffer.limit(); + int recordLength = 1 + (int) Serialization.SIZE_OF_INT + buffer.limit(); usableSpace.decrement(recordLength); preallocate(recordLength); ByteBuffer toWrite = ByteBuffer.allocate(recordLength); @@ -323,15 +334,16 @@ synchronized boolean isRollRequired(ByteBuffer buffer) throws IOException { * Sync the underlying log file to disk. Expensive call, * should be used only on commits. If a sync has already happened after * the last commit, this method is a no-op + * * @throws IOException * @throws LogFileRetryableIOException - if this log file is closed. */ synchronized void sync() throws IOException { if (!fsyncPerTransaction && !dirty) { - if(LOG.isDebugEnabled()) { + if (LOG.isDebugEnabled()) { LOG.debug( - "No events written to file, " + getFile().toString() + - " in last " + fsyncInterval + " or since last commit."); + "No events written to file, " + getFile().toString() + + " in last " + fsyncInterval + " or since last commit."); } return; } @@ -346,27 +358,29 @@ synchronized void sync() throws IOException { } } - protected boolean isOpen() { return open; } + protected RandomAccessFile getFileHandle() { return writeFileHandle; } + protected FileChannel getFileChannel() { return writeFileChannel; } + synchronized void close() { - if(open) { + if (open) { open = false; if (!fsyncPerTransaction) { // Shutdown the executor before attempting to close. - if(syncExecutor != null) { + if (syncExecutor != null) { // No need to wait for it to shutdown. syncExecutor.shutdown(); } } - if(writeFileChannel.isOpen()) { + if (writeFileChannel.isOpen()) { LOG.info("Closing " + file); try { writeFileChannel.force(true); @@ -381,9 +395,10 @@ synchronized void close() { } } } + protected void preallocate(int size) throws IOException { long position = position(); - if(position + size > getFileChannel().size()) { + if (position + size > getFileChannel().size()) { LOG.debug("Preallocating at position " + position); synchronized (FILL) { FILL.position(0); @@ -404,7 +419,7 @@ public static class OperationRecordUpdater { public OperationRecordUpdater(File file) throws FileNotFoundException { Preconditions.checkState(file.exists(), "File to update, " + - file.toString() + " does not exist."); + file.toString() + " does not exist."); this.file = file; fileHandle = new RandomAccessFile(file, "rw"); } @@ -417,10 +432,10 @@ public void markRecordAsNoop(long offset) throws IOException { fileHandle.seek(offset); byte byteRead = fileHandle.readByte(); Preconditions.checkState(byteRead == OP_RECORD || byteRead == OP_NOOP, - "Expected to read a record but the byte read indicates EOF"); + "Expected to read a record but the byte read indicates EOF"); fileHandle.seek(offset); LOG.info("Marking event as " + OP_NOOP + " at " + offset + " for file " + - file.toString()); + file.toString()); fileHandle.writeByte(OP_NOOP); } @@ -430,20 +445,21 @@ public void close() { fileHandle.close(); } catch (IOException e) { LOG.error("Could not close file handle to file " + - fileHandle.toString(), e); + fileHandle.toString(), e); } } } - static abstract class RandomReader { + abstract static class RandomReader { private final File file; private final BlockingQueue readFileHandles = new ArrayBlockingQueue(50, true); private final KeyProvider encryptionKeyProvider; private final boolean fsyncPerTransaction; private volatile boolean open; + public RandomReader(File file, @Nullable KeyProvider - encryptionKeyProvider, boolean fsyncPerTransaction) + encryptionKeyProvider, boolean fsyncPerTransaction) throws IOException { this.file = file; this.encryptionKeyProvider = encryptionKeyProvider; @@ -466,31 +482,31 @@ protected KeyProvider getKeyProvider() { } FlumeEvent get(int offset) throws IOException, InterruptedException, - CorruptEventException, NoopRecordException { + CorruptEventException, NoopRecordException { Preconditions.checkState(open, "File closed"); RandomAccessFile fileHandle = checkOut(); boolean error = true; try { fileHandle.seek(offset); byte operation = fileHandle.readByte(); - if(operation == OP_NOOP) { + if (operation == OP_NOOP) { throw new NoopRecordException("No op record found. Corrupt record " + - "may have been repaired by File Channel Integrity tool"); + "may have been repaired by File Channel Integrity tool"); } if (operation != OP_RECORD) { throw new CorruptEventException( - "Operation code is invalid. File " + - "is corrupt. Please run File Channel Integrity tool."); + "Operation code is invalid. File " + + "is corrupt. Please run File Channel Integrity tool."); } TransactionEventRecord record = doGet(fileHandle); - if(!(record instanceof Put)) { + if (!(record instanceof Put)) { Preconditions.checkState(false, "Record is " + record.getClass().getSimpleName()); } error = false; - return ((Put)record).getEvent(); + return ((Put) record).getEvent(); } finally { - if(error) { + if (error) { close(fileHandle, file); } else { checkIn(fileHandle); @@ -499,12 +515,12 @@ FlumeEvent get(int offset) throws IOException, InterruptedException, } synchronized void close() { - if(open) { + if (open) { open = false; LOG.info("Closing RandomReader " + file); List fileHandles = Lists.newArrayList(); - while(readFileHandles.drainTo(fileHandles) > 0) { - for(RandomAccessFile fileHandle : fileHandles) { + while (readFileHandles.drainTo(fileHandles) > 0) { + for (RandomAccessFile fileHandle : fileHandles) { synchronized (fileHandle) { try { fileHandle.close(); @@ -528,7 +544,7 @@ private RandomAccessFile open() throws IOException { } private void checkIn(RandomAccessFile fileHandle) { - if(!readFileHandles.offer(fileHandle)) { + if (!readFileHandles.offer(fileHandle)) { close(fileHandle, file); } } @@ -536,19 +552,20 @@ private void checkIn(RandomAccessFile fileHandle) { private RandomAccessFile checkOut() throws IOException, InterruptedException { RandomAccessFile fileHandle = readFileHandles.poll(); - if(fileHandle != null) { + if (fileHandle != null) { return fileHandle; } int remaining = readFileHandles.remainingCapacity(); - if(remaining > 0) { + if (remaining > 0) { LOG.info("Opening " + file + " for read, remaining number of file " + - "handles available for reads of this file is " + remaining); + "handles available for reads of this file is " + remaining); return open(); } return readFileHandles.take(); } + private static void close(RandomAccessFile fileHandle, File file) { - if(fileHandle != null) { + if (fileHandle != null) { try { fileHandle.close(); } catch (IOException e) { @@ -558,7 +575,7 @@ private static void close(RandomAccessFile fileHandle, File file) { } } - public static abstract class SequentialReader { + public abstract static class SequentialReader { private final RandomAccessFile fileHandle; private final FileChannel fileChannel; @@ -573,8 +590,9 @@ public static abstract class SequentialReader { /** * Construct a Sequential Log Reader object + * * @param file - * @throws IOException if an I/O error occurs + * @throws IOException if an I/O error occurs * @throws EOFException if the file is empty */ SequentialReader(File file, @Nullable KeyProvider encryptionKeyProvider) @@ -584,6 +602,7 @@ public static abstract class SequentialReader { fileHandle = new RandomAccessFile(file, "r"); fileChannel = fileHandle.getChannel(); } + abstract LogRecord doNext(int offset) throws IOException, CorruptEventException; abstract int getVersion(); @@ -591,50 +610,57 @@ public static abstract class SequentialReader { protected void setLastCheckpointPosition(long lastCheckpointPosition) { this.lastCheckpointPosition = lastCheckpointPosition; } + protected void setLastCheckpointWriteOrderID(long lastCheckpointWriteOrderID) { this.lastCheckpointWriteOrderID = lastCheckpointWriteOrderID; } + protected void setPreviousCheckpointPosition( - long backupCheckpointPosition) { + long backupCheckpointPosition) { this.backupCheckpointPosition = backupCheckpointPosition; } + protected void setPreviousCheckpointWriteOrderID( - long backupCheckpointWriteOrderID) { + long backupCheckpointWriteOrderID) { this.backupCheckpointWriteOrderID = backupCheckpointWriteOrderID; } + protected void setLogFileID(int logFileID) { this.logFileID = logFileID; Preconditions.checkArgument(logFileID >= 0, "LogFileID is not positive: " + Integer.toHexString(logFileID)); } + protected KeyProvider getKeyProvider() { return encryptionKeyProvider; } + protected RandomAccessFile getFileHandle() { return fileHandle; } + int getLogFileID() { return logFileID; } void skipToLastCheckpointPosition(long checkpointWriteOrderID) - throws IOException { + throws IOException { if (lastCheckpointPosition > 0L) { long position = 0; if (lastCheckpointWriteOrderID <= checkpointWriteOrderID) { position = lastCheckpointPosition; } else if (backupCheckpointWriteOrderID <= checkpointWriteOrderID - && backupCheckpointPosition > 0) { + && backupCheckpointPosition > 0) { position = backupCheckpointPosition; } fileChannel.position(position); LOG.info("fast-forward to checkpoint position: " + position); } else { LOG.info("Checkpoint for file(" + file.getAbsolutePath() + ") " - + "is: " + lastCheckpointWriteOrderID + ", which is beyond the " - + "requested checkpoint time: " + checkpointWriteOrderID - + " and position " + lastCheckpointPosition); + + "is: " + lastCheckpointWriteOrderID + ", which is beyond the " + + "requested checkpoint time: " + checkpointWriteOrderID + + " and position " + lastCheckpointPosition); } } @@ -644,8 +670,8 @@ public LogRecord next() throws IOException, CorruptEventException { long position = fileChannel.position(); if (position > FileChannelConfiguration.DEFAULT_MAX_FILE_SIZE) { LOG.info("File position exceeds the threshold: " - + FileChannelConfiguration.DEFAULT_MAX_FILE_SIZE - + ", position: " + position); + + FileChannelConfiguration.DEFAULT_MAX_FILE_SIZE + + ", position: " + position); } offset = (int) position; Preconditions.checkState(offset >= 0); @@ -658,21 +684,21 @@ public LogRecord next() throws IOException, CorruptEventException { return null; } else if (operation == OP_NOOP) { LOG.info("No op event found in file: " + file.toString() + - " at " + offset + ". Skipping event."); + " at " + offset + ". Skipping event."); skipRecord(fileHandle, offset + 1); offset = (int) fileHandle.getFilePointer(); continue; } else { LOG.error("Encountered non op-record at " + offset + " " + - Integer.toHexString(operation) + " in " + file); + Integer.toHexString(operation) + " in " + file); return null; } } - if(offset >= fileHandle.length()) { + if (offset >= fileHandle.length()) { return null; } return doNext(offset); - } catch(EOFException e) { + } catch (EOFException e) { return null; } catch (IOException e) { throw new IOException("Unable to read next Transaction from log file " + @@ -683,33 +709,36 @@ public LogRecord next() throws IOException, CorruptEventException { public long getPosition() throws IOException { return fileChannel.position(); } + public void close() { - if(fileHandle != null) { + if (fileHandle != null) { try { fileHandle.close(); - } catch (IOException e) {} + } catch (IOException e) { + } } } } + protected static void writeDelimitedBuffer(ByteBuffer output, ByteBuffer buffer) throws IOException { output.putInt(buffer.limit()); output.put(buffer); } + protected static byte[] readDelimitedBuffer(RandomAccessFile fileHandle) throws IOException, CorruptEventException { int length = fileHandle.readInt(); if (length < 0) { - throw new CorruptEventException("Length of event is: " + String.valueOf - (length) + ". Event must have length >= 0. Possible corruption of " + - "data or partial fsync."); + throw new CorruptEventException("Length of event is: " + String.valueOf(length) + + ". Event must have length >= 0. Possible corruption of data or partial fsync."); } byte[] buffer = new byte[length]; try { fileHandle.readFully(buffer); } catch (EOFException ex) { throw new CorruptEventException("Remaining data in file less than " + - "expected size of event.", ex); + "expected size of event.", ex); } return buffer; } diff --git a/flume-ng-channels/flume-file-channel/src/main/java/org/apache/flume/channel/file/LogFileFactory.java b/flume-ng-channels/flume-file-channel/src/main/java/org/apache/flume/channel/file/LogFileFactory.java index 7d7fd8504f..f2fcad6492 100644 --- a/flume-ng-channels/flume-file-channel/src/main/java/org/apache/flume/channel/file/LogFileFactory.java +++ b/flume-ng-channels/flume-file-channel/src/main/java/org/apache/flume/channel/file/LogFileFactory.java @@ -18,24 +18,23 @@ */ package org.apache.flume.channel.file; +import com.google.common.base.Preconditions; +import org.apache.flume.channel.file.encryption.KeyProvider; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +import javax.annotation.Nullable; import java.io.EOFException; import java.io.File; import java.io.IOException; import java.io.RandomAccessFile; import java.security.Key; -import javax.annotation.Nullable; - -import org.apache.flume.channel.file.encryption.KeyProvider; -import org.slf4j.Logger; -import org.slf4j.LoggerFactory; - -import com.google.common.base.Preconditions; - @SuppressWarnings("deprecation") class LogFileFactory { private static final Logger LOGGER = LoggerFactory.getLogger(LogFileFactory.class); + private LogFileFactory() {} static LogFile.MetaDataWriter getMetaDataWriter(File file, int logFileID) @@ -43,21 +42,21 @@ static LogFile.MetaDataWriter getMetaDataWriter(File file, int logFileID) RandomAccessFile logFile = null; try { File metaDataFile = Serialization.getMetaDataFile(file); - if(metaDataFile.exists()) { + if (metaDataFile.exists()) { return new LogFileV3.MetaDataWriter(file, logFileID); } logFile = new RandomAccessFile(file, "r"); int version = logFile.readInt(); - if(Serialization.VERSION_2 == version) { + if (Serialization.VERSION_2 == version) { return new LogFileV2.MetaDataWriter(file, logFileID); } throw new IOException("File " + file + " has bad version " + Integer.toHexString(version)); } finally { - if(logFile != null) { + if (logFile != null) { try { logFile.close(); - } catch(IOException e) { + } catch (IOException e) { LOGGER.warn("Unable to close " + file, e); } } @@ -65,13 +64,13 @@ static LogFile.MetaDataWriter getMetaDataWriter(File file, int logFileID) } static LogFile.Writer getWriter(File file, int logFileID, - long maxFileSize, @Nullable Key encryptionKey, - @Nullable String encryptionKeyAlias, - @Nullable String encryptionCipherProvider, - long usableSpaceRefreshInterval, boolean fsyncPerTransaction, - int fsyncInterval) throws IOException { - Preconditions.checkState(!file.exists(), "File already exists " + - file.getAbsolutePath()); + long maxFileSize, @Nullable Key encryptionKey, + @Nullable String encryptionKeyAlias, + @Nullable String encryptionCipherProvider, + long usableSpaceRefreshInterval, boolean fsyncPerTransaction, + int fsyncInterval) throws IOException { + Preconditions.checkState(!file.exists(), "File already exists " + + file.getAbsolutePath()); Preconditions.checkState(file.createNewFile(), "File could not be created " + file.getAbsolutePath()); return new LogFileV3.Writer(file, logFileID, maxFileSize, encryptionKey, @@ -80,28 +79,29 @@ static LogFile.Writer getWriter(File file, int logFileID, } static LogFile.RandomReader getRandomReader(File file, - @Nullable KeyProvider encryptionKeyProvider, boolean fsyncPerTransaction) + @Nullable KeyProvider encryptionKeyProvider, + boolean fsyncPerTransaction) throws IOException { RandomAccessFile logFile = new RandomAccessFile(file, "r"); try { File metaDataFile = Serialization.getMetaDataFile(file); // either this is a rr for a just created file or // the metadata file exists and as such it's V3 - if(logFile.length() == 0L || metaDataFile.exists()) { + if (logFile.length() == 0L || metaDataFile.exists()) { return new LogFileV3.RandomReader(file, encryptionKeyProvider, - fsyncPerTransaction); + fsyncPerTransaction); } int version = logFile.readInt(); - if(Serialization.VERSION_2 == version) { + if (Serialization.VERSION_2 == version) { return new LogFileV2.RandomReader(file); } throw new IOException("File " + file + " has bad version " + Integer.toHexString(version)); } finally { - if(logFile != null) { + if (logFile != null) { try { logFile.close(); - } catch(IOException e) { + } catch (IOException e) { LOGGER.warn("Unable to close " + file, e); } } @@ -109,7 +109,8 @@ static LogFile.RandomReader getRandomReader(File file, } static LogFile.SequentialReader getSequentialReader(File file, - @Nullable KeyProvider encryptionKeyProvider, boolean fsyncPerTransaction) + @Nullable KeyProvider encryptionKeyProvider, + boolean fsyncPerTransaction) throws IOException { RandomAccessFile logFile = null; try { @@ -134,27 +135,27 @@ static LogFile.SequentialReader getSequentialReader(File file, hasMeta = true; } else { throw new IOException("Renaming of " + tempMetadataFile.getName() - + " to " + metaDataFile.getName() + " failed"); + + " to " + metaDataFile.getName() + " failed"); } } else if (oldMetadataFile.exists()) { if (oldMetadataFile.renameTo(metaDataFile)) { hasMeta = true; } else { throw new IOException("Renaming of " + oldMetadataFile.getName() - + " to " + metaDataFile.getName() + " failed"); + + " to " + metaDataFile.getName() + " failed"); } } if (hasMeta) { // Now the metadata file has been found, delete old or temp files // so it does not interfere with normal operation. - if(oldMetadataFile.exists()) { + if (oldMetadataFile.exists()) { oldMetadataFile.delete(); } - if(tempMetadataFile.exists()) { + if (tempMetadataFile.exists()) { tempMetadataFile.delete(); } - if(metaDataFile.length() == 0L) { - if(file.length() != 0L) { + if (metaDataFile.length() == 0L) { + if (file.length() != 0L) { String msg = String.format("MetaData file %s is empty, but log %s" + " is of size %d", metaDataFile, file, file.length()); throw new IllegalStateException(msg); @@ -163,20 +164,20 @@ static LogFile.SequentialReader getSequentialReader(File file, metaDataFile)); } return new LogFileV3.SequentialReader(file, encryptionKeyProvider, - fsyncPerTransaction); + fsyncPerTransaction); } logFile = new RandomAccessFile(file, "r"); int version = logFile.readInt(); - if(Serialization.VERSION_2 == version) { + if (Serialization.VERSION_2 == version) { return new LogFileV2.SequentialReader(file); } throw new IOException("File " + file + " has bad version " + Integer.toHexString(version)); } finally { - if(logFile != null) { + if (logFile != null) { try { logFile.close(); - } catch(IOException e) { + } catch (IOException e) { LOGGER.warn("Unable to close " + file, e); } } diff --git a/flume-ng-channels/flume-file-channel/src/main/java/org/apache/flume/channel/file/LogFileRetryableIOException.java b/flume-ng-channels/flume-file-channel/src/main/java/org/apache/flume/channel/file/LogFileRetryableIOException.java index 9447652412..b0377ab2fe 100644 --- a/flume-ng-channels/flume-file-channel/src/main/java/org/apache/flume/channel/file/LogFileRetryableIOException.java +++ b/flume-ng-channels/flume-file-channel/src/main/java/org/apache/flume/channel/file/LogFileRetryableIOException.java @@ -22,12 +22,15 @@ public class LogFileRetryableIOException extends IOException { private static final long serialVersionUID = -2747112999806160431L; + public LogFileRetryableIOException() { super(); } + public LogFileRetryableIOException(String msg) { super(msg); } + public LogFileRetryableIOException(String msg, Throwable t) { super(msg, t); } diff --git a/flume-ng-channels/flume-file-channel/src/main/java/org/apache/flume/channel/file/LogFileV2.java b/flume-ng-channels/flume-file-channel/src/main/java/org/apache/flume/channel/file/LogFileV2.java index bb25e95dc5..62b8cb9f7b 100644 --- a/flume-ng-channels/flume-file-channel/src/main/java/org/apache/flume/channel/file/LogFileV2.java +++ b/flume-ng-channels/flume-file-channel/src/main/java/org/apache/flume/channel/file/LogFileV2.java @@ -65,7 +65,7 @@ protected MetaDataWriter(File file, int logFileID) throws IOException { + ", logWriteOrderID: " + getLastCheckpointWriteOrderID()); error = false; } finally { - if(error) { + if (error) { close(); } } @@ -108,6 +108,7 @@ static class Writer extends LogFile.Writer { getFileChannel().force(true); } + @Override int getVersion() { return Serialization.VERSION_2; @@ -115,29 +116,27 @@ int getVersion() { } static class RandomReader extends LogFile.RandomReader { - RandomReader(File file) - throws IOException { + RandomReader(File file) throws IOException { super(file, null, true); } + @Override int getVersion() { return Serialization.VERSION_2; } + @Override - protected TransactionEventRecord doGet(RandomAccessFile fileHandle) - throws IOException { + protected TransactionEventRecord doGet(RandomAccessFile fileHandle) throws IOException { return TransactionEventRecord.fromDataInputV2(fileHandle); } } static class SequentialReader extends LogFile.SequentialReader { - - SequentialReader(File file) - throws EOFException, IOException { + SequentialReader(File file) throws EOFException, IOException { super(file, null); RandomAccessFile fileHandle = getFileHandle(); int version = fileHandle.readInt(); - if(version != getVersion()) { + if (version != getVersion()) { throw new IOException("Version is " + Integer.toHexString(version) + " expected " + Integer.toHexString(getVersion()) + " file: " + file.getCanonicalPath()); @@ -146,10 +145,12 @@ static class SequentialReader extends LogFile.SequentialReader { setLastCheckpointPosition(fileHandle.readLong()); setLastCheckpointWriteOrderID(fileHandle.readLong()); } + @Override public int getVersion() { return Serialization.VERSION_2; } + @Override LogRecord doNext(int offset) throws IOException { TransactionEventRecord event = diff --git a/flume-ng-channels/flume-file-channel/src/main/java/org/apache/flume/channel/file/LogFileV3.java b/flume-ng-channels/flume-file-channel/src/main/java/org/apache/flume/channel/file/LogFileV3.java index 9b0ef93a9e..b4599476e1 100644 --- a/flume-ng-channels/flume-file-channel/src/main/java/org/apache/flume/channel/file/LogFileV3.java +++ b/flume-ng-channels/flume-file-channel/src/main/java/org/apache/flume/channel/file/LogFileV3.java @@ -21,7 +21,6 @@ import com.google.common.base.Preconditions; import com.google.protobuf.ByteString; import com.google.protobuf.GeneratedMessage; -import org.apache.flume.Transaction; import org.apache.flume.annotations.InterfaceAudience; import org.apache.flume.annotations.InterfaceStability; import org.apache.flume.channel.file.encryption.CipherProvider; @@ -58,13 +57,14 @@ private LogFileV3() {} static class MetaDataWriter extends LogFile.MetaDataWriter { private ProtosFactory.LogFileMetaData logFileMetaData; private final File metaDataFile; + protected MetaDataWriter(File logFile, int logFileID) throws IOException { super(logFile, logFileID); metaDataFile = Serialization.getMetaDataFile(logFile); MetaDataReader metaDataReader = new MetaDataReader(logFile, logFileID); logFileMetaData = metaDataReader.read(); int version = logFileMetaData.getVersion(); - if(version != getVersion()) { + if (version != getVersion()) { throw new IOException("Version is " + Integer.toHexString(version) + " expected " + Integer.toHexString(getVersion()) + " file: " + logFile); @@ -90,9 +90,9 @@ void markCheckpoint(long currentPosition, long logWriteOrderID) * would be possible to recover from a backup. */ metaDataBuilder.setBackupCheckpointPosition(logFileMetaData - .getCheckpointPosition()); + .getCheckpointPosition()); metaDataBuilder.setBackupCheckpointWriteOrderID(logFileMetaData - .getCheckpointWriteOrderID()); + .getCheckpointWriteOrderID()); logFileMetaData = metaDataBuilder.build(); writeDelimitedTo(logFileMetaData, metaDataFile); } @@ -102,17 +102,19 @@ static class MetaDataReader { private final File logFile; private final File metaDataFile; private final int logFileID; + protected MetaDataReader(File logFile, int logFileID) throws IOException { this.logFile = logFile; metaDataFile = Serialization.getMetaDataFile(logFile); this.logFileID = logFileID; } + ProtosFactory.LogFileMetaData read() throws IOException { FileInputStream inputStream = new FileInputStream(metaDataFile); try { ProtosFactory.LogFileMetaData metaData = Preconditions.checkNotNull( - ProtosFactory.LogFileMetaData. - parseDelimitedFrom(inputStream), "Metadata cannot be null"); + ProtosFactory.LogFileMetaData.parseDelimitedFrom(inputStream), + "Metadata cannot be null"); if (metaData.getLogFileID() != logFileID) { throw new IOException("The file id of log file: " + logFile + " is different from expected " @@ -123,7 +125,7 @@ ProtosFactory.LogFileMetaData read() throws IOException { } finally { try { inputStream.close(); - } catch(IOException e) { + } catch (IOException e) { LOGGER.warn("Unable to close " + metaDataFile, e); } } @@ -133,13 +135,14 @@ ProtosFactory.LogFileMetaData read() throws IOException { /** * Writes a GeneratedMessage to a temp file, synchronizes it to disk * and then renames the file over file. - * @param msg GeneratedMessage to write to the file + * + * @param msg GeneratedMessage to write to the file * @param file destination file * @throws IOException if a write error occurs or the File.renameTo - * method returns false meaning the file could not be overwritten. + * method returns false meaning the file could not be overwritten. */ public static void writeDelimitedTo(GeneratedMessage msg, File file) - throws IOException { + throws IOException { File tmp = Serialization.getMetaDataTempFile(file); FileOutputStream outputStream = new FileOutputStream(tmp); boolean closed = false; @@ -148,26 +151,26 @@ public static void writeDelimitedTo(GeneratedMessage msg, File file) outputStream.getChannel().force(true); outputStream.close(); closed = true; - if(!tmp.renameTo(file)) { + if (!tmp.renameTo(file)) { //Some platforms don't support moving over an existing file. //So: //log.meta -> log.meta.old //log.meta.tmp -> log.meta //delete log.meta.old File oldFile = Serialization.getOldMetaDataFile(file); - if(!file.renameTo(oldFile)){ + if (!file.renameTo(oldFile)) { throw new IOException("Unable to rename " + file + " to " + oldFile); } - if(!tmp.renameTo(file)) { + if (!tmp.renameTo(file)) { throw new IOException("Unable to rename " + tmp + " over " + file); } oldFile.delete(); } } finally { - if(!closed) { + if (!closed) { try { outputStream.close(); - } catch(IOException e) { + } catch (IOException e) { LOGGER.warn("Unable to close " + tmp, e); } } @@ -177,17 +180,17 @@ public static void writeDelimitedTo(GeneratedMessage msg, File file) static class Writer extends LogFile.Writer { Writer(File file, int logFileID, long maxFileSize, - @Nullable Key encryptionKey, - @Nullable String encryptionKeyAlias, - @Nullable String encryptionCipherProvider, - long usableSpaceRefreshInterval, boolean fsyncPerTransaction, - int fsyncInterval) throws IOException { - super(file, logFileID, maxFileSize, CipherProviderFactory. - getEncrypter(encryptionCipherProvider, encryptionKey), - usableSpaceRefreshInterval, fsyncPerTransaction, fsyncInterval); + @Nullable Key encryptionKey, + @Nullable String encryptionKeyAlias, + @Nullable String encryptionCipherProvider, + long usableSpaceRefreshInterval, boolean fsyncPerTransaction, + int fsyncInterval) throws IOException { + super(file, logFileID, maxFileSize, + CipherProviderFactory.getEncrypter(encryptionCipherProvider, encryptionKey), + usableSpaceRefreshInterval, fsyncPerTransaction, fsyncInterval); ProtosFactory.LogFileMetaData.Builder metaDataBuilder = ProtosFactory.LogFileMetaData.newBuilder(); - if(encryptionKey != null) { + if (encryptionKey != null) { Preconditions.checkNotNull(encryptionKeyAlias, "encryptionKeyAlias"); Preconditions.checkNotNull(encryptionCipherProvider, "encryptionCipherProvider"); @@ -208,6 +211,7 @@ static class Writer extends LogFile.Writer { File metaDataFile = Serialization.getMetaDataFile(file); writeDelimitedTo(metaDataBuilder.build(), metaDataFile); } + @Override int getVersion() { return Serialization.VERSION_3; @@ -221,28 +225,29 @@ static class RandomReader extends LogFile.RandomReader { private volatile String cipherProvider; private volatile byte[] parameters; private BlockingQueue decryptors = - new LinkedBlockingDeque(); + new LinkedBlockingDeque(); RandomReader(File file, @Nullable KeyProvider encryptionKeyProvider, - boolean fsyncPerTransaction) throws IOException { + boolean fsyncPerTransaction) throws IOException { super(file, encryptionKeyProvider, fsyncPerTransaction); } + private void initialize() throws IOException { File metaDataFile = Serialization.getMetaDataFile(getFile()); FileInputStream inputStream = new FileInputStream(metaDataFile); try { - ProtosFactory.LogFileMetaData metaData = - Preconditions.checkNotNull(ProtosFactory.LogFileMetaData. - parseDelimitedFrom(inputStream), "MetaData cannot be null"); + ProtosFactory.LogFileMetaData metaData = Preconditions.checkNotNull( + ProtosFactory.LogFileMetaData.parseDelimitedFrom(inputStream), + "MetaData cannot be null"); int version = metaData.getVersion(); - if(version != getVersion()) { + if (version != getVersion()) { throw new IOException("Version is " + Integer.toHexString(version) + " expected " + Integer.toHexString(getVersion()) + " file: " + getFile().getCanonicalPath()); } encryptionEnabled = false; - if(metaData.hasEncryption()) { - if(getKeyProvider() == null) { + if (metaData.hasEncryption()) { + if (getKeyProvider() == null) { throw new IllegalStateException("Data file is encrypted but no " + " provider was specified"); } @@ -255,23 +260,26 @@ private void initialize() throws IOException { } finally { try { inputStream.close(); - } catch(IOException e) { + } catch (IOException e) { LOGGER.warn("Unable to close " + metaDataFile, e); } } } + private CipherProvider.Decryptor getDecryptor() { CipherProvider.Decryptor decryptor = decryptors.poll(); - if(decryptor == null) { + if (decryptor == null) { decryptor = CipherProviderFactory.getDecrypter(cipherProvider, key, parameters); } return decryptor; } + @Override int getVersion() { return Serialization.VERSION_3; } + @Override protected TransactionEventRecord doGet(RandomAccessFile fileHandle) throws IOException, CorruptEventException { @@ -279,7 +287,7 @@ protected TransactionEventRecord doGet(RandomAccessFile fileHandle) // empty. As such we wait to initialize until there is some // data before we we initialize synchronized (this) { - if(!initialized) { + if (!initialized) { initialized = true; initialize(); } @@ -288,18 +296,17 @@ protected TransactionEventRecord doGet(RandomAccessFile fileHandle) CipherProvider.Decryptor decryptor = null; try { byte[] buffer = readDelimitedBuffer(fileHandle); - if(encryptionEnabled) { + if (encryptionEnabled) { decryptor = getDecryptor(); buffer = decryptor.decrypt(buffer); } - TransactionEventRecord event = TransactionEventRecord. - fromByteArray(buffer); + TransactionEventRecord event = TransactionEventRecord.fromByteArray(buffer); success = true; return event; - } catch(DecryptionFailureException ex) { + } catch (DecryptionFailureException ex) { throw new CorruptEventException("Error decrypting event", ex); } finally { - if(success && encryptionEnabled && decryptor != null) { + if (success && encryptionEnabled && decryptor != null) { decryptors.offer(decryptor); } } @@ -309,9 +316,10 @@ protected TransactionEventRecord doGet(RandomAccessFile fileHandle) public static class SequentialReader extends LogFile.SequentialReader { private CipherProvider.Decryptor decryptor; private final boolean fsyncPerTransaction; + public SequentialReader(File file, @Nullable KeyProvider - encryptionKeyProvider, boolean fsyncPerTransaction) throws EOFException, - IOException { + encryptionKeyProvider, boolean fsyncPerTransaction) throws EOFException, + IOException { super(file, encryptionKeyProvider); this.fsyncPerTransaction = fsyncPerTransaction; File metaDataFile = Serialization.getMetaDataFile(file); @@ -321,32 +329,31 @@ public SequentialReader(File file, @Nullable KeyProvider ProtosFactory.LogFileMetaData.parseDelimitedFrom(inputStream), "MetaData cannot be null"); int version = metaData.getVersion(); - if(version != getVersion()) { + if (version != getVersion()) { throw new IOException("Version is " + Integer.toHexString(version) + " expected " + Integer.toHexString(getVersion()) + " file: " + file.getCanonicalPath()); } - if(metaData.hasEncryption()) { - if(getKeyProvider() == null) { + if (metaData.hasEncryption()) { + if (getKeyProvider() == null) { throw new IllegalStateException("Data file is encrypted but no " + " provider was specified"); } ProtosFactory.LogFileEncryption encryption = metaData.getEncryption(); Key key = getKeyProvider().getKey(encryption.getKeyAlias()); - decryptor = CipherProviderFactory. - getDecrypter(encryption.getCipherProvider(), key, - encryption.getParameters().toByteArray()); + decryptor = CipherProviderFactory.getDecrypter( + encryption.getCipherProvider(), key, encryption.getParameters().toByteArray()); } setLogFileID(metaData.getLogFileID()); setLastCheckpointPosition(metaData.getCheckpointPosition()); setLastCheckpointWriteOrderID(metaData.getCheckpointWriteOrderID()); setPreviousCheckpointPosition(metaData.getBackupCheckpointPosition()); setPreviousCheckpointWriteOrderID( - metaData.getBackupCheckpointWriteOrderID()); + metaData.getBackupCheckpointWriteOrderID()); } finally { try { inputStream.close(); - } catch(IOException e) { + } catch (IOException e) { LOGGER.warn("Unable to close " + metaDataFile, e); } } @@ -359,7 +366,7 @@ public int getVersion() { @Override LogRecord doNext(int offset) throws IOException, CorruptEventException, - DecryptionFailureException { + DecryptionFailureException { byte[] buffer = null; TransactionEventRecord event = null; try { @@ -370,7 +377,7 @@ LogRecord doNext(int offset) throws IOException, CorruptEventException, event = TransactionEventRecord.fromByteArray(buffer); } catch (CorruptEventException ex) { LOGGER.warn("Corrupt file found. File id: log-" + this.getLogFileID(), - ex); + ex); // Return null so that replay handler thinks all events in this file // have been taken. if (!fsyncPerTransaction) { @@ -380,7 +387,7 @@ LogRecord doNext(int offset) throws IOException, CorruptEventException, } catch (DecryptionFailureException ex) { if (!fsyncPerTransaction) { LOGGER.warn("Could not decrypt even read from channel. Skipping " + - "event.", ex); + "event.", ex); return null; } throw ex; diff --git a/flume-ng-channels/flume-file-channel/src/main/java/org/apache/flume/channel/file/LogRecord.java b/flume-ng-channels/flume-file-channel/src/main/java/org/apache/flume/channel/file/LogRecord.java index 19ad0d696f..5a75627d84 100644 --- a/flume-ng-channels/flume-file-channel/src/main/java/org/apache/flume/channel/file/LogRecord.java +++ b/flume-ng-channels/flume-file-channel/src/main/java/org/apache/flume/channel/file/LogRecord.java @@ -20,15 +20,17 @@ import java.util.Arrays; - public class LogRecord implements Comparable { - private int fileID, offset; + private int fileID; + private int offset; private TransactionEventRecord event; + public LogRecord(int fileID, int offset, TransactionEventRecord event) { this.fileID = fileID; this.offset = offset; this.event = event; } + public int getFileID() { return fileID; } @@ -41,20 +43,16 @@ public TransactionEventRecord getEvent() { @Override public int compareTo(LogRecord o) { - int result = new Long(event.getLogWriteOrderID()) - .compareTo(o.getEvent().getLogWriteOrderID()); - if(result == 0) { + int result = new Long(event.getLogWriteOrderID()).compareTo(o.getEvent().getLogWriteOrderID()); + if (result == 0) { // oops we have hit a flume-1.2 bug. let's try and use the txid // to replay the events - result = new Long(event.getTransactionID()) - .compareTo(o.getEvent().getTransactionID()); - if(result == 0) { + result = new Long(event.getTransactionID()).compareTo(o.getEvent().getTransactionID()); + if (result == 0) { // events are within the same transaction. Basically we want commit // and rollback to come after take and put - Integer thisIndex = Arrays.binarySearch(replaySortOrder, - event.getRecordType()); - Integer thatIndex = Arrays.binarySearch(replaySortOrder, - o.getEvent().getRecordType()); + Integer thisIndex = Arrays.binarySearch(replaySortOrder, event.getRecordType()); + Integer thatIndex = Arrays.binarySearch(replaySortOrder, o.getEvent().getRecordType()); return thisIndex.compareTo(thatIndex); } } diff --git a/flume-ng-channels/flume-file-channel/src/main/java/org/apache/flume/channel/file/LogUtils.java b/flume-ng-channels/flume-file-channel/src/main/java/org/apache/flume/channel/file/LogUtils.java index d1498c2c36..48177d0809 100644 --- a/flume-ng-channels/flume-file-channel/src/main/java/org/apache/flume/channel/file/LogUtils.java +++ b/flume-ng-channels/flume-file-channel/src/main/java/org/apache/flume/channel/file/LogUtils.java @@ -64,7 +64,7 @@ static int getIDForFile(File file) { static List getLogs(File logDir) { List result = Lists.newArrayList(); File[] files = logDir.listFiles(); - if(files == null) { + if (files == null) { String msg = logDir + ".listFiles() returned null: "; msg += "File = " + logDir.isFile() + ", "; msg += "Exists = " + logDir.exists() + ", "; diff --git a/flume-ng-channels/flume-file-channel/src/main/java/org/apache/flume/channel/file/Pair.java b/flume-ng-channels/flume-file-channel/src/main/java/org/apache/flume/channel/file/Pair.java index dfcdd73849..b74ff7b9ae 100644 --- a/flume-ng-channels/flume-file-channel/src/main/java/org/apache/flume/channel/file/Pair.java +++ b/flume-ng-channels/flume-file-channel/src/main/java/org/apache/flume/channel/file/Pair.java @@ -19,19 +19,21 @@ package org.apache.flume.channel.file; class Pair { - private final L left; private final R right; + Pair(L l, R r) { left = l; right = r; } + L getLeft() { return left; } R getRight() { return right; } + static Pair of(L left, R right) { return new Pair(left, right); } diff --git a/flume-ng-channels/flume-file-channel/src/main/java/org/apache/flume/channel/file/Put.java b/flume-ng-channels/flume-file-channel/src/main/java/org/apache/flume/channel/file/Put.java index f08f0244ea..0a70a2404a 100644 --- a/flume-ng-channels/flume-file-channel/src/main/java/org/apache/flume/channel/file/Put.java +++ b/flume-ng-channels/flume-file-channel/src/main/java/org/apache/flume/channel/file/Put.java @@ -78,8 +78,8 @@ void writeProtos(OutputStream out) throws IOException { Map headers = event.getHeaders(); ProtosFactory.FlumeEventHeader.Builder headerBuilder = ProtosFactory.FlumeEventHeader.newBuilder(); - if(headers != null) { - for(String key : headers.keySet()) { + if (headers != null) { + for (String key : headers.keySet()) { String value = headers.get(key); headerBuilder.clear(); eventBuilder.addHeaders(headerBuilder.setKey(key) @@ -93,13 +93,12 @@ void writeProtos(OutputStream out) throws IOException { putBuilder.build().writeDelimitedTo(out); } @Override - void readProtos(InputStream in) throws IOException, - CorruptEventException { - ProtosFactory.Put put = Preconditions.checkNotNull(ProtosFactory. - Put.parseDelimitedFrom(in), "Put cannot be null"); + void readProtos(InputStream in) throws IOException, CorruptEventException { + ProtosFactory.Put put = Preconditions.checkNotNull( + ProtosFactory.Put.parseDelimitedFrom(in), "Put cannot be null"); Map headers = Maps.newHashMap(); ProtosFactory.FlumeEvent protosEvent = put.getEvent(); - for(ProtosFactory.FlumeEventHeader header : protosEvent.getHeadersList()) { + for (ProtosFactory.FlumeEventHeader header : protosEvent.getHeadersList()) { headers.put(header.getKey(), header.getValue()); } byte[] eventBody = protosEvent.getBody().toByteArray(); diff --git a/flume-ng-channels/flume-file-channel/src/main/java/org/apache/flume/channel/file/ReplayHandler.java b/flume-ng-channels/flume-file-channel/src/main/java/org/apache/flume/channel/file/ReplayHandler.java index a5595034b3..662fd4246d 100644 --- a/flume-ng-channels/flume-file-channel/src/main/java/org/apache/flume/channel/file/ReplayHandler.java +++ b/flume-ng-channels/flume-file-channel/src/main/java/org/apache/flume/channel/file/ReplayHandler.java @@ -99,8 +99,8 @@ public int getRollbackCount() { } ReplayHandler(FlumeEventQueue queue, - @Nullable KeyProvider encryptionKeyProvider, - boolean fsyncPerTransaction) { + @Nullable KeyProvider encryptionKeyProvider, + boolean fsyncPerTransaction) { this.queue = queue; this.lastCheckpoint = queue.getLogWriteOrderID(); pendingTakes = Lists.newArrayList(); @@ -109,6 +109,7 @@ public int getRollbackCount() { this.encryptionKeyProvider = encryptionKeyProvider; this.fsyncPerTransaction = fsyncPerTransaction; } + /** * Replay logic from Flume1.2 which can be activated if the v2 logic * is failing on ol logs for some reason. @@ -165,9 +166,8 @@ void replayLogv1(List logs) throws Exception { commitCount++; @SuppressWarnings("unchecked") Collection pointers = - (Collection) transactionMap.remove(trans); - if (((Commit) record).getType() - == TransactionEventRecord.Type.TAKE.get()) { + (Collection) transactionMap.remove(trans); + if (((Commit) record).getType() == TransactionEventRecord.Type.TAKE.get()) { if (inflightTakes.containsKey(trans)) { if (pointers == null) { pointers = Sets.newHashSet(); @@ -185,8 +185,8 @@ void replayLogv1(List logs) throws Exception { count += pointers.size(); } } else { - Preconditions.checkArgument(false, "Unknown record type: " - + Integer.toHexString(type)); + Preconditions.checkArgument(false, + "Unknown record type: " + Integer.toHexString(type)); } } else { @@ -196,8 +196,8 @@ void replayLogv1(List logs) throws Exception { LOG.info("Replayed " + count + " from " + log); if (LOG.isDebugEnabled()) { LOG.debug("read: " + readCount + ", put: " + putCount + ", take: " - + takeCount + ", rollback: " + rollbackCount + ", commit: " - + commitCount + ", skipp: " + skipCount); + + takeCount + ", rollback: " + rollbackCount + ", commit: " + + commitCount + ", skipp: " + skipCount); } } catch (EOFException e) { LOG.warn("Hit EOF on " + log); @@ -262,21 +262,20 @@ void replayLog(List logs) throws Exception { LOG.info("Replaying " + log); try { LogFile.SequentialReader reader = - LogFileFactory.getSequentialReader(log, encryptionKeyProvider, - fsyncPerTransaction); + LogFileFactory.getSequentialReader(log, encryptionKeyProvider, fsyncPerTransaction); reader.skipToLastCheckpointPosition(queue.getLogWriteOrderID()); Preconditions.checkState(!readers.containsKey(reader.getLogFileID()), "Readers " + readers + " already contains " + reader.getLogFileID()); readers.put(reader.getLogFileID(), reader); LogRecord logRecord = reader.next(); - if(logRecord == null) { + if (logRecord == null) { readers.remove(reader.getLogFileID()); reader.close(); } else { logRecordBuffer.add(logRecord); } - } catch(EOFException e) { + } catch (EOFException e) { LOG.warn("Ignoring " + log + " due to EOF", e); } } @@ -294,7 +293,7 @@ void replayLog(List logs) throws Exception { writeOrderIDSeed = Math.max(writeOrderIDSeed, record.getLogWriteOrderID()); readCount++; - if(readCount % 10000 == 0 && readCount > 0) { + if (readCount % 10000 == 0 && readCount > 0) { LOG.info("read: " + readCount + ", put: " + putCount + ", take: " + takeCount + ", rollback: " + rollbackCount + ", commit: " + commitCount + ", skip: " + skipCount + ", eventCount:" + count); @@ -316,11 +315,11 @@ void replayLog(List logs) throws Exception { commitCount++; @SuppressWarnings("unchecked") Collection pointers = - (Collection) transactionMap.remove(trans); + (Collection) transactionMap.remove(trans); if (((Commit) record).getType() == TransactionEventRecord.Type.TAKE.get()) { if (inflightTakes.containsKey(trans)) { - if(pointers == null){ + if (pointers == null) { pointers = Sets.newHashSet(); } Set takes = inflightTakes.removeAll(trans); @@ -350,8 +349,8 @@ void replayLog(List logs) throws Exception { } finally { TransactionIDOracle.setSeed(transactionIDSeed); WriteOrderOracle.setSeed(writeOrderIDSeed); - for(LogFile.SequentialReader reader : readers.values()) { - if(reader != null) { + for (LogFile.SequentialReader reader : readers.values()) { + if (reader != null) { reader.close(); } } @@ -378,11 +377,11 @@ void replayLog(List logs) throws Exception { } private LogRecord next() throws IOException, CorruptEventException { LogRecord resultLogRecord = logRecordBuffer.poll(); - if(resultLogRecord != null) { + if (resultLogRecord != null) { // there is more log records to read LogFile.SequentialReader reader = readers.get(resultLogRecord.getFileID()); LogRecord nextLogRecord; - if((nextLogRecord = reader.next()) != null) { + if ((nextLogRecord = reader.next()) != null) { logRecordBuffer.add(nextLogRecord); } } @@ -391,7 +390,7 @@ private LogRecord next() throws IOException, CorruptEventException { private void processCommit(short type, Collection pointers) { if (type == TransactionEventRecord.Type.PUT.get()) { for (FlumeEventPointer pointer : pointers) { - if(!queue.addTail(pointer)) { + if (!queue.addTail(pointer)) { throw new IllegalStateException("Unable to add " + pointer + ". Queue depth = " + queue.getSize() + ", Capacity = " + queue.getCapacity()); diff --git a/flume-ng-channels/flume-file-channel/src/main/java/org/apache/flume/channel/file/Rollback.java b/flume-ng-channels/flume-file-channel/src/main/java/org/apache/flume/channel/file/Rollback.java index 335ad0bd4a..2fca755aca 100644 --- a/flume-ng-channels/flume-file-channel/src/main/java/org/apache/flume/channel/file/Rollback.java +++ b/flume-ng-channels/flume-file-channel/src/main/java/org/apache/flume/channel/file/Rollback.java @@ -35,6 +35,7 @@ class Rollback extends TransactionEventRecord { Rollback(Long transactionID, Long logWriteOrderID) { super(transactionID, logWriteOrderID); } + @Override public void readFields(DataInput in) throws IOException { super.readFields(in); @@ -44,22 +45,26 @@ public void readFields(DataInput in) throws IOException { public void write(DataOutput out) throws IOException { super.write(out); } + @Override void writeProtos(OutputStream out) throws IOException { ProtosFactory.Rollback.Builder rollbackBuilder = ProtosFactory.Rollback.newBuilder(); rollbackBuilder.build().writeDelimitedTo(out); } + @Override void readProtos(InputStream in) throws IOException { @SuppressWarnings("unused") - ProtosFactory.Rollback rollback = Preconditions.checkNotNull(ProtosFactory. - Rollback.parseDelimitedFrom(in), "Rollback cannot be null"); + ProtosFactory.Rollback rollback = Preconditions.checkNotNull( + ProtosFactory.Rollback.parseDelimitedFrom(in), "Rollback cannot be null"); } + @Override short getRecordType() { return Type.ROLLBACK.get(); } + @Override public String toString() { StringBuilder builder = new StringBuilder(); diff --git a/flume-ng-channels/flume-file-channel/src/main/java/org/apache/flume/channel/file/Serialization.java b/flume-ng-channels/flume-file-channel/src/main/java/org/apache/flume/channel/file/Serialization.java index a6eda75fa6..19303cc59c 100644 --- a/flume-ng-channels/flume-file-channel/src/main/java/org/apache/flume/channel/file/Serialization.java +++ b/flume-ng-channels/flume-file-channel/src/main/java/org/apache/flume/channel/file/Serialization.java @@ -46,14 +46,12 @@ private Serialization() {} static final long SIZE_OF_INT = 4; static final int SIZE_OF_LONG = 8; - static final int VERSION_2 = 2; static final int VERSION_3 = 3; public static final String METADATA_FILENAME = ".meta"; public static final String METADATA_TMP_FILENAME = ".tmp"; - public static final String OLD_METADATA_FILENAME = METADATA_FILENAME + - ".old"; + public static final String OLD_METADATA_FILENAME = METADATA_FILENAME + ".old"; // 64 K buffer to copy and compress files. private static final int FILE_BUFFER_SIZE = 64 * 1024; @@ -63,12 +61,11 @@ private Serialization() {} static File getMetaDataTempFile(File metaDataFile) { String metaDataFileName = metaDataFile.getName() + METADATA_TMP_FILENAME; return new File(metaDataFile.getParentFile(), metaDataFileName); - } + static File getMetaDataFile(File file) { String metaDataFileName = file.getName() + METADATA_FILENAME; return new File(file.getParentFile(), metaDataFileName); - } // Support platforms that cannot do atomic renames - FLUME-1699 @@ -79,19 +76,20 @@ static File getOldMetaDataFile(File file) { /** * Deletes all files in given directory. + * * @param checkpointDir - The directory whose files are to be deleted - * @param excludes - Names of files which should not be deleted from this - * directory. + * @param excludes - Names of files which should not be deleted from this + * directory. * @return - true if all files were successfully deleted, false otherwise. */ static boolean deleteAllFiles(File checkpointDir, - @Nullable Set excludes) { + @Nullable Set excludes) { if (!checkpointDir.isDirectory()) { return false; } File[] files = checkpointDir.listFiles(); - if(files == null) { + if (files == null) { return false; } StringBuilder builder; @@ -100,13 +98,13 @@ static boolean deleteAllFiles(File checkpointDir, } else { builder = new StringBuilder("Deleted the following files: "); } - if(excludes == null) { + if (excludes == null) { excludes = Collections.emptySet(); } for (File file : files) { - if(excludes.contains(file.getName())) { + if (excludes.contains(file.getName())) { LOG.info("Skipping " + file.getName() + " because it is in excludes " + - "set"); + "set"); continue; } if (!FileUtils.deleteQuietly(file)) { @@ -125,18 +123,19 @@ static boolean deleteAllFiles(File checkpointDir, /** * Copy a file using a 64K size buffer. This method will copy the file and * then fsync to disk + * * @param from File to copy - this file should exist - * @param to Destination file - this file should not exist + * @param to Destination file - this file should not exist * @return true if the copy was successful */ public static boolean copyFile(File from, File to) throws IOException { Preconditions.checkNotNull(from, "Source file is null, file copy failed."); Preconditions.checkNotNull(to, "Destination file is null, " + - "file copy failed."); + "file copy failed."); Preconditions.checkState(from.exists(), "Source file: " + from.toString() + - " does not exist."); + " does not exist."); Preconditions.checkState(!to.exists(), "Destination file: " - + to.toString() + " unexpectedly exists."); + + to.toString() + " unexpectedly exists."); BufferedInputStream in = null; RandomAccessFile out = null; //use a RandomAccessFile for easy fsync @@ -145,7 +144,7 @@ public static boolean copyFile(File from, File to) throws IOException { out = new RandomAccessFile(to, "rw"); byte[] buf = new byte[FILE_BUFFER_SIZE]; int total = 0; - while(true) { + while (true) { int read = in.read(buf); if (read == -1) { break; @@ -155,11 +154,11 @@ public static boolean copyFile(File from, File to) throws IOException { } out.getFD().sync(); Preconditions.checkState(total == from.length(), - "The size of the origin file and destination file are not equal."); + "The size of the origin file and destination file are not equal."); return true; } catch (Exception ex) { LOG.error("Error while attempting to copy " + from.toString() + " to " - + to.toString() + ".", ex); + + to.toString() + ".", ex); Throwables.propagate(ex); } finally { Throwable th = null; @@ -185,26 +184,26 @@ public static boolean copyFile(File from, File to) throws IOException { } // Should never reach here. throw new IOException("Copying file: " + from.toString() + " to: " + to - .toString() + " may have failed."); + .toString() + " may have failed."); } /** * Compress file using Snappy + * * @param uncompressed File to compress - this file should exist - * @param compressed Compressed file - this file should not exist + * @param compressed Compressed file - this file should not exist * @return true if compression was successful */ public static boolean compressFile(File uncompressed, File compressed) - throws IOException { + throws IOException { Preconditions.checkNotNull(uncompressed, - "Source file is null, compression failed."); + "Source file is null, compression failed."); Preconditions.checkNotNull(compressed, - "Destination file is null, compression failed."); + "Destination file is null, compression failed."); Preconditions.checkState(uncompressed.exists(), "Source file: " + - uncompressed.toString() + " does not exist."); + uncompressed.toString() + " does not exist."); Preconditions.checkState(!compressed.exists(), - "Compressed file: " + compressed.toString() + " unexpectedly " + - "exists."); + "Compressed file: " + compressed.toString() + " unexpectedly " + "exists."); BufferedInputStream in = null; FileOutputStream out = null; @@ -215,7 +214,7 @@ public static boolean compressFile(File uncompressed, File compressed) snappyOut = new SnappyOutputStream(out); byte[] buf = new byte[FILE_BUFFER_SIZE]; - while(true) { + while (true) { int read = in.read(buf); if (read == -1) { break; @@ -226,8 +225,7 @@ public static boolean compressFile(File uncompressed, File compressed) return true; } catch (Exception ex) { LOG.error("Error while attempting to compress " + - uncompressed.toString() + " to " + compressed.toString() - + ".", ex); + uncompressed.toString() + " to " + compressed.toString() + ".", ex); Throwables.propagate(ex); } finally { Throwable th = null; @@ -253,26 +251,24 @@ public static boolean compressFile(File uncompressed, File compressed) } // Should never reach here. throw new IOException("Copying file: " + uncompressed.toString() - + " to: " + compressed.toString() + " may have failed."); + + " to: " + compressed.toString() + " may have failed."); } /** * Decompress file using Snappy - * @param compressed File to compress - this file should exist + * + * @param compressed File to compress - this file should exist * @param decompressed Compressed file - this file should not exist * @return true if decompression was successful */ - public static boolean decompressFile(File compressed, File decompressed) - throws IOException { - Preconditions.checkNotNull(compressed, - "Source file is null, decompression failed."); + public static boolean decompressFile(File compressed, File decompressed) throws IOException { + Preconditions.checkNotNull(compressed, "Source file is null, decompression failed."); Preconditions.checkNotNull(decompressed, "Destination file is " + - "null, decompression failed."); + "null, decompression failed."); Preconditions.checkState(compressed.exists(), "Source file: " + - compressed.toString() + " does not exist."); + compressed.toString() + " does not exist."); Preconditions.checkState(!decompressed.exists(), - "Decompressed file: " + decompressed.toString() + - " unexpectedly exists."); + "Decompressed file: " + decompressed.toString() + " unexpectedly exists."); BufferedInputStream in = null; SnappyInputStream snappyIn = null; @@ -283,7 +279,7 @@ public static boolean decompressFile(File compressed, File decompressed) out = new FileOutputStream(decompressed); byte[] buf = new byte[FILE_BUFFER_SIZE]; - while(true) { + while (true) { int read = snappyIn.read(buf); if (read == -1) { break; @@ -294,8 +290,8 @@ public static boolean decompressFile(File compressed, File decompressed) return true; } catch (Exception ex) { LOG.error("Error while attempting to compress " + - compressed.toString() + " to " + decompressed.toString() + - ".", ex); + compressed.toString() + " to " + decompressed.toString() + + ".", ex); Throwables.propagate(ex); } finally { Throwable th = null; @@ -321,7 +317,7 @@ public static boolean decompressFile(File compressed, File decompressed) } // Should never reach here. throw new IOException("Decompressing file: " + - compressed.toString() + " to: " + decompressed.toString() + - " may have failed."); + compressed.toString() + " to: " + decompressed.toString() + + " may have failed."); } } diff --git a/flume-ng-channels/flume-file-channel/src/main/java/org/apache/flume/channel/file/Take.java b/flume-ng-channels/flume-file-channel/src/main/java/org/apache/flume/channel/file/Take.java index 143143a546..ee7fcc80c1 100644 --- a/flume-ng-channels/flume-file-channel/src/main/java/org/apache/flume/channel/file/Take.java +++ b/flume-ng-channels/flume-file-channel/src/main/java/org/apache/flume/channel/file/Take.java @@ -34,14 +34,17 @@ class Take extends TransactionEventRecord { private int offset; private int fileID; + Take(Long transactionID, Long logWriteOrderID) { super(transactionID, logWriteOrderID); } + Take(Long transactionID, Long logWriteOrderID, int offset, int fileID) { this(transactionID, logWriteOrderID); this.offset = offset; this.fileID = fileID; } + int getOffset() { return offset; } @@ -70,17 +73,20 @@ void writeProtos(OutputStream out) throws IOException { takeBuilder.setOffset(offset); takeBuilder.build().writeDelimitedTo(out); } + @Override void readProtos(InputStream in) throws IOException { - ProtosFactory.Take take = Preconditions.checkNotNull(ProtosFactory. - Take.parseDelimitedFrom(in), "Take cannot be null"); + ProtosFactory.Take take = Preconditions.checkNotNull( + ProtosFactory.Take.parseDelimitedFrom(in), "Take cannot be null"); fileID = take.getFileID(); offset = take.getOffset(); } + @Override short getRecordType() { return Type.TAKE.get(); } + @Override public String toString() { StringBuilder builder = new StringBuilder(); diff --git a/flume-ng-channels/flume-file-channel/src/main/java/org/apache/flume/channel/file/TransactionEventRecord.java b/flume-ng-channels/flume-file-channel/src/main/java/org/apache/flume/channel/file/TransactionEventRecord.java index 1eb3f4fa00..0f7c3c80ac 100644 --- a/flume-ng-channels/flume-file-channel/src/main/java/org/apache/flume/channel/file/TransactionEventRecord.java +++ b/flume-ng-channels/flume-file-channel/src/main/java/org/apache/flume/channel/file/TransactionEventRecord.java @@ -91,13 +91,16 @@ static enum Type { COMMIT((short)4); private short id; + Type(short id) { this.id = id; } + public short get() { return id; } } + private static final ImmutableMap> TYPES; static { @@ -131,11 +134,11 @@ static ByteBuffer toByteBufferV2(TransactionEventRecord record) { dataOutput.flush(); // TODO toByteArray does an unneeded copy return ByteBuffer.wrap(byteOutput.toByteArray()); - } catch(IOException e) { + } catch (IOException e) { // near impossible throw Throwables.propagate(e); } finally { - if(dataOutput != null) { + if (dataOutput != null) { try { dataOutput.close(); } catch (IOException e) { @@ -149,7 +152,7 @@ static ByteBuffer toByteBufferV2(TransactionEventRecord record) { static TransactionEventRecord fromDataInputV2(DataInput in) throws IOException { int header = in.readInt(); - if(header != MAGIC_HEADER) { + if (header != MAGIC_HEADER) { throw new IOException("Header " + Integer.toHexString(header) + " is not the required value: " + Integer.toHexString(MAGIC_HEADER)); } @@ -176,10 +179,10 @@ static ByteBuffer toByteBuffer(TransactionEventRecord record) { ProtosFactory.TransactionEventFooter.newBuilder().build(); footer.writeDelimitedTo(byteOutput); return ByteBuffer.wrap(byteOutput.toByteArray()); - } catch(IOException e) { + } catch (IOException e) { throw Throwables.propagate(e); } finally { - if(byteOutput != null) { + if (byteOutput != null) { try { byteOutput.close(); } catch (IOException e) { @@ -194,23 +197,19 @@ static TransactionEventRecord fromByteArray(byte[] buffer) throws IOException, CorruptEventException { ByteArrayInputStream in = new ByteArrayInputStream(buffer); try { - ProtosFactory.TransactionEventHeader header = Preconditions. - checkNotNull(ProtosFactory.TransactionEventHeader. - parseDelimitedFrom(in), "Header cannot be null"); + ProtosFactory.TransactionEventHeader header = Preconditions.checkNotNull( + ProtosFactory.TransactionEventHeader.parseDelimitedFrom(in), "Header cannot be null"); short type = (short)header.getType(); long transactionID = header.getTransactionID(); long writeOrderID = header.getWriteOrderID(); - TransactionEventRecord transactionEvent = - newRecordForType(type, transactionID, writeOrderID); + TransactionEventRecord transactionEvent = newRecordForType(type, transactionID, writeOrderID); transactionEvent.readProtos(in); @SuppressWarnings("unused") ProtosFactory.TransactionEventFooter footer = Preconditions.checkNotNull( - ProtosFactory.TransactionEventFooter. - parseDelimitedFrom(in), "Footer cannot be null"); + ProtosFactory.TransactionEventFooter.parseDelimitedFrom(in), "Footer cannot be null"); return transactionEvent; } catch (InvalidProtocolBufferException ex) { - throw new CorruptEventException( - "Could not parse event from data file.", ex); + throw new CorruptEventException("Could not parse event from data file.", ex); } finally { try { in.close(); diff --git a/flume-ng-channels/flume-file-channel/src/main/java/org/apache/flume/channel/file/TransactionIDOracle.java b/flume-ng-channels/flume-file-channel/src/main/java/org/apache/flume/channel/file/TransactionIDOracle.java index a9f6be6517..12e5c7d508 100644 --- a/flume-ng-channels/flume-file-channel/src/main/java/org/apache/flume/channel/file/TransactionIDOracle.java +++ b/flume-ng-channels/flume-file-channel/src/main/java/org/apache/flume/channel/file/TransactionIDOracle.java @@ -23,15 +23,17 @@ public final class TransactionIDOracle { private TransactionIDOracle() {} + private static final AtomicLong TRANSACTION_ID = new AtomicLong(System.currentTimeMillis()); public static void setSeed(long highest) { long previous; - while(highest > (previous = TRANSACTION_ID.get())) { + while (highest > (previous = TRANSACTION_ID.get())) { TRANSACTION_ID.compareAndSet(previous, highest); } } + public static long next() { return TRANSACTION_ID.incrementAndGet(); } diff --git a/flume-ng-channels/flume-file-channel/src/main/java/org/apache/flume/channel/file/WritableUtils.java b/flume-ng-channels/flume-file-channel/src/main/java/org/apache/flume/channel/file/WritableUtils.java index 69072dba4b..2ebd42d898 100644 --- a/flume-ng-channels/flume-file-channel/src/main/java/org/apache/flume/channel/file/WritableUtils.java +++ b/flume-ng-channels/flume-file-channel/src/main/java/org/apache/flume/channel/file/WritableUtils.java @@ -75,7 +75,7 @@ public static void writeVLong(DataOutput stream, long i) throws IOException { long tmp = i; while (tmp != 0) { tmp = tmp >> 8; - len--; + len--; } stream.writeByte((byte)len); @@ -92,8 +92,8 @@ public static void writeVLong(DataOutput stream, long i) throws IOException { /** * Reads a zero-compressed encoded long from input stream and returns it. * @param stream Binary input stream - * @throws java.io.IOException * @return deserialized long from stream. + * @throws java.io.IOException */ public static long readVLong(DataInput stream) throws IOException { byte firstByte = stream.readByte(); @@ -102,7 +102,7 @@ public static long readVLong(DataInput stream) throws IOException { return firstByte; } long i = 0; - for (int idx = 0; idx < len-1; idx++) { + for (int idx = 0; idx < len - 1; idx++) { byte b = stream.readByte(); i = i << 8; i = i | (b & 0xFF); @@ -113,8 +113,8 @@ public static long readVLong(DataInput stream) throws IOException { /** * Reads a zero-compressed encoded integer from input stream and returns it. * @param stream Binary input stream - * @throws java.io.IOException * @return deserialized integer from stream. + * @throws java.io.IOException */ public static int readVInt(DataInput stream) throws IOException { long n = readVLong(stream); diff --git a/flume-ng-channels/flume-file-channel/src/main/java/org/apache/flume/channel/file/WriteOrderOracle.java b/flume-ng-channels/flume-file-channel/src/main/java/org/apache/flume/channel/file/WriteOrderOracle.java index dbf1c1e84a..b26cbb4f65 100644 --- a/flume-ng-channels/flume-file-channel/src/main/java/org/apache/flume/channel/file/WriteOrderOracle.java +++ b/flume-ng-channels/flume-file-channel/src/main/java/org/apache/flume/channel/file/WriteOrderOracle.java @@ -23,15 +23,17 @@ public final class WriteOrderOracle { private WriteOrderOracle() {} + private static final AtomicLong WRITER_ORDERER = new AtomicLong(System.currentTimeMillis()); public static void setSeed(long highest) { long previous; - while(highest > (previous = WRITER_ORDERER.get())) { + while (highest > (previous = WRITER_ORDERER.get())) { WRITER_ORDERER.compareAndSet(previous, highest); } } + public static long next() { return WRITER_ORDERER.incrementAndGet(); } diff --git a/flume-ng-channels/flume-file-channel/src/main/java/org/apache/flume/channel/file/encryption/AESCTRNoPaddingProvider.java b/flume-ng-channels/flume-file-channel/src/main/java/org/apache/flume/channel/file/encryption/AESCTRNoPaddingProvider.java index 9ee4245521..e1116d2148 100644 --- a/flume-ng-channels/flume-file-channel/src/main/java/org/apache/flume/channel/file/encryption/AESCTRNoPaddingProvider.java +++ b/flume-ng-channels/flume-file-channel/src/main/java/org/apache/flume/channel/file/encryption/AESCTRNoPaddingProvider.java @@ -46,7 +46,8 @@ public Decryptor.Builder newDecryptorBuilder() { } public static class EncryptorBuilder - extends CipherProvider.Encryptor.Builder { + extends CipherProvider.Encryptor.Builder { + @Override public AESCTRNoPaddingEncryptor build() { ByteBuffer buffer = ByteBuffer.allocate(16); @@ -58,9 +59,8 @@ public AESCTRNoPaddingEncryptor build() { } } - public static class DecryptorBuilder - extends CipherProvider.Decryptor.Builder { + extends CipherProvider.Decryptor.Builder { @Override public AESCTRNoPaddingDecryptor build() { return new AESCTRNoPaddingDecryptor(key, parameters); @@ -70,18 +70,22 @@ public AESCTRNoPaddingDecryptor build() { private static class AESCTRNoPaddingEncryptor extends Encryptor { private byte[] parameters; private Cipher cipher; + private AESCTRNoPaddingEncryptor(Key key, byte[] parameters) { this.parameters = parameters; cipher = getCipher(key, Cipher.ENCRYPT_MODE, parameters); } + @Override public byte[] getParameters() { return parameters; } + @Override public String getCodec() { return TYPE; } + @Override public byte[] encrypt(byte[] clearText) { return doFinal(cipher, clearText); @@ -90,21 +94,23 @@ public byte[] encrypt(byte[] clearText) { private static class AESCTRNoPaddingDecryptor extends Decryptor { private Cipher cipher; + private AESCTRNoPaddingDecryptor(Key key, byte[] parameters) { cipher = getCipher(key, Cipher.DECRYPT_MODE, parameters); } + @Override public byte[] decrypt(byte[] cipherText) { return doFinal(cipher, cipherText); } + @Override public String getCodec() { return TYPE; } } - private static byte[] doFinal(Cipher cipher, byte[] input) - throws DecryptionFailureException{ + private static byte[] doFinal(Cipher cipher, byte[] input) throws DecryptionFailureException { try { return cipher.doFinal(input); } catch (Exception e) { diff --git a/flume-ng-channels/flume-file-channel/src/main/java/org/apache/flume/channel/file/encryption/CipherProvider.java b/flume-ng-channels/flume-file-channel/src/main/java/org/apache/flume/channel/file/encryption/CipherProvider.java index 2c71bc9b2a..2c516f9601 100644 --- a/flume-ng-channels/flume-file-channel/src/main/java/org/apache/flume/channel/file/encryption/CipherProvider.java +++ b/flume-ng-channels/flume-file-channel/src/main/java/org/apache/flume/channel/file/encryption/CipherProvider.java @@ -27,44 +27,45 @@ public abstract class CipherProvider { public abstract Encryptor.Builder newEncryptorBuilder(); public abstract Decryptor.Builder newDecryptorBuilder(); - public static abstract class Encryptor { + public abstract static class Encryptor { public abstract byte[] encrypt(byte[] clearText); public abstract byte[] getParameters(); public abstract String getCodec(); /** Builder implementations MUST have a no-arg constructor */ - public static abstract class Builder { + public abstract static class Builder { protected Key key; + public Builder setKey(Key key) { this.key = Preconditions.checkNotNull(key, "key cannot be null"); return this; } + public abstract T build(); } - } - public static abstract class Decryptor { + public abstract static class Decryptor { public abstract byte[] decrypt(byte[] cipherText); public abstract String getCodec(); /** Builder implementations MUST have a no-arg constructor */ - public static abstract class Builder { + public abstract static class Builder { protected byte[] parameters; protected Key key; + public Builder setKey(Key key) { this.key = Preconditions.checkNotNull(key, "key cannot be null"); return this; } + public Builder setParameters(byte[] parameters) { this.parameters = parameters; return this; } + public abstract T build(); } - } - - } diff --git a/flume-ng-channels/flume-file-channel/src/main/java/org/apache/flume/channel/file/encryption/CipherProviderFactory.java b/flume-ng-channels/flume-file-channel/src/main/java/org/apache/flume/channel/file/encryption/CipherProviderFactory.java index ca11f6bbac..85b0fbb65c 100644 --- a/flume-ng-channels/flume-file-channel/src/main/java/org/apache/flume/channel/file/encryption/CipherProviderFactory.java +++ b/flume-ng-channels/flume-file-channel/src/main/java/org/apache/flume/channel/file/encryption/CipherProviderFactory.java @@ -33,7 +33,7 @@ public class CipherProviderFactory { public static CipherProvider.Encryptor getEncrypter(String cipherProviderType, Key key) { - if(cipherProviderType == null) { + if (cipherProviderType == null) { return null; } CipherProvider provider = getProvider(cipherProviderType); @@ -41,7 +41,7 @@ public static CipherProvider.Encryptor getEncrypter(String cipherProviderType, } public static CipherProvider.Decryptor getDecrypter(String cipherProviderType, Key key, byte[] parameters) { - if(cipherProviderType == null) { + if (cipherProviderType == null) { return null; } CipherProvider provider = getProvider(cipherProviderType); diff --git a/flume-ng-channels/flume-file-channel/src/main/java/org/apache/flume/channel/file/encryption/CipherProviderType.java b/flume-ng-channels/flume-file-channel/src/main/java/org/apache/flume/channel/file/encryption/CipherProviderType.java index 87834d7a96..73bc720322 100644 --- a/flume-ng-channels/flume-file-channel/src/main/java/org/apache/flume/channel/file/encryption/CipherProviderType.java +++ b/flume-ng-channels/flume-file-channel/src/main/java/org/apache/flume/channel/file/encryption/CipherProviderType.java @@ -18,7 +18,6 @@ */ package org.apache.flume.channel.file.encryption; - public enum CipherProviderType { AESCTRNOPADDING(AESCTRNoPaddingProvider.class), OTHER(null); diff --git a/flume-ng-channels/flume-file-channel/src/main/java/org/apache/flume/channel/file/encryption/DecryptionFailureException.java b/flume-ng-channels/flume-file-channel/src/main/java/org/apache/flume/channel/file/encryption/DecryptionFailureException.java index 0155c39581..beffd9ec73 100644 --- a/flume-ng-channels/flume-file-channel/src/main/java/org/apache/flume/channel/file/encryption/DecryptionFailureException.java +++ b/flume-ng-channels/flume-file-channel/src/main/java/org/apache/flume/channel/file/encryption/DecryptionFailureException.java @@ -27,7 +27,6 @@ public class DecryptionFailureException extends FlumeException { private static final long serialVersionUID = 6646810195384793646L; - public DecryptionFailureException(String msg) { super(msg); } diff --git a/flume-ng-channels/flume-file-channel/src/main/java/org/apache/flume/channel/file/encryption/JCEFileKeyProvider.java b/flume-ng-channels/flume-file-channel/src/main/java/org/apache/flume/channel/file/encryption/JCEFileKeyProvider.java index f961ef9769..c96cf0a0a0 100644 --- a/flume-ng-channels/flume-file-channel/src/main/java/org/apache/flume/channel/file/encryption/JCEFileKeyProvider.java +++ b/flume-ng-channels/flume-file-channel/src/main/java/org/apache/flume/channel/file/encryption/JCEFileKeyProvider.java @@ -55,7 +55,7 @@ public JCEFileKeyProvider(File keyStoreFile, File keyStorePasswordFile, keyStorePassword = Files.toString(keyStorePasswordFile, Charsets.UTF_8) .trim().toCharArray(); ks.load(new FileInputStream(keyStoreFile), keyStorePassword); - } catch(Exception ex) { + } catch (Exception ex) { throw Throwables.propagate(ex); } } @@ -65,14 +65,14 @@ public Key getKey(String alias) { String passwordFile = keyStorePasswordFile.getAbsolutePath(); try { char[] keyPassword = keyStorePassword; - if(aliasPasswordFileMap.containsKey(alias)) { + if (aliasPasswordFileMap.containsKey(alias)) { File keyPasswordFile = aliasPasswordFileMap.get(alias); keyPassword = Files.toString(keyPasswordFile, Charsets.UTF_8).trim().toCharArray(); passwordFile = keyPasswordFile.getAbsolutePath(); } Key key = ks.getKey(alias, keyPassword); - if(key == null) { + if (key == null) { throw new IllegalStateException("KeyStore returned null for " + alias); } return key; @@ -99,13 +99,13 @@ public KeyProvider build(Context context) { EncryptionConfiguration.JCE_FILE_KEYS); Preconditions.checkState(!Strings.isNullOrEmpty(passwordProtectedKeys), "Keys available to KeyStore was not specified or empty"); - for(String passwordName : passwordProtectedKeys.trim().split("\\s+")) { + for (String passwordName : passwordProtectedKeys.trim().split("\\s+")) { String propertyName = Joiner.on(".").join(EncryptionConfiguration.JCE_FILE_KEYS, passwordName, EncryptionConfiguration.JCE_FILE_KEY_PASSWORD_FILE); String passwordFileName = context.getString(propertyName, keyStorePasswordFileName); File passwordFile = new File(passwordFileName.trim()); - if(passwordFile.isFile()) { + if (passwordFile.isFile()) { aliasPasswordFileMap.put(passwordName, passwordFile); } else { logger.warn("Password file for alias " + passwordName + diff --git a/flume-ng-channels/flume-file-channel/src/main/java/org/apache/flume/channel/file/encryption/KeyProviderType.java b/flume-ng-channels/flume-file-channel/src/main/java/org/apache/flume/channel/file/encryption/KeyProviderType.java index 0fef6dc0da..326361501f 100644 --- a/flume-ng-channels/flume-file-channel/src/main/java/org/apache/flume/channel/file/encryption/KeyProviderType.java +++ b/flume-ng-channels/flume-file-channel/src/main/java/org/apache/flume/channel/file/encryption/KeyProviderType.java @@ -18,7 +18,6 @@ */ package org.apache.flume.channel.file.encryption; - public enum KeyProviderType { JCEKSFILE(JCEFileKeyProvider.Builder.class), OTHER(null); diff --git a/flume-ng-channels/flume-file-channel/src/main/java/org/apache/flume/channel/file/proto/ProtosFactory.java b/flume-ng-channels/flume-file-channel/src/main/java/org/apache/flume/channel/file/proto/ProtosFactory.java index 34f93d9b93..50492cc5f9 100644 --- a/flume-ng-channels/flume-file-channel/src/main/java/org/apache/flume/channel/file/proto/ProtosFactory.java +++ b/flume-ng-channels/flume-file-channel/src/main/java/org/apache/flume/channel/file/proto/ProtosFactory.java @@ -5010,18 +5010,21 @@ public Builder clearOffset() { public interface RollbackOrBuilder extends com.google.protobuf.MessageOrBuilder { } + /** * Protobuf type {@code Rollback} */ - public static final class Rollback extends - com.google.protobuf.GeneratedMessage - implements RollbackOrBuilder { + public static final class Rollback extends com.google.protobuf.GeneratedMessage + implements RollbackOrBuilder { // Use Rollback.newBuilder() to construct. private Rollback(com.google.protobuf.GeneratedMessage.Builder builder) { super(builder); this.unknownFields = builder.getUnknownFields(); } - private Rollback(boolean noInit) { this.unknownFields = com.google.protobuf.UnknownFieldSet.getDefaultInstance(); } + + private Rollback(boolean noInit) { + this.unknownFields = com.google.protobuf.UnknownFieldSet.getDefaultInstance(); + } private static final Rollback defaultInstance; public static Rollback getDefaultInstance() { diff --git a/flume-ng-channels/flume-jdbc-channel/src/main/java/org/apache/flume/channel/jdbc/ConfigurationConstants.java b/flume-ng-channels/flume-jdbc-channel/src/main/java/org/apache/flume/channel/jdbc/ConfigurationConstants.java index 3b9768429f..7138b41908 100644 --- a/flume-ng-channels/flume-jdbc-channel/src/main/java/org/apache/flume/channel/jdbc/ConfigurationConstants.java +++ b/flume-ng-channels/flume-jdbc-channel/src/main/java/org/apache/flume/channel/jdbc/ConfigurationConstants.java @@ -89,7 +89,6 @@ public final class ConfigurationConstants { public static final String OLD_CONFIG_CREATE_SCHEMA = PREFIX + CONFIG_CREATE_SCHEMA; - public static final String CONFIG_CREATE_INDEX = "create.index"; /** diff --git a/flume-ng-channels/flume-jdbc-channel/src/main/java/org/apache/flume/channel/jdbc/JdbcChannel.java b/flume-ng-channels/flume-jdbc-channel/src/main/java/org/apache/flume/channel/jdbc/JdbcChannel.java index 1192452ec9..fba6e7b20c 100644 --- a/flume-ng-channels/flume-jdbc-channel/src/main/java/org/apache/flume/channel/jdbc/JdbcChannel.java +++ b/flume-ng-channels/flume-jdbc-channel/src/main/java/org/apache/flume/channel/jdbc/JdbcChannel.java @@ -27,6 +27,7 @@ import org.apache.flume.channel.AbstractChannel; import org.slf4j.Logger; import org.slf4j.LoggerFactory; + /** *

    * A JDBC based channel implementation. diff --git a/flume-ng-channels/flume-jdbc-channel/src/main/java/org/apache/flume/channel/jdbc/JdbcChannelProvider.java b/flume-ng-channels/flume-jdbc-channel/src/main/java/org/apache/flume/channel/jdbc/JdbcChannelProvider.java index e445d613d3..76bc627f38 100644 --- a/flume-ng-channels/flume-jdbc-channel/src/main/java/org/apache/flume/channel/jdbc/JdbcChannelProvider.java +++ b/flume-ng-channels/flume-jdbc-channel/src/main/java/org/apache/flume/channel/jdbc/JdbcChannelProvider.java @@ -29,7 +29,7 @@ public interface JdbcChannelProvider { /** * Initializes the channel provider. This method must be called before * the channel can be used in any way. - * @param properties the configuration for the system + * @param context the configuration for the system */ public void initialize(Context context); diff --git a/flume-ng-channels/flume-jdbc-channel/src/main/java/org/apache/flume/channel/jdbc/impl/DerbySchemaHandler.java b/flume-ng-channels/flume-jdbc-channel/src/main/java/org/apache/flume/channel/jdbc/impl/DerbySchemaHandler.java index 2dc3fcc6e1..56eebfdeaf 100644 --- a/flume-ng-channels/flume-jdbc-channel/src/main/java/org/apache/flume/channel/jdbc/impl/DerbySchemaHandler.java +++ b/flume-ng-channels/flume-jdbc-channel/src/main/java/org/apache/flume/channel/jdbc/impl/DerbySchemaHandler.java @@ -127,7 +127,6 @@ public class DerbySchemaHandler implements SchemaHandler { private static final Logger LOGGER = LoggerFactory.getLogger(DerbySchemaHandler.class); - private static final String QUREY_SYSCHEMA_FLUME = "SELECT SCHEMAID FROM SYS.SYSSCHEMAS WHERE SCHEMANAME = 'FLUME'"; @@ -613,15 +612,15 @@ public void storeEvent(PersistableEvent pe, Connection connection) { // Persist the payload spill if (hasSpillPayload) { - spillEventStmt = connection.prepareStatement(STMT_INSERT_EVENT_SPILL); - spillEventStmt.setLong(1, eventId); - spillEventStmt.setBinaryStream(2, - new ByteArrayInputStream(spillPayload), spillPayload.length); - int spillEventCount = spillEventStmt.executeUpdate(); - if (spillEventCount != 1) { - throw new JdbcChannelException("Invalid update count on spill " - + "event insert: " + spillEventCount); - } + spillEventStmt = connection.prepareStatement(STMT_INSERT_EVENT_SPILL); + spillEventStmt.setLong(1, eventId); + spillEventStmt.setBinaryStream(2, + new ByteArrayInputStream(spillPayload), spillPayload.length); + int spillEventCount = spillEventStmt.executeUpdate(); + if (spillEventCount != 1) { + throw new JdbcChannelException("Invalid update count on spill " + + "event insert: " + spillEventCount); + } } // Persist the headers @@ -645,8 +644,7 @@ public void storeEvent(PersistableEvent pe, Connection connection) { int updateCount = baseHeaderStmt.executeUpdate(); if (updateCount != 1) { - throw new JdbcChannelException("Unexpected update header count: " - + updateCount); + throw new JdbcChannelException("Unexpected update header count: " + updateCount); } ResultSet headerIdResultSet = baseHeaderStmt.getGeneratedKeys(); if (!headerIdResultSet.next()) { @@ -705,7 +703,7 @@ public void storeEvent(PersistableEvent pe, Connection connection) { headerValueSpillStmt = connection.prepareStatement(STMT_INSERT_HEADER_VALUE_SPILL); - for(HeaderEntry entry : headerWithValueSpill) { + for (HeaderEntry entry : headerWithValueSpill) { String valueSpill = entry.getValue().getSpill(); headerValueSpillStmt.setLong(1, entry.getId()); diff --git a/flume-ng-channels/flume-jdbc-channel/src/main/java/org/apache/flume/channel/jdbc/impl/JdbcChannelProviderImpl.java b/flume-ng-channels/flume-jdbc-channel/src/main/java/org/apache/flume/channel/jdbc/impl/JdbcChannelProviderImpl.java index f42b4ddd78..845b794170 100644 --- a/flume-ng-channels/flume-jdbc-channel/src/main/java/org/apache/flume/channel/jdbc/impl/JdbcChannelProviderImpl.java +++ b/flume-ng-channels/flume-jdbc-channel/src/main/java/org/apache/flume/channel/jdbc/impl/JdbcChannelProviderImpl.java @@ -54,12 +54,10 @@ public class JdbcChannelProviderImpl implements JdbcChannelProvider { private static final Logger LOGGER = LoggerFactory.getLogger(JdbcChannelProviderImpl.class); - private static final String EMBEDDED_DERBY_DRIVER_CLASSNAME - = "org.apache.derby.jdbc.EmbeddedDriver"; + = "org.apache.derby.jdbc.EmbeddedDriver"; - private static final String DEFAULT_DRIVER_CLASSNAME - = EMBEDDED_DERBY_DRIVER_CLASSNAME; + private static final String DEFAULT_DRIVER_CLASSNAME = EMBEDDED_DERBY_DRIVER_CLASSNAME; private static final String DEFAULT_USERNAME = "sa"; private static final String DEFAULT_PASSWORD = ""; private static final String DEFAULT_DBTYPE = "DERBY"; @@ -133,7 +131,7 @@ private void initializeSystemProperties(Context context) { for (String key: sysProps.keySet()) { String value = sysProps.get(key); - if(key != null && value != null) { + if (key != null && value != null) { System.setProperty(key, value); } } @@ -254,7 +252,7 @@ public void close() { int index = connectUrl.indexOf(";"); String baseUrl = null; if (index != -1) { - baseUrl = connectUrl.substring(0, index+1); + baseUrl = connectUrl.substring(0, index + 1); } else { baseUrl = connectUrl + ";"; } @@ -440,12 +438,12 @@ private void initializeDataSource(Context context) { databaseType = DatabaseType.getByName(dbTypeName); switch (databaseType) { - case DERBY: - case MYSQL: - break; - default: - throw new JdbcChannelException("Database " + databaseType - + " not supported at this time"); + case DERBY: + case MYSQL: + break; + default: + throw new JdbcChannelException("Database " + databaseType + + " not supported at this time"); } // Register driver diff --git a/flume-ng-channels/flume-jdbc-channel/src/main/java/org/apache/flume/channel/jdbc/impl/JdbcTransactionImpl.java b/flume-ng-channels/flume-jdbc-channel/src/main/java/org/apache/flume/channel/jdbc/impl/JdbcTransactionImpl.java index 13b14f5b30..6f3aecd486 100644 --- a/flume-ng-channels/flume-jdbc-channel/src/main/java/org/apache/flume/channel/jdbc/impl/JdbcTransactionImpl.java +++ b/flume-ng-channels/flume-jdbc-channel/src/main/java/org/apache/flume/channel/jdbc/impl/JdbcTransactionImpl.java @@ -28,7 +28,6 @@ import org.slf4j.Logger; import org.slf4j.LoggerFactory; - public class JdbcTransactionImpl implements Transaction { private static final Logger LOGGER = diff --git a/flume-ng-channels/flume-jdbc-channel/src/main/java/org/apache/flume/channel/jdbc/impl/SchemaHandler.java b/flume-ng-channels/flume-jdbc-channel/src/main/java/org/apache/flume/channel/jdbc/impl/SchemaHandler.java index dba96fca3a..9bfc227ff2 100644 --- a/flume-ng-channels/flume-jdbc-channel/src/main/java/org/apache/flume/channel/jdbc/impl/SchemaHandler.java +++ b/flume-ng-channels/flume-jdbc-channel/src/main/java/org/apache/flume/channel/jdbc/impl/SchemaHandler.java @@ -26,14 +26,12 @@ public interface SchemaHandler { /** - * @param connection the connection to check for schema. * @return true if the schema exists. False otherwise. */ public boolean schemaExists(); /** * Validates the schema. - * @param connection */ public void validateSchema(); @@ -74,8 +72,6 @@ public PersistableEvent fetchAndDeleteEvent( * must have an active transaction ongoing. This allows the provider impl to * enforce channel capacity limits when persisting events. * @return the current size of the channel. - * @param connection - * @return */ public long getChannelSize(Connection connection); } diff --git a/flume-ng-channels/flume-jdbc-channel/src/main/java/org/apache/flume/channel/jdbc/impl/SchemaHandlerFactory.java b/flume-ng-channels/flume-jdbc-channel/src/main/java/org/apache/flume/channel/jdbc/impl/SchemaHandlerFactory.java index 2543848ce3..35f4c61178 100644 --- a/flume-ng-channels/flume-jdbc-channel/src/main/java/org/apache/flume/channel/jdbc/impl/SchemaHandlerFactory.java +++ b/flume-ng-channels/flume-jdbc-channel/src/main/java/org/apache/flume/channel/jdbc/impl/SchemaHandlerFactory.java @@ -27,19 +27,17 @@ */ public final class SchemaHandlerFactory { - public static SchemaHandler getHandler(DatabaseType dbType, - DataSource dataSource) { + public static SchemaHandler getHandler(DatabaseType dbType, DataSource dataSource) { SchemaHandler handler = null; - switch(dbType) { - case DERBY: - handler = new DerbySchemaHandler(dataSource); - break; - case MYSQL: - handler = new MySQLSchemaHandler(dataSource); - break; - default: - throw new JdbcChannelException("Database " + dbType - + " not supported yet"); + switch (dbType) { + case DERBY: + handler = new DerbySchemaHandler(dataSource); + break; + case MYSQL: + handler = new MySQLSchemaHandler(dataSource); + break; + default: + throw new JdbcChannelException("Database " + dbType + " not supported yet"); } return handler; diff --git a/flume-ng-channels/flume-kafka-channel/src/main/java/org/apache/flume/channel/kafka/KafkaChannel.java b/flume-ng-channels/flume-kafka-channel/src/main/java/org/apache/flume/channel/kafka/KafkaChannel.java index dfc95bc25e..90e3288a5e 100644 --- a/flume-ng-channels/flume-kafka-channel/src/main/java/org/apache/flume/channel/kafka/KafkaChannel.java +++ b/flume-ng-channels/flume-kafka-channel/src/main/java/org/apache/flume/channel/kafka/KafkaChannel.java @@ -73,7 +73,7 @@ public class KafkaChannel extends BasicChannelSemantics { - private final static Logger logger = + private static final Logger logger = LoggerFactory.getLogger(KafkaChannel.class); private final Properties consumerProps = new Properties(); @@ -97,27 +97,27 @@ public class KafkaChannel extends BasicChannelSemantics { private KafkaChannelCounter counter; - /* Each Consumer commit will commit all partitions owned by it. To - * ensure that each partition is only committed when all events are - * actually done, we will need to keep a Consumer per thread. - */ + /* Each Consumer commit will commit all partitions owned by it. To + * ensure that each partition is only committed when all events are + * actually done, we will need to keep a Consumer per thread. + */ - private final ThreadLocal consumerAndRecords = new - ThreadLocal() { - @Override - public ConsumerAndRecords initialValue() { - return createConsumerAndRecords(); - } - }; + private final ThreadLocal consumerAndRecords = + new ThreadLocal() { + @Override + public ConsumerAndRecords initialValue() { + return createConsumerAndRecords(); + } + }; @Override public void start() { - logger.info("Starting Kafka Channel: {}", getName()); - producer = new KafkaProducer(producerProps); - // We always have just one topic being read by one thread - logger.info("Topic = {}", topic.get()); - counter.start(); - super.start(); + logger.info("Starting Kafka Channel: {}", getName()); + producer = new KafkaProducer(producerProps); + // We always have just one topic being read by one thread + logger.info("Topic = {}", topic.get()); + counter.start(); + super.start(); } @Override @@ -185,17 +185,19 @@ private void translateOldProps(Context ctx) { throw new ConfigurationException("Bootstrap Servers must be specified"); } else { ctx.put(BOOTSTRAP_SERVERS_CONFIG, brokerList); - logger.warn("{} is deprecated. Please use the parameter {}", BROKER_LIST_FLUME_KEY, BOOTSTRAP_SERVERS_CONFIG); + logger.warn("{} is deprecated. Please use the parameter {}", + BROKER_LIST_FLUME_KEY, BOOTSTRAP_SERVERS_CONFIG); } } //GroupId // If there is an old Group Id set, then use that if no groupId is set. if (!(ctx.containsKey(KAFKA_CONSUMER_PREFIX + ConsumerConfig.GROUP_ID_CONFIG))) { - String oldGroupId = ctx.getString(GROUP_ID_FLUME); - if ( oldGroupId != null && !oldGroupId.isEmpty()) { + String oldGroupId = ctx.getString(GROUP_ID_FLUME); + if (oldGroupId != null && !oldGroupId.isEmpty()) { ctx.put(KAFKA_CONSUMER_PREFIX + ConsumerConfig.GROUP_ID_CONFIG, oldGroupId); - logger.warn("{} is deprecated. Please use the parameter {}", GROUP_ID_FLUME, KAFKA_CONSUMER_PREFIX + ConsumerConfig.GROUP_ID_CONFIG); + logger.warn("{} is deprecated. Please use the parameter {}", + GROUP_ID_FLUME, KAFKA_CONSUMER_PREFIX + ConsumerConfig.GROUP_ID_CONFIG); } } @@ -209,7 +211,9 @@ private void translateOldProps(Context ctx) { auto = "latest"; } ctx.put(KAFKA_CONSUMER_PREFIX + ConsumerConfig.AUTO_OFFSET_RESET_CONFIG,auto); - logger.warn("{} is deprecated. Please use the parameter {}", READ_SMALLEST_OFFSET,KAFKA_CONSUMER_PREFIX + ConsumerConfig.AUTO_OFFSET_RESET_CONFIG); + logger.warn("{} is deprecated. Please use the parameter {}", + READ_SMALLEST_OFFSET, + KAFKA_CONSUMER_PREFIX + ConsumerConfig.AUTO_OFFSET_RESET_CONFIG); } } @@ -249,15 +253,17 @@ private void setConsumerProps(Context ctx, String bootStrapServers) { logger.info(consumerProps.toString()); } - protected Properties getConsumerProps() { return consumerProps; } - + protected Properties getConsumerProps() { + return consumerProps; + } private synchronized ConsumerAndRecords createConsumerAndRecords() { try { KafkaConsumer consumer = new KafkaConsumer(consumerProps); ConsumerAndRecords car = new ConsumerAndRecords(consumer, channelUUID); logger.info("Created new consumer to connect to Kafka"); - car.consumer.subscribe(Arrays.asList(topic.get()), new ChannelRebalanceListener(rebalanceFlag)); + car.consumer.subscribe(Arrays.asList(topic.get()), + new ChannelRebalanceListener(rebalanceFlag)); car.offsets = new HashMap(); consumers.add(car); return car; @@ -286,14 +292,14 @@ private enum TransactionType { NONE } - private class KafkaTransaction extends BasicTransactionSemantics { private TransactionType type = TransactionType.NONE; private Optional tempOutStream = Optional .absent(); // For put transactions, serialize the events and hold them until the commit goes is requested. - private Optional>> producerRecords = Optional.absent(); + private Optional>> producerRecords = + Optional.absent(); // For take transactions, deserialize and hold them till commit goes through private Optional> events = Optional.absent(); private Optional> writer = @@ -323,8 +329,9 @@ protected void doPut(Event event) throws InterruptedException { } String key = event.getHeaders().get(KEY_HEADER); try { - producerRecords.get().add(new ProducerRecord - (topic.get(), key, serializeValue(event, parseAsFlumeEvent))); + producerRecords.get().add( + new ProducerRecord(topic.get(), key, + serializeValue(event, parseAsFlumeEvent))); } catch (Exception e) { throw new ChannelException("Error while serializing event", e); } @@ -382,7 +389,8 @@ protected Event doTake() throws InterruptedException { } if (logger.isDebugEnabled()) { - logger.debug("Processed output from partition {} offset {}", record.partition(), record.offset()); + logger.debug("Processed output from partition {} offset {}", + record.partition(), record.offset()); } long endTime = System.nanoTime(); @@ -391,10 +399,10 @@ protected Event doTake() throws InterruptedException { return null; } } catch (Exception ex) { - logger.warn("Error while getting events from Kafka. This is usually caused by trying to read " + - "a non-flume event. Ensure the setting for parseAsFlumeEvent is correct", ex); - throw new ChannelException("Error while getting events from Kafka", - ex); + logger.warn("Error while getting events from Kafka. This is usually caused by " + + "trying to read a non-flume event. Ensure the setting for " + + "parseAsFlumeEvent is correct", ex); + throw new ChannelException("Error while getting events from Kafka", ex); } } eventTaken = true; @@ -564,8 +572,9 @@ public void printCurrentAssignment() { StringBuilder sb = new StringBuilder(); for (TopicPartition tp : this.consumer.assignment()) { try { - sb.append("Committed: [").append(tp).append(",").append(this.consumer.committed(tp).offset()) - .append(",").append(this.consumer.committed(tp).metadata()).append("]"); + sb.append("Committed: [").append(tp).append(",") + .append(this.consumer.committed(tp).offset()) + .append(",").append(this.consumer.committed(tp).metadata()).append("]"); if (logger.isDebugEnabled()) { logger.debug(sb.toString()); } @@ -596,8 +605,8 @@ public void onCompletion(RecordMetadata metadata, Exception exception) { } if (log.isDebugEnabled()) { long batchElapsedTime = System.currentTimeMillis() - startTime; - log.debug("Acked message_no " + index + ": " + metadata.topic() + "-" + metadata.partition() + "-" + - metadata.offset() + "-" + batchElapsedTime); + log.debug("Acked message_no " + index + ": " + metadata.topic() + "-" + + metadata.partition() + "-" + metadata.offset() + "-" + batchElapsedTime); } } } diff --git a/flume-ng-channels/flume-kafka-channel/src/main/java/org/apache/flume/channel/kafka/KafkaChannelConfiguration.java b/flume-ng-channels/flume-kafka-channel/src/main/java/org/apache/flume/channel/kafka/KafkaChannelConfiguration.java index faf46b6232..ccf46d9132 100644 --- a/flume-ng-channels/flume-kafka-channel/src/main/java/org/apache/flume/channel/kafka/KafkaChannelConfiguration.java +++ b/flume-ng-channels/flume-kafka-channel/src/main/java/org/apache/flume/channel/kafka/KafkaChannelConfiguration.java @@ -26,12 +26,17 @@ public class KafkaChannelConfiguration { public static final String KAFKA_CONSUMER_PREFIX = KAFKA_PREFIX + "consumer."; public static final String KAFKA_PRODUCER_PREFIX = KAFKA_PREFIX + "producer."; public static final String DEFAULT_ACKS = "all"; - public static final String DEFAULT_KEY_SERIALIZER = "org.apache.kafka.common.serialization.StringSerializer"; - public static final String DEFAULT_VALUE_SERIAIZER = "org.apache.kafka.common.serialization.ByteArraySerializer"; - public static final String DEFAULT_KEY_DESERIALIZER = "org.apache.kafka.common.serialization.StringDeserializer"; - public static final String DEFAULT_VALUE_DESERIAIZER = "org.apache.kafka.common.serialization.ByteArrayDeserializer"; + public static final String DEFAULT_KEY_SERIALIZER = + "org.apache.kafka.common.serialization.StringSerializer"; + public static final String DEFAULT_VALUE_SERIAIZER = + "org.apache.kafka.common.serialization.ByteArraySerializer"; + public static final String DEFAULT_KEY_DESERIALIZER = + "org.apache.kafka.common.serialization.StringDeserializer"; + public static final String DEFAULT_VALUE_DESERIAIZER = + "org.apache.kafka.common.serialization.ByteArrayDeserializer"; public static final String TOPIC_CONFIG = KAFKA_PREFIX + "topic"; - public static final String BOOTSTRAP_SERVERS_CONFIG = KAFKA_PREFIX + CommonClientConfigs.BOOTSTRAP_SERVERS_CONFIG; + public static final String BOOTSTRAP_SERVERS_CONFIG = + KAFKA_PREFIX + CommonClientConfigs.BOOTSTRAP_SERVERS_CONFIG; public static final String DEFAULT_TOPIC = "flume-channel"; public static final String DEFAULT_GROUP_ID = "flume"; public static final String POLL_TIMEOUT = KAFKA_PREFIX + "pollTimeout"; diff --git a/flume-ng-channels/flume-spillable-memory-channel/src/main/java/org/apache/flume/channel/SpillableMemoryChannel.java b/flume-ng-channels/flume-spillable-memory-channel/src/main/java/org/apache/flume/channel/SpillableMemoryChannel.java index bdf42cdcb1..b46d6469fc 100644 --- a/flume-ng-channels/flume-spillable-memory-channel/src/main/java/org/apache/flume/channel/SpillableMemoryChannel.java +++ b/flume-ng-channels/flume-spillable-memory-channel/src/main/java/org/apache/flume/channel/SpillableMemoryChannel.java @@ -18,26 +18,26 @@ */ package org.apache.flume.channel; -import java.util.ArrayDeque; -import java.util.concurrent.Semaphore; -import java.util.concurrent.TimeUnit; - -import javax.annotation.concurrent.GuardedBy; - import com.google.common.annotations.VisibleForTesting; -import org.apache.flume.*; -import org.apache.flume.annotations.Recyclable; - +import com.google.common.base.Preconditions; +import org.apache.flume.ChannelException; +import org.apache.flume.ChannelFullException; +import org.apache.flume.Context; +import org.apache.flume.Event; +import org.apache.flume.Transaction; import org.apache.flume.annotations.InterfaceAudience; import org.apache.flume.annotations.InterfaceStability; +import org.apache.flume.annotations.Recyclable; import org.apache.flume.channel.file.FileChannel; import org.apache.flume.instrumentation.ChannelCounter; - import org.apache.flume.lifecycle.LifecycleState; import org.slf4j.Logger; import org.slf4j.LoggerFactory; -import com.google.common.base.Preconditions; +import javax.annotation.concurrent.GuardedBy; +import java.util.ArrayDeque; +import java.util.concurrent.Semaphore; +import java.util.concurrent.TimeUnit; /** *

    @@ -50,28 +50,45 @@ @Recyclable public class SpillableMemoryChannel extends FileChannel { // config settings - /** Max number of events to be stored in memory */ + /** + * Max number of events to be stored in memory + */ public static final String MEMORY_CAPACITY = "memoryCapacity"; - /** Seconds to wait before enabling disk overflow when memory fills up */ + /** + * Seconds to wait before enabling disk overflow when memory fills up + */ public static final String OVERFLOW_TIMEOUT = "overflowTimeout"; - /** Internal use only. To remain undocumented in User guide. Determines the + /** + * Internal use only. To remain undocumented in User guide. Determines the * percent free space available in mem queue when we stop spilling to overflow */ public static final String OVERFLOW_DEACTIVATION_THRESHOLD - = "overflowDeactivationThreshold"; - /** percent of buffer between byteCapacity and the estimated event size. */ + = "overflowDeactivationThreshold"; + /** + * percent of buffer between byteCapacity and the estimated event size. + */ public static final String BYTE_CAPACITY_BUFFER_PERCENTAGE - = "byteCapacityBufferPercentage"; + = "byteCapacityBufferPercentage"; - /** max number of bytes used for all events in the queue. */ + /** + * max number of bytes used for all events in the queue. + */ public static final String BYTE_CAPACITY = "byteCapacity"; - /** max number of events in overflow. */ + /** + * max number of events in overflow. + */ public static final String OVERFLOW_CAPACITY = "overflowCapacity"; - /** file channel setting that is overriden by Spillable Channel */ + /** + * file channel setting that is overriden by Spillable Channel + */ public static final String KEEP_ALIVE = "keep-alive"; - /** file channel capacity overridden by Spillable Channel */ + /** + * file channel capacity overridden by Spillable Channel + */ public static final String CAPACITY = "capacity"; - /** Estimated average size of events expected to be in the channel */ + /** + * Estimated average size of events expected to be in the channel + */ public static final String AVG_EVENT_SIZE = "avgEventSize"; private static Logger LOGGER = LoggerFactory.getLogger(SpillableMemoryChannel.class); @@ -84,7 +101,7 @@ public class SpillableMemoryChannel extends FileChannel { // memory consumption control private static final int defaultAvgEventSize = 500; private static final Long defaultByteCapacity - = (long)(Runtime.getRuntime().maxMemory() * .80); + = (long) (Runtime.getRuntime().maxMemory() * .80); private static final int defaultByteCapacityBufferPercentage = 20; private volatile int byteCapacity; @@ -94,7 +111,7 @@ public class SpillableMemoryChannel extends FileChannel { private Semaphore bytesRemaining; // for synchronizing access to primary/overflow channels & drain order - final private Object queueLock = new Object(); + private final Object queueLock = new Object(); @GuardedBy(value = "queueLock") public ArrayDeque memQueue; @@ -109,8 +126,10 @@ public class SpillableMemoryChannel extends FileChannel { private int maxMemQueueSize = 0; // max sie of memory Queue - private boolean overflowDisabled; // if true indicates the overflow should not be used at all. - private boolean overflowActivated=false; // indicates if overflow can be used. invariant: false if overflowDisabled is true. + private boolean overflowDisabled; // if true indicates the overflow should not be used at all. + + // indicates if overflow can be used. invariant: false if overflowDisabled is true. + private boolean overflowActivated = false; // if true overflow can be used. invariant: false if overflowDisabled is true. private int memoryCapacity = -1; // max events that the channel can hold in memory @@ -120,7 +139,7 @@ public class SpillableMemoryChannel extends FileChannel { // mem full % at which we stop spill to overflow private double overflowDeactivationThreshold - = defaultOverflowDeactivationThreshold / 100; + = defaultOverflowDeactivationThreshold / 100; public SpillableMemoryChannel() { super(); @@ -133,6 +152,7 @@ protected int getTotalStored() { public int getMemoryCapacity() { return memoryCapacity; } + public int getOverflowTimeout() { return overflowTimeout; } @@ -160,7 +180,6 @@ public int queueSize() { } } - private static class MutableInteger { private int value; @@ -186,7 +205,7 @@ public static class DrainOrderQueue { public int totalPuts = 0; // for debugging only private long overflowCounter = 0; // # of items in overflow channel - public String dump() { + public String dump() { StringBuilder sb = new StringBuilder(); sb.append(" [ "); @@ -195,12 +214,12 @@ public String dump() { sb.append(" "); } sb.append("]"); - return sb.toString(); + return sb.toString(); } public void putPrimary(Integer eventCount) { totalPuts += eventCount; - if ( (queue.peekLast() == null) || queue.getLast().intValue() < 0) { + if ((queue.peekLast() == null) || queue.getLast().intValue() < 0) { queue.addLast(new MutableInteger(eventCount)); } else { queue.getLast().add(eventCount); @@ -208,7 +227,7 @@ public void putPrimary(Integer eventCount) { } public void putFirstPrimary(Integer eventCount) { - if ( (queue.peekFirst() == null) || queue.getFirst().intValue() < 0) { + if ((queue.peekFirst() == null) || queue.getFirst().intValue() < 0) { queue.addFirst(new MutableInteger(eventCount)); } else { queue.getFirst().add(eventCount); @@ -217,7 +236,7 @@ public void putFirstPrimary(Integer eventCount) { public void putOverflow(Integer eventCount) { totalPuts += eventCount; - if ( (queue.peekLast() == null) || queue.getLast().intValue() > 0) { + if ((queue.peekLast() == null) || queue.getLast().intValue() > 0) { queue.addLast(new MutableInteger(-eventCount)); } else { queue.getLast().add(-eventCount); @@ -226,9 +245,9 @@ public void putOverflow(Integer eventCount) { } public void putFirstOverflow(Integer eventCount) { - if ( (queue.peekFirst() == null) || queue.getFirst().intValue() > 0) { + if ((queue.peekFirst() == null) || queue.getFirst().intValue() > 0) { queue.addFirst(new MutableInteger(-eventCount)); - } else { + } else { queue.getFirst().add(-eventCount); } overflowCounter += eventCount; @@ -247,9 +266,9 @@ public void takePrimary(int takeCount) { // this condition is optimization to avoid redundant conversions of // int -> Integer -> string in hot path - if (headValue.intValue() < takeCount) { + if (headValue.intValue() < takeCount) { throw new IllegalStateException("Cannot take " + takeCount + - " from " + headValue.intValue() + " in DrainOrder Queue"); + " from " + headValue.intValue() + " in DrainOrder Queue"); } headValue.add(-takeCount); @@ -260,9 +279,9 @@ public void takePrimary(int takeCount) { public void takeOverflow(int takeCount) { MutableInteger headValue = queue.getFirst(); - if(headValue.intValue() > -takeCount) { + if (headValue.intValue() > -takeCount) { throw new IllegalStateException("Cannot take " + takeCount + " from " - + headValue.intValue() + " in DrainOrder Queue head " ); + + headValue.intValue() + " in DrainOrder Queue head "); } headValue.add(takeCount); @@ -293,7 +312,6 @@ private class SpillableMemoryTransaction extends BasicTransactionSemantics { ArrayDeque putList; private final ChannelCounter channelCounter; - public SpillableMemoryTransaction(ChannelCounter counter) { takeList = new ArrayDeque(largestTakeTxSize); putList = new ArrayDeque(largestPutTxSize); @@ -307,26 +325,25 @@ public void begin() { @Override public void close() { - if (overflowTakeTx!=null) { + if (overflowTakeTx != null) { overflowTakeTx.close(); } - if (overflowPutTx!=null) { + if (overflowPutTx != null) { overflowPutTx.close(); } super.close(); } - @Override protected void doPut(Event event) throws InterruptedException { channelCounter.incrementEventPutAttemptCount(); putCalled = true; - int eventByteSize = (int)Math.ceil(estimateEventSize(event)/ avgEventSize); + int eventByteSize = (int) Math.ceil(estimateEventSize(event) / avgEventSize); if (!putList.offer(event)) { throw new ChannelFullException("Put queue in " + getName() + - " channel's Transaction having capacity " + putList.size() + - " full, consider reducing batch size of sources"); + " channel's Transaction having capacity " + putList.size() + + " full, consider reducing batch size of sources"); } putListByteCount += eventByteSize; } @@ -344,7 +361,7 @@ protected Event doTake() throws InterruptedException { boolean takeSuceeded = false; try { Event event; - synchronized(queueLock) { + synchronized (queueLock) { int drainOrderTop = drainOrder.front(); if (!takeCalled) { @@ -375,11 +392,11 @@ protected Event doTake() throws InterruptedException { ++takeCount; drainOrder.takePrimary(1); Preconditions.checkNotNull(event, "Queue.poll returned NULL despite" - + " semaphore signalling existence of entry"); + + " semaphore signalling existence of entry"); } } - int eventByteSize = (int)Math.ceil(estimateEventSize(event)/ avgEventSize); + int eventByteSize = (int) Math.ceil(estimateEventSize(event) / avgEventSize); if (!useOverflow) { // takeList is thd pvt, so no need to do this in synchronized block takeList.offer(event); @@ -389,7 +406,7 @@ protected Event doTake() throws InterruptedException { takeSuceeded = true; return event; } finally { - if(!takeSuceeded) { + if (!takeSuceeded) { totalStored.release(); } } @@ -400,37 +417,35 @@ protected void doCommit() throws InterruptedException { if (putCalled) { putCommit(); if (LOGGER.isDebugEnabled()) { - LOGGER.debug("Put Committed. Drain Order Queue state : " - + drainOrder.dump()); + LOGGER.debug("Put Committed. Drain Order Queue state : " + drainOrder.dump()); } } else if (takeCalled) { takeCommit(); if (LOGGER.isDebugEnabled()) { - LOGGER.debug("Take Committed. Drain Order Queue state : " - + drainOrder.dump()); + LOGGER.debug("Take Committed. Drain Order Queue state : " + drainOrder.dump()); } } } private void takeCommit() { - if (takeCount > largestTakeTxSize) + if (takeCount > largestTakeTxSize) { largestTakeTxSize = takeCount; + } synchronized (queueLock) { - if (overflowTakeTx!=null) { + if (overflowTakeTx != null) { overflowTakeTx.commit(); } - double memoryPercentFree = (memoryCapacity == 0) ? 0 - : (memoryCapacity - memQueue.size() + takeCount ) / (double)memoryCapacity ; + double memoryPercentFree = (memoryCapacity == 0) ? 0 : + (memoryCapacity - memQueue.size() + takeCount) / (double) memoryCapacity; - if (overflowActivated - && memoryPercentFree >= overflowDeactivationThreshold) { + if (overflowActivated && memoryPercentFree >= overflowDeactivationThreshold) { overflowActivated = false; LOGGER.info("Overflow Deactivated"); } channelCounter.setChannelSize(getTotalStored()); } - if (!useOverflow) { + if (!useOverflow) { memQueRemaining.release(takeCount); bytesRemaining.release(takeListByteCount); } @@ -440,30 +455,29 @@ private void takeCommit() { private void putCommit() throws InterruptedException { // decide if overflow needs to be used - int timeout = overflowActivated ? 0 : overflowTimeout; + int timeout = overflowActivated ? 0 : overflowTimeout; if (memoryCapacity != 0) { // check for enough event slots(memoryCapacity) for using memory queue if (!memQueRemaining.tryAcquire(putList.size(), timeout, - TimeUnit.SECONDS)) { + TimeUnit.SECONDS)) { if (overflowDisabled) { throw new ChannelFullException("Spillable Memory Channel's " + - "memory capacity has been reached and overflow is " + - "disabled. Consider increasing memoryCapacity."); + "memory capacity has been reached and overflow is " + + "disabled. Consider increasing memoryCapacity."); } overflowActivated = true; useOverflow = true; - } // check if we have enough byteCapacity for using memory queue - else if (!bytesRemaining.tryAcquire(putListByteCount, overflowTimeout - , TimeUnit.SECONDS)) { + } else if (!bytesRemaining.tryAcquire(putListByteCount, + overflowTimeout, TimeUnit.SECONDS)) { memQueRemaining.release(putList.size()); if (overflowDisabled) { throw new ChannelFullException("Spillable Memory Channel's " - + "memory capacity has been reached. " - + (bytesRemaining.availablePermits() * (int) avgEventSize) - + " bytes are free and overflow is disabled. Consider " - + "increasing byteCapacity or capacity."); + + "memory capacity has been reached. " + + (bytesRemaining.availablePermits() * (int) avgEventSize) + + " bytes are free and overflow is disabled. Consider " + + "increasing byteCapacity or capacity."); } overflowActivated = true; useOverflow = true; @@ -496,22 +510,21 @@ private void commitPutsToOverflow() throws InterruptedException { } private void commitPutsToOverflow_core(Transaction overflowPutTx) - throws InterruptedException { + throws InterruptedException { // reattempt only once if overflow is full first time around - for (int i = 0; i < 2; ++i) { + for (int i = 0; i < 2; ++i) { try { - synchronized(queueLock) { + synchronized (queueLock) { overflowPutTx.commit(); drainOrder.putOverflow(putList.size()); channelCounter.setChannelSize(memQueue.size() - + drainOrder.overflowCounter); + + drainOrder.overflowCounter); break; } - } catch (ChannelFullException e) { // drop lock & reattempt - if (i==0) { - Thread.sleep(overflowTimeout *1000); - } - else { + } catch (ChannelFullException e) { // drop lock & reattempt + if (i == 0) { + Thread.sleep(overflowTimeout * 1000); + } else { throw e; } } @@ -523,14 +536,14 @@ private void commitPutsToPrimary() { for (Event e : putList) { if (!memQueue.offer(e)) { throw new ChannelException("Unable to insert event into memory " + - "queue in spite of spare capacity, this is very unexpected"); + "queue in spite of spare capacity, this is very unexpected"); } } drainOrder.putPrimary(putList.size()); - maxMemQueueSize = (memQueue.size() > maxMemQueueSize) ? memQueue.size() - : maxMemQueueSize; + maxMemQueueSize = (memQueue.size() > maxMemQueueSize) ? memQueue.size() + : maxMemQueueSize; channelCounter.setChannelSize(memQueue.size() - + drainOrder.overflowCounter); + + drainOrder.overflowCounter); } // update counters and semaphores totalStored.release(putList.size()); @@ -540,10 +553,10 @@ private void commitPutsToPrimary() { @Override protected void doRollback() { LOGGER.debug("Rollback() of " + - (takeCalled ? " Take Tx" : (putCalled ? " Put Tx" : "Empty Tx"))); + (takeCalled ? " Take Tx" : (putCalled ? " Put Tx" : "Empty Tx"))); if (putCalled) { - if (overflowPutTx!=null) { + if (overflowPutTx != null) { overflowPutTx.rollback(); } if (!useOverflow) { @@ -552,8 +565,8 @@ protected void doRollback() { } putListByteCount = 0; } else if (takeCalled) { - synchronized(queueLock) { - if (overflowTakeTx!=null) { + synchronized (queueLock) { + if (overflowTakeTx != null) { overflowTakeTx.rollback(); } if (useOverflow) { @@ -561,8 +574,8 @@ protected void doRollback() { } else { int remainingCapacity = memoryCapacity - memQueue.size(); Preconditions.checkState(remainingCapacity >= takeCount, - "Not enough space in memory queue to rollback takes. This" + - " should never happen, please report"); + "Not enough space in memory queue to rollback takes. This" + + " should never happen, please report"); while (!takeList.isEmpty()) { memQueue.addFirst(takeList.removeLast()); } @@ -582,15 +595,18 @@ protected void doRollback() { *

  • memoryCapacity = total number of events allowed at one time in the memory queue. *
  • overflowCapacity = total number of events allowed at one time in the overflow file channel. *
  • byteCapacity = the max number of bytes used for events in the memory queue. - *
  • byteCapacityBufferPercentage = type int. Defines the percent of buffer between byteCapacity and the estimated event size. - *
  • overflowTimeout = type int. Number of seconds to wait on a full memory before deciding to enable overflow + *
  • byteCapacityBufferPercentage = type int. Defines the percent of buffer between byteCapacity + * and the estimated event size. + *
  • overflowTimeout = type int. Number of seconds to wait on a full memory before deciding to + * enable overflow */ @Override public void configure(Context context) { - if (getLifecycleState() == LifecycleState.START // does not support reconfig when running - || getLifecycleState() == LifecycleState.ERROR) + if (getLifecycleState() == LifecycleState.START || // does not support reconfig when running + getLifecycleState() == LifecycleState.ERROR) { stop(); + } if (totalStored == null) { totalStored = new Semaphore(0); @@ -603,8 +619,7 @@ public void configure(Context context) { // 1) Memory Capacity Integer newMemoryCapacity; try { - newMemoryCapacity = context.getInteger(MEMORY_CAPACITY - , defaultMemoryCapacity); + newMemoryCapacity = context.getInteger(MEMORY_CAPACITY, defaultMemoryCapacity); if (newMemoryCapacity == null) { newMemoryCapacity = defaultMemoryCapacity; } @@ -612,7 +627,7 @@ public void configure(Context context) { throw new NumberFormatException(MEMORY_CAPACITY + " must be >= 0"); } - } catch(NumberFormatException e) { + } catch (NumberFormatException e) { newMemoryCapacity = defaultMemoryCapacity; LOGGER.warn("Invalid " + MEMORY_CAPACITY + " specified, initializing " + getName() + " channel to default value of {}", defaultMemoryCapacity); @@ -626,60 +641,60 @@ public void configure(Context context) { // overflowTimeout - wait time before switching to overflow when mem is full try { Integer newOverflowTimeout = - context.getInteger(OVERFLOW_TIMEOUT, defaultOverflowTimeout); + context.getInteger(OVERFLOW_TIMEOUT, defaultOverflowTimeout); overflowTimeout = (newOverflowTimeout != null) ? newOverflowTimeout - : defaultOverflowTimeout; - } catch(NumberFormatException e) { + : defaultOverflowTimeout; + } catch (NumberFormatException e) { LOGGER.warn("Incorrect value for " + getName() + "'s " + OVERFLOW_TIMEOUT - + " setting. Using default value {}", defaultOverflowTimeout); + + " setting. Using default value {}", defaultOverflowTimeout); overflowTimeout = defaultOverflowTimeout; } try { Integer newThreshold = context.getInteger(OVERFLOW_DEACTIVATION_THRESHOLD); - overflowDeactivationThreshold = (newThreshold != null) ? - newThreshold/100.0 - : defaultOverflowDeactivationThreshold / 100.0; - } catch(NumberFormatException e) { + overflowDeactivationThreshold = (newThreshold != null) ? + newThreshold / 100.0 + : defaultOverflowDeactivationThreshold / 100.0; + } catch (NumberFormatException e) { LOGGER.warn("Incorrect value for " + getName() + "'s " + OVERFLOW_DEACTIVATION_THRESHOLD + ". Using default value {} %", - defaultOverflowDeactivationThreshold); + defaultOverflowDeactivationThreshold); overflowDeactivationThreshold = defaultOverflowDeactivationThreshold / 100.0; } // 3) Memory consumption control try { byteCapacityBufferPercentage = - context.getInteger(BYTE_CAPACITY_BUFFER_PERCENTAGE - , defaultByteCapacityBufferPercentage); - } catch(NumberFormatException e) { + context.getInteger(BYTE_CAPACITY_BUFFER_PERCENTAGE, defaultByteCapacityBufferPercentage); + } catch (NumberFormatException e) { LOGGER.warn("Error parsing " + BYTE_CAPACITY_BUFFER_PERCENTAGE + " for " - + getName() + ". Using default=" - + defaultByteCapacityBufferPercentage + ". " + e.getMessage()); + + getName() + ". Using default=" + + defaultByteCapacityBufferPercentage + ". " + e.getMessage()); byteCapacityBufferPercentage = defaultByteCapacityBufferPercentage; } try { avgEventSize = context.getInteger(AVG_EVENT_SIZE, defaultAvgEventSize); - } catch ( NumberFormatException e) { + } catch (NumberFormatException e) { LOGGER.warn("Error parsing " + AVG_EVENT_SIZE + " for " + getName() - + ". Using default = " + defaultAvgEventSize + ". " - + e.getMessage()); + + ". Using default = " + defaultAvgEventSize + ". " + + e.getMessage()); avgEventSize = defaultAvgEventSize; } try { - byteCapacity = (int) ((context.getLong(BYTE_CAPACITY, defaultByteCapacity) * (1 - byteCapacityBufferPercentage * .01 )) / avgEventSize); + byteCapacity = (int) ((context.getLong(BYTE_CAPACITY, defaultByteCapacity) * + (1 - byteCapacityBufferPercentage * .01)) / avgEventSize); if (byteCapacity < 1) { byteCapacity = Integer.MAX_VALUE; } - } catch(NumberFormatException e) { + } catch (NumberFormatException e) { LOGGER.warn("Error parsing " + BYTE_CAPACITY + " setting for " + getName() - + ". Using default = " + defaultByteCapacity + ". " - + e.getMessage()); + + ". Using default = " + defaultByteCapacity + ". " + + e.getMessage()); byteCapacity = (int) - ( (defaultByteCapacity * (1 - byteCapacityBufferPercentage * .01 )) - / avgEventSize); + ((defaultByteCapacity * (1 - byteCapacityBufferPercentage * .01)) + / avgEventSize); } @@ -692,8 +707,10 @@ public void configure(Context context) { lastByteCapacity = byteCapacity; } else { try { - if (!bytesRemaining.tryAcquire(lastByteCapacity - byteCapacity, overflowTimeout, TimeUnit.SECONDS)) { - LOGGER.warn("Couldn't acquire permits to downsize the byte capacity, resizing has been aborted"); + if (!bytesRemaining.tryAcquire(lastByteCapacity - byteCapacity, + overflowTimeout, TimeUnit.SECONDS)) { + LOGGER.warn("Couldn't acquire permits to downsize the byte capacity, " + + "resizing has been aborted"); } else { lastByteCapacity = byteCapacity; } @@ -704,51 +721,53 @@ public void configure(Context context) { } try { - overflowCapacity = context.getInteger(OVERFLOW_CAPACITY, defaultOverflowCapacity); // file channel capacity + // file channel capacity + overflowCapacity = context.getInteger(OVERFLOW_CAPACITY, defaultOverflowCapacity); // Determine if File channel needs to be disabled - if ( memoryCapacity < 1 && overflowCapacity < 1) { - LOGGER.warn("For channel " + getName() + OVERFLOW_CAPACITY + - " cannot be set to 0 if " + MEMORY_CAPACITY + " is also 0. " + - "Using default value " + OVERFLOW_CAPACITY + " = " + - defaultOverflowCapacity); - overflowCapacity = defaultOverflowCapacity; - } - overflowDisabled = (overflowCapacity < 1) ; - if (overflowDisabled) { - overflowActivated = false; - } - } catch(NumberFormatException e) { + if (memoryCapacity < 1 && overflowCapacity < 1) { + LOGGER.warn("For channel " + getName() + OVERFLOW_CAPACITY + + " cannot be set to 0 if " + MEMORY_CAPACITY + " is also 0. " + + "Using default value " + OVERFLOW_CAPACITY + " = " + + defaultOverflowCapacity); + overflowCapacity = defaultOverflowCapacity; + } + overflowDisabled = (overflowCapacity < 1); + if (overflowDisabled) { + overflowActivated = false; + } + } catch (NumberFormatException e) { overflowCapacity = defaultOverflowCapacity; } // Configure File channel - context.put(KEEP_ALIVE,"0"); // override keep-alive for File channel - context.put(CAPACITY, Integer.toString(overflowCapacity) ); // file channel capacity + context.put(KEEP_ALIVE, "0"); // override keep-alive for File channel + context.put(CAPACITY, Integer.toString(overflowCapacity)); // file channel capacity super.configure(context); } private void resizePrimaryQueue(int newMemoryCapacity) throws InterruptedException { - if (memQueue != null && memoryCapacity == newMemoryCapacity) { + if (memQueue != null && memoryCapacity == newMemoryCapacity) { return; } if (memoryCapacity > newMemoryCapacity) { int diff = memoryCapacity - newMemoryCapacity; if (!memQueRemaining.tryAcquire(diff, overflowTimeout, TimeUnit.SECONDS)) { - LOGGER.warn("Memory buffer currently contains more events than the new size. Downsizing has been aborted."); + LOGGER.warn("Memory buffer currently contains more events than the new size. " + + "Downsizing has been aborted."); return; } - synchronized(queueLock) { + synchronized (queueLock) { ArrayDeque newQueue = new ArrayDeque(newMemoryCapacity); newQueue.addAll(memQueue); memQueue = newQueue; memoryCapacity = newMemoryCapacity; } - } else { // if (memoryCapacity <= newMemoryCapacity) - synchronized(queueLock) { + } else { // if (memoryCapacity <= newMemoryCapacity) + synchronized (queueLock) { ArrayDeque newQueue = new ArrayDeque(newMemoryCapacity); - if (memQueue !=null) { + if (memQueue != null) { newQueue.addAll(memQueue); } memQueue = newQueue; @@ -771,14 +790,14 @@ public synchronized void start() { drainOrder.putOverflow(overFlowCount); totalStored.release(overFlowCount); } - int totalCount = overFlowCount + memQueue.size(); + int totalCount = overFlowCount + memQueue.size(); channelCounter.setChannelCapacity(memoryCapacity + getOverflowCapacity()); channelCounter.setChannelSize(totalCount); } @Override public synchronized void stop() { - if (getLifecycleState()==LifecycleState.STOP) { + if (getLifecycleState() == LifecycleState.STOP) { return; } channelCounter.setChannelSize(memQueue.size() + drainOrder.overflowCounter); diff --git a/flume-ng-clients/flume-ng-log4jappender/src/main/java/org/apache/flume/clients/log4jappender/LoadBalancingLog4jAppender.java b/flume-ng-clients/flume-ng-log4jappender/src/main/java/org/apache/flume/clients/log4jappender/LoadBalancingLog4jAppender.java index 713234fa86..ae31916eef 100644 --- a/flume-ng-clients/flume-ng-log4jappender/src/main/java/org/apache/flume/clients/log4jappender/LoadBalancingLog4jAppender.java +++ b/flume-ng-clients/flume-ng-log4jappender/src/main/java/org/apache/flume/clients/log4jappender/LoadBalancingLog4jAppender.java @@ -99,11 +99,11 @@ public void setMaxBackoff(String maxBackoff) { @Override public synchronized void append(LoggingEvent event) { - if(!configured) { + if (!configured) { String errorMsg = "Flume Log4jAppender not configured correctly! Cannot" + - " send events to Flume."; + " send events to Flume."; LogLog.error(errorMsg); - if(getUnsafeMode()) { + if (getUnsafeMode()) { return; } throw new FlumeException(errorMsg); @@ -121,10 +121,9 @@ public synchronized void append(LoggingEvent event) { @Override public void activateOptions() throws FlumeException { try { - final Properties properties = getProperties(hosts, selector, - maxBackoff, getTimeout()); + final Properties properties = getProperties(hosts, selector, maxBackoff, getTimeout()); rpcClient = RpcClientFactory.getInstance(properties); - if(layout != null) { + if (layout != null) { layout.activateOptions(); } configured = true; @@ -169,14 +168,13 @@ private Properties getProperties(String hosts, String selector, throw new FlumeException( "Misconfigured max backoff, value must be greater than 0"); } - props.put(RpcClientConfigurationConstants.CONFIG_BACKOFF, - String.valueOf(true)); + props.put(RpcClientConfigurationConstants.CONFIG_BACKOFF, String.valueOf(true)); props.put(RpcClientConfigurationConstants.CONFIG_MAX_BACKOFF, maxBackoff); } props.setProperty(RpcClientConfigurationConstants.CONFIG_CONNECT_TIMEOUT, - String.valueOf(timeout)); + String.valueOf(timeout)); props.setProperty(RpcClientConfigurationConstants.CONFIG_REQUEST_TIMEOUT, - String.valueOf(timeout)); + String.valueOf(timeout)); return props; } } diff --git a/flume-ng-clients/flume-ng-log4jappender/src/main/java/org/apache/flume/clients/log4jappender/Log4jAppender.java b/flume-ng-clients/flume-ng-log4jappender/src/main/java/org/apache/flume/clients/log4jappender/Log4jAppender.java index 7c483db5db..f9803e424c 100644 --- a/flume-ng-clients/flume-ng-log4jappender/src/main/java/org/apache/flume/clients/log4jappender/Log4jAppender.java +++ b/flume-ng-clients/flume-ng-log4jappender/src/main/java/org/apache/flume/clients/log4jappender/Log4jAppender.java @@ -75,8 +75,7 @@ public class Log4jAppender extends AppenderSkeleton { private String hostname; private int port; private boolean unsafeMode = false; - private long timeout = RpcClientConfigurationConstants - .DEFAULT_REQUEST_TIMEOUT_MILLIS; + private long timeout = RpcClientConfigurationConstants.DEFAULT_REQUEST_TIMEOUT_MILLIS; private boolean avroReflectionEnabled; private String avroSchemaUrl; @@ -99,7 +98,7 @@ public Log4jAppender(){ * @param port The port to connect on the host. * */ - public Log4jAppender(String hostname, int port){ + public Log4jAppender(String hostname, int port) { this.hostname = hostname; this.port = port; } @@ -112,14 +111,14 @@ public Log4jAppender(String hostname, int port){ * was a connection error. */ @Override - public synchronized void append(LoggingEvent event) throws FlumeException{ + public synchronized void append(LoggingEvent event) throws FlumeException { //If rpcClient is null, it means either this appender object was never //setup by setting hostname and port and then calling activateOptions //or this appender object was closed by calling close(), so we throw an //exception to show the appender is no longer accessible. if (rpcClient == null) { String errorMsg = "Cannot Append to Appender! Appender either closed or" + - " not setup correctly!"; + " not setup correctly!"; LogLog.error(errorMsg); if (unsafeMode) { return; @@ -127,7 +126,7 @@ public synchronized void append(LoggingEvent event) throws FlumeException{ throw new FlumeException(errorMsg); } - if(!rpcClient.isActive()){ + if (!rpcClient.isActive()) { reconnect(); } @@ -231,7 +230,7 @@ public synchronized void close() throws FlumeException { } else { String errorMsg = "Flume log4jappender already closed!"; LogLog.error(errorMsg); - if(unsafeMode) { + if (unsafeMode) { return; } throw new FlumeException(errorMsg); @@ -251,7 +250,7 @@ public boolean requiresLayout() { * Set the first flume hop hostname. * @param hostname The first hop where the client should connect to. */ - public void setHostname(String hostname){ + public void setHostname(String hostname) { this.hostname = hostname; } @@ -259,7 +258,7 @@ public void setHostname(String hostname){ * Set the port on the hostname to connect to. * @param port The port to connect on the host. */ - public void setPort(int port){ + public void setPort(int port) { this.port = port; } @@ -299,19 +298,18 @@ public void activateOptions() throws FlumeException { Properties props = new Properties(); props.setProperty(RpcClientConfigurationConstants.CONFIG_HOSTS, "h1"); props.setProperty(RpcClientConfigurationConstants.CONFIG_HOSTS_PREFIX + "h1", - hostname + ":" + port); + hostname + ":" + port); props.setProperty(RpcClientConfigurationConstants.CONFIG_CONNECT_TIMEOUT, - String.valueOf(timeout)); + String.valueOf(timeout)); props.setProperty(RpcClientConfigurationConstants.CONFIG_REQUEST_TIMEOUT, - String.valueOf(timeout)); + String.valueOf(timeout)); try { rpcClient = RpcClientFactory.getInstance(props); if (layout != null) { layout.activateOptions(); } } catch (FlumeException e) { - String errormsg = "RPC client creation failed! " + - e.getMessage(); + String errormsg = "RPC client creation failed! " + e.getMessage(); LogLog.error(errormsg); if (unsafeMode) { return; diff --git a/flume-ng-clients/flume-ng-log4jappender/src/main/java/org/apache/flume/clients/log4jappender/Log4jAvroHeaders.java b/flume-ng-clients/flume-ng-log4jappender/src/main/java/org/apache/flume/clients/log4jappender/Log4jAvroHeaders.java index b68e7496d2..22983d3e70 100644 --- a/flume-ng-clients/flume-ng-log4jappender/src/main/java/org/apache/flume/clients/log4jappender/Log4jAvroHeaders.java +++ b/flume-ng-clients/flume-ng-log4jappender/src/main/java/org/apache/flume/clients/log4jappender/Log4jAvroHeaders.java @@ -30,24 +30,23 @@ public enum Log4jAvroHeaders { AVRO_SCHEMA_URL("flume.avro.schema.url"); private String headerName; - private Log4jAvroHeaders(String headerName){ + private Log4jAvroHeaders(String headerName) { this.headerName = headerName; } - public String getName(){ + public String getName() { return headerName; } - public String toString(){ + public String toString() { return getName(); } - public static Log4jAvroHeaders getByName(String headerName){ + public static Log4jAvroHeaders getByName(String headerName) { Log4jAvroHeaders hdrs = null; - try{ + try { hdrs = Log4jAvroHeaders.valueOf(headerName.toLowerCase(Locale.ENGLISH).trim()); - } - catch(IllegalArgumentException e){ + } catch (IllegalArgumentException e) { hdrs = Log4jAvroHeaders.OTHER; } return hdrs; diff --git a/flume-ng-configuration/src/main/java/org/apache/flume/Context.java b/flume-ng-configuration/src/main/java/org/apache/flume/Context.java index c0460d2199..f00b571aba 100644 --- a/flume-ng-configuration/src/main/java/org/apache/flume/Context.java +++ b/flume-ng-configuration/src/main/java/org/apache/flume/Context.java @@ -86,7 +86,7 @@ public ImmutableMap getSubProperties(String prefix) { Preconditions.checkArgument(prefix.endsWith("."), "The given prefix does not end with a period (" + prefix + ")"); Map result = Maps.newHashMap(); - synchronized(parameters) { + synchronized (parameters) { for (String key : parameters.keySet()) { if (key.startsWith(prefix)) { String name = key.substring(prefix.length()); @@ -129,7 +129,7 @@ public boolean containsKey(String key) { */ public Boolean getBoolean(String key, Boolean defaultValue) { String value = get(key); - if(value != null) { + if (value != null) { return Boolean.parseBoolean(value.trim()); } return defaultValue; @@ -158,7 +158,7 @@ public Boolean getBoolean(String key) { */ public Integer getInteger(String key, Integer defaultValue) { String value = get(key); - if(value != null) { + if (value != null) { return Integer.parseInt(value.trim()); } return defaultValue; @@ -187,7 +187,7 @@ public Integer getInteger(String key) { */ public Long getLong(String key, Long defaultValue) { String value = get(key); - if(value != null) { + if (value != null) { return Long.parseLong(value.trim()); } return defaultValue; @@ -227,7 +227,7 @@ public String getString(String key) { } private String get(String key, String defaultValue) { String result = parameters.get(key); - if(result != null) { + if (result != null) { return result; } return defaultValue; diff --git a/flume-ng-configuration/src/main/java/org/apache/flume/conf/BasicConfigurationConstants.java b/flume-ng-configuration/src/main/java/org/apache/flume/conf/BasicConfigurationConstants.java index d6aa33adb4..908912237c 100644 --- a/flume-ng-configuration/src/main/java/org/apache/flume/conf/BasicConfigurationConstants.java +++ b/flume-ng-configuration/src/main/java/org/apache/flume/conf/BasicConfigurationConstants.java @@ -22,7 +22,6 @@ public final class BasicConfigurationConstants { public static final String CONFIG_SOURCES_PREFIX = CONFIG_SOURCES + "."; public static final String CONFIG_SOURCE_CHANNELSELECTOR_PREFIX = "selector."; - public static final String CONFIG_SINKS = "sinks"; public static final String CONFIG_SINKS_PREFIX = CONFIG_SINKS + "."; public static final String CONFIG_SINK_PROCESSOR_PREFIX = "processor."; diff --git a/flume-ng-configuration/src/main/java/org/apache/flume/conf/ComponentConfiguration.java b/flume-ng-configuration/src/main/java/org/apache/flume/conf/ComponentConfiguration.java index 0e0614e5e2..477a3e6e6c 100644 --- a/flume-ng-configuration/src/main/java/org/apache/flume/conf/ComponentConfiguration.java +++ b/flume-ng-configuration/src/main/java/org/apache/flume/conf/ComponentConfiguration.java @@ -64,7 +64,7 @@ public void configure(Context context) throws ConfigurationException { failIfConfigured(); String confType = context.getString( BasicConfigurationConstants.CONFIG_TYPE); - if (confType != null && !confType.isEmpty()){ + if (confType != null && !confType.isEmpty()) { this.type = confType; } // Type can be set by child class constructors, so check if it was. @@ -74,12 +74,12 @@ public void configure(Context context) throws ConfigurationException { FlumeConfigurationErrorType.ATTRS_MISSING, ErrorOrWarning.ERROR)); throw new ConfigurationException( - "Component has no type. Cannot configure. "+ componentName); + "Component has no type. Cannot configure. " + componentName); } } protected void failIfConfigured() throws ConfigurationException { - if (configured){ + if (configured) { throw new ConfigurationException("Already configured component." + componentName); } @@ -134,12 +134,13 @@ public enum ComponentType { CHANNELSELECTOR("ChannelSelector"); private final String componentType; - private ComponentType(String type){ + + private ComponentType(String type) { componentType = type; } + public String getComponentType() { return componentType; } - } } diff --git a/flume-ng-configuration/src/main/java/org/apache/flume/conf/ComponentConfigurationFactory.java b/flume-ng-configuration/src/main/java/org/apache/flume/conf/ComponentConfigurationFactory.java index 0433c9ca58..16860c3773 100644 --- a/flume-ng-configuration/src/main/java/org/apache/flume/conf/ComponentConfigurationFactory.java +++ b/flume-ng-configuration/src/main/java/org/apache/flume/conf/ComponentConfigurationFactory.java @@ -27,9 +27,9 @@ import org.apache.flume.conf.source.SourceConfiguration.SourceConfigurationType; public class ComponentConfigurationFactory { + @SuppressWarnings("unchecked") - public static ComponentConfiguration - create(String name, String type, ComponentType component) + public static ComponentConfiguration create(String name, String type, ComponentType component) throws ConfigurationException { Class confType = null; @@ -43,7 +43,7 @@ public class ComponentConfigurationFactory { } catch (Exception ignored) { try { type = type.toUpperCase(Locale.ENGLISH); - switch(component){ + switch (component) { case SOURCE: return SourceConfigurationType.valueOf(type.toUpperCase(Locale.ENGLISH)) .getConfiguration(name); @@ -63,8 +63,7 @@ public class ComponentConfigurationFactory { return new SinkGroupConfiguration(name); default: throw new ConfigurationException( - "Cannot create configuration. Unknown Type specified: " + - type); + "Cannot create configuration. Unknown Type specified: " + type); } } catch (ConfigurationException e) { throw e; diff --git a/flume-ng-configuration/src/main/java/org/apache/flume/conf/FlumeConfiguration.java b/flume-ng-configuration/src/main/java/org/apache/flume/conf/FlumeConfiguration.java index 5997406c68..9b3a434db9 100644 --- a/flume-ng-configuration/src/main/java/org/apache/flume/conf/FlumeConfiguration.java +++ b/flume-ng-configuration/src/main/java/org/apache/flume/conf/FlumeConfiguration.java @@ -5,9 +5,9 @@ * licenses this file to you under the Apache License, Version 2.0 (the * "License"); you may not use this file except in compliance with the License. * You may obtain a copy of the License at - * + *

    * http://www.apache.org/licenses/LICENSE-2.0 - * + *

    * Unless required by applicable law or agreed to in writing, software * distributed under the License is distributed on an "AS IS" BASIS, WITHOUT * WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. See the @@ -16,6 +16,19 @@ */ package org.apache.flume.conf; +import org.apache.flume.Context; +import org.apache.flume.conf.ComponentConfiguration.ComponentType; +import org.apache.flume.conf.FlumeConfigurationError.ErrorOrWarning; +import org.apache.flume.conf.channel.ChannelConfiguration; +import org.apache.flume.conf.channel.ChannelType; +import org.apache.flume.conf.sink.SinkConfiguration; +import org.apache.flume.conf.sink.SinkGroupConfiguration; +import org.apache.flume.conf.sink.SinkType; +import org.apache.flume.conf.source.SourceConfiguration; +import org.apache.flume.conf.source.SourceType; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + import java.util.ArrayList; import java.util.Arrays; import java.util.Collections; @@ -30,19 +43,6 @@ import java.util.Set; import java.util.StringTokenizer; -import org.apache.flume.Context; -import org.apache.flume.conf.ComponentConfiguration.ComponentType; -import org.apache.flume.conf.FlumeConfigurationError.ErrorOrWarning; -import org.apache.flume.conf.channel.ChannelConfiguration; -import org.apache.flume.conf.channel.ChannelType; -import org.apache.flume.conf.sink.SinkConfiguration; -import org.apache.flume.conf.sink.SinkGroupConfiguration; -import org.apache.flume.conf.sink.SinkType; -import org.apache.flume.conf.source.SourceConfiguration; -import org.apache.flume.conf.source.SourceType; -import org.slf4j.Logger; -import org.slf4j.LoggerFactory; - /** *

    * FlumeConfiguration is an in memory representation of the hierarchical @@ -58,14 +58,13 @@ */ public class FlumeConfiguration { - private static final Logger logger = LoggerFactory - .getLogger(FlumeConfiguration.class); + private static final Logger logger = LoggerFactory.getLogger(FlumeConfiguration.class); private final Map agentConfigMap; private final LinkedList errors; - public static final String NEWLINE = System.getProperty("line.separator", - "\n"); + public static final String NEWLINE = System.getProperty("line.separator", "\n"); public static final String INDENTSTEP = " "; + /** * Creates a populated Flume Configuration object. * @deprecated please use the other constructor @@ -75,7 +74,7 @@ public FlumeConfiguration(Properties properties) { agentConfigMap = new HashMap(); errors = new LinkedList(); // Construct the in-memory component hierarchy - for(Object name : properties.keySet()) { + for (Object name : properties.keySet()) { Object value = properties.get(name); if (!addRawProperty(name.toString(), value.toString())) { logger.warn("Configuration property ignored: " + name + " = " + value); @@ -86,8 +85,8 @@ public FlumeConfiguration(Properties properties) { // validate and remove improperly configured components validateConfiguration(); - } + /** * Creates a populated Flume Configuration object. */ @@ -95,7 +94,7 @@ public FlumeConfiguration(Map properties) { agentConfigMap = new HashMap(); errors = new LinkedList(); // Construct the in-memory component hierarchy - for(String name : properties.keySet()) { + for (String name : properties.keySet()) { String value = properties.get(name); if (!addRawProperty(name, value)) { @@ -146,18 +145,18 @@ private boolean addRawProperty(String name, String value) { // Null names and values not supported if (name == null || value == null) { errors - .add(new FlumeConfigurationError("", "", - FlumeConfigurationErrorType.AGENT_NAME_MISSING, - ErrorOrWarning.ERROR)); + .add(new FlumeConfigurationError("", "", + FlumeConfigurationErrorType.AGENT_NAME_MISSING, + ErrorOrWarning.ERROR)); return false; } // Empty values are not supported if (value.trim().length() == 0) { errors - .add(new FlumeConfigurationError(name, "", - FlumeConfigurationErrorType.PROPERTY_VALUE_NULL, - ErrorOrWarning.ERROR)); + .add(new FlumeConfigurationError(name, "", + FlumeConfigurationErrorType.PROPERTY_VALUE_NULL, + ErrorOrWarning.ERROR)); return false; } @@ -170,9 +169,9 @@ private boolean addRawProperty(String name, String value) { // All configuration keys must have a prefix defined as agent name if (index == -1) { errors - .add(new FlumeConfigurationError(name, "", - FlumeConfigurationErrorType.AGENT_NAME_MISSING, - ErrorOrWarning.ERROR)); + .add(new FlumeConfigurationError(name, "", + FlumeConfigurationErrorType.AGENT_NAME_MISSING, + ErrorOrWarning.ERROR)); return false; } @@ -181,9 +180,9 @@ private boolean addRawProperty(String name, String value) { // Agent name must be specified for all properties if (agentName.length() == 0) { errors - .add(new FlumeConfigurationError(name, "", - FlumeConfigurationErrorType.AGENT_NAME_MISSING, - ErrorOrWarning.ERROR)); + .add(new FlumeConfigurationError(name, "", + FlumeConfigurationErrorType.AGENT_NAME_MISSING, + ErrorOrWarning.ERROR)); return false; } @@ -192,9 +191,9 @@ private boolean addRawProperty(String name, String value) { // Configuration key must be specified for every property if (configKey.length() == 0) { errors - .add(new FlumeConfigurationError(name, "", - FlumeConfigurationErrorType.PROPERTY_NAME_NULL, - ErrorOrWarning.ERROR)); + .add(new FlumeConfigurationError(name, "", + FlumeConfigurationErrorType.PROPERTY_NAME_NULL, + ErrorOrWarning.ERROR)); return false; } @@ -236,7 +235,7 @@ public static class AgentConfiguration { private final List errorList; private AgentConfiguration(String agentName, - List errorList) { + List errorList) { this.agentName = agentName; this.errorList = errorList; sourceConfigMap = new HashMap(); @@ -411,9 +410,6 @@ private SourceType getKnownSource(String type) { /** * If it is a known component it will do the full validation required for * that component, else it will do the validation required for that class. - * - * @param channelSet - * @return */ private Set validateChannels(Set channelSet) { Iterator iter = channelSet.iterator(); @@ -445,13 +441,12 @@ private Set validateChannels(Set channelSet) { if (channelContext != null) { // Get the configuration object for the channel: ChannelType chType = getKnownChannel(channelContext.getString( - BasicConfigurationConstants.CONFIG_TYPE)); + BasicConfigurationConstants.CONFIG_TYPE)); boolean configSpecified = false; String config = null; // Not a known channel - cannot do specific validation to this channel if (chType == null) { - config = channelContext.getString - (BasicConfigurationConstants.CONFIG_CONFIG); + config = channelContext.getString(BasicConfigurationConstants.CONFIG_CONFIG); if (config == null || config.isEmpty()) { config = "OTHER"; } else { @@ -470,14 +465,15 @@ private Set validateChannels(Set channelSet) { if (conf != null) { conf.configure(channelContext); } - if((configSpecified && conf.isNotFoundConfigClass()) || - !configSpecified){ + if ((configSpecified && conf.isNotFoundConfigClass()) || + !configSpecified) { newContextMap.put(channelName, channelContext); } else if (configSpecified) { channelConfigMap.put(channelName, conf); } - if (conf != null) + if (conf != null) { errorList.addAll(conf.getErrors()); + } } catch (ConfigurationException e) { // Could not configure channel - skip it. // No need to add to error list - already added before exception is @@ -504,7 +500,7 @@ private Set validateChannels(Set channelSet) { private Set validateSources(Set channelSet) { //Arrays.split() call will throw NPE if the sources string is empty - if(sources == null || sources.isEmpty()){ + if (sources == null || sources.isEmpty()) { logger.warn("Agent configuration for '" + agentName + "' has no sources."); errorList.add(new FlumeConfigurationError(agentName, @@ -570,17 +566,17 @@ private Set validateSources(Set channelSet) { channels.addAll(srcConf.getChannels()); } channels.retainAll(channelSet); - if(channels.isEmpty()){ + if (channels.isEmpty()) { throw new ConfigurationException( - "No Channels configured for " + sourceName); + "No Channels configured for " + sourceName); } srcContext.put(BasicConfigurationConstants.CONFIG_CHANNELS, - this.getSpaceDelimitedList(channels)); + this.getSpaceDelimitedList(channels)); } if ((configSpecified && srcConf.isNotFoundConfigClass()) || !configSpecified) { newContextMap.put(sourceName, srcContext); - } else if (configSpecified){ + } else if (configSpecified) { sourceConfigMap.put(sourceName, srcConf); } if (srcConf != null) errorList.addAll(srcConf.getErrors()); @@ -664,7 +660,7 @@ private Set validateSinks(Set channelSet) { BasicConfigurationConstants.CONFIG_CONFIG); if (config == null || config.isEmpty()) { config = "OTHER"; - } else{ + } else { configSpecified = true; } } else { @@ -681,9 +677,9 @@ private Set validateSinks(Set channelSet) { sinkConf.configure(sinkContext); } - if(!channelSet.contains(sinkConf.getChannel())){ + if (!channelSet.contains(sinkConf.getChannel())) { throw new ConfigurationException("Channel " + - sinkConf.getChannel() + " not in active set."); + sinkConf.getChannel() + " not in active set."); } if ((configSpecified && sinkConf.isNotFoundConfigClass()) || !configSpecified) { @@ -753,15 +749,15 @@ private Set validateGroups(Set sinkSet) { if (conf != null) errorList.addAll(conf.getErrors()); throw new ConfigurationException( "No available sinks for sinkgroup: " + sinkgroupName - + ". Sinkgroup will be removed"); + + ". Sinkgroup will be removed"); } } catch (ConfigurationException e) { iter.remove(); errorList - .add(new FlumeConfigurationError(agentName, sinkgroupName, - FlumeConfigurationErrorType.CONFIG_ERROR, - ErrorOrWarning.ERROR)); + .add(new FlumeConfigurationError(agentName, sinkgroupName, + FlumeConfigurationErrorType.CONFIG_ERROR, + ErrorOrWarning.ERROR)); logger.warn("Could not configure sink group " + sinkgroupName + " due to: " + e.getMessage(), e); } @@ -791,10 +787,10 @@ private Set validateGroups(Set sinkSet) { * @return List of sinks available and reserved for group */ private Set validGroupSinks(Set sinkSet, - Map usedSinks, SinkGroupConfiguration groupConf) { + Map usedSinks, + SinkGroupConfiguration groupConf) { Set groupSinks = - Collections - .synchronizedSet(new HashSet(groupConf.getSinks())); + Collections.synchronizedSet(new HashSet(groupConf.getSinks())); if (groupSinks.isEmpty()) return null; Iterator sinkIt = groupSinks.iterator(); @@ -866,40 +862,40 @@ public String getPrevalidationConfig() { public String getPostvalidationConfig() { StringBuilder sb = new StringBuilder( "AgentConfiguration created without Configuration stubs " + - "for which only basic syntactical validation was performed["); + "for which only basic syntactical validation was performed["); sb.append(agentName).append("]").append(NEWLINE); - if(!sourceContextMap.isEmpty() || + if (!sourceContextMap.isEmpty() || !sinkContextMap.isEmpty() || !channelContextMap.isEmpty()) { - if(!sourceContextMap.isEmpty()){ + if (!sourceContextMap.isEmpty()) { sb.append("SOURCES: ").append(sourceContextMap).append(NEWLINE); } - if(!channelContextMap.isEmpty()){ + if (!channelContextMap.isEmpty()) { sb.append("CHANNELS: ").append(channelContextMap).append(NEWLINE); } - if(!sinkContextMap.isEmpty()){ + if (!sinkContextMap.isEmpty()) { sb.append("SINKS: ").append(sinkContextMap).append(NEWLINE); } } - if(!sourceConfigMap.isEmpty() || + if (!sourceConfigMap.isEmpty() || !sinkConfigMap.isEmpty() || !channelConfigMap.isEmpty()) { sb.append("AgentConfiguration created with Configuration stubs " + "for which full validation was performed["); sb.append(agentName).append("]").append(NEWLINE); - if(!sourceConfigMap.isEmpty()){ + if (!sourceConfigMap.isEmpty()) { sb.append("SOURCES: ").append(sourceConfigMap).append(NEWLINE); } - if(!channelConfigMap.isEmpty()){ + if (!channelConfigMap.isEmpty()) { sb.append("CHANNELS: ").append(channelConfigMap).append(NEWLINE); } - if(!sinkConfigMap.isEmpty()){ + if (!sinkConfigMap.isEmpty()) { sb.append("SINKS: ").append(sinkConfigMap).append(NEWLINE); } } @@ -915,7 +911,7 @@ private boolean addProperty(String key, String value) { return true; } else { logger - .warn("Duplicate source list specified for agent: " + agentName); + .warn("Duplicate source list specified for agent: " + agentName); errorList.add(new FlumeConfigurationError(agentName, BasicConfigurationConstants.CONFIG_SOURCES, FlumeConfigurationErrorType.DUPLICATE_PROPERTY, @@ -965,7 +961,7 @@ private boolean addProperty(String key, String value) { return true; } else { logger - .warn("Duplicate sinkgroup list specfied for agent: " + agentName); + .warn("Duplicate sinkgroup list specfied for agent: " + agentName); errorList.add(new FlumeConfigurationError(agentName, BasicConfigurationConstants.CONFIG_SINKGROUPS, FlumeConfigurationErrorType.DUPLICATE_PROPERTY, diff --git a/flume-ng-configuration/src/main/java/org/apache/flume/conf/FlumeConfigurationError.java b/flume-ng-configuration/src/main/java/org/apache/flume/conf/FlumeConfigurationError.java index 94025a48b7..5fac11c5f2 100644 --- a/flume-ng-configuration/src/main/java/org/apache/flume/conf/FlumeConfigurationError.java +++ b/flume-ng-configuration/src/main/java/org/apache/flume/conf/FlumeConfigurationError.java @@ -32,7 +32,7 @@ public class FlumeConfigurationError { public FlumeConfigurationError(String component, String key, FlumeConfigurationErrorType error, ErrorOrWarning err) { this.error = err; - if (component != null){ + if (component != null) { this.componentName = component; } else { this.componentName = ""; diff --git a/flume-ng-configuration/src/main/java/org/apache/flume/conf/sink/SinkConfiguration.java b/flume-ng-configuration/src/main/java/org/apache/flume/conf/sink/SinkConfiguration.java index b34a3673b9..1acd2916f6 100644 --- a/flume-ng-configuration/src/main/java/org/apache/flume/conf/sink/SinkConfiguration.java +++ b/flume-ng-configuration/src/main/java/org/apache/flume/conf/sink/SinkConfiguration.java @@ -54,7 +54,7 @@ public void configure(Context context) throws ConfigurationException { } @Override - public String toString(int indentCount){ + public String toString(int indentCount) { StringBuilder indentSb = new StringBuilder(""); for (int i = 0; i < indentCount; i++) { @@ -187,7 +187,7 @@ public SinkConfiguration getConfiguration(String name) instance = new SinkConfiguration(name); // Let the caller know that this was created because of this exception. instance.setNotFoundConfigClass(); - } catch (Exception e){ + } catch (Exception e) { throw new ConfigurationException("Couldn't create configuration", e); } return instance; diff --git a/flume-ng-configuration/src/main/java/org/apache/flume/conf/sink/SinkGroupConfiguration.java b/flume-ng-configuration/src/main/java/org/apache/flume/conf/sink/SinkGroupConfiguration.java index 6b487e51a4..c3fc2bb8e8 100644 --- a/flume-ng-configuration/src/main/java/org/apache/flume/conf/sink/SinkGroupConfiguration.java +++ b/flume-ng-configuration/src/main/java/org/apache/flume/conf/sink/SinkGroupConfiguration.java @@ -92,8 +92,7 @@ private SinkProcessorType getKnownSinkProcessor(String type) { for (SinkProcessorType value : values) { if (value.toString().equalsIgnoreCase(type)) return value; String sinkProcessClassName = value.getSinkProcessorClassName(); - if (sinkProcessClassName != null - && sinkProcessClassName.equalsIgnoreCase(type)){ + if (sinkProcessClassName != null && sinkProcessClassName.equalsIgnoreCase(type)) { return value; } } diff --git a/flume-ng-configuration/src/main/java/org/apache/flume/conf/source/SourceConfiguration.java b/flume-ng-configuration/src/main/java/org/apache/flume/conf/source/SourceConfiguration.java index 068bd69b2d..8d7318e25c 100644 --- a/flume-ng-configuration/src/main/java/org/apache/flume/conf/source/SourceConfiguration.java +++ b/flume-ng-configuration/src/main/java/org/apache/flume/conf/source/SourceConfiguration.java @@ -113,11 +113,11 @@ public void configure(Context context) throws ConfigurationException { } @Override - public String toString(int indentCount){ + public String toString(int indentCount) { String basicStr = super.toString(indentCount); StringBuilder sb = new StringBuilder(); sb.append(basicStr).append("CHANNELS:"); - for(String channel : this.channels){ + for (String channel : this.channels) { sb.append(FlumeConfiguration.INDENTSTEP).append( channel).append(FlumeConfiguration.NEWLINE); } diff --git a/flume-ng-core/src/main/java/org/apache/flume/ChannelFactory.java b/flume-ng-core/src/main/java/org/apache/flume/ChannelFactory.java index 26af8e1efe..e08e4f3b2a 100644 --- a/flume-ng-core/src/main/java/org/apache/flume/ChannelFactory.java +++ b/flume-ng-core/src/main/java/org/apache/flume/ChannelFactory.java @@ -1,4 +1,4 @@ -/** +/* * Licensed to the Apache Software Foundation (ASF) under one * or more contributor license agreements. See the NOTICE file * distributed with this work for additional information @@ -17,12 +17,9 @@ */ package org.apache.flume; - public interface ChannelFactory { - public Channel create(String name, String type) throws FlumeException; - - public Class getClass(String type) - throws FlumeException; + Channel create(String name, String type) throws FlumeException; + Class getClass(String type) throws FlumeException; } diff --git a/flume-ng-core/src/main/java/org/apache/flume/ChannelSelector.java b/flume-ng-core/src/main/java/org/apache/flume/ChannelSelector.java index f86aec7b22..fba2dcb6e5 100644 --- a/flume-ng-core/src/main/java/org/apache/flume/ChannelSelector.java +++ b/flume-ng-core/src/main/java/org/apache/flume/ChannelSelector.java @@ -21,7 +21,6 @@ import java.util.List; import org.apache.flume.conf.Configurable; -import org.apache.flume.conf.ConfigurableComponent; /** *

    diff --git a/flume-ng-core/src/main/java/org/apache/flume/Clock.java b/flume-ng-core/src/main/java/org/apache/flume/Clock.java index fc719bc4d5..119df48bba 100644 --- a/flume-ng-core/src/main/java/org/apache/flume/Clock.java +++ b/flume-ng-core/src/main/java/org/apache/flume/Clock.java @@ -22,7 +22,5 @@ * Facade for System.currentTimeMillis for Testing */ public interface Clock { - - public long currentTimeMillis(); - + long currentTimeMillis(); } diff --git a/flume-ng-core/src/main/java/org/apache/flume/SinkFactory.java b/flume-ng-core/src/main/java/org/apache/flume/SinkFactory.java index e9e52a8dfd..db4a49b4bc 100644 --- a/flume-ng-core/src/main/java/org/apache/flume/SinkFactory.java +++ b/flume-ng-core/src/main/java/org/apache/flume/SinkFactory.java @@ -19,12 +19,10 @@ package org.apache.flume; - public interface SinkFactory { - - public Sink create(String name, String type) + Sink create(String name, String type) throws FlumeException; - public Class getClass(String type) - throws FlumeException; + Class getClass(String type) + throws FlumeException; } diff --git a/flume-ng-core/src/main/java/org/apache/flume/SinkRunner.java b/flume-ng-core/src/main/java/org/apache/flume/SinkRunner.java index c353d1fd00..f513bebc19 100644 --- a/flume-ng-core/src/main/java/org/apache/flume/SinkRunner.java +++ b/flume-ng-core/src/main/java/org/apache/flume/SinkRunner.java @@ -104,10 +104,8 @@ public void stop() { logger.debug("Waiting for runner thread to exit"); runnerThread.join(500); } catch (InterruptedException e) { - logger - .debug( - "Interrupted while waiting for runner thread to exit. Exception follows.", - e); + logger.debug("Interrupted while waiting for runner thread to exit. Exception follows.", + e); } } } diff --git a/flume-ng-core/src/main/java/org/apache/flume/SourceFactory.java b/flume-ng-core/src/main/java/org/apache/flume/SourceFactory.java index e147410c9a..946cc37891 100644 --- a/flume-ng-core/src/main/java/org/apache/flume/SourceFactory.java +++ b/flume-ng-core/src/main/java/org/apache/flume/SourceFactory.java @@ -19,12 +19,11 @@ package org.apache.flume; - public interface SourceFactory { - public Source create(String sourceName, String type) + Source create(String sourceName, String type) throws FlumeException; - public Class getClass(String type) - throws FlumeException; + Class getClass(String type) + throws FlumeException; } diff --git a/flume-ng-core/src/main/java/org/apache/flume/SourceRunner.java b/flume-ng-core/src/main/java/org/apache/flume/SourceRunner.java index 32461517e9..2687287f91 100644 --- a/flume-ng-core/src/main/java/org/apache/flume/SourceRunner.java +++ b/flume-ng-core/src/main/java/org/apache/flume/SourceRunner.java @@ -28,7 +28,7 @@ * * This is an abstract class used for instantiating derived classes. */ -abstract public class SourceRunner implements LifecycleAware { +public abstract class SourceRunner implements LifecycleAware { private Source source; diff --git a/flume-ng-core/src/main/java/org/apache/flume/SystemClock.java b/flume-ng-core/src/main/java/org/apache/flume/SystemClock.java index f17680713c..4e326ccae7 100644 --- a/flume-ng-core/src/main/java/org/apache/flume/SystemClock.java +++ b/flume-ng-core/src/main/java/org/apache/flume/SystemClock.java @@ -23,8 +23,8 @@ */ public class SystemClock implements Clock { - public long currentTimeMillis() { - return System.currentTimeMillis(); - } + public long currentTimeMillis() { + return System.currentTimeMillis(); + } } diff --git a/flume-ng-core/src/main/java/org/apache/flume/Transaction.java b/flume-ng-core/src/main/java/org/apache/flume/Transaction.java index 24f12a3c5d..ffb07bdfda 100644 --- a/flume-ng-core/src/main/java/org/apache/flume/Transaction.java +++ b/flume-ng-core/src/main/java/org/apache/flume/Transaction.java @@ -52,7 +52,8 @@ */ public interface Transaction { -public enum TransactionState {Started, Committed, RolledBack, Closed }; + enum TransactionState { Started, Committed, RolledBack, Closed } + /** *

    Starts a transaction boundary for the current channel operation. If a * transaction is already in progress, this method will join that transaction @@ -62,19 +63,19 @@ public enum TransactionState {Started, Committed, RolledBack, Closed }; * to ensure this can lead to dangling transactions and unpredictable results. *

    */ - public void begin(); + void begin(); /** * Indicates that the transaction can be successfully committed. It is * required that a transaction be in progress when this method is invoked. */ - public void commit(); + void commit(); /** * Indicates that the transaction can must be aborted. It is * required that a transaction be in progress when this method is invoked. */ - public void rollback(); + void rollback(); /** *

    Ends a transaction boundary for the current channel operation. If a @@ -86,5 +87,5 @@ public enum TransactionState {Started, Committed, RolledBack, Closed }; * to ensure this can lead to dangling transactions and unpredictable results. *

    */ - public void close(); + void close(); } diff --git a/flume-ng-core/src/main/java/org/apache/flume/annotations/Disposable.java b/flume-ng-core/src/main/java/org/apache/flume/annotations/Disposable.java index d8db82c878..10b18840b9 100644 --- a/flume-ng-core/src/main/java/org/apache/flume/annotations/Disposable.java +++ b/flume-ng-core/src/main/java/org/apache/flume/annotations/Disposable.java @@ -18,8 +18,10 @@ */ package org.apache.flume.annotations; + import java.lang.annotation.Target; import java.lang.annotation.Retention; + import static java.lang.annotation.RetentionPolicy.RUNTIME; import static java.lang.annotation.ElementType.TYPE; diff --git a/flume-ng-core/src/main/java/org/apache/flume/annotations/InterfaceStability.java b/flume-ng-core/src/main/java/org/apache/flume/annotations/InterfaceStability.java index e946ac7c32..70bc3471ad 100644 --- a/flume-ng-core/src/main/java/org/apache/flume/annotations/InterfaceStability.java +++ b/flume-ng-core/src/main/java/org/apache/flume/annotations/InterfaceStability.java @@ -29,6 +29,7 @@ *
  • Classes that are {@link Private} are to be considered unstable unless * a different InterfaceStability annotation states otherwise.
  • *
  • Incompatible changes must not be made to classes marked as stable.
  • + * */ @InterfaceAudience.Public @InterfaceStability.Evolving diff --git a/flume-ng-core/src/main/java/org/apache/flume/annotations/Recyclable.java b/flume-ng-core/src/main/java/org/apache/flume/annotations/Recyclable.java index a732c83552..bc9ed0e2d5 100644 --- a/flume-ng-core/src/main/java/org/apache/flume/annotations/Recyclable.java +++ b/flume-ng-core/src/main/java/org/apache/flume/annotations/Recyclable.java @@ -18,8 +18,10 @@ */ package org.apache.flume.annotations; + import java.lang.annotation.Target; import java.lang.annotation.Retention; + import static java.lang.annotation.RetentionPolicy.RUNTIME; import static java.lang.annotation.ElementType.TYPE; diff --git a/flume-ng-core/src/main/java/org/apache/flume/channel/AbstractChannel.java b/flume-ng-core/src/main/java/org/apache/flume/channel/AbstractChannel.java index b991650b09..750cd91016 100644 --- a/flume-ng-core/src/main/java/org/apache/flume/channel/AbstractChannel.java +++ b/flume-ng-core/src/main/java/org/apache/flume/channel/AbstractChannel.java @@ -67,12 +67,10 @@ public synchronized String getName() { } @Override - public void configure(Context context) { - - } + public void configure(Context context) {} public String toString() { - return this.getClass().getName() + "{name: " + name + "}"; + return this.getClass().getName() + "{name: " + name + "}"; } } diff --git a/flume-ng-core/src/main/java/org/apache/flume/channel/AbstractChannelSelector.java b/flume-ng-core/src/main/java/org/apache/flume/channel/AbstractChannelSelector.java index d69087fcf4..3b1e6e0dff 100644 --- a/flume-ng-core/src/main/java/org/apache/flume/channel/AbstractChannelSelector.java +++ b/flume-ng-core/src/main/java/org/apache/flume/channel/AbstractChannelSelector.java @@ -73,7 +73,7 @@ protected Map getChannelNameMap() { protected List getChannelListFromNames(String channels, Map channelNameMap) { List configuredChannels = new ArrayList(); - if(channels == null || channels.isEmpty()) { + if (channels == null || channels.isEmpty()) { return configuredChannels; } String[] chNames = channels.split(" "); diff --git a/flume-ng-core/src/main/java/org/apache/flume/channel/BasicTransactionSemantics.java b/flume-ng-core/src/main/java/org/apache/flume/channel/BasicTransactionSemantics.java index 403cbca535..20231f4cb6 100644 --- a/flume-ng-core/src/main/java/org/apache/flume/channel/BasicTransactionSemantics.java +++ b/flume-ng-core/src/main/java/org/apache/flume/channel/BasicTransactionSemantics.java @@ -210,6 +210,7 @@ public String toString() { * perform any further operations beyond closing it. *
    CLOSED
    *
    A closed transaction. No further operations are permitted.
    + * */ protected static enum State { NEW, OPEN, COMPLETED, CLOSED diff --git a/flume-ng-core/src/main/java/org/apache/flume/channel/ChannelSelectorFactory.java b/flume-ng-core/src/main/java/org/apache/flume/channel/ChannelSelectorFactory.java index 80b1453c82..fac6271397 100644 --- a/flume-ng-core/src/main/java/org/apache/flume/channel/ChannelSelectorFactory.java +++ b/flume-ng-core/src/main/java/org/apache/flume/channel/ChannelSelectorFactory.java @@ -56,7 +56,7 @@ public static ChannelSelector create(List channels, public static ChannelSelector create(List channels, ChannelSelectorConfiguration conf) { String type = ChannelSelectorType.REPLICATING.toString(); - if (conf != null){ + if (conf != null) { type = conf.getType(); } ChannelSelector selector = getSelectorForType(type); diff --git a/flume-ng-core/src/main/java/org/apache/flume/channel/DefaultChannelFactory.java b/flume-ng-core/src/main/java/org/apache/flume/channel/DefaultChannelFactory.java index 1dd124efb7..640fe2125e 100644 --- a/flume-ng-core/src/main/java/org/apache/flume/channel/DefaultChannelFactory.java +++ b/flume-ng-core/src/main/java/org/apache/flume/channel/DefaultChannelFactory.java @@ -51,8 +51,7 @@ public Channel create(String name, String type) throws FlumeException { @SuppressWarnings("unchecked") @Override - public Class getClass(String type) - throws FlumeException { + public Class getClass(String type) throws FlumeException { String channelClassName = type; ChannelType channelType = ChannelType.OTHER; try { diff --git a/flume-ng-core/src/main/java/org/apache/flume/channel/MemoryChannel.java b/flume-ng-core/src/main/java/org/apache/flume/channel/MemoryChannel.java index 6575d10b61..4393783b6c 100644 --- a/flume-ng-core/src/main/java/org/apache/flume/channel/MemoryChannel.java +++ b/flume-ng-core/src/main/java/org/apache/flume/channel/MemoryChannel.java @@ -18,12 +18,7 @@ */ package org.apache.flume.channel; -import java.util.concurrent.LinkedBlockingDeque; -import java.util.concurrent.Semaphore; -import java.util.concurrent.TimeUnit; - -import javax.annotation.concurrent.GuardedBy; - +import com.google.common.base.Preconditions; import org.apache.flume.ChannelException; import org.apache.flume.ChannelFullException; import org.apache.flume.Context; @@ -35,7 +30,10 @@ import org.slf4j.Logger; import org.slf4j.LoggerFactory; -import com.google.common.base.Preconditions; +import javax.annotation.concurrent.GuardedBy; +import java.util.concurrent.LinkedBlockingDeque; +import java.util.concurrent.Semaphore; +import java.util.concurrent.TimeUnit; /** *

    @@ -78,11 +76,11 @@ public MemoryTransaction(int transCapacity, ChannelCounter counter) { @Override protected void doPut(Event event) throws InterruptedException { channelCounter.incrementEventPutAttemptCount(); - int eventByteSize = (int)Math.ceil(estimateEventSize(event)/byteCapacitySlotSize); + int eventByteSize = (int) Math.ceil(estimateEventSize(event) / byteCapacitySlotSize); if (!putList.offer(event)) { throw new ChannelException( - "Put queue for MemoryTransaction of capacity " + + "Put queue for MemoryTransaction of capacity " + putList.size() + " full, consider committing more frequently, " + "increasing capacity or increasing thread count"); } @@ -92,23 +90,23 @@ protected void doPut(Event event) throws InterruptedException { @Override protected Event doTake() throws InterruptedException { channelCounter.incrementEventTakeAttemptCount(); - if(takeList.remainingCapacity() == 0) { + if (takeList.remainingCapacity() == 0) { throw new ChannelException("Take list for MemoryTransaction, capacity " + takeList.size() + " full, consider committing more frequently, " + "increasing capacity, or increasing thread count"); } - if(!queueStored.tryAcquire(keepAlive, TimeUnit.SECONDS)) { + if (!queueStored.tryAcquire(keepAlive, TimeUnit.SECONDS)) { return null; } Event event; - synchronized(queueLock) { + synchronized (queueLock) { event = queue.poll(); } Preconditions.checkNotNull(event, "Queue.poll returned NULL despite semaphore " + "signalling existence of entry"); takeList.put(event); - int eventByteSize = (int)Math.ceil(estimateEventSize(event)/byteCapacitySlotSize); + int eventByteSize = (int) Math.ceil(estimateEventSize(event) / byteCapacitySlotSize); takeByteCounter += eventByteSize; return event; @@ -117,15 +115,14 @@ protected Event doTake() throws InterruptedException { @Override protected void doCommit() throws InterruptedException { int remainingChange = takeList.size() - putList.size(); - if(remainingChange < 0) { - if(!bytesRemaining.tryAcquire(putByteCounter, keepAlive, - TimeUnit.SECONDS)) { + if (remainingChange < 0) { + if (!bytesRemaining.tryAcquire(putByteCounter, keepAlive, TimeUnit.SECONDS)) { throw new ChannelException("Cannot commit transaction. Byte capacity " + - "allocated to store event body " + byteCapacity * byteCapacitySlotSize + - "reached. Please increase heap space/byte capacity allocated to " + - "the channel as the sinks may not be keeping up with the sources"); + "allocated to store event body " + byteCapacity * byteCapacitySlotSize + + "reached. Please increase heap space/byte capacity allocated to " + + "the channel as the sinks may not be keeping up with the sources"); } - if(!queueRemaining.tryAcquire(-remainingChange, keepAlive, TimeUnit.SECONDS)) { + if (!queueRemaining.tryAcquire(-remainingChange, keepAlive, TimeUnit.SECONDS)) { bytesRemaining.release(putByteCounter); throw new ChannelFullException("Space for commit to queue couldn't be acquired." + " Sinks are likely not keeping up with sources, or the buffer size is too tight"); @@ -133,10 +130,10 @@ protected void doCommit() throws InterruptedException { } int puts = putList.size(); int takes = takeList.size(); - synchronized(queueLock) { - if(puts > 0 ) { - while(!putList.isEmpty()) { - if(!queue.offer(putList.removeFirst())) { + synchronized (queueLock) { + if (puts > 0) { + while (!putList.isEmpty()) { + if (!queue.offer(putList.removeFirst())) { throw new RuntimeException("Queue add failed, this shouldn't be able to happen"); } } @@ -149,7 +146,7 @@ protected void doCommit() throws InterruptedException { putByteCounter = 0; queueStored.release(puts); - if(remainingChange > 0) { + if (remainingChange > 0) { queueRemaining.release(remainingChange); } if (puts > 0) { @@ -165,10 +162,11 @@ protected void doCommit() throws InterruptedException { @Override protected void doRollback() { int takes = takeList.size(); - synchronized(queueLock) { - Preconditions.checkState(queue.remainingCapacity() >= takeList.size(), "Not enough space in memory channel " + + synchronized (queueLock) { + Preconditions.checkState(queue.remainingCapacity() >= takeList.size(), + "Not enough space in memory channel " + "queue to rollback takes. This should never happen, please report"); - while(!takeList.isEmpty()) { + while (!takeList.isEmpty()) { queue.addFirst(takeList.removeLast()); } putList.clear(); @@ -195,10 +193,12 @@ protected void doRollback() { // this allows local threads waiting for space in the queue to commit without denying access to the // shared lock to threads that would make more space on the queue private Semaphore queueRemaining; + // used to make "reservations" to grab data from the queue. // by using this we can block for a while to get data without locking all other threads out // like we would if we tried to use a blocking call on queue private Semaphore queueStored; + // maximum items in a transaction queue private volatile Integer transCapacity; private volatile int keepAlive; @@ -208,7 +208,6 @@ protected void doRollback() { private Semaphore bytesRemaining; private ChannelCounter channelCounter; - public MemoryChannel() { super(); } @@ -226,7 +225,7 @@ public void configure(Context context) { Integer capacity = null; try { capacity = context.getInteger("capacity", defaultCapacity); - } catch(NumberFormatException e) { + } catch (NumberFormatException e) { capacity = defaultCapacity; LOGGER.warn("Invalid capacity specified, initializing channel to " + "default capacity of {}", defaultCapacity); @@ -239,7 +238,7 @@ public void configure(Context context) { } try { transCapacity = context.getInteger("transactionCapacity", defaultTransCapacity); - } catch(NumberFormatException e) { + } catch (NumberFormatException e) { transCapacity = defaultTransCapacity; LOGGER.warn("Invalid transation capacity specified, initializing channel" + " to default capacity of {}", defaultTransCapacity); @@ -255,34 +254,37 @@ public void configure(Context context) { "the capacity."); try { - byteCapacityBufferPercentage = context.getInteger("byteCapacityBufferPercentage", defaultByteCapacityBufferPercentage); - } catch(NumberFormatException e) { + byteCapacityBufferPercentage = context.getInteger("byteCapacityBufferPercentage", + defaultByteCapacityBufferPercentage); + } catch (NumberFormatException e) { byteCapacityBufferPercentage = defaultByteCapacityBufferPercentage; } try { - byteCapacity = (int)((context.getLong("byteCapacity", defaultByteCapacity).longValue() * (1 - byteCapacityBufferPercentage * .01 )) /byteCapacitySlotSize); + byteCapacity = (int) ((context.getLong("byteCapacity", defaultByteCapacity).longValue() * + (1 - byteCapacityBufferPercentage * .01)) / byteCapacitySlotSize); if (byteCapacity < 1) { byteCapacity = Integer.MAX_VALUE; } - } catch(NumberFormatException e) { - byteCapacity = (int)((defaultByteCapacity * (1 - byteCapacityBufferPercentage * .01 )) /byteCapacitySlotSize); + } catch (NumberFormatException e) { + byteCapacity = (int) ((defaultByteCapacity * (1 - byteCapacityBufferPercentage * .01)) / + byteCapacitySlotSize); } try { keepAlive = context.getInteger("keep-alive", defaultKeepAlive); - } catch(NumberFormatException e) { + } catch (NumberFormatException e) { keepAlive = defaultKeepAlive; } - if(queue != null) { + if (queue != null) { try { resizeQueue(capacity); } catch (InterruptedException e) { Thread.currentThread().interrupt(); } } else { - synchronized(queueLock) { + synchronized (queueLock) { queue = new LinkedBlockingDeque(capacity); queueRemaining = new Semaphore(capacity); queueStored = new Semaphore(0); @@ -298,7 +300,8 @@ public void configure(Context context) { lastByteCapacity = byteCapacity; } else { try { - if(!bytesRemaining.tryAcquire(lastByteCapacity - byteCapacity, keepAlive, TimeUnit.SECONDS)) { + if (!bytesRemaining.tryAcquire(lastByteCapacity - byteCapacity, keepAlive, + TimeUnit.SECONDS)) { LOGGER.warn("Couldn't acquire permits to downsize the byte capacity, resizing has been aborted"); } else { lastByteCapacity = byteCapacity; @@ -316,24 +319,24 @@ public void configure(Context context) { private void resizeQueue(int capacity) throws InterruptedException { int oldCapacity; - synchronized(queueLock) { + synchronized (queueLock) { oldCapacity = queue.size() + queue.remainingCapacity(); } - if(oldCapacity == capacity) { + if (oldCapacity == capacity) { return; } else if (oldCapacity > capacity) { - if(!queueRemaining.tryAcquire(oldCapacity - capacity, keepAlive, TimeUnit.SECONDS)) { + if (!queueRemaining.tryAcquire(oldCapacity - capacity, keepAlive, TimeUnit.SECONDS)) { LOGGER.warn("Couldn't acquire permits to downsize the queue, resizing has been aborted"); } else { - synchronized(queueLock) { + synchronized (queueLock) { LinkedBlockingDeque newQueue = new LinkedBlockingDeque(capacity); newQueue.addAll(queue); queue = newQueue; } } } else { - synchronized(queueLock) { + synchronized (queueLock) { LinkedBlockingDeque newQueue = new LinkedBlockingDeque(capacity); newQueue.addAll(queue); queue = newQueue; @@ -363,10 +366,9 @@ protected BasicTransactionSemantics createTransaction() { return new MemoryTransaction(transCapacity, channelCounter); } - private long estimateEventSize(Event event) - { + private long estimateEventSize(Event event) { byte[] body = event.getBody(); - if(body != null && body.length != 0) { + if (body != null && body.length != 0) { return body.length; } //Each event occupies at least 1 slot, so return 1. diff --git a/flume-ng-core/src/main/java/org/apache/flume/channel/MultiplexingChannelSelector.java b/flume-ng-core/src/main/java/org/apache/flume/channel/MultiplexingChannelSelector.java index 3e32804483..e57a22354d 100644 --- a/flume-ng-core/src/main/java/org/apache/flume/channel/MultiplexingChannelSelector.java +++ b/flume-ng-core/src/main/java/org/apache/flume/channel/MultiplexingChannelSelector.java @@ -40,8 +40,7 @@ public class MultiplexingChannelSelector extends AbstractChannelSelector { public static final String CONFIG_PREFIX_OPTIONAL = "optional"; @SuppressWarnings("unused") - private static final Logger LOG = LoggerFactory - .getLogger(MultiplexingChannelSelector.class); + private static final Logger LOG = LoggerFactory.getLogger(MultiplexingChannelSelector.class); private static final List EMPTY_LIST = Collections.emptyList(); @@ -51,6 +50,7 @@ public class MultiplexingChannelSelector extends AbstractChannelSelector { private Map> channelMapping; private Map> optionalChannels; private List defaultChannels; + @Override public List getRequiredChannels(Event event) { String headerValue = event.getHeaders().get(headerName); @@ -74,7 +74,7 @@ public List getOptionalChannels(Event event) { String hdr = event.getHeaders().get(headerName); List channels = optionalChannels.get(hdr); - if(channels == null) { + if (channels == null) { channels = EMPTY_LIST; } return channels; @@ -128,7 +128,7 @@ public void configure(Context context) { List reqdChannels = channelMapping.get(hdr); //Check if there are required channels, else defaults to default channels - if(reqdChannels == null || reqdChannels.isEmpty()) { + if (reqdChannels == null || reqdChannels.isEmpty()) { reqdChannels = defaultChannels; } for (Channel c : reqdChannels) { diff --git a/flume-ng-core/src/main/java/org/apache/flume/channel/PseudoTxnMemoryChannel.java b/flume-ng-core/src/main/java/org/apache/flume/channel/PseudoTxnMemoryChannel.java index cc391c4bc6..22e100238e 100644 --- a/flume-ng-core/src/main/java/org/apache/flume/channel/PseudoTxnMemoryChannel.java +++ b/flume-ng-core/src/main/java/org/apache/flume/channel/PseudoTxnMemoryChannel.java @@ -100,13 +100,13 @@ public void configure(Context context) { } queue = new ArrayBlockingQueue(capacity); - if(channelCounter == null) { + if (channelCounter == null) { channelCounter = new ChannelCounter(getName()); } } @Override - public void start(){ + public void start() { channelCounter.start(); channelCounter.setChannelSize(queue.size()); channelCounter.setChannelSize( @@ -115,7 +115,7 @@ public void start(){ } @Override - public void stop(){ + public void stop() { channelCounter.setChannelSize(queue.size()); channelCounter.stop(); super.stop(); diff --git a/flume-ng-core/src/main/java/org/apache/flume/channel/ReplicatingChannelSelector.java b/flume-ng-core/src/main/java/org/apache/flume/channel/ReplicatingChannelSelector.java index 8a0d2bd968..47c6aca386 100644 --- a/flume-ng-core/src/main/java/org/apache/flume/channel/ReplicatingChannelSelector.java +++ b/flume-ng-core/src/main/java/org/apache/flume/channel/ReplicatingChannelSelector.java @@ -47,7 +47,7 @@ public List getRequiredChannels(Event event) { * configure method. It is conceiveable that custom component tests too * do that. So in that case, revert to old behavior. */ - if(requiredChannels == null) { + if (requiredChannels == null) { return getAllChannels(); } return requiredChannels; @@ -63,8 +63,8 @@ public void configure(Context context) { String optionalList = context.getString(CONFIG_OPTIONAL); requiredChannels = new ArrayList(getAllChannels()); Map channelNameMap = getChannelNameMap(); - if(optionalList != null && !optionalList.isEmpty()) { - for(String optional : optionalList.split("\\s+")) { + if (optionalList != null && !optionalList.isEmpty()) { + for (String optional : optionalList.split("\\s+")) { Channel optionalChannel = channelNameMap.get(optional); requiredChannels.remove(optionalChannel); if (!optionalChannels.contains(optionalChannel)) { diff --git a/flume-ng-core/src/main/java/org/apache/flume/client/avro/AvroCLIClient.java b/flume-ng-core/src/main/java/org/apache/flume/client/avro/AvroCLIClient.java index 3c8c267145..dd2aeef8ab 100644 --- a/flume-ng-core/src/main/java/org/apache/flume/client/avro/AvroCLIClient.java +++ b/flume-ng-core/src/main/java/org/apache/flume/client/avro/AvroCLIClient.java @@ -98,8 +98,7 @@ private void parseHeaders(CommandLine commandLine) { for (Map.Entry propertiesEntry : properties.entrySet()) { String key = (String) propertiesEntry.getKey(); String value = (String) propertiesEntry.getValue(); - logger.debug("Inserting Header Key [" + key + "] header value [" + - value + "]"); + logger.debug("Inserting Header Key [" + key + "] header value [" + value + "]"); headers.put(key, value); } } @@ -108,12 +107,12 @@ private void parseHeaders(CommandLine commandLine) { return; } finally { if (fs != null) { - try { - fs.close(); - }catch (Exception e) { - logger.error("Unable to close headerFile", e); - return; - } + try { + fs.close(); + } catch (Exception e) { + logger.error("Unable to close headerFile", e); + return; + } } } } @@ -179,7 +178,7 @@ private boolean parseCommandLine(String[] args) throws ParseException { fileName = commandLine.getOptionValue("filename"); dirName = commandLine.getOptionValue("dirname"); - if (commandLine.hasOption("headerFile")){ + if (commandLine.hasOption("headerFile")) { parseHeaders(commandLine); } diff --git a/flume-ng-core/src/main/java/org/apache/flume/client/avro/ReliableSpoolingFileEventReader.java b/flume-ng-core/src/main/java/org/apache/flume/client/avro/ReliableSpoolingFileEventReader.java index 36d80f0d00..4dc0207723 100644 --- a/flume-ng-core/src/main/java/org/apache/flume/client/avro/ReliableSpoolingFileEventReader.java +++ b/flume-ng-core/src/main/java/org/apache/flume/client/avro/ReliableSpoolingFileEventReader.java @@ -24,15 +24,18 @@ import com.google.common.base.Optional; import com.google.common.base.Preconditions; import com.google.common.io.Files; -import org.apache.commons.io.FileUtils; -import org.apache.commons.io.filefilter.IOFileFilter; -import org.apache.commons.lang.StringUtils; import org.apache.flume.Context; import org.apache.flume.Event; import org.apache.flume.FlumeException; import org.apache.flume.annotations.InterfaceAudience; import org.apache.flume.annotations.InterfaceStability; -import org.apache.flume.serialization.*; +import org.apache.flume.serialization.DecodeErrorPolicy; +import org.apache.flume.serialization.DurablePositionTracker; +import org.apache.flume.serialization.EventDeserializer; +import org.apache.flume.serialization.EventDeserializerFactory; +import org.apache.flume.serialization.PositionTracker; +import org.apache.flume.serialization.ResettableFileInputStream; +import org.apache.flume.serialization.ResettableInputStream; import org.apache.flume.source.SpoolDirectorySourceConfigurationConstants; import org.apache.flume.source.SpoolDirectorySourceConfigurationConstants.ConsumeOrder; import org.apache.flume.tools.PlatformDetect; @@ -44,9 +47,12 @@ import java.io.FileNotFoundException; import java.io.IOException; import java.nio.charset.Charset; -import java.util.*; -import java.util.regex.Pattern; import java.util.ArrayList; +import java.util.Collections; +import java.util.Iterator; +import java.util.List; +import java.util.Locale; +import java.util.regex.Pattern; /** *

    A {@link ReliableEventReader} which reads log data from files stored @@ -137,9 +143,11 @@ private ReliableSpoolingFileEventReader(File spoolDirectory, if (logger.isDebugEnabled()) { logger.debug("Initializing {} with directory={}, metaDir={}, " + - "deserializer={}", - new Object[] { ReliableSpoolingFileEventReader.class.getSimpleName(), - spoolDirectory, trackerDirPath, deserializerType }); + "deserializer={}", + new Object[] { + ReliableSpoolingFileEventReader.class.getSimpleName(), + spoolDirectory, trackerDirPath, deserializerType + }); } // Verify directory exists and is readable/writable @@ -202,7 +210,7 @@ private ReliableSpoolingFileEventReader(File spoolDirectory, this.metaFile = new File(trackerDirectory, metaFileName); - if(metaFile.exists() && metaFile.length() == 0) { + if (metaFile.exists() && metaFile.length() == 0) { deleteMetaFile(); } } @@ -214,7 +222,8 @@ private ReliableSpoolingFileEventReader(File spoolDirectory, public boolean accept(File candidate) { if (candidate.isDirectory()) { String directoryName = candidate.getName(); - if (!recursiveDirectorySearch || directoryName.startsWith(".") || + if (!recursiveDirectorySearch || + directoryName.startsWith(".") || ignorePattern.matcher(directoryName).matches()) { return false; @@ -222,7 +231,8 @@ public boolean accept(File candidate) { return true; } String fileName = candidate.getName(); - if (fileName.endsWith(completedSuffix) || fileName.startsWith(".") || + if (fileName.endsWith(completedSuffix) || + fileName.startsWith(".") || ignorePattern.matcher(fileName).matches()) { return false; } @@ -243,11 +253,10 @@ private List getCandidateFiles(File directory) { return candidateFiles; } - for(File file : directory.listFiles(filter)){ + for (File file : directory.listFiles(filter)) { if (file.isDirectory()) { candidateFiles.addAll(getCandidateFiles(file)); - } - else { + } else { candidateFiles.add(file); } } @@ -555,8 +564,8 @@ private Optional openFile(File file) { new ResettableFileInputStream(file, tracker, ResettableFileInputStream.DEFAULT_BUF_SIZE, inputCharset, decodeErrorPolicy); - EventDeserializer deserializer = EventDeserializerFactory.getInstance - (deserializerType, deserializerContext, in); + EventDeserializer deserializer = + EventDeserializerFactory.getInstance(deserializerType, deserializerContext, in); return Optional.of(new FileInfo(file, deserializer)); } catch (FileNotFoundException e) { @@ -589,10 +598,21 @@ public FileInfo(File file, EventDeserializer deserializer) { this.deserializer = deserializer; } - public long getLength() { return length; } - public long getLastModified() { return lastModified; } - public EventDeserializer getDeserializer() { return deserializer; } - public File getFile() { return file; } + public long getLength() { + return length; + } + + public long getLastModified() { + return lastModified; + } + + public EventDeserializer getDeserializer() { + return deserializer; + } + + public File getFile() { + return file; + } } @InterfaceAudience.Private diff --git a/flume-ng-core/src/main/java/org/apache/flume/event/EventHelper.java b/flume-ng-core/src/main/java/org/apache/flume/event/EventHelper.java index a2c0a578a7..0e3398efa1 100644 --- a/flume-ng-core/src/main/java/org/apache/flume/event/EventHelper.java +++ b/flume-ng-core/src/main/java/org/apache/flume/event/EventHelper.java @@ -53,18 +53,18 @@ public static String dumpEvent(Event event, int maxBytes) { HexDump.dump(data, 0, out, 0); String hexDump = new String(out.toByteArray()); // remove offset since it's not relevant for such a small dataset - if(hexDump.startsWith(HEXDUMP_OFFSET)) { + if (hexDump.startsWith(HEXDUMP_OFFSET)) { hexDump = hexDump.substring(HEXDUMP_OFFSET.length()); } buffer.append(hexDump); } catch (Exception e) { - if(LOGGER.isInfoEnabled()) { - LOGGER.info("Exception while dumping event", e); - } + if (LOGGER.isInfoEnabled()) { + LOGGER.info("Exception while dumping event", e); + } buffer.append("...Exception while dumping: ").append(e.getMessage()); } String result = buffer.toString(); - if(result.endsWith(EOL) && buffer.length() > EOL.length()) { + if (result.endsWith(EOL) && buffer.length() > EOL.length()) { buffer.delete(buffer.length() - EOL.length(), buffer.length()).toString(); } } diff --git a/flume-ng-core/src/main/java/org/apache/flume/formatter/output/BucketPath.java b/flume-ng-core/src/main/java/org/apache/flume/formatter/output/BucketPath.java index e565192739..b2fe3f0693 100644 --- a/flume-ng-core/src/main/java/org/apache/flume/formatter/output/BucketPath.java +++ b/flume-ng-core/src/main/java/org/apache/flume/formatter/output/BucketPath.java @@ -18,32 +18,29 @@ package org.apache.flume.formatter.output; +import com.google.common.annotations.VisibleForTesting; +import com.google.common.base.Preconditions; +import org.apache.flume.Clock; +import org.apache.flume.SystemClock; +import org.apache.flume.tools.TimestampRoundDownUtil; + import java.text.SimpleDateFormat; import java.util.Calendar; import java.util.Date; import java.util.HashMap; import java.util.Map; import java.util.TimeZone; -import java.util.concurrent.ConcurrentHashMap; import java.util.regex.Matcher; import java.util.regex.Pattern; -import com.google.common.annotations.VisibleForTesting; - -import org.apache.flume.Clock; -import org.apache.flume.SystemClock; -import org.apache.flume.tools.TimestampRoundDownUtil; - -import com.google.common.base.Preconditions; - public class BucketPath { /** * These are useful to other classes which might want to search for tags in * strings. */ - final public static String TAG_REGEX = "\\%(\\w|\\%)|\\%\\{([\\w\\.-]+)\\}"; - final public static Pattern tagPattern = Pattern.compile(TAG_REGEX); + public static final String TAG_REGEX = "\\%(\\w|\\%)|\\%\\{([\\w\\.-]+)\\}"; + public static final Pattern tagPattern = Pattern.compile(TAG_REGEX); private static Clock clock = new SystemClock(); @@ -62,57 +59,57 @@ public static boolean containsTag(String in) { public static String expandShorthand(char c) { // It's a date switch (c) { - case 'a': - return "weekday_short"; - case 'A': - return "weekday_full"; - case 'b': - return "monthname_short"; - case 'B': - return "monthname_full"; - case 'c': - return "datetime"; - case 'd': - return "day_of_month_xx"; // two digit - case 'e': - return "day_of_month_x"; // 1 or 2 digit - case 'D': - return "date_short"; // "MM/dd/yy"; - case 'H': - return "hour_24_xx"; - case 'I': - return "hour_12_xx"; - case 'j': - return "day_of_year_xxx"; // three digits - case 'k': - return "hour_24"; // 1 or 2 digits - case 'l': - return "hour_12"; // 1 or 2 digits - case 'm': - return "month_xx"; - case 'n': - return "month_x"; // 1 or 2 digits - case 'M': - return "minute_xx"; - case 'p': - return "am_pm"; - case 's': - return "unix_seconds"; - case 'S': - return "seconds_xx"; - case 't': - // This is different from unix date (which would insert a tab character - // here) - return "unix_millis"; - case 'y': - return "year_xx"; - case 'Y': - return "year_xxxx"; - case 'z': - return "timezone_delta"; - default: -// LOG.warn("Unrecognized escape in event format string: %" + c); - return "" + c; + case 'a': + return "weekday_short"; + case 'A': + return "weekday_full"; + case 'b': + return "monthname_short"; + case 'B': + return "monthname_full"; + case 'c': + return "datetime"; + case 'd': + return "day_of_month_xx"; // two digit + case 'e': + return "day_of_month_x"; // 1 or 2 digit + case 'D': + return "date_short"; // "MM/dd/yy"; + case 'H': + return "hour_24_xx"; + case 'I': + return "hour_12_xx"; + case 'j': + return "day_of_year_xxx"; // three digits + case 'k': + return "hour_24"; // 1 or 2 digits + case 'l': + return "hour_12"; // 1 or 2 digits + case 'm': + return "month_xx"; + case 'n': + return "month_x"; // 1 or 2 digits + case 'M': + return "minute_xx"; + case 'p': + return "am_pm"; + case 's': + return "unix_seconds"; + case 'S': + return "seconds_xx"; + case 't': + // This is different from unix date (which would insert a tab character + // here) + return "unix_millis"; + case 'y': + return "year_xx"; + case 'Y': + return "year_xxxx"; + case 'z': + return "timezone_delta"; + default: + // LOG.warn("Unrecognized escape in event format string: %" + c); + return "" + c; } } @@ -181,8 +178,8 @@ public static String replaceShorthand(char c, Map headers, @VisibleForTesting @Deprecated public static String replaceShorthand(char c, Map headers, - TimeZone timeZone, boolean needRounding, int unit, int roundDown, - boolean useLocalTimestamp) { + TimeZone timeZone, boolean needRounding, int unit, int roundDown, + boolean useLocalTimestamp) { long ts = 0; if (useLocalTimestamp) { ts = clock.currentTimeMillis(); @@ -191,27 +188,27 @@ public static String replaceShorthand(char c, Map headers, roundDown, false, ts); } - protected static final ThreadLocal> simpleDateFormatCache = new ThreadLocal>() { - - @Override - protected HashMap initialValue() { - return new HashMap(); - } - }; - + protected static final ThreadLocal> simpleDateFormatCache = + new ThreadLocal>() { + + @Override + protected HashMap initialValue() { + return new HashMap(); + } + }; + protected static SimpleDateFormat getSimpleDateFormat(String string) { - HashMap localCache = simpleDateFormatCache.get(); - - SimpleDateFormat simpleDateFormat = localCache.get(string); - if (simpleDateFormat == null) { - simpleDateFormat = new SimpleDateFormat(string); - localCache.put(string, simpleDateFormat); - simpleDateFormatCache.set(localCache); - } - - return simpleDateFormat; + HashMap localCache = simpleDateFormatCache.get(); + + SimpleDateFormat simpleDateFormat = localCache.get(string); + if (simpleDateFormat == null) { + simpleDateFormat = new SimpleDateFormat(string); + localCache.put(string, simpleDateFormat); + simpleDateFormatCache.set(localCache); + } + + return simpleDateFormat; } - /** * Not intended as a public API @@ -223,10 +220,10 @@ protected static String replaceShorthand(char c, Map headers, String timestampHeader = null; try { - if(!useLocalTimestamp) { + if (!useLocalTimestamp) { timestampHeader = headers.get("timestamp"); Preconditions.checkNotNull(timestampHeader, "Expected timestamp in " + - "the Flume event headers, but it was null"); + "the Flume event headers, but it was null"); ts = Long.valueOf(timestampHeader); } else { timestampHeader = String.valueOf(ts); @@ -238,87 +235,87 @@ protected static String replaceShorthand(char c, Map headers, + " TimestampInterceptor source interceptor).", e); } - if(needRounding){ + if (needRounding) { ts = roundDown(roundDown, unit, ts); } // It's a date String formatString = ""; switch (c) { - case '%': - return "%"; - case 'a': - formatString = "EEE"; - break; - case 'A': - formatString = "EEEE"; - break; - case 'b': - formatString = "MMM"; - break; - case 'B': - formatString = "MMMM"; - break; - case 'c': - formatString = "EEE MMM d HH:mm:ss yyyy"; - break; - case 'd': - formatString = "dd"; - break; - case 'e': - formatString = "d"; - break; - case 'D': - formatString = "MM/dd/yy"; - break; - case 'H': - formatString = "HH"; - break; - case 'I': - formatString = "hh"; - break; - case 'j': - formatString = "DDD"; - break; - case 'k': - formatString = "H"; - break; - case 'l': - formatString = "h"; - break; - case 'm': - formatString = "MM"; - break; - case 'M': - formatString = "mm"; - break; - case 'n': - formatString = "M"; - break; - case 'p': - formatString = "a"; - break; - case 's': - return "" + (ts/1000); - case 'S': - formatString = "ss"; - break; - case 't': - // This is different from unix date (which would insert a tab character - // here) - return timestampHeader; - case 'y': - formatString = "yy"; - break; - case 'Y': - formatString = "yyyy"; - break; - case 'z': - formatString = "ZZZ"; - break; - default: -// LOG.warn("Unrecognized escape in event format string: %" + c); - return ""; + case '%': + return "%"; + case 'a': + formatString = "EEE"; + break; + case 'A': + formatString = "EEEE"; + break; + case 'b': + formatString = "MMM"; + break; + case 'B': + formatString = "MMMM"; + break; + case 'c': + formatString = "EEE MMM d HH:mm:ss yyyy"; + break; + case 'd': + formatString = "dd"; + break; + case 'e': + formatString = "d"; + break; + case 'D': + formatString = "MM/dd/yy"; + break; + case 'H': + formatString = "HH"; + break; + case 'I': + formatString = "hh"; + break; + case 'j': + formatString = "DDD"; + break; + case 'k': + formatString = "H"; + break; + case 'l': + formatString = "h"; + break; + case 'm': + formatString = "MM"; + break; + case 'M': + formatString = "mm"; + break; + case 'n': + formatString = "M"; + break; + case 'p': + formatString = "a"; + break; + case 's': + return "" + (ts / 1000); + case 'S': + formatString = "ss"; + break; + case 't': + // This is different from unix date (which would insert a tab character + // here) + return timestampHeader; + case 'y': + formatString = "yy"; + break; + case 'Y': + formatString = "yyyy"; + break; + case 'z': + formatString = "ZZZ"; + break; + default: + // LOG.warn("Unrecognized escape in event format string: %" + c); + return ""; } SimpleDateFormat format = getSimpleDateFormat(formatString); @@ -332,9 +329,9 @@ protected static String replaceShorthand(char c, Map headers, return format.format(date); } - private static long roundDown(int roundDown, int unit, long ts){ + private static long roundDown(int roundDown, int unit, long ts) { long timestamp = ts; - if(roundDown <= 0){ + if (roundDown <= 0) { roundDown = 1; } switch (unit) { @@ -366,7 +363,7 @@ private static long roundDown(int roundDown, int unit, long ts){ * TODO(henry): we may want to consider taking this out of Event and into a * more general class when we get more use cases for this pattern. */ - public static String escapeString(String in, Map headers){ + public static String escapeString(String in, Map headers) { return escapeString(in, headers, false, 0, 0); } @@ -404,8 +401,8 @@ public static String escapeString(String in, Map headers, * @return Escaped string. */ public static String escapeString(String in, Map headers, - TimeZone timeZone, boolean needRounding, int unit, int roundDown, - boolean useLocalTimeStamp) { + TimeZone timeZone, boolean needRounding, int unit, int roundDown, + boolean useLocalTimeStamp) { long ts = clock.currentTimeMillis(); diff --git a/flume-ng-core/src/main/java/org/apache/flume/formatter/output/DefaultPathManager.java b/flume-ng-core/src/main/java/org/apache/flume/formatter/output/DefaultPathManager.java index 176db7f9cf..a6e203a694 100644 --- a/flume-ng-core/src/main/java/org/apache/flume/formatter/output/DefaultPathManager.java +++ b/flume-ng-core/src/main/java/org/apache/flume/formatter/output/DefaultPathManager.java @@ -51,7 +51,7 @@ public File nextFile() { sb.append(filePrefix).append(seriesTimestamp).append("-"); sb.append(fileIndex.incrementAndGet()); if (extension.length() > 0) { - sb.append(".").append(extension); + sb.append(".").append(extension); } currentFile = new File(baseDirectory, sb.toString()); diff --git a/flume-ng-core/src/main/java/org/apache/flume/formatter/output/PathManager.java b/flume-ng-core/src/main/java/org/apache/flume/formatter/output/PathManager.java index 5a3066a1d5..baad5d0870 100644 --- a/flume-ng-core/src/main/java/org/apache/flume/formatter/output/PathManager.java +++ b/flume-ng-core/src/main/java/org/apache/flume/formatter/output/PathManager.java @@ -25,26 +25,26 @@ * Creates the files used by the RollingFileSink. */ public interface PathManager { - /** - * {@link Context} prefix - */ - public static String CTX_PREFIX = "pathManager."; + /** + * {@link Context} prefix + */ + public static String CTX_PREFIX = "pathManager."; - File nextFile(); + File nextFile(); - File getCurrentFile(); + File getCurrentFile(); - void rotate(); + void rotate(); - File getBaseDirectory(); + File getBaseDirectory(); - void setBaseDirectory(File baseDirectory); + void setBaseDirectory(File baseDirectory); - /** - * Knows how to construct this path manager.
    - * Note: Implementations MUST provide a public a no-arg constructor. - */ - public interface Builder { - public PathManager build(Context context); - } + /** + * Knows how to construct this path manager.
    + * Note: Implementations MUST provide a public a no-arg constructor. + */ + public interface Builder { + public PathManager build(Context context); + } } diff --git a/flume-ng-core/src/main/java/org/apache/flume/formatter/output/PathManagerFactory.java b/flume-ng-core/src/main/java/org/apache/flume/formatter/output/PathManagerFactory.java index 4dbe08314a..4446e5ff9a 100644 --- a/flume-ng-core/src/main/java/org/apache/flume/formatter/output/PathManagerFactory.java +++ b/flume-ng-core/src/main/java/org/apache/flume/formatter/output/PathManagerFactory.java @@ -19,64 +19,65 @@ package org.apache.flume.formatter.output; import com.google.common.base.Preconditions; -import java.util.Locale; import org.apache.flume.Context; import org.apache.flume.FlumeException; import org.slf4j.Logger; import org.slf4j.LoggerFactory; +import java.util.Locale; + /** * Create PathManager instances. */ public class PathManagerFactory { - private static final Logger logger = LoggerFactory.getLogger(PathManagerFactory.class); + private static final Logger logger = LoggerFactory.getLogger(PathManagerFactory.class); - public static PathManager getInstance(String managerType, Context context) { + public static PathManager getInstance(String managerType, Context context) { - Preconditions.checkNotNull(managerType, "path manager type must not be null"); - - // try to find builder class in enum of known output serializers - PathManagerType type; - try { - type = PathManagerType.valueOf(managerType.toUpperCase(Locale.ENGLISH)); - } catch (IllegalArgumentException e) { - logger.debug("Not in enum, loading builder class: {}", managerType); - type = PathManagerType.OTHER; - } - Class builderClass = type.getBuilderClass(); + Preconditions.checkNotNull(managerType, "path manager type must not be null"); - // handle the case where they have specified their own builder in the config - if (builderClass == null) { - try { - Class c = Class.forName(managerType); - if (c != null && PathManager.Builder.class.isAssignableFrom(c)) { - builderClass = (Class) c; - } else { - String errMessage = "Unable to instantiate Builder from " + - managerType + ": does not appear to implement " + - PathManager.Builder.class.getName(); - throw new FlumeException(errMessage); - } - } catch (ClassNotFoundException ex) { - logger.error("Class not found: " + managerType, ex); - throw new FlumeException(ex); - } - } + // try to find builder class in enum of known output serializers + PathManagerType type; + try { + type = PathManagerType.valueOf(managerType.toUpperCase(Locale.ENGLISH)); + } catch (IllegalArgumentException e) { + logger.debug("Not in enum, loading builder class: {}", managerType); + type = PathManagerType.OTHER; + } + Class builderClass = type.getBuilderClass(); - // build the builder - PathManager.Builder builder; - try { - builder = builderClass.newInstance(); - } catch (InstantiationException ex) { - String errMessage = "Cannot instantiate builder: " + managerType; - logger.error(errMessage, ex); - throw new FlumeException(errMessage, ex); - } catch (IllegalAccessException ex) { - String errMessage = "Cannot instantiate builder: " + managerType; - logger.error(errMessage, ex); - throw new FlumeException(errMessage, ex); + // handle the case where they have specified their own builder in the config + if (builderClass == null) { + try { + Class c = Class.forName(managerType); + if (c != null && PathManager.Builder.class.isAssignableFrom(c)) { + builderClass = (Class) c; + } else { + String errMessage = "Unable to instantiate Builder from " + + managerType + ": does not appear to implement " + + PathManager.Builder.class.getName(); + throw new FlumeException(errMessage); } + } catch (ClassNotFoundException ex) { + logger.error("Class not found: " + managerType, ex); + throw new FlumeException(ex); + } + } - return builder.build(context); + // build the builder + PathManager.Builder builder; + try { + builder = builderClass.newInstance(); + } catch (InstantiationException ex) { + String errMessage = "Cannot instantiate builder: " + managerType; + logger.error(errMessage, ex); + throw new FlumeException(errMessage, ex); + } catch (IllegalAccessException ex) { + String errMessage = "Cannot instantiate builder: " + managerType; + logger.error(errMessage, ex); + throw new FlumeException(errMessage, ex); } + + return builder.build(context); + } } diff --git a/flume-ng-core/src/main/java/org/apache/flume/formatter/output/PathManagerType.java b/flume-ng-core/src/main/java/org/apache/flume/formatter/output/PathManagerType.java index 4f1fa9399b..f131ff3b50 100644 --- a/flume-ng-core/src/main/java/org/apache/flume/formatter/output/PathManagerType.java +++ b/flume-ng-core/src/main/java/org/apache/flume/formatter/output/PathManagerType.java @@ -27,17 +27,17 @@ @InterfaceAudience.Private @InterfaceStability.Unstable public enum PathManagerType { - DEFAULT(DefaultPathManager.Builder.class), - ROLLTIME(RollTimePathManager.Builder.class), - OTHER(null); + DEFAULT(DefaultPathManager.Builder.class), + ROLLTIME(RollTimePathManager.Builder.class), + OTHER(null); - private final Class builderClass; + private final Class builderClass; - PathManagerType(Class builderClass) { - this.builderClass = builderClass; - } + PathManagerType(Class builderClass) { + this.builderClass = builderClass; + } - public Class getBuilderClass() { - return builderClass; - } + public Class getBuilderClass() { + return builderClass; + } } diff --git a/flume-ng-core/src/main/java/org/apache/flume/formatter/output/RollTimePathManager.java b/flume-ng-core/src/main/java/org/apache/flume/formatter/output/RollTimePathManager.java index 6883a9c13e..b14a812fa0 100644 --- a/flume-ng-core/src/main/java/org/apache/flume/formatter/output/RollTimePathManager.java +++ b/flume-ng-core/src/main/java/org/apache/flume/formatter/output/RollTimePathManager.java @@ -19,48 +19,48 @@ package org.apache.flume.formatter.output; -import java.io.File; - import org.apache.flume.Context; import org.joda.time.LocalDateTime; import org.joda.time.format.DateTimeFormat; import org.joda.time.format.DateTimeFormatter; +import java.io.File; + /** * */ public class RollTimePathManager extends DefaultPathManager { - private final DateTimeFormatter formatter = DateTimeFormat.forPattern("yyyyMMddHHmmss"); - private String lastRoll; - - public RollTimePathManager(Context context) { - super(context); - } + private final DateTimeFormatter formatter = DateTimeFormat.forPattern("yyyyMMddHHmmss"); + private String lastRoll; - @Override - public File nextFile() { - StringBuilder sb = new StringBuilder(); - String date = formatter.print(LocalDateTime.now()); - if (!date.equals(lastRoll)) { - getFileIndex().set(0); - lastRoll = date; - } - sb.append(getPrefix()).append(date).append("-"); - sb.append(getFileIndex().incrementAndGet()); - if (getExtension().length() > 0) { - sb.append(".").append(getExtension()); - } - currentFile = new File(getBaseDirectory(), sb.toString()); + public RollTimePathManager(Context context) { + super(context); + } - return currentFile; + @Override + public File nextFile() { + StringBuilder sb = new StringBuilder(); + String date = formatter.print(LocalDateTime.now()); + if (!date.equals(lastRoll)) { + getFileIndex().set(0); + lastRoll = date; + } + sb.append(getPrefix()).append(date).append("-"); + sb.append(getFileIndex().incrementAndGet()); + if (getExtension().length() > 0) { + sb.append(".").append(getExtension()); } + currentFile = new File(getBaseDirectory(), sb.toString()); - public static class Builder implements PathManager.Builder { - @Override - public PathManager build(Context context) { - return new RollTimePathManager(context); - } + return currentFile; + } + + public static class Builder implements PathManager.Builder { + @Override + public PathManager build(Context context) { + return new RollTimePathManager(context); } + } } diff --git a/flume-ng-core/src/main/java/org/apache/flume/instrumentation/ChannelCounter.java b/flume-ng-core/src/main/java/org/apache/flume/instrumentation/ChannelCounter.java index 977ad6c6a5..c4fe6c63cc 100644 --- a/flume-ng-core/src/main/java/org/apache/flume/instrumentation/ChannelCounter.java +++ b/flume-ng-core/src/main/java/org/apache/flume/instrumentation/ChannelCounter.java @@ -100,20 +100,20 @@ public long addToEventTakeSuccessCount(long delta) { return addAndGet(COUNTER_EVENT_TAKE_SUCCESS, delta); } - public void setChannelCapacity(long capacity){ + public void setChannelCapacity(long capacity) { set(COUNTER_CHANNEL_CAPACITY, capacity); } @Override - public long getChannelCapacity(){ + public long getChannelCapacity() { return get(COUNTER_CHANNEL_CAPACITY); } @Override - public double getChannelFillPercentage(){ + public double getChannelFillPercentage() { long capacity = getChannelCapacity(); - if(capacity != 0L) { - return ((getChannelSize()/(double)capacity) * 100); + if (capacity != 0L) { + return (getChannelSize() / (double)capacity) * 100; } return Double.MAX_VALUE; } diff --git a/flume-ng-core/src/main/java/org/apache/flume/instrumentation/GangliaServer.java b/flume-ng-core/src/main/java/org/apache/flume/instrumentation/GangliaServer.java index 7d4be55159..bd9cd88a11 100644 --- a/flume-ng-core/src/main/java/org/apache/flume/instrumentation/GangliaServer.java +++ b/flume-ng-core/src/main/java/org/apache/flume/instrumentation/GangliaServer.java @@ -85,12 +85,6 @@ public class GangliaServer implements MonitorService { public final String CONF_ISGANGLIA3 = "isGanglia3"; private static final String GANGLIA_CONTEXT = "flume."; - /** - * - * @param hosts List of hosts to send the metrics to. All of them have to be - * running the version of ganglia specified by the configuration. - * @throws FlumeException - */ public GangliaServer() throws FlumeException { collectorRunnable = new GangliaCollector(); } diff --git a/flume-ng-core/src/main/java/org/apache/flume/instrumentation/MonitoredCounterGroup.java b/flume-ng-core/src/main/java/org/apache/flume/instrumentation/MonitoredCounterGroup.java index 44e26e4aeb..633513a619 100644 --- a/flume-ng-core/src/main/java/org/apache/flume/instrumentation/MonitoredCounterGroup.java +++ b/flume-ng-core/src/main/java/org/apache/flume/instrumentation/MonitoredCounterGroup.java @@ -59,7 +59,6 @@ public abstract class MonitoredCounterGroup { private AtomicLong stopTime; private volatile boolean registered = false; - protected MonitoredCounterGroup(Type type, String name, String... attrs) { this.type = type; this.name = name; @@ -154,15 +153,15 @@ public void stop() { // Print out the startTime for this component logger.info("Shutdown Metric for type: " + type + ", " - + "name: " + name + ". " - + typePrefix + "." + COUNTER_GROUP_START_TIME - + " == " + startTime); + + "name: " + name + ". " + + typePrefix + "." + COUNTER_GROUP_START_TIME + + " == " + startTime); // Print out the stopTime for this component logger.info("Shutdown Metric for type: " + type + ", " - + "name: " + name + ". " - + typePrefix + "." + COUNTER_GROUP_STOP_TIME - + " == " + stopTime); + + "name: " + name + ". " + + typePrefix + "." + COUNTER_GROUP_STOP_TIME + + " == " + stopTime); // Retrieve and sort counter group map keys final List mapKeys = new ArrayList(counterMap.keySet()); @@ -176,8 +175,8 @@ public void stop() { final long counterMapValue = get(counterMapKey); logger.info("Shutdown Metric for type: " + type + ", " - + "name: " + name + ". " - + counterMapKey + " == " + counterMapValue); + + "name: " + name + ". " + + counterMapKey + " == " + counterMapValue); } } @@ -276,9 +275,9 @@ public static enum Type { INTERCEPTOR, SERIALIZER, OTHER - }; + } - public String getType(){ + public String getType() { return type.name(); } } diff --git a/flume-ng-core/src/main/java/org/apache/flume/instrumentation/MonitoringType.java b/flume-ng-core/src/main/java/org/apache/flume/instrumentation/MonitoringType.java index 443335c930..4e1a28c26f 100644 --- a/flume-ng-core/src/main/java/org/apache/flume/instrumentation/MonitoringType.java +++ b/flume-ng-core/src/main/java/org/apache/flume/instrumentation/MonitoringType.java @@ -33,7 +33,7 @@ private MonitoringType(Class klass) { this.monitoringClass = klass; } - public Class getMonitorClass(){ + public Class getMonitorClass() { return this.monitoringClass; } } diff --git a/flume-ng-core/src/main/java/org/apache/flume/instrumentation/SinkCounter.java b/flume-ng-core/src/main/java/org/apache/flume/instrumentation/SinkCounter.java index 54f4a4c30f..534adc880d 100644 --- a/flume-ng-core/src/main/java/org/apache/flume/instrumentation/SinkCounter.java +++ b/flume-ng-core/src/main/java/org/apache/flume/instrumentation/SinkCounter.java @@ -53,7 +53,6 @@ public class SinkCounter extends MonitoredCounterGroup implements COUNTER_EVENT_DRAIN_ATTEMPT, COUNTER_EVENT_DRAIN_SUCCESS }; - public SinkCounter(String name) { super(MonitoredCounterGroup.Type.SINK, name, ATTRIBUTES); } @@ -63,7 +62,6 @@ public SinkCounter(String name, String[] attributes) { (String[]) ArrayUtils.addAll(attributes,ATTRIBUTES)); } - @Override public long getConnectionCreatedCount() { return get(COUNTER_CONNECTION_CREATED); diff --git a/flume-ng-core/src/main/java/org/apache/flume/instrumentation/SourceCounter.java b/flume-ng-core/src/main/java/org/apache/flume/instrumentation/SourceCounter.java index 02ef6ed679..f96694e779 100644 --- a/flume-ng-core/src/main/java/org/apache/flume/instrumentation/SourceCounter.java +++ b/flume-ng-core/src/main/java/org/apache/flume/instrumentation/SourceCounter.java @@ -41,15 +41,12 @@ public class SourceCounter extends MonitoredCounterGroup implements private static final String COUNTER_OPEN_CONNECTION_COUNT = "src.open-connection.count"; - - private static final String[] ATTRIBUTES = - { - COUNTER_EVENTS_RECEIVED, COUNTER_EVENTS_ACCEPTED, - COUNTER_APPEND_RECEIVED, COUNTER_APPEND_ACCEPTED, - COUNTER_APPEND_BATCH_RECEIVED, COUNTER_APPEND_BATCH_ACCEPTED, - COUNTER_OPEN_CONNECTION_COUNT - }; - + private static final String[] ATTRIBUTES = { + COUNTER_EVENTS_RECEIVED, COUNTER_EVENTS_ACCEPTED, + COUNTER_APPEND_RECEIVED, COUNTER_APPEND_ACCEPTED, + COUNTER_APPEND_BATCH_RECEIVED, COUNTER_APPEND_BATCH_ACCEPTED, + COUNTER_OPEN_CONNECTION_COUNT + }; public SourceCounter(String name) { super(MonitoredCounterGroup.Type.SOURCE, name, ATTRIBUTES); @@ -126,7 +123,7 @@ public long getOpenConnectionCount() { return get(COUNTER_OPEN_CONNECTION_COUNT); } - public void setOpenConnectionCount(long openConnectionCount){ + public void setOpenConnectionCount(long openConnectionCount) { set(COUNTER_OPEN_CONNECTION_COUNT, openConnectionCount); } } diff --git a/flume-ng-core/src/main/java/org/apache/flume/instrumentation/http/HTTPMetricsServer.java b/flume-ng-core/src/main/java/org/apache/flume/instrumentation/http/HTTPMetricsServer.java index 7c0afb046b..921a1f74f7 100644 --- a/flume-ng-core/src/main/java/org/apache/flume/instrumentation/http/HTTPMetricsServer.java +++ b/flume-ng-core/src/main/java/org/apache/flume/instrumentation/http/HTTPMetricsServer.java @@ -94,9 +94,7 @@ public void configure(Context context) { private class HTTPMetricsHandler extends AbstractHandler { - Type mapType = - new TypeToken>>() { - }.getType(); + Type mapType = new TypeToken>>() {}.getType(); Gson gson = new Gson(); @Override @@ -108,8 +106,8 @@ public void handle(String target, //If we want to use any other url for something else, we should make sure //that for metrics only /metrics is used to prevent backward //compatibility issues. - if(request.getMethod().equalsIgnoreCase("TRACE") || request.getMethod() - .equalsIgnoreCase("OPTIONS")) { + if (request.getMethod().equalsIgnoreCase("TRACE") || + request.getMethod().equalsIgnoreCase("OPTIONS")) { response.sendError(HttpServletResponse.SC_FORBIDDEN); response.flushBuffer(); ((Request) request).setHandled(true); diff --git a/flume-ng-core/src/main/java/org/apache/flume/instrumentation/kafka/KafkaChannelCounter.java b/flume-ng-core/src/main/java/org/apache/flume/instrumentation/kafka/KafkaChannelCounter.java index 6e142cfd12..28d3c8ce72 100644 --- a/flume-ng-core/src/main/java/org/apache/flume/instrumentation/kafka/KafkaChannelCounter.java +++ b/flume-ng-core/src/main/java/org/apache/flume/instrumentation/kafka/KafkaChannelCounter.java @@ -23,7 +23,6 @@ public class KafkaChannelCounter extends ChannelCounter implements KafkaChannelCounterMBean { - private static final String TIMER_KAFKA_EVENT_GET = "channel.kafka.event.get.time"; @@ -36,13 +35,10 @@ public class KafkaChannelCounter extends ChannelCounter private static final String COUNT_ROLLBACK = "channel.rollback.count"; - private static String[] ATTRIBUTES = { - TIMER_KAFKA_COMMIT,TIMER_KAFKA_EVENT_SEND,TIMER_KAFKA_EVENT_GET, - COUNT_ROLLBACK + TIMER_KAFKA_COMMIT,TIMER_KAFKA_EVENT_SEND,TIMER_KAFKA_EVENT_GET, COUNT_ROLLBACK }; - public KafkaChannelCounter(String name) { super(name,ATTRIBUTES); } diff --git a/flume-ng-core/src/main/java/org/apache/flume/instrumentation/util/JMXPollUtil.java b/flume-ng-core/src/main/java/org/apache/flume/instrumentation/util/JMXPollUtil.java index cbd6c3546d..a7794964e4 100644 --- a/flume-ng-core/src/main/java/org/apache/flume/instrumentation/util/JMXPollUtil.java +++ b/flume-ng-core/src/main/java/org/apache/flume/instrumentation/util/JMXPollUtil.java @@ -37,8 +37,7 @@ public class JMXPollUtil { private static Logger LOG = LoggerFactory.getLogger(JMXPollUtil.class); - private static MBeanServer mbeanServer = ManagementFactory. - getPlatformMBeanServer(); + private static MBeanServer mbeanServer = ManagementFactory.getPlatformMBeanServer(); public static Map> getAllMBeans() { Map> mbeanMap = Maps.newHashMap(); @@ -54,23 +53,20 @@ public static Map> getAllMBeans() { if (!obj.getObjectName().toString().startsWith("org.apache.flume")) { continue; } - MBeanAttributeInfo[] attrs = mbeanServer. - getMBeanInfo(obj.getObjectName()).getAttributes(); - String strAtts[] = new String[attrs.length]; + MBeanAttributeInfo[] attrs = mbeanServer.getMBeanInfo(obj.getObjectName()).getAttributes(); + String[] strAtts = new String[attrs.length]; for (int i = 0; i < strAtts.length; i++) { strAtts[i] = attrs[i].getName(); } - AttributeList attrList = mbeanServer.getAttributes( - obj.getObjectName(), strAtts); + AttributeList attrList = mbeanServer.getAttributes(obj.getObjectName(), strAtts); String component = obj.getObjectName().toString().substring( - obj.getObjectName().toString().indexOf('=') + 1); + obj.getObjectName().toString().indexOf('=') + 1); Map attrMap = Maps.newHashMap(); - for (Object attr : attrList) { Attribute localAttr = (Attribute) attr; - if(localAttr.getName().equalsIgnoreCase("type")){ - component = localAttr.getValue()+ "." + component; + if (localAttr.getName().equalsIgnoreCase("type")) { + component = localAttr.getValue() + "." + component; } attrMap.put(localAttr.getName(), localAttr.getValue().toString()); } diff --git a/flume-ng-core/src/main/java/org/apache/flume/interceptor/HostInterceptor.java b/flume-ng-core/src/main/java/org/apache/flume/interceptor/HostInterceptor.java index 269312309c..a2ad018644 100644 --- a/flume-ng-core/src/main/java/org/apache/flume/interceptor/HostInterceptor.java +++ b/flume-ng-core/src/main/java/org/apache/flume/interceptor/HostInterceptor.java @@ -88,7 +88,6 @@ private HostInterceptor(boolean preserveExisting, logger.warn("Could not get local host address. Exception follows.", e); } - } @Override @@ -106,7 +105,7 @@ public Event intercept(Event event) { if (preserveExisting && headers.containsKey(header)) { return event; } - if(host != null) { + if (host != null) { headers.put(header, host); } diff --git a/flume-ng-core/src/main/java/org/apache/flume/interceptor/RegexExtractorInterceptor.java b/flume-ng-core/src/main/java/org/apache/flume/interceptor/RegexExtractorInterceptor.java index 67cfc43758..7fda90d9eb 100644 --- a/flume-ng-core/src/main/java/org/apache/flume/interceptor/RegexExtractorInterceptor.java +++ b/flume-ng-core/src/main/java/org/apache/flume/interceptor/RegexExtractorInterceptor.java @@ -61,10 +61,10 @@ * agent.sources.r1.interceptors.i1.serializers = s1 s2 * agent.sources.r1.interceptors.i1.serializers.s1.type = com.blah.SomeSerializer * agent.sources.r1.interceptors.i1.serializers.s1.name = warning - * agent.sources.r1.interceptors.i1.serializers.s2.type = org.apache.flume.interceptor.RegexExtractorInterceptorTimestampSerializer + * agent.sources.r1.interceptors.i1.serializers.s2.type = + * org.apache.flume.interceptor.RegexExtractorInterceptorTimestampSerializer * agent.sources.r1.interceptors.i1.serializers.s2.name = error * agent.sources.r1.interceptors.i1.serializers.s2.dateFormat = yyyy-MM-dd - *
    *

    *
      * Example 1:
    @@ -167,7 +167,8 @@ public static class Builder implements Interceptor.Builder {
     
         private Pattern regex;
         private List serializerList;
    -    private final RegexExtractorInterceptorSerializer defaultSerializer = new RegexExtractorInterceptorPassThroughSerializer();
    +    private final RegexExtractorInterceptorSerializer defaultSerializer =
    +        new RegexExtractorInterceptorPassThroughSerializer();
     
         @Override
         public void configure(Context context) {
    @@ -191,7 +192,7 @@ private void configureSerializers(Context context) {
               new Context(context.getSubProperties(SERIALIZERS + "."));
     
           serializerList = Lists.newArrayListWithCapacity(serializerNames.length);
    -      for(String serializerName : serializerNames) {
    +      for (String serializerName : serializerNames) {
             Context serializerContext = new Context(
                 serializerContexts.getSubProperties(serializerName + "."));
             String type = serializerContext.getString("type", "DEFAULT");
    @@ -199,7 +200,7 @@ private void configureSerializers(Context context) {
             Preconditions.checkArgument(!StringUtils.isEmpty(name),
                 "Supplied name cannot be empty.");
     
    -        if("DEFAULT".equals(type)) {
    +        if ("DEFAULT".equals(type)) {
               serializerList.add(new NameAndSerializer(name, defaultSerializer));
             } else {
               serializerList.add(new NameAndSerializer(name, getCustomSerializer(
    diff --git a/flume-ng-core/src/main/java/org/apache/flume/interceptor/RegexFilteringInterceptor.java b/flume-ng-core/src/main/java/org/apache/flume/interceptor/RegexFilteringInterceptor.java
    index 8a3b6ce5f4..d8327d497a 100644
    --- a/flume-ng-core/src/main/java/org/apache/flume/interceptor/RegexFilteringInterceptor.java
    +++ b/flume-ng-core/src/main/java/org/apache/flume/interceptor/RegexFilteringInterceptor.java
    @@ -103,16 +103,13 @@ public Event intercept(Event event) {
         if (!excludeEvents) {
           if (regex.matcher(new String(event.getBody())).find()) {
             return event;
    -      }
    -      else {
    +      } else {
             return null;
           }
    -    }
    -    else {
    +    } else {
           if (regex.matcher(new String(event.getBody())).find()) {
             return null;
    -      }
    -      else {
    +      } else {
             return event;
           }
         }
    @@ -129,7 +126,9 @@ public List intercept(List events) {
         List out = Lists.newArrayList();
         for (Event event : events) {
           Event outEvent = intercept(event);
    -      if (outEvent != null) { out.add(outEvent); }
    +      if (outEvent != null) {
    +        out.add(outEvent);
    +      }
         }
         return out;
       }
    diff --git a/flume-ng-core/src/main/java/org/apache/flume/interceptor/SearchAndReplaceInterceptor.java b/flume-ng-core/src/main/java/org/apache/flume/interceptor/SearchAndReplaceInterceptor.java
    index b8588cd358..c4ec43b64f 100644
    --- a/flume-ng-core/src/main/java/org/apache/flume/interceptor/SearchAndReplaceInterceptor.java
    +++ b/flume-ng-core/src/main/java/org/apache/flume/interceptor/SearchAndReplaceInterceptor.java
    @@ -56,8 +56,8 @@ public class SearchAndReplaceInterceptor implements Interceptor {
       private final Charset charset;
     
       private SearchAndReplaceInterceptor(Pattern searchPattern,
    -    String replaceString,
    -    Charset charset) {
    +                                      String replaceString,
    +                                      Charset charset) {
         this.searchPattern = searchPattern;
         this.replaceString = replaceString;
         this.charset = charset;
    @@ -107,7 +107,7 @@ public void configure(Context context) {
           replaceString = context.getString(REPLACE_STRING_KEY);
           // Empty replacement String value or if the property itself is not present
           // assign empty string as replacement
    -      if(replaceString == null) {
    +      if (replaceString == null) {
             replaceString = "";
           }
     
    diff --git a/flume-ng-core/src/main/java/org/apache/flume/interceptor/StaticInterceptor.java b/flume-ng-core/src/main/java/org/apache/flume/interceptor/StaticInterceptor.java
    index 97df46740a..d2eb523ea5 100644
    --- a/flume-ng-core/src/main/java/org/apache/flume/interceptor/StaticInterceptor.java
    +++ b/flume-ng-core/src/main/java/org/apache/flume/interceptor/StaticInterceptor.java
    @@ -26,8 +26,6 @@
     import org.slf4j.Logger;
     import org.slf4j.LoggerFactory;
     
    -import static org.apache.flume.interceptor.StaticInterceptor.Constants.*;
    -
     /**
      * Interceptor class that appends a static, pre-configured header to all events.
      *
    @@ -57,8 +55,7 @@
      */
     public class StaticInterceptor implements Interceptor {
     
    -  private static final Logger logger = LoggerFactory
    -      .getLogger(StaticInterceptor.class);
    +  private static final Logger logger = LoggerFactory.getLogger(StaticInterceptor.class);
     
       private final boolean preserveExisting;
       private final String key;
    @@ -123,9 +120,9 @@ public static class Builder implements Interceptor.Builder {
     
         @Override
         public void configure(Context context) {
    -      preserveExisting = context.getBoolean(PRESERVE, PRESERVE_DEFAULT);
    -      key = context.getString(KEY, KEY_DEFAULT);
    -      value = context.getString(VALUE, VALUE_DEFAULT);
    +      preserveExisting = context.getBoolean(Constants.PRESERVE, Constants.PRESERVE_DEFAULT);
    +      key = context.getString(Constants.KEY, Constants.KEY_DEFAULT);
    +      value = context.getString(Constants.VALUE, Constants.VALUE_DEFAULT);
         }
     
         @Override
    @@ -136,11 +133,9 @@ public Interceptor build() {
           return new StaticInterceptor(preserveExisting, key, value);
         }
     
    -
       }
     
       public static class Constants {
    -
         public static final String KEY = "key";
         public static final String KEY_DEFAULT = "key";
     
    diff --git a/flume-ng-core/src/main/java/org/apache/flume/interceptor/TimestampInterceptor.java b/flume-ng-core/src/main/java/org/apache/flume/interceptor/TimestampInterceptor.java
    index 9d942f6ba0..50c36952e2 100644
    --- a/flume-ng-core/src/main/java/org/apache/flume/interceptor/TimestampInterceptor.java
    +++ b/flume-ng-core/src/main/java/org/apache/flume/interceptor/TimestampInterceptor.java
    @@ -22,6 +22,7 @@
     import java.util.Map;
     import org.apache.flume.Context;
     import org.apache.flume.Event;
    +
     import static org.apache.flume.interceptor.TimestampInterceptor.Constants.*;
     
     /**
    diff --git a/flume-ng-core/src/main/java/org/apache/flume/lifecycle/LifecycleAware.java b/flume-ng-core/src/main/java/org/apache/flume/lifecycle/LifecycleAware.java
    index d7fe7ac0d1..f42fd86dc0 100644
    --- a/flume-ng-core/src/main/java/org/apache/flume/lifecycle/LifecycleAware.java
    +++ b/flume-ng-core/src/main/java/org/apache/flume/lifecycle/LifecycleAware.java
    @@ -44,44 +44,42 @@
      * 

    * Example usage *

    - * - * public class MyService implements LifecycleAware { + *
    + * {@code
    + * public class MyService implements LifecycleAware {
      *
    - *    private LifecycleState lifecycleState;
    + *   private LifecycleState lifecycleState;
      *
    - *    public MyService() {
    - *      lifecycleState = LifecycleState.IDLE;
    - *    }
    + *   public MyService() {
    + *     lifecycleState = LifecycleState.IDLE;
    + *   }
      *
    - *    @Override
    - *    public void start(Context context) throws LifecycleException,
    - *      InterruptedException {
    + *   @Override
    + *   public void start(Context context) throws LifecycleException, InterruptedException {
    + *     // ...your code does something.
    + *     lifecycleState = LifecycleState.START;
    + *   }
      *
    - *      ...your code does something.
    + *   @Override
    + *   public void stop(Context context) throws LifecycleException, InterruptedException {
      *
    - *      lifecycleState = LifecycleState.START;
    - *    }
    + *     try {
    + *       // ...you stop services here.
    + *     } catch (SomethingException) {
    + *       lifecycleState = LifecycleState.ERROR;
    + *     }
      *
    - *    @Override
    - *    public void stop(Context context) throws LifecycleException,
    - *      InterruptedException {
    + *     lifecycleState = LifecycleState.STOP;
    + *   }
      *
    - *      try {
    - *        ...you stop services here.
    - *      } catch (SomethingException) {
    - *        lifecycleState = LifecycleState.ERROR;
    - *      }
    + *   @Override
    + *   public LifecycleState getLifecycleState() {
    + *     return lifecycleState;
    + *   }
      *
    - *      lifecycleState = LifecycleState.STOP;
    - *    }
    - *
    - *    @Override
    - *    public LifecycleState getLifecycleState() {
    - *      return lifecycleState;
    - *    }
    - *
    - *  }
    - * 
    + * }
    + * }
    + * 
    */ @InterfaceAudience.Public @InterfaceStability.Stable diff --git a/flume-ng-core/src/main/java/org/apache/flume/lifecycle/LifecycleSupervisor.java b/flume-ng-core/src/main/java/org/apache/flume/lifecycle/LifecycleSupervisor.java index 59d780abea..773d671b44 100644 --- a/flume-ng-core/src/main/java/org/apache/flume/lifecycle/LifecycleSupervisor.java +++ b/flume-ng-core/src/main/java/org/apache/flume/lifecycle/LifecycleSupervisor.java @@ -36,8 +36,7 @@ public class LifecycleSupervisor implements LifecycleAware { - private static final Logger logger = LoggerFactory - .getLogger(LifecycleSupervisor.class); + private static final Logger logger = LoggerFactory.getLogger(LifecycleSupervisor.class); private Map supervisedProcesses; private Map> monitorFutures; @@ -81,15 +80,15 @@ public synchronized void stop() { if (monitorService != null) { monitorService.shutdown(); - try{ + try { monitorService.awaitTermination(10, TimeUnit.SECONDS); } catch (InterruptedException e) { logger.error("Interrupted while waiting for monitor service to stop"); } - if(!monitorService.isTerminated()) { + if (!monitorService.isTerminated()) { monitorService.shutdownNow(); try { - while(!monitorService.isTerminated()) { + while (!monitorService.isTerminated()) { monitorService.awaitTermination(10, TimeUnit.SECONDS); } } catch (InterruptedException e) { @@ -98,8 +97,7 @@ public synchronized void stop() { } } - for (final Entry entry : supervisedProcesses - .entrySet()) { + for (final Entry entry : supervisedProcesses.entrySet()) { if (entry.getKey().getLifecycleState().equals(LifecycleState.START)) { entry.getValue().status.desiredState = LifecycleState.STOP; @@ -122,9 +120,9 @@ public synchronized void fail() { public synchronized void supervise(LifecycleAware lifecycleAware, SupervisorPolicy policy, LifecycleState desiredState) { - if(this.monitorService.isShutdown() + if (this.monitorService.isShutdown() || this.monitorService.isTerminated() - || this.monitorService.isTerminating()){ + || this.monitorService.isTerminating()) { throw new FlumeException("Supervise called on " + lifecycleAware + " " + "after shutdown has been initiated. " + lifecycleAware + " will not" + " be started"); @@ -165,8 +163,8 @@ public synchronized void unsupervise(LifecycleAware lifecycleAware) { logger.debug("Unsupervising service:{}", lifecycleAware); synchronized (lifecycleAware) { - Supervisoree supervisoree = supervisedProcesses.get(lifecycleAware); - supervisoree.status.discard = true; + Supervisoree supervisoree = supervisedProcesses.get(lifecycleAware); + supervisoree.status.discard = true; this.setDesiredState(lifecycleAware, LifecycleState.STOP); logger.info("Stopping component: {}", lifecycleAware); lifecycleAware.stop(); @@ -199,7 +197,7 @@ public synchronized LifecycleState getLifecycleState() { return lifecycleState; } - public synchronized boolean isComponentInErrorState(LifecycleAware component){ + public synchronized boolean isComponentInErrorState(LifecycleAware component) { return supervisedProcesses.get(component).status.error; } @@ -301,18 +299,18 @@ public void run() { } } } - } catch(Throwable t) { + } catch (Throwable t) { logger.error("Unexpected error", t); } logger.debug("Status check complete"); } } - private class Purger implements Runnable{ + private class Purger implements Runnable { @Override public void run() { - if(needToPurge){ + if (needToPurge) { monitorService.purge(); needToPurge = false; } @@ -338,7 +336,7 @@ public String toString() { } - public static abstract class SupervisorPolicy { + public abstract static class SupervisorPolicy { abstract boolean isValid(LifecycleAware object, Status status); @@ -372,5 +370,4 @@ public String toString() { } - } diff --git a/flume-ng-core/src/main/java/org/apache/flume/serialization/AvroEventDeserializer.java b/flume-ng-core/src/main/java/org/apache/flume/serialization/AvroEventDeserializer.java index 5faf449f3a..c6489580f5 100644 --- a/flume-ng-core/src/main/java/org/apache/flume/serialization/AvroEventDeserializer.java +++ b/flume-ng-core/src/main/java/org/apache/flume/serialization/AvroEventDeserializer.java @@ -52,8 +52,7 @@ */ public class AvroEventDeserializer implements EventDeserializer { - private static final Logger logger = LoggerFactory.getLogger - (AvroEventDeserializer.class); + private static final Logger logger = LoggerFactory.getLogger(AvroEventDeserializer.class); private final AvroSchemaType schemaType; private final ResettableInputStream ris; @@ -180,8 +179,8 @@ public EventDeserializer build(Context context, ResettableInputStream in) { } private static class SeekableResettableInputBridge implements SeekableInput { - ResettableInputStream ris; + public SeekableResettableInputBridge(ResettableInputStream ris) { this.ris = ris; } diff --git a/flume-ng-core/src/main/java/org/apache/flume/serialization/BodyTextEventSerializer.java b/flume-ng-core/src/main/java/org/apache/flume/serialization/BodyTextEventSerializer.java index d09f02d0db..4d7a525c63 100644 --- a/flume-ng-core/src/main/java/org/apache/flume/serialization/BodyTextEventSerializer.java +++ b/flume-ng-core/src/main/java/org/apache/flume/serialization/BodyTextEventSerializer.java @@ -31,8 +31,7 @@ */ public class BodyTextEventSerializer implements EventSerializer { - private final static Logger logger = - LoggerFactory.getLogger(BodyTextEventSerializer.class); + private static final Logger logger = LoggerFactory.getLogger(BodyTextEventSerializer.class); // for legacy reasons, by default, append a newline to each event written out private final String APPEND_NEWLINE = "appendNewline"; diff --git a/flume-ng-core/src/main/java/org/apache/flume/serialization/HeaderAndBodyTextEventSerializer.java b/flume-ng-core/src/main/java/org/apache/flume/serialization/HeaderAndBodyTextEventSerializer.java index 9c6003c2c1..8e3621d30e 100644 --- a/flume-ng-core/src/main/java/org/apache/flume/serialization/HeaderAndBodyTextEventSerializer.java +++ b/flume-ng-core/src/main/java/org/apache/flume/serialization/HeaderAndBodyTextEventSerializer.java @@ -31,7 +31,7 @@ */ public class HeaderAndBodyTextEventSerializer implements EventSerializer { - private final static Logger logger = + private static final Logger logger = LoggerFactory.getLogger(HeaderAndBodyTextEventSerializer.class); // for legacy reasons, by default, append a newline to each event written out diff --git a/flume-ng-core/src/main/java/org/apache/flume/serialization/LineDeserializer.java b/flume-ng-core/src/main/java/org/apache/flume/serialization/LineDeserializer.java index 7c872353a2..8f236858c9 100644 --- a/flume-ng-core/src/main/java/org/apache/flume/serialization/LineDeserializer.java +++ b/flume-ng-core/src/main/java/org/apache/flume/serialization/LineDeserializer.java @@ -38,8 +38,7 @@ @InterfaceStability.Evolving public class LineDeserializer implements EventDeserializer { - private static final Logger logger = LoggerFactory.getLogger - (LineDeserializer.class); + private static final Logger logger = LoggerFactory.getLogger(LineDeserializer.class); private final ResettableInputStream in; private final Charset outputCharset; diff --git a/flume-ng-core/src/main/java/org/apache/flume/serialization/ResettableFileInputStream.java b/flume-ng-core/src/main/java/org/apache/flume/serialization/ResettableFileInputStream.java index 618913ef6b..7d6d95c8d0 100644 --- a/flume-ng-core/src/main/java/org/apache/flume/serialization/ResettableFileInputStream.java +++ b/flume-ng-core/src/main/java/org/apache/flume/serialization/ResettableFileInputStream.java @@ -64,12 +64,13 @@ *

    Thus the behaviour of mark and reset is as follows:

    * *
      - *
    1. If {@link #mark()} is called after a high surrogate pair has been returned by {@link #readChar()}, - * the marked position will be that of the character following the low surrogate, - * not that of the low surrogate itself.
    2. - *
    3. If {@link #reset()} is called after a high surrogate pair has been returned by {@link #readChar()}, - * the low surrogate is always returned by the next call to {@link #readChar()}, - * before the stream is actually reset to the last marked position.
    4. + *
    5. If {@link #mark()} is called after a high surrogate pair has been returned by + * {@link #readChar()}, the marked position will be that of the character following + * the low surrogate, not that of the low surrogate itself.
    6. + *
    7. If {@link #reset()} is called after a high surrogate pair has been returned by + * {@link #readChar()}, the low surrogate is always returned by the next call to + * {@link #readChar()}, before the stream is actually reset to the last marked + * position.
    8. *
    * *

    This ensures that no dangling high surrogate could ever be read as long as @@ -181,13 +182,13 @@ public ResettableFileInputStream(File file, PositionTracker tracker, this.decoder = charset.newDecoder(); this.position = 0; this.syncPosition = 0; - if(charset.name().startsWith("UTF-8")) { + if (charset.name().startsWith("UTF-8")) { // some JDKs wrongly report 3 bytes max this.maxCharWidth = 4; - } else if(charset.name().startsWith("UTF-16")) { + } else if (charset.name().startsWith("UTF-16")) { // UTF_16BE and UTF_16LE wrongly report 2 bytes max this.maxCharWidth = 4; - } else if(charset.name().startsWith("UTF-32")) { + } else if (charset.name().startsWith("UTF-32")) { // UTF_32BE and UTF_32LE wrongly report 4 bytes max this.maxCharWidth = 8; } else { @@ -254,7 +255,7 @@ public synchronized int readChar() throws IOException { // Check whether we are in the middle of a surrogate pair, // in which case, return the last (low surrogate) char of the pair. - if(hasLowSurrogate) { + if (hasLowSurrogate) { hasLowSurrogate = false; return lowSurrogate; } @@ -296,7 +297,7 @@ public synchronized int readChar() throws IOException { // Found nothing, but the byte buffer has not been entirely consumed. // This situation denotes the presence of a surrogate pair // that can only be decoded if we have a 2-char buffer. - if(buf.hasRemaining()) { + if (buf.hasRemaining()) { charBuf.clear(); // increase the limit to 2 charBuf.limit(2); @@ -312,9 +313,10 @@ public synchronized int readChar() throws IOException { // save second (low surrogate) char for later consumption lowSurrogate = charBuf.get(); // Check if we really have a surrogate pair - if( ! Character.isHighSurrogate(highSurrogate) || ! Character.isLowSurrogate(lowSurrogate)) { + if (!Character.isHighSurrogate(highSurrogate) || !Character.isLowSurrogate(lowSurrogate)) { // This should only happen in case of bad sequences (dangling surrogate, etc.) - logger.warn("Decoded a pair of chars, but it does not seem to be a surrogate pair: {} {}", (int)highSurrogate, (int)lowSurrogate); + logger.warn("Decoded a pair of chars, but it does not seem to be a surrogate pair: {} {}", + (int)highSurrogate, (int)lowSurrogate); } hasLowSurrogate = true; // consider the pair as a single unit and increment position normally diff --git a/flume-ng-core/src/main/java/org/apache/flume/sink/AbstractRpcSink.java b/flume-ng-core/src/main/java/org/apache/flume/sink/AbstractRpcSink.java index 5146834998..f6024aa888 100644 --- a/flume-ng-core/src/main/java/org/apache/flume/sink/AbstractRpcSink.java +++ b/flume-ng-core/src/main/java/org/apache/flume/sink/AbstractRpcSink.java @@ -115,14 +115,16 @@ * * * compression-type - * Select compression type. Default is "none" and the only compression type available is "deflate" + * Select compression type. Default is "none" and the only compression type available + * is "deflate" * compression type * none * * * compression-level - * In the case compression type is "deflate" this value can be between 0-9. 0 being no compression and - * 1-9 is compression. The higher the number the better the compression. 6 is the default. + * In the case compression type is "deflate" this value can be between 0-9. + * 0 being no compression and 1-9 is compression. + * The higher the number the better the compression. 6 is the default. * compression level * 6 * @@ -139,11 +141,9 @@ * This method will be called whenever this sink needs to create a new * connection to the source. */ -public abstract class AbstractRpcSink extends AbstractSink - implements Configurable { +public abstract class AbstractRpcSink extends AbstractSink implements Configurable { - private static final Logger logger = LoggerFactory.getLogger - (AbstractRpcSink.class); + private static final Logger logger = LoggerFactory.getLogger(AbstractRpcSink.class); private String hostname; private Integer port; private RpcClient client; @@ -152,9 +152,9 @@ public abstract class AbstractRpcSink extends AbstractSink private int cxnResetInterval; private AtomicBoolean resetConnectionFlag; private final int DEFAULT_CXN_RESET_INTERVAL = 0; - private final ScheduledExecutorService cxnResetExecutor = Executors - .newSingleThreadScheduledExecutor(new ThreadFactoryBuilder() - .setNameFormat("Rpc Sink Reset Thread").build()); + private final ScheduledExecutorService cxnResetExecutor = + Executors.newSingleThreadScheduledExecutor( + new ThreadFactoryBuilder().setNameFormat("Rpc Sink Reset Thread").build()); @Override public void configure(Context context) { @@ -179,10 +179,9 @@ public void configure(Context context) { } cxnResetInterval = context.getInteger("reset-connection-interval", DEFAULT_CXN_RESET_INTERVAL); - if(cxnResetInterval == DEFAULT_CXN_RESET_INTERVAL) { - logger.info("Connection reset is set to " + String.valueOf - (DEFAULT_CXN_RESET_INTERVAL) +". Will not reset connection to next " + - "hop"); + if (cxnResetInterval == DEFAULT_CXN_RESET_INTERVAL) { + logger.info("Connection reset is set to " + String.valueOf(DEFAULT_CXN_RESET_INTERVAL) + + ". Will not reset connection to next hop"); } } @@ -210,7 +209,7 @@ private void createConnection() throws FlumeException { resetConnectionFlag = new AtomicBoolean(false); client = initializeRpcClient(clientProps); Preconditions.checkNotNull(client, "Rpc Client could not be " + - "initialized. " + getName() + " could not be started"); + "initialized. " + getName() + " could not be started"); sinkCounter.incrementConnectionCreatedCount(); if (cxnResetInterval > 0) { cxnResetExecutor.schedule(new Runnable() { @@ -228,20 +227,19 @@ public void run() { throw new FlumeException(ex); } } - logger.debug("Rpc sink {}: Created RpcClient: {}", getName(), client); + logger.debug("Rpc sink {}: Created RpcClient: {}", getName(), client); } } private void resetConnection() { - try { - destroyConnection(); - createConnection(); - } catch (Throwable throwable) { - //Don't rethrow, else this runnable won't get scheduled again. - logger.error("Error while trying to expire connection", - throwable); - } + try { + destroyConnection(); + createConnection(); + } catch (Throwable throwable) { + // Don't rethrow, else this runnable won't get scheduled again. + logger.error("Error while trying to expire connection", throwable); + } } private void destroyConnection() { @@ -314,8 +312,7 @@ public void stop() { cxnResetExecutor.shutdownNow(); } } catch (Exception ex) { - logger.error("Interrupted while waiting for connection reset executor " + - "to shut down"); + logger.error("Interrupted while waiting for connection reset executor to shut down"); } sinkCounter.stop(); super.stop(); @@ -335,7 +332,7 @@ public Status process() throws EventDeliveryException { Channel channel = getChannel(); Transaction transaction = channel.getTransaction(); - if(resetConnectionFlag.get()) { + if (resetConnectionFlag.get()) { resetConnection(); // if the time to reset is long and the timeout is short // this may cancel the next reset request diff --git a/flume-ng-core/src/main/java/org/apache/flume/sink/AbstractSink.java b/flume-ng-core/src/main/java/org/apache/flume/sink/AbstractSink.java index 1112643787..585044b7a3 100644 --- a/flume-ng-core/src/main/java/org/apache/flume/sink/AbstractSink.java +++ b/flume-ng-core/src/main/java/org/apache/flume/sink/AbstractSink.java @@ -30,7 +30,7 @@ @InterfaceAudience.Public @InterfaceStability.Stable -abstract public class AbstractSink implements Sink, LifecycleAware { +public abstract class AbstractSink implements Sink, LifecycleAware { private Channel channel; private String name; @@ -78,7 +78,8 @@ public synchronized String getName() { return name; } + @Override public String toString() { - return this.getClass().getName() + "{name:" + name + ", channel:" + channel.getName() + "}"; + return this.getClass().getName() + "{name:" + name + ", channel:" + channel.getName() + "}"; } } diff --git a/flume-ng-core/src/main/java/org/apache/flume/sink/AbstractSinkProcessor.java b/flume-ng-core/src/main/java/org/apache/flume/sink/AbstractSinkProcessor.java index 1c30592982..3de653adfb 100644 --- a/flume-ng-core/src/main/java/org/apache/flume/sink/AbstractSinkProcessor.java +++ b/flume-ng-core/src/main/java/org/apache/flume/sink/AbstractSinkProcessor.java @@ -37,7 +37,7 @@ public abstract class AbstractSinkProcessor implements SinkProcessor { @Override public void start() { - for(Sink s : sinkList) { + for (Sink s : sinkList) { s.start(); } @@ -46,7 +46,7 @@ public void start() { @Override public void stop() { - for(Sink s : sinkList) { + for (Sink s : sinkList) { s.stop(); } state = LifecycleState.STOP; diff --git a/flume-ng-core/src/main/java/org/apache/flume/sink/AbstractSinkSelector.java b/flume-ng-core/src/main/java/org/apache/flume/sink/AbstractSinkSelector.java index 9ddeef4ed4..f236a8abdc 100644 --- a/flume-ng-core/src/main/java/org/apache/flume/sink/AbstractSinkSelector.java +++ b/flume-ng-core/src/main/java/org/apache/flume/sink/AbstractSinkSelector.java @@ -38,7 +38,7 @@ public abstract class AbstractSinkSelector implements SinkSelector { @Override public void configure(Context context) { Long timeOut = context.getLong("maxTimeOut"); - if(timeOut != null){ + if (timeOut != null) { maxTimeOut = timeOut; } } diff --git a/flume-ng-core/src/main/java/org/apache/flume/sink/DefaultSinkFactory.java b/flume-ng-core/src/main/java/org/apache/flume/sink/DefaultSinkFactory.java index 6a5be921fd..6f405fba0c 100644 --- a/flume-ng-core/src/main/java/org/apache/flume/sink/DefaultSinkFactory.java +++ b/flume-ng-core/src/main/java/org/apache/flume/sink/DefaultSinkFactory.java @@ -53,8 +53,7 @@ public Sink create(String name, String type) throws FlumeException { @SuppressWarnings("unchecked") @Override - public Class getClass(String type) - throws FlumeException { + public Class getClass(String type) throws FlumeException { String sinkClassName = type; SinkType sinkType = SinkType.OTHER; try { diff --git a/flume-ng-core/src/main/java/org/apache/flume/sink/DefaultSinkProcessor.java b/flume-ng-core/src/main/java/org/apache/flume/sink/DefaultSinkProcessor.java index 00a362bc2c..2da9264bc5 100644 --- a/flume-ng-core/src/main/java/org/apache/flume/sink/DefaultSinkProcessor.java +++ b/flume-ng-core/src/main/java/org/apache/flume/sink/DefaultSinkProcessor.java @@ -35,8 +35,7 @@ * results without any additional handling. Suitable for all sinks that aren't * assigned to a group. */ -public class DefaultSinkProcessor implements SinkProcessor, -ConfigurableComponent { +public class DefaultSinkProcessor implements SinkProcessor, ConfigurableComponent { private Sink sink; private LifecycleState lifecycleState; diff --git a/flume-ng-core/src/main/java/org/apache/flume/sink/FailoverSinkProcessor.java b/flume-ng-core/src/main/java/org/apache/flume/sink/FailoverSinkProcessor.java index 3bd52f20dd..69541e6feb 100644 --- a/flume-ng-core/src/main/java/org/apache/flume/sink/FailoverSinkProcessor.java +++ b/flume-ng-core/src/main/java/org/apache/flume/sink/FailoverSinkProcessor.java @@ -70,12 +70,14 @@ private class FailedSink implements Comparable { private Integer priority; private Sink sink; private Integer sequentialFailures; + public FailedSink(Integer priority, Sink sink, int seqFailures) { this.sink = sink; this.priority = priority; this.sequentialFailures = seqFailures; adjustRefresh(); } + @Override public int compareTo(FailedSink arg0) { return refresh.compareTo(arg0.refresh); @@ -88,24 +90,25 @@ public Long getRefresh() { public Sink getSink() { return sink; } + public Integer getPriority() { return priority; } + public void incFails() { sequentialFailures++; adjustRefresh(); - logger.debug("Sink {} failed again, new refresh is at {}, " + - "current time {}", new Object[] { - sink.getName(), refresh, System.currentTimeMillis()}); + logger.debug("Sink {} failed again, new refresh is at {}, current time {}", + new Object[] { sink.getName(), refresh, System.currentTimeMillis() }); } + private void adjustRefresh() { refresh = System.currentTimeMillis() - + Math.min(maxPenalty, (1 << sequentialFailures) * FAILURE_PENALTY); + + Math.min(maxPenalty, (1 << sequentialFailures) * FAILURE_PENALTY); } } - private static final Logger logger = LoggerFactory - .getLogger(FailoverSinkProcessor.class); + private static final Logger logger = LoggerFactory.getLogger(FailoverSinkProcessor.class); private static final String PRIORITY_PREFIX = "priority."; private static final String MAX_PENALTY_PREFIX = "maxpenalty"; @@ -121,15 +124,15 @@ public void configure(Context context) { failedSinks = new PriorityQueue(); Integer nextPrio = 0; String maxPenaltyStr = context.getString(MAX_PENALTY_PREFIX); - if(maxPenaltyStr == null) { + if (maxPenaltyStr == null) { maxPenalty = DEFAULT_MAX_PENALTY; } else { try { maxPenalty = Integer.parseInt(maxPenaltyStr); } catch (NumberFormatException e) { logger.warn("{} is not a valid value for {}", - new Object[] { maxPenaltyStr, MAX_PENALTY_PREFIX }); - maxPenalty = DEFAULT_MAX_PENALTY; + new Object[] { maxPenaltyStr, MAX_PENALTY_PREFIX }); + maxPenalty = DEFAULT_MAX_PENALTY; } } for (Entry entry : sinks.entrySet()) { @@ -140,7 +143,7 @@ public void configure(Context context) { } catch (Exception e) { priority = --nextPrio; } - if(!liveSinks.containsKey(priority)) { + if (!liveSinks.containsKey(priority)) { liveSinks.put(priority, sinks.get(entry.getKey())); } else { logger.warn("Sink {} not added to FailverSinkProcessor as priority" + @@ -155,7 +158,7 @@ public void configure(Context context) { public Status process() throws EventDeliveryException { // Retry any failed sinks that have gone through their "cooldown" period Long now = System.currentTimeMillis(); - while(!failedSinks.isEmpty() && failedSinks.peek().getRefresh() < now) { + while (!failedSinks.isEmpty() && failedSinks.peek().getRefresh() < now) { FailedSink cur = failedSinks.poll(); Status s; try { @@ -177,7 +180,7 @@ public Status process() throws EventDeliveryException { } Status ret = null; - while(activeSink != null) { + while (activeSink != null) { try { ret = activeSink.process(); return ret; @@ -196,8 +199,8 @@ private Sink moveActiveToDeadAndGetNext() { Integer key = liveSinks.lastKey(); failedSinks.add(new FailedSink(key, activeSink, 1)); liveSinks.remove(key); - if(liveSinks.isEmpty()) return null; - if(liveSinks.lastKey() != null) { + if (liveSinks.isEmpty()) return null; + if (liveSinks.lastKey() != null) { return liveSinks.get(liveSinks.lastKey()); } else { return null; diff --git a/flume-ng-core/src/main/java/org/apache/flume/sink/LoadBalancingSinkProcessor.java b/flume-ng-core/src/main/java/org/apache/flume/sink/LoadBalancingSinkProcessor.java index 2d85756fc2..ac0781e660 100644 --- a/flume-ng-core/src/main/java/org/apache/flume/sink/LoadBalancingSinkProcessor.java +++ b/flume-ng-core/src/main/java/org/apache/flume/sink/LoadBalancingSinkProcessor.java @@ -56,12 +56,14 @@ * *

    * Sample configuration: - *

    - *  host1.sinkgroups.group1.sinks = sink1 sink2
    - *  host1.sinkgroups.group1.processor.type = load_balance
    - *  host1.sinkgroups.group1.processor.selector = 
    - *  host1.sinkgroups.group1.processor.selector.selector_property = 
    - *  
    + *
    + * {@code
    + * host1.sinkgroups.group1.sinks = sink1 sink2
    + * host1.sinkgroups.group1.processor.type = load_balance
    + * host1.sinkgroups.group1.processor.selector = 
    + * host1.sinkgroups.group1.processor.selector.selector_property = 
    + * }
    + * 
    * * The value of processor.selector could be either round_robin for * round-robin scheme of load-balancing or random for random @@ -200,26 +202,28 @@ public interface SinkSelector extends Configurable, LifecycleAware { } /** - * A sink selector that implements the round-robin sink selection policy. - * This implementation is not MT safe. + *

    A sink selector that implements the round-robin sink selection policy. + * This implementation is not MT safe.

    + * + *

    Unfortunately both implementations need to override the base implementation + * in AbstractSinkSelector class, because any custom sink selectors + * will break if this stuff is moved to that class.

    */ - //Unfortunately both implementations need to override the base implementation - //in AbstractSinkSelector class, because any custom sink selectors - //will break if this stuff is moved to that class. private static class RoundRobinSinkSelector extends AbstractSinkSelector { - private OrderSelector selector; - RoundRobinSinkSelector(boolean backoff){ + + RoundRobinSinkSelector(boolean backoff) { selector = new RoundRobinOrderSelector(backoff); } @Override - public void configure(Context context){ + public void configure(Context context) { super.configure(context); if (maxTimeOut != 0) { selector.setMaxTimeOut(maxTimeOut); } } + @Override public Iterator createSinkIterator() { return selector.createIterator(); @@ -245,7 +249,7 @@ private static class RandomOrderSinkSelector extends AbstractSinkSelector { private OrderSelector selector; - RandomOrderSinkSelector(boolean backoff){ + RandomOrderSinkSelector(boolean backoff) { selector = new RandomOrderSelector(backoff); } diff --git a/flume-ng-core/src/main/java/org/apache/flume/sink/LoggerSink.java b/flume-ng-core/src/main/java/org/apache/flume/sink/LoggerSink.java index 9cf9bc2b1d..d219be7914 100644 --- a/flume-ng-core/src/main/java/org/apache/flume/sink/LoggerSink.java +++ b/flume-ng-core/src/main/java/org/apache/flume/sink/LoggerSink.java @@ -71,8 +71,9 @@ public void configure(Context context) { try { maxBytesToLog = Integer.parseInt(strMaxBytes); } catch (NumberFormatException e) { - logger.warn(String.format("Unable to convert %s to integer, using default value(%d) for maxByteToDump", - strMaxBytes, DEFAULT_MAX_BYTE_DUMP)); + logger.warn(String.format( + "Unable to convert %s to integer, using default value(%d) for maxByteToDump", + strMaxBytes, DEFAULT_MAX_BYTE_DUMP)); maxBytesToLog = DEFAULT_MAX_BYTE_DUMP; } } diff --git a/flume-ng-core/src/main/java/org/apache/flume/sink/NullSink.java b/flume-ng-core/src/main/java/org/apache/flume/sink/NullSink.java index cada6ecef7..eb00e158e1 100644 --- a/flume-ng-core/src/main/java/org/apache/flume/sink/NullSink.java +++ b/flume-ng-core/src/main/java/org/apache/flume/sink/NullSink.java @@ -92,7 +92,7 @@ public Status process() throws EventDeliveryException { if (++eventCounter % logEveryNEvents == 0) { logger.info("Null sink {} successful processed {} events.", getName(), eventCounter); } - if(event == null) { + if (event == null) { status = Status.BACKOFF; break; } diff --git a/flume-ng-core/src/main/java/org/apache/flume/sink/RollingFileSink.java b/flume-ng-core/src/main/java/org/apache/flume/sink/RollingFileSink.java index b97d404a03..ee4b94761a 100644 --- a/flume-ng-core/src/main/java/org/apache/flume/sink/RollingFileSink.java +++ b/flume-ng-core/src/main/java/org/apache/flume/sink/RollingFileSink.java @@ -112,7 +112,7 @@ public void start() { super.start(); pathController.setBaseDirectory(directory); - if(rollInterval > 0){ + if (rollInterval > 0) { rollService = Executors.newScheduledThreadPool( 1, @@ -136,7 +136,7 @@ public void run() { } }, rollInterval, rollInterval, TimeUnit.SECONDS); - } else{ + } else { logger.info("RollInterval is not valid, file rolling will not happen."); } logger.info("RollingFileSink {} started.", getName()); @@ -251,17 +251,15 @@ public void stop() { serializer = null; } } - if(rollInterval > 0){ + if (rollInterval > 0) { rollService.shutdown(); while (!rollService.isTerminated()) { try { rollService.awaitTermination(1, TimeUnit.SECONDS); } catch (InterruptedException e) { - logger - .debug( - "Interrupted while waiting for roll service to stop. " + - "Please report this.", e); + logger.debug("Interrupted while waiting for roll service to stop. " + + "Please report this.", e); } } } diff --git a/flume-ng-core/src/main/java/org/apache/flume/sink/SinkProcessorFactory.java b/flume-ng-core/src/main/java/org/apache/flume/sink/SinkProcessorFactory.java index dcdcad2160..084072f19b 100644 --- a/flume-ng-core/src/main/java/org/apache/flume/sink/SinkProcessorFactory.java +++ b/flume-ng-core/src/main/java/org/apache/flume/sink/SinkProcessorFactory.java @@ -55,8 +55,7 @@ public class SinkProcessorFactory { * processor configuration */ @SuppressWarnings("unchecked") - public static SinkProcessor getProcessor(Context context, - List sinks) { + public static SinkProcessor getProcessor(Context context, List sinks) { Preconditions.checkNotNull(context); Preconditions.checkNotNull(sinks); Preconditions.checkArgument(!sinks.isEmpty()); @@ -71,7 +70,7 @@ public static SinkProcessor getProcessor(Context context, logger.warn("Sink Processor type {} is a custom type", typeStr); } - if(!type.equals(SinkProcessorType.OTHER)) { + if (!type.equals(SinkProcessorType.OTHER)) { processorClassName = type.getSinkProcessorClassName(); } diff --git a/flume-ng-core/src/main/java/org/apache/flume/sink/ThriftSink.java b/flume-ng-core/src/main/java/org/apache/flume/sink/ThriftSink.java index 32021d37df..bcab731ac7 100644 --- a/flume-ng-core/src/main/java/org/apache/flume/sink/ThriftSink.java +++ b/flume-ng-core/src/main/java/org/apache/flume/sink/ThriftSink.java @@ -106,15 +106,15 @@ public class ThriftSink extends AbstractRpcSink { protected RpcClient initializeRpcClient(Properties props) { // Only one thread is enough, since only one sink thread processes // transactions at any given time. Each sink owns its own Rpc client. - props.setProperty(RpcClientConfigurationConstants - .CONFIG_CONNECTION_POOL_SIZE, String.valueOf(1)); - boolean enableKerberos = Boolean.parseBoolean(props.getProperty( - RpcClientConfigurationConstants.KERBEROS_KEY, "false")); - if(enableKerberos) { + props.setProperty(RpcClientConfigurationConstants.CONFIG_CONNECTION_POOL_SIZE, + String.valueOf(1)); + boolean enableKerberos = Boolean.parseBoolean( + props.getProperty(RpcClientConfigurationConstants.KERBEROS_KEY, "false")); + if (enableKerberos) { return SecureRpcClientFactory.getThriftInstance(props); } else { props.setProperty(RpcClientConfigurationConstants.CONFIG_CLIENT_TYPE, - RpcClientFactory.ClientType.THRIFT.name()); + RpcClientFactory.ClientType.THRIFT.name()); return RpcClientFactory.getInstance(props); } } diff --git a/flume-ng-core/src/main/java/org/apache/flume/source/AbstractEventDrivenSource.java b/flume-ng-core/src/main/java/org/apache/flume/source/AbstractEventDrivenSource.java index 89bd3573a9..08f9b84cf5 100644 --- a/flume-ng-core/src/main/java/org/apache/flume/source/AbstractEventDrivenSource.java +++ b/flume-ng-core/src/main/java/org/apache/flume/source/AbstractEventDrivenSource.java @@ -29,9 +29,9 @@ */ @InterfaceAudience.Public @InterfaceStability.Evolving -public abstract class AbstractEventDrivenSource extends BasicSourceSemantics - implements EventDrivenSource { - +public abstract class AbstractEventDrivenSource + extends BasicSourceSemantics + implements EventDrivenSource { public AbstractEventDrivenSource() { super(); } diff --git a/flume-ng-core/src/main/java/org/apache/flume/source/AbstractPollableSource.java b/flume-ng-core/src/main/java/org/apache/flume/source/AbstractPollableSource.java index 33e1acc0f2..97f6c99fcb 100644 --- a/flume-ng-core/src/main/java/org/apache/flume/source/AbstractPollableSource.java +++ b/flume-ng-core/src/main/java/org/apache/flume/source/AbstractPollableSource.java @@ -37,8 +37,8 @@ */ @InterfaceAudience.Public @InterfaceStability.Evolving -public abstract class AbstractPollableSource extends BasicSourceSemantics - implements PollableSource { +public abstract class AbstractPollableSource + extends BasicSourceSemantics implements PollableSource { long backoffSleepIncrement = PollableSourceConstants.DEFAULT_BACKOFF_SLEEP_INCREMENT; long maxBackoffSleep = PollableSourceConstants.DEFAULT_MAX_BACKOFF_SLEEP; @@ -46,14 +46,16 @@ public abstract class AbstractPollableSource extends BasicSourceSemantics public AbstractPollableSource() { super(); } + public Status process() throws EventDeliveryException { Exception exception = getStartException(); if (exception != null) { throw new FlumeException("Source had error configuring or starting", exception); } - if(!isStarted()) { - throw new EventDeliveryException("Source is not started. It is in '" + getLifecycleState() + "' state"); + if (!isStarted()) { + throw new EventDeliveryException("Source is not started. It is in '" + + getLifecycleState() + "' state"); } return doProcess(); } diff --git a/flume-ng-core/src/main/java/org/apache/flume/source/AbstractSource.java b/flume-ng-core/src/main/java/org/apache/flume/source/AbstractSource.java index 0855de3a9a..88ef665215 100644 --- a/flume-ng-core/src/main/java/org/apache/flume/source/AbstractSource.java +++ b/flume-ng-core/src/main/java/org/apache/flume/source/AbstractSource.java @@ -29,7 +29,7 @@ @InterfaceAudience.Public @InterfaceStability.Stable -abstract public class AbstractSource implements Source { +public abstract class AbstractSource implements Source { private ChannelProcessor channelProcessor; private String name; @@ -79,6 +79,6 @@ public synchronized String getName() { } public String toString() { - return this.getClass().getName() + "{name:" + name + ",state:" + lifecycleState +"}"; + return this.getClass().getName() + "{name:" + name + ",state:" + lifecycleState + "}"; } } diff --git a/flume-ng-core/src/main/java/org/apache/flume/source/AvroSource.java b/flume-ng-core/src/main/java/org/apache/flume/source/AvroSource.java index 6eb6a0a180..8b9b956dee 100644 --- a/flume-ng-core/src/main/java/org/apache/flume/source/AvroSource.java +++ b/flume-ng-core/src/main/java/org/apache/flume/source/AvroSource.java @@ -21,18 +21,6 @@ import com.google.common.base.Preconditions; import com.google.common.base.Throwables; -import java.io.FileInputStream; -import java.net.InetSocketAddress; -import java.security.KeyStore; -import java.security.Security; -import java.util.*; -import java.util.concurrent.Executors; -import java.util.concurrent.ScheduledExecutorService; -import java.util.concurrent.TimeUnit; -import javax.net.ssl.KeyManagerFactory; -import javax.net.ssl.SSLContext; -import javax.net.ssl.SSLEngine; - import com.google.common.util.concurrent.ThreadFactoryBuilder; import org.apache.avro.ipc.NettyServer; import org.apache.avro.ipc.NettyTransceiver; @@ -53,10 +41,10 @@ import org.apache.flume.source.avro.AvroFlumeEvent; import org.apache.flume.source.avro.AvroSourceProtocol; import org.apache.flume.source.avro.Status; -import org.jboss.netty.channel.ChannelPipelineFactory; import org.jboss.netty.channel.ChannelPipeline; -import org.jboss.netty.channel.socket.nio.NioServerSocketChannelFactory; +import org.jboss.netty.channel.ChannelPipelineFactory; import org.jboss.netty.channel.Channels; +import org.jboss.netty.channel.socket.nio.NioServerSocketChannelFactory; import org.jboss.netty.handler.codec.compression.ZlibDecoder; import org.jboss.netty.handler.codec.compression.ZlibEncoder; import org.jboss.netty.handler.ipfilter.IpFilterRule; @@ -66,6 +54,23 @@ import org.slf4j.Logger; import org.slf4j.LoggerFactory; +import javax.net.ssl.KeyManagerFactory; +import javax.net.ssl.SSLContext; +import javax.net.ssl.SSLEngine; +import java.io.FileInputStream; +import java.net.InetSocketAddress; +import java.security.KeyStore; +import java.security.Security; +import java.util.ArrayList; +import java.util.Arrays; +import java.util.HashMap; +import java.util.LinkedList; +import java.util.List; +import java.util.Map; +import java.util.concurrent.Executors; +import java.util.concurrent.ScheduledExecutorService; +import java.util.concurrent.TimeUnit; + /** *

    * A {@link Source} implementation that receives Avro events from clients that @@ -205,7 +210,7 @@ public void configure(Context context) { if (enableIpFilter) { patternRuleConfigDefinition = context.getString(IP_FILTER_RULES_KEY); if (patternRuleConfigDefinition == null || - patternRuleConfigDefinition.trim().isEmpty()) { + patternRuleConfigDefinition.trim().isEmpty()) { throw new FlumeException( "ipFilter is configured with true but ipFilterRules is not defined:" + " "); @@ -241,7 +246,7 @@ public void start() { sourceCounter.start(); super.start(); final NettyServer srv = (NettyServer)server; - connectionCountUpdater.scheduleWithFixedDelay(new Runnable(){ + connectionCountUpdater.scheduleWithFixedDelay(new Runnable() { @Override public void run() { @@ -256,22 +261,17 @@ public void run() { private NioServerSocketChannelFactory initSocketChannelFactory() { NioServerSocketChannelFactory socketChannelFactory; if (maxThreads <= 0) { - socketChannelFactory = new NioServerSocketChannelFactory - (Executors.newCachedThreadPool(new ThreadFactoryBuilder(). - setNameFormat("Avro " + NettyTransceiver.class.getSimpleName() - + " Boss-%d").build()), - Executors.newCachedThreadPool(new ThreadFactoryBuilder(). - setNameFormat("Avro " + NettyTransceiver.class.getSimpleName() - + " I/O Worker-%d").build())); + socketChannelFactory = new NioServerSocketChannelFactory( + Executors.newCachedThreadPool(new ThreadFactoryBuilder().setNameFormat( + "Avro " + NettyTransceiver.class.getSimpleName() + " Boss-%d").build()), + Executors.newCachedThreadPool(new ThreadFactoryBuilder().setNameFormat( + "Avro " + NettyTransceiver.class.getSimpleName() + " I/O Worker-%d").build())); } else { socketChannelFactory = new NioServerSocketChannelFactory( - Executors.newCachedThreadPool(new ThreadFactoryBuilder(). - setNameFormat( - "Avro " + NettyTransceiver.class.getSimpleName() - + " Boss-%d").build()), - Executors.newFixedThreadPool(maxThreads, new ThreadFactoryBuilder(). - setNameFormat("Avro " + NettyTransceiver.class.getSimpleName() + - " I/O Worker-%d").build())); + Executors.newCachedThreadPool(new ThreadFactoryBuilder().setNameFormat( + "Avro " + NettyTransceiver.class.getSimpleName() + " Boss-%d").build()), + Executors.newFixedThreadPool(maxThreads, new ThreadFactoryBuilder().setNameFormat( + "Avro " + NettyTransceiver.class.getSimpleName() + " I/O Worker-%d").build())); } return socketChannelFactory; } @@ -309,7 +309,7 @@ public void stop() { } sourceCounter.stop(); connectionCountUpdater.shutdown(); - while(!connectionCountUpdater.isTerminated()){ + while (!connectionCountUpdater.isTerminated()) { try { Thread.sleep(100); } catch (InterruptedException ex) { @@ -399,44 +399,40 @@ public Status appendBatch(List events) { return Status.OK; } - private PatternRule generateRule( - String patternRuleDefinition) throws FlumeException { + private PatternRule generateRule(String patternRuleDefinition) throws FlumeException { patternRuleDefinition = patternRuleDefinition.trim(); //first validate the format int firstColonIndex = patternRuleDefinition.indexOf(":"); if (firstColonIndex == -1) { throw new FlumeException( - "Invalid ipFilter patternRule '" + patternRuleDefinition + + "Invalid ipFilter patternRule '" + patternRuleDefinition + "' should look like <'allow' or 'deny'>:<'ip' or " + "'name'>:"); } else { - String ruleAccessFlag = patternRuleDefinition.substring(0, - firstColonIndex); - int secondColonIndex = patternRuleDefinition.indexOf(":", - firstColonIndex + 1); - if ((!ruleAccessFlag.equals("allow") && - !ruleAccessFlag.equals("deny")) || secondColonIndex == -1) { + String ruleAccessFlag = patternRuleDefinition.substring(0, firstColonIndex); + int secondColonIndex = patternRuleDefinition.indexOf(":", firstColonIndex + 1); + if ((!ruleAccessFlag.equals("allow") && !ruleAccessFlag.equals("deny")) || + secondColonIndex == -1) { throw new FlumeException( - "Invalid ipFilter patternRule '" + patternRuleDefinition + + "Invalid ipFilter patternRule '" + patternRuleDefinition + "' should look like <'allow' or 'deny'>:<'ip' or " + "'name'>:"); } String patternTypeFlag = patternRuleDefinition.substring( - firstColonIndex + 1, secondColonIndex); - if ((!patternTypeFlag.equals("ip") && - !patternTypeFlag.equals("name"))) { + firstColonIndex + 1, secondColonIndex); + if ((!patternTypeFlag.equals("ip") && !patternTypeFlag.equals("name"))) { throw new FlumeException( - "Invalid ipFilter patternRule '" + patternRuleDefinition + + "Invalid ipFilter patternRule '" + patternRuleDefinition + "' should look like <'allow' or 'deny'>:<'ip' or " + "'name'>:"); } boolean isAllow = ruleAccessFlag.equals("allow"); String patternRuleString = (patternTypeFlag.equals("ip") ? "i" : "n") - + ":" + patternRuleDefinition.substring(secondColonIndex + 1); + + ":" + patternRuleDefinition.substring(secondColonIndex + 1); logger.info("Adding ipFilter PatternRule: " - + (isAllow ? "Allow" : "deny") + " " + patternRuleString); + + (isAllow ? "Allow" : "deny") + " " + patternRuleString); return new PatternRule(isAllow, patternRuleString); } } @@ -458,9 +454,9 @@ private class AdvancedChannelPipelineFactory private String patternRuleConfigDefinition; public AdvancedChannelPipelineFactory(boolean enableCompression, - boolean enableSsl, String keystore, String keystorePassword, - String keystoreType, boolean enableIpFilter, - String patternRuleConfigDefinition) { + boolean enableSsl, String keystore, String keystorePassword, + String keystoreType, boolean enableIpFilter, + String patternRuleConfigDefinition) { this.enableCompression = enableCompression; this.enableSsl = enableSsl; this.keystore = keystore; @@ -505,7 +501,6 @@ public ChannelPipeline getPipeline() throws Exception { pipeline.addFirst("inflater", new ZlibDecoder()); } - if (enableSsl) { SSLEngine sslEngine = createServerSSLContext().createSSLEngine(); sslEngine.setUseClientMode(false); @@ -527,11 +522,10 @@ public ChannelPipeline getPipeline() throws Exception { if (enableIpFilter) { logger.info("Setting up ipFilter with the following rule definition: " + - patternRuleConfigDefinition); + patternRuleConfigDefinition); IpFilterRuleHandler ipFilterHandler = new IpFilterRuleHandler(); ipFilterHandler.addAll(rules); - logger.info( - "Adding ipFilter with " + ipFilterHandler.size() + " rules"); + logger.info("Adding ipFilter with " + ipFilterHandler.size() + " rules"); pipeline.addFirst("ipFilter", ipFilterHandler); } diff --git a/flume-ng-core/src/main/java/org/apache/flume/source/BasicSourceSemantics.java b/flume-ng-core/src/main/java/org/apache/flume/source/BasicSourceSemantics.java index d2672b5f24..931e1e4b6d 100644 --- a/flume-ng-core/src/main/java/org/apache/flume/source/BasicSourceSemantics.java +++ b/flume-ng-core/src/main/java/org/apache/flume/source/BasicSourceSemantics.java @@ -44,8 +44,7 @@ @InterfaceAudience.Public @InterfaceStability.Evolving public abstract class BasicSourceSemantics implements Source, Configurable { - private static final Logger logger = LoggerFactory - .getLogger(BasicSourceSemantics.class); + private static final Logger logger = LoggerFactory.getLogger(BasicSourceSemantics.class); private Exception exception; private ChannelProcessor channelProcessor; private String name; @@ -54,9 +53,10 @@ public abstract class BasicSourceSemantics implements Source, Configurable { public BasicSourceSemantics() { lifecycleState = LifecycleState.IDLE; } + @Override public synchronized void configure(Context context) { - if(isStarted()) { + if (isStarted()) { throw new IllegalStateException("Configure called when started"); } else { try { @@ -126,8 +126,7 @@ public synchronized LifecycleState getLifecycleState() { } public String toString() { - return this.getClass().getName() + "{name:" + name + ",state:" - + lifecycleState +"}"; + return this.getClass().getName() + "{name:" + name + ",state:" + lifecycleState + "}"; } protected boolean isStarted() { diff --git a/flume-ng-core/src/main/java/org/apache/flume/source/DefaultSourceFactory.java b/flume-ng-core/src/main/java/org/apache/flume/source/DefaultSourceFactory.java index bb9d3f1693..f2da332692 100644 --- a/flume-ng-core/src/main/java/org/apache/flume/source/DefaultSourceFactory.java +++ b/flume-ng-core/src/main/java/org/apache/flume/source/DefaultSourceFactory.java @@ -46,13 +46,12 @@ public Source create(String name, String type) throws FlumeException { return source; } catch (Exception ex) { throw new FlumeException("Unable to create source: " + name - +", type: " + type + ", class: " + sourceClass.getName(), ex); + + ", type: " + type + ", class: " + sourceClass.getName(), ex); } } @SuppressWarnings("unchecked") @Override - public Class getClass(String type) - throws FlumeException { + public Class getClass(String type) throws FlumeException { String sourceClassName = type; SourceType srcType = SourceType.OTHER; try { diff --git a/flume-ng-core/src/main/java/org/apache/flume/source/ExecSource.java b/flume-ng-core/src/main/java/org/apache/flume/source/ExecSource.java index 18e662c79c..52ea8087cc 100644 --- a/flume-ng-core/src/main/java/org/apache/flume/source/ExecSource.java +++ b/flume-ng-core/src/main/java/org/apache/flume/source/ExecSource.java @@ -133,7 +133,8 @@ * * * batchTimeout - * Amount of time (in milliseconds) to wait, if the buffer size was not reached, before data is pushed downstream. + * Amount of time (in milliseconds) to wait, if the buffer size was not reached, + * before data is pushed downstream. * long * 3000 * @@ -145,11 +146,9 @@ * TODO *

    */ -public class ExecSource extends AbstractSource implements EventDrivenSource, -Configurable { +public class ExecSource extends AbstractSource implements EventDrivenSource, Configurable { - private static final Logger logger = LoggerFactory - .getLogger(ExecSource.class); + private static final Logger logger = LoggerFactory.getLogger(ExecSource.class); private String shell; private String command; @@ -190,7 +189,7 @@ public void start() { @Override public void stop() { logger.info("Stopping exec source with command:{}", command); - if(runner != null) { + if (runner != null) { runner.setRestart(false); runner.kill(); } @@ -298,7 +297,7 @@ public void run() { "timedFlushExecService" + Thread.currentThread().getId() + "-%d").build()); try { - if(shell != null) { + if (shell != null) { String[] commandArgs = formulateShellCommand(shell, command); process = Runtime.getRuntime().exec(commandArgs); } else { @@ -320,14 +319,14 @@ public void run() { public void run() { try { synchronized (eventList) { - if(!eventList.isEmpty() && timeout()) { + if (!eventList.isEmpty() && timeout()) { flushEventBatch(eventList); } } } catch (Exception e) { logger.error("Exception occured when processing event batch", e); - if(e instanceof InterruptedException) { - Thread.currentThread().interrupt(); + if (e instanceof InterruptedException) { + Thread.currentThread().interrupt(); } } } @@ -338,20 +337,20 @@ public void run() { synchronized (eventList) { sourceCounter.incrementEventReceivedCount(); eventList.add(EventBuilder.withBody(line.getBytes(charset))); - if(eventList.size() >= bufferCount || timeout()) { + if (eventList.size() >= bufferCount || timeout()) { flushEventBatch(eventList); } } } synchronized (eventList) { - if(!eventList.isEmpty()) { - flushEventBatch(eventList); - } + if (!eventList.isEmpty()) { + flushEventBatch(eventList); + } } } catch (Exception e) { logger.error("Failed while running command: " + command, e); - if(e instanceof InterruptedException) { + if (e instanceof InterruptedException) { Thread.currentThread().interrupt(); } } finally { @@ -364,7 +363,7 @@ public void run() { } exitCode = String.valueOf(kill()); } - if(restart) { + if (restart) { logger.info("Restarting in {}ms, exit code {}", restartThrottle, exitCode); try { @@ -375,17 +374,17 @@ public void run() { } else { logger.info("Command [" + command + "] exited with " + exitCode); } - } while(restart); + } while (restart); } - private void flushEventBatch(List eventList){ + private void flushEventBatch(List eventList) { channelProcessor.processEventBatch(eventList); sourceCounter.addToEventAcceptedCount(eventList.size()); eventList.clear(); lastPushToChannel = systemClock.currentTimeMillis(); } - private boolean timeout(){ + private boolean timeout() { return (systemClock.currentTimeMillis() - lastPushToChannel) >= batchTimeout; } @@ -398,7 +397,7 @@ private static String[] formulateShellCommand(String shell, String command) { } public int kill() { - if(process != null) { + if (process != null) { synchronized (process) { process.destroy(); @@ -407,7 +406,7 @@ public int kill() { // Stop the Thread that flushes periodically if (future != null) { - future.cancel(true); + future.cancel(true); } if (timedFlushService != null) { @@ -417,7 +416,7 @@ public int kill() { timedFlushService.awaitTermination(500, TimeUnit.MILLISECONDS); } catch (InterruptedException e) { logger.debug("Interrupted while waiting for exec executor service " - + "to stop. Just exiting."); + + "to stop. Just exiting."); Thread.currentThread().interrupt(); } } @@ -435,6 +434,7 @@ public void setRestart(boolean restart) { this.restart = restart; } } + private static class StderrReader extends Thread { private BufferedReader input; private boolean logStderr; @@ -449,8 +449,8 @@ public void run() { try { int i = 0; String line = null; - while((line = input.readLine()) != null) { - if(logStderr) { + while ((line = input.readLine()) != null) { + if (logStderr) { // There is no need to read 'line' with a charset // as we do not to propagate it. // It is in UTF-16 and would be printed in UTF-8 format. @@ -461,7 +461,7 @@ public void run() { logger.info("StderrLogger exiting", e); } finally { try { - if(input != null) { + if (input != null) { input.close(); } } catch (IOException ex) { diff --git a/flume-ng-core/src/main/java/org/apache/flume/source/ExecSourceConfigurationConstants.java b/flume-ng-core/src/main/java/org/apache/flume/source/ExecSourceConfigurationConstants.java index 957ec7fd97..ef1d51c315 100644 --- a/flume-ng-core/src/main/java/org/apache/flume/source/ExecSourceConfigurationConstants.java +++ b/flume-ng-core/src/main/java/org/apache/flume/source/ExecSourceConfigurationConstants.java @@ -18,7 +18,6 @@ */ package org.apache.flume.source; - public class ExecSourceConfigurationConstants { /** @@ -50,7 +49,7 @@ public class ExecSourceConfigurationConstants { * to data is pushed downstream: : default 3000 ms */ public static final String CONFIG_BATCH_TIME_OUT = "batchTimeout"; - public static final long DEFAULT_BATCH_TIME_OUT = 3000l; + public static final long DEFAULT_BATCH_TIME_OUT = 3000L; /** * Charset for reading input diff --git a/flume-ng-core/src/main/java/org/apache/flume/source/MultiportSyslogTCPSource.java b/flume-ng-core/src/main/java/org/apache/flume/source/MultiportSyslogTCPSource.java index 87f0db1609..b9f2438aa7 100644 --- a/flume-ng-core/src/main/java/org/apache/flume/source/MultiportSyslogTCPSource.java +++ b/flume-ng-core/src/main/java/org/apache/flume/source/MultiportSyslogTCPSource.java @@ -113,7 +113,7 @@ public void configure(Context context) { portCharsets.clear(); { ImmutableMap portCharsetCfg = context.getSubProperties( - SyslogSourceConfigurationConstants.CONFIG_PORT_CHARSET_PREFIX); + SyslogSourceConfigurationConstants.CONFIG_PORT_CHARSET_PREFIX); for (Map.Entry entry : portCharsetCfg.entrySet()) { String portStr = entry.getKey(); String charsetStr = entry.getValue(); @@ -386,7 +386,7 @@ Event parseEvent(ParsedBuffer parsedBuf, CharsetDecoder decoder) { */ static class LineSplitter { - private final static byte NEWLINE = '\n'; + private static final byte NEWLINE = '\n'; private final int maxLineLength; public LineSplitter(int maxLineLength) { diff --git a/flume-ng-core/src/main/java/org/apache/flume/source/NetcatSourceConfigurationConstants.java b/flume-ng-core/src/main/java/org/apache/flume/source/NetcatSourceConfigurationConstants.java index 1720d5f3cb..f3efddb096 100644 --- a/flume-ng-core/src/main/java/org/apache/flume/source/NetcatSourceConfigurationConstants.java +++ b/flume-ng-core/src/main/java/org/apache/flume/source/NetcatSourceConfigurationConstants.java @@ -42,9 +42,9 @@ public class NetcatSourceConfigurationConstants { public static final String CONFIG_MAX_LINE_LENGTH = "max-line-length"; public static final int DEFAULT_MAX_LINE_LENGTH = 512; - /** - * Encoding for the netcat source - */ + /** + * Encoding for the netcat source + */ public static final String CONFIG_SOURCE_ENCODING = "encoding"; public static final String DEFAULT_ENCODING = "utf-8"; } diff --git a/flume-ng-core/src/main/java/org/apache/flume/source/PollableSourceRunner.java b/flume-ng-core/src/main/java/org/apache/flume/source/PollableSourceRunner.java index ea37703aff..7357793c63 100644 --- a/flume-ng-core/src/main/java/org/apache/flume/source/PollableSourceRunner.java +++ b/flume-ng-core/src/main/java/org/apache/flume/source/PollableSourceRunner.java @@ -48,8 +48,7 @@ */ public class PollableSourceRunner extends SourceRunner { - private static final Logger logger = LoggerFactory - .getLogger(PollableSourceRunner.class); + private static final Logger logger = LoggerFactory.getLogger(PollableSourceRunner.class); private AtomicBoolean shouldStop; @@ -94,10 +93,7 @@ public void stop() { runnerThread.interrupt(); runnerThread.join(); } catch (InterruptedException e) { - logger - .warn( - "Interrupted while waiting for polling runner to stop. Please report this.", - e); + logger.warn("Interrupted while waiting for polling runner to stop. Please report this.", e); Thread.currentThread().interrupt(); } diff --git a/flume-ng-core/src/main/java/org/apache/flume/source/SequenceGeneratorSource.java b/flume-ng-core/src/main/java/org/apache/flume/source/SequenceGeneratorSource.java index 121463532c..9f831bdfd8 100644 --- a/flume-ng-core/src/main/java/org/apache/flume/source/SequenceGeneratorSource.java +++ b/flume-ng-core/src/main/java/org/apache/flume/source/SequenceGeneratorSource.java @@ -71,7 +71,7 @@ protected Status doProcess() throws EventDeliveryException { int i = 0; try { if (batchSize <= 1) { - if(eventsSent < totalEvents) { + if (eventsSent < totalEvents) { getChannelProcessor().processEvent( EventBuilder.withBody(String.valueOf(sequence++).getBytes())); sourceCounter.incrementEventAcceptedCount(); @@ -82,7 +82,7 @@ protected Status doProcess() throws EventDeliveryException { } else { batchArrayList.clear(); for (i = 0; i < batchSize; i++) { - if(eventsSent < totalEvents){ + if (eventsSent < totalEvents) { batchArrayList.add(i, EventBuilder.withBody(String .valueOf(sequence++).getBytes())); eventsSent++; @@ -90,7 +90,7 @@ protected Status doProcess() throws EventDeliveryException { status = Status.BACKOFF; } } - if(!batchArrayList.isEmpty()) { + if (!batchArrayList.isEmpty()) { getChannelProcessor().processEventBatch(batchArrayList); sourceCounter.incrementAppendBatchAcceptedCount(); sourceCounter.addToEventAcceptedCount(batchArrayList.size()); diff --git a/flume-ng-core/src/main/java/org/apache/flume/source/SpoolDirectorySource.java b/flume-ng-core/src/main/java/org/apache/flume/source/SpoolDirectorySource.java index 3af3e53f3f..d88cc1d8b7 100644 --- a/flume-ng-core/src/main/java/org/apache/flume/source/SpoolDirectorySource.java +++ b/flume-ng-core/src/main/java/org/apache/flume/source/SpoolDirectorySource.java @@ -20,7 +20,11 @@ import com.google.common.annotations.VisibleForTesting; import com.google.common.base.Preconditions; import com.google.common.base.Throwables; -import org.apache.flume.*; +import org.apache.flume.ChannelException; +import org.apache.flume.Context; +import org.apache.flume.Event; +import org.apache.flume.EventDrivenSource; +import org.apache.flume.FlumeException; import org.apache.flume.client.avro.ReliableSpoolingFileEventReader; import org.apache.flume.conf.Configurable; import org.apache.flume.instrumentation.SourceCounter; @@ -39,11 +43,10 @@ import static org.apache.flume.source.SpoolDirectorySourceConfigurationConstants.*; -public class SpoolDirectorySource extends AbstractSource implements -Configurable, EventDrivenSource { +public class SpoolDirectorySource extends AbstractSource + implements Configurable, EventDrivenSource { - private static final Logger logger = LoggerFactory - .getLogger(SpoolDirectorySource.class); + private static final Logger logger = LoggerFactory.getLogger(SpoolDirectorySource.class); /* Config options */ private String completedSuffix; @@ -124,8 +127,7 @@ public synchronized void stop() { super.stop(); sourceCounter.stop(); - logger.info("SpoolDir source {} stopped. Metrics: {}", getName(), - sourceCounter); + logger.info("SpoolDir source {} stopped. Metrics: {}", getName(), sourceCounter); } @Override @@ -247,8 +249,8 @@ public void run() { reader.commit(); } catch (ChannelException ex) { logger.warn("The channel is full, and cannot write data now. The " + - "source will try again after " + String.valueOf(backoffInterval) + - " milliseconds"); + "source will try again after " + String.valueOf(backoffInterval) + + " milliseconds"); hitChannelException = true; if (backoff) { TimeUnit.MILLISECONDS.sleep(backoffInterval); diff --git a/flume-ng-core/src/main/java/org/apache/flume/source/SpoolDirectorySourceConfigurationConstants.java b/flume-ng-core/src/main/java/org/apache/flume/source/SpoolDirectorySourceConfigurationConstants.java index 32b7837df6..5859aa239d 100644 --- a/flume-ng-core/src/main/java/org/apache/flume/source/SpoolDirectorySourceConfigurationConstants.java +++ b/flume-ng-core/src/main/java/org/apache/flume/source/SpoolDirectorySourceConfigurationConstants.java @@ -91,6 +91,7 @@ public class SpoolDirectorySourceConfigurationConstants { public enum ConsumeOrder { OLDEST, YOUNGEST, RANDOM } + public static final String CONSUME_ORDER = "consumeOrder"; public static final ConsumeOrder DEFAULT_CONSUME_ORDER = ConsumeOrder.OLDEST; diff --git a/flume-ng-core/src/main/java/org/apache/flume/source/StressSource.java b/flume-ng-core/src/main/java/org/apache/flume/source/StressSource.java index 9aa14779cd..aa9529462c 100644 --- a/flume-ng-core/src/main/java/org/apache/flume/source/StressSource.java +++ b/flume-ng-core/src/main/java/org/apache/flume/source/StressSource.java @@ -53,11 +53,9 @@ * * See {@link StressSource#configure(Context)} for configuration options. */ -public class StressSource extends AbstractPollableSource implements - Configurable { +public class StressSource extends AbstractPollableSource implements Configurable { - private static final Logger logger = LoggerFactory - .getLogger(StressSource.class); + private static final Logger logger = LoggerFactory.getLogger(StressSource.class); private CounterGroup counterGroup; private byte[] buffer; @@ -102,8 +100,7 @@ private void prepEventData(int bufferSize) { //Create event objects in case of batch test eventBatchList = new ArrayList(); - for (int i = 0; i < batchSize; i++) - { + for (int i = 0; i < batchSize; i++) { eventBatchList.add(EventBuilder.withBody(buffer)); } } else { diff --git a/flume-ng-core/src/main/java/org/apache/flume/source/SyslogParser.java b/flume-ng-core/src/main/java/org/apache/flume/source/SyslogParser.java index b57ffac73b..de727f60ba 100644 --- a/flume-ng-core/src/main/java/org/apache/flume/source/SyslogParser.java +++ b/flume-ng-core/src/main/java/org/apache/flume/source/SyslogParser.java @@ -115,7 +115,7 @@ public Event parseMessage(String msg, Charset charset, Set keepFields) { // remember version string String version = null; if (msgLen > curPos + 2 && "1 ".equals(msg.substring(curPos, curPos + 2))) { - version = msg.substring(curPos, curPos+1); + version = msg.substring(curPos, curPos + 1); headers.put(SyslogUtils.SYSLOG_VERSION, version); curPos += 2; } @@ -313,18 +313,19 @@ protected long parseRfc3164Time(String ts) { try { date = rfc3164Format.parseDateTime(ts); } catch (IllegalArgumentException e) { - logger.debug("rfc3164 date parse failed on ("+ts+"): invalid format", e); + logger.debug("rfc3164 date parse failed on (" + ts + "): invalid format", e); return 0; } // rfc3164 dates are really dumb. /* - * Some code to try and add some smarts to the year insertion as without a year in the message we - * need to make some educated guessing. + * Some code to try and add some smarts to the year insertion as without a year in the message + * we need to make some educated guessing. * First set the "fixed" to be the timestamp with the current year. * If the "fixed" time is more than one month in the future then roll it back a year. * If the "fixed" time is more than eleven months in the past then roll it forward a year. - * This gives us a 12 month rolling window (11 months in the past, 1 month in the future) of timestamps. + * This gives us a 12 month rolling window (11 months in the past, 1 month in the future) of + * timestamps. */ if (date != null) { @@ -332,7 +333,7 @@ protected long parseRfc3164Time(String ts) { // flume clock is ahead or there is some latency, and the year rolled if (fixed.isAfter(now) && fixed.minusMonths(1).isAfter(now)) { - fixed = date.minusYears(1); + fixed = date.minusYears(1); // flume clock is behind and the year rolled } else if (fixed.isBefore(now) && fixed.plusMonths(1).isBefore(now)) { fixed = date.plusYears(1); diff --git a/flume-ng-core/src/main/java/org/apache/flume/source/SyslogTcpSource.java b/flume-ng-core/src/main/java/org/apache/flume/source/SyslogTcpSource.java index bd87151738..185c00ca32 100644 --- a/flume-ng-core/src/main/java/org/apache/flume/source/SyslogTcpSource.java +++ b/flume-ng-core/src/main/java/org/apache/flume/source/SyslogTcpSource.java @@ -48,11 +48,9 @@ import org.slf4j.LoggerFactory; public class SyslogTcpSource extends AbstractSource -implements EventDrivenSource, Configurable { + implements EventDrivenSource, Configurable { + private static final Logger logger = LoggerFactory.getLogger(SyslogTcpSource.class); - - private static final Logger logger = LoggerFactory - .getLogger(SyslogTcpSource.class); private int port; private String host = null; private Channel nettyChannel; @@ -65,7 +63,7 @@ public class syslogTcpHandler extends SimpleChannelHandler { private SyslogUtils syslogUtils = new SyslogUtils(); - public void setEventSize(int eventSize){ + public void setEventSize(int eventSize) { syslogUtils.setEventSize(eventSize); } diff --git a/flume-ng-core/src/main/java/org/apache/flume/source/SyslogUDPSource.java b/flume-ng-core/src/main/java/org/apache/flume/source/SyslogUDPSource.java index 47993dd46c..175bebbcc5 100644 --- a/flume-ng-core/src/main/java/org/apache/flume/source/SyslogUDPSource.java +++ b/flume-ng-core/src/main/java/org/apache/flume/source/SyslogUDPSource.java @@ -35,14 +35,21 @@ import org.apache.flume.conf.Configurables; import org.jboss.netty.bootstrap.ConnectionlessBootstrap; import org.jboss.netty.buffer.ChannelBuffer; -import org.jboss.netty.channel.*; +import org.jboss.netty.channel.AdaptiveReceiveBufferSizePredictorFactory; +import org.jboss.netty.channel.Channel; +import org.jboss.netty.channel.ChannelHandlerContext; +import org.jboss.netty.channel.ChannelPipeline; +import org.jboss.netty.channel.ChannelPipelineFactory; +import org.jboss.netty.channel.Channels; +import org.jboss.netty.channel.MessageEvent; +import org.jboss.netty.channel.SimpleChannelHandler; import org.jboss.netty.channel.socket.oio.OioDatagramChannelFactory; import org.slf4j.Logger; import org.slf4j.LoggerFactory; public class SyslogUDPSource extends AbstractSource - implements EventDrivenSource, Configurable { + implements EventDrivenSource, Configurable { private int port; private int maxsize = 1 << 16; // 64k is max allowable in RFC 5426 @@ -51,8 +58,7 @@ public class SyslogUDPSource extends AbstractSource private Map formaterProp; private Set keepFields; - private static final Logger logger = LoggerFactory - .getLogger(SyslogUDPSource.class); + private static final Logger logger = LoggerFactory.getLogger(SyslogUDPSource.class); private CounterGroup counterGroup = new CounterGroup(); @@ -96,20 +102,20 @@ public void messageReceived(ChannelHandlerContext ctx, MessageEvent mEvent) { @Override public void start() { // setup Netty server - ConnectionlessBootstrap serverBootstrap = new ConnectionlessBootstrap - (new OioDatagramChannelFactory(Executors.newCachedThreadPool())); + ConnectionlessBootstrap serverBootstrap = new ConnectionlessBootstrap( + new OioDatagramChannelFactory(Executors.newCachedThreadPool())); final syslogHandler handler = new syslogHandler(); handler.setFormater(formaterProp); handler.setKeepFields(keepFields); serverBootstrap.setOption("receiveBufferSizePredictorFactory", - new AdaptiveReceiveBufferSizePredictorFactory(DEFAULT_MIN_SIZE, - DEFAULT_INITIAL_SIZE, maxsize)); + new AdaptiveReceiveBufferSizePredictorFactory(DEFAULT_MIN_SIZE, + DEFAULT_INITIAL_SIZE, maxsize)); serverBootstrap.setPipelineFactory(new ChannelPipelineFactory() { @Override public ChannelPipeline getPipeline() { - return Channels.pipeline(handler); + return Channels.pipeline(handler); } - }); + }); if (host == null) { nettyChannel = serverBootstrap.bind(new InetSocketAddress(port)); diff --git a/flume-ng-core/src/main/java/org/apache/flume/source/SyslogUtils.java b/flume-ng-core/src/main/java/org/apache/flume/source/SyslogUtils.java index 4866183fdb..43a10e1b4d 100644 --- a/flume-ng-core/src/main/java/org/apache/flume/source/SyslogUtils.java +++ b/flume-ng-core/src/main/java/org/apache/flume/source/SyslogUtils.java @@ -36,7 +36,6 @@ import java.util.Date; import java.util.HashMap; import java.util.HashSet; -import java.util.List; import java.util.Locale; import java.util.Map; import java.util.Set; @@ -47,38 +46,38 @@ @InterfaceAudience.Private @InterfaceStability.Evolving public class SyslogUtils { - final public static String SYSLOG_TIMESTAMP_FORMAT_RFC5424_2 = "yyyy-MM-dd'T'HH:mm:ss.SZ"; - final public static String SYSLOG_TIMESTAMP_FORMAT_RFC5424_1 = "yyyy-MM-dd'T'HH:mm:ss.S"; - final public static String SYSLOG_TIMESTAMP_FORMAT_RFC5424_3 = "yyyy-MM-dd'T'HH:mm:ssZ"; - final public static String SYSLOG_TIMESTAMP_FORMAT_RFC5424_4 = "yyyy-MM-dd'T'HH:mm:ss"; - final public static String SYSLOG_TIMESTAMP_FORMAT_RFC3164_1 = "yyyyMMM d HH:mm:ss"; + public static final String SYSLOG_TIMESTAMP_FORMAT_RFC5424_2 = "yyyy-MM-dd'T'HH:mm:ss.SZ"; + public static final String SYSLOG_TIMESTAMP_FORMAT_RFC5424_1 = "yyyy-MM-dd'T'HH:mm:ss.S"; + public static final String SYSLOG_TIMESTAMP_FORMAT_RFC5424_3 = "yyyy-MM-dd'T'HH:mm:ssZ"; + public static final String SYSLOG_TIMESTAMP_FORMAT_RFC5424_4 = "yyyy-MM-dd'T'HH:mm:ss"; + public static final String SYSLOG_TIMESTAMP_FORMAT_RFC3164_1 = "yyyyMMM d HH:mm:ss"; - final public static String SYSLOG_MSG_RFC5424_0 = + public static final String SYSLOG_MSG_RFC5424_0 = "(?:\\<(\\d{1,3})\\>)" + // priority - "(?:(\\d?)\\s?)" + // version - /* yyyy-MM-dd'T'HH:mm:ss.SZ or yyyy-MM-dd'T'HH:mm:ss.S+hh:mm or - (null stamp) */ - "(?:" + - "(\\d{4}[-]\\d{2}[-]\\d{2}[T]\\d{2}[:]\\d{2}[:]\\d{2}" + - "(?:\\.\\d{1,6})?(?:[+-]\\d{2}[:]\\d{2}|Z)?)|-)" + // stamp - "\\s" + // separator - "(?:([\\w][\\w\\d\\.@\\-]*)|-)" + // host name or - (null) - "\\s" + // separator - "(.*)$"; // body - - final public static String SYSLOG_MSG_RFC3164_0 = + "(?:(\\d?)\\s?)" + // version + /* yyyy-MM-dd'T'HH:mm:ss.SZ or yyyy-MM-dd'T'HH:mm:ss.S+hh:mm or - (null stamp) */ + "(?:" + + "(\\d{4}[-]\\d{2}[-]\\d{2}[T]\\d{2}[:]\\d{2}[:]\\d{2}" + + "(?:\\.\\d{1,6})?(?:[+-]\\d{2}[:]\\d{2}|Z)?)|-)" + // stamp + "\\s" + // separator + "(?:([\\w][\\w\\d\\.@\\-]*)|-)" + // host name or - (null) + "\\s" + // separator + "(.*)$"; // body + + public static final String SYSLOG_MSG_RFC3164_0 = "(?:\\<(\\d{1,3})\\>)" + - "(?:(\\d)?\\s?)" + // version - // stamp MMM d HH:mm:ss, single digit date has two spaces - "([A-Z][a-z][a-z]\\s{1,2}\\d{1,2}\\s\\d{2}[:]\\d{2}[:]\\d{2})" + - "\\s" + // separator - "([\\w][\\w\\d\\.@-]*)" + // host - "\\s(.*)$"; // body - - final public static int SYSLOG_PRIORITY_POS = 1; - final public static int SYSLOG_VERSION_POS = 2; - final public static int SYSLOG_TIMESTAMP_POS = 3; - final public static int SYSLOG_HOSTNAME_POS = 4; - final public static int SYSLOG_BODY_POS = 5; + "(?:(\\d)?\\s?)" + // version + // stamp MMM d HH:mm:ss, single digit date has two spaces + "([A-Z][a-z][a-z]\\s{1,2}\\d{1,2}\\s\\d{2}[:]\\d{2}[:]\\d{2})" + + "\\s" + // separator + "([\\w][\\w\\d\\.@-]*)" + // host + "\\s(.*)$"; // body + + public static final int SYSLOG_PRIORITY_POS = 1; + public static final int SYSLOG_VERSION_POS = 2; + public static final int SYSLOG_TIMESTAMP_POS = 3; + public static final int SYSLOG_HOSTNAME_POS = 4; + public static final int SYSLOG_BODY_POS = 5; private Mode m = Mode.START; private StringBuilder prio = new StringBuilder(); @@ -86,13 +85,13 @@ public class SyslogUtils { private static final Logger logger = LoggerFactory .getLogger(SyslogUtils.class); - final public static String SYSLOG_FACILITY = "Facility"; - final public static String SYSLOG_SEVERITY = "Severity"; - final public static String SYSLOG_PRIORITY = "Priority"; - final public static String SYSLOG_VERSION = "Version"; - final public static String EVENT_STATUS = "flume.syslog.status"; - final public static Integer MIN_SIZE = 10; - final public static Integer DEFAULT_SIZE = 2500; + public static final String SYSLOG_FACILITY = "Facility"; + public static final String SYSLOG_SEVERITY = "Severity"; + public static final String SYSLOG_PRIORITY = "Priority"; + public static final String SYSLOG_VERSION = "Version"; + public static final String EVENT_STATUS = "flume.syslog.status"; + public static final Integer MIN_SIZE = 10; + public static final Integer DEFAULT_SIZE = 2500; private final boolean isUdp; private boolean isBadEvent; private boolean isIncompleteEvent; @@ -106,6 +105,7 @@ private class SyslogFormatter { public ArrayList dateFormat = new ArrayList(); public boolean addYear; } + private ArrayList formats = new ArrayList(); private String priority = null; @@ -115,10 +115,10 @@ private class SyslogFormatter { private String msgBody = null; private static final String[] DEFAULT_FIELDS_TO_KEEP = { - SyslogSourceConfigurationConstants.CONFIG_KEEP_FIELDS_PRIORITY, - SyslogSourceConfigurationConstants.CONFIG_KEEP_FIELDS_VERSION, - SyslogSourceConfigurationConstants.CONFIG_KEEP_FIELDS_TIMESTAMP, - SyslogSourceConfigurationConstants.CONFIG_KEEP_FIELDS_HOSTNAME + SyslogSourceConfigurationConstants.CONFIG_KEEP_FIELDS_PRIORITY, + SyslogSourceConfigurationConstants.CONFIG_KEEP_FIELDS_VERSION, + SyslogSourceConfigurationConstants.CONFIG_KEEP_FIELDS_TIMESTAMP, + SyslogSourceConfigurationConstants.CONFIG_KEEP_FIELDS_HOSTNAME }; public static final String KEEP_FIELDS_ALL = "--all--"; @@ -211,22 +211,19 @@ public void addFormats(Map formatProp) { return; } SyslogFormatter fmt1 = new SyslogFormatter(); - fmt1.regexPattern = Pattern.compile( formatProp.get( - SyslogSourceConfigurationConstants.CONFIG_REGEX) ); - if (formatProp.containsKey( - SyslogSourceConfigurationConstants.CONFIG_SEARCH)) { - fmt1.searchPattern.add(formatProp.get( - SyslogSourceConfigurationConstants.CONFIG_SEARCH)); + fmt1.regexPattern = Pattern.compile( + formatProp.get(SyslogSourceConfigurationConstants.CONFIG_REGEX)); + if (formatProp.containsKey(SyslogSourceConfigurationConstants.CONFIG_SEARCH)) { + fmt1.searchPattern.add( + formatProp.get(SyslogSourceConfigurationConstants.CONFIG_SEARCH)); } - if (formatProp.containsKey( - SyslogSourceConfigurationConstants.CONFIG_REPLACE)) { - fmt1.replacePattern.add(formatProp.get( - SyslogSourceConfigurationConstants.CONFIG_REPLACE)); + if (formatProp.containsKey(SyslogSourceConfigurationConstants.CONFIG_REPLACE)) { + fmt1.replacePattern.add( + formatProp.get(SyslogSourceConfigurationConstants.CONFIG_REPLACE)); } - if (formatProp.containsKey( - SyslogSourceConfigurationConstants.CONFIG_DATEFORMAT)) { - fmt1.dateFormat.add(new SimpleDateFormat(formatProp.get( - SyslogSourceConfigurationConstants.CONFIG_DATEFORMAT))); + if (formatProp.containsKey(SyslogSourceConfigurationConstants.CONFIG_DATEFORMAT)) { + fmt1.dateFormat.add(new SimpleDateFormat( + formatProp.get(SyslogSourceConfigurationConstants.CONFIG_DATEFORMAT))); } formats.add(0, fmt1); } @@ -266,20 +263,22 @@ private void initHeaderFormats() { enum Mode { START, PRIO, DATA - }; + } - public enum SyslogStatus{ + ; + + public enum SyslogStatus { OTHER("Unknown"), INVALID("Invalid"), INCOMPLETE("Incomplete"); private final String syslogStatus; - private SyslogStatus(String status){ + private SyslogStatus(String status) { syslogStatus = status; } - public String getSyslogStatus(){ + public String getSyslogStatus() { return this.syslogStatus; } } @@ -292,14 +291,14 @@ Event buildEvent() { int sev = 0; int facility = 0; - if(!isBadEvent){ + if (!isBadEvent) { pri = Integer.parseInt(prio.toString()); sev = pri % 8; facility = pri / 8; formatHeaders(); } - Map headers = new HashMap(); + Map headers = new HashMap(); headers.put(SYSLOG_FACILITY, String.valueOf(facility)); headers.put(SYSLOG_SEVERITY, String.valueOf(sev)); if ((priority != null) && (priority.length() > 0)) { @@ -314,10 +313,10 @@ Event buildEvent() { if ((hostName != null) && (hostName.length() > 0)) { headers.put("host", hostName); } - if(isBadEvent){ + if (isBadEvent) { logger.warn("Event created from Invalid Syslog data."); headers.put(EVENT_STATUS, SyslogStatus.INVALID.getSyslogStatus()); - } else if(isIncompleteEvent){ + } else if (isIncompleteEvent) { logger.warn("Event size larger than specified event size: {}. You should " + "consider increasing your event size.", maxSize); headers.put(EVENT_STATUS, SyslogStatus.INCOMPLETE.getSyslogStatus()); @@ -345,22 +344,22 @@ private void formatHeaders() { String eventStr = baos.toString(); String timeStampString = null; - for(int p=0; p < formats.size(); p++) { + for (int p = 0; p < formats.size(); p++) { SyslogFormatter fmt = formats.get(p); Pattern pattern = fmt.regexPattern; Matcher matcher = pattern.matcher(eventStr); - if (! matcher.matches()) { + if (!matcher.matches()) { continue; } MatchResult res = matcher.toMatchResult(); - for (int grp=1; grp <= res.groupCount(); grp++) { + for (int grp = 1; grp <= res.groupCount(); grp++) { String value = res.group(grp); if (grp == SYSLOG_TIMESTAMP_POS) { timeStampString = value; // apply available format replacements to timestamp if (value != null) { - for (int sp=0; sp < fmt.searchPattern.size(); sp++) { + for (int sp = 0; sp < fmt.searchPattern.size(); sp++) { value = value.replaceAll(fmt.searchPattern.get(sp), fmt.replacePattern.get(sp)); } // Add year to timestamp if needed @@ -373,14 +372,16 @@ private void formatHeaders() { Date parsedDate = fmt.dateFormat.get(dt).parse(value); /* * Some code to try and add some smarts to the year insertion. - * Original code just added the current year which was okay-ish, but around January 1st becomes - * pretty naïve. - * The current year is added above. This code, if the year has been added does the following: + * Original code just added the current year which was okay-ish, but around + * January 1st becomes pretty naïve. + * The current year is added above. This code, if the year has been added does + * the following: * 1. Compute what the computed time, but one month in the past would be. * 2. Compute what the computed time, but eleven months in the future would be. - * If the computed time is more than one month in the future then roll it back a year. - * If the computed time is more than eleven months in the past then roll it forward a year. - * This gives us a 12 month rolling window (11 months in the past, 1 month in the future) of timestamps. + * If the computed time is more than one month in the future then roll it back a + * year. If the computed time is more than eleven months in the past then roll it + * forward a year. This gives us a 12 month rolling window (11 months in the past, + * 1 month in the future) of timestamps. */ if (fmt.addYear) { Calendar cal = Calendar.getInstance(); @@ -393,13 +394,15 @@ private void formatHeaders() { calPlusElevenMonths.setTime(parsedDate); calPlusElevenMonths.add(Calendar.MONTH, +11); - if (cal.getTimeInMillis() > System.currentTimeMillis() && calMinusOneMonth.getTimeInMillis() > System.currentTimeMillis()) { + if (cal.getTimeInMillis() > System.currentTimeMillis() && + calMinusOneMonth.getTimeInMillis() > System.currentTimeMillis()) { //Need to roll back a year Calendar c1 = Calendar.getInstance(); c1.setTime(parsedDate); c1.add(Calendar.YEAR, -1); parsedDate = c1.getTime(); - } else if (cal.getTimeInMillis() < System.currentTimeMillis() && calPlusElevenMonths.getTimeInMillis() < System.currentTimeMillis() ) { + } else if (cal.getTimeInMillis() < System.currentTimeMillis() && + calPlusElevenMonths.getTimeInMillis() < System.currentTimeMillis()) { //Need to roll forward a year Calendar c1 = Calendar.getInstance(); c1.setTime(parsedDate); @@ -422,14 +425,15 @@ private void formatHeaders() { } else if (grp == SYSLOG_VERSION_POS) { version = value; } else if (grp == SYSLOG_BODY_POS) { - msgBody = addFieldsToBody(keepFields, value, priority, version, timeStampString, hostName); + msgBody = addFieldsToBody(keepFields, value, priority, version, + timeStampString, hostName); } } break; // we successfully parsed the message using this pattern } } - private void reset(){ + private void reset() { baos.reset(); m = Mode.START; prio.delete(0, prio.length()); @@ -441,7 +445,7 @@ private void reset(){ } // extract relevant syslog data needed for building Flume event - public Event extractEvent(ChannelBuffer in){ + public Event extractEvent(ChannelBuffer in) { /* for protocol debugging ByteBuffer bb = in.toByteBuffer(); @@ -459,61 +463,61 @@ public Event extractEvent(ChannelBuffer in){ while (!doneReading && in.readable()) { b = in.readByte(); switch (m) { - case START: - if (b == '<') { - baos.write(b); - m = Mode.PRIO; - } else if(b == '\n'){ - //If the character is \n, it was because the last event was exactly - //as long as the maximum size allowed and - //the only remaining character was the delimiter - '\n', or - //multiple delimiters were sent in a row. - //Just ignore it, and move forward, don't change the mode. - //This is a no-op, just ignore it. - logger.debug("Delimiter found while in START mode, ignoring.."); - - } else { - isBadEvent = true; - baos.write(b); - //Bad event, just dump everything as if it is data. - m = Mode.DATA; - } - break; - case PRIO: - baos.write(b); - if (b == '>') { - if (prio.length() == 0) { - isBadEvent = true; - } - m = Mode.DATA; - } else { - char ch = (char) b; - prio.append(ch); - // Priority is max 3 digits per both RFC 3164 and 5424 - // With this check there is basically no danger of - // boas.size() exceeding this.maxSize before getting to the - // DATA state where this is actually checked - if (!Character.isDigit(ch) || prio.length() > 3) { + case START: + if (b == '<') { + baos.write(b); + m = Mode.PRIO; + } else if (b == '\n') { + //If the character is \n, it was because the last event was exactly + //as long as the maximum size allowed and + //the only remaining character was the delimiter - '\n', or + //multiple delimiters were sent in a row. + //Just ignore it, and move forward, don't change the mode. + //This is a no-op, just ignore it. + logger.debug("Delimiter found while in START mode, ignoring.."); + + } else { isBadEvent = true; - //If we hit a bad priority, just write as if everything is data. + baos.write(b); + //Bad event, just dump everything as if it is data. m = Mode.DATA; } - } - break; - case DATA: - // TCP syslog entries are separated by '\n' - if (b == '\n') { - e = buildEvent(); - doneReading = true; - } else { + break; + case PRIO: baos.write(b); - } - if(baos.size() == this.maxSize && !doneReading) { - isIncompleteEvent = true; - e = buildEvent(); - doneReading = true; - } - break; + if (b == '>') { + if (prio.length() == 0) { + isBadEvent = true; + } + m = Mode.DATA; + } else { + char ch = (char) b; + prio.append(ch); + // Priority is max 3 digits per both RFC 3164 and 5424 + // With this check there is basically no danger of + // boas.size() exceeding this.maxSize before getting to the + // DATA state where this is actually checked + if (!Character.isDigit(ch) || prio.length() > 3) { + isBadEvent = true; + //If we hit a bad priority, just write as if everything is data. + m = Mode.DATA; + } + } + break; + case DATA: + // TCP syslog entries are separated by '\n' + if (b == '\n') { + e = buildEvent(); + doneReading = true; + } else { + baos.write(b); + } + if (baos.size() == this.maxSize && !doneReading) { + isIncompleteEvent = true; + e = buildEvent(); + doneReading = true; + } + break; } } diff --git a/flume-ng-core/src/main/java/org/apache/flume/source/ThriftSource.java b/flume-ng-core/src/main/java/org/apache/flume/source/ThriftSource.java index 7df5ddbaad..6a25e64d1f 100644 --- a/flume-ng-core/src/main/java/org/apache/flume/source/ThriftSource.java +++ b/flume-ng-core/src/main/java/org/apache/flume/source/ThriftSource.java @@ -75,11 +75,10 @@ import java.util.concurrent.TimeUnit; import java.security.PrivilegedAction; -public class ThriftSource extends AbstractSource implements Configurable, - EventDrivenSource { +public class ThriftSource extends AbstractSource implements Configurable, EventDrivenSource { + + public static final Logger logger = LoggerFactory.getLogger(ThriftSource.class); - public static final Logger logger = LoggerFactory.getLogger(ThriftSource - .class); /** * Config param for the maximum number of threads this source should use to * handle incoming data. @@ -131,17 +130,17 @@ public void configure(Context context) { logger.info("Configuring thrift source."); port = context.getInteger(CONFIG_PORT); Preconditions.checkNotNull(port, "Port must be specified for Thrift " + - "Source."); + "Source."); bindAddress = context.getString(CONFIG_BIND); Preconditions.checkNotNull(bindAddress, "Bind address must be specified " + - "for Thrift Source."); + "for Thrift Source."); try { maxThreads = context.getInteger(CONFIG_THREADS, 0); maxThreads = (maxThreads <= 0) ? Integer.MAX_VALUE : maxThreads; } catch (NumberFormatException e) { logger.warn("Thrift source\'s \"threads\" property must specify an " + - "integer value: " + context.getString(CONFIG_THREADS)); + "integer value: " + context.getString(CONFIG_THREADS)); } if (sourceCounter == null) { @@ -190,8 +189,8 @@ public void configure(Context context) { String keytab = context.getString(AGENT_KEYTAB); enableKerberos = context.getBoolean(KERBEROS_KEY, false); this.flumeAuth = FlumeAuthenticationUtil.getAuthenticator(principal, keytab); - if(enableKerberos) { - if(!flumeAuth.isAuthenticated()) { + if (enableKerberos) { + if (!flumeAuth.isAuthenticated()) { throw new FlumeException("Authentication failed in Kerberos mode for " + "principal " + principal + " keytab " + keytab); } @@ -221,29 +220,27 @@ public void start() { servingExecutor.submit(new Runnable() { @Override public void run() { - flumeAuth.execute( - new PrivilegedAction() { - @Override - public Object run() { - server.serve(); - return null; - } + flumeAuth.execute(new PrivilegedAction() { + @Override + public Object run() { + server.serve(); + return null; } - ); + }); } }); long timeAfterStart = System.currentTimeMillis(); - while(!server.isServing()) { + while (!server.isServing()) { try { - if(System.currentTimeMillis() - timeAfterStart >=10000) { + if (System.currentTimeMillis() - timeAfterStart >= 10000) { throw new FlumeException("Thrift server failed to start!"); } TimeUnit.MILLISECONDS.sleep(1000); } catch (InterruptedException e) { Thread.currentThread().interrupt(); throw new FlumeException("Interrupted while waiting for Thrift server" + - " to start.", e); + " to start.", e); } } sourceCounter.start(); @@ -287,8 +284,7 @@ private TServerTransport getSSLServerTransport() { private TServerTransport getTServerTransport() { try { - return new TServerSocket(new InetSocketAddress - (bindAddress, port)); + return new TServerSocket(new InetSocketAddress(bindAddress, port)); } catch (Throwable throwable) { throw new FlumeException("Cannot start Thrift source.", throwable); } @@ -305,7 +301,7 @@ private TProtocolFactory getProtocolFactory() { } private TServer getTThreadedSelectorServer() { - if(enableSsl || enableKerberos) { + if (enableSsl || enableKerberos) { return null; } Class serverClass; @@ -345,7 +341,7 @@ private TServer getTThreadedSelectorServer() { * */ server = (TServer) serverClass.getConstructor(argsClass).newInstance(args); - } catch(ClassNotFoundException e) { + } catch (ClassNotFoundException e) { return null; } catch (Throwable ex) { throw new FlumeException("Cannot start Thrift Source.", ex); @@ -371,7 +367,7 @@ private void populateServerParams(TServer.AbstractServerArgs args) { args.protocolFactory(getProtocolFactory()); //populate the transportFactory - if(enableKerberos) { + if (enableKerberos) { args.transportFactory(getSASLTransportFactory()); } else { args.transportFactory(new TFastFramedTransport.Factory()); @@ -402,7 +398,7 @@ private TTransportFactory getSASLTransportFactory() { @Override public void stop() { - if(server != null && server.isServing()) { + if (server != null && server.isServing()) { server.stop(); } if (servingExecutor != null) { @@ -424,8 +420,7 @@ private class ThriftSourceHandler implements ThriftSourceProtocol.Iface { @Override public Status append(ThriftFlumeEvent event) throws TException { - Event flumeEvent = EventBuilder.withBody(event.getBody(), - event.getHeaders()); + Event flumeEvent = EventBuilder.withBody(event.getBody(), event.getHeaders()); sourceCounter.incrementAppendReceivedCount(); sourceCounter.incrementEventReceivedCount(); @@ -434,7 +429,7 @@ public Status append(ThriftFlumeEvent event) throws TException { getChannelProcessor().processEvent(flumeEvent); } catch (ChannelException ex) { logger.warn("Thrift source " + getName() + " could not append events " + - "to the channel.", ex); + "to the channel.", ex); return Status.FAILED; } sourceCounter.incrementAppendAcceptedCount(); @@ -448,16 +443,14 @@ public Status appendBatch(List events) throws TException { sourceCounter.addToEventReceivedCount(events.size()); List flumeEvents = Lists.newArrayList(); - for(ThriftFlumeEvent event : events) { - flumeEvents.add(EventBuilder.withBody(event.getBody(), - event.getHeaders())); + for (ThriftFlumeEvent event : events) { + flumeEvents.add(EventBuilder.withBody(event.getBody(), event.getHeaders())); } try { getChannelProcessor().processEventBatch(flumeEvents); } catch (ChannelException ex) { - logger.warn("Thrift source %s could not append events to the " + - "channel.", getName()); + logger.warn("Thrift source %s could not append events to the channel.", getName()); return Status.FAILED; } diff --git a/flume-ng-core/src/main/java/org/apache/flume/source/http/BLOBHandler.java b/flume-ng-core/src/main/java/org/apache/flume/source/http/BLOBHandler.java index a8163639a4..e24d4c6814 100644 --- a/flume-ng-core/src/main/java/org/apache/flume/source/http/BLOBHandler.java +++ b/flume-ng-core/src/main/java/org/apache/flume/source/http/BLOBHandler.java @@ -77,7 +77,7 @@ public List getEvents(HttpServletRequest request) throws Exception { } ByteArrayOutputStream outputStream = new ByteArrayOutputStream(); - try{ + try { IOUtils.copy(inputStream, outputStream); LOG.debug("Building an Event with stream of size -- {}", outputStream.size()); Event event = EventBuilder.withBody(outputStream.toByteArray(), headers); @@ -85,8 +85,7 @@ public List getEvents(HttpServletRequest request) throws Exception { List eventList = new ArrayList(); eventList.add(event); return eventList; - } - finally { + } finally { outputStream.close(); inputStream.close(); } @@ -94,7 +93,8 @@ public List getEvents(HttpServletRequest request) throws Exception { @Override public void configure(Context context) { - this.commaSeparatedHeaders = context.getString(MANDATORY_PARAMETERS, DEFAULT_MANDATORY_PARAMETERS); + this.commaSeparatedHeaders = context.getString(MANDATORY_PARAMETERS, + DEFAULT_MANDATORY_PARAMETERS); this.mandatoryHeaders = commaSeparatedHeaders.split(PARAMETER_SEPARATOR); } diff --git a/flume-ng-core/src/main/java/org/apache/flume/source/http/HTTPSource.java b/flume-ng-core/src/main/java/org/apache/flume/source/http/HTTPSource.java index b520b03282..38bdfda601 100644 --- a/flume-ng-core/src/main/java/org/apache/flume/source/http/HTTPSource.java +++ b/flume-ng-core/src/main/java/org/apache/flume/source/http/HTTPSource.java @@ -41,7 +41,12 @@ import javax.servlet.http.HttpServletResponse; import java.io.IOException; import java.net.ServerSocket; -import java.util.*; +import java.util.ArrayList; +import java.util.Arrays; +import java.util.Collections; +import java.util.LinkedList; +import java.util.List; +import java.util.Map; /** * A source which accepts Flume Events by HTTP POST and GET. GET should be used @@ -104,27 +109,28 @@ public void configure(Context context) { port = context.getInteger(HTTPSourceConfigurationConstants.CONFIG_PORT); host = context.getString(HTTPSourceConfigurationConstants.CONFIG_BIND, - HTTPSourceConfigurationConstants.DEFAULT_BIND); + HTTPSourceConfigurationConstants.DEFAULT_BIND); Preconditions.checkState(host != null && !host.isEmpty(), "HTTPSource hostname specified is empty"); Preconditions.checkNotNull(port, "HTTPSource requires a port number to be" - + " specified"); + + " specified"); String handlerClassName = context.getString( HTTPSourceConfigurationConstants.CONFIG_HANDLER, HTTPSourceConfigurationConstants.DEFAULT_HANDLER).trim(); - if(sslEnabled) { + if (sslEnabled) { LOG.debug("SSL configuration enabled"); keyStorePath = context.getString(HTTPSourceConfigurationConstants.SSL_KEYSTORE); Preconditions.checkArgument(keyStorePath != null && !keyStorePath.isEmpty(), - "Keystore is required for SSL Conifguration" ); - keyStorePassword = context.getString(HTTPSourceConfigurationConstants.SSL_KEYSTORE_PASSWORD); + "Keystore is required for SSL Conifguration" ); + keyStorePassword = + context.getString(HTTPSourceConfigurationConstants.SSL_KEYSTORE_PASSWORD); Preconditions.checkArgument(keyStorePassword != null, - "Keystore password is required for SSL Configuration"); - String excludeProtocolsStr = context.getString(HTTPSourceConfigurationConstants - .EXCLUDE_PROTOCOLS); + "Keystore password is required for SSL Configuration"); + String excludeProtocolsStr = + context.getString(HTTPSourceConfigurationConstants.EXCLUDE_PROTOCOLS); if (excludeProtocolsStr == null) { excludedProtocols.add("SSLv3"); } else { @@ -166,9 +172,9 @@ public void configure(Context context) { private void checkHostAndPort() { Preconditions.checkState(host != null && !host.isEmpty(), - "HTTPSource hostname specified is empty"); + "HTTPSource hostname specified is empty"); Preconditions.checkNotNull(port, "HTTPSource requires a port number to be" - + " specified"); + + " specified"); } @Override @@ -199,8 +205,7 @@ public void start() { connectors[0].setPort(port); srv.setConnectors(connectors); try { - org.mortbay.jetty.servlet.Context root = - new org.mortbay.jetty.servlet.Context( + org.mortbay.jetty.servlet.Context root = new org.mortbay.jetty.servlet.Context( srv, "/", org.mortbay.jetty.servlet.Context.SESSIONS); root.addServlet(new ServletHolder(new FlumeHTTPServlet()), "/"); HTTPServerConstraintUtil.enforceConstraints(root); @@ -285,26 +290,23 @@ public void doGet(HttpServletRequest request, HttpServletResponse response) } private static class HTTPSourceSocketConnector extends SslSocketConnector { - private final List excludedProtocols; + HTTPSourceSocketConnector(List excludedProtocols) { this.excludedProtocols = excludedProtocols; } @Override - public ServerSocket newServerSocket(String host, int port, - int backlog) throws IOException { - SSLServerSocket socket = (SSLServerSocket)super.newServerSocket(host, - port, backlog); + public ServerSocket newServerSocket(String host, int port, int backlog) throws IOException { + SSLServerSocket socket = (SSLServerSocket)super.newServerSocket(host, port, backlog); String[] protocols = socket.getEnabledProtocols(); List newProtocols = new ArrayList(protocols.length); - for(String protocol: protocols) { + for (String protocol: protocols) { if (!excludedProtocols.contains(protocol)) { newProtocols.add(protocol); } } - socket.setEnabledProtocols( - newProtocols.toArray(new String[newProtocols.size()])); + socket.setEnabledProtocols(newProtocols.toArray(new String[newProtocols.size()])); return socket; } } diff --git a/flume-ng-core/src/main/java/org/apache/flume/source/http/JSONHandler.java b/flume-ng-core/src/main/java/org/apache/flume/source/http/JSONHandler.java index 197f66a827..c99eb18cf5 100644 --- a/flume-ng-core/src/main/java/org/apache/flume/source/http/JSONHandler.java +++ b/flume-ng-core/src/main/java/org/apache/flume/source/http/JSONHandler.java @@ -35,13 +35,11 @@ import org.slf4j.LoggerFactory; /** - * * JSONHandler for HTTPSource that accepts an array of events. * * This handler throws exception if the deserialization fails because of bad * format or any other reason. * - * * Each event must be encoded as a map with two key-value pairs.

    1. headers * - the key for this key-value pair is "headers". The value for this key is * another map, which represent the event headers. These headers are inserted @@ -69,17 +67,15 @@ * {@linkplain Gson#toJson(java.lang.Object, java.lang.reflect.Type) } * method. The type token to pass as the 2nd argument of this method * for list of events can be created by:

    - * - * Type type = new TypeToken>() {}.getType();

    - * + * {@code + * Type type = new TypeToken>() {}.getType(); + * } */ public class JSONHandler implements HTTPSourceHandler { private static final Logger LOG = LoggerFactory.getLogger(JSONHandler.class); - private final Type listType = - new TypeToken>() { - }.getType(); + private final Type listType = new TypeToken>() {}.getType(); private final Gson gson; public JSONHandler() { @@ -131,7 +127,7 @@ public void configure(Context context) { private List getSimpleEvents(List events) { List newEvents = new ArrayList(events.size()); - for(Event e:events) { + for (Event e:events) { newEvents.add(EventBuilder.withBody(e.getBody(), e.getHeaders())); } return newEvents; diff --git a/flume-ng-core/src/main/java/org/apache/flume/tools/DirectMemoryUtils.java b/flume-ng-core/src/main/java/org/apache/flume/tools/DirectMemoryUtils.java index 8c2db2c0b7..dfa2229c00 100644 --- a/flume-ng-core/src/main/java/org/apache/flume/tools/DirectMemoryUtils.java +++ b/flume-ng-core/src/main/java/org/apache/flume/tools/DirectMemoryUtils.java @@ -54,7 +54,7 @@ public static ByteBuffer allocate(int size) { ByteBuffer result = ByteBuffer.allocateDirect(size); allocated.addAndGet(size); return result; - } catch(OutOfMemoryError error) { + } catch (OutOfMemoryError error) { LOG.error("Error allocating " + size + ", you likely want" + " to increase " + MAX_DIRECT_MEMORY_PARAM, error); throw error; @@ -88,11 +88,9 @@ public static long getDirectMemorySize() { if (memSize.contains("k")) { multiplier = 1024; - } - else if (memSize.contains("m")) { + } else if (memSize.contains("m")) { multiplier = 1048576; - } - else if (memSize.contains("g")) { + } else if (memSize.contains("g")) { multiplier = 1073741824; } memSize = memSize.replaceAll("[^\\d]", ""); @@ -107,7 +105,7 @@ private static long getDefaultDirectMemorySize() { Class VM = Class.forName("sun.misc.VM"); Method maxDirectMemory = VM.getDeclaredMethod("maxDirectMemory", (Class)null); Object result = maxDirectMemory.invoke(null, (Object[])null); - if(result != null && result instanceof Long) { + if (result != null && result instanceof Long) { return (Long)result; } } catch (Exception e) { diff --git a/flume-ng-core/src/main/java/org/apache/flume/tools/GetJavaProperty.java b/flume-ng-core/src/main/java/org/apache/flume/tools/GetJavaProperty.java index 5d0ea745db..bc073bb4ed 100644 --- a/flume-ng-core/src/main/java/org/apache/flume/tools/GetJavaProperty.java +++ b/flume-ng-core/src/main/java/org/apache/flume/tools/GetJavaProperty.java @@ -21,7 +21,7 @@ * A generic way for querying Java properties. */ public class GetJavaProperty { - public static void main(String args[]) { + public static void main(String[] args) { if (args.length == 0) { for (Object prop : System.getProperties().keySet()) { System.out.println(prop + "=" + System.getProperty((String)prop, "")); diff --git a/flume-ng-core/src/main/java/org/apache/flume/tools/TimestampRoundDownUtil.java b/flume-ng-core/src/main/java/org/apache/flume/tools/TimestampRoundDownUtil.java index 3a59953c16..daa9606ea9 100644 --- a/flume-ng-core/src/main/java/org/apache/flume/tools/TimestampRoundDownUtil.java +++ b/flume-ng-core/src/main/java/org/apache/flume/tools/TimestampRoundDownUtil.java @@ -35,7 +35,7 @@ public class TimestampRoundDownUtil { */ public static long roundDownTimeStampSeconds(long timestamp, int roundDownSec) throws IllegalStateException { - Preconditions.checkArgument(roundDownSec > 0 && roundDownSec <=60, + Preconditions.checkArgument(roundDownSec > 0 && roundDownSec <= 60, "RoundDownSec must be > 0 and <=60"); Calendar cal = roundDownField(timestamp, Calendar.SECOND, roundDownSec); cal.set(Calendar.MILLISECOND, 0); @@ -53,7 +53,7 @@ public static long roundDownTimeStampSeconds(long timestamp, */ public static long roundDownTimeStampMinutes(long timestamp, int roundDownMins) throws IllegalStateException { - Preconditions.checkArgument(roundDownMins > 0 && roundDownMins <=60, + Preconditions.checkArgument(roundDownMins > 0 && roundDownMins <= 60, "RoundDown must be > 0 and <=60"); Calendar cal = roundDownField(timestamp, Calendar.MINUTE, roundDownMins); cal.set(Calendar.SECOND, 0); @@ -73,7 +73,7 @@ public static long roundDownTimeStampMinutes(long timestamp, */ public static long roundDownTimeStampHours(long timestamp, int roundDownHours) throws IllegalStateException { - Preconditions.checkArgument(roundDownHours > 0 && roundDownHours <=24, + Preconditions.checkArgument(roundDownHours > 0 && roundDownHours <= 24, "RoundDown must be > 0 and <=24"); Calendar cal = roundDownField(timestamp, Calendar.HOUR_OF_DAY, roundDownHours); @@ -83,8 +83,7 @@ public static long roundDownTimeStampHours(long timestamp, return cal.getTimeInMillis(); } - private static Calendar roundDownField( - long timestamp, int field, int roundDown){ + private static Calendar roundDownField(long timestamp, int field, int roundDown) { Preconditions.checkArgument(timestamp > 0, "Timestamp must be positive"); Calendar cal = Calendar.getInstance(); cal.setTimeInMillis(timestamp); diff --git a/flume-ng-core/src/main/java/org/apache/flume/tools/VersionInfo.java b/flume-ng-core/src/main/java/org/apache/flume/tools/VersionInfo.java index c12cf8d3c6..95aa29d3f1 100644 --- a/flume-ng-core/src/main/java/org/apache/flume/tools/VersionInfo.java +++ b/flume-ng-core/src/main/java/org/apache/flume/tools/VersionInfo.java @@ -54,9 +54,9 @@ public static String getVersion() { * @return the revision number, eg. "100755" */ public static String getRevision() { - if(version != null - && version.revision() != null - && !version.revision().isEmpty()){ + if (version != null + && version.revision() != null + && !version.revision().isEmpty()) { return version.revision(); } return "Unknown"; @@ -105,12 +105,12 @@ public static String getSrcChecksum() { * Returns the build version info which includes version, * revision, user, date and source checksum */ - public static String getBuildVersion(){ + public static String getBuildVersion() { return VersionInfo.getVersion() + - " from " + VersionInfo.getRevision() + - " by " + VersionInfo.getUser() + - " on " + VersionInfo.getDate() + - " source checksum " + VersionInfo.getSrcChecksum(); + " from " + VersionInfo.getRevision() + + " by " + VersionInfo.getUser() + + " on " + VersionInfo.getDate() + + " source checksum " + VersionInfo.getSrcChecksum(); } public static void main(String[] args) { diff --git a/flume-ng-embedded-agent/src/main/java/org/apache/flume/agent/embedded/EmbeddedAgent.java b/flume-ng-embedded-agent/src/main/java/org/apache/flume/agent/embedded/EmbeddedAgent.java index 32c9f18cfd..ad3e138198 100644 --- a/flume-ng-embedded-agent/src/main/java/org/apache/flume/agent/embedded/EmbeddedAgent.java +++ b/flume-ng-embedded-agent/src/main/java/org/apache/flume/agent/embedded/EmbeddedAgent.java @@ -76,9 +76,11 @@ public class EmbeddedAgent { supervisor = new LifecycleSupervisor(); } + public EmbeddedAgent(String name) { this(new MaterializedConfigurationProvider(), name); } + /** * Configures the embedded agent. Can only be called after the object * is created or after the stop() method is called. @@ -89,12 +91,13 @@ public EmbeddedAgent(String name) { */ public void configure(Map properties) throws FlumeException { - if(state == State.STARTED) { + if (state == State.STARTED) { throw new IllegalStateException("Cannot be configured while started"); } doConfigure(properties); state = State.STOPPED; } + /** * Started the agent. Can only be called after a successful call to * configure(). @@ -105,9 +108,9 @@ public void configure(Map properties) */ public void start() throws FlumeException { - if(state == State.STARTED) { + if (state == State.STARTED) { throw new IllegalStateException("Cannot be started while started"); - } else if(state == State.NEW) { + } else if (state == State.NEW) { throw new IllegalStateException("Cannot be started before being " + "configured"); } @@ -115,15 +118,15 @@ public void start() // as doStart() accesses sourceRunner.getSource() Source source = Preconditions.checkNotNull(sourceRunner.getSource(), "Source runner returned null source"); - if(source instanceof EmbeddedSource) { + if (source instanceof EmbeddedSource) { embeddedSource = (EmbeddedSource)source; } else { - throw new IllegalStateException("Unknown source type: " + source. - getClass().getName()); + throw new IllegalStateException("Unknown source type: " + source.getClass().getName()); } doStart(); state = State.STARTED; } + /** * Stops the agent. Can only be called after a successful call to start(). * After a call to stop(), the agent can be re-configured with the @@ -134,7 +137,7 @@ public void start() */ public void stop() throws FlumeException { - if(state != State.STARTED) { + if (state != State.STARTED) { throw new IllegalStateException("Cannot be stopped unless started"); } supervisor.stop(); @@ -146,9 +149,9 @@ private void doConfigure(Map properties) { properties = EmbeddedAgentConfiguration.configure(name, properties); - if(LOGGER.isDebugEnabled()) { + if (LOGGER.isDebugEnabled()) { LOGGER.debug("Agent configuration values"); - for(String key : new TreeSet(properties.keySet())) { + for (String key : new TreeSet(properties.keySet())) { LOGGER.debug(key + " = " + properties.get(key)); } } @@ -156,17 +159,17 @@ private void doConfigure(Map properties) { MaterializedConfiguration conf = configurationProvider.get(name, properties); Map sources = conf.getSourceRunners(); - if(sources.size() != 1) { + if (sources.size() != 1) { throw new FlumeException("Expected one source and got " + sources.size()); } Map channels = conf.getChannels(); - if(channels.size() != 1) { + if (channels.size() != 1) { throw new FlumeException("Expected one channel and got " + channels.size()); } Map sinks = conf.getSinkRunners(); - if(sinks.size() != 1) { + if (sinks.size() != 1) { throw new FlumeException("Expected one sink group and got " + sinks.size()); } @@ -174,6 +177,7 @@ private void doConfigure(Map properties) { this.channel = channels.values().iterator().next(); this.sinkRunner = sinks.values().iterator().next(); } + /** * Adds event to the channel owned by the agent. Note however, that the * event is not copied and as such, the byte array and headers cannot @@ -182,7 +186,7 @@ private void doConfigure(Map properties) { * @throws EventDeliveryException if unable to add event to channel */ public void put(Event event) throws EventDeliveryException { - if(state != State.STARTED) { + if (state != State.STARTED) { throw new IllegalStateException("Cannot put events unless started"); } try { @@ -192,6 +196,7 @@ public void put(Event event) throws EventDeliveryException { ": Unable to process event: " + ex.getMessage(), ex); } } + /** * Adds events to the channel owned by the agent. Note however, that the * event is not copied and as such, the byte array and headers cannot @@ -200,7 +205,7 @@ public void put(Event event) throws EventDeliveryException { * @throws EventDeliveryException if unable to add event to channel */ public void putAll(List events) throws EventDeliveryException { - if(state != State.STARTED) { + if (state != State.STARTED) { throw new IllegalStateException("Cannot put events unless started"); } try { @@ -226,7 +231,7 @@ private void doStart() { new SupervisorPolicy.AlwaysRestartPolicy(), LifecycleState.START); error = false; } finally { - if(error) { + if (error) { stopLogError(sourceRunner); stopLogError(channel); stopLogError(sinkRunner); @@ -234,9 +239,10 @@ private void doStart() { } } } + private void stopLogError(LifecycleAware lifeCycleAware) { try { - if(LifecycleState.START.equals(lifeCycleAware.getLifecycleState())) { + if (LifecycleState.START.equals(lifeCycleAware.getLifecycleState())) { lifeCycleAware.stop(); } } catch (Exception e) { diff --git a/flume-ng-embedded-agent/src/main/java/org/apache/flume/agent/embedded/EmbeddedAgentConfiguration.java b/flume-ng-embedded-agent/src/main/java/org/apache/flume/agent/embedded/EmbeddedAgentConfiguration.java index 4a49fa0921..05d2d04443 100644 --- a/flume-ng-embedded-agent/src/main/java/org/apache/flume/agent/embedded/EmbeddedAgentConfiguration.java +++ b/flume-ng-embedded-agent/src/main/java/org/apache/flume/agent/embedded/EmbeddedAgentConfiguration.java @@ -136,8 +136,8 @@ public class EmbeddedAgentConfiguration { * Load balancing sink processor. See Flume User Guide for configuration * information. */ - public static final String SINK_PROCESSOR_TYPE_LOAD_BALANCE = SinkProcessorType.LOAD_BALANCE.name(); - + public static final String SINK_PROCESSOR_TYPE_LOAD_BALANCE = + SinkProcessorType.LOAD_BALANCE.name(); private static final String[] ALLOWED_SOURCES = { SOURCE_TYPE_EMBEDDED_ALIAS, @@ -165,22 +165,21 @@ public class EmbeddedAgentConfiguration { private static void validate(String name, Map properties) throws FlumeException { - if(properties.containsKey(SOURCE_TYPE)) { + if (properties.containsKey(SOURCE_TYPE)) { checkAllowed(ALLOWED_SOURCES, properties.get(SOURCE_TYPE)); } checkRequired(properties, CHANNEL_TYPE); checkAllowed(ALLOWED_CHANNELS, properties.get(CHANNEL_TYPE)); checkRequired(properties, SINKS); String sinkNames = properties.get(SINKS); - for(String sink : sinkNames.split("\\s+")) { - if(DISALLOWED_SINK_NAMES.contains(sink.toLowerCase(Locale.ENGLISH))) { + for (String sink : sinkNames.split("\\s+")) { + if (DISALLOWED_SINK_NAMES.contains(sink.toLowerCase(Locale.ENGLISH))) { throw new FlumeException("Sink name " + sink + " is one of the" + " disallowed sink names: " + DISALLOWED_SINK_NAMES); } String key = join(sink, TYPE); checkRequired(properties, key); checkAllowed(ALLOWED_SINKS, properties.get(key)); - } checkRequired(properties, SINK_PROCESSOR_TYPE); checkAllowed(ALLOWED_SINK_PROCESSORS, properties.get(SINK_PROCESSOR_TYPE)); @@ -201,8 +200,8 @@ static Map configure(String name, // we are going to modify the properties as we parse the config properties = new HashMap(properties); - if(!properties.containsKey(SOURCE_TYPE) || SOURCE_TYPE_EMBEDDED_ALIAS. - equalsIgnoreCase(properties.get(SOURCE_TYPE))) { + if (!properties.containsKey(SOURCE_TYPE) || + SOURCE_TYPE_EMBEDDED_ALIAS.equalsIgnoreCase(properties.get(SOURCE_TYPE))) { properties.put(SOURCE_TYPE, SOURCE_TYPE_EMBEDDED); } String sinkNames = properties.remove(SINKS); @@ -220,9 +219,6 @@ static Map configure(String name, // user supplied config -> agent configuration Map result = Maps.newHashMap(); - // properties will be modified during iteration so we need a - // copy of the keys - Set userProvidedKeys; /* * First we are going to setup all the root level pointers. I.E * point the agent at the components, sink group at sinks, and @@ -247,15 +243,19 @@ static Map configure(String name, result.put(join(name, BasicConfigurationConstants.CONFIG_SOURCES, sourceName, BasicConfigurationConstants.CONFIG_CHANNELS), channelName); + + // Properties will be modified during iteration so we need a + // copy of the keys. + Set userProvidedKeys = new HashSet(properties.keySet()); + /* * Second process the sink configuration and point the sinks * at the channel. */ - userProvidedKeys = new HashSet(properties.keySet()); - for(String sink : sinkNames.split("\\s+")) { - for(String key : userProvidedKeys) { + for (String sink : sinkNames.split("\\s+")) { + for (String key : userProvidedKeys) { String value = properties.get(key); - if(key.startsWith(sink + SEPERATOR)) { + if (key.startsWith(sink + SEPERATOR)) { properties.remove(key); result.put(join(name, BasicConfigurationConstants.CONFIG_SINKS, key), value); @@ -271,19 +271,19 @@ static Map configure(String name, * correctly and then passing them on to the agent. */ userProvidedKeys = new HashSet(properties.keySet()); - for(String key : userProvidedKeys) { + for (String key : userProvidedKeys) { String value = properties.get(key); - if(key.startsWith(SOURCE_PREFIX)) { + if (key.startsWith(SOURCE_PREFIX)) { // users use `source' but agent needs the actual source name key = key.replaceFirst(SOURCE, sourceName); result.put(join(name, BasicConfigurationConstants.CONFIG_SOURCES, key), value); - } else if(key.startsWith(CHANNEL_PREFIX)) { + } else if (key.startsWith(CHANNEL_PREFIX)) { // users use `channel' but agent needs the actual channel name key = key.replaceFirst(CHANNEL, channelName); result.put(join(name, BasicConfigurationConstants.CONFIG_CHANNELS, key), value); - } else if(key.startsWith(SINK_PROCESSOR_PREFIX)) { + } else if (key.startsWith(SINK_PROCESSOR_PREFIX)) { // agent.sinkgroups.sinkgroup.processor.* result.put(join(name, BasicConfigurationConstants.CONFIG_SINKGROUPS, sinkGroupName, key), value); @@ -297,20 +297,19 @@ static Map configure(String name, private static void checkAllowed(String[] allowedTypes, String type) { boolean isAllowed = false; type = type.trim(); - for(String allowedType : allowedTypes) { - if(allowedType.equalsIgnoreCase(type)) { + for (String allowedType : allowedTypes) { + if (allowedType.equalsIgnoreCase(type)) { isAllowed = true; break; } } - if(!isAllowed) { + if (!isAllowed) { throw new FlumeException("Component type of " + type + " is not in " + "allowed types of " + Arrays.toString(allowedTypes)); } } - private static void checkRequired(Map properties, - String name) { - if(!properties.containsKey(name)) { + private static void checkRequired(Map properties, String name) { + if (!properties.containsKey(name)) { throw new FlumeException("Required parameter not found " + name); } } @@ -319,7 +318,5 @@ private static String join(String... parts) { return JOINER.join(parts); } - private EmbeddedAgentConfiguration() { - - } + private EmbeddedAgentConfiguration() {} } \ No newline at end of file diff --git a/flume-ng-embedded-agent/src/main/java/org/apache/flume/agent/embedded/EmbeddedSource.java b/flume-ng-embedded-agent/src/main/java/org/apache/flume/agent/embedded/EmbeddedSource.java index 53389d20c0..71a88ec25c 100644 --- a/flume-ng-embedded-agent/src/main/java/org/apache/flume/agent/embedded/EmbeddedSource.java +++ b/flume-ng-embedded-agent/src/main/java/org/apache/flume/agent/embedded/EmbeddedSource.java @@ -35,16 +35,16 @@ */ @InterfaceAudience.Private @InterfaceStability.Unstable -public class EmbeddedSource extends AbstractSource - implements EventDrivenSource, Configurable { +public class EmbeddedSource extends AbstractSource implements EventDrivenSource, Configurable { @Override public void configure(Context context) { - } + public void put(Event event) throws ChannelException { getChannelProcessor().processEvent(event); } + public void putAll(List events) throws ChannelException { getChannelProcessor().processEventBatch(events); } diff --git a/flume-ng-legacy-sources/flume-thrift-source/src/main/java/org/apache/flume/source/thriftLegacy/ThriftLegacySource.java b/flume-ng-legacy-sources/flume-thrift-source/src/main/java/org/apache/flume/source/thriftLegacy/ThriftLegacySource.java index 47913dc1a7..7140f07204 100644 --- a/flume-ng-legacy-sources/flume-thrift-source/src/main/java/org/apache/flume/source/thriftLegacy/ThriftLegacySource.java +++ b/flume-ng-legacy-sources/flume-thrift-source/src/main/java/org/apache/flume/source/thriftLegacy/ThriftLegacySource.java @@ -27,6 +27,8 @@ import java.util.Map; import java.util.Map.Entry; +import com.cloudera.flume.handlers.thrift.ThriftFlumeEvent; +import com.cloudera.flume.handlers.thrift.ThriftFlumeEventServer; import org.apache.flume.ChannelException; import org.apache.flume.Context; import org.apache.flume.CounterGroup; @@ -45,8 +47,6 @@ import org.slf4j.Logger; import org.slf4j.LoggerFactory; -import com.cloudera.flume.handlers.thrift.*; - public class ThriftLegacySource extends AbstractSource implements EventDrivenSource, Configurable { @@ -86,7 +86,7 @@ public void append(ThriftFlumeEvent evt ) { headers.put(NANOS, Long.toString(evt.getNanos())); for (Entry entry: evt.getFields().entrySet()) { headers.put(entry.getKey().toString(), - UTF_8.decode(entry.getValue()).toString()); + UTF_8.decode(entry.getValue()).toString()); } headers.put(OG_EVENT, "yes"); @@ -139,8 +139,8 @@ public void start() { serverTransport = new TServerSocket(bindAddr); ThriftFlumeEventServer.Processor processor = new ThriftFlumeEventServer.Processor(new ThriftFlumeEventServerImpl()); - server = new TThreadPoolServer(new TThreadPoolServer. - Args(serverTransport).processor(processor)); + server = new TThreadPoolServer( + new TThreadPoolServer.Args(serverTransport).processor(processor)); } catch (TTransportException e) { throw new FlumeException("Failed starting source", e); } diff --git a/flume-ng-node/src/main/java/org/apache/flume/node/AbstractConfigurationProvider.java b/flume-ng-node/src/main/java/org/apache/flume/node/AbstractConfigurationProvider.java index 40abba2886..130bc641d9 100644 --- a/flume-ng-node/src/main/java/org/apache/flume/node/AbstractConfigurationProvider.java +++ b/flume-ng-node/src/main/java/org/apache/flume/node/AbstractConfigurationProvider.java @@ -17,8 +17,15 @@ */ package org.apache.flume.node; -import java.util.*; +import java.util.ArrayList; +import java.util.Enumeration; +import java.util.HashMap; +import java.util.HashSet; +import java.util.List; +import java.util.Map; import java.util.Map.Entry; +import java.util.Properties; +import java.util.Set; import org.apache.flume.Channel; import org.apache.flume.ChannelFactory; @@ -58,18 +65,15 @@ import com.google.common.collect.Lists; import com.google.common.collect.Maps; -public abstract class AbstractConfigurationProvider implements - ConfigurationProvider { +public abstract class AbstractConfigurationProvider implements ConfigurationProvider { - private static final Logger LOGGER = LoggerFactory - .getLogger(AbstractConfigurationProvider.class); + private static final Logger LOGGER = LoggerFactory.getLogger(AbstractConfigurationProvider.class); private final String agentName; private final SourceFactory sourceFactory; private final SinkFactory sinkFactory; private final ChannelFactory channelFactory; - private final Map, Map> channelCache; public AbstractConfigurationProvider(String agentName) { @@ -96,18 +100,16 @@ public MaterializedConfiguration getConfiguration() { loadChannels(agentConf, channelComponentMap); loadSources(agentConf, channelComponentMap, sourceRunnerMap); loadSinks(agentConf, channelComponentMap, sinkRunnerMap); - Set channelNames = - new HashSet(channelComponentMap.keySet()); - for(String channelName : channelNames) { - ChannelComponent channelComponent = channelComponentMap. - get(channelName); - if(channelComponent.components.isEmpty()) { + Set channelNames = new HashSet(channelComponentMap.keySet()); + for (String channelName : channelNames) { + ChannelComponent channelComponent = channelComponentMap.get(channelName); + if (channelComponent.components.isEmpty()) { LOGGER.warn(String.format("Channel %s has no components connected" + " and has been removed.", channelName)); channelComponentMap.remove(channelName); - Map nameChannelMap = channelCache. - get(channelComponent.channel.getClass()); - if(nameChannelMap != null) { + Map nameChannelMap = + channelCache.get(channelComponent.channel.getClass()); + if (nameChannelMap != null) { nameChannelMap.remove(channelName); } } else { @@ -116,10 +118,10 @@ public MaterializedConfiguration getConfiguration() { conf.addChannel(channelName, channelComponent.channel); } } - for(Map.Entry entry : sourceRunnerMap.entrySet()) { + for (Map.Entry entry : sourceRunnerMap.entrySet()) { conf.addSourceRunner(entry.getKey(), entry.getValue()); } - for(Map.Entry entry : sinkRunnerMap.entrySet()) { + for (Map.Entry entry : sinkRunnerMap.entrySet()) { conf.addSinkRunner(entry.getKey(), entry.getValue()); } } catch (InstantiationException ex) { @@ -155,21 +157,21 @@ private void loadChannels(AgentConfiguration agentConf, ListMultimap, String> channelsNotReused = ArrayListMultimap.create(); // assume all channels will not be re-used - for(Map.Entry, Map> entry : channelCache.entrySet()) { + for (Map.Entry, Map> entry : + channelCache.entrySet()) { Class channelKlass = entry.getKey(); Set channelNames = entry.getValue().keySet(); channelsNotReused.get(channelKlass).addAll(channelNames); } Set channelNames = agentConf.getChannelSet(); - Map compMap = - agentConf.getChannelConfigMap(); + Map compMap = agentConf.getChannelConfigMap(); /* * Components which have a ComponentConfiguration object */ for (String chName : channelNames) { ComponentConfiguration comp = compMap.get(chName); - if(comp != null) { + if (comp != null) { Channel channel = getOrCreateChannel(channelsNotReused, comp.getComponentName(), comp.getType()); try { @@ -190,17 +192,16 @@ private void loadChannels(AgentConfiguration agentConf, */ for (String chName : channelNames) { Context context = agentConf.getChannelContext().get(chName); - if(context != null){ - Channel channel = - getOrCreateChannel(channelsNotReused, chName, context.getString( - BasicConfigurationConstants.CONFIG_TYPE)); + if (context != null) { + Channel channel = getOrCreateChannel(channelsNotReused, chName, + context.getString(BasicConfigurationConstants.CONFIG_TYPE)); try { Configurables.configure(channel, context); channelComponentMap.put(chName, new ChannelComponent(channel)); LOGGER.info("Created channel " + chName); } catch (Exception e) { String msg = String.format("Channel %s has been removed due to an " + - "error during configuration", chName); + "error during configuration", chName); LOGGER.error(msg, e); } } @@ -212,7 +213,7 @@ private void loadChannels(AgentConfiguration agentConf, Map channelMap = channelCache.get(channelKlass); if (channelMap != null) { for (String channelName : channelsNotReused.get(channelKlass)) { - if(channelMap.remove(channelName) != null) { + if (channelMap.remove(channelName) != null) { LOGGER.info("Removed {} of type {}", channelName, channelKlass); } } @@ -228,12 +229,11 @@ private Channel getOrCreateChannel( String name, String type) throws FlumeException { - Class channelClass = channelFactory. - getClass(type); + Class channelClass = channelFactory.getClass(type); /* * Channel has requested a new instance on each re-configuration */ - if(channelClass.isAnnotationPresent(Disposable.class)) { + if (channelClass.isAnnotationPresent(Disposable.class)) { Channel channel = channelFactory.create(name, type); channel.setName(name); return channel; @@ -244,7 +244,7 @@ private Channel getOrCreateChannel( channelCache.put(channelClass, channelMap); } Channel channel = channelMap.get(name); - if(channel == null) { + if (channel == null) { channel = channelFactory.create(name, type); channel.setName(name); channelMap.put(name, channel); @@ -266,7 +266,7 @@ private void loadSources(AgentConfiguration agentConf, */ for (String sourceName : sourceNames) { ComponentConfiguration comp = compMap.get(sourceName); - if(comp != null) { + if (comp != null) { SourceConfiguration config = (SourceConfiguration) comp; Source source = sourceFactory.create(comp.getComponentName(), @@ -277,11 +277,11 @@ private void loadSources(AgentConfiguration agentConf, List sourceChannels = new ArrayList(); for (String chName : channelNames) { ChannelComponent channelComponent = channelComponentMap.get(chName); - if(channelComponent != null) { + if (channelComponent != null) { sourceChannels.add(channelComponent.channel); } } - if(sourceChannels.isEmpty()) { + if (sourceChannels.isEmpty()) { String msg = String.format("Source %s is not connected to a " + "channel", sourceName); throw new IllegalStateException(msg); @@ -298,10 +298,10 @@ private void loadSources(AgentConfiguration agentConf, source.setChannelProcessor(channelProcessor); sourceRunnerMap.put(comp.getComponentName(), SourceRunner.forSource(source)); - for(Channel channel : sourceChannels) { - ChannelComponent channelComponent = Preconditions. - checkNotNull(channelComponentMap.get(channel.getName()), - String.format("Channel %s", channel.getName())); + for (Channel channel : sourceChannels) { + ChannelComponent channelComponent = + Preconditions.checkNotNull(channelComponentMap.get(channel.getName()), + String.format("Channel %s", channel.getName())); channelComponent.components.add(sourceName); } } catch (Exception e) { @@ -318,10 +318,10 @@ private void loadSources(AgentConfiguration agentConf, Map sourceContexts = agentConf.getSourceContext(); for (String sourceName : sourceNames) { Context context = sourceContexts.get(sourceName); - if(context != null){ + if (context != null) { Source source = sourceFactory.create(sourceName, - context.getString(BasicConfigurationConstants.CONFIG_TYPE)); + context.getString(BasicConfigurationConstants.CONFIG_TYPE)); try { Configurables.configure(source, context); List sourceChannels = new ArrayList(); @@ -329,11 +329,11 @@ private void loadSources(AgentConfiguration agentConf, BasicConfigurationConstants.CONFIG_CHANNELS).split("\\s+"); for (String chName : channelNames) { ChannelComponent channelComponent = channelComponentMap.get(chName); - if(channelComponent != null) { + if (channelComponent != null) { sourceChannels.add(channelComponent.channel); } } - if(sourceChannels.isEmpty()) { + if (sourceChannels.isEmpty()) { String msg = String.format("Source %s is not connected to a " + "channel", sourceName); throw new IllegalStateException(msg); @@ -349,10 +349,10 @@ private void loadSources(AgentConfiguration agentConf, source.setChannelProcessor(channelProcessor); sourceRunnerMap.put(sourceName, SourceRunner.forSource(source)); - for(Channel channel : sourceChannels) { - ChannelComponent channelComponent = Preconditions. - checkNotNull(channelComponentMap.get(channel.getName()), - String.format("Channel %s", channel.getName())); + for (Channel channel : sourceChannels) { + ChannelComponent channelComponent = + Preconditions.checkNotNull(channelComponentMap.get(channel.getName()), + String.format("Channel %s", channel.getName())); channelComponent.components.add(sourceName); } } catch (Exception e) { @@ -376,15 +376,13 @@ private void loadSinks(AgentConfiguration agentConf, */ for (String sinkName : sinkNames) { ComponentConfiguration comp = compMap.get(sinkName); - if(comp != null) { + if (comp != null) { SinkConfiguration config = (SinkConfiguration) comp; - Sink sink = sinkFactory.create(comp.getComponentName(), - comp.getType()); + Sink sink = sinkFactory.create(comp.getComponentName(), comp.getType()); try { Configurables.configure(sink, config); - ChannelComponent channelComponent = channelComponentMap. - get(config.getChannel()); - if(channelComponent == null) { + ChannelComponent channelComponent = channelComponentMap.get(config.getChannel()); + if (channelComponent == null) { String msg = String.format("Sink %s is not connected to a " + "channel", sinkName); throw new IllegalStateException(msg); @@ -406,14 +404,15 @@ private void loadSinks(AgentConfiguration agentConf, Map sinkContexts = agentConf.getSinkContext(); for (String sinkName : sinkNames) { Context context = sinkContexts.get(sinkName); - if(context != null) { + if (context != null) { Sink sink = sinkFactory.create(sinkName, context.getString( BasicConfigurationConstants.CONFIG_TYPE)); try { Configurables.configure(sink, context); - ChannelComponent channelComponent = channelComponentMap. - get(context.getString(BasicConfigurationConstants.CONFIG_CHANNEL)); - if(channelComponent == null) { + ChannelComponent channelComponent = + channelComponentMap.get( + context.getString(BasicConfigurationConstants.CONFIG_CHANNEL)); + if (channelComponent == null) { String msg = String.format("Sink %s is not connected to a " + "channel", sinkName); throw new IllegalStateException(msg); @@ -441,7 +440,7 @@ private void loadSinkGroups(AgentConfiguration agentConf, Map usedSinks = new HashMap(); for (String groupName: sinkGroupNames) { ComponentConfiguration comp = compMap.get(groupName); - if(comp != null) { + if (comp != null) { SinkGroupConfiguration groupConf = (SinkGroupConfiguration) comp; List groupSinks = new ArrayList(); for (String sink : groupConf.getSinks()) { @@ -475,7 +474,7 @@ private void loadSinkGroups(AgentConfiguration agentConf, } } // add any unassigned sinks to solo collectors - for(Entry entry : sinks.entrySet()) { + for (Entry entry : sinks.entrySet()) { if (!usedSinks.containsValue(entry.getKey())) { try { SinkProcessor pr = new DefaultSinkProcessor(); @@ -483,9 +482,8 @@ private void loadSinkGroups(AgentConfiguration agentConf, sinkMap.add(entry.getValue()); pr.setSinks(sinkMap); Configurables.configure(pr, new Context()); - sinkRunnerMap.put(entry.getKey(), - new SinkRunner(pr)); - } catch(Exception e) { + sinkRunnerMap.put(entry.getKey(), new SinkRunner(pr)); + } catch (Exception e) { String msg = String.format("SinkGroup %s has been removed due to " + "an error during configuration", entry.getKey()); LOGGER.error(msg, e); @@ -496,6 +494,7 @@ private void loadSinkGroups(AgentConfiguration agentConf, private static class ChannelComponent { final Channel channel; final List components; + ChannelComponent(Channel channel) { this.channel = channel; components = Lists.newArrayList(); diff --git a/flume-ng-node/src/main/java/org/apache/flume/node/Application.java b/flume-ng-node/src/main/java/org/apache/flume/node/Application.java index 959fa779d8..d6d92f0295 100644 --- a/flume-ng-node/src/main/java/org/apache/flume/node/Application.java +++ b/flume-ng-node/src/main/java/org/apache/flume/node/Application.java @@ -19,15 +19,10 @@ package org.apache.flume.node; -import java.io.File; -import java.io.IOException; -import java.util.ArrayList; -import java.util.List; -import java.util.Locale; -import java.util.Map.Entry; -import java.util.Properties; -import java.util.Set; - +import com.google.common.base.Throwables; +import com.google.common.collect.Lists; +import com.google.common.eventbus.EventBus; +import com.google.common.eventbus.Subscribe; import org.apache.commons.cli.CommandLine; import org.apache.commons.cli.CommandLineParser; import org.apache.commons.cli.GnuParser; @@ -49,10 +44,14 @@ import org.slf4j.Logger; import org.slf4j.LoggerFactory; -import com.google.common.base.Throwables; -import com.google.common.collect.Lists; -import com.google.common.eventbus.EventBus; -import com.google.common.eventbus.Subscribe; +import java.io.File; +import java.io.IOException; +import java.util.ArrayList; +import java.util.List; +import java.util.Locale; +import java.util.Map.Entry; +import java.util.Properties; +import java.util.Set; public class Application { @@ -77,7 +76,7 @@ public Application(List components) { } public synchronized void start() { - for(LifecycleAware component : components) { + for (LifecycleAware component : components) { supervisor.supervise(component, new SupervisorPolicy.AlwaysRestartPolicy(), LifecycleState.START); } @@ -91,7 +90,7 @@ public synchronized void handleConfigurationEvent(MaterializedConfiguration conf public synchronized void stop() { supervisor.stop(); - if(monitorServer != null) { + if (monitorServer != null) { monitorServer.stop(); } } @@ -99,37 +98,37 @@ public synchronized void stop() { private void stopAllComponents() { if (this.materializedConfiguration != null) { logger.info("Shutting down configuration: {}", this.materializedConfiguration); - for (Entry entry : this.materializedConfiguration - .getSourceRunners().entrySet()) { - try{ + for (Entry entry : + this.materializedConfiguration.getSourceRunners().entrySet()) { + try { logger.info("Stopping Source " + entry.getKey()); supervisor.unsupervise(entry.getValue()); - } catch (Exception e){ + } catch (Exception e) { logger.error("Error while stopping {}", entry.getValue(), e); } } for (Entry entry : - this.materializedConfiguration.getSinkRunners().entrySet()) { - try{ + this.materializedConfiguration.getSinkRunners().entrySet()) { + try { logger.info("Stopping Sink " + entry.getKey()); supervisor.unsupervise(entry.getValue()); - } catch (Exception e){ + } catch (Exception e) { logger.error("Error while stopping {}", entry.getValue(), e); } } for (Entry entry : - this.materializedConfiguration.getChannels().entrySet()) { - try{ + this.materializedConfiguration.getChannels().entrySet()) { + try { logger.info("Stopping Channel " + entry.getKey()); supervisor.unsupervise(entry.getValue()); - } catch (Exception e){ + } catch (Exception e) { logger.error("Error while stopping {}", entry.getValue(), e); } } } - if(monitorServer != null) { + if (monitorServer != null) { monitorServer.stop(); } } @@ -140,12 +139,12 @@ private void startAllComponents(MaterializedConfiguration materializedConfigurat this.materializedConfiguration = materializedConfiguration; for (Entry entry : - materializedConfiguration.getChannels().entrySet()) { - try{ + materializedConfiguration.getChannels().entrySet()) { + try { logger.info("Starting Channel " + entry.getKey()); supervisor.supervise(entry.getValue(), new SupervisorPolicy.AlwaysRestartPolicy(), LifecycleState.START); - } catch (Exception e){ + } catch (Exception e) { logger.error("Error while starting {}", entry.getValue(), e); } } @@ -153,9 +152,9 @@ private void startAllComponents(MaterializedConfiguration materializedConfigurat /* * Wait for all channels to start. */ - for(Channel ch: materializedConfiguration.getChannels().values()){ - while(ch.getLifecycleState() != LifecycleState.START - && !supervisor.isComponentInErrorState(ch)){ + for (Channel ch : materializedConfiguration.getChannels().values()) { + while (ch.getLifecycleState() != LifecycleState.START + && !supervisor.isComponentInErrorState(ch)) { try { logger.info("Waiting for channel: " + ch.getName() + " to start. Sleeping for 500 ms"); @@ -167,23 +166,22 @@ private void startAllComponents(MaterializedConfiguration materializedConfigurat } } - for (Entry entry : materializedConfiguration.getSinkRunners() - .entrySet()) { - try{ + for (Entry entry : materializedConfiguration.getSinkRunners().entrySet()) { + try { logger.info("Starting Sink " + entry.getKey()); supervisor.supervise(entry.getValue(), - new SupervisorPolicy.AlwaysRestartPolicy(), LifecycleState.START); + new SupervisorPolicy.AlwaysRestartPolicy(), LifecycleState.START); } catch (Exception e) { logger.error("Error while starting {}", entry.getValue(), e); } } - for (Entry entry : materializedConfiguration - .getSourceRunners().entrySet()) { - try{ + for (Entry entry : + materializedConfiguration.getSourceRunners().entrySet()) { + try { logger.info("Starting Source " + entry.getKey()); supervisor.supervise(entry.getValue(), - new SupervisorPolicy.AlwaysRestartPolicy(), LifecycleState.START); + new SupervisorPolicy.AlwaysRestartPolicy(), LifecycleState.START); } catch (Exception e) { logger.error("Error while starting {}", entry.getValue(), e); } @@ -203,7 +201,7 @@ private void loadMonitoring() { try { //Is it a known type? klass = MonitoringType.valueOf( - monitorType.toUpperCase(Locale.ENGLISH)).getMonitorClass(); + monitorType.toUpperCase(Locale.ENGLISH)).getMonitorClass(); } catch (Exception e) { //Not a known type, use FQCN klass = (Class) Class.forName(monitorType); @@ -213,7 +211,7 @@ private void loadMonitoring() { for (String key : keys) { if (key.startsWith(CONF_MONITOR_PREFIX)) { context.put(key.substring(CONF_MONITOR_PREFIX.length()), - systemProps.getProperty(key)); + systemProps.getProperty(key)); } } monitorServer.configure(context); @@ -221,7 +219,7 @@ private void loadMonitoring() { } } catch (Exception e) { logger.warn("Error starting monitoring. " - + "Monitoring might not be available.", e); + + "Monitoring might not be available.", e); } } @@ -285,18 +283,17 @@ public static void main(String[] args) { EventBus eventBus = new EventBus(agentName + "-event-bus"); List components = Lists.newArrayList(); PollingZooKeeperConfigurationProvider zookeeperConfigurationProvider = - new PollingZooKeeperConfigurationProvider( - agentName, zkConnectionStr, baseZkPath, eventBus); + new PollingZooKeeperConfigurationProvider( + agentName, zkConnectionStr, baseZkPath, eventBus); components.add(zookeeperConfigurationProvider); application = new Application(components); eventBus.register(application); } else { StaticZooKeeperConfigurationProvider zookeeperConfigurationProvider = - new StaticZooKeeperConfigurationProvider( - agentName, zkConnectionStr, baseZkPath); + new StaticZooKeeperConfigurationProvider( + agentName, zkConnectionStr, baseZkPath); application = new Application(); - application.handleConfigurationEvent(zookeeperConfigurationProvider - .getConfiguration()); + application.handleConfigurationEvent(zookeeperConfigurationProvider.getConfiguration()); } } else { File configurationFile = new File(commandLine.getOptionValue('f')); @@ -308,16 +305,16 @@ public static void main(String[] args) { if (!configurationFile.exists()) { // If command line invocation, then need to fail fast if (System.getProperty(Constants.SYSPROP_CALLED_FROM_SERVICE) == - null) { + null) { String path = configurationFile.getPath(); try { path = configurationFile.getCanonicalPath(); } catch (IOException ex) { logger.error("Failed to read canonical path for file: " + path, - ex); + ex); } throw new ParseException( - "The specified configuration file does not exist: " + path); + "The specified configuration file does not exist: " + path); } } List components = Lists.newArrayList(); @@ -325,18 +322,16 @@ public static void main(String[] args) { if (reload) { EventBus eventBus = new EventBus(agentName + "-event-bus"); PollingPropertiesFileConfigurationProvider configurationProvider = - new PollingPropertiesFileConfigurationProvider( - agentName, configurationFile, eventBus, 30); + new PollingPropertiesFileConfigurationProvider( + agentName, configurationFile, eventBus, 30); components.add(configurationProvider); application = new Application(components); eventBus.register(application); } else { PropertiesFileConfigurationProvider configurationProvider = - new PropertiesFileConfigurationProvider( - agentName, configurationFile); + new PropertiesFileConfigurationProvider(agentName, configurationFile); application = new Application(); - application.handleConfigurationEvent(configurationProvider - .getConfiguration()); + application.handleConfigurationEvent(configurationProvider.getConfiguration()); } } application.start(); @@ -350,8 +345,7 @@ public void run() { }); } catch (Exception e) { - logger.error("A fatal error occurred while running. Exception follows.", - e); + logger.error("A fatal error occurred while running. Exception follows.", e); } } } \ No newline at end of file diff --git a/flume-ng-node/src/main/java/org/apache/flume/node/ConfigurationProvider.java b/flume-ng-node/src/main/java/org/apache/flume/node/ConfigurationProvider.java index 6a27898a7f..9528cb7cc3 100644 --- a/flume-ng-node/src/main/java/org/apache/flume/node/ConfigurationProvider.java +++ b/flume-ng-node/src/main/java/org/apache/flume/node/ConfigurationProvider.java @@ -19,11 +19,6 @@ package org.apache.flume.node; - public interface ConfigurationProvider { - - - public MaterializedConfiguration getConfiguration(); - - + MaterializedConfiguration getConfiguration(); } diff --git a/flume-ng-node/src/main/java/org/apache/flume/node/PollingPropertiesFileConfigurationProvider.java b/flume-ng-node/src/main/java/org/apache/flume/node/PollingPropertiesFileConfigurationProvider.java index 857c8a504a..91a09f0064 100644 --- a/flume-ng-node/src/main/java/org/apache/flume/node/PollingPropertiesFileConfigurationProvider.java +++ b/flume-ng-node/src/main/java/org/apache/flume/node/PollingPropertiesFileConfigurationProvider.java @@ -32,11 +32,12 @@ import com.google.common.eventbus.EventBus; import com.google.common.util.concurrent.ThreadFactoryBuilder; -public class PollingPropertiesFileConfigurationProvider extends - PropertiesFileConfigurationProvider implements LifecycleAware { +public class PollingPropertiesFileConfigurationProvider + extends PropertiesFileConfigurationProvider + implements LifecycleAware { - private static final Logger LOGGER = LoggerFactory - .getLogger(PollingPropertiesFileConfigurationProvider.class); + private static final Logger LOGGER = + LoggerFactory.getLogger(PollingPropertiesFileConfigurationProvider.class); private final EventBus eventBus; private final File file; @@ -83,8 +84,8 @@ public void stop() { LOGGER.info("Configuration provider stopping"); executorService.shutdown(); - try{ - while(!executorService.awaitTermination(500, TimeUnit.MILLISECONDS)) { + try { + while (!executorService.awaitTermination(500, TimeUnit.MILLISECONDS)) { LOGGER.debug("Waiting for file watcher to terminate"); } } catch (InterruptedException e) { diff --git a/flume-ng-node/src/main/java/org/apache/flume/node/SimpleMaterializedConfiguration.java b/flume-ng-node/src/main/java/org/apache/flume/node/SimpleMaterializedConfiguration.java index 536dcc4d51..a6523904eb 100644 --- a/flume-ng-node/src/main/java/org/apache/flume/node/SimpleMaterializedConfiguration.java +++ b/flume-ng-node/src/main/java/org/apache/flume/node/SimpleMaterializedConfiguration.java @@ -56,7 +56,7 @@ public void addSinkRunner(String name, SinkRunner sinkRunner) { } @Override - public void addChannel(String name, Channel channel){ + public void addChannel(String name, Channel channel) { channels.put(name, channel); } diff --git a/flume-ng-sdk/src/main/java/org/apache/flume/api/AbstractRpcClient.java b/flume-ng-sdk/src/main/java/org/apache/flume/api/AbstractRpcClient.java index 5cc292ae34..f20462bda2 100644 --- a/flume-ng-sdk/src/main/java/org/apache/flume/api/AbstractRpcClient.java +++ b/flume-ng-sdk/src/main/java/org/apache/flume/api/AbstractRpcClient.java @@ -35,7 +35,7 @@ public abstract class AbstractRpcClient implements RpcClient { RpcClientConfigurationConstants.DEFAULT_REQUEST_TIMEOUT_MILLIS; @Override - public int getBatchSize(){ + public int getBatchSize() { return batchSize; } @Override diff --git a/flume-ng-sdk/src/main/java/org/apache/flume/api/FailoverRpcClient.java b/flume-ng-sdk/src/main/java/org/apache/flume/api/FailoverRpcClient.java index db6905a288..9d82acbf2c 100644 --- a/flume-ng-sdk/src/main/java/org/apache/flume/api/FailoverRpcClient.java +++ b/flume-ng-sdk/src/main/java/org/apache/flume/api/FailoverRpcClient.java @@ -70,7 +70,7 @@ protected FailoverRpcClient() { //since shared data structures are created here. private synchronized void configureHosts(Properties properties) throws FlumeException { - if(isActive){ + if (isActive) { logger.error("This client was already configured, " + "cannot reconfigure."); throw new FlumeException("This client was already configured, " + @@ -79,7 +79,7 @@ private synchronized void configureHosts(Properties properties) hosts = HostInfo.getHostInfoList(properties); String tries = properties.getProperty( RpcClientConfigurationConstants.CONFIG_MAX_ATTEMPTS); - if (tries == null || tries.isEmpty()){ + if (tries == null || tries.isEmpty()) { maxTries = hosts.size(); } else { try { @@ -269,7 +269,7 @@ private RpcClient getNextClient() throws FlumeException { continue; } } - for(int count = 0; count <= lastCheckedhost; count++) { + for (int count = 0; count <= lastCheckedhost; count++) { HostInfo hostInfo = hosts.get(count); try { setDefaultProperties(hostInfo, props); diff --git a/flume-ng-sdk/src/main/java/org/apache/flume/api/HostInfo.java b/flume-ng-sdk/src/main/java/org/apache/flume/api/HostInfo.java index 8a81208724..53d99a260a 100644 --- a/flume-ng-sdk/src/main/java/org/apache/flume/api/HostInfo.java +++ b/flume-ng-sdk/src/main/java/org/apache/flume/api/HostInfo.java @@ -76,7 +76,7 @@ public static List getHostInfoList(Properties properties) { // Ignore that host if value is not there if (hostAndPortStr != null) { String[] hostAndPort = hostAndPortStr.split(":"); - if (hostAndPort.length != 2){ + if (hostAndPort.length != 2) { LOGGER.error("Invalid host address" + hostAndPortStr); throw new FlumeException("Invalid host address" + hostAndPortStr); } diff --git a/flume-ng-sdk/src/main/java/org/apache/flume/api/LoadBalancingRpcClient.java b/flume-ng-sdk/src/main/java/org/apache/flume/api/LoadBalancingRpcClient.java index e5fcc369fa..d3ccf74153 100644 --- a/flume-ng-sdk/src/main/java/org/apache/flume/api/LoadBalancingRpcClient.java +++ b/flume-ng-sdk/src/main/java/org/apache/flume/api/LoadBalancingRpcClient.java @@ -161,7 +161,7 @@ protected void configure(Properties properties) throws FlumeException { RpcClientConfigurationConstants.CONFIG_MAX_BACKOFF); long maxBackoff = 0; - if(maxBackoffStr != null) { + if (maxBackoffStr != null) { maxBackoff = Long.parseLong(maxBackoffStr); } @@ -240,12 +240,13 @@ private static class RoundRobinHostSelector implements HostSelector { private OrderSelector selector; - RoundRobinHostSelector(boolean backoff, long maxBackoff){ + RoundRobinHostSelector(boolean backoff, long maxBackoff) { selector = new RoundRobinOrderSelector(backoff); - if(maxBackoff != 0){ + if (maxBackoff != 0) { selector.setMaxTimeOut(maxBackoff); } } + @Override public synchronized Iterator createHostIterator() { return selector.createIterator(); @@ -256,7 +257,7 @@ public synchronized void setHosts(List hosts) { selector.setObjects(hosts); } - public synchronized void informFailure(HostInfo failedHost){ + public synchronized void informFailure(HostInfo failedHost) { selector.informFailure(failedHost); } } diff --git a/flume-ng-sdk/src/main/java/org/apache/flume/api/NettyAvroRpcClient.java b/flume-ng-sdk/src/main/java/org/apache/flume/api/NettyAvroRpcClient.java index 36616729ae..21a9553be8 100644 --- a/flume-ng-sdk/src/main/java/org/apache/flume/api/NettyAvroRpcClient.java +++ b/flume-ng-sdk/src/main/java/org/apache/flume/api/NettyAvroRpcClient.java @@ -79,8 +79,7 @@ * The connections are intended to be opened before clients are given access so * that the object cannot ever be in an inconsistent when exposed to users. */ -public class NettyAvroRpcClient extends AbstractRpcClient -implements RpcClient { +public class NettyAvroRpcClient extends AbstractRpcClient implements RpcClient { private ExecutorService callTimeoutPool; private final ReentrantLock stateLock = new ReentrantLock(); @@ -135,11 +134,11 @@ private void connect(long timeout, TimeUnit tu) throws FlumeException { try { ExecutorService bossExecutor = - Executors.newCachedThreadPool(new TransceiverThreadFactory( - "Avro " + NettyTransceiver.class.getSimpleName() + " Boss")); + Executors.newCachedThreadPool(new TransceiverThreadFactory( + "Avro " + NettyTransceiver.class.getSimpleName() + " Boss")); ExecutorService workerExecutor = - Executors.newCachedThreadPool(new TransceiverThreadFactory( - "Avro " + NettyTransceiver.class.getSimpleName() + " I/O Worker")); + Executors.newCachedThreadPool(new TransceiverThreadFactory( + "Avro " + NettyTransceiver.class.getSimpleName() + " I/O Worker")); if (enableDeflateCompression || enableSsl) { if (maxIoWorkers >= 1) { @@ -468,7 +467,7 @@ private static enum ConnState { } - /** + /** *

    * Configure the actual client using the properties. * properties should have at least 2 params: @@ -479,13 +478,13 @@ private static enum ConnState { * batch-size = batchSize * @param properties The properties to instantiate the client with. * @return - */ + */ @Override public synchronized void configure(Properties properties) throws FlumeException { stateLock.lock(); - try{ - if(connState == ConnState.READY || connState == ConnState.DEAD){ + try { + if (connState == ConnState.READY || connState == ConnState.DEAD) { throw new FlumeException("This client was already configured, " + "cannot reconfigure."); } @@ -529,12 +528,12 @@ public synchronized void configure(Properties properties) } String host = properties.getProperty( - RpcClientConfigurationConstants.CONFIG_HOSTS_PREFIX+hosts[0]); + RpcClientConfigurationConstants.CONFIG_HOSTS_PREFIX + hosts[0]); if (host == null || host.isEmpty()) { throw new FlumeException("Host not found: " + hosts[0]); } String[] hostAndPort = host.split(":"); - if (hostAndPort.length != 2){ + if (hostAndPort.length != 2) { throw new FlumeException("Invalid hostname: " + hosts[0]); } Integer port = null; @@ -583,10 +582,12 @@ public synchronized void configure(Properties properties) } } - String enableCompressionStr = properties.getProperty(RpcClientConfigurationConstants.CONFIG_COMPRESSION_TYPE); + String enableCompressionStr = + properties.getProperty(RpcClientConfigurationConstants.CONFIG_COMPRESSION_TYPE); if (enableCompressionStr != null && enableCompressionStr.equalsIgnoreCase("deflate")) { this.enableDeflateCompression = true; - String compressionLvlStr = properties.getProperty(RpcClientConfigurationConstants.CONFIG_COMPRESSION_LEVEL); + String compressionLvlStr = + properties.getProperty(RpcClientConfigurationConstants.CONFIG_COMPRESSION_LEVEL); compressionLevel = RpcClientConfigurationConstants.DEFAULT_COMPRESSION_LEVEL; if (compressionLvlStr != null) { try { @@ -608,7 +609,7 @@ public synchronized void configure(Properties properties) truststoreType = properties.getProperty( RpcClientConfigurationConstants.CONFIG_TRUSTSTORE_TYPE, "JKS"); String excludeProtocolsStr = properties.getProperty( - RpcClientConfigurationConstants.CONFIG_EXCLUDE_PROTOCOLS); + RpcClientConfigurationConstants.CONFIG_EXCLUDE_PROTOCOLS); if (excludeProtocolsStr == null) { excludeProtocols.add("SSLv3"); } else { @@ -618,14 +619,13 @@ public synchronized void configure(Properties properties) } } - String maxIoWorkersStr = properties.getProperty( - RpcClientConfigurationConstants.MAX_IO_WORKERS); + String maxIoWorkersStr = properties.getProperty(RpcClientConfigurationConstants.MAX_IO_WORKERS); if (!StringUtils.isEmpty(maxIoWorkersStr)) { try { maxIoWorkers = Integer.parseInt(maxIoWorkersStr); } catch (NumberFormatException ex) { - logger.warn ("Invalid maxIOWorkers:" + maxIoWorkersStr + " Using " + - "default maxIOWorkers."); + logger.warn("Invalid maxIOWorkers:" + maxIoWorkersStr + " Using " + + "default maxIOWorkers."); maxIoWorkers = -1; } } diff --git a/flume-ng-sdk/src/main/java/org/apache/flume/api/RpcClientConfigurationConstants.java b/flume-ng-sdk/src/main/java/org/apache/flume/api/RpcClientConfigurationConstants.java index 343e07b1d5..d83cf19429 100644 --- a/flume-ng-sdk/src/main/java/org/apache/flume/api/RpcClientConfigurationConstants.java +++ b/flume-ng-sdk/src/main/java/org/apache/flume/api/RpcClientConfigurationConstants.java @@ -66,18 +66,18 @@ public final class RpcClientConfigurationConstants { /** * Default batch size. */ - public final static Integer DEFAULT_BATCH_SIZE = 100; + public static final Integer DEFAULT_BATCH_SIZE = 100; /** * Default connection, handshake, and initial request timeout in milliseconds. */ - public final static long DEFAULT_CONNECT_TIMEOUT_MILLIS = + public static final long DEFAULT_CONNECT_TIMEOUT_MILLIS = TimeUnit.MILLISECONDS.convert(20, TimeUnit.SECONDS); /** * Default request timeout in milliseconds. */ - public final static long DEFAULT_REQUEST_TIMEOUT_MILLIS = + public static final long DEFAULT_REQUEST_TIMEOUT_MILLIS = TimeUnit.MILLISECONDS.convert(20, TimeUnit.SECONDS); /** diff --git a/flume-ng-sdk/src/main/java/org/apache/flume/api/RpcClientFactory.java b/flume-ng-sdk/src/main/java/org/apache/flume/api/RpcClientFactory.java index 11bc94c495..5cb3332aef 100644 --- a/flume-ng-sdk/src/main/java/org/apache/flume/api/RpcClientFactory.java +++ b/flume-ng-sdk/src/main/java/org/apache/flume/api/RpcClientFactory.java @@ -18,6 +18,8 @@ */ package org.apache.flume.api; +import org.apache.flume.FlumeException; + import java.io.File; import java.io.FileNotFoundException; import java.io.FileReader; @@ -25,7 +27,6 @@ import java.io.Reader; import java.util.Locale; import java.util.Properties; -import org.apache.flume.FlumeException; /** * Factory class to construct Flume {@link RPCClient} implementations. @@ -63,12 +64,12 @@ public static RpcClient getInstance(Properties properties) try { String clientClassType = type; ClientType clientType = null; - try{ + try { clientType = ClientType.valueOf(type.toUpperCase(Locale.ENGLISH)); - } catch (IllegalArgumentException e){ + } catch (IllegalArgumentException e) { clientType = ClientType.OTHER; } - if (!clientType.equals(ClientType.OTHER)){ + if (!clientType.equals(ClientType.OTHER)) { clientClassType = clientType.getClientClassName(); } clazz = @@ -181,8 +182,7 @@ public static RpcClient getDefaultInstance(String hostname, Integer port, * @return an {@linkplain RpcClient} which uses thrift configured with the * given parameters. */ - public static RpcClient getThriftInstance(String hostname, Integer port, - Integer batchSize) { + public static RpcClient getThriftInstance(String hostname, Integer port, Integer batchSize) { if (hostname == null) { throw new NullPointerException("hostname must not be null"); } @@ -196,7 +196,7 @@ public static RpcClient getThriftInstance(String hostname, Integer port, Properties props = new Properties(); props.setProperty(RpcClientConfigurationConstants.CONFIG_HOSTS, "h1"); props.setProperty(RpcClientConfigurationConstants.CONFIG_HOSTS_PREFIX + "h1", - hostname + ":" + port.intValue()); + hostname + ":" + port.intValue()); props.setProperty(RpcClientConfigurationConstants.CONFIG_BATCH_SIZE, batchSize.toString()); ThriftRpcClient client = new ThriftRpcClient(); client.configure(props); @@ -227,7 +227,7 @@ public static RpcClient getThriftInstance(String hostname, Integer port) { */ public static RpcClient getThriftInstance(Properties props) { props.setProperty(RpcClientConfigurationConstants.CONFIG_CLIENT_TYPE, - ClientType.THRIFT.clientClassName); + ClientType.THRIFT.clientClassName); return getInstance(props); } diff --git a/flume-ng-sdk/src/main/java/org/apache/flume/api/ThriftRpcClient.java b/flume-ng-sdk/src/main/java/org/apache/flume/api/ThriftRpcClient.java index 857948f182..1d21d5fd6f 100644 --- a/flume-ng-sdk/src/main/java/org/apache/flume/api/ThriftRpcClient.java +++ b/flume-ng-sdk/src/main/java/org/apache/flume/api/ThriftRpcClient.java @@ -32,16 +32,15 @@ import org.slf4j.Logger; import org.slf4j.LoggerFactory; -import javax.net.ssl.KeyManagerFactory; -import javax.net.ssl.TrustManagerFactory; import javax.net.ssl.SSLContext; import javax.net.ssl.SSLSocket; import javax.net.ssl.SSLSocketFactory; - +import javax.net.ssl.TrustManagerFactory; import java.io.FileInputStream; import java.nio.ByteBuffer; import java.security.KeyStore; import java.util.ArrayList; +import java.util.Arrays; import java.util.HashSet; import java.util.Iterator; import java.util.LinkedList; @@ -50,7 +49,6 @@ import java.util.Queue; import java.util.Random; import java.util.Set; -import java.util.Arrays; import java.util.concurrent.Callable; import java.util.concurrent.ExecutionException; import java.util.concurrent.ExecutorService; @@ -65,8 +63,7 @@ import java.util.concurrent.locks.ReentrantLock; public class ThriftRpcClient extends AbstractRpcClient { - private static final Logger LOGGER = - LoggerFactory.getLogger(ThriftRpcClient.class); + private static final Logger LOGGER = LoggerFactory.getLogger(ThriftRpcClient.class); /** * Config param for the thrift protocol to use. @@ -104,8 +101,7 @@ public ThriftRpcClient() { @Override public Thread newThread(Runnable r) { Thread t = new Thread(r); - t.setName("Flume Thrift RPC thread - " + String.valueOf( - threadCounter.incrementAndGet())); + t.setName("Flume Thrift RPC thread - " + String.valueOf(threadCounter.incrementAndGet())); return t; } }); @@ -126,11 +122,11 @@ public void append(Event event) throws EventDeliveryException { try { if (!isActive()) { throw new EventDeliveryException("Client was closed due to error. " + - "Please create a new client"); + "Please create a new client"); } client = connectionManager.checkout(); final ThriftFlumeEvent thriftEvent = new ThriftFlumeEvent(event - .getHeaders(), ByteBuffer.wrap(event.getBody())); + .getHeaders(), ByteBuffer.wrap(event.getBody())); doAppend(client, thriftEvent).get(requestTimeout, TimeUnit.MILLISECONDS); } catch (Throwable e) { if (e instanceof ExecutionException) { @@ -169,22 +165,22 @@ public void appendBatch(List events) throws EventDeliveryException { try { if (!isActive()) { throw new EventDeliveryException("Client was closed " + - "due to error or is not yet configured."); + "due to error or is not yet configured."); } client = connectionManager.checkout(); final List thriftFlumeEvents = new ArrayList - (); + (); Iterator eventsIter = events.iterator(); while (eventsIter.hasNext()) { thriftFlumeEvents.clear(); for (int i = 0; i < batchSize && eventsIter.hasNext(); i++) { Event event = eventsIter.next(); thriftFlumeEvents.add(new ThriftFlumeEvent(event.getHeaders(), - ByteBuffer.wrap(event.getBody()))); + ByteBuffer.wrap(event.getBody()))); } if (!thriftFlumeEvents.isEmpty()) { doAppendBatch(client, thriftFlumeEvents).get(requestTimeout, - TimeUnit.MILLISECONDS); + TimeUnit.MILLISECONDS); } } } catch (Throwable e) { @@ -216,7 +212,7 @@ public void appendBatch(List events) throws EventDeliveryException { } private Future doAppend(final ClientWrapper client, - final ThriftFlumeEvent e) throws Exception { + final ThriftFlumeEvent e) throws Exception { return callTimeoutPool.submit(new Callable() { @Override @@ -224,7 +220,7 @@ public Void call() throws Exception { Status status = client.client.append(e); if (status != Status.OK) { throw new EventDeliveryException("Failed to deliver events. Server " + - "returned status : " + status.name()); + "returned status : " + status.name()); } return null; } @@ -232,7 +228,7 @@ public Void call() throws Exception { } private Future doAppendBatch(final ClientWrapper client, - final List e) throws Exception { + final List e) throws Exception { return callTimeoutPool.submit(new Callable() { @Override @@ -240,7 +236,7 @@ public Void call() throws Exception { Status status = client.client.appendBatch(e); if (status != Status.OK) { throw new EventDeliveryException("Failed to deliver events. Server " + - "returned status : " + status.name()); + "returned status : " + status.name()); } return null; } @@ -265,11 +261,11 @@ public void close() throws FlumeException { connState = State.DEAD; connectionManager.closeAll(); callTimeoutPool.shutdown(); - if(!callTimeoutPool.awaitTermination(5, TimeUnit.SECONDS)) { + if (!callTimeoutPool.awaitTermination(5, TimeUnit.SECONDS)) { callTimeoutPool.shutdownNow(); } } catch (Throwable ex) { - if(ex instanceof Error) { + if (ex instanceof Error) { throw (Error) ex; } else if (ex instanceof RuntimeException) { throw (RuntimeException) ex; @@ -284,7 +280,7 @@ public void close() throws FlumeException { protected void configure(Properties properties) throws FlumeException { if (isActive()) { throw new FlumeException("Attempting to re-configured an already " + - "configured client!"); + "configured client!"); } stateLock.lock(); try { @@ -304,40 +300,40 @@ protected void configure(Properties properties) throws FlumeException { protocol = COMPACT_PROTOCOL; } batchSize = Integer.parseInt(properties.getProperty( - RpcClientConfigurationConstants.CONFIG_BATCH_SIZE, - RpcClientConfigurationConstants.DEFAULT_BATCH_SIZE.toString())); + RpcClientConfigurationConstants.CONFIG_BATCH_SIZE, + RpcClientConfigurationConstants.DEFAULT_BATCH_SIZE.toString())); requestTimeout = Long.parseLong(properties.getProperty( - RpcClientConfigurationConstants.CONFIG_REQUEST_TIMEOUT, - String.valueOf( - RpcClientConfigurationConstants.DEFAULT_REQUEST_TIMEOUT_MILLIS))); + RpcClientConfigurationConstants.CONFIG_REQUEST_TIMEOUT, + String.valueOf( + RpcClientConfigurationConstants.DEFAULT_REQUEST_TIMEOUT_MILLIS))); if (requestTimeout < 1000) { LOGGER.warn("Request timeout specified less than 1s. " + - "Using default value instead."); + "Using default value instead."); requestTimeout = - RpcClientConfigurationConstants.DEFAULT_REQUEST_TIMEOUT_MILLIS; + RpcClientConfigurationConstants.DEFAULT_REQUEST_TIMEOUT_MILLIS; } int connectionPoolSize = Integer.parseInt(properties.getProperty( - RpcClientConfigurationConstants.CONFIG_CONNECTION_POOL_SIZE, - String.valueOf(RpcClientConfigurationConstants - .DEFAULT_CONNECTION_POOL_SIZE))); - if(connectionPoolSize < 1) { + RpcClientConfigurationConstants.CONFIG_CONNECTION_POOL_SIZE, + String.valueOf(RpcClientConfigurationConstants + .DEFAULT_CONNECTION_POOL_SIZE))); + if (connectionPoolSize < 1) { LOGGER.warn("Connection Pool Size specified is less than 1. " + - "Using default value instead."); + "Using default value instead."); connectionPoolSize = RpcClientConfigurationConstants - .DEFAULT_CONNECTION_POOL_SIZE; + .DEFAULT_CONNECTION_POOL_SIZE; } enableSsl = Boolean.parseBoolean(properties.getProperty( - RpcClientConfigurationConstants.CONFIG_SSL)); - if(enableSsl) { + RpcClientConfigurationConstants.CONFIG_SSL)); + if (enableSsl) { truststore = properties.getProperty( - RpcClientConfigurationConstants.CONFIG_TRUSTSTORE); + RpcClientConfigurationConstants.CONFIG_TRUSTSTORE); truststorePassword = properties.getProperty( - RpcClientConfigurationConstants.CONFIG_TRUSTSTORE_PASSWORD); + RpcClientConfigurationConstants.CONFIG_TRUSTSTORE_PASSWORD); truststoreType = properties.getProperty( - RpcClientConfigurationConstants.CONFIG_TRUSTSTORE_TYPE, "JKS"); + RpcClientConfigurationConstants.CONFIG_TRUSTSTORE_TYPE, "JKS"); String excludeProtocolsStr = properties.getProperty( - RpcClientConfigurationConstants.CONFIG_EXCLUDE_PROTOCOLS); + RpcClientConfigurationConstants.CONFIG_EXCLUDE_PROTOCOLS); if (excludeProtocolsStr == null) { excludeProtocols.add("SSLv3"); } else { @@ -353,7 +349,7 @@ protected void configure(Properties properties) throws FlumeException { } catch (Throwable ex) { //Failed to configure, kill the client. connState = State.DEAD; - if(ex instanceof Error) { + if (ex instanceof Error) { throw (Error) ex; } else if (ex instanceof RuntimeException) { throw (RuntimeException) ex; @@ -381,40 +377,37 @@ private class ClientWrapper { public final TTransport transport; private final int hashCode; - public ClientWrapper() throws Exception{ + public ClientWrapper() throws Exception { TSocket tsocket; - if(enableSsl) { + if (enableSsl) { // JDK6's factory doesn't appear to pass the protocol onto the Socket // properly so we have to do some magic to make sure that happens. // Not an issue in JDK7 Lifted from thrift-0.9.1 to make the SSLContext SSLContext sslContext = createSSLContext(truststore, truststorePassword, - truststoreType); + truststoreType); // Create the factory from it SSLSocketFactory sslSockFactory = sslContext.getSocketFactory(); // Create the TSocket from that tsocket = createSSLSocket( - sslSockFactory, hostname, port, 120000, excludeProtocols); + sslSockFactory, hostname, port, 120000, excludeProtocols); } else { tsocket = new TSocket(hostname, port); } - - transport = getTransport(tsocket); + transport = getTransport(tsocket); // The transport is already open for SSL as part of TSSLTransportFactory.getClientSocket - if(!transport.isOpen()) { + if (!transport.isOpen()) { transport.open(); } if (protocol.equals(BINARY_PROTOCOL)) { LOGGER.info("Using TBinaryProtocol"); - client = new ThriftSourceProtocol.Client(new TBinaryProtocol - (transport)); + client = new ThriftSourceProtocol.Client(new TBinaryProtocol(transport)); } else { LOGGER.info("Using TCompactProtocol"); - client = new ThriftSourceProtocol.Client(new TCompactProtocol - (transport)); + client = new ThriftSourceProtocol.Client(new TCompactProtocol(transport)); } // Not a great hash code, but since this class is immutable and there // is at most one instance of the components of this class, @@ -423,12 +416,12 @@ public ClientWrapper() throws Exception{ } public boolean equals(Object o) { - if(o == null) { + if (o == null) { return false; } // Since there is only one wrapper with any given client, // direct comparison is good enough. - if(this == o) { + if (this == o) { return true; } return false; @@ -507,10 +500,8 @@ public void closeAll() { c.transport.close(); currentPoolSize--; } - /* - * Be cruel and close even the checked out clients. The threads writing - * using these will now get an exception. - */ + // Be cruel and close even the checked out clients. The threads writing + // using these will now get an exception. for (ClientWrapper c : checkedOutClients) { c.transport.close(); currentPoolSize--; @@ -522,12 +513,14 @@ public void closeAll() { } /** - * Lifted from ACCUMULO-3318 - Lifted from TSSLTransportFactory in Thrift-0.9.1. The method to create a client socket with an SSLContextFactory object is not visibile to us. Have to use - * SslConnectionParams instead of TSSLTransportParameters because no getters exist on TSSLTransportParameters. - * + * Lifted from ACCUMULO-3318 - Lifted from TSSLTransportFactory in Thrift-0.9.1. + * The method to create a client socket with an SSLContextFactory object is not visible to us. + * Have to use * SslConnectionParams instead of TSSLTransportParameters because no getters exist + * on TSSLTransportParameters. */ private static SSLContext createSSLContext(String truststore, - String truststorePassword, String truststoreType) throws FlumeException { + String truststorePassword, + String truststoreType) throws FlumeException { SSLContext ctx; try { ctx = SSLContext.getInstance("TLS"); @@ -550,7 +543,8 @@ private static SSLContext createSSLContext(String truststore, } private static TSocket createSSLSocket(SSLSocketFactory factory, String host, - int port, int timeout, List excludeProtocols) throws FlumeException { + int port, int timeout, List excludeProtocols) + throws FlumeException { try { SSLSocket socket = (SSLSocket) factory.createSocket(host, port); socket.setSoTimeout(timeout); diff --git a/flume-ng-sdk/src/main/java/org/apache/flume/event/EventBuilder.java b/flume-ng-sdk/src/main/java/org/apache/flume/event/EventBuilder.java index a5e01fc085..c19925a9f5 100644 --- a/flume-ng-sdk/src/main/java/org/apache/flume/event/EventBuilder.java +++ b/flume-ng-sdk/src/main/java/org/apache/flume/event/EventBuilder.java @@ -37,7 +37,7 @@ public class EventBuilder { public static Event withBody(byte[] body, Map headers) { Event event = new SimpleEvent(); - if(body == null) { + if (body == null) { body = new byte[0]; } event.setBody(body); diff --git a/flume-ng-sdk/src/main/java/org/apache/flume/event/JSONEvent.java b/flume-ng-sdk/src/main/java/org/apache/flume/event/JSONEvent.java index 0ec1678a3a..9ee90aec6c 100644 --- a/flume-ng-sdk/src/main/java/org/apache/flume/event/JSONEvent.java +++ b/flume-ng-sdk/src/main/java/org/apache/flume/event/JSONEvent.java @@ -26,7 +26,7 @@ /** * */ -public class JSONEvent implements Event{ +public class JSONEvent implements Event { private Map headers; private String body; private transient String charset = "UTF-8"; @@ -43,7 +43,7 @@ public void setHeaders(Map headers) { @Override public byte[] getBody() { - if(body != null) { + if (body != null) { try { return body.getBytes(charset); } catch (UnsupportedEncodingException ex) { @@ -57,7 +57,7 @@ public byte[] getBody() { @Override public void setBody(byte[] body) { - if(body != null) { + if (body != null) { this.body = new String(body); } else { this.body = ""; diff --git a/flume-ng-sdk/src/main/java/org/apache/flume/event/SimpleEvent.java b/flume-ng-sdk/src/main/java/org/apache/flume/event/SimpleEvent.java index a7ac36fbe4..61f848d7e9 100644 --- a/flume-ng-sdk/src/main/java/org/apache/flume/event/SimpleEvent.java +++ b/flume-ng-sdk/src/main/java/org/apache/flume/event/SimpleEvent.java @@ -51,7 +51,7 @@ public byte[] getBody() { @Override public void setBody(byte[] body) { - if(body == null){ + if (body == null) { body = new byte[0]; } this.body = body; diff --git a/flume-ng-sdk/src/main/java/org/apache/flume/util/OrderSelector.java b/flume-ng-sdk/src/main/java/org/apache/flume/util/OrderSelector.java index fd9e81fddd..806a553013 100644 --- a/flume-ng-sdk/src/main/java/org/apache/flume/util/OrderSelector.java +++ b/flume-ng-sdk/src/main/java/org/apache/flume/util/OrderSelector.java @@ -40,9 +40,8 @@ public abstract class OrderSelector { private static final int EXP_BACKOFF_COUNTER_LIMIT = 16; - private static final long CONSIDER_SEQUENTIAL_RANGE = TimeUnit.HOURS - .toMillis(1); - private static final long MAX_TIMEOUT = 30000l; + private static final long CONSIDER_SEQUENTIAL_RANGE = TimeUnit.HOURS.toMillis(1); + private static final long MAX_TIMEOUT = 30000L; private final Map stateMap = new LinkedHashMap(); private long maxTimeout = MAX_TIMEOUT; diff --git a/flume-ng-sinks/flume-dataset-sink/src/main/java/org/apache/flume/sink/kite/DatasetSink.java b/flume-ng-sinks/flume-dataset-sink/src/main/java/org/apache/flume/sink/kite/DatasetSink.java index a9f42b8992..e3b57c323f 100644 --- a/flume-ng-sinks/flume-dataset-sink/src/main/java/org/apache/flume/sink/kite/DatasetSink.java +++ b/flume-ng-sinks/flume-dataset-sink/src/main/java/org/apache/flume/sink/kite/DatasetSink.java @@ -104,12 +104,12 @@ public class DatasetSink extends AbstractSink implements Configurable { /** * The last time the writer rolled. */ - private long lastRolledMillis = 0l; + private long lastRolledMillis = 0L; /** * The raw number of bytes parsed. */ - private long bytesParsed = 0l; + private long bytesParsed = 0L; /** * A class for parsing Kite entities from Flume Events. @@ -225,7 +225,7 @@ public synchronized void start() { */ @VisibleForTesting void roll() { - this.lastRolledMillis = 0l; + this.lastRolledMillis = 0L; } @VisibleForTesting @@ -434,7 +434,7 @@ public Dataset run() { // Reset the last rolled time and the metrics this.lastRolledMillis = System.currentTimeMillis(); - this.bytesParsed = 0l; + this.bytesParsed = 0L; } catch (DatasetNotFoundException ex) { throw new EventDeliveryException("Dataset " + datasetUri + " not found." + " The dataset must be created before Flume can write to it.", ex); @@ -558,7 +558,7 @@ private void rollbackTransaction() { try { // If the transaction wasn't committed before we got the exception, we // need to rollback. - transaction.rollback(); + transaction.rollback(); } catch (RuntimeException ex) { LOG.error("Transaction rollback failed: " + ex.getLocalizedMessage()); LOG.debug("Exception follows.", ex); @@ -567,7 +567,7 @@ private void rollbackTransaction() { this.transaction = null; } } -} + } /** * Get the name of the dataset from the URI diff --git a/flume-ng-sinks/flume-dataset-sink/src/main/java/org/apache/flume/sink/kite/NonRecoverableEventException.java b/flume-ng-sinks/flume-dataset-sink/src/main/java/org/apache/flume/sink/kite/NonRecoverableEventException.java index 8f6c0aeeae..43734293db 100644 --- a/flume-ng-sinks/flume-dataset-sink/src/main/java/org/apache/flume/sink/kite/NonRecoverableEventException.java +++ b/flume-ng-sinks/flume-dataset-sink/src/main/java/org/apache/flume/sink/kite/NonRecoverableEventException.java @@ -50,5 +50,4 @@ public NonRecoverableEventException(Throwable t) { super(t); } - } diff --git a/flume-ng-sinks/flume-dataset-sink/src/main/java/org/apache/flume/sink/kite/parser/EntityParserFactory.java b/flume-ng-sinks/flume-dataset-sink/src/main/java/org/apache/flume/sink/kite/parser/EntityParserFactory.java index cfb734943e..3720ff3635 100644 --- a/flume-ng-sinks/flume-dataset-sink/src/main/java/org/apache/flume/sink/kite/parser/EntityParserFactory.java +++ b/flume-ng-sinks/flume-dataset-sink/src/main/java/org/apache/flume/sink/kite/parser/EntityParserFactory.java @@ -25,7 +25,6 @@ import static org.apache.flume.sink.kite.DatasetSinkConstants.*; - public class EntityParserFactory { public EntityParser newParser(Schema datasetSchema, Context config) { diff --git a/flume-ng-sinks/flume-dataset-sink/src/main/java/org/apache/flume/sink/kite/policy/FailurePolicyFactory.java b/flume-ng-sinks/flume-dataset-sink/src/main/java/org/apache/flume/sink/kite/policy/FailurePolicyFactory.java index a8b2008126..d3b1fe83a5 100644 --- a/flume-ng-sinks/flume-dataset-sink/src/main/java/org/apache/flume/sink/kite/policy/FailurePolicyFactory.java +++ b/flume-ng-sinks/flume-dataset-sink/src/main/java/org/apache/flume/sink/kite/policy/FailurePolicyFactory.java @@ -23,7 +23,6 @@ import static org.apache.flume.sink.kite.DatasetSinkConstants.*; - public class FailurePolicyFactory { public FailurePolicy newPolicy(Context config) { diff --git a/flume-ng-sinks/flume-hdfs-sink/src/main/java/org/apache/flume/sink/hdfs/AbstractHDFSWriter.java b/flume-ng-sinks/flume-hdfs-sink/src/main/java/org/apache/flume/sink/hdfs/AbstractHDFSWriter.java index e367e1271c..2fe309fb77 100644 --- a/flume-ng-sinks/flume-hdfs-sink/src/main/java/org/apache/flume/sink/hdfs/AbstractHDFSWriter.java +++ b/flume-ng-sinks/flume-hdfs-sink/src/main/java/org/apache/flume/sink/hdfs/AbstractHDFSWriter.java @@ -50,7 +50,7 @@ public abstract class AbstractHDFSWriter implements HDFSWriter { private Integer numberOfCloseRetries = null; private long timeBetweenCloseRetries = Long.MAX_VALUE; - final static Object [] NO_ARGS = new Object []{}; + static final Object[] NO_ARGS = new Object[]{}; @Override public void configure(Context context) { @@ -63,11 +63,12 @@ public void configure(Context context) { if (numberOfCloseRetries > 1) { try { - timeBetweenCloseRetries = context.getLong("hdfs.callTimeout", 10000l); + timeBetweenCloseRetries = context.getLong("hdfs.callTimeout", 10000L); } catch (NumberFormatException e) { - logger.warn("hdfs.callTimeout can not be parsed to a long: " + context.getLong("hdfs.callTimeout")); + logger.warn("hdfs.callTimeout can not be parsed to a long: " + + context.getLong("hdfs.callTimeout")); } - timeBetweenCloseRetries = Math.max(timeBetweenCloseRetries/numberOfCloseRetries, 1000); + timeBetweenCloseRetries = Math.max(timeBetweenCloseRetries / numberOfCloseRetries, 1000); } } @@ -232,7 +233,7 @@ private Method reflectGetDefaultReplication(FileSystem fileSystem) { private Method reflectHflushOrSync(FSDataOutputStream os) { Method m = null; - if(os != null) { + if (os != null) { Class fsDataOutputStreamClass = os.getClass(); try { m = fsDataOutputStreamClass.getMethod("hflush"); @@ -242,7 +243,7 @@ private Method reflectHflushOrSync(FSDataOutputStream os) { m = fsDataOutputStreamClass.getMethod("sync"); } catch (Exception ex1) { String msg = "Neither hflush not sync were found. That seems to be " + - "a problem!"; + "a problem!"; logger.error(msg); throw new FlumeException(msg, ex1); } @@ -266,7 +267,7 @@ protected void hflushOrSync(FSDataOutputStream os) throws IOException { String msg = "Error while trying to hflushOrSync!"; logger.error(msg); Throwable cause = e.getCause(); - if(cause != null && cause instanceof IOException) { + if (cause != null && cause instanceof IOException) { throw (IOException)cause; } throw new FlumeException(msg, e); diff --git a/flume-ng-sinks/flume-hdfs-sink/src/main/java/org/apache/flume/sink/hdfs/BucketClosedException.java b/flume-ng-sinks/flume-hdfs-sink/src/main/java/org/apache/flume/sink/hdfs/BucketClosedException.java index 1aca58f630..1d8a9e4c5b 100644 --- a/flume-ng-sinks/flume-hdfs-sink/src/main/java/org/apache/flume/sink/hdfs/BucketClosedException.java +++ b/flume-ng-sinks/flume-hdfs-sink/src/main/java/org/apache/flume/sink/hdfs/BucketClosedException.java @@ -20,7 +20,7 @@ import org.apache.flume.FlumeException; -public class BucketClosedException extends FlumeException{ +public class BucketClosedException extends FlumeException { private static final long serialVersionUID = -4216667125119540357L; diff --git a/flume-ng-sinks/flume-hdfs-sink/src/main/java/org/apache/flume/sink/hdfs/BucketWriter.java b/flume-ng-sinks/flume-hdfs-sink/src/main/java/org/apache/flume/sink/hdfs/BucketWriter.java index 6b97de6e12..b09641020d 100644 --- a/flume-ng-sinks/flume-hdfs-sink/src/main/java/org/apache/flume/sink/hdfs/BucketWriter.java +++ b/flume-ng-sinks/flume-hdfs-sink/src/main/java/org/apache/flume/sink/hdfs/BucketWriter.java @@ -18,22 +18,8 @@ package org.apache.flume.sink.hdfs; -import java.io.IOException; -import java.lang.reflect.Method; -import java.security.PrivilegedExceptionAction; -import java.util.concurrent.Callable; -import java.util.concurrent.CancellationException; -import java.util.concurrent.ExecutionException; -import java.util.concurrent.ExecutorService; -import java.util.concurrent.Future; -import java.util.concurrent.ScheduledExecutorService; -import java.util.concurrent.ScheduledFuture; -import java.util.concurrent.TimeUnit; -import java.util.concurrent.TimeoutException; -import java.util.concurrent.atomic.AtomicInteger; -import java.util.concurrent.atomic.AtomicLong; - import com.google.common.annotations.VisibleForTesting; +import com.google.common.base.Throwables; import org.apache.flume.Clock; import org.apache.flume.Context; import org.apache.flume.Event; @@ -49,7 +35,20 @@ import org.slf4j.Logger; import org.slf4j.LoggerFactory; -import com.google.common.base.Throwables; +import java.io.IOException; +import java.lang.reflect.Method; +import java.security.PrivilegedExceptionAction; +import java.util.concurrent.Callable; +import java.util.concurrent.CancellationException; +import java.util.concurrent.ExecutionException; +import java.util.concurrent.ExecutorService; +import java.util.concurrent.Future; +import java.util.concurrent.ScheduledExecutorService; +import java.util.concurrent.ScheduledFuture; +import java.util.concurrent.TimeUnit; +import java.util.concurrent.TimeoutException; +import java.util.concurrent.atomic.AtomicInteger; +import java.util.concurrent.atomic.AtomicLong; /** * Internal API intended for HDFSSink use. @@ -117,14 +116,14 @@ class BucketWriter { AtomicInteger renameTries = new AtomicInteger(0); BucketWriter(long rollInterval, long rollSize, long rollCount, long batchSize, - Context context, String filePath, String fileName, String inUsePrefix, - String inUseSuffix, String fileSuffix, CompressionCodec codeC, - CompressionType compType, HDFSWriter writer, - ScheduledExecutorService timedRollerPool, PrivilegedExecutor proxyUser, - SinkCounter sinkCounter, int idleTimeout, WriterCallback onCloseCallback, - String onCloseCallbackPath, long callTimeout, - ExecutorService callTimeoutPool, long retryInterval, - int maxCloseTries) { + Context context, String filePath, String fileName, String inUsePrefix, + String inUseSuffix, String fileSuffix, CompressionCodec codeC, + CompressionType compType, HDFSWriter writer, + ScheduledExecutorService timedRollerPool, PrivilegedExecutor proxyUser, + SinkCounter sinkCounter, int idleTimeout, WriterCallback onCloseCallback, + String onCloseCallbackPath, long callTimeout, + ExecutorService callTimeoutPool, long retryInterval, + int maxCloseTries) { this.rollInterval = rollInterval; this.rollSize = rollSize; this.rollCount = rollCount; @@ -181,17 +180,15 @@ private Method getRefIsClosed() { Path.class); } catch (Exception e) { LOG.warn("isFileClosed is not available in the " + - "version of HDFS being used. Flume will not " + - "attempt to close files if the close fails on " + - "the first attempt",e); + "version of HDFS being used. Flume will not " + + "attempt to close files if the close fails on " + + "the first attempt",e); return null; } } - private Boolean isFileClosed(FileSystem fs, - Path tmpFilePath) throws Exception { - return (Boolean)(isClosedMethod.invoke(fs, - tmpFilePath)); + private Boolean isFileClosed(FileSystem fs, Path tmpFilePath) throws Exception { + return (Boolean)(isClosedMethod.invoke(fs, tmpFilePath)); } /** @@ -239,17 +236,15 @@ public Void call() throws Exception { // Need to get reference to FS using above config before underlying // writer does in order to avoid shutdown hook & // IllegalStateExceptions - if(!mockFsInjected) { - fileSystem = new Path(bucketPath).getFileSystem( - config); + if (!mockFsInjected) { + fileSystem = new Path(bucketPath).getFileSystem(config); } writer.open(bucketPath); } else { // need to get reference to FS before writer does to // avoid shutdown hook - if(!mockFsInjected) { - fileSystem = new Path(bucketPath).getFileSystem( - config); + if (!mockFsInjected) { + fileSystem = new Path(bucketPath).getFileSystem(config); } writer.open(bucketPath, codeC, compType); } @@ -278,7 +273,7 @@ public Void call() throws Exception { try { // Roll the file and remove reference from sfWriters map. close(true); - } catch(Throwable t) { + } catch (Throwable t) { LOG.error("Unexpected error", t); } return null; @@ -327,7 +322,7 @@ private Callable createScheduledRenameCallable() { public Void call() throws Exception { if (renameTries >= maxRenameTries) { LOG.warn("Unsuccessfully attempted to rename " + path + " " + - maxRenameTries + " times. File may still be open."); + maxRenameTries + " times. File may still be open."); return null; } renameTries++; @@ -335,16 +330,15 @@ public Void call() throws Exception { renameBucket(path, finalPath, fs); } catch (Exception e) { LOG.warn("Renaming file: " + path + " failed. Will " + - "retry again in " + retryInterval + " seconds.", e); - timedRollerPool.schedule(this, retryInterval, - TimeUnit.SECONDS); + "retry again in " + retryInterval + " seconds.", e); + timedRollerPool.schedule(this, retryInterval, TimeUnit.SECONDS); return null; } return null; } }; - } + /** * Close the file handle and rename the temp file to the permanent filename. * Safe to call multiple times. Logs HDFSWriter.close() exceptions. @@ -352,7 +346,7 @@ public Void call() throws Exception { * @throws InterruptedException */ public synchronized void close(boolean callCloseCallback) - throws IOException, InterruptedException { + throws IOException, InterruptedException { checkAndThrowInterruptedException(); try { flush(); @@ -367,9 +361,8 @@ public synchronized void close(boolean callCloseCallback) callWithTimeout(closeCallRunner); sinkCounter.incrementConnectionClosedCount(); } catch (IOException e) { - LOG.warn( - "failed to close() HDFSWriter for file (" + bucketPath + - "). Exception follows.", e); + LOG.warn("failed to close() HDFSWriter for file (" + bucketPath + + "). Exception follows.", e); sinkCounter.incrementConnectionFailedCount(); failedToClose = true; } @@ -393,15 +386,12 @@ public synchronized void close(boolean callCloseCallback) // could block or throw IOException try { renameBucket(bucketPath, targetPath, fileSystem); - } catch(Exception e) { - LOG.warn( - "failed to rename() file (" + bucketPath + - "). Exception follows.", e); + } catch (Exception e) { + LOG.warn("failed to rename() file (" + bucketPath + + "). Exception follows.", e); sinkCounter.incrementConnectionFailedCount(); - final Callable scheduledRename = - createScheduledRenameCallable(); - timedRollerPool.schedule(scheduledRename, retryInterval, - TimeUnit.SECONDS); + final Callable scheduledRename = createScheduledRenameCallable(); + timedRollerPool.schedule(scheduledRename, retryInterval, TimeUnit.SECONDS); } } if (callCloseCallback) { @@ -420,14 +410,14 @@ public synchronized void flush() throws IOException, InterruptedException { if (!isBatchComplete()) { doFlush(); - if(idleTimeout > 0) { + if (idleTimeout > 0) { // if the future exists and couldn't be cancelled, that would mean it has already run // or been cancelled - if(idleFuture == null || idleFuture.cancel(false)) { + if (idleFuture == null || idleFuture.cancel(false)) { Callable idleAction = new Callable() { public Void call() throws Exception { LOG.info("Closing idle bucketWriter {} at {}", bucketPath, - System.currentTimeMillis()); + System.currentTimeMillis()); if (isOpen) { close(true); } @@ -443,10 +433,10 @@ public Void call() throws Exception { private void runCloseAction() { try { - if(onCloseCallback != null) { + if (onCloseCallback != null) { onCloseCallback.run(onCloseCallbackPath); } - } catch(Throwable t) { + } catch (Throwable t) { LOG.error("Unexpected error", t); } } @@ -483,19 +473,19 @@ public synchronized void append(final Event event) checkAndThrowInterruptedException(); // If idleFuture is not null, cancel it before we move forward to avoid a // close call in the middle of the append. - if(idleFuture != null) { + if (idleFuture != null) { idleFuture.cancel(false); // There is still a small race condition - if the idleFuture is already // running, interrupting it can cause HDFS close operation to throw - // so we cannot interrupt it while running. If the future could not be // cancelled, it is already running - wait for it to finish before // attempting to write. - if(!idleFuture.isDone()) { + if (!idleFuture.isDone()) { try { idleFuture.get(callTimeout, TimeUnit.MILLISECONDS); } catch (TimeoutException ex) { LOG.warn("Timeout while trying to cancel closing of idle file. Idle" + - " file close may have failed", ex); + " file close may have failed", ex); } catch (Exception ex) { LOG.warn("Error while trying to cancel closing of idle file. ", ex); } @@ -612,10 +602,9 @@ private boolean shouldRotate() { // this method can get called from the scheduled thread so the // file gets closed later - so an implicit reference to this // bucket writer would still be alive in the Callable instance. - private void renameBucket(String bucketPath, - String targetPath, final FileSystem fs) throws IOException, - InterruptedException { - if(bucketPath.equals(targetPath)) { + private void renameBucket(String bucketPath, String targetPath, final FileSystem fs) + throws IOException, InterruptedException { + if (bucketPath.equals(targetPath)) { return; } @@ -646,7 +635,7 @@ private boolean isBatchComplete() { } void setClock(Clock clock) { - this.clock = clock; + this.clock = clock; } /** @@ -669,7 +658,7 @@ private static void checkAndThrowInterruptedException() * cancel the callable and throw an IOException */ private T callWithTimeout(final CallRunner callRunner) - throws IOException, InterruptedException { + throws IOException, InterruptedException { Future future = callTimeoutPool.submit(new Callable() { @Override public T call() throws Exception { diff --git a/flume-ng-sinks/flume-hdfs-sink/src/main/java/org/apache/flume/sink/hdfs/HDFSCompressedDataStream.java b/flume-ng-sinks/flume-hdfs-sink/src/main/java/org/apache/flume/sink/hdfs/HDFSCompressedDataStream.java index f1287958c8..80b7cb4c32 100644 --- a/flume-ng-sinks/flume-hdfs-sink/src/main/java/org/apache/flume/sink/hdfs/HDFSCompressedDataStream.java +++ b/flume-ng-sinks/flume-hdfs-sink/src/main/java/org/apache/flume/sink/hdfs/HDFSCompressedDataStream.java @@ -78,8 +78,8 @@ public void open(String filePath, CompressionCodec codec, Configuration conf = new Configuration(); Path dstPath = new Path(filePath); FileSystem hdfs = dstPath.getFileSystem(conf); - if(useRawLocalFileSystem) { - if(hdfs instanceof LocalFileSystem) { + if (useRawLocalFileSystem) { + if (hdfs instanceof LocalFileSystem) { hdfs = ((LocalFileSystem)hdfs).getRaw(); } else { logger.warn("useRawLocalFileSystem is set to true but file system " + @@ -87,14 +87,13 @@ public void open(String filePath, CompressionCodec codec, } } boolean appending = false; - if (conf.getBoolean("hdfs.append.support", false) == true && hdfs.isFile - (dstPath)) { + if (conf.getBoolean("hdfs.append.support", false) == true && hdfs.isFile(dstPath)) { fsOut = hdfs.append(dstPath); appending = true; } else { fsOut = hdfs.create(dstPath); } - if(compressor == null) { + if (compressor == null) { compressor = CodecPool.getCompressor(codec, conf); } cmpOut = codec.createOutputStream(fsOut, compressor); diff --git a/flume-ng-sinks/flume-hdfs-sink/src/main/java/org/apache/flume/sink/hdfs/HDFSDataStream.java b/flume-ng-sinks/flume-hdfs-sink/src/main/java/org/apache/flume/sink/hdfs/HDFSDataStream.java index 7054bfc0fc..c4ad9192d7 100644 --- a/flume-ng-sinks/flume-hdfs-sink/src/main/java/org/apache/flume/sink/hdfs/HDFSDataStream.java +++ b/flume-ng-sinks/flume-hdfs-sink/src/main/java/org/apache/flume/sink/hdfs/HDFSDataStream.java @@ -37,8 +37,7 @@ public class HDFSDataStream extends AbstractHDFSWriter { - private static final Logger logger = - LoggerFactory.getLogger(HDFSDataStream.class); + private static final Logger logger = LoggerFactory.getLogger(HDFSDataStream.class); private FSDataOutputStream outStream; private String serializerType; @@ -60,16 +59,13 @@ public void configure(Context context) { } @VisibleForTesting - protected FileSystem getDfs(Configuration conf, - Path dstPath) throws IOException{ - return dstPath.getFileSystem(conf); + protected FileSystem getDfs(Configuration conf, Path dstPath) throws IOException { + return dstPath.getFileSystem(conf); } - protected void doOpen(Configuration conf, - Path dstPath, FileSystem hdfs) throws - IOException { - if(useRawLocalFileSystem) { - if(hdfs instanceof LocalFileSystem) { + protected void doOpen(Configuration conf, Path dstPath, FileSystem hdfs) throws IOException { + if (useRawLocalFileSystem) { + if (hdfs instanceof LocalFileSystem) { hdfs = ((LocalFileSystem)hdfs).getRaw(); } else { logger.warn("useRawLocalFileSystem is set to true but file system " + @@ -78,8 +74,7 @@ protected void doOpen(Configuration conf, } boolean appending = false; - if (conf.getBoolean("hdfs.append.support", false) == true && hdfs.isFile - (dstPath)) { + if (conf.getBoolean("hdfs.append.support", false) == true && hdfs.isFile(dstPath)) { outStream = hdfs.append(dstPath); appending = true; } else { diff --git a/flume-ng-sinks/flume-hdfs-sink/src/main/java/org/apache/flume/sink/hdfs/HDFSEventSink.java b/flume-ng-sinks/flume-hdfs-sink/src/main/java/org/apache/flume/sink/hdfs/HDFSEventSink.java index 9a48841c4c..741f01e7b8 100644 --- a/flume-ng-sinks/flume-hdfs-sink/src/main/java/org/apache/flume/sink/hdfs/HDFSEventSink.java +++ b/flume-ng-sinks/flume-hdfs-sink/src/main/java/org/apache/flume/sink/hdfs/HDFSEventSink.java @@ -63,8 +63,7 @@ public interface WriterCallback { public void run(String filePath); } - private static final Logger LOG = LoggerFactory - .getLogger(HDFSEventSink.class); + private static final Logger LOG = LoggerFactory.getLogger(HDFSEventSink.class); private static String DIRECTORY_DELIMITER = System.getProperty("file.separator"); @@ -98,7 +97,6 @@ public interface WriterCallback { private static final int defaultThreadPoolSize = 10; private static final int defaultRollTimerPoolSize = 1; - private final HDFSWriterFactory writerFactory; private WriterLinkedHashMap sfWriters; @@ -217,23 +215,21 @@ public void configure(Context context) { String kerbKeytab = context.getString("hdfs.kerberosKeytab"); String proxyUser = context.getString("hdfs.proxyUser"); tryCount = context.getInteger("hdfs.closeTries", defaultTryCount); - if(tryCount <= 0) { + if (tryCount <= 0) { LOG.warn("Retry count value : " + tryCount + " is not " + - "valid. The sink will try to close the file until the file " + - "is eventually closed."); + "valid. The sink will try to close the file until the file " + + "is eventually closed."); tryCount = defaultTryCount; } - retryInterval = context.getLong("hdfs.retryInterval", - defaultRetryInterval); - if(retryInterval <= 0) { + retryInterval = context.getLong("hdfs.retryInterval", defaultRetryInterval); + if (retryInterval <= 0) { LOG.warn("Retry Interval value: " + retryInterval + " is not " + - "valid. If the first close of a file fails, " + - "it may remain open and will not be renamed."); + "valid. If the first close of a file fails, " + + "it may remain open and will not be renamed."); tryCount = 1; } - Preconditions.checkArgument(batchSize > 0, - "batchSize must be greater than 0"); + Preconditions.checkArgument(batchSize > 0, "batchSize must be greater than 0"); if (codecName == null) { codeC = null; compType = CompressionType.NONE; @@ -245,14 +241,13 @@ public void configure(Context context) { // Do not allow user to set fileType DataStream with codeC together // To prevent output file with compress extension (like .snappy) - if(fileType.equalsIgnoreCase(HDFSWriterFactory.DataStreamType) - && codecName != null) { + if (fileType.equalsIgnoreCase(HDFSWriterFactory.DataStreamType) && codecName != null) { throw new IllegalArgumentException("fileType: " + fileType + " which does NOT support compressed output. Please don't set codeC" + " or change the fileType if compressed output is desired."); } - if(fileType.equalsIgnoreCase(HDFSWriterFactory.CompStreamType)) { + if (fileType.equalsIgnoreCase(HDFSWriterFactory.CompStreamType)) { Preconditions.checkNotNull(codeC, "It's essential to set compress codec" + " when fileType is: " + fileType); } @@ -261,18 +256,15 @@ public void configure(Context context) { this.privExecutor = FlumeAuthenticationUtil.getAuthenticator( kerbConfPrincipal, kerbKeytab).proxyAs(proxyUser); - - - needRounding = context.getBoolean("hdfs.round", false); - if(needRounding) { + if (needRounding) { String unit = context.getString("hdfs.roundUnit", "second"); if (unit.equalsIgnoreCase("hour")) { this.roundUnit = Calendar.HOUR_OF_DAY; } else if (unit.equalsIgnoreCase("minute")) { this.roundUnit = Calendar.MINUTE; - } else if (unit.equalsIgnoreCase("second")){ + } else if (unit.equalsIgnoreCase("second")) { this.roundUnit = Calendar.SECOND; } else { LOG.warn("Rounding unit is not valid, please set one of" + @@ -280,11 +272,11 @@ public void configure(Context context) { needRounding = false; } this.roundValue = context.getInteger("hdfs.roundValue", 1); - if(roundUnit == Calendar.SECOND || roundUnit == Calendar.MINUTE){ + if (roundUnit == Calendar.SECOND || roundUnit == Calendar.MINUTE) { Preconditions.checkArgument(roundValue > 0 && roundValue <= 60, "Round value" + "must be > 0 and <= 60"); - } else if (roundUnit == Calendar.HOUR_OF_DAY){ + } else if (roundUnit == Calendar.HOUR_OF_DAY) { Preconditions.checkArgument(roundValue > 0 && roundValue <= 24, "Round value" + "must be > 0 and <= 24"); @@ -292,7 +284,7 @@ public void configure(Context context) { } this.useLocalTime = context.getBoolean("hdfs.useLocalTimeStamp", false); - if(useLocalTime) { + if (useLocalTime) { clock = new SystemClock(); } @@ -301,16 +293,13 @@ public void configure(Context context) { } } - private static boolean codecMatches(Class cls, - String codecName) { + private static boolean codecMatches(Class cls, String codecName) { String simpleName = cls.getSimpleName(); - if (cls.getName().equals(codecName) - || simpleName.equalsIgnoreCase(codecName)) { + if (cls.getName().equals(codecName) || simpleName.equalsIgnoreCase(codecName)) { return true; } if (simpleName.endsWith("Codec")) { - String prefix = simpleName.substring(0, - simpleName.length() - "Codec".length()); + String prefix = simpleName.substring(0, simpleName.length() - "Codec".length()); if (prefix.equalsIgnoreCase(codecName)) { return true; } @@ -321,8 +310,7 @@ private static boolean codecMatches(Class cls, @VisibleForTesting static CompressionCodec getCodec(String codecName) { Configuration conf = new Configuration(); - List> codecs = CompressionCodecFactory - .getCodecClasses(conf); + List> codecs = CompressionCodecFactory.getCodecClasses(conf); // Wish we could base this on DefaultCodec but appears not all codec's // extend DefaultCodec(Lzo) CompressionCodec codec = null; @@ -380,7 +368,7 @@ public Status process() throws EventDeliveryException { String realPath = BucketPath.escapeString(filePath, event.getHeaders(), timeZone, needRounding, roundUnit, roundValue, useLocalTime); String realName = BucketPath.escapeString(fileName, event.getHeaders(), - timeZone, needRounding, roundUnit, roundValue, useLocalTime); + timeZone, needRounding, roundUnit, roundValue, useLocalTime); String lookupPath = realPath + DIRECTORY_DELIMITER + realName; BucketWriter bucketWriter; @@ -418,7 +406,7 @@ public void run(String bucketPath) { bucketWriter.append(event); } catch (BucketClosedException ex) { LOG.info("Bucket was closed while trying to append, " + - "reinitializing bucket and writing event."); + "reinitializing bucket and writing event."); hdfsWriter = writerFactory.getWriter(fileType); bucketWriter = initializeBucketWriter(realPath, realName, lookupPath, hdfsWriter, closeCallback); @@ -468,16 +456,16 @@ public void run(String bucketPath) { } private BucketWriter initializeBucketWriter(String realPath, - String realName, String lookupPath, HDFSWriter hdfsWriter, - WriterCallback closeCallback) { + String realName, String lookupPath, HDFSWriter hdfsWriter, + WriterCallback closeCallback) { BucketWriter bucketWriter = new BucketWriter(rollInterval, - rollSize, rollCount, - batchSize, context, realPath, realName, inUsePrefix, inUseSuffix, - suffix, codeC, compType, hdfsWriter, timedRollerPool, - privExecutor, sinkCounter, idleTimeout, closeCallback, - lookupPath, callTimeout, callTimeoutPool, retryInterval, - tryCount); - if(mockFs != null) { + rollSize, rollCount, + batchSize, context, realPath, realName, inUsePrefix, inUseSuffix, + suffix, codeC, compType, hdfsWriter, timedRollerPool, + privExecutor, sinkCounter, idleTimeout, closeCallback, + lookupPath, callTimeout, callTimeoutPool, retryInterval, + tryCount); + if (mockFs != null) { bucketWriter.setFileSystem(mockFs); bucketWriter.setMockStream(mockWriter); } @@ -504,7 +492,7 @@ public void stop() { } // shut down all our thread pools - ExecutorService toShutdown[] = {callTimeoutPool, timedRollerPool}; + ExecutorService[] toShutdown = { callTimeoutPool, timedRollerPool }; for (ExecutorService execService : toShutdown) { execService.shutdown(); try { diff --git a/flume-ng-sinks/flume-hdfs-sink/src/main/java/org/apache/flume/sink/hdfs/HDFSSequenceFile.java b/flume-ng-sinks/flume-hdfs-sink/src/main/java/org/apache/flume/sink/hdfs/HDFSSequenceFile.java index a261cce6b3..ba8b30d7e0 100644 --- a/flume-ng-sinks/flume-hdfs-sink/src/main/java/org/apache/flume/sink/hdfs/HDFSSequenceFile.java +++ b/flume-ng-sinks/flume-hdfs-sink/src/main/java/org/apache/flume/sink/hdfs/HDFSSequenceFile.java @@ -81,16 +81,15 @@ public void open(String filePath, CompressionCodec codeC, protected void open(Path dstPath, CompressionCodec codeC, CompressionType compType, Configuration conf, FileSystem hdfs) throws IOException { - if(useRawLocalFileSystem) { - if(hdfs instanceof LocalFileSystem) { + if (useRawLocalFileSystem) { + if (hdfs instanceof LocalFileSystem) { hdfs = ((LocalFileSystem)hdfs).getRaw(); } else { logger.warn("useRawLocalFileSystem is set to true but file system " + "is not of type LocalFileSystem: " + hdfs.getClass().getName()); } } - if (conf.getBoolean("hdfs.append.support", false) == true && hdfs.isFile - (dstPath)) { + if (conf.getBoolean("hdfs.append.support", false) == true && hdfs.isFile(dstPath)) { outStream = hdfs.append(dstPath); } else { outStream = hdfs.create(dstPath); diff --git a/flume-ng-sinks/flume-hdfs-sink/src/main/java/org/apache/flume/sink/hdfs/KerberosUser.java b/flume-ng-sinks/flume-hdfs-sink/src/main/java/org/apache/flume/sink/hdfs/KerberosUser.java index 516988e45b..43297e25b7 100644 --- a/flume-ng-sinks/flume-hdfs-sink/src/main/java/org/apache/flume/sink/hdfs/KerberosUser.java +++ b/flume-ng-sinks/flume-hdfs-sink/src/main/java/org/apache/flume/sink/hdfs/KerberosUser.java @@ -46,7 +46,9 @@ public boolean equals(Object obj) { return false; } final KerberosUser other = (KerberosUser) obj; - if ((this.principal == null) ? (other.principal != null) : !this.principal.equals(other.principal)) { + if ((this.principal == null) ? + (other.principal != null) : + !this.principal.equals(other.principal)) { return false; } if ((this.keyTab == null) ? (other.keyTab != null) : !this.keyTab.equals(other.keyTab)) { diff --git a/flume-ng-sinks/flume-hdfs-sink/src/main/java/org/apache/flume/sink/hdfs/SequenceFileSerializerType.java b/flume-ng-sinks/flume-hdfs-sink/src/main/java/org/apache/flume/sink/hdfs/SequenceFileSerializerType.java index 4351488399..2ad7689f27 100644 --- a/flume-ng-sinks/flume-hdfs-sink/src/main/java/org/apache/flume/sink/hdfs/SequenceFileSerializerType.java +++ b/flume-ng-sinks/flume-hdfs-sink/src/main/java/org/apache/flume/sink/hdfs/SequenceFileSerializerType.java @@ -25,8 +25,7 @@ public enum SequenceFileSerializerType { private final Class builderClass; - SequenceFileSerializerType( - Class builderClass) { + SequenceFileSerializerType(Class builderClass) { this.builderClass = builderClass; } diff --git a/flume-ng-sinks/flume-hive-sink/src/main/java/org/apache/flume/sink/hive/HiveDelimitedTextSerializer.java b/flume-ng-sinks/flume-hive-sink/src/main/java/org/apache/flume/sink/hive/HiveDelimitedTextSerializer.java index 4d70aaabd1..59520e7d88 100644 --- a/flume-ng-sinks/flume-hive-sink/src/main/java/org/apache/flume/sink/hive/HiveDelimitedTextSerializer.java +++ b/flume-ng-sinks/flume-hive-sink/src/main/java/org/apache/flume/sink/hive/HiveDelimitedTextSerializer.java @@ -18,7 +18,6 @@ package org.apache.flume.sink.hive; - import org.apache.flume.Context; import org.apache.flume.Event; import org.apache.hive.hcatalog.streaming.DelimitedInputWriter; @@ -60,12 +59,11 @@ public void write(TransactionBatch txnBatch, Collection events) @Override public RecordWriter createRecordWriter(HiveEndPoint endPoint) - throws StreamingException, IOException, ClassNotFoundException { + throws StreamingException, IOException, ClassNotFoundException { if (serdeSeparator == null) { return new DelimitedInputWriter(fieldToColMapping, delimiter, endPoint); } - return new DelimitedInputWriter(fieldToColMapping, delimiter, endPoint, null - , serdeSeparator); + return new DelimitedInputWriter(fieldToColMapping, delimiter, endPoint, null, serdeSeparator); } @Override @@ -90,8 +88,8 @@ private static String parseDelimiterSpec(String delimiter) { return null; } if (delimiter.charAt(0) == '"' && - delimiter.charAt(delimiter.length()-1) == '"') { - return delimiter.substring(1,delimiter.length()-1); + delimiter.charAt(delimiter.length() - 1) == '"') { + return delimiter.substring(1,delimiter.length() - 1); } return delimiter; } @@ -105,9 +103,9 @@ private static Character parseSerdeSeparatorSpec(String separatorStr) { return separatorStr.charAt(0); } if (separatorStr.length() == 3 && - separatorStr.charAt(2) == '\'' && - separatorStr.charAt(separatorStr.length()-1) == '\'') { - return separatorStr.charAt(1); + separatorStr.charAt(2) == '\'' && + separatorStr.charAt(separatorStr.length() - 1) == '\'') { + return separatorStr.charAt(1); } throw new IllegalArgumentException("serializer.serdeSeparator spec is invalid " + diff --git a/flume-ng-sinks/flume-hive-sink/src/main/java/org/apache/flume/sink/hive/HiveEventSerializer.java b/flume-ng-sinks/flume-hive-sink/src/main/java/org/apache/flume/sink/hive/HiveEventSerializer.java index 386484c636..7ed2c82763 100644 --- a/flume-ng-sinks/flume-hive-sink/src/main/java/org/apache/flume/sink/hive/HiveEventSerializer.java +++ b/flume-ng-sinks/flume-hive-sink/src/main/java/org/apache/flume/sink/hive/HiveEventSerializer.java @@ -18,7 +18,6 @@ package org.apache.flume.sink.hive; - import org.apache.flume.Event; import org.apache.flume.conf.Configurable; import org.apache.hive.hcatalog.streaming.HiveEndPoint; diff --git a/flume-ng-sinks/flume-hive-sink/src/main/java/org/apache/flume/sink/hive/HiveSink.java b/flume-ng-sinks/flume-hive-sink/src/main/java/org/apache/flume/sink/hive/HiveSink.java index d93bca3b3d..cc5cdca844 100644 --- a/flume-ng-sinks/flume-hive-sink/src/main/java/org/apache/flume/sink/hive/HiveSink.java +++ b/flume-ng-sinks/flume-hive-sink/src/main/java/org/apache/flume/sink/hive/HiveSink.java @@ -32,7 +32,7 @@ import org.apache.flume.formatter.output.BucketPath; import org.apache.flume.instrumentation.SinkCounter; import org.apache.flume.sink.AbstractSink; -import org.apache.hive.hcatalog.streaming.*; +import org.apache.hive.hcatalog.streaming.HiveEndPoint; import org.slf4j.Logger; import org.slf4j.LoggerFactory; @@ -52,8 +52,7 @@ public class HiveSink extends AbstractSink implements Configurable { - private static final Logger LOG = LoggerFactory - .getLogger(HiveSink.class); + private static final Logger LOG = LoggerFactory.getLogger(HiveSink.class); private static final int DEFAULT_MAXOPENCONNECTIONS = 500; private static final int DEFAULT_TXNSPERBATCH = 100; @@ -62,7 +61,6 @@ public class HiveSink extends AbstractSink implements Configurable { private static final int DEFAULT_IDLETIMEOUT = 0; private static final int DEFAULT_HEARTBEATINTERVAL = 240; // seconds - private Map allWriters; private SinkCounter sinkCounter; @@ -162,7 +160,8 @@ public void configure(Context context) { + DEFAULT_HEARTBEATINTERVAL); heartBeatInterval = DEFAULT_HEARTBEATINTERVAL; } - maxOpenConnections = context.getInteger(Config.MAX_OPEN_CONNECTIONS, DEFAULT_MAXOPENCONNECTIONS); + maxOpenConnections = context.getInteger(Config.MAX_OPEN_CONNECTIONS, + DEFAULT_MAXOPENCONNECTIONS); autoCreatePartitions = context.getBoolean("autoCreatePartitions", true); // Timestamp processing @@ -177,7 +176,7 @@ public void configure(Context context) { this.roundUnit = Calendar.HOUR_OF_DAY; } else if (unit.equalsIgnoreCase(Config.MINUTE)) { this.roundUnit = Calendar.MINUTE; - } else if (unit.equalsIgnoreCase(Config.SECOND)){ + } else if (unit.equalsIgnoreCase(Config.SECOND)) { this.roundUnit = Calendar.SECOND; } else { LOG.warn(getName() + ". Rounding unit is not valid, please set one of " + @@ -185,10 +184,10 @@ public void configure(Context context) { needRounding = false; } this.roundValue = context.getInteger(Config.ROUND_VALUE, 1); - if (roundUnit == Calendar.SECOND || roundUnit == Calendar.MINUTE){ + if (roundUnit == Calendar.SECOND || roundUnit == Calendar.MINUTE) { Preconditions.checkArgument(roundValue > 0 && roundValue <= 60, "Round value must be > 0 and <= 60"); - } else if (roundUnit == Calendar.HOUR_OF_DAY){ + } else if (roundUnit == Calendar.HOUR_OF_DAY) { Preconditions.checkArgument(roundValue > 0 && roundValue <= 24, "Round value must be > 0 and <= 24"); } @@ -215,8 +214,8 @@ protected SinkCounter getCounter() { return sinkCounter; } private HiveEventSerializer createSerializer(String serializerName) { - if(serializerName.compareToIgnoreCase(HiveDelimitedTextSerializer.ALIAS) == 0 || - serializerName.compareTo(HiveDelimitedTextSerializer.class.getName()) == 0) { + if (serializerName.compareToIgnoreCase(HiveDelimitedTextSerializer.ALIAS) == 0 || + serializerName.compareTo(HiveDelimitedTextSerializer.class.getName()) == 0) { return new HiveDelimitedTextSerializer(); } else if (serializerName.compareToIgnoreCase(HiveJsonSerializer.ALIAS) == 0 || serializerName.compareTo(HiveJsonSerializer.class.getName()) == 0) { @@ -345,7 +344,7 @@ private HiveWriter getOrCreateWriter(Map activeWriters callTimeout, callTimeoutPool, proxyUser, serializer, sinkCounter); sinkCounter.incrementConnectionCreatedCount(); - if (allWriters.size() > maxOpenConnections){ + if (allWriters.size() > maxOpenConnections) { int retired = closeIdleWriters(); if (retired == 0) { closeEldestWriter(); @@ -353,8 +352,7 @@ private HiveWriter getOrCreateWriter(Map activeWriters } allWriters.put(endPoint, writer); activeWriters.put(endPoint, writer); - } - else { + } else { if (activeWriters.get(endPoint) == null) { activeWriters.put(endPoint,writer); } @@ -425,7 +423,7 @@ private int closeIdleWriters() throws InterruptedException { } } //2) Retire them - for(HiveEndPoint ep : retirees) { + for (HiveEndPoint ep : retirees) { sinkCounter.incrementConnectionClosedCount(); LOG.info(getName() + ": Closing idle Writer to Hive end point : {}", ep); allWriters.remove(ep).close(); @@ -440,7 +438,7 @@ private int closeIdleWriters() throws InterruptedException { private void closeAllWriters() throws InterruptedException { //1) Retire writers for (Entry entry : allWriters.entrySet()) { - entry.getValue().close(); + entry.getValue().close(); } //2) Clear cache @@ -453,7 +451,7 @@ private void closeAllWriters() throws InterruptedException { */ private void abortAllWriters() throws InterruptedException { for (Entry entry : allWriters.entrySet()) { - entry.getValue().abort(); + entry.getValue().abort(); } } diff --git a/flume-ng-sinks/flume-hive-sink/src/main/java/org/apache/flume/sink/hive/HiveWriter.java b/flume-ng-sinks/flume-hive-sink/src/main/java/org/apache/flume/sink/hive/HiveWriter.java index ec30c98515..7106696a13 100644 --- a/flume-ng-sinks/flume-hive-sink/src/main/java/org/apache/flume/sink/hive/HiveWriter.java +++ b/flume-ng-sinks/flume-hive-sink/src/main/java/org/apache/flume/sink/hive/HiveWriter.java @@ -6,9 +6,9 @@ * to you under the Apache License, Version 2.0 (the * "License"); you may not use this file except in compliance * with the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * + *

    + * http://www.apache.org/licenses/LICENSE-2.0 + *

    * Unless required by applicable law or agreed to in writing, software * distributed under the License is distributed on an "AS IS" BASIS, * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. @@ -18,6 +18,18 @@ package org.apache.flume.sink.hive; +import org.apache.flume.Event; +import org.apache.flume.instrumentation.SinkCounter; +import org.apache.hive.hcatalog.streaming.HiveEndPoint; +import org.apache.hive.hcatalog.streaming.RecordWriter; +import org.apache.hive.hcatalog.streaming.SerializationError; +import org.apache.hive.hcatalog.streaming.StreamingConnection; +import org.apache.hive.hcatalog.streaming.StreamingException; +import org.apache.hive.hcatalog.streaming.StreamingIOFailure; +import org.apache.hive.hcatalog.streaming.TransactionBatch; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + import java.io.IOException; import java.util.ArrayList; import java.util.concurrent.Callable; @@ -27,24 +39,12 @@ import java.util.concurrent.TimeUnit; import java.util.concurrent.TimeoutException; -import org.apache.hive.hcatalog.streaming.*; - -import org.apache.flume.Event; - -import org.apache.flume.instrumentation.SinkCounter; - - -import org.slf4j.Logger; -import org.slf4j.LoggerFactory; - - /** * Internal API intended for HiveSink use. */ class HiveWriter { - private static final Logger LOG = LoggerFactory - .getLogger(HiveWriter.class); + private static final Logger LOG = LoggerFactory.getLogger(HiveWriter.class); private final HiveEndPoint endPoint; private HiveEventSerializer serializer; @@ -76,7 +76,7 @@ class HiveWriter { boolean autoCreatePartitions, long callTimeout, ExecutorService callTimeoutPool, String hiveUser, HiveEventSerializer serializer, SinkCounter sinkCounter) - throws ConnectException, InterruptedException { + throws ConnectException, InterruptedException { try { this.autoCreatePartitions = autoCreatePartitions; this.sinkCounter = sinkCounter; @@ -130,13 +130,13 @@ public int getRemainingTxns() { * @throws InterruptedException */ public synchronized void write(final Event event) - throws WriteException, InterruptedException { + throws WriteException, InterruptedException { if (closed) { throw new IllegalStateException("Writer closed. Cannot write to : " + endPoint); } batch.add(event); - if(batch.size()== writeBatchSz) { + if (batch.size() == writeBatchSz) { // write the event writeEventBatchToSerializer(); } @@ -147,7 +147,7 @@ public synchronized void write(final Event event) } private void writeEventBatchToSerializer() - throws InterruptedException, WriteException { + throws InterruptedException, WriteException { try { timedCall(new CallRunner1() { @Override @@ -180,15 +180,15 @@ public Void call() throws InterruptedException, StreamingException { * new TxnBatch if current Txn batch is exhausted */ public void flush(boolean rollToNext) - throws CommitException, TxnBatchException, TxnFailure, InterruptedException, - WriteException { - if(!batch.isEmpty()) { + throws CommitException, TxnBatchException, TxnFailure, InterruptedException, + WriteException { + if (!batch.isEmpty()) { writeEventBatchToSerializer(); batch.clear(); } //0 Heart beat on TxnBatch - if(hearbeatNeeded) { + if (hearbeatNeeded) { hearbeatNeeded = false; heartBeat(); } @@ -197,16 +197,16 @@ public void flush(boolean rollToNext) try { //1 commit txn & close batch if needed commitTxn(); - if(txnBatch.remainingTransactions() == 0) { + if (txnBatch.remainingTransactions() == 0) { closeTxnBatch(); txnBatch = null; - if(rollToNext) { + if (rollToNext) { txnBatch = nextTxnBatch(recordWriter); } } //2 roll to next Txn - if(rollToNext) { + if (rollToNext) { LOG.debug("Switching to next Txn for {}", endPoint); txnBatch.beginNextTransaction(); // does not block } @@ -219,7 +219,7 @@ public void flush(boolean rollToNext) * Aborts the current Txn * @throws InterruptedException */ - public void abort() throws InterruptedException { + public void abort() throws InterruptedException { batch.clear(); abortTxn(); } @@ -227,7 +227,7 @@ public void abort() throws InterruptedException { /** Queues up a heartbeat request on the current and remaining txns using the * heartbeatThdPool and returns immediately */ - public void heartBeat() throws InterruptedException { + public void heartBeat() throws InterruptedException { // 1) schedule the heartbeat on one thread in pool try { timedCall(new CallRunner1() { @@ -261,43 +261,43 @@ public void close() throws InterruptedException { private void abortRemainingTxns() throws InterruptedException { - try { - if ( !isClosed(txnBatch.getCurrentTransactionState()) ) { - abortCurrTxnHelper(); - } + try { + if (!isClosed(txnBatch.getCurrentTransactionState())) { + abortCurrTxnHelper(); + } - // recursively abort remaining txns - if(txnBatch.remainingTransactions()>0) { - timedCall( - new CallRunner1() { - @Override - public Void call() throws StreamingException, InterruptedException { - txnBatch.beginNextTransaction(); - return null; - } - }); - abortRemainingTxns(); - } - } catch (StreamingException e) { - LOG.warn("Error when aborting remaining transactions in batch " + txnBatch, e); - return; - } catch (TimeoutException e) { - LOG.warn("Timed out when aborting remaining transactions in batch " + txnBatch, e); - return; + // recursively abort remaining txns + if (txnBatch.remainingTransactions() > 0) { + timedCall( + new CallRunner1() { + @Override + public Void call() throws StreamingException, InterruptedException { + txnBatch.beginNextTransaction(); + return null; + } + }); + abortRemainingTxns(); } + } catch (StreamingException e) { + LOG.warn("Error when aborting remaining transactions in batch " + txnBatch, e); + return; + } catch (TimeoutException e) { + LOG.warn("Timed out when aborting remaining transactions in batch " + txnBatch, e); + return; + } } private void abortCurrTxnHelper() throws TimeoutException, InterruptedException { try { timedCall( - new CallRunner1() { - @Override - public Void call() throws StreamingException, InterruptedException { - txnBatch.abort(); - LOG.info("Aborted txn " + txnBatch.getCurrentTxnId()); - return null; - } - } + new CallRunner1() { + @Override + public Void call() throws StreamingException, InterruptedException { + txnBatch.abort(); + LOG.info("Aborted txn " + txnBatch.getCurrentTxnId()); + return null; + } + } ); } catch (StreamingException e) { LOG.warn("Unable to abort transaction " + txnBatch.getCurrentTxnId(), e); @@ -306,10 +306,12 @@ public Void call() throws StreamingException, InterruptedException { } private boolean isClosed(TransactionBatch.TxnState txnState) { - if(txnState == TransactionBatch.TxnState.COMMITTED) + if (txnState == TransactionBatch.TxnState.COMMITTED) { return true; - if(txnState == TransactionBatch.TxnState.ABORTED) + } + if (txnState == TransactionBatch.TxnState.ABORTED) { return true; + } return false; } @@ -360,7 +362,8 @@ public Void call() throws StreamingException, InterruptedException { } catch (InterruptedException e) { throw e; } catch (TimeoutException e) { - LOG.warn("Timeout while aborting Txn " + txnBatch.getCurrentTxnId() + " on EndPoint: " + endPoint, e); + LOG.warn("Timeout while aborting Txn " + txnBatch.getCurrentTxnId() + + " on EndPoint: " + endPoint, e); } catch (Exception e) { LOG.warn("Error aborting Txn " + txnBatch.getCurrentTxnId() + " on EndPoint: " + endPoint, e); // Suppressing exceptions as we don't care for errors on abort @@ -368,9 +371,9 @@ public Void call() throws StreamingException, InterruptedException { } private StreamingConnection newConnection(final String proxyUser) - throws InterruptedException, ConnectException { + throws InterruptedException, ConnectException { try { - return timedCall(new CallRunner1() { + return timedCall(new CallRunner1() { @Override public StreamingConnection call() throws InterruptedException, StreamingException { return endPoint.newConnection(autoCreatePartitions); // could block @@ -382,7 +385,7 @@ public StreamingConnection call() throws InterruptedException, StreamingExceptio } private TransactionBatch nextTxnBatch(final RecordWriter recordWriter) - throws InterruptedException, TxnBatchException { + throws InterruptedException, TxnBatchException { LOG.debug("Fetching new Txn Batch for {}", endPoint); TransactionBatch batch = null; try { @@ -418,7 +421,7 @@ public Void call() throws InterruptedException, StreamingException { } private T timedCall(final CallRunner1 callRunner) - throws TimeoutException, InterruptedException, StreamingException { + throws TimeoutException, InterruptedException, StreamingException { Future future = callTimeoutPool.submit(new Callable() { @Override public T call() throws StreamingException, InterruptedException, Failure { @@ -439,7 +442,7 @@ public T call() throws StreamingException, InterruptedException, Failure { } catch (ExecutionException e1) { sinkCounter.incrementConnectionFailedCount(); Throwable cause = e1.getCause(); - if (cause instanceof IOException ) { + if (cause instanceof IOException) { throw new StreamingException("I/O Failure", (IOException) cause); } else if (cause instanceof StreamingException) { throw (StreamingException) cause; @@ -468,12 +471,10 @@ private interface CallRunner { T call() throws Exception; } - private interface CallRunner1 { T call() throws StreamingException, InterruptedException, Failure; } - public static class Failure extends Exception { public Failure(String msg, Throwable cause) { super(msg, cause); @@ -488,7 +489,7 @@ public WriteException(HiveEndPoint endPoint, Long currentTxnId, Throwable cause) public static class CommitException extends Failure { public CommitException(HiveEndPoint endPoint, Long txnID, Throwable cause) { - super("Commit of Txn " + txnID + " failed on EndPoint: " + endPoint, cause); + super("Commit of Txn " + txnID + " failed on EndPoint: " + endPoint, cause); } } diff --git a/flume-ng-sinks/flume-irc-sink/src/main/java/org/apache/flume/sink/irc/IRCSink.java b/flume-ng-sinks/flume-irc-sink/src/main/java/org/apache/flume/sink/irc/IRCSink.java index 40657b40b3..52bbfc853a 100644 --- a/flume-ng-sinks/flume-irc-sink/src/main/java/org/apache/flume/sink/irc/IRCSink.java +++ b/flume-ng-sinks/flume-irc-sink/src/main/java/org/apache/flume/sink/irc/IRCSink.java @@ -60,7 +60,7 @@ public class IRCSink extends AbstractSink implements Configurable { private CounterGroup counterGroup; - static public class IRCConnectionListener implements IRCEventListener { + public static class IRCConnectionListener implements IRCEventListener { public void onRegistered() { } @@ -214,7 +214,7 @@ private void sendLine(Event event) { if (splitLines) { String[] lines = body.split(splitChars); - for(String line: lines) { + for (String line: lines) { connection.doPrivmsg(IRC_CHANNEL_PREFIX + this.chan, line); } } else { diff --git a/flume-ng-sinks/flume-ng-elasticsearch-sink/src/main/java/org/apache/flume/sink/elasticsearch/AbstractElasticSearchIndexRequestBuilderFactory.java b/flume-ng-sinks/flume-ng-elasticsearch-sink/src/main/java/org/apache/flume/sink/elasticsearch/AbstractElasticSearchIndexRequestBuilderFactory.java index 99961423ee..754155cc9f 100644 --- a/flume-ng-sinks/flume-ng-elasticsearch-sink/src/main/java/org/apache/flume/sink/elasticsearch/AbstractElasticSearchIndexRequestBuilderFactory.java +++ b/flume-ng-sinks/flume-ng-elasticsearch-sink/src/main/java/org/apache/flume/sink/elasticsearch/AbstractElasticSearchIndexRequestBuilderFactory.java @@ -49,8 +49,7 @@ public abstract class AbstractElasticSearchIndexRequestBuilderFactory * Constructor for subclasses * @param fastDateFormat {@link FastDateFormat} to use for index names */ - protected AbstractElasticSearchIndexRequestBuilderFactory( - FastDateFormat fastDateFormat) { + protected AbstractElasticSearchIndexRequestBuilderFactory(FastDateFormat fastDateFormat) { this.fastDateFormat = fastDateFormat; } @@ -94,11 +93,11 @@ IndexRequestBuilder prepareIndex(Client client) { /** * Gets the name of the index to use for an index request - * @return index name of the form 'indexPrefix-formattedTimestamp' * @param indexPrefix * Prefix of index name to use -- as configured on the sink * @param timestamp * timestamp (millis) to format / use + * @return index name of the form 'indexPrefix-formattedTimestamp' */ protected String getIndexName(String indexPrefix, long timestamp) { return new StringBuilder(indexPrefix).append('-') diff --git a/flume-ng-sinks/flume-ng-elasticsearch-sink/src/main/java/org/apache/flume/sink/elasticsearch/ElasticSearchIndexRequestBuilderFactory.java b/flume-ng-sinks/flume-ng-elasticsearch-sink/src/main/java/org/apache/flume/sink/elasticsearch/ElasticSearchIndexRequestBuilderFactory.java index 1ca227a897..f76308ceab 100644 --- a/flume-ng-sinks/flume-ng-elasticsearch-sink/src/main/java/org/apache/flume/sink/elasticsearch/ElasticSearchIndexRequestBuilderFactory.java +++ b/flume-ng-sinks/flume-ng-elasticsearch-sink/src/main/java/org/apache/flume/sink/elasticsearch/ElasticSearchIndexRequestBuilderFactory.java @@ -40,7 +40,6 @@ public interface ElasticSearchIndexRequestBuilderFactory extends Configurable, TimeZone.getTimeZone("Etc/UTC")); /** - * @return prepared ElasticSearch {@link IndexRequestBuilder} instance * @param client * ElasticSearch {@link Client} to prepare index from * @param indexPrefix @@ -49,6 +48,7 @@ public interface ElasticSearchIndexRequestBuilderFactory extends Configurable, * Index type to use -- as configured on the sink * @param event * Flume event to serialize and add to index request + * @return prepared ElasticSearch {@link IndexRequestBuilder} instance * @throws IOException * If an error occurs e.g. during serialization */ diff --git a/flume-ng-sinks/flume-ng-elasticsearch-sink/src/main/java/org/apache/flume/sink/elasticsearch/ElasticSearchSink.java b/flume-ng-sinks/flume-ng-elasticsearch-sink/src/main/java/org/apache/flume/sink/elasticsearch/ElasticSearchSink.java index 1d9dfce765..ebafb9f901 100644 --- a/flume-ng-sinks/flume-ng-elasticsearch-sink/src/main/java/org/apache/flume/sink/elasticsearch/ElasticSearchSink.java +++ b/flume-ng-sinks/flume-ng-elasticsearch-sink/src/main/java/org/apache/flume/sink/elasticsearch/ElasticSearchSink.java @@ -54,6 +54,7 @@ import java.util.concurrent.TimeUnit; import java.util.regex.Matcher; import java.util.regex.Pattern; + import static org.apache.flume.sink.elasticsearch.ElasticSearchSinkConstants.CLIENT_PREFIX; import static org.apache.flume.sink.elasticsearch.ElasticSearchSinkConstants.CLIENT_TYPE; import static org.apache.flume.sink.elasticsearch.ElasticSearchSinkConstants.DEFAULT_CLIENT_TYPE; diff --git a/flume-ng-sinks/flume-ng-elasticsearch-sink/src/main/java/org/apache/flume/sink/elasticsearch/EventSerializerIndexRequestBuilderFactory.java b/flume-ng-sinks/flume-ng-elasticsearch-sink/src/main/java/org/apache/flume/sink/elasticsearch/EventSerializerIndexRequestBuilderFactory.java index c71b2e55a4..d6cca50a35 100644 --- a/flume-ng-sinks/flume-ng-elasticsearch-sink/src/main/java/org/apache/flume/sink/elasticsearch/EventSerializerIndexRequestBuilderFactory.java +++ b/flume-ng-sinks/flume-ng-elasticsearch-sink/src/main/java/org/apache/flume/sink/elasticsearch/EventSerializerIndexRequestBuilderFactory.java @@ -33,7 +33,7 @@ * {@link ElasticSearchEventSerializer} instance configured on the sink. */ public class EventSerializerIndexRequestBuilderFactory - extends AbstractElasticSearchIndexRequestBuilderFactory { + extends AbstractElasticSearchIndexRequestBuilderFactory { protected final ElasticSearchEventSerializer serializer; diff --git a/flume-ng-sinks/flume-ng-elasticsearch-sink/src/main/java/org/apache/flume/sink/elasticsearch/client/ElasticSearchClientFactory.java b/flume-ng-sinks/flume-ng-elasticsearch-sink/src/main/java/org/apache/flume/sink/elasticsearch/client/ElasticSearchClientFactory.java index 873157ae85..cb34394eba 100644 --- a/flume-ng-sinks/flume-ng-elasticsearch-sink/src/main/java/org/apache/flume/sink/elasticsearch/client/ElasticSearchClientFactory.java +++ b/flume-ng-sinks/flume-ng-elasticsearch-sink/src/main/java/org/apache/flume/sink/elasticsearch/client/ElasticSearchClientFactory.java @@ -63,8 +63,10 @@ public ElasticSearchClient getClient(String clientType, String[] hostNames, * * @return Local elastic search instance client */ - public ElasticSearchClient getLocalClient(String clientType, ElasticSearchEventSerializer serializer, - ElasticSearchIndexRequestBuilderFactory indexBuilder) throws NoSuchClientTypeException { + public ElasticSearchClient getLocalClient(String clientType, + ElasticSearchEventSerializer serializer, + ElasticSearchIndexRequestBuilderFactory indexBuilder) + throws NoSuchClientTypeException { if (clientType.equalsIgnoreCase(TransportClient) && serializer != null) { return new ElasticSearchTransportClient(serializer); } else if (clientType.equalsIgnoreCase(TransportClient) && indexBuilder != null) { diff --git a/flume-ng-sinks/flume-ng-elasticsearch-sink/src/main/java/org/apache/flume/sink/elasticsearch/client/ElasticSearchRestClient.java b/flume-ng-sinks/flume-ng-elasticsearch-sink/src/main/java/org/apache/flume/sink/elasticsearch/client/ElasticSearchRestClient.java index 0d1c37f9ea..e51efe23c8 100644 --- a/flume-ng-sinks/flume-ng-elasticsearch-sink/src/main/java/org/apache/flume/sink/elasticsearch/client/ElasticSearchRestClient.java +++ b/flume-ng-sinks/flume-ng-elasticsearch-sink/src/main/java/org/apache/flume/sink/elasticsearch/client/ElasticSearchRestClient.java @@ -25,12 +25,6 @@ import org.apache.flume.EventDeliveryException; import org.apache.flume.sink.elasticsearch.ElasticSearchEventSerializer; import org.apache.flume.sink.elasticsearch.IndexNameBuilder; -import org.slf4j.Logger; -import org.slf4j.LoggerFactory; - -import java.util.Arrays; -import java.util.HashMap; -import java.util.Map; import org.apache.http.HttpResponse; import org.apache.http.HttpStatus; import org.apache.http.client.HttpClient; @@ -39,6 +33,12 @@ import org.apache.http.impl.client.DefaultHttpClient; import org.apache.http.util.EntityUtils; import org.elasticsearch.common.bytes.BytesReference; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +import java.util.Arrays; +import java.util.HashMap; +import java.util.Map; /** * Rest ElasticSearch client which is responsible for sending bulks of events to @@ -92,7 +92,8 @@ public void close() { } @Override - public void addEvent(Event event, IndexNameBuilder indexNameBuilder, String indexType, long ttlMs) throws Exception { + public void addEvent(Event event, IndexNameBuilder indexNameBuilder, String indexType, + long ttlMs) throws Exception { BytesReference content = serializer.getContentBuilder(event).bytes(); Map> parameters = new HashMap>(); Map indexParameters = new HashMap(); @@ -104,7 +105,7 @@ public void addEvent(Event event, IndexNameBuilder indexNameBuilder, String inde parameters.put(INDEX_OPERATION_NAME, indexParameters); Gson gson = new Gson(); - synchronized(bulkBuilder) { + synchronized (bulkBuilder) { bulkBuilder.append(gson.toJson(parameters)); bulkBuilder.append("\n"); bulkBuilder.append(content.toBytesArray().toUtf8()); @@ -131,8 +132,10 @@ public void execute() throws Exception { response = httpClient.execute(httpRequest); statusCode = response.getStatusLine().getStatusCode(); logger.info("Status code from elasticsearch: " + statusCode); - if (response.getEntity() != null) - logger.debug("Status message from elasticsearch: " + EntityUtils.toString(response.getEntity(), "UTF-8")); + if (response.getEntity() != null) { + logger.debug("Status message from elasticsearch: " + + EntityUtils.toString(response.getEntity(), "UTF-8")); + } } if (statusCode != HttpStatus.SC_OK) { diff --git a/flume-ng-sinks/flume-ng-elasticsearch-sink/src/main/java/org/apache/flume/sink/elasticsearch/client/ElasticSearchTransportClient.java b/flume-ng-sinks/flume-ng-elasticsearch-sink/src/main/java/org/apache/flume/sink/elasticsearch/client/ElasticSearchTransportClient.java index d44c8ad9cc..2cf365e7ba 100644 --- a/flume-ng-sinks/flume-ng-elasticsearch-sink/src/main/java/org/apache/flume/sink/elasticsearch/client/ElasticSearchTransportClient.java +++ b/flume-ng-sinks/flume-ng-elasticsearch-sink/src/main/java/org/apache/flume/sink/elasticsearch/client/ElasticSearchTransportClient.java @@ -122,12 +122,10 @@ public ElasticSearchTransportClient(Client client, /** * Used for testing - * - * @param client ElasticSearch Client - * @param serializer Event Serializer */ public ElasticSearchTransportClient(Client client, - ElasticSearchIndexRequestBuilderFactory requestBuilderFactory) throws IOException { + ElasticSearchIndexRequestBuilderFactory requestBuilderFactory) + throws IOException { this.client = client; requestBuilderFactory.createIndexRequest(client, null, null, null); } diff --git a/flume-ng-sinks/flume-ng-elasticsearch-sink/src/main/java/org/apache/flume/sink/elasticsearch/client/RoundRobinList.java b/flume-ng-sinks/flume-ng-elasticsearch-sink/src/main/java/org/apache/flume/sink/elasticsearch/client/RoundRobinList.java index dbad8d841d..4cbbe91195 100644 --- a/flume-ng-sinks/flume-ng-elasticsearch-sink/src/main/java/org/apache/flume/sink/elasticsearch/client/RoundRobinList.java +++ b/flume-ng-sinks/flume-ng-elasticsearch-sink/src/main/java/org/apache/flume/sink/elasticsearch/client/RoundRobinList.java @@ -29,7 +29,7 @@ public RoundRobinList(Collection elements) { iterator = this.elements.iterator(); } - synchronized public T get() { + public synchronized T get() { if (iterator.hasNext()) { return iterator.next(); } else { diff --git a/flume-ng-sinks/flume-ng-hbase-sink/src/main/java/org/apache/flume/sink/hbase/AsyncHBaseSink.java b/flume-ng-sinks/flume-ng-hbase-sink/src/main/java/org/apache/flume/sink/hbase/AsyncHBaseSink.java index 28f0de1744..280d0b0ffd 100644 --- a/flume-ng-sinks/flume-ng-hbase-sink/src/main/java/org/apache/flume/sink/hbase/AsyncHBaseSink.java +++ b/flume-ng-sinks/flume-ng-hbase-sink/src/main/java/org/apache/flume/sink/hbase/AsyncHBaseSink.java @@ -18,27 +18,23 @@ */ package org.apache.flume.sink.hbase; -import java.util.Arrays; -import java.util.Collection; -import java.util.Comparator; -import java.util.List; -import java.util.Map; -import java.util.concurrent.CountDownLatch; -import java.util.concurrent.ExecutorService; -import java.util.concurrent.Executors; -import java.util.concurrent.atomic.AtomicBoolean; - import com.google.common.annotations.VisibleForTesting; +import com.google.common.base.Charsets; +import com.google.common.base.Preconditions; +import com.google.common.base.Throwables; import com.google.common.collect.Maps; import com.google.common.primitives.UnsignedBytes; import com.google.common.util.concurrent.ThreadFactoryBuilder; +import com.stumbleupon.async.Callback; import org.apache.flume.Channel; +import org.apache.flume.ChannelException; import org.apache.flume.Context; import org.apache.flume.Event; import org.apache.flume.EventDeliveryException; import org.apache.flume.FlumeException; import org.apache.flume.Transaction; import org.apache.flume.conf.Configurable; +import org.apache.flume.instrumentation.SinkCounter; import org.apache.flume.sink.AbstractSink; import org.apache.hadoop.conf.Configuration; import org.apache.hadoop.hbase.HBaseConfiguration; @@ -47,68 +43,68 @@ import org.hbase.async.AtomicIncrementRequest; import org.hbase.async.HBaseClient; import org.hbase.async.PutRequest; -import org.jboss.netty.channel.socket.nio - .NioClientSocketChannelFactory; +import org.jboss.netty.channel.socket.nio.NioClientSocketChannelFactory; import org.slf4j.Logger; import org.slf4j.LoggerFactory; -import com.google.common.base.Charsets; -import com.google.common.base.Preconditions; -import com.google.common.base.Throwables; -import com.stumbleupon.async.Callback; +import java.util.Arrays; +import java.util.Collection; +import java.util.Comparator; +import java.util.List; +import java.util.Map; +import java.util.concurrent.CountDownLatch; +import java.util.concurrent.ExecutorService; +import java.util.concurrent.Executors; import java.util.concurrent.TimeUnit; +import java.util.concurrent.atomic.AtomicBoolean; import java.util.concurrent.atomic.AtomicInteger; import java.util.concurrent.locks.Condition; import java.util.concurrent.locks.Lock; import java.util.concurrent.locks.ReentrantLock; -import org.apache.flume.ChannelException; -import org.apache.flume.instrumentation.SinkCounter; /** -* -* A simple sink which reads events from a channel and writes them to HBase. -* This Sink uses an aysnchronous API internally and is likely to -* perform better. -* The Hbase configution is picked up from the first hbase-site.xml -* encountered in the classpath. This sink supports batch reading of -* events from the channel, and writing them to Hbase, to minimize the number -* of flushes on the hbase tables. To use this sink, it has to be configured -* with certain mandatory parameters:

    -* -* table: The name of the table in Hbase to write to.

    -* columnFamily: The column family in Hbase to write to.

    -* Other optional parameters are:

    -* serializer: A class implementing -* {@link AsyncHbaseEventSerializer}. -* An instance of -* this class will be used to serialize events which are written to hbase.

    -* serializer.*: Passed in the configure() method to -* serializer -* as an object of {@link org.apache.flume.Context}.

    -* batchSize: This is the batch size used by the client. This is the -* maximum number of events the sink will commit per transaction. The default -* batch size is 100 events. -*

    -* timeout: The length of time in milliseconds the sink waits for -* callbacks from hbase for all events in a transaction. -* If no timeout is specified, the sink will wait forever.

    -* -* Note: Hbase does not guarantee atomic commits on multiple -* rows. So if a subset of events in a batch are written to disk by Hbase and -* Hbase fails, the flume transaction is rolled back, causing flume to write -* all the events in the transaction all over again, which will cause -* duplicates. The serializer is expected to take care of the handling of -* duplicates etc. HBase also does not support batch increments, so if -* multiple increments are returned by the serializer, then HBase failure -* will cause them to be re-written, when HBase comes back up. -*/ + * A simple sink which reads events from a channel and writes them to HBase. + * This Sink uses an aysnchronous API internally and is likely to + * perform better. + * The Hbase configution is picked up from the first hbase-site.xml + * encountered in the classpath. This sink supports batch reading of + * events from the channel, and writing them to Hbase, to minimize the number + * of flushes on the hbase tables. To use this sink, it has to be configured + * with certain mandatory parameters:

    + *

    + * table: The name of the table in Hbase to write to.

    + * columnFamily: The column family in Hbase to write to.

    + * Other optional parameters are:

    + * serializer: A class implementing + * {@link AsyncHbaseEventSerializer}. + * An instance of + * this class will be used to serialize events which are written to hbase.

    + * serializer.*: Passed in the configure() method to + * serializer + * as an object of {@link org.apache.flume.Context}.

    + * batchSize: This is the batch size used by the client. This is the + * maximum number of events the sink will commit per transaction. The default + * batch size is 100 events. + *

    + * timeout: The length of time in milliseconds the sink waits for + * callbacks from hbase for all events in a transaction. + * If no timeout is specified, the sink will wait forever.

    + *

    + * Note: Hbase does not guarantee atomic commits on multiple + * rows. So if a subset of events in a batch are written to disk by Hbase and + * Hbase fails, the flume transaction is rolled back, causing flume to write + * all the events in the transaction all over again, which will cause + * duplicates. The serializer is expected to take care of the handling of + * duplicates etc. HBase also does not support batch increments, so if + * multiple increments are returned by the serializer, then HBase failure + * will cause them to be re-written, when HBase comes back up. + */ public class AsyncHBaseSink extends AbstractSink implements Configurable { private String tableName; private byte[] columnFamily; private long batchSize; - private static final Logger logger = - LoggerFactory.getLogger(AsyncHBaseSink.class); + private static final Logger logger = LoggerFactory.getLogger(AsyncHBaseSink.class); private AsyncHbaseEventSerializer serializer; private String eventSerializerType; private Context serializerContext; @@ -138,10 +134,9 @@ public class AsyncHBaseSink extends AbstractSink implements Configurable { // Does not need to be thread-safe. Always called only from the sink's // process method. - private final Comparator COMPARATOR = UnsignedBytes - .lexicographicalComparator(); + private final Comparator COMPARATOR = UnsignedBytes.lexicographicalComparator(); - public AsyncHBaseSink(){ + public AsyncHBaseSink() { this(null); } @@ -151,7 +146,7 @@ public AsyncHBaseSink(Configuration conf) { @VisibleForTesting AsyncHBaseSink(Configuration conf, boolean isTimeoutTest, - boolean isCoalesceTest) { + boolean isCoalesceTest) { this.conf = conf; this.isTimeoutTest = isTimeoutTest; this.isCoalesceTest = isCoalesceTest; @@ -189,17 +184,17 @@ public Status process() throws EventDeliveryException { * locks and conditions. */ Callback putSuccessCallback = - new SuccessCallback( + new SuccessCallback( lock, callbacksReceived, condition); Callback putFailureCallback = - new FailureCallback( + new FailureCallback( lock, callbacksReceived, txnFail, condition); Callback incrementSuccessCallback = - new SuccessCallback( + new SuccessCallback( lock, callbacksReceived, condition); Callback incrementFailureCallback = - new FailureCallback( + new FailureCallback( lock, callbacksReceived, txnFail, condition); Status status = Status.READY; @@ -235,9 +230,9 @@ public Status process() throws EventDeliveryException { for (AtomicIncrementRequest increment : increments) { if (batchIncrements) { CellIdentifier identifier = new CellIdentifier(increment.key(), - increment.qualifier()); + increment.qualifier()); AtomicIncrementRequest request - = incrementBuffer.get(identifier); + = incrementBuffer.get(identifier); if (request == null) { incrementBuffer.put(identifier, increment); } else { @@ -245,7 +240,7 @@ public Status process() throws EventDeliveryException { } } else { client.atomicIncrement(increment).addCallbacks( - incrementSuccessCallback, incrementFailureCallback); + incrementSuccessCallback, incrementFailureCallback); } } } @@ -254,7 +249,7 @@ public Status process() throws EventDeliveryException { Collection increments = incrementBuffer.values(); for (AtomicIncrementRequest increment : increments) { client.atomicIncrement(increment).addCallbacks( - incrementSuccessCallback, incrementFailureCallback); + incrementSuccessCallback, incrementFailureCallback); } callbacksExpected.addAndGet(increments.size()); } @@ -273,14 +268,14 @@ public Status process() throws EventDeliveryException { long timeRemaining; try { while ((callbacksReceived.get() < callbacksExpected.get()) - && !txnFail.get()) { + && !txnFail.get()) { timeRemaining = timeout - (System.nanoTime() - startTime); timeRemaining = (timeRemaining >= 0) ? timeRemaining : 0; try { if (!condition.await(timeRemaining, TimeUnit.NANOSECONDS)) { txnFail.set(true); logger.warn("HBase callbacks timed out. " - + "Transaction will be rolled back."); + + "Transaction will be rolled back."); } } catch (Exception ex) { logger.error("Exception while waiting for callbacks from HBase."); @@ -348,35 +343,35 @@ public void configure(Context context) { Preconditions.checkNotNull(cf, "Column family cannot be empty, please specify in configuration file"); //Check foe event serializer, if null set event serializer type - if(eventSerializerType == null || eventSerializerType.isEmpty()) { + if (eventSerializerType == null || eventSerializerType.isEmpty()) { eventSerializerType = "org.apache.flume.sink.hbase.SimpleAsyncHbaseEventSerializer"; logger.info("No serializer defined, Will use default"); } serializerContext.putAll(context.getSubProperties( - HBaseSinkConfigurationConstants.CONFIG_SERIALIZER_PREFIX)); + HBaseSinkConfigurationConstants.CONFIG_SERIALIZER_PREFIX)); columnFamily = cf.getBytes(Charsets.UTF_8); try { @SuppressWarnings("unchecked") Class clazz = - (Class) - Class.forName(eventSerializerType); + (Class) + Class.forName(eventSerializerType); serializer = clazz.newInstance(); serializer.configure(serializerContext); serializer.initialize(tableName.getBytes(Charsets.UTF_8), columnFamily); } catch (Exception e) { - logger.error("Could not instantiate event serializer." , e); + logger.error("Could not instantiate event serializer.", e); Throwables.propagate(e); } - if(sinkCounter == null) { + if (sinkCounter == null) { sinkCounter = new SinkCounter(this.getName()); } timeout = context.getLong(HBaseSinkConfigurationConstants.CONFIG_TIMEOUT, - HBaseSinkConfigurationConstants.DEFAULT_TIMEOUT); - if(timeout <= 0){ + HBaseSinkConfigurationConstants.DEFAULT_TIMEOUT); + if (timeout <= 0) { logger.warn("Timeout should be positive for Hbase sink. " - + "Sink will not timeout."); + + "Sink will not timeout."); timeout = HBaseSinkConfigurationConstants.DEFAULT_TIMEOUT; } //Convert to nanos. @@ -384,7 +379,7 @@ public void configure(Context context) { zkQuorum = context.getString( HBaseSinkConfigurationConstants.ZK_QUORUM, "").trim(); - if(!zkQuorum.isEmpty()) { + if (!zkQuorum.isEmpty()) { zkBaseDir = context.getString( HBaseSinkConfigurationConstants.ZK_ZNODE_PARENT, HBaseSinkConfigurationConstants.DEFAULT_ZK_ZNODE_PARENT); @@ -394,32 +389,33 @@ public void configure(Context context) { } zkQuorum = ZKConfig.getZKQuorumServersString(conf); zkBaseDir = conf.get(HConstants.ZOOKEEPER_ZNODE_PARENT, - HConstants.DEFAULT_ZOOKEEPER_ZNODE_PARENT); + HConstants.DEFAULT_ZOOKEEPER_ZNODE_PARENT); } Preconditions.checkState(zkQuorum != null && !zkQuorum.isEmpty(), "The Zookeeper quorum cannot be null and should be specified."); enableWal = context.getBoolean(HBaseSinkConfigurationConstants - .CONFIG_ENABLE_WAL, HBaseSinkConfigurationConstants.DEFAULT_ENABLE_WAL); + .CONFIG_ENABLE_WAL, HBaseSinkConfigurationConstants.DEFAULT_ENABLE_WAL); logger.info("The write to WAL option is set to: " + String.valueOf(enableWal)); - if(!enableWal) { + if (!enableWal) { logger.warn("AsyncHBaseSink's enableWal configuration is set to false. " + - "All writes to HBase will have WAL disabled, and any data in the " + - "memstore of this region in the Region Server could be lost!"); + "All writes to HBase will have WAL disabled, and any data in the " + + "memstore of this region in the Region Server could be lost!"); } batchIncrements = context.getBoolean( - HBaseSinkConfigurationConstants.CONFIG_COALESCE_INCREMENTS, - HBaseSinkConfigurationConstants.DEFAULT_COALESCE_INCREMENTS); + HBaseSinkConfigurationConstants.CONFIG_COALESCE_INCREMENTS, + HBaseSinkConfigurationConstants.DEFAULT_COALESCE_INCREMENTS); - if(batchIncrements) { + if (batchIncrements) { incrementBuffer = Maps.newHashMap(); logger.info("Increment coalescing is enabled. Increments will be " + - "buffered."); + "buffered."); } - maxConsecutiveFails = context.getInteger(HBaseSinkConfigurationConstants.CONFIG_MAX_CONSECUTIVE_FAILS, - HBaseSinkConfigurationConstants.DEFAULT_MAX_CONSECUTIVE_FAILS); + maxConsecutiveFails = + context.getInteger(HBaseSinkConfigurationConstants.CONFIG_MAX_CONSECUTIVE_FAILS, + HBaseSinkConfigurationConstants.DEFAULT_MAX_CONSECUTIVE_FAILS); } @@ -432,10 +428,11 @@ int getTotalCallbacksReceived() { boolean isConfNull() { return conf == null; } + @Override - public void start(){ + public void start() { Preconditions.checkArgument(client == null, "Please call stop " - + "before calling start on an old instance."); + + "before calling start on an old instance."); sinkCounter.start(); sinkCounter.incrementConnectionCreatedCount(); client = initHBaseClient(); @@ -446,31 +443,31 @@ private HBaseClient initHBaseClient() { logger.info("Initializing HBase Client"); sinkCallbackPool = Executors.newCachedThreadPool(new ThreadFactoryBuilder() - .setNameFormat(this.getName() + " HBase Call Pool").build()); + .setNameFormat(this.getName() + " HBase Call Pool").build()); logger.info("Callback pool created"); client = new HBaseClient(zkQuorum, zkBaseDir, - new NioClientSocketChannelFactory(sinkCallbackPool, sinkCallbackPool)); + new NioClientSocketChannelFactory(sinkCallbackPool, sinkCallbackPool)); final CountDownLatch latch = new CountDownLatch(1); final AtomicBoolean fail = new AtomicBoolean(false); client.ensureTableFamilyExists( - tableName.getBytes(Charsets.UTF_8), columnFamily).addCallbacks( - new Callback() { - @Override - public Object call(Object arg) throws Exception { - latch.countDown(); - logger.info("table found"); - return null; - } - }, - new Callback() { - @Override - public Object call(Object arg) throws Exception { - fail.set(true); - latch.countDown(); - return null; - } - }); + tableName.getBytes(Charsets.UTF_8), columnFamily).addCallbacks( + new Callback() { + @Override + public Object call(Object arg) throws Exception { + latch.countDown(); + logger.info("table found"); + return null; + } + }, + new Callback() { + @Override + public Object call(Object arg) throws Exception { + fail.set(true); + latch.countDown(); + return null; + } + }); try { logger.info("waiting on callback"); @@ -481,14 +478,14 @@ public Object call(Object arg) throws Exception { throw new FlumeException( "Interrupted while waiting for Hbase Callbacks", e); } - if(fail.get()){ + if (fail.get()) { sinkCounter.incrementConnectionFailedCount(); if (client != null) { shutdownHBaseClient(); } throw new FlumeException( "Could not start sink. " + - "Table or column family does not exist in Hbase."); + "Table or column family does not exist in Hbase."); } else { open = true; } @@ -497,7 +494,7 @@ public Object call(Object arg) throws Exception { } @Override - public void stop(){ + public void stop() { serializer.cleanUp(); if (client != null) { shutdownHBaseClient(); @@ -514,7 +511,7 @@ public void stop(){ } } catch (InterruptedException e) { logger.error("Interrupted while waiting for asynchbase sink pool to " + - "die", e); + "die", e); if (sinkCallbackPool != null) { sinkCallbackPool.shutdownNow(); } @@ -546,7 +543,7 @@ public Object call(Object arg) throws Exception { }); if (!waiter.await(timeout, TimeUnit.NANOSECONDS)) { logger.error("HBase connection could not be closed within timeout! HBase cluster might " + - "be down!"); + "be down!"); } } catch (Exception ex) { logger.warn("Error while attempting to close connections to HBase"); @@ -569,7 +566,7 @@ private void handleTransactionFailure(Transaction txn) } catch (Throwable e) { logger.error("Failed to commit transaction." + "Transaction rolled back.", e); - if(e instanceof Error || e instanceof RuntimeException){ + if (e instanceof Error || e instanceof RuntimeException) { logger.error("Failed to commit transaction." + "Transaction rolled back.", e); Throwables.propagate(e); @@ -583,14 +580,15 @@ private void handleTransactionFailure(Transaction txn) txn.close(); } } - private class SuccessCallback implements Callback { + + private class SuccessCallback implements Callback { private Lock lock; private AtomicInteger callbacksReceived; private Condition condition; private final boolean isTimeoutTesting; public SuccessCallback(Lock lck, AtomicInteger callbacksReceived, - Condition condition) { + Condition condition) { lock = lck; this.callbacksReceived = callbacksReceived; this.condition = condition; @@ -614,7 +612,7 @@ public R call(T arg) throws Exception { private void doCall() throws Exception { callbacksReceived.incrementAndGet(); lock.lock(); - try{ + try { condition.signal(); } finally { lock.unlock(); @@ -622,14 +620,15 @@ private void doCall() throws Exception { } } - private class FailureCallback implements Callback { + private class FailureCallback implements Callback { private Lock lock; private AtomicInteger callbacksReceived; private AtomicBoolean txnFail; private Condition condition; private final boolean isTimeoutTesting; + public FailureCallback(Lock lck, AtomicInteger callbacksReceived, - AtomicBoolean txnFail, Condition condition){ + AtomicBoolean txnFail, Condition condition) { this.lock = lck; this.callbacksReceived = callbacksReceived; this.txnFail = txnFail; @@ -665,7 +664,7 @@ private void doCall() throws Exception { } private void checkIfChannelExceptionAndThrow(Throwable e) - throws EventDeliveryException { + throws EventDeliveryException { if (e instanceof ChannelException) { throw new EventDeliveryException("Error in processing transaction.", e); } else if (e instanceof Error || e instanceof RuntimeException) { @@ -678,13 +677,14 @@ private class CellIdentifier { private final byte[] row; private final byte[] column; private final int hashCode; + // Since the sink operates only on one table and one cf, // we use the data from the owning sink public CellIdentifier(byte[] row, byte[] column) { this.row = row; this.column = column; this.hashCode = - (Arrays.hashCode(row) * 31) * (Arrays.hashCode(column) * 31); + (Arrays.hashCode(row) * 31) * (Arrays.hashCode(column) * 31); } @Override @@ -701,7 +701,7 @@ public boolean equals(Object other) { return false; } else { return (COMPARATOR.compare(row, o.row) == 0 - && COMPARATOR.compare(column, o.column) == 0); + && COMPARATOR.compare(column, o.column) == 0); } } } diff --git a/flume-ng-sinks/flume-ng-hbase-sink/src/main/java/org/apache/flume/sink/hbase/AsyncHbaseEventSerializer.java b/flume-ng-sinks/flume-ng-hbase-sink/src/main/java/org/apache/flume/sink/hbase/AsyncHbaseEventSerializer.java index 9ae6c289da..481fce8f89 100644 --- a/flume-ng-sinks/flume-ng-hbase-sink/src/main/java/org/apache/flume/sink/hbase/AsyncHbaseEventSerializer.java +++ b/flume-ng-sinks/flume-ng-hbase-sink/src/main/java/org/apache/flume/sink/hbase/AsyncHbaseEventSerializer.java @@ -34,8 +34,7 @@ * of this interface is expected by the {@linkplain AsyncHBaseSink} to serialize * the events. */ -public interface AsyncHbaseEventSerializer extends Configurable, -ConfigurableComponent { +public interface AsyncHbaseEventSerializer extends Configurable, ConfigurableComponent { /** * Initialize the event serializer. @@ -47,7 +46,7 @@ public interface AsyncHbaseEventSerializer extends Configurable, public void initialize(byte[] table, byte[] cf); /** - * @param Event to be written to HBase. + * @param event Event to be written to HBase */ public void setEvent(Event event); diff --git a/flume-ng-sinks/flume-ng-hbase-sink/src/main/java/org/apache/flume/sink/hbase/HBaseSink.java b/flume-ng-sinks/flume-ng-hbase-sink/src/main/java/org/apache/flume/sink/hbase/HBaseSink.java index e659ada928..4c8b52bf35 100644 --- a/flume-ng-sinks/flume-ng-hbase-sink/src/main/java/org/apache/flume/sink/hbase/HBaseSink.java +++ b/flume-ng-sinks/flume-ng-hbase-sink/src/main/java/org/apache/flume/sink/hbase/HBaseSink.java @@ -18,15 +18,10 @@ */ package org.apache.flume.sink.hbase; -import java.io.IOException; -import java.lang.reflect.InvocationTargetException; -import java.lang.reflect.Method; -import java.util.LinkedList; -import java.util.List; -import java.util.Map; -import java.util.NavigableMap; - import com.google.common.annotations.VisibleForTesting; +import com.google.common.base.Charsets; +import com.google.common.base.Preconditions; +import com.google.common.base.Throwables; import com.google.common.collect.Lists; import com.google.common.collect.Maps; import org.apache.flume.Channel; @@ -52,14 +47,16 @@ import org.slf4j.Logger; import org.slf4j.LoggerFactory; -import com.google.common.base.Charsets; -import com.google.common.base.Preconditions; -import com.google.common.base.Throwables; +import java.io.IOException; +import java.lang.reflect.InvocationTargetException; +import java.lang.reflect.Method; import java.security.PrivilegedExceptionAction; - +import java.util.LinkedList; +import java.util.List; +import java.util.Map; +import java.util.NavigableMap; /** - * * A simple sink which reads events from a channel and writes them to HBase. * The Hbase configuration is picked up from the first hbase-site.xml * encountered in the classpath. This sink supports batch reading of @@ -73,7 +70,7 @@ * batch size, whichever comes first.

    * Other optional parameters are:

    * serializer: A class implementing {@link HbaseEventSerializer}. - * An instance of + * An instance of * this class will be used to write out events to hbase.

    * serializer.*: Passed in the configure() method to serializer * as an object of {@link org.apache.flume.Context}.

    @@ -81,7 +78,7 @@ * maximum number of events the sink will commit per transaction. The default * batch size is 100 events. *

    - * + *

    * Note: While this sink flushes all events in a transaction * to HBase in one shot, Hbase does not guarantee atomic commits on multiple * rows. So if a subset of events in a batch are written to disk by Hbase and @@ -113,11 +110,11 @@ public class HBaseSink extends AbstractSink implements Configurable { // Internal hooks used for unit testing. private DebugIncrementsCallback debugIncrCallback = null; - public HBaseSink(){ + public HBaseSink() { this(HBaseConfiguration.create()); } - public HBaseSink(Configuration conf){ + public HBaseSink(Configuration conf) { this.config = conf; } @@ -129,15 +126,16 @@ public HBaseSink(Configuration conf){ } @Override - public void start(){ + public void start() { Preconditions.checkArgument(table == null, "Please call stop " + "before calling start on an old instance."); try { - privilegedExecutor = FlumeAuthenticationUtil.getAuthenticator(kerberosPrincipal, kerberosKeytab); + privilegedExecutor = + FlumeAuthenticationUtil.getAuthenticator(kerberosPrincipal, kerberosKeytab); } catch (Exception ex) { sinkCounter.incrementConnectionFailedCount(); throw new FlumeException("Failed to login to HBase using " - + "provided credentials.", ex); + + "provided credentials.", ex); } try { table = privilegedExecutor.execute(new PrivilegedExceptionAction() { @@ -165,16 +163,16 @@ public Boolean run() throws IOException { } })) { throw new IOException("Table " + tableName - + " has no such column family " + Bytes.toString(columnFamily)); + + " has no such column family " + Bytes.toString(columnFamily)); } } catch (Exception e) { //Get getTableDescriptor also throws IOException, so catch the IOException //thrown above or by the getTableDescriptor() call. sinkCounter.incrementConnectionFailedCount(); throw new FlumeException("Error getting column family from HBase." - + "Please verify that the table " + tableName + " and Column Family, " - + Bytes.toString(columnFamily) + " exists in HBase, and the" - + " current user has permissions to access that table.", e); + + "Please verify that the table " + tableName + " and Column Family, " + + Bytes.toString(columnFamily) + " exists in HBase, and the" + + " current user has permissions to access that table.", e); } super.start(); @@ -183,7 +181,7 @@ public Boolean run() throws IOException { } @Override - public void stop(){ + public void stop() { try { if (table != null) { table.close(); @@ -198,7 +196,7 @@ public void stop(){ @SuppressWarnings("unchecked") @Override - public void configure(Context context){ + public void configure(Context context) { tableName = context.getString(HBaseSinkConfigurationConstants.CONFIG_TABLE); String cf = context.getString( HBaseSinkConfigurationConstants.CONFIG_COLUMN_FAMILY); @@ -213,48 +211,48 @@ public void configure(Context context){ Preconditions.checkNotNull(cf, "Column family cannot be empty, please specify in configuration file"); //Check foe event serializer, if null set event serializer type - if(eventSerializerType == null || eventSerializerType.isEmpty()) { + if (eventSerializerType == null || eventSerializerType.isEmpty()) { eventSerializerType = "org.apache.flume.sink.hbase.SimpleHbaseEventSerializer"; logger.info("No serializer defined, Will use default"); } serializerContext.putAll(context.getSubProperties( - HBaseSinkConfigurationConstants.CONFIG_SERIALIZER_PREFIX)); + HBaseSinkConfigurationConstants.CONFIG_SERIALIZER_PREFIX)); columnFamily = cf.getBytes(Charsets.UTF_8); try { Class clazz = (Class) - Class.forName(eventSerializerType); + Class.forName(eventSerializerType); serializer = clazz.newInstance(); serializer.configure(serializerContext); } catch (Exception e) { - logger.error("Could not instantiate event serializer." , e); + logger.error("Could not instantiate event serializer.", e); Throwables.propagate(e); } kerberosKeytab = context.getString(HBaseSinkConfigurationConstants.CONFIG_KEYTAB); kerberosPrincipal = context.getString(HBaseSinkConfigurationConstants.CONFIG_PRINCIPAL); enableWal = context.getBoolean(HBaseSinkConfigurationConstants - .CONFIG_ENABLE_WAL, HBaseSinkConfigurationConstants.DEFAULT_ENABLE_WAL); + .CONFIG_ENABLE_WAL, HBaseSinkConfigurationConstants.DEFAULT_ENABLE_WAL); logger.info("The write to WAL option is set to: " + String.valueOf(enableWal)); - if(!enableWal) { + if (!enableWal) { logger.warn("HBase Sink's enableWal configuration is set to false. All " + - "writes to HBase will have WAL disabled, and any data in the " + - "memstore of this region in the Region Server could be lost!"); + "writes to HBase will have WAL disabled, and any data in the " + + "memstore of this region in the Region Server could be lost!"); } batchIncrements = context.getBoolean( - HBaseSinkConfigurationConstants.CONFIG_COALESCE_INCREMENTS, - HBaseSinkConfigurationConstants.DEFAULT_COALESCE_INCREMENTS); + HBaseSinkConfigurationConstants.CONFIG_COALESCE_INCREMENTS, + HBaseSinkConfigurationConstants.DEFAULT_COALESCE_INCREMENTS); if (batchIncrements) { logger.info("Increment coalescing is enabled. Increments will be " + - "buffered."); + "buffered."); refGetFamilyMap = reflectLookupGetFamilyMap(); } String zkQuorum = context.getString(HBaseSinkConfigurationConstants - .ZK_QUORUM); + .ZK_QUORUM); Integer port = null; /** * HBase allows multiple nodes in the quorum, but all need to use the @@ -267,10 +265,10 @@ public void configure(Context context){ logger.info("Using ZK Quorum: " + zkQuorum); String[] zkHosts = zkQuorum.split(","); int length = zkHosts.length; - for(int i = 0; i < length; i++) { + for (int i = 0; i < length; i++) { String[] zkHostAndPort = zkHosts[i].split(":"); zkBuilder.append(zkHostAndPort[0].trim()); - if(i != length-1) { + if (i != length - 1) { zkBuilder.append(","); } else { zkQuorum = zkBuilder.toString(); @@ -282,18 +280,18 @@ public void configure(Context context){ port = Integer.parseInt(zkHostAndPort[1].trim()); } else if (!port.equals(Integer.parseInt(zkHostAndPort[1].trim()))) { throw new FlumeException("All Zookeeper nodes in the quorum must " + - "use the same client port."); + "use the same client port."); } } - if(port == null) { + if (port == null) { port = HConstants.DEFAULT_ZOOKEPER_CLIENT_PORT; } this.config.set(HConstants.ZOOKEEPER_QUORUM, zkQuorum); this.config.setInt(HConstants.ZOOKEEPER_CLIENT_PORT, port); } String hbaseZnode = context.getString( - HBaseSinkConfigurationConstants.ZK_ZNODE_PARENT); - if(hbaseZnode != null && !hbaseZnode.isEmpty()) { + HBaseSinkConfigurationConstants.ZK_ZNODE_PARENT); + if (hbaseZnode != null && !hbaseZnode.isEmpty()) { this.config.set(HConstants.ZOOKEEPER_ZNODE_PARENT, hbaseZnode); } sinkCounter = new SinkCounter(this.getName()); @@ -314,7 +312,7 @@ public Status process() throws EventDeliveryException { txn.begin(); if (serializer instanceof BatchAware) { - ((BatchAware)serializer).onBatchStart(); + ((BatchAware) serializer).onBatchStart(); } long i = 0; @@ -342,15 +340,15 @@ public Status process() throws EventDeliveryException { putEventsAndCommit(actions, incs, txn); } catch (Throwable e) { - try{ + try { txn.rollback(); } catch (Exception e2) { logger.error("Exception in rollback. Rollback might not have been " + - "successful." , e2); + "successful.", e2); } logger.error("Failed to commit transaction." + "Transaction rolled back.", e); - if(e instanceof Error || e instanceof RuntimeException){ + if (e instanceof Error || e instanceof RuntimeException) { logger.error("Failed to commit transaction." + "Transaction rolled back.", e); Throwables.propagate(e); @@ -367,7 +365,7 @@ public Status process() throws EventDeliveryException { } private void putEventsAndCommit(final List actions, - final List incs, Transaction txn) throws Exception { + final List incs, Transaction txn) throws Exception { privilegedExecutor.execute(new PrivilegedExceptionAction() { @Override @@ -421,7 +419,7 @@ public Void run() throws Exception { @VisibleForTesting static Method reflectLookupGetFamilyMap() { Method m = null; - String[] methodNames = { "getFamilyMapOfLongs", "getFamilyMap" }; + String[] methodNames = {"getFamilyMapOfLongs", "getFamilyMap"}; for (String methodName : methodNames) { try { m = Increment.class.getMethod(methodName); @@ -447,7 +445,7 @@ static Method reflectLookupGetFamilyMap() { @SuppressWarnings("unchecked") private Map> getFamilyMap(Increment inc) { Preconditions.checkNotNull(refGetFamilyMap, - "Increment.getFamilymap() not found"); + "Increment.getFamilymap() not found"); Preconditions.checkNotNull(inc, "Increment required"); Map> familyMap = null; try { @@ -466,6 +464,7 @@ private Map> getFamilyMap(Increment inc) { /** * Perform "compression" on the given set of increments so that Flume sends * the minimum possible number of RPC operations to HBase per batch. + * * @param incs Input: Increment objects to coalesce. * @return List of new Increment objects after coalescing the unique counts. */ @@ -478,7 +477,7 @@ private List coalesceIncrements(Iterable incs) { for (Increment inc : incs) { byte[] row = inc.getRow(); Map> families = getFamilyMap(inc); - for (Map.Entry> familyEntry : families.entrySet()) { + for (Map.Entry> familyEntry : families.entrySet()) { byte[] family = familyEntry.getKey(); NavigableMap qualifiers = familyEntry.getValue(); for (Map.Entry qualifierEntry : qualifiers.entrySet()) { @@ -491,9 +490,10 @@ private List coalesceIncrements(Iterable incs) { // Reconstruct list of Increments per unique row/family/qualifier. List coalesced = Lists.newLinkedList(); - for (Map.Entry>> rowEntry : counters.entrySet()) { + for (Map.Entry>> rowEntry : + counters.entrySet()) { byte[] row = rowEntry.getKey(); - Map > families = rowEntry.getValue(); + Map> families = rowEntry.getValue(); Increment inc = new Increment(row); for (Map.Entry> familyEntry : families.entrySet()) { byte[] family = familyEntry.getKey(); @@ -513,11 +513,12 @@ private List coalesceIncrements(Iterable incs) { /** * Helper function for {@link #coalesceIncrements} to increment a counter * value in the passed data structure. - * @param counters Nested data structure containing the counters. - * @param row Row key to increment. - * @param family Column family to increment. + * + * @param counters Nested data structure containing the counters. + * @param row Row key to increment. + * @param family Column family to increment. * @param qualifier Column qualifier to increment. - * @param count Amount to increment by. + * @param count Amount to increment by. */ private void incrementCounter( Map>> counters, diff --git a/flume-ng-sinks/flume-ng-hbase-sink/src/main/java/org/apache/flume/sink/hbase/HbaseEventSerializer.java b/flume-ng-sinks/flume-ng-hbase-sink/src/main/java/org/apache/flume/sink/hbase/HbaseEventSerializer.java index 2c0f0e6c5a..d4e3f84937 100644 --- a/flume-ng-sinks/flume-ng-hbase-sink/src/main/java/org/apache/flume/sink/hbase/HbaseEventSerializer.java +++ b/flume-ng-sinks/flume-ng-hbase-sink/src/main/java/org/apache/flume/sink/hbase/HbaseEventSerializer.java @@ -32,13 +32,12 @@ * params required should be taken through this. Only the column family is * passed in. The columns should exist in the table and column family * specified in the configuration for the HbaseSink. - * */ -public interface HbaseEventSerializer extends Configurable, - ConfigurableComponent { +public interface HbaseEventSerializer extends Configurable, ConfigurableComponent { /** * Initialize the event serializer. - * @param Event to be written to HBase. + * @param event Event to be written to HBase + * @param columnFamily Column family to write to */ public void initialize(Event event, byte[] columnFamily); @@ -54,10 +53,9 @@ public interface HbaseEventSerializer extends Configurable, public List getActions(); public List getIncrements(); + /* * Clean up any state. This will be called when the sink is being stopped. */ public void close(); - - } diff --git a/flume-ng-sinks/flume-ng-hbase-sink/src/main/java/org/apache/flume/sink/hbase/RegexHbaseEventSerializer.java b/flume-ng-sinks/flume-ng-hbase-sink/src/main/java/org/apache/flume/sink/hbase/RegexHbaseEventSerializer.java index 7d2b8b7fb8..8342d67ebd 100644 --- a/flume-ng-sinks/flume-ng-hbase-sink/src/main/java/org/apache/flume/sink/hbase/RegexHbaseEventSerializer.java +++ b/flume-ng-sinks/flume-ng-hbase-sink/src/main/java/org/apache/flume/sink/hbase/RegexHbaseEventSerializer.java @@ -18,14 +18,8 @@ */ package org.apache.flume.sink.hbase; -import java.nio.charset.Charset; -import java.util.Calendar; -import java.util.List; -import java.util.Map; -import java.util.concurrent.atomic.AtomicInteger; -import java.util.regex.Matcher; -import java.util.regex.Pattern; - +import com.google.common.base.Charsets; +import com.google.common.collect.Lists; import org.apache.commons.lang.RandomStringUtils; import org.apache.flume.Context; import org.apache.flume.Event; @@ -35,20 +29,25 @@ import org.apache.hadoop.hbase.client.Put; import org.apache.hadoop.hbase.client.Row; -import com.google.common.base.Charsets; -import com.google.common.collect.Lists; +import java.nio.charset.Charset; +import java.util.Calendar; +import java.util.List; +import java.util.Map; +import java.util.concurrent.atomic.AtomicInteger; +import java.util.regex.Matcher; +import java.util.regex.Pattern; /** - * An {@link HbaseEventSerializer} which parses columns based on a supplied - * regular expression and column name list. - * + * An {@link HbaseEventSerializer} which parses columns based on a supplied + * regular expression and column name list. + *

    * Note that if the regular expression does not return the correct number of * groups for a particular event, or it does not correctly match an event, * the event is silently dropped. - * + *

    * Row keys for each event consist of a timestamp concatenated with an * identifier which enforces uniqueness of keys across flume agents. - * + *

    * See static constant variables for configuration options. */ public class RegexHbaseEventSerializer implements HbaseEventSerializer { @@ -108,21 +107,21 @@ public void configure(Context context) { String colNameStr = context.getString(COL_NAME_CONFIG, COLUMN_NAME_DEFAULT); String[] columnNames = colNameStr.split(","); - for (String s: columnNames) { + for (String s : columnNames) { colNames.add(s.getBytes(charset)); } //Rowkey is optional, default is -1 rowKeyIndex = context.getInteger(ROW_KEY_INDEX_CONFIG, -1); //if row key is being used, make sure it is specified correct - if(rowKeyIndex >=0){ - if(rowKeyIndex >= columnNames.length) { + if (rowKeyIndex >= 0) { + if (rowKeyIndex >= columnNames.length) { throw new IllegalArgumentException(ROW_KEY_INDEX_CONFIG + " must be " + - "less than num columns " + columnNames.length); + "less than num columns " + columnNames.length); } - if(!ROW_KEY_NAME.equalsIgnoreCase(columnNames[rowKeyIndex])) { + if (!ROW_KEY_NAME.equalsIgnoreCase(columnNames[rowKeyIndex])) { throw new IllegalArgumentException("Column at " + rowKeyIndex + " must be " - + ROW_KEY_NAME + " and is " + columnNames[rowKeyIndex]); + + ROW_KEY_NAME + " and is " + columnNames[rowKeyIndex]); } } } @@ -181,15 +180,15 @@ public List getActions() throws FlumeException { } try { - if(rowKeyIndex < 0){ + if (rowKeyIndex < 0) { rowKey = getRowKey(); - }else{ + } else { rowKey = m.group(rowKeyIndex + 1).getBytes(Charsets.UTF_8); } Put put = new Put(rowKey); for (int i = 0; i < colNames.size(); i++) { - if(i != rowKeyIndex) { + if (i != rowKeyIndex) { put.add(cf, colNames.get(i), m.group(i + 1).getBytes(Charsets.UTF_8)); } } @@ -211,5 +210,6 @@ public List getIncrements() { } @Override - public void close() { } + public void close() { + } } \ No newline at end of file diff --git a/flume-ng-sinks/flume-ng-hbase-sink/src/main/java/org/apache/flume/sink/hbase/SimpleAsyncHbaseEventSerializer.java b/flume-ng-sinks/flume-ng-hbase-sink/src/main/java/org/apache/flume/sink/hbase/SimpleAsyncHbaseEventSerializer.java index 96095d11a8..3f442e8409 100644 --- a/flume-ng-sinks/flume-ng-hbase-sink/src/main/java/org/apache/flume/sink/hbase/SimpleAsyncHbaseEventSerializer.java +++ b/flume-ng-sinks/flume-ng-hbase-sink/src/main/java/org/apache/flume/sink/hbase/SimpleAsyncHbaseEventSerializer.java @@ -18,18 +18,17 @@ */ package org.apache.flume.sink.hbase; -import java.util.ArrayList; -import java.util.List; - +import com.google.common.base.Charsets; import org.apache.flume.Context; import org.apache.flume.Event; import org.apache.flume.FlumeException; -import org.hbase.async.AtomicIncrementRequest; -import org.hbase.async.PutRequest; import org.apache.flume.conf.ComponentConfiguration; import org.apache.flume.sink.hbase.SimpleHbaseEventSerializer.KeyType; +import org.hbase.async.AtomicIncrementRequest; +import org.hbase.async.PutRequest; -import com.google.common.base.Charsets; +import java.util.ArrayList; +import java.util.List; /** * A simple serializer to be used with the AsyncHBaseSink @@ -69,7 +68,7 @@ public void initialize(byte[] table, byte[] cf) { @Override public List getActions() { List actions = new ArrayList(); - if(payloadColumn != null){ + if (payloadColumn != null) { byte[] rowKey; try { switch (keyType) { @@ -89,17 +88,16 @@ public List getActions() { PutRequest putRequest = new PutRequest(table, rowKey, cf, payloadColumn, payload); actions.add(putRequest); - } catch (Exception e){ + } catch (Exception e) { throw new FlumeException("Could not get row key!", e); } } return actions; } - public List getIncrements(){ - List actions = new - ArrayList(); - if(incrementColumn != null) { + public List getIncrements() { + List actions = new ArrayList(); + if (incrementColumn != null) { AtomicIncrementRequest inc = new AtomicIncrementRequest(table, incrementRow, cf, incrementColumn); actions.add(inc); @@ -119,23 +117,22 @@ public void configure(Context context) { String iCol = context.getString("incrementColumn", "iCol"); rowPrefix = context.getString("rowPrefix", "default"); String suffix = context.getString("suffix", "uuid"); - if(pCol != null && !pCol.isEmpty()) { - if(suffix.equals("timestamp")){ + if (pCol != null && !pCol.isEmpty()) { + if (suffix.equals("timestamp")) { keyType = KeyType.TS; } else if (suffix.equals("random")) { keyType = KeyType.RANDOM; - } else if(suffix.equals("nano")){ + } else if (suffix.equals("nano")) { keyType = KeyType.TSNANO; } else { keyType = KeyType.UUID; } payloadColumn = pCol.getBytes(Charsets.UTF_8); } - if(iCol != null && !iCol.isEmpty()) { + if (iCol != null && !iCol.isEmpty()) { incrementColumn = iCol.getBytes(Charsets.UTF_8); } - incrementRow = - context.getString("incrementRow", "incRow").getBytes(Charsets.UTF_8); + incrementRow = context.getString("incrementRow", "incRow").getBytes(Charsets.UTF_8); } @Override diff --git a/flume-ng-sinks/flume-ng-hbase-sink/src/main/java/org/apache/flume/sink/hbase/SimpleHbaseEventSerializer.java b/flume-ng-sinks/flume-ng-hbase-sink/src/main/java/org/apache/flume/sink/hbase/SimpleHbaseEventSerializer.java index 758252b843..dc89fd725a 100644 --- a/flume-ng-sinks/flume-ng-hbase-sink/src/main/java/org/apache/flume/sink/hbase/SimpleHbaseEventSerializer.java +++ b/flume-ng-sinks/flume-ng-hbase-sink/src/main/java/org/apache/flume/sink/hbase/SimpleHbaseEventSerializer.java @@ -19,9 +19,7 @@ package org.apache.flume.sink.hbase; -import java.util.LinkedList; -import java.util.List; - +import com.google.common.base.Charsets; import org.apache.flume.Context; import org.apache.flume.Event; import org.apache.flume.FlumeException; @@ -30,19 +28,18 @@ import org.apache.hadoop.hbase.client.Put; import org.apache.hadoop.hbase.client.Row; -import com.google.common.base.Charsets; +import java.util.LinkedList; +import java.util.List; /** * A simple serializer that returns puts from an event, by writing the event * body into it. The headers are discarded. It also updates a row in hbase * which acts as an event counter. - * - * Takes optional parameters:

    + *

    Takes optional parameters:

    * rowPrefix: The prefix to be used. Default: default

    * incrementRow The row to increment. Default: incRow

    * suffix: uuid/random/timestamp.Default: uuid

    - * - * Mandatory parameters:

    + *

    Mandatory parameters:

    * cf:Column family.

    * Components that have no defaults and will not be used if null: * payloadColumn: Which column to put payload in. If it is null, @@ -59,8 +56,7 @@ public class SimpleHbaseEventSerializer implements HbaseEventSerializer { private KeyType keyType; private byte[] payload; - public SimpleHbaseEventSerializer(){ - + public SimpleHbaseEventSerializer() { } @Override @@ -70,21 +66,21 @@ public void configure(Context context) { context.getString("incrementRow", "incRow").getBytes(Charsets.UTF_8); String suffix = context.getString("suffix", "uuid"); - String payloadColumn = context.getString("payloadColumn","pCol"); - String incColumn = context.getString("incrementColumn","iCol"); - if(payloadColumn != null && !payloadColumn.isEmpty()) { - if(suffix.equals("timestamp")){ + String payloadColumn = context.getString("payloadColumn", "pCol"); + String incColumn = context.getString("incrementColumn", "iCol"); + if (payloadColumn != null && !payloadColumn.isEmpty()) { + if (suffix.equals("timestamp")) { keyType = KeyType.TS; } else if (suffix.equals("random")) { keyType = KeyType.RANDOM; - } else if(suffix.equals("nano")){ + } else if (suffix.equals("nano")) { keyType = KeyType.TSNANO; } else { keyType = KeyType.UUID; } plCol = payloadColumn.getBytes(Charsets.UTF_8); } - if(incColumn != null && !incColumn.isEmpty()) { + if (incColumn != null && !incColumn.isEmpty()) { incCol = incColumn.getBytes(Charsets.UTF_8); } } @@ -102,14 +98,14 @@ public void initialize(Event event, byte[] cf) { @Override public List getActions() throws FlumeException { List actions = new LinkedList(); - if(plCol != null){ + if (plCol != null) { byte[] rowKey; try { if (keyType == KeyType.TS) { rowKey = SimpleRowKeyGenerator.getTimestampKey(rowPrefix); - } else if(keyType == KeyType.RANDOM) { + } else if (keyType == KeyType.RANDOM) { rowKey = SimpleRowKeyGenerator.getRandomKey(rowPrefix); - } else if(keyType == KeyType.TSNANO) { + } else if (keyType == KeyType.TSNANO) { rowKey = SimpleRowKeyGenerator.getNanoTimestampKey(rowPrefix); } else { rowKey = SimpleRowKeyGenerator.getUUIDKey(rowPrefix); @@ -117,33 +113,34 @@ public List getActions() throws FlumeException { Put put = new Put(rowKey); put.add(cf, plCol, payload); actions.add(put); - } catch (Exception e){ + } catch (Exception e) { throw new FlumeException("Could not get row key!", e); } } return actions; } - @Override - public List getIncrements(){ - List incs = new LinkedList(); - if(incCol != null) { - Increment inc = new Increment(incrementRow); - inc.addColumn(cf, incCol, 1); - incs.add(inc); - } - return incs; - } - @Override - public void close() { + @Override + public List getIncrements() { + List incs = new LinkedList(); + if (incCol != null) { + Increment inc = new Increment(incrementRow); + inc.addColumn(cf, incCol, 1); + incs.add(inc); } + return incs; + } - public enum KeyType{ - UUID, - RANDOM, - TS, - TSNANO; - } + @Override + public void close() { + } + public enum KeyType { + UUID, + RANDOM, + TS, + TSNANO; } + +} diff --git a/flume-ng-sinks/flume-ng-hbase-sink/src/main/java/org/apache/flume/sink/hbase/SimpleRowKeyGenerator.java b/flume-ng-sinks/flume-ng-hbase-sink/src/main/java/org/apache/flume/sink/hbase/SimpleRowKeyGenerator.java index b25eb6aeef..2d654f274f 100644 --- a/flume-ng-sinks/flume-ng-hbase-sink/src/main/java/org/apache/flume/sink/hbase/SimpleRowKeyGenerator.java +++ b/flume-ng-sinks/flume-ng-hbase-sink/src/main/java/org/apache/flume/sink/hbase/SimpleRowKeyGenerator.java @@ -25,28 +25,22 @@ /** * Utility class for users to generate their own keys. Any key can be used, * this is just a utility that provides a set of simple keys. - * - * */ public class SimpleRowKeyGenerator { - public static byte[] getUUIDKey(String prefix) - throws UnsupportedEncodingException{ + public static byte[] getUUIDKey(String prefix) throws UnsupportedEncodingException { return (prefix + UUID.randomUUID().toString()).getBytes("UTF8"); } - public static byte[] getRandomKey(String prefix) - throws UnsupportedEncodingException{ + public static byte[] getRandomKey(String prefix) throws UnsupportedEncodingException { return (prefix + String.valueOf(new Random().nextLong())).getBytes("UTF8"); } - public static byte[] getTimestampKey(String prefix) - throws UnsupportedEncodingException { - return (prefix + String.valueOf( - System.currentTimeMillis())).getBytes("UTF8"); + + public static byte[] getTimestampKey(String prefix) throws UnsupportedEncodingException { + return (prefix + String.valueOf(System.currentTimeMillis())).getBytes("UTF8"); } - public static byte[] getNanoTimestampKey(String prefix) - throws UnsupportedEncodingException{ - return (prefix + String.valueOf( - System.nanoTime())).getBytes("UTF8"); + + public static byte[] getNanoTimestampKey(String prefix) throws UnsupportedEncodingException { + return (prefix + String.valueOf(System.nanoTime())).getBytes("UTF8"); } } diff --git a/flume-ng-sinks/flume-ng-kafka-sink/src/main/java/org/apache/flume/sink/kafka/KafkaSink.java b/flume-ng-sinks/flume-ng-kafka-sink/src/main/java/org/apache/flume/sink/kafka/KafkaSink.java index 7bef7f3bcb..9453546ba0 100644 --- a/flume-ng-sinks/flume-ng-kafka-sink/src/main/java/org/apache/flume/sink/kafka/KafkaSink.java +++ b/flume-ng-sinks/flume-ng-kafka-sink/src/main/java/org/apache/flume/sink/kafka/KafkaSink.java @@ -157,7 +157,7 @@ public Status process() throws EventDeliveryException { if (event == null) { // no events available in channel - if(processedEvents == 0) { + if (processedEvents == 0) { result = Status.BACKOFF; counter.incrementBatchEmptyCount(); } else { @@ -177,7 +177,7 @@ public Status process() throws EventDeliveryException { if (logger.isDebugEnabled()) { logger.debug("{Event} " + eventTopic + " : " + eventKey + " : " - + new String(eventBody, "UTF-8")); + + new String(eventBody, "UTF-8")); logger.debug("event #{}", processedEvents); } @@ -185,8 +185,10 @@ public Status process() throws EventDeliveryException { long startTime = System.currentTimeMillis(); try { - kafkaFutures.add(producer.send(new ProducerRecord (eventTopic, eventKey, serializeEvent(event, useAvroEventFormat)), - new SinkCallback(startTime))); + kafkaFutures.add(producer.send( + new ProducerRecord(eventTopic, eventKey, + serializeEvent(event, useAvroEventFormat)), + new SinkCallback(startTime))); } catch (IOException ex) { throw new EventDeliveryException("Could not serialize event", ex); } @@ -197,11 +199,11 @@ public Status process() throws EventDeliveryException { // publish batch and commit. if (processedEvents > 0) { - for (Future future : kafkaFutures) { - future.get(); - } + for (Future future : kafkaFutures) { + future.get(); + } long endTime = System.nanoTime(); - counter.addToKafkaEventSendTimer((endTime-batchStartTime)/(1000*1000)); + counter.addToKafkaEventSendTimer((endTime - batchStartTime) / (1000 * 1000)); counter.addToEventDrainSuccessCount(Long.valueOf(kafkaFutures.size())); } @@ -270,8 +272,7 @@ public void configure(Context context) { if (topicStr == null || topicStr.isEmpty()) { topicStr = DEFAULT_TOPIC; logger.warn("Topic was not specified. Using {} as the topic.", topicStr); - } - else { + } else { logger.info("Using the static topic {}. This may be overridden by event headers", topicStr); } @@ -283,7 +284,8 @@ public void configure(Context context) { logger.debug("Using batch size: {}", batchSize); } - useAvroEventFormat = context.getBoolean(KafkaSinkConstants.AVRO_EVENT, KafkaSinkConstants.DEFAULT_AVRO_EVENT); + useAvroEventFormat = context.getBoolean(KafkaSinkConstants.AVRO_EVENT, + KafkaSinkConstants.DEFAULT_AVRO_EVENT); if (logger.isDebugEnabled()) { logger.debug(KafkaSinkConstants.AVRO_EVENT + " set to: {}", useAvroEventFormat); @@ -322,7 +324,8 @@ private void translateOldProps(Context ctx) { throw new ConfigurationException("Bootstrap Servers must be specified"); } else { ctx.put(BOOTSTRAP_SERVERS_CONFIG, brokerList); - logger.warn("{} is deprecated. Please use the parameter {}", BROKER_LIST_FLUME_KEY, BOOTSTRAP_SERVERS_CONFIG); + logger.warn("{} is deprecated. Please use the parameter {}", + BROKER_LIST_FLUME_KEY, BOOTSTRAP_SERVERS_CONFIG); } } @@ -348,21 +351,18 @@ private void translateOldProps(Context ctx) { if (ctx.containsKey(KEY_SERIALIZER_KEY )) { logger.warn("{} is deprecated. Flume now uses the latest Kafka producer which implements " + - "a different interface for serializers. Please use the parameter {}", - KEY_SERIALIZER_KEY,KAFKA_PRODUCER_PREFIX + ProducerConfig.KEY_SERIALIZER_CLASS_CONFIG); + "a different interface for serializers. Please use the parameter {}", + KEY_SERIALIZER_KEY,KAFKA_PRODUCER_PREFIX + ProducerConfig.KEY_SERIALIZER_CLASS_CONFIG); } if (ctx.containsKey(MESSAGE_SERIALIZER_KEY)) { logger.warn("{} is deprecated. Flume now uses the latest Kafka producer which implements " + - "a different interface for serializers. Please use the parameter {}", - MESSAGE_SERIALIZER_KEY,KAFKA_PRODUCER_PREFIX + ProducerConfig.VALUE_SERIALIZER_CLASS_CONFIG); + "a different interface for serializers. Please use the parameter {}", + MESSAGE_SERIALIZER_KEY, + KAFKA_PRODUCER_PREFIX + ProducerConfig.VALUE_SERIALIZER_CLASS_CONFIG); } - - - - - } + private void setProducerProps(Context context, String bootStrapServers) { kafkaProps.put(ProducerConfig.ACKS_CONFIG, DEFAULT_ACKS); //Defaults overridden based on config @@ -387,7 +387,8 @@ private byte[] serializeEvent(Event event, boolean useAvroEventFormat) throws IO writer = Optional.of(new SpecificDatumWriter(AvroFlumeEvent.class)); } tempOutStream.get().reset(); - AvroFlumeEvent e = new AvroFlumeEvent(toCharSeqMap(event.getHeaders()), ByteBuffer.wrap(event.getBody())); + AvroFlumeEvent e = new AvroFlumeEvent(toCharSeqMap(event.getHeaders()), + ByteBuffer.wrap(event.getBody())); encoder = EncoderFactory.get().directBinaryEncoder(tempOutStream.get(), encoder); writer.get().write(e, encoder); encoder.flush(); diff --git a/flume-ng-sinks/flume-ng-kafka-sink/src/main/java/org/apache/flume/sink/kafka/KafkaSinkConstants.java b/flume-ng-sinks/flume-ng-kafka-sink/src/main/java/org/apache/flume/sink/kafka/KafkaSinkConstants.java index 6b64bc14e2..1bf380c8f7 100644 --- a/flume-ng-sinks/flume-ng-kafka-sink/src/main/java/org/apache/flume/sink/kafka/KafkaSinkConstants.java +++ b/flume-ng-sinks/flume-ng-kafka-sink/src/main/java/org/apache/flume/sink/kafka/KafkaSinkConstants.java @@ -29,7 +29,8 @@ public class KafkaSinkConstants { public static final String TOPIC_CONFIG = KAFKA_PREFIX + "topic"; public static final String BATCH_SIZE = "flumeBatchSize"; - public static final String BOOTSTRAP_SERVERS_CONFIG = KAFKA_PREFIX + CommonClientConfigs.BOOTSTRAP_SERVERS_CONFIG; + public static final String BOOTSTRAP_SERVERS_CONFIG = + KAFKA_PREFIX + CommonClientConfigs.BOOTSTRAP_SERVERS_CONFIG; public static final String KEY_HEADER = "key"; public static final String TOPIC_HEADER = "topic"; @@ -37,25 +38,23 @@ public class KafkaSinkConstants { public static final String AVRO_EVENT = "useFlumeEventFormat"; public static final boolean DEFAULT_AVRO_EVENT = false; - public static final String DEFAULT_KEY_SERIALIZER = "org.apache.kafka.common.serialization.StringSerializer"; - public static final String DEFAULT_VALUE_SERIAIZER = "org.apache.kafka.common.serialization.ByteArraySerializer"; + public static final String DEFAULT_KEY_SERIALIZER = + "org.apache.kafka.common.serialization.StringSerializer"; + public static final String DEFAULT_VALUE_SERIAIZER = + "org.apache.kafka.common.serialization.ByteArraySerializer"; public static final int DEFAULT_BATCH_SIZE = 100; public static final String DEFAULT_TOPIC = "default-flume-topic"; public static final String DEFAULT_ACKS = "1"; - /* Old Properties */ - /* Properties */ + /* Properties */ public static final String OLD_BATCH_SIZE = "batchSize"; public static final String MESSAGE_SERIALIZER_KEY = "serializer.class"; public static final String KEY_SERIALIZER_KEY = "key.serializer.class"; public static final String BROKER_LIST_FLUME_KEY = "brokerList"; public static final String REQUIRED_ACKS_FLUME_KEY = "requiredAcks"; - - - } diff --git a/flume-ng-sinks/flume-ng-morphline-solr-sink/src/main/java/org/apache/flume/sink/solr/morphline/BlobDeserializer.java b/flume-ng-sinks/flume-ng-morphline-solr-sink/src/main/java/org/apache/flume/sink/solr/morphline/BlobDeserializer.java index 12bdc409ed..095f889cba 100644 --- a/flume-ng-sinks/flume-ng-morphline-solr-sink/src/main/java/org/apache/flume/sink/solr/morphline/BlobDeserializer.java +++ b/flume-ng-sinks/flume-ng-morphline-solr-sink/src/main/java/org/apache/flume/sink/solr/morphline/BlobDeserializer.java @@ -84,7 +84,8 @@ public Event readEvent() throws IOException { blob.write(buf, 0, n); blobLength += n; if (blobLength >= maxBlobLength) { - LOGGER.warn("File length exceeds maxBlobLength ({}), truncating BLOB event!", maxBlobLength); + LOGGER.warn("File length exceeds maxBlobLength ({}), truncating BLOB event!", + maxBlobLength); break; } } diff --git a/flume-ng-sinks/flume-ng-morphline-solr-sink/src/main/java/org/apache/flume/sink/solr/morphline/BlobHandler.java b/flume-ng-sinks/flume-ng-morphline-solr-sink/src/main/java/org/apache/flume/sink/solr/morphline/BlobHandler.java index e84dec1b70..ca7614a0c1 100644 --- a/flume-ng-sinks/flume-ng-morphline-solr-sink/src/main/java/org/apache/flume/sink/solr/morphline/BlobHandler.java +++ b/flume-ng-sinks/flume-ng-morphline-solr-sink/src/main/java/org/apache/flume/sink/solr/morphline/BlobHandler.java @@ -87,7 +87,8 @@ public List getEvents(HttpServletRequest request) throws Exception { blob.write(buf, 0, n); blobLength += n; if (blobLength >= maxBlobLength) { - LOGGER.warn("Request length exceeds maxBlobLength ({}), truncating BLOB event!", maxBlobLength); + LOGGER.warn("Request length exceeds maxBlobLength ({}), truncating BLOB event!", + maxBlobLength); break; } } diff --git a/flume-ng-sinks/flume-ng-morphline-solr-sink/src/main/java/org/apache/flume/sink/solr/morphline/MorphlineHandlerImpl.java b/flume-ng-sinks/flume-ng-morphline-solr-sink/src/main/java/org/apache/flume/sink/solr/morphline/MorphlineHandlerImpl.java index d3154af261..d877814147 100644 --- a/flume-ng-sinks/flume-ng-morphline-solr-sink/src/main/java/org/apache/flume/sink/solr/morphline/MorphlineHandlerImpl.java +++ b/flume-ng-sinks/flume-ng-morphline-solr-sink/src/main/java/org/apache/flume/sink/solr/morphline/MorphlineHandlerImpl.java @@ -97,8 +97,10 @@ public void configure(Context context) { .build(); } - Config override = ConfigFactory.parseMap(context.getSubProperties(MORPHLINE_VARIABLE_PARAM + ".")); - morphline = new Compiler().compile(new File(morphlineFile), morphlineId, morphlineContext, finalChild, override); + Config override = ConfigFactory.parseMap( + context.getSubProperties(MORPHLINE_VARIABLE_PARAM + ".")); + morphline = new Compiler().compile( + new File(morphlineFile), morphlineId, morphlineContext, finalChild, override); this.mappingTimer = morphlineContext.getMetricRegistry().timer( MetricRegistry.name("morphline.app", Metrics.ELAPSED_TIME)); diff --git a/flume-ng-sinks/flume-ng-morphline-solr-sink/src/main/java/org/apache/flume/sink/solr/morphline/MorphlineInterceptor.java b/flume-ng-sinks/flume-ng-morphline-solr-sink/src/main/java/org/apache/flume/sink/solr/morphline/MorphlineInterceptor.java index ef8f716517..3b941335fa 100644 --- a/flume-ng-sinks/flume-ng-morphline-solr-sink/src/main/java/org/apache/flume/sink/solr/morphline/MorphlineInterceptor.java +++ b/flume-ng-sinks/flume-ng-morphline-solr-sink/src/main/java/org/apache/flume/sink/solr/morphline/MorphlineInterceptor.java @@ -47,12 +47,13 @@ public class MorphlineInterceptor implements Interceptor { private final Context context; - private final Queue pool = new ConcurrentLinkedQueue(); + private final Queue pool = new ConcurrentLinkedQueue<>(); protected MorphlineInterceptor(Context context) { Preconditions.checkNotNull(context); this.context = context; - returnToPool(new LocalMorphlineInterceptor(context)); // fail fast on morphline compilation exception + // fail fast on morphline compilation exception + returnToPool(new LocalMorphlineInterceptor(context)); } @Override diff --git a/flume-ng-sinks/flume-ng-morphline-solr-sink/src/main/java/org/apache/flume/sink/solr/morphline/MorphlineSink.java b/flume-ng-sinks/flume-ng-morphline-solr-sink/src/main/java/org/apache/flume/sink/solr/morphline/MorphlineSink.java index 9c4dc25189..f7a73f3537 100644 --- a/flume-ng-sinks/flume-ng-morphline-solr-sink/src/main/java/org/apache/flume/sink/solr/morphline/MorphlineSink.java +++ b/flume-ng-sinks/flume-ng-morphline-solr-sink/src/main/java/org/apache/flume/sink/solr/morphline/MorphlineSink.java @@ -160,15 +160,15 @@ public Status process() throws EventDeliveryException { return numEventsTaken == 0 ? Status.BACKOFF : Status.READY; } catch (Throwable t) { // Ooops - need to rollback and back off - LOGGER.error("Morphline Sink " + getName() + ": Unable to process event from channel " + myChannel.getName() - + ". Exception follows.", t); + LOGGER.error("Morphline Sink " + getName() + ": Unable to process event from channel " + + myChannel.getName() + ". Exception follows.", t); try { if (!isMorphlineTransactionCommitted) { handler.rollbackTransaction(); } } catch (Throwable t2) { - LOGGER.error("Morphline Sink " + getName() + ": Unable to rollback morphline transaction. " + - "Exception follows.", t2); + LOGGER.error("Morphline Sink " + getName() + + ": Unable to rollback morphline transaction. Exception follows.", t2); } finally { try { txn.rollback(); diff --git a/flume-ng-sources/flume-jms-source/src/main/java/org/apache/flume/source/jms/DefaultJMSMessageConverter.java b/flume-ng-sources/flume-jms-source/src/main/java/org/apache/flume/source/jms/DefaultJMSMessageConverter.java index 6b327ce2b1..acb51186ba 100644 --- a/flume-ng-sources/flume-jms-source/src/main/java/org/apache/flume/source/jms/DefaultJMSMessageConverter.java +++ b/flume-ng-sources/flume-jms-source/src/main/java/org/apache/flume/source/jms/DefaultJMSMessageConverter.java @@ -57,15 +57,17 @@ public class DefaultJMSMessageConverter implements JMSMessageConverter { private final Charset charset; + private DefaultJMSMessageConverter(String charset) { this.charset = Charset.forName(charset); } + public static class Builder implements JMSMessageConverter.Builder { @Override public JMSMessageConverter build(Context context) { - return new DefaultJMSMessageConverter(context. - getString(JMSSourceConfiguration.CONVERTER_CHARSET, - JMSSourceConfiguration.CONVERTER_CHARSET_DEFAULT).trim()); + return new DefaultJMSMessageConverter(context.getString( + JMSSourceConfiguration.CONVERTER_CHARSET, + JMSSourceConfiguration.CONVERTER_CHARSET_DEFAULT).trim()); } } @@ -75,52 +77,52 @@ public List convert(Message message) throws JMSException { Map headers = event.getHeaders(); @SuppressWarnings("rawtypes") Enumeration propertyNames = message.getPropertyNames(); - while(propertyNames.hasMoreElements()) { + while (propertyNames.hasMoreElements()) { String name = propertyNames.nextElement().toString(); String value = message.getStringProperty(name); headers.put(name, value); } - if(message instanceof BytesMessage) { + if (message instanceof BytesMessage) { BytesMessage bytesMessage = (BytesMessage)message; long length = bytesMessage.getBodyLength(); - if(length > 0L) { + if (length > 0L) { if (length > Integer.MAX_VALUE) { throw new JMSException("Unable to process message " + "of size " + length); } byte[] body = new byte[(int)length]; int count = bytesMessage.readBytes(body); - if(count != length) { + if (count != length) { throw new JMSException("Unable to read full message. " + "Read " + count + " of total " + length); } event.setBody(body); } - } else if(message instanceof TextMessage) { + } else if (message instanceof TextMessage) { TextMessage textMessage = (TextMessage)message; event.setBody(textMessage.getText().getBytes(charset)); - } else if(message instanceof ObjectMessage) { + } else if (message instanceof ObjectMessage) { ObjectMessage objectMessage = (ObjectMessage)message; Object object = objectMessage.getObject(); - if(object != null) { + if (object != null) { ByteArrayOutputStream bos = new ByteArrayOutputStream(); ObjectOutput out = null; try { out = new ObjectOutputStream(bos); out.writeObject(object); event.setBody(bos.toByteArray()); - } catch(IOException e) { + } catch (IOException e) { throw new FlumeException("Error serializing object", e); } finally { try { - if(out != null) { + if (out != null) { out.close(); } } catch (IOException e) { throw new FlumeException("Error closing ObjectOutputStream", e); } try { - if(bos != null) { + if (bos != null) { bos.close(); } } catch (IOException e) { diff --git a/flume-ng-sources/flume-jms-source/src/main/java/org/apache/flume/source/jms/InitialContextFactory.java b/flume-ng-sources/flume-jms-source/src/main/java/org/apache/flume/source/jms/InitialContextFactory.java index 2f0220a262..8874dd15ae 100644 --- a/flume-ng-sources/flume-jms-source/src/main/java/org/apache/flume/source/jms/InitialContextFactory.java +++ b/flume-ng-sources/flume-jms-source/src/main/java/org/apache/flume/source/jms/InitialContextFactory.java @@ -22,7 +22,6 @@ import javax.naming.InitialContext; import javax.naming.NamingException; - public class InitialContextFactory { public InitialContext create(Properties properties) throws NamingException { diff --git a/flume-ng-sources/flume-jms-source/src/main/java/org/apache/flume/source/jms/JMSMessageConsumer.java b/flume-ng-sources/flume-jms-source/src/main/java/org/apache/flume/source/jms/JMSMessageConsumer.java index 7a9461ba1c..6b3a1cf6f6 100644 --- a/flume-ng-sources/flume-jms-source/src/main/java/org/apache/flume/source/jms/JMSMessageConsumer.java +++ b/flume-ng-sources/flume-jms-source/src/main/java/org/apache/flume/source/jms/JMSMessageConsumer.java @@ -18,8 +18,12 @@ */ package org.apache.flume.source.jms; -import java.util.ArrayList; -import java.util.List; +import com.google.common.base.Optional; +import com.google.common.base.Preconditions; +import org.apache.flume.Event; +import org.apache.flume.FlumeException; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; import javax.jms.Connection; import javax.jms.ConnectionFactory; @@ -30,14 +34,8 @@ import javax.jms.Session; import javax.naming.InitialContext; import javax.naming.NamingException; - -import org.apache.flume.Event; -import org.apache.flume.FlumeException; -import org.slf4j.Logger; -import org.slf4j.LoggerFactory; - -import com.google.common.base.Optional; -import com.google.common.base.Preconditions; +import java.util.ArrayList; +import java.util.List; class JMSMessageConsumer { private static final Logger logger = LoggerFactory @@ -52,11 +50,11 @@ class JMSMessageConsumer { private final Destination destination; private final MessageConsumer messageConsumer; - JMSMessageConsumer(InitialContext initialContext, ConnectionFactory connectionFactory, String destinationName, - JMSDestinationLocator destinationLocator, JMSDestinationType destinationType, - String messageSelector, int batchSize, long pollTimeout, - JMSMessageConverter messageConverter, - Optional userName, Optional password) { + JMSMessageConsumer(InitialContext initialContext, ConnectionFactory connectionFactory, + String destinationName, JMSDestinationLocator destinationLocator, + JMSDestinationType destinationType, String messageSelector, int batchSize, + long pollTimeout, JMSMessageConverter messageConverter, + Optional userName, Optional password) { this.batchSize = batchSize; this.pollTimeout = pollTimeout; this.messageConverter = messageConverter; @@ -65,7 +63,7 @@ class JMSMessageConsumer { Preconditions.checkArgument(pollTimeout >= 0, "Poll timeout cannot be " + "negative"); try { - if(userName.isPresent()) { + if (userName.isPresent()) { connection = connectionFactory.createConnection(userName.get(), password.get()); } else { @@ -82,37 +80,37 @@ class JMSMessageConsumer { throw new FlumeException("Could not create session", e); } - try { - if (destinationLocator.equals(JMSDestinationLocator.CDI)) { - switch (destinationType) { - case QUEUE: - destination = session.createQueue(destinationName); - break; - case TOPIC: - destination = session.createTopic(destinationName); - break; - default: - throw new IllegalStateException(String.valueOf(destinationType)); + try { + if (destinationLocator.equals(JMSDestinationLocator.CDI)) { + switch (destinationType) { + case QUEUE: + destination = session.createQueue(destinationName); + break; + case TOPIC: + destination = session.createTopic(destinationName); + break; + default: + throw new IllegalStateException(String.valueOf(destinationType)); + } + } else { + destination = (Destination) initialContext.lookup(destinationName); } - } else { - destination = (Destination) initialContext.lookup(destinationName); + } catch (JMSException e) { + throw new FlumeException("Could not create destination " + destinationName, e); + } catch (NamingException e) { + throw new FlumeException("Could not find destination " + destinationName, e); } - } catch (JMSException e) { - throw new FlumeException("Could not create destination " + destinationName, e); - } catch (NamingException e) { - throw new FlumeException("Could not find destination " + destinationName, e); - } - try { + try { messageConsumer = session.createConsumer(destination, - messageSelector.isEmpty() ? null: messageSelector); + messageSelector.isEmpty() ? null : messageSelector); } catch (JMSException e) { throw new FlumeException("Could not create consumer", e); } String startupMsg = String.format("Connected to '%s' of type '%s' with " + - "user '%s', batch size '%d', selector '%s' ", destinationName, + "user '%s', batch size '%d', selector '%s' ", destinationName, destinationType, userName.isPresent() ? userName.get() : "null", - batchSize, messageSelector.isEmpty() ? null : messageSelector); + batchSize, messageSelector.isEmpty() ? null : messageSelector); logger.info(startupMsg); } @@ -120,23 +118,23 @@ List take() throws JMSException { List result = new ArrayList(batchSize); Message message; message = messageConsumer.receive(pollTimeout); - if(message != null) { + if (message != null) { result.addAll(messageConverter.convert(message)); int max = batchSize - 1; for (int i = 0; i < max; i++) { message = messageConsumer.receiveNoWait(); - if(message == null) { + if (message == null) { break; } result.addAll(messageConverter.convert(message)); } } - if(logger.isDebugEnabled()) { - logger.debug(String.format("Took batch of %s from %s", result.size(), - destination)); + if (logger.isDebugEnabled()) { + logger.debug(String.format("Took batch of %s from %s", result.size(), destination)); } return result; } + void commit() { try { session.commit(); diff --git a/flume-ng-sources/flume-jms-source/src/main/java/org/apache/flume/source/jms/JMSMessageConsumerFactory.java b/flume-ng-sources/flume-jms-source/src/main/java/org/apache/flume/source/jms/JMSMessageConsumerFactory.java index af74bf4515..9747a31024 100644 --- a/flume-ng-sources/flume-jms-source/src/main/java/org/apache/flume/source/jms/JMSMessageConsumerFactory.java +++ b/flume-ng-sources/flume-jms-source/src/main/java/org/apache/flume/source/jms/JMSMessageConsumerFactory.java @@ -22,15 +22,15 @@ import com.google.common.base.Optional; - public class JMSMessageConsumerFactory { JMSMessageConsumer create(InitialContext initialContext, ConnectionFactory connectionFactory, - String destinationName, JMSDestinationType destinationType, JMSDestinationLocator destinationLocator, - String messageSelector, int batchSize, long pollTimeout, JMSMessageConverter messageConverter, - Optional userName, Optional password) { + String destinationName, JMSDestinationType destinationType, + JMSDestinationLocator destinationLocator, String messageSelector, int batchSize, + long pollTimeout, JMSMessageConverter messageConverter, + Optional userName, Optional password) { return new JMSMessageConsumer(initialContext, connectionFactory, destinationName, - destinationLocator, destinationType, messageSelector, batchSize, pollTimeout, + destinationLocator, destinationType, messageSelector, batchSize, pollTimeout, messageConverter, userName, password); } } diff --git a/flume-ng-sources/flume-jms-source/src/main/java/org/apache/flume/source/jms/JMSSource.java b/flume-ng-sources/flume-jms-source/src/main/java/org/apache/flume/source/jms/JMSSource.java index c1cc9cf5e9..763182777c 100644 --- a/flume-ng-sources/flume-jms-source/src/main/java/org/apache/flume/source/jms/JMSSource.java +++ b/flume-ng-sources/flume-jms-source/src/main/java/org/apache/flume/source/jms/JMSSource.java @@ -76,40 +76,39 @@ public class JMSSource extends AbstractPollableSource { private int jmsExceptionCounter; private InitialContext initialContext; - public JMSSource() { this(new JMSMessageConsumerFactory(), new InitialContextFactory()); } + @VisibleForTesting - public JMSSource(JMSMessageConsumerFactory consumerFactory, InitialContextFactory initialContextFactory) { + public JMSSource(JMSMessageConsumerFactory consumerFactory, + InitialContextFactory initialContextFactory) { super(); this.consumerFactory = consumerFactory; this.initialContextFactory = initialContextFactory; - } @Override protected void doConfigure(Context context) throws FlumeException { sourceCounter = new SourceCounter(getName()); - initialContextFactoryName = context.getString(JMSSourceConfiguration. - INITIAL_CONTEXT_FACTORY, "").trim(); + initialContextFactoryName = context.getString( + JMSSourceConfiguration.INITIAL_CONTEXT_FACTORY, "").trim(); - providerUrl = context.getString(JMSSourceConfiguration.PROVIDER_URL, "") - .trim(); + providerUrl = context.getString(JMSSourceConfiguration.PROVIDER_URL, "").trim(); - destinationName = context.getString(JMSSourceConfiguration. - DESTINATION_NAME, "").trim(); + destinationName = context.getString(JMSSourceConfiguration.DESTINATION_NAME, "").trim(); - String destinationTypeName = context.getString(JMSSourceConfiguration. - DESTINATION_TYPE, "").trim().toUpperCase(Locale.ENGLISH); + String destinationTypeName = context.getString( + JMSSourceConfiguration.DESTINATION_TYPE, "").trim().toUpperCase(Locale.ENGLISH); - String destinationLocatorName = context.getString(JMSSourceConfiguration. - DESTINATION_LOCATOR, JMSSourceConfiguration.DESTINATION_LOCATOR_DEFAULT) - .trim().toUpperCase(Locale.ENGLISH); + String destinationLocatorName = context.getString( + JMSSourceConfiguration.DESTINATION_LOCATOR, + JMSSourceConfiguration.DESTINATION_LOCATOR_DEFAULT) + .trim().toUpperCase(Locale.ENGLISH); - messageSelector = context.getString(JMSSourceConfiguration. - MESSAGE_SELECTOR, "").trim(); + messageSelector = context.getString( + JMSSourceConfiguration.MESSAGE_SELECTOR, "").trim(); batchSize = context.getInteger(JMSSourceConfiguration.BATCH_SIZE, JMSSourceConfiguration.BATCH_SIZE_DEFAULT); @@ -117,16 +116,14 @@ protected void doConfigure(Context context) throws FlumeException { errorThreshold = context.getInteger(JMSSourceConfiguration.ERROR_THRESHOLD, JMSSourceConfiguration.ERROR_THRESHOLD_DEFAULT); - userName = Optional.fromNullable(context.getString(JMSSourceConfiguration. - USERNAME)); + userName = Optional.fromNullable(context.getString(JMSSourceConfiguration.USERNAME)); pollTimeout = context.getLong(JMSSourceConfiguration.POLL_TIMEOUT, JMSSourceConfiguration.POLL_TIMEOUT_DEFAULT); - String passwordFile = context.getString(JMSSourceConfiguration. - PASSWORD_FILE, "").trim(); + String passwordFile = context.getString(JMSSourceConfiguration.PASSWORD_FILE, "").trim(); - if(passwordFile.isEmpty()) { + if (passwordFile.isEmpty()) { password = Optional.of(""); } else { try { @@ -140,45 +137,38 @@ protected void doConfigure(Context context) throws FlumeException { String converterClassName = context.getString( JMSSourceConfiguration.CONVERTER_TYPE, - JMSSourceConfiguration.CONVERTER_TYPE_DEFAULT) - .trim(); - if(JMSSourceConfiguration.CONVERTER_TYPE_DEFAULT. - equalsIgnoreCase(converterClassName)) { + JMSSourceConfiguration.CONVERTER_TYPE_DEFAULT).trim(); + if (JMSSourceConfiguration.CONVERTER_TYPE_DEFAULT.equalsIgnoreCase(converterClassName)) { converterClassName = DefaultJMSMessageConverter.Builder.class.getName(); } - Context converterContext = new Context(context. - getSubProperties(JMSSourceConfiguration.CONVERTER + ".")); + Context converterContext = new Context(context.getSubProperties( + JMSSourceConfiguration.CONVERTER + ".")); try { @SuppressWarnings("rawtypes") Class clazz = Class.forName(converterClassName); boolean isBuilder = JMSMessageConverter.Builder.class .isAssignableFrom(clazz); - if(isBuilder) { - JMSMessageConverter.Builder builder = (JMSMessageConverter.Builder) - clazz.newInstance(); + if (isBuilder) { + JMSMessageConverter.Builder builder = (JMSMessageConverter.Builder)clazz.newInstance(); converter = builder.build(converterContext); } else { - Preconditions.checkState(JMSMessageConverter.class. - isAssignableFrom(clazz), String. - format("Class %s is not a subclass of JMSMessageConverter", - clazz.getName())); + Preconditions.checkState(JMSMessageConverter.class.isAssignableFrom(clazz), + String.format("Class %s is not a subclass of JMSMessageConverter", clazz.getName())); converter = (JMSMessageConverter)clazz.newInstance(); - boolean configured = Configurables.configure(converter, - converterContext); - if(logger.isDebugEnabled()) { - logger.debug(String. - format("Attempted configuration of %s, result = %s", - converterClassName, String.valueOf(configured))); + boolean configured = Configurables.configure(converter, converterContext); + if (logger.isDebugEnabled()) { + logger.debug(String.format("Attempted configuration of %s, result = %s", + converterClassName, String.valueOf(configured))); } } - } catch(Exception e) { + } catch (Exception e) { throw new FlumeException(String.format( "Unable to create instance of converter %s", converterClassName), e); } - String connectionFactoryName = context.getString(JMSSourceConfiguration. - CONNECTION_FACTORY, JMSSourceConfiguration.CONNECTION_FACTORY_DEFAULT) - .trim(); + String connectionFactoryName = context.getString( + JMSSourceConfiguration.CONNECTION_FACTORY, + JMSSourceConfiguration.CONNECTION_FACTORY_DEFAULT).trim(); assertNotEmpty(initialContextFactoryName, String.format( "Initial Context Factory is empty. This is specified by %s", @@ -210,8 +200,7 @@ protected void doConfigure(Context context) throws FlumeException { "invalid.", destinationLocatorName), e); } - Preconditions.checkArgument(batchSize > 0, "Batch size must be greater " + - "than 0"); + Preconditions.checkArgument(batchSize > 0, "Batch size must be greater than 0"); try { Properties contextProperties = new Properties(); @@ -223,12 +212,12 @@ protected void doConfigure(Context context) throws FlumeException { // Provide properties for connecting via JNDI if (this.userName.isPresent()) { - contextProperties.setProperty( - javax.naming.Context.SECURITY_PRINCIPAL, this.userName.get()); + contextProperties.setProperty(javax.naming.Context.SECURITY_PRINCIPAL, + this.userName.get()); } if (this.password.isPresent()) { - contextProperties.setProperty( - javax.naming.Context.SECURITY_CREDENTIALS, this.password.get()); + contextProperties.setProperty(javax.naming.Context.SECURITY_CREDENTIALS, + this.password.get()); } initialContext = initialContextFactory.create(contextProperties); @@ -239,28 +228,26 @@ protected void doConfigure(Context context) throws FlumeException { } try { - connectionFactory = (ConnectionFactory) initialContext. - lookup(connectionFactoryName); + connectionFactory = (ConnectionFactory) initialContext.lookup(connectionFactoryName); } catch (NamingException e) { throw new FlumeException("Could not lookup ConnectionFactory", e); } } private void assertNotEmpty(String arg, String msg) { - Preconditions.checkArgument(!arg.isEmpty(), - msg); + Preconditions.checkArgument(!arg.isEmpty(), msg); } @Override protected synchronized Status doProcess() throws EventDeliveryException { boolean error = true; try { - if(consumer == null) { + if (consumer == null) { consumer = createConsumer(); } List events = consumer.take(); int size = events.size(); - if(size == 0) { + if (size == 0) { error = false; return Status.BACKOFF; } @@ -275,28 +262,28 @@ protected synchronized Status doProcess() throws EventDeliveryException { logger.warn("Error appending event to channel. " + "Channel might be full. Consider increasing the channel " + "capacity or make sure the sinks perform faster.", channelException); - } catch(JMSException jmsException) { + } catch (JMSException jmsException) { logger.warn("JMSException consuming events", jmsException); - if(++jmsExceptionCounter > errorThreshold) { - if(consumer != null) { + if (++jmsExceptionCounter > errorThreshold) { + if (consumer != null) { logger.warn("Exceeded JMSException threshold, closing consumer"); consumer.rollback(); consumer.close(); consumer = null; } } - } catch(Throwable throwable) { + } catch (Throwable throwable) { logger.error("Unexpected error processing events", throwable); - if(throwable instanceof Error) { + if (throwable instanceof Error) { throw (Error) throwable; } } finally { - if(error) { - if(consumer != null) { + if (error) { + if (consumer != null) { consumer.rollback(); } } else { - if(consumer != null) { + if (consumer != null) { consumer.commit(); jmsExceptionCounter = 0; } @@ -304,6 +291,7 @@ protected synchronized Status doProcess() throws EventDeliveryException { } return Status.BACKOFF; } + @Override protected synchronized void doStart() { try { @@ -317,18 +305,18 @@ protected synchronized void doStart() { @Override protected synchronized void doStop() { - if(consumer != null) { + if (consumer != null) { consumer.close(); consumer = null; } sourceCounter.stop(); } + private JMSMessageConsumer createConsumer() throws JMSException { logger.info("Creating new consumer for " + destinationName); JMSMessageConsumer consumer = consumerFactory.create(initialContext, - connectionFactory, destinationName, destinationType, destinationLocator, - messageSelector, batchSize, - pollTimeout, converter, userName, password); + connectionFactory, destinationName, destinationType, destinationLocator, + messageSelector, batchSize, pollTimeout, converter, userName, password); jmsExceptionCounter = 0; return consumer; } diff --git a/flume-ng-sources/flume-kafka-source/src/main/java/org/apache/flume/source/kafka/KafkaSource.java b/flume-ng-sources/flume-kafka-source/src/main/java/org/apache/flume/source/kafka/KafkaSource.java index 84fef524e4..90e47159cf 100644 --- a/flume-ng-sources/flume-kafka-source/src/main/java/org/apache/flume/source/kafka/KafkaSource.java +++ b/flume-ng-sources/flume-kafka-source/src/main/java/org/apache/flume/source/kafka/KafkaSource.java @@ -112,18 +112,24 @@ public class KafkaSource extends AbstractPollableSource */ public abstract class Subscriber { public abstract void subscribe(KafkaConsumer consumer, SourceRebalanceListener listener); - public T get() {return null;} + + public T get() { + return null; + } } private class TopicListSubscriber extends Subscriber> { private List topicList; + public TopicListSubscriber(String commaSeparatedTopics) { this.topicList = Arrays.asList(commaSeparatedTopics.split("^\\s+|\\s*,\\s*|\\s+$")); } + @Override public void subscribe(KafkaConsumer consumer, SourceRebalanceListener listener) { consumer.subscribe(topicList, listener); } + @Override public List get() { return topicList; @@ -132,13 +138,16 @@ public List get() { private class PatternSubscriber extends Subscriber { private Pattern pattern; + public PatternSubscriber(String regex) { this.pattern = Pattern.compile(regex); } + @Override public void subscribe(KafkaConsumer consumer, SourceRebalanceListener listener) { consumer.subscribe(pattern, listener); } + @Override public Pattern get() { return pattern; @@ -232,10 +241,11 @@ protected Status doProcess() throws EventDeliveryException { } if (log.isDebugEnabled()) { - log.debug("Topic: {} Partition: {} Message: {}", new String[]{ - message.topic(), - String.valueOf(message.partition()), - new String(eventBody)}); + log.debug("Topic: {} Partition: {} Message: {}", new String[] { + message.topic(), + String.valueOf(message.partition()), + new String(eventBody) + }); } event = EventBuilder.withBody(eventBody, headers); @@ -305,21 +315,21 @@ protected void doConfigure(Context context) throws FlumeException { if (topicProperty != null && !topicProperty.isEmpty()) { // create subscriber that uses pattern-based subscription subscriber = new PatternSubscriber(topicProperty); - } else - if((topicProperty = context.getString(KafkaSourceConstants.TOPICS)) != null && !topicProperty.isEmpty()) { + } else if ((topicProperty = context.getString(KafkaSourceConstants.TOPICS)) != null && + !topicProperty.isEmpty()) { // create subscriber that uses topic list subscription subscriber = new TopicListSubscriber(topicProperty); - } else - if (subscriber == null) { + } else if (subscriber == null) { throw new ConfigurationException("At least one Kafka topic must be specified."); } batchUpperLimit = context.getInteger(KafkaSourceConstants.BATCH_SIZE, - KafkaSourceConstants.DEFAULT_BATCH_SIZE); + KafkaSourceConstants.DEFAULT_BATCH_SIZE); maxBatchDurationMillis = context.getInteger(KafkaSourceConstants.BATCH_DURATION_MS, - KafkaSourceConstants.DEFAULT_BATCH_DURATION); + KafkaSourceConstants.DEFAULT_BATCH_DURATION); - useAvroEventFormat = context.getBoolean(KafkaSourceConstants.AVRO_EVENT, KafkaSourceConstants.DEFAULT_AVRO_EVENT); + useAvroEventFormat = context.getBoolean(KafkaSourceConstants.AVRO_EVENT, + KafkaSourceConstants.DEFAULT_AVRO_EVENT); if (log.isDebugEnabled()) { log.debug(KafkaSourceConstants.AVRO_EVENT + " set to: {}", useAvroEventFormat); @@ -337,7 +347,6 @@ protected void doConfigure(Context context) throws FlumeException { } } - // We can remove this once the properties are officially deprecated private void translateOldProperties(Context ctx) { // topic @@ -358,16 +367,18 @@ private void translateOldProperties(Context ctx) { } } - private void setConsumerProps(Context ctx, String bootStrapServers) { - String groupId = ctx.getString(KafkaSourceConstants.KAFKA_CONSUMER_PREFIX + ConsumerConfig.GROUP_ID_CONFIG); + String groupId = ctx.getString( + KafkaSourceConstants.KAFKA_CONSUMER_PREFIX + ConsumerConfig.GROUP_ID_CONFIG); if ((groupId == null || groupId.isEmpty()) && - kafkaProps.getProperty(ConsumerConfig.GROUP_ID_CONFIG) == null) { - groupId = KafkaSourceConstants.DEFAULT_GROUP_ID; - log.info("Group ID was not specified. Using " + groupId + " as the group id."); + kafkaProps.getProperty(ConsumerConfig.GROUP_ID_CONFIG) == null) { + groupId = KafkaSourceConstants.DEFAULT_GROUP_ID; + log.info("Group ID was not specified. Using " + groupId + " as the group id."); } - kafkaProps.put(ConsumerConfig.KEY_DESERIALIZER_CLASS_CONFIG, KafkaSourceConstants.DEFAULT_KEY_DESERIALIZER); - kafkaProps.put(ConsumerConfig.VALUE_DESERIALIZER_CLASS_CONFIG, KafkaSourceConstants.DEFAULT_VALUE_DESERIALIZER); + kafkaProps.put(ConsumerConfig.KEY_DESERIALIZER_CLASS_CONFIG, + KafkaSourceConstants.DEFAULT_KEY_DESERIALIZER); + kafkaProps.put(ConsumerConfig.VALUE_DESERIALIZER_CLASS_CONFIG, + KafkaSourceConstants.DEFAULT_VALUE_DESERIALIZER); //Defaults overridden based on config kafkaProps.putAll(ctx.getSubProperties(KafkaSourceConstants.KAFKA_CONSUMER_PREFIX)); //These always take precedence over config @@ -375,7 +386,8 @@ private void setConsumerProps(Context ctx, String bootStrapServers) { if (groupId != null) { kafkaProps.put(ConsumerConfig.GROUP_ID_CONFIG, groupId); } - kafkaProps.put(ConsumerConfig.ENABLE_AUTO_COMMIT_CONFIG, KafkaSourceConstants.DEFAULT_AUTO_COMMIT); + kafkaProps.put(ConsumerConfig.ENABLE_AUTO_COMMIT_CONFIG, + KafkaSourceConstants.DEFAULT_AUTO_COMMIT); log.info(kafkaProps.toString()); } @@ -426,7 +438,6 @@ protected void doStop() throws FlumeException { } } - class SourceRebalanceListener implements ConsumerRebalanceListener { private static final Logger log = LoggerFactory.getLogger(SourceRebalanceListener.class); private AtomicBoolean rebalanceFlag; diff --git a/flume-ng-sources/flume-kafka-source/src/main/java/org/apache/flume/source/kafka/KafkaSourceConstants.java b/flume-ng-sources/flume-kafka-source/src/main/java/org/apache/flume/source/kafka/KafkaSourceConstants.java index 9f20f614c8..1f255f9471 100644 --- a/flume-ng-sources/flume-kafka-source/src/main/java/org/apache/flume/source/kafka/KafkaSourceConstants.java +++ b/flume-ng-sources/flume-kafka-source/src/main/java/org/apache/flume/source/kafka/KafkaSourceConstants.java @@ -22,9 +22,12 @@ public class KafkaSourceConstants { public static final String KAFKA_PREFIX = "kafka."; public static final String KAFKA_CONSUMER_PREFIX = KAFKA_PREFIX + "consumer."; - public static final String DEFAULT_KEY_DESERIALIZER = "org.apache.kafka.common.serialization.StringDeserializer"; - public static final String DEFAULT_VALUE_DESERIALIZER = "org.apache.kafka.common.serialization.ByteArrayDeserializer"; - public static final String BOOTSTRAP_SERVERS = KAFKA_PREFIX + CommonClientConfigs.BOOTSTRAP_SERVERS_CONFIG; + public static final String DEFAULT_KEY_DESERIALIZER = + "org.apache.kafka.common.serialization.StringDeserializer"; + public static final String DEFAULT_VALUE_DESERIALIZER = + "org.apache.kafka.common.serialization.ByteArrayDeserializer"; + public static final String BOOTSTRAP_SERVERS = + KAFKA_PREFIX + CommonClientConfigs.BOOTSTRAP_SERVERS_CONFIG; public static final String TOPICS = KAFKA_PREFIX + "topics"; public static final String TOPICS_REGEX = TOPICS + "." + "regex"; public static final String DEFAULT_AUTO_COMMIT = "false"; diff --git a/flume-ng-sources/flume-taildir-source/src/main/java/org/apache/flume/source/taildir/ReliableTaildirEventReader.java b/flume-ng-sources/flume-taildir-source/src/main/java/org/apache/flume/source/taildir/ReliableTaildirEventReader.java index 8128df4c82..1409f2522a 100644 --- a/flume-ng-sources/flume-taildir-source/src/main/java/org/apache/flume/source/taildir/ReliableTaildirEventReader.java +++ b/flume-ng-sources/flume-taildir-source/src/main/java/org/apache/flume/source/taildir/ReliableTaildirEventReader.java @@ -19,6 +19,20 @@ package org.apache.flume.source.taildir; +import com.google.common.annotations.VisibleForTesting; +import com.google.common.base.Preconditions; +import com.google.common.collect.Lists; +import com.google.common.collect.Maps; +import com.google.common.collect.Table; +import com.google.gson.stream.JsonReader; +import org.apache.flume.Event; +import org.apache.flume.FlumeException; +import org.apache.flume.annotations.InterfaceAudience; +import org.apache.flume.annotations.InterfaceStability; +import org.apache.flume.client.avro.ReliableEventReader; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + import java.io.File; import java.io.FileNotFoundException; import java.io.FileReader; @@ -29,21 +43,6 @@ import java.util.Map; import java.util.Map.Entry; -import org.apache.flume.Event; -import org.apache.flume.FlumeException; -import org.apache.flume.annotations.InterfaceAudience; -import org.apache.flume.annotations.InterfaceStability; -import org.apache.flume.client.avro.ReliableEventReader; -import org.slf4j.Logger; -import org.slf4j.LoggerFactory; - -import com.google.common.annotations.VisibleForTesting; -import com.google.common.base.Preconditions; -import com.google.common.collect.Lists; -import com.google.common.collect.Maps; -import com.google.common.collect.Table; -import com.google.gson.stream.JsonReader; - @InterfaceAudience.Private @InterfaceStability.Evolving public class ReliableTaildirEventReader implements ReliableEventReader { @@ -111,15 +110,15 @@ public void loadPositionFile(String filePath) { jr.beginObject(); while (jr.hasNext()) { switch (jr.nextName()) { - case "inode": - inode = jr.nextLong(); - break; - case "pos": - pos = jr.nextLong(); - break; - case "file": - path = jr.nextString(); - break; + case "inode": + inode = jr.nextLong(); + break; + case "pos": + pos = jr.nextLong(); + break; + case "file": + path = jr.nextString(); + break; } } jr.endObject(); @@ -238,7 +237,7 @@ public List updateTailFiles(boolean skipToEnd) throws IOException { if (tf == null || !tf.getPath().equals(f.getAbsolutePath())) { long startPos = skipToEnd ? f.length() : 0; tf = openFile(f, headers, inode, startPos); - } else{ + } else { boolean updated = tf.getLastUpdated() < f.lastModified(); if (updated) { if (tf.getRaf() == null) { @@ -320,7 +319,8 @@ public Builder cachePatternMatching(boolean cachePatternMatching) { } public ReliableTaildirEventReader build() throws IOException { - return new ReliableTaildirEventReader(filePaths, headerTable, positionFilePath, skipToEnd, addByteOffset, cachePatternMatching); + return new ReliableTaildirEventReader(filePaths, headerTable, positionFilePath, skipToEnd, + addByteOffset, cachePatternMatching); } } diff --git a/flume-ng-sources/flume-taildir-source/src/main/java/org/apache/flume/source/taildir/TailFile.java b/flume-ng-sources/flume-taildir-source/src/main/java/org/apache/flume/source/taildir/TailFile.java index cb36e411a7..42474c4f75 100644 --- a/flume-ng-sources/flume-taildir-source/src/main/java/org/apache/flume/source/taildir/TailFile.java +++ b/flume-ng-sources/flume-taildir-source/src/main/java/org/apache/flume/source/taildir/TailFile.java @@ -19,21 +19,19 @@ package org.apache.flume.source.taildir; -import static org.apache.flume.source.taildir.TaildirSourceConfigurationConstants.*; +import com.google.common.collect.Lists; +import org.apache.flume.Event; +import org.apache.flume.event.EventBuilder; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; import java.io.File; import java.io.IOException; import java.io.RandomAccessFile; -import java.util.Comparator; import java.util.List; import java.util.Map; -import org.apache.flume.Event; -import org.apache.flume.event.EventBuilder; -import org.slf4j.Logger; -import org.slf4j.LoggerFactory; - -import com.google.common.collect.Lists; +import static org.apache.flume.source.taildir.TaildirSourceConfigurationConstants.BYTE_OFFSET_HEADER_KEY; public class TailFile { private static final Logger logger = LoggerFactory.getLogger(TailFile.class); @@ -41,8 +39,8 @@ public class TailFile { private static final byte BYTE_NL = (byte) 10; private static final byte BYTE_CR = (byte) 13; - private final static int BUFFER_SIZE = 8192; - private final static int NEED_READING = -1; + private static final int BUFFER_SIZE = 8192; + private static final int NEED_READING = -1; private RandomAccessFile raf; private final String path; @@ -61,7 +59,7 @@ public TailFile(File file, Map headers, long inode, long pos) this.raf = new RandomAccessFile(file, "r"); if (pos > 0) { raf.seek(pos); - lineReadPos=pos; + lineReadPos = pos; } this.path = file.getAbsolutePath(); this.inode = inode; @@ -70,22 +68,56 @@ public TailFile(File file, Map headers, long inode, long pos) this.needTail = true; this.headers = headers; this.oldBuffer = new byte[0]; - this.bufferPos= NEED_READING; + this.bufferPos = NEED_READING; + } + + public RandomAccessFile getRaf() { + return raf; + } + + public String getPath() { + return path; + } + + public long getInode() { + return inode; } - public RandomAccessFile getRaf() { return raf; } - public String getPath() { return path; } - public long getInode() { return inode; } - public long getPos() { return pos; } - public long getLastUpdated() { return lastUpdated; } - public boolean needTail() { return needTail; } - public Map getHeaders() { return headers; } - public long getLineReadPos() { return lineReadPos; } + public long getPos() { + return pos; + } + + public long getLastUpdated() { + return lastUpdated; + } + + public boolean needTail() { + return needTail; + } + + public Map getHeaders() { + return headers; + } + + public long getLineReadPos() { + return lineReadPos; + } - public void setPos(long pos) { this.pos = pos; } - public void setLastUpdated(long lastUpdated) { this.lastUpdated = lastUpdated; } - public void setNeedTail(boolean needTail) { this.needTail = needTail; } - public void setLineReadPos(long lineReadPos) { this.lineReadPos = lineReadPos; } + public void setPos(long pos) { + this.pos = pos; + } + + public void setLastUpdated(long lastUpdated) { + this.lastUpdated = lastUpdated; + } + + public void setNeedTail(boolean needTail) { + this.needTail = needTail; + } + + public void setLineReadPos(long lineReadPos) { + this.lineReadPos = lineReadPos; + } public boolean updatePos(String path, long inode, long pos) throws IOException { if (this.inode == inode && this.path.equals(path)) { @@ -99,7 +131,7 @@ public boolean updatePos(String path, long inode, long pos) throws IOException { public void updateFilePos(long pos) throws IOException { raf.seek(pos); lineReadPos = pos; - bufferPos= NEED_READING; + bufferPos = NEED_READING; oldBuffer = new byte[0]; } @@ -146,7 +178,8 @@ private void readFile() throws IOException { bufferPos = 0; } - private byte[] concatByteArrays(byte[] a, int startIdxA, int lenA, byte[] b, int startIdxB, int lenB) { + private byte[] concatByteArrays(byte[] a, int startIdxA, int lenA, + byte[] b, int startIdxB, int lenB) { byte[] c = new byte[lenA + lenB]; System.arraycopy(a, startIdxA, c, 0, lenA); System.arraycopy(b, startIdxB, c, lenA, lenB); @@ -195,7 +228,8 @@ public LineResult readLine() throws IOException { break; } // NEW_LINE not showed up at the end of the buffer - oldBuffer = concatByteArrays(oldBuffer, 0, oldBuffer.length, buffer, bufferPos, (buffer.length - bufferPos)); + oldBuffer = concatByteArrays(oldBuffer, 0, oldBuffer.length, + buffer, bufferPos, buffer.length - bufferPos); bufferPos = NEED_READING; } return lineResult; diff --git a/flume-ng-sources/flume-taildir-source/src/main/java/org/apache/flume/source/taildir/TaildirMatcher.java b/flume-ng-sources/flume-taildir-source/src/main/java/org/apache/flume/source/taildir/TaildirMatcher.java index 245aef5156..ad9f720170 100644 --- a/flume-ng-sources/flume-taildir-source/src/main/java/org/apache/flume/source/taildir/TaildirMatcher.java +++ b/flume-ng-sources/flume-taildir-source/src/main/java/org/apache/flume/source/taildir/TaildirMatcher.java @@ -27,7 +27,6 @@ import org.slf4j.LoggerFactory; import java.io.File; -import java.io.FileFilter; import java.io.IOException; import java.nio.file.DirectoryStream; import java.nio.file.FileSystem; @@ -40,25 +39,25 @@ import java.util.HashMap; import java.util.List; import java.util.concurrent.TimeUnit; -import java.util.regex.Pattern; /** - * Identifies and caches the files matched by single file pattern for TAILDIR source. + * Identifies and caches the files matched by single file pattern for {@code TAILDIR} source. *

    - * Since file patterns only apply to the fileNames and not the parent dictionaries, this implementation - * checks the parent directory for modification (additional or removed files update modification time of parent dir) - * If no modification happened to the parent dir that means the underlying files could only be written to but no need - * to rerun the pattern matching on fileNames. + * Since file patterns only apply to the fileNames and not the parent dictionaries, this + * implementation checks the parent directory for modification (additional or removed files + * update modification time of parent dir) + * If no modification happened to the parent dir that means the underlying files could only be + * written to but no need to rerun the pattern matching on fileNames. *

    - * This implementation provides lazy caching or no caching. Instances of this class keep the result - * file list from the last successful execution - * of {@linkplain #getMatchingFiles()} function invocation, - * and may serve the content without hitting the FileSystem for performance optimization. + * This implementation provides lazy caching or no caching. Instances of this class keep the + * result file list from the last successful execution of {@linkplain #getMatchingFiles()} + * function invocation, and may serve the content without hitting the FileSystem for performance + * optimization. *

    - * IMPORTANT: It is assumed that the hosting system provides at least second granularity for both - * System.currentTimeMillis() and File.lastModified(). Also that system clock is used - * for file system timestamps. If it is not the case then configure it as uncached. - * Class is solely for package only usage. Member functions are not thread safe. + * IMPORTANT: It is assumed that the hosting system provides at least second granularity + * for both {@code System.currentTimeMillis()} and {@code File.lastModified()}. Also + * that system clock is used for file system timestamps. If it is not the case then configure it + * as uncached. Class is solely for package only usage. Member functions are not thread safe. * * @see TaildirSource * @see ReliableTaildirEventReader @@ -84,29 +83,33 @@ public class TaildirMatcher { // system time in milliseconds, stores the last modification time of the // parent directory seen by the last check, rounded to seconds - // initial value is used in first check only when it will be replaced instantly (system time is positive) + // initial value is used in first check only when it will be replaced instantly + // (system time is positive) private long lastSeenParentDirMTime = -1; // system time in milliseconds, time of the last check, rounded to seconds - // initial value is used in first check only when it will be replaced instantly (system time is positive) + // initial value is used in first check only when it will be replaced instantly + // (system time is positive) private long lastCheckedTime = -1; // cached content, files which matched the pattern within the parent directory private List lastMatchedFiles = Lists.newArrayList(); /** - * Package accessible constructor. From configuration context it represents a single filegroup - * and encapsulates the corresponding filePattern. - * filePattern consists of two parts: first part has to be a valid path to - * an existing parent directory, second part has to be a - * valid regex {@link java.util.regex.Pattern} that match any non-hidden file names within parent directory. - * A valid example for filePattern is /dir0/dir1/.* given /dir0/dir1 - * is an existing directory structure readable by the running user. + * Package accessible constructor. From configuration context it represents a single + * filegroup and encapsulates the corresponding filePattern. + * filePattern consists of two parts: first part has to be a valid path to an + * existing parent directory, second part has to be a valid regex + * {@link java.util.regex.Pattern} that match any non-hidden file names within parent directory + * . A valid example for filePattern is /dir0/dir1/.* given + * /dir0/dir1 is an existing directory structure readable by the running user. *

    * An instance of this class is created for each fileGroup * * @param fileGroup arbitrary name of the group given by the config - * @param filePattern parent directory plus regex pattern. No wildcards are allowed in directory name - * @param cachePatternMatching default true, recommended in every setup especially with huge parent directories. - * Don't set when local system clock is not used for stamping mtime (eg: remote filesystems) + * @param filePattern parent directory plus regex pattern. No wildcards are allowed in directory + * name + * @param cachePatternMatching default true, recommended in every setup especially with huge + * parent directories. Don't set when local system clock is not used + * for stamping mtime (eg: remote filesystems) * @see TaildirSourceConfigurationConstants */ TaildirMatcher(String fileGroup, String filePattern, boolean cachePatternMatching) { @@ -133,11 +136,12 @@ public boolean accept(Path entry) throws IOException { } /** - * Lists those files within the parentDir that match regex pattern passed in during object instantiation. - * Designed for frequent periodic invocation {@link org.apache.flume.source.PollableSourceRunner}. + * Lists those files within the parentDir that match regex pattern passed in during object + * instantiation. Designed for frequent periodic invocation + * {@link org.apache.flume.source.PollableSourceRunner}. *

    - * Based on the modification of the parentDir this function may trigger cache recalculation by calling - * {@linkplain #getMatchingFilesNoCache()} or + * Based on the modification of the parentDir this function may trigger cache recalculation by + * calling {@linkplain #getMatchingFilesNoCache()} or * return the value stored in {@linkplain #lastMatchedFiles}. * Parentdir is allowed to be a symbolic link. *

    @@ -146,7 +150,8 @@ public boolean accept(Path entry) throws IOException { * so it may (or may not) reflect updates to the directory that occur during the call, * In which case next call * will return those files (as mtime is increasing it won't hit cache but trigger recalculation). - * It is guaranteed that invocation reflects every change which was observable at the time of invocation. + * It is guaranteed that invocation reflects every change which was observable at the time of + * invocation. *

    * Matching file list recalculation is triggered when caching was turned off or * if mtime is greater than the previously seen mtime @@ -156,8 +161,8 @@ public boolean accept(Path entry) throws IOException { * within the same second so in such case (assuming at least second granularity of reported mtime) * it is impossible to tell whether a change of the dir happened before the check or after * (unless the check happened after that second). - * Having said that implementation also stores system time of the previous invocation and previous invocation has to - * happen strictly after the current mtime to avoid further cache refresh + * Having said that implementation also stores system time of the previous invocation and previous + * invocation has to happen strictly after the current mtime to avoid further cache refresh * (because then it is guaranteed that previous invocation resulted in valid cache content). * If system clock hasn't passed the second of * the current mtime then logic expects more changes as well @@ -166,23 +171,26 @@ public boolean accept(Path entry) throws IOException { * hence it recalculates matching files. If system clock finally * passed actual mtime then a subsequent invocation guarantees that it picked up every * change from the passed second so - * any further invocations can be served from cache associated with that second (given mtime is not updated again). + * any further invocations can be served from cache associated with that second + * (given mtime is not updated again). * - * @return List of files matching the pattern sorted by last modification time. No recursion. No directories. - * If nothing matches then returns an empty list. If I/O issue occurred then returns the list collected to the point - * when exception was thrown. + * @return List of files matching the pattern sorted by last modification time. No recursion. + * No directories. If nothing matches then returns an empty list. If I/O issue occurred then + * returns the list collected to the point when exception was thrown. * * @see #getMatchingFilesNoCache() */ List getMatchingFiles() { - long now = TimeUnit.SECONDS.toMillis(TimeUnit.MILLISECONDS.toSeconds(System.currentTimeMillis())); + long now = TimeUnit.SECONDS.toMillis( + TimeUnit.MILLISECONDS.toSeconds(System.currentTimeMillis())); long currentParentDirMTime = parentDir.lastModified(); List result; // calculate matched files if // - we don't want to use cache (recalculate every time) OR // - directory was clearly updated after the last check OR - // - last mtime change wasn't already checked for sure (system clock hasn't passed that second yet) + // - last mtime change wasn't already checked for sure + // (system clock hasn't passed that second yet) if (!cachePatternMatching || lastSeenParentDirMTime < currentParentDirMTime || !(currentParentDirMTime < lastCheckedTime)) { @@ -199,13 +207,13 @@ List getMatchingFiles() { * files are matching the regex pattern. Each invocation uses {@link DirectoryStream} * to identify matching files. * - * Files returned by this call are weakly consistent (see {@link DirectoryStream}). It does not freeze the directory while iterating, - * so it may (or may not) reflect updates to the directory that occur during the call. In which case next call - * will return those files. + * Files returned by this call are weakly consistent (see {@link DirectoryStream}). + * It does not freeze the directory while iterating, so it may (or may not) reflect updates + * to the directory that occur during the call. In which case next call will return those files. * * @return List of files matching the pattern unsorted. No recursion. No directories. - * If nothing matches then returns an empty list. If I/O issue occurred then returns the list collected to the point - * when exception was thrown. + * If nothing matches then returns an empty list. If I/O issue occurred then returns the list + * collected to the point when exception was thrown. * * @see DirectoryStream * @see DirectoryStream.Filter @@ -217,16 +225,16 @@ private List getMatchingFilesNoCache() { result.add(entry.toFile()); } } catch (IOException e) { - logger.error("I/O exception occurred while listing parent directory. Files already matched will be returned. " + - parentDir.toPath(), e); + logger.error("I/O exception occurred while listing parent directory. " + + "Files already matched will be returned. " + parentDir.toPath(), e); } return result; } /** * Utility function to sort matched files based on last modification time. - * Sorting itself use only a snapshot of last modification times captured before the sorting to keep the - * number of stat system calls to the required minimum. + * Sorting itself use only a snapshot of last modification times captured before the sorting + * to keep the number of stat system calls to the required minimum. * * @param files list of files in any order * @return sorted list diff --git a/flume-ng-sources/flume-taildir-source/src/main/java/org/apache/flume/source/taildir/TaildirSource.java b/flume-ng-sources/flume-taildir-source/src/main/java/org/apache/flume/source/taildir/TaildirSource.java index dfb5b2901f..eae1b1a40f 100644 --- a/flume-ng-sources/flume-taildir-source/src/main/java/org/apache/flume/source/taildir/TaildirSource.java +++ b/flume-ng-sources/flume-taildir-source/src/main/java/org/apache/flume/source/taildir/TaildirSource.java @@ -150,7 +150,8 @@ public synchronized void configure(Context context) { String fileGroups = context.getString(FILE_GROUPS); Preconditions.checkState(fileGroups != null, "Missing param: " + FILE_GROUPS); - filePaths = selectByKeys(context.getSubProperties(FILE_GROUPS_PREFIX), fileGroups.split("\\s+")); + filePaths = selectByKeys(context.getSubProperties(FILE_GROUPS_PREFIX), + fileGroups.split("\\s+")); Preconditions.checkState(!filePaths.isEmpty(), "Mapping for tailing files is empty or invalid: '" + FILE_GROUPS_PREFIX + "'"); @@ -168,12 +169,13 @@ public synchronized void configure(Context context) { byteOffsetHeader = context.getBoolean(BYTE_OFFSET_HEADER, DEFAULT_BYTE_OFFSET_HEADER); idleTimeout = context.getInteger(IDLE_TIMEOUT, DEFAULT_IDLE_TIMEOUT); writePosInterval = context.getInteger(WRITE_POS_INTERVAL, DEFAULT_WRITE_POS_INTERVAL); - cachePatternMatching = context.getBoolean(CACHE_PATTERN_MATCHING, DEFAULT_CACHE_PATTERN_MATCHING); + cachePatternMatching = context.getBoolean(CACHE_PATTERN_MATCHING, + DEFAULT_CACHE_PATTERN_MATCHING); - backoffSleepIncrement = context.getLong(PollableSourceConstants.BACKOFF_SLEEP_INCREMENT - , PollableSourceConstants.DEFAULT_BACKOFF_SLEEP_INCREMENT); - maxBackOffSleepInterval = context.getLong(PollableSourceConstants.MAX_BACKOFF_SLEEP - , PollableSourceConstants.DEFAULT_MAX_BACKOFF_SLEEP); + backoffSleepIncrement = context.getLong(PollableSourceConstants.BACKOFF_SLEEP_INCREMENT, + PollableSourceConstants.DEFAULT_BACKOFF_SLEEP_INCREMENT); + maxBackOffSleepInterval = context.getLong(PollableSourceConstants.MAX_BACKOFF_SLEEP, + PollableSourceConstants.DEFAULT_MAX_BACKOFF_SLEEP); if (sourceCounter == null) { sourceCounter = new SourceCounter(getName()); @@ -254,7 +256,7 @@ private void tailFileProcess(TailFile tf, boolean backoffWithoutNL) reader.commit(); } catch (ChannelException ex) { logger.warn("The channel is full or unexpected failure. " + - "The source will try again after " + retryInterval + " ms"); + "The source will try again after " + retryInterval + " ms"); TimeUnit.MILLISECONDS.sleep(retryInterval); retryInterval = retryInterval << 1; retryInterval = Math.min(retryInterval, maxRetryInterval); @@ -320,7 +322,7 @@ private void writePosition() { String json = toPosInfoJson(); writer.write(json); } - } catch (Throwable t){ + } catch (Throwable t) { logger.error("Failed writing positionFile", t); } finally { try { diff --git a/flume-ng-sources/flume-taildir-source/src/main/java/org/apache/flume/source/taildir/TaildirSourceConfigurationConstants.java b/flume-ng-sources/flume-taildir-source/src/main/java/org/apache/flume/source/taildir/TaildirSourceConfigurationConstants.java index b0c934dd37..2c49540862 100644 --- a/flume-ng-sources/flume-taildir-source/src/main/java/org/apache/flume/source/taildir/TaildirSourceConfigurationConstants.java +++ b/flume-ng-sources/flume-taildir-source/src/main/java/org/apache/flume/source/taildir/TaildirSourceConfigurationConstants.java @@ -50,7 +50,9 @@ public class TaildirSourceConfigurationConstants { public static final String BYTE_OFFSET_HEADER_KEY = "byteoffset"; public static final boolean DEFAULT_BYTE_OFFSET_HEADER = false; - /** Whether to cache the list of files matching the specified file patterns till parent directory is modified. */ + /** Whether to cache the list of files matching the specified file patterns till parent directory + * is modified. + */ public static final String CACHE_PATTERN_MATCHING = "cachePatternMatching"; public static final boolean DEFAULT_CACHE_PATTERN_MATCHING = true; } diff --git a/flume-ng-tests/src/test/java/org/apache/flume/test/agent/TestRpcClientCommunicationFailure.java b/flume-ng-tests/src/test/java/org/apache/flume/test/agent/TestRpcClientCommunicationFailure.java index f488baee27..efb6457272 100644 --- a/flume-ng-tests/src/test/java/org/apache/flume/test/agent/TestRpcClientCommunicationFailure.java +++ b/flume-ng-tests/src/test/java/org/apache/flume/test/agent/TestRpcClientCommunicationFailure.java @@ -29,39 +29,39 @@ public class TestRpcClientCommunicationFailure { - public static final String CONFIG_FILE_PRCCLIENT_TEST = - "rpc-client-test.properties"; + public static final String CONFIG_FILE_PRCCLIENT_TEST = + "rpc-client-test.properties"; - @Test - public void testFailure() throws Exception { - try { + @Test + public void testFailure() throws Exception { + try { - StagedInstall.getInstance().startAgent( - "rpccagent", CONFIG_FILE_PRCCLIENT_TEST); - StagedInstall.waitUntilPortOpens("localhost", 12121, 20000); - RpcClient client = RpcClientFactory.getDefaultInstance( - "localhost", 12121); - String[] text = {"foo", "bar", "xyz", "abc"}; - for (String str : text) { - client.append(EventBuilder.withBody(str.getBytes())); - } + StagedInstall.getInstance().startAgent( + "rpccagent", CONFIG_FILE_PRCCLIENT_TEST); + StagedInstall.waitUntilPortOpens("localhost", 12121, 20000); + RpcClient client = RpcClientFactory.getDefaultInstance( + "localhost", 12121); + String[] text = {"foo", "bar", "xyz", "abc"}; + for (String str : text) { + client.append(EventBuilder.withBody(str.getBytes())); + } - // Stop the agent - StagedInstall.getInstance().stopAgent(); + // Stop the agent + StagedInstall.getInstance().stopAgent(); - // Try sending the event which should fail - try { - client.append(EventBuilder.withBody("test".getBytes())); - Assert.fail("EventDeliveryException expected but not raised"); - } catch (EventDeliveryException ex) { - System.out.println("Attempting to close client"); - client.close(); - } - } finally { - if (StagedInstall.getInstance().isRunning()) { - StagedInstall.getInstance().stopAgent(); - } - } - } + // Try sending the event which should fail + try { + client.append(EventBuilder.withBody("test".getBytes())); + Assert.fail("EventDeliveryException expected but not raised"); + } catch (EventDeliveryException ex) { + System.out.println("Attempting to close client"); + client.close(); + } + } finally { + if (StagedInstall.getInstance().isRunning()) { + StagedInstall.getInstance().stopAgent(); + } + } + } } diff --git a/flume-tools/src/main/java/org/apache/flume/tools/FileChannelIntegrityTool.java b/flume-tools/src/main/java/org/apache/flume/tools/FileChannelIntegrityTool.java index 7abb7ebeb8..f892d89092 100644 --- a/flume-tools/src/main/java/org/apache/flume/tools/FileChannelIntegrityTool.java +++ b/flume-tools/src/main/java/org/apache/flume/tools/FileChannelIntegrityTool.java @@ -49,8 +49,7 @@ import java.util.Set; public class FileChannelIntegrityTool implements FlumeTool { - public static final Logger LOG = LoggerFactory.getLogger - (FileChannelIntegrityTool.class); + public static final Logger LOG = LoggerFactory.getLogger(FileChannelIntegrityTool.class); private final List dataDirs = new ArrayList(); @@ -66,18 +65,18 @@ public class FileChannelIntegrityTool implements FlumeTool { @Override public void run(String[] args) throws IOException, ParseException { boolean shouldContinue = parseCommandLineOpts(args); - if(!shouldContinue) { + if (!shouldContinue) { LOG.error("Could not parse command line options. Exiting ..."); System.exit(1); } - for(File dataDir : dataDirs) { + for (File dataDir : dataDirs) { File[] dataFiles = dataDir.listFiles(new FilenameFilter() { @Override public boolean accept(File dir, String name) { - if(!name.endsWith(Serialization.METADATA_FILENAME) - && !name.endsWith(Serialization.METADATA_TMP_FILENAME) - && !name.endsWith(Serialization.OLD_METADATA_FILENAME) - && !name.equals(Log.FILE_LOCK)) { + if (!name.endsWith(Serialization.METADATA_FILENAME) + && !name.endsWith(Serialization.METADATA_TMP_FILENAME) + && !name.endsWith(Serialization.OLD_METADATA_FILENAME) + && !name.equals(Log.FILE_LOCK)) { return true; } return false; @@ -86,10 +85,8 @@ public boolean accept(File dir, String name) { if (dataFiles != null && dataFiles.length > 0) { for (File dataFile : dataFiles) { LOG.info("Checking for corruption in " + dataFile.toString()); - LogFile.SequentialReader reader = - new LogFileV3.SequentialReader(dataFile, null, true); - LogFile.OperationRecordUpdater updater = new LogFile - .OperationRecordUpdater(dataFile); + LogFile.SequentialReader reader = new LogFileV3.SequentialReader(dataFile, null, true); + LogFile.OperationRecordUpdater updater = new LogFile.OperationRecordUpdater(dataFile); boolean fileDone = false; boolean fileBackedup = false; while (!fileDone) { @@ -106,7 +103,7 @@ public boolean accept(File dir, String name) { if (record != null) { TransactionEventRecord recordEvent = record.getEvent(); Event event = EventUtils.getEventFromTransactionEvent(recordEvent); - if(event != null) { + if (event != null) { totalPutEvents++; try { if (!eventValidator.validateEvent(event)) { @@ -124,7 +121,8 @@ public boolean accept(File dir, String name) { // OOPS, didn't expected an exception // considering as failure case // marking as noop - System.err.println("Encountered Exception while validating event, marking as invalid"); + System.err.println("Encountered Exception while validating event, " + + "marking as invalid"); updater.markRecordAsNoop(eventPosition); eventsWithException++; } @@ -135,11 +133,10 @@ public boolean accept(File dir, String name) { } catch (CorruptEventException e) { corruptEvents++; totalChannelEvents++; - LOG.warn("Corruption found in " + dataFile.toString() + " at " - + eventPosition); + LOG.warn("Corruption found in " + dataFile.toString() + " at " + eventPosition); if (!fileBackedup) { Serialization.copyFile(dataFile, new File(dataFile.getParent(), - dataFile.getName() + ".bak")); + dataFile.getName() + ".bak")); fileBackedup = true; } updater.markRecordAsNoop(eventPosition); @@ -155,49 +152,46 @@ public boolean accept(File dir, String name) { private boolean parseCommandLineOpts(String[] args) throws ParseException { Options options = new Options(); - options - .addOption("l", "dataDirs", true, "Comma-separated list of data " + - "directories which the tool must verify. This option is mandatory") - .addOption("h", "help", false, "Display help") - .addOption("e", "eventValidator", true, "Fully Qualified Name of Event Validator Implementation");; + options.addOption("l", "dataDirs", true, "Comma-separated list of data " + + "directories which the tool must verify. This option is mandatory") + .addOption("h", "help", false, "Display help") + .addOption("e", "eventValidator", true, + "Fully Qualified Name of Event Validator Implementation"); - - Option property = OptionBuilder.withArgName("property=value") + Option property = OptionBuilder.withArgName("property=value") .hasArgs(2) .withValueSeparator() - .withDescription( "custom properties" ) - .create( "D" ); + .withDescription("custom properties") + .create("D"); options.addOption(property); CommandLineParser parser = new GnuParser(); CommandLine commandLine = parser.parse(options, args); - if(commandLine.hasOption("help")) { - new HelpFormatter().printHelp("bin/flume-ng tool fcintegritytool ", - options, true); + if (commandLine.hasOption("help")) { + new HelpFormatter().printHelp("bin/flume-ng tool fcintegritytool ", options, true); return false; } - if(!commandLine.hasOption("dataDirs")) { + if (!commandLine.hasOption("dataDirs")) { new HelpFormatter().printHelp("bin/flume-ng tool fcintegritytool ", "", - options, "dataDirs is required.", true); + options, "dataDirs is required.", true); return false; } else { - String dataDirStr[] = commandLine.getOptionValue("dataDirs").split(","); - for(String dataDir : dataDirStr) { + String[] dataDirStr = commandLine.getOptionValue("dataDirs").split(","); + for (String dataDir : dataDirStr) { File f = new File(dataDir); - if(!f.exists()) { - throw new FlumeException("Data directory, " + dataDir + " does not " + - "exist."); + if (!f.exists()) { + throw new FlumeException("Data directory, " + dataDir + " does not exist."); } dataDirs.add(f); } } - if(commandLine.hasOption("eventValidator")) { + if (commandLine.hasOption("eventValidator")) { try { Class eventValidatorClassName = - (Class)Class.forName( - commandLine.getOptionValue("eventValidator")); + (Class)Class.forName( + commandLine.getOptionValue("eventValidator")); EventValidator.Builder eventValidatorBuilder = eventValidatorClassName.newInstance(); // Pass on the configuration parameter @@ -226,12 +220,13 @@ private boolean parseCommandLineOpts(String[] args) throws ParseException { */ private void printSummary() { System.out.println("---------- Summary --------------------"); - System.out.println("Number of Events in the Channel = "+totalChannelEvents++); - System.out.println("Number of Put Events Processed = "+totalPutEvents); - System.out.println("Number of Valid Put Events = "+validEvents); - System.out.println("Number of Invalid Put Events = "+invalidEvents); - System.out.println("Number of Put Events that threw Exception during validation = "+eventsWithException); - System.out.println("Number of Corrupt Events = "+corruptEvents); + System.out.println("Number of Events in the Channel = " + totalChannelEvents++); + System.out.println("Number of Put Events Processed = " + totalPutEvents); + System.out.println("Number of Valid Put Events = " + validEvents); + System.out.println("Number of Invalid Put Events = " + invalidEvents); + System.out.println("Number of Put Events that threw Exception during validation = " + + eventsWithException); + System.out.println("Number of Corrupt Events = " + corruptEvents); System.out.println("---------------------------------------"); } } diff --git a/flume-tools/src/main/java/org/apache/flume/tools/FlumeToolType.java b/flume-tools/src/main/java/org/apache/flume/tools/FlumeToolType.java index 1bb122dcee..e9ba27d39a 100644 --- a/flume-tools/src/main/java/org/apache/flume/tools/FlumeToolType.java +++ b/flume-tools/src/main/java/org/apache/flume/tools/FlumeToolType.java @@ -34,7 +34,7 @@ public Class getClassInstance() { public static String getNames() { StringBuilder builder = new StringBuilder(); - for(FlumeToolType type: values()) { + for (FlumeToolType type: values()) { builder.append(type.name().toLowerCase(Locale.ENGLISH) + "\n"); } return builder.toString(); diff --git a/flume-tools/src/main/java/org/apache/flume/tools/FlumeToolsMain.java b/flume-tools/src/main/java/org/apache/flume/tools/FlumeToolsMain.java index 799ce8574c..426f55f031 100644 --- a/flume-tools/src/main/java/org/apache/flume/tools/FlumeToolsMain.java +++ b/flume-tools/src/main/java/org/apache/flume/tools/FlumeToolsMain.java @@ -36,33 +36,32 @@ private FlumeToolsMain() { } @Override - public void run(String[] args) throws Exception{ + public void run(String[] args) throws Exception { String error = "Expected name of tool and arguments for" + - " tool to be passed in on the command line. Please pass one of the " + - "following as arguments to this command: \n"; + " tool to be passed in on the command line. Please pass one of the " + + "following as arguments to this command: \n"; StringBuilder builder = new StringBuilder(error); - for(FlumeToolType type : FlumeToolType.values()) { + for (FlumeToolType type : FlumeToolType.values()) { builder.append(type.name()).append("\n"); } - if(args == null || args.length == 0) { + if (args == null || args.length == 0) { System.out.println(builder.toString()); System.exit(1); } String toolName = args[0]; FlumeTool tool = null; - for(FlumeToolType type : FlumeToolType.values()) { - if(toolName.equalsIgnoreCase(type.name())) { + for (FlumeToolType type : FlumeToolType.values()) { + if (toolName.equalsIgnoreCase(type.name())) { tool = type.getClassInstance().newInstance(); break; } } Preconditions.checkNotNull(tool, "Cannot find tool matching " + toolName - + ". Please select one of: \n " + FlumeToolType.getNames()); + + ". Please select one of: \n " + FlumeToolType.getNames()); if (args.length == 1) { tool.run(new String[0]); } else { - tool.run(Arrays.asList(args).subList(1, args.length). - toArray(new String[0])); + tool.run(Arrays.asList(args).subList(1, args.length).toArray(new String[0])); } } } diff --git a/pom.xml b/pom.xml index 99543534a9..85c0dc8a64 100644 --- a/pom.xml +++ b/pom.xml @@ -47,6 +47,8 @@ limitations under the License. ${project.basedir}/target/docs 1.7.4 + 2.17 + 6.19 0.90.1 2.4.0 0.7.0 @@ -58,6 +60,7 @@ limitations under the License. + flume-checkstyle flume-ng-core flume-ng-configuration flume-ng-embedded-agent @@ -638,6 +641,49 @@ limitations under the License. apache-rat-plugin + + + org.apache.maven.plugins + maven-checkstyle-plugin + ${checkstyle.plugin.version} + + + + org.apache.flume + flume-checkstyle + ${project.version} + + + + com.puppycrawl.tools + checkstyle + ${checkstyle.tool.version} + + + + + verify + verify + + flume/checkstyle.xml + flume/checkstyle-suppressions.xml + checkstyle.suppressions.file + UTF-8 + true + true + false + false + + + check + + + + + @@ -776,6 +822,7 @@ limitations under the License. + junit junit @@ -1051,6 +1098,11 @@ limitations under the License. 1.7.0-SNAPSHOT + + org.apache.flume + flume-checkstyle + 1.7.0-SNAPSHOT + org.apache.flume flume-tools @@ -1413,6 +1465,20 @@ limitations under the License. + + org.apache.maven.plugins + maven-checkstyle-plugin + ${checkstyle.plugin.version} + + flume/checkstyle.xml + flume/checkstyle-suppressions.xml + checkstyle.suppressions.file + UTF-8 + false + false + + + From c8c0f9b8404ef60222f49aa91f74f5a846b57ca7 Mon Sep 17 00:00:00 2001 From: Mike Percy Date: Thu, 30 Jun 2016 10:27:54 -0700 Subject: [PATCH 296/341] Minor README tweaks --- README.md | 6 ++---- 1 file changed, 2 insertions(+), 4 deletions(-) diff --git a/README.md b/README.md index 790890d1a0..7198a65bc6 100644 --- a/README.md +++ b/README.md @@ -63,7 +63,5 @@ We recommend you set the following Maven options: export MAVEN_OPTS="-Xms512m -Xmx1024m -XX:PermSize=256m -XX:MaxPermSize=512m" -To compile Flume, run `mvn compile`. -To build a distribution, run `mvn install`. - -The final Flume distribution artifacts will be in $project/flume-ng-dist/target/. +To compile Flume and build a distribution tarball, run `mvn install` from the +top level directory. The artifacts will be placed under `flume-ng-dist/target/`. From cfbf1156858af9ae26975fefc94594d91c8cd3f4 Mon Sep 17 00:00:00 2001 From: Mike Percy Date: Wed, 29 Jun 2016 21:18:20 -0700 Subject: [PATCH 297/341] FLUME-2941. Integrate checkstyle for test classes Also make test code conform to style guidelines. Additionally, this patch makes style violations fatal to the build. This patch is whitespace-only from a code perspective. After stripping line numbers, the generated test bytecode before and after these changes is identical. Code review: https://reviews.apache.org/r/49830/ Reviewed by Hari. --- flume-checkstyle/pom.xml | 8 - .../flume/checkstyle-suppressions.xml | 18 +- .../src/main/resources/flume/checkstyle.xml | 2 +- .../flume/auth/TestFlumeAuthenticator.java | 16 +- .../channel/file/CountingSinkRunner.java | 22 +- .../channel/file/CountingSourceRunner.java | 21 +- .../flume/channel/file/TestCheckpoint.java | 4 +- .../TestEventQueueBackingStoreFactory.java | 140 +++---- .../flume/channel/file/TestEventUtils.java | 4 +- .../flume/channel/file/TestFileChannel.java | 139 ++++--- .../file/TestFileChannelFormatRegression.java | 21 +- .../channel/file/TestFileChannelRestart.java | 208 +++++----- .../channel/file/TestFileChannelRollback.java | 16 +- .../channel/file/TestFlumeEventQueue.java | 200 +++++---- .../flume/channel/file/TestIntegration.java | 33 +- .../apache/flume/channel/file/TestLog.java | 308 ++++++++------ .../flume/channel/file/TestLogFile.java | 153 ++++--- .../file/TestTransactionEventRecordV2.java | 10 +- .../file/TestTransactionEventRecordV3.java | 34 +- .../apache/flume/channel/file/TestUtils.java | 122 +++--- .../encryption/CipherProviderTestSuite.java | 5 + .../file/encryption/EncryptionTestUtils.java | 95 +++-- .../TestAESCTRNoPaddingProvider.java | 10 +- .../encryption/TestFileChannelEncryption.java | 98 +++-- .../encryption/TestJCEFileKeyProvider.java | 69 ++-- .../jdbc/BaseJdbcChannelProviderTest.java | 28 +- .../apache/flume/channel/jdbc/MockEvent.java | 3 +- .../flume/channel/jdbc/MockEventUtils.java | 22 +- .../jdbc/TestDerbySchemaHandlerQueries.java | 2 - .../flume/channel/kafka/TestKafkaChannel.java | 67 +-- .../channel/TestSpillableMemoryChannel.java | 386 ++++++++---------- .../TestLoadBalancingLog4jAppender.java | 51 ++- .../log4jappender/TestLog4jAppender.java | 32 +- .../TestLog4jAppenderWithAvro.java | 5 +- .../AbstractBasicChannelSemanticsTest.java | 7 +- .../flume/channel/TestChannelProcessor.java | 32 +- .../flume/channel/TestMemoryChannel.java | 40 +- .../channel/TestMemoryChannelConcurrency.java | 100 ++--- .../channel/TestMemoryChannelTransaction.java | 12 +- .../TestReliableSpoolingFileEventReader.java | 201 +++++---- .../formatter/output/TestBucketPath.java | 52 +-- .../TestMonitoredCounterGroup.java | 50 +-- .../http/TestHTTPMetricsServer.java | 24 +- .../kafka/KafkaSourceCounterTest.java | 78 ++-- ...xExtractorInterceptorMillisSerializer.java | 10 +- ...actorInterceptorPassThroughSerializer.java | 3 +- .../TestSearchAndReplaceInterceptor.java | 2 +- .../SyslogAvroEventSerializer.java | 45 +- .../TestAvroEventDeserializer.java | 1 + .../TestDurablePositionTracker.java | 3 +- .../TestFlumeEventAvroEventSerializer.java | 24 +- .../TestResettableFileInputStream.java | 26 +- .../TestSyslogAvroEventSerializer.java | 2 +- .../org/apache/flume/sink/TestAvroSink.java | 87 ++-- .../flume/sink/TestDefaultSinkFactory.java | 3 +- .../flume/sink/TestFailoverSinkProcessor.java | 10 +- .../sink/TestLoadBalancingSinkProcessor.java | 60 ++- .../flume/sink/TestRollingFileSink.java | 30 +- .../org/apache/flume/sink/TestThriftSink.java | 11 +- .../source/TestAbstractPollableSource.java | 24 +- .../apache/flume/source/TestAvroSource.java | 34 +- .../apache/flume/source/TestExecSource.java | 336 ++++++++------- .../source/TestMultiportSyslogTCPSource.java | 1 + .../apache/flume/source/TestNetcatSource.java | 79 ++-- .../source/TestSequenceGeneratorSource.java | 8 +- .../source/TestSpoolDirectorySource.java | 42 +- .../apache/flume/source/TestStressSource.java | 2 +- .../apache/flume/source/TestSyslogParser.java | 12 +- .../flume/source/TestSyslogTcpSource.java | 12 +- .../flume/source/TestSyslogUdpSource.java | 19 +- .../apache/flume/source/TestSyslogUtils.java | 162 ++++---- .../apache/flume/source/TestThriftSource.java | 29 +- .../http/FlumeHttpServletRequestWrapper.java | 4 +- .../flume/source/http/TestHTTPSource.java | 153 +++---- .../tools/TestTimestampRoundDownUtil.java | 10 +- .../apache/flume/tools/TestVersionInfo.java | 4 +- .../agent/embedded/TestEmbeddedAgent.java | 24 +- .../TestEmbeddedAgentConfiguration.java | 18 +- .../TestEmbeddedAgentEmbeddedSource.java | 15 +- .../embedded/TestEmbeddedAgentState.java | 30 +- .../avroLegacy/TestLegacyAvroSource.java | 27 +- .../thriftLegacy/TestThriftLegacySource.java | 30 +- .../TestAbstractConfigurationProvider.java | 71 ++-- ...bstractZooKeeperConfigurationProvider.java | 25 +- .../apache/flume/node/TestApplication.java | 2 +- ...ngPropertiesFileConfigurationProvider.java | 1 - ...stPropertiesFileConfigurationProvider.java | 24 +- .../apache/flume/source/TestNetcatSource.java | 38 +- .../org/apache/flume/api/RpcTestUtils.java | 77 ++-- .../flume/api/TestFailoverRpcClient.java | 4 +- .../flume/api/TestLoadBalancingRpcClient.java | 107 ++--- .../flume/api/TestNettyAvroRpcClient.java | 26 +- .../apache/flume/api/TestThriftRpcClient.java | 41 +- .../apache/flume/api/ThriftTestingSource.java | 27 +- .../flume/sink/kite/TestDatasetSink.java | 79 ++-- .../flume/sink/hdfs/HDFSTestSeqWriter.java | 16 +- .../flume/sink/hdfs/MockDataStream.java | 4 +- .../flume/sink/hdfs/MockFileSystem.java | 23 +- .../sink/hdfs/MockFsDataOutputStream.java | 12 +- .../flume/sink/hdfs/MockHDFSWriter.java | 3 +- .../flume/sink/hdfs/TestBucketWriter.java | 274 ++++++------- .../flume/sink/hdfs/TestHDFSEventSink.java | 127 +++--- .../TestSequenceFileSerializerFactory.java | 3 +- .../apache/flume/sink/hive/TestHiveSink.java | 21 +- .../flume/sink/hive/TestHiveWriter.java | 50 ++- .../org/apache/flume/sink/hive/TestUtil.java | 33 +- .../apache/flume/sink/irc/TestIRCSink.java | 11 +- .../AbstractElasticSearchSinkTest.java | 25 +- ...asticSearchIndexRequestBuilderFactory.java | 30 +- ...tElasticSearchLogStashEventSerializer.java | 82 ++-- .../elasticsearch/TestElasticSearchSink.java | 52 +-- .../TestElasticSearchSinkCreation.java | 2 +- .../client/RoundRobinListTest.java | 3 +- .../TestElasticSearchClientFactory.java | 10 +- .../client/TestElasticSearchRestClient.java | 35 +- .../hbase/IncrementAsyncHBaseSerializer.java | 6 +- .../flume/sink/hbase/TestAsyncHBaseSink.java | 114 +++--- .../flume/sink/hbase/TestHBaseSink.java | 129 +++--- .../hbase/TestRegexHbaseEventSerializer.java | 37 +- .../flume/sink/kafka/TestKafkaSink.java | 89 ++-- .../flume/sink/kafka/util/KafkaLocal.java | 57 ++- .../flume/sink/kafka/util/ZooKeeperLocal.java | 75 ++-- .../solr/morphline/TestBlobDeserializer.java | 10 +- .../morphline/TestMorphlineInterceptor.java | 57 +-- .../solr/morphline/TestMorphlineSolrSink.java | 19 +- .../jms/JMSMessageConsumerTestBase.java | 13 +- .../jms/TestDefaultJMSMessageConverter.java | 2 +- .../source/jms/TestIntegrationActiveMQ.java | 23 +- .../source/jms/TestJMSMessageConsumer.java | 3 +- .../flume/source/jms/TestJMSSource.java | 53 ++- .../kafka/KafkaSourceEmbeddedKafka.java | 11 +- .../kafka/KafkaSourceEmbeddedZookeeper.java | 1 - .../flume/source/kafka/TestKafkaSource.java | 152 ++++--- .../taildir/TestTaildirEventReader.java | 36 +- .../source/taildir/TestTaildirMatcher.java | 57 ++- .../source/taildir/TestTaildirSource.java | 38 +- .../flume/test/agent/TestFileChannel.java | 178 ++++---- .../apache/flume/test/util/StagedInstall.java | 39 +- .../apache/flume/test/util/SyslogAgent.java | 9 +- .../tools/TestFileChannelIntegrityTool.java | 54 ++- pom.xml | 6 +- 141 files changed, 3523 insertions(+), 3423 deletions(-) diff --git a/flume-checkstyle/pom.xml b/flume-checkstyle/pom.xml index 31db3c0f2b..74ebf6b0d1 100644 --- a/flume-checkstyle/pom.xml +++ b/flume-checkstyle/pom.xml @@ -21,14 +21,6 @@ xsi:schemaLocation="http://maven.apache.org/POM/4.0.0 http://maven.apache.org/xsd/maven-4.0.0.xsd"> 4.0.0 - - org.apache.flume flume-checkstyle Flume checkstyle project diff --git a/flume-checkstyle/src/main/resources/flume/checkstyle-suppressions.xml b/flume-checkstyle/src/main/resources/flume/checkstyle-suppressions.xml index 49c88347fb..2642baa873 100644 --- a/flume-checkstyle/src/main/resources/flume/checkstyle-suppressions.xml +++ b/flume-checkstyle/src/main/resources/flume/checkstyle-suppressions.xml @@ -12,21 +12,29 @@ + + + + files="channel/file|RpcClientFactory\.java|BucketPath\.java|SinkGroup\.java|DefaultSinkProcessor\.java|RegexExtractorInterceptorMillisSerializer\.java|SimpleAsyncHbaseEventSerializer\.java|hdfs/BucketWriter\.java|AbstractBasicChannelSemanticsTest\.java"/> + + + + files="SyslogUtils\.java|ReliableTaildirEventReader\.java|AbstractBasicChannelSemanticsTest\.java"/> + files="channel/file/LogFile\.java|TestDatasetSink\.java|CountingSourceRunner\.java|CountingSinkRunner\.java|TestKafkaChannel\.java|TestTaildirSource\.java|TestChannelProcessor\.java|TestHiveSink\.java|AbstractBasicChannelSemanticsTest\.java|TestJMSSource\.java|TestEmbeddedAgent\.java|TestAsyncHBaseSink\.java"/> + files="channel/MemoryChannel\.java|ReliableSpoolingFileEventReader\.java|TestAvroSink\.java"/> + files="KafkaSource\.java|KafkaChannel\.java|KafkaSink\.java|TestElasticSearchSink\.java"/> diff --git a/flume-checkstyle/src/main/resources/flume/checkstyle.xml b/flume-checkstyle/src/main/resources/flume/checkstyle.xml index e8913f0d8d..fdbcb5dae0 100644 --- a/flume-checkstyle/src/main/resources/flume/checkstyle.xml +++ b/flume-checkstyle/src/main/resources/flume/checkstyle.xml @@ -18,7 +18,7 @@ - + diff --git a/flume-ng-auth/src/test/java/org/apache/flume/auth/TestFlumeAuthenticator.java b/flume-ng-auth/src/test/java/org/apache/flume/auth/TestFlumeAuthenticator.java index 5a8860de09..0dc88724e6 100644 --- a/flume-ng-auth/src/test/java/org/apache/flume/auth/TestFlumeAuthenticator.java +++ b/flume-ng-auth/src/test/java/org/apache/flume/auth/TestFlumeAuthenticator.java @@ -17,15 +17,19 @@ */ package org.apache.flume.auth; -import java.io.File; -import java.io.IOException; -import java.util.Properties; - import org.apache.hadoop.minikdc.MiniKdc; import org.junit.AfterClass; import org.junit.BeforeClass; import org.junit.Test; -import static org.junit.Assert.*; + +import java.io.File; +import java.io.IOException; +import java.util.Properties; + +import static org.junit.Assert.assertEquals; +import static org.junit.Assert.assertFalse; +import static org.junit.Assert.assertTrue; +import static org.junit.Assert.fail; public class TestFlumeAuthenticator { @@ -132,7 +136,7 @@ public void testFlumeLoginPrincipalWithoutRealm() throws Exception { String principal = "flume"; File keytab = new File(workDir, "flume2.keytab"); kdc.createPrincipal(keytab, principal); - String expResult = principal+"@" + kdc.getRealm(); + String expResult = principal + "@" + kdc.getRealm(); // Clear the previous statically stored logged in credentials FlumeAuthenticationUtil.clearCredentials(); diff --git a/flume-ng-channels/flume-file-channel/src/test/java/org/apache/flume/channel/file/CountingSinkRunner.java b/flume-ng-channels/flume-file-channel/src/test/java/org/apache/flume/channel/file/CountingSinkRunner.java index 0733dc4269..a303994f3c 100644 --- a/flume-ng-channels/flume-file-channel/src/test/java/org/apache/flume/channel/file/CountingSinkRunner.java +++ b/flume-ng-channels/flume-file-channel/src/test/java/org/apache/flume/channel/file/CountingSinkRunner.java @@ -18,11 +18,10 @@ */ package org.apache.flume.channel.file; -import java.util.List; - +import com.google.common.collect.Lists; import org.apache.flume.Sink; -import com.google.common.collect.Lists; +import java.util.List; public class CountingSinkRunner extends Thread { private int count; @@ -30,39 +29,46 @@ public class CountingSinkRunner extends Thread { private final Sink sink; private volatile boolean run; private final List errors = Lists.newArrayList(); + public CountingSinkRunner(Sink sink) { this(sink, Integer.MAX_VALUE); } + public CountingSinkRunner(Sink sink, int until) { this.sink = sink; this.until = until; } + @Override public void run() { run = true; - while(run && count < until) { + while (run && count < until) { boolean error = true; try { - if(Sink.Status.READY.equals(sink.process())) { + if (Sink.Status.READY.equals(sink.process())) { count++; error = false; } - } catch(Exception ex) { + } catch (Exception ex) { errors.add(ex); } - if(error) { + if (error) { try { Thread.sleep(1000L); - } catch (InterruptedException e) {} + } catch (InterruptedException e) { + } } } } + public void shutdown() { run = false; } + public int getCount() { return count; } + public List getErrors() { return errors; } diff --git a/flume-ng-channels/flume-file-channel/src/test/java/org/apache/flume/channel/file/CountingSourceRunner.java b/flume-ng-channels/flume-file-channel/src/test/java/org/apache/flume/channel/file/CountingSourceRunner.java index b6abc350ed..1119990ab1 100644 --- a/flume-ng-channels/flume-file-channel/src/test/java/org/apache/flume/channel/file/CountingSourceRunner.java +++ b/flume-ng-channels/flume-file-channel/src/test/java/org/apache/flume/channel/file/CountingSourceRunner.java @@ -33,19 +33,23 @@ public class CountingSourceRunner extends Thread { private final PollableSource source; private volatile boolean run; private final List errors = Lists.newArrayList(); + public CountingSourceRunner(PollableSource source) { this(source, Integer.MAX_VALUE); } + public CountingSourceRunner(PollableSource source, int until) { this(source, until, null); } + public CountingSourceRunner(PollableSource source, Channel channel) { this(source, Integer.MAX_VALUE, channel); } + public CountingSourceRunner(PollableSource source, int until, Channel channel) { this.source = source; this.until = until; - if(channel != null) { + if (channel != null) { ReplicatingChannelSelector selector = new ReplicatingChannelSelector(); List channels = Lists.newArrayList(); channels.add(channel); @@ -53,32 +57,37 @@ public CountingSourceRunner(PollableSource source, int until, Channel channel) { this.source.setChannelProcessor(new ChannelProcessor(selector)); } } + @Override public void run() { run = true; - while(run && count < until) { + while (run && count < until) { boolean error = true; try { - if(PollableSource.Status.READY.equals(source.process())) { + if (PollableSource.Status.READY.equals(source.process())) { count++; error = false; } - } catch(Exception ex) { + } catch (Exception ex) { errors.add(ex); } - if(error) { + if (error) { try { Thread.sleep(1000L); - } catch (InterruptedException e) {} + } catch (InterruptedException e) { + } } } } + public void shutdown() { run = false; } + public int getCount() { return count; } + public List getErrors() { return errors; } diff --git a/flume-ng-channels/flume-file-channel/src/test/java/org/apache/flume/channel/file/TestCheckpoint.java b/flume-ng-channels/flume-file-channel/src/test/java/org/apache/flume/channel/file/TestCheckpoint.java index c1de12e350..cd1dcd910d 100644 --- a/flume-ng-channels/flume-file-channel/src/test/java/org/apache/flume/channel/file/TestCheckpoint.java +++ b/flume-ng-channels/flume-file-channel/src/test/java/org/apache/flume/channel/file/TestCheckpoint.java @@ -28,11 +28,11 @@ import org.junit.Test; public class TestCheckpoint { - File file; File inflightPuts; File inflightTakes; File queueSet; + @Before public void setup() throws IOException { file = File.createTempFile("Checkpoint", ""); @@ -42,10 +42,12 @@ public void setup() throws IOException { Assert.assertTrue(file.isFile()); Assert.assertTrue(file.canWrite()); } + @After public void cleanup() { file.delete(); } + @Test public void testSerialization() throws Exception { EventQueueBackingStore backingStore = diff --git a/flume-ng-channels/flume-file-channel/src/test/java/org/apache/flume/channel/file/TestEventQueueBackingStoreFactory.java b/flume-ng-channels/flume-file-channel/src/test/java/org/apache/flume/channel/file/TestEventQueueBackingStoreFactory.java index 52c706d59a..0939454bc1 100644 --- a/flume-ng-channels/flume-file-channel/src/test/java/org/apache/flume/channel/file/TestEventQueueBackingStoreFactory.java +++ b/flume-ng-channels/flume-file-channel/src/test/java/org/apache/flume/channel/file/TestEventQueueBackingStoreFactory.java @@ -18,31 +18,29 @@ */ package org.apache.flume.channel.file; -import java.io.DataInputStream; -import java.io.File; -import java.io.FileInputStream; -import java.io.IOException; -import java.util.Arrays; -import java.util.Collections; -import java.util.List; - +import com.google.common.collect.Lists; +import com.google.common.io.Files; +import com.google.protobuf.InvalidProtocolBufferException; import junit.framework.Assert; - import org.apache.commons.io.FileUtils; +import org.apache.flume.channel.file.proto.ProtosFactory; import org.junit.After; import org.junit.Before; import org.junit.Test; -import com.google.common.collect.Lists; -import com.google.common.io.Files; -import com.google.protobuf.InvalidProtocolBufferException; +import java.io.DataInputStream; +import java.io.File; +import java.io.FileInputStream; import java.io.FileOutputStream; +import java.io.IOException; import java.io.RandomAccessFile; +import java.util.Arrays; +import java.util.Collections; +import java.util.List; import java.util.Random; -import org.apache.flume.channel.file.proto.ProtosFactory; public class TestEventQueueBackingStoreFactory { - static final List pointersInTestCheckpoint = Arrays.asList(new Long[] { + static final List pointersInTestCheckpoint = Arrays.asList(new Long[]{ 8589936804L, 4294969563L, 12884904153L, @@ -59,6 +57,7 @@ public class TestEventQueueBackingStoreFactory { File inflightTakes; File inflightPuts; File queueSetDir; + @Before public void setup() throws IOException { baseDir = Files.createTempDir(); @@ -67,42 +66,46 @@ public void setup() throws IOException { inflightPuts = new File(baseDir, "puts"); queueSetDir = new File(baseDir, "queueset"); TestUtils.copyDecompressed("fileformat-v2-checkpoint.gz", checkpoint); - } + @After public void teardown() { FileUtils.deleteQuietly(baseDir); } + @Test public void testWithNoFlag() throws Exception { verify(EventQueueBackingStoreFactory.get(checkpoint, 10, "test"), - Serialization.VERSION_3, pointersInTestCheckpoint); + Serialization.VERSION_3, pointersInTestCheckpoint); } + @Test public void testWithFlag() throws Exception { verify(EventQueueBackingStoreFactory.get(checkpoint, 10, "test", true), - Serialization.VERSION_3, pointersInTestCheckpoint); + Serialization.VERSION_3, pointersInTestCheckpoint); } + @Test public void testNoUprade() throws Exception { verify(EventQueueBackingStoreFactory.get(checkpoint, 10, "test", false), - Serialization.VERSION_2, pointersInTestCheckpoint); + Serialization.VERSION_2, pointersInTestCheckpoint); } - @Test (expected = BadCheckpointException.class) + + @Test(expected = BadCheckpointException.class) public void testDecreaseCapacity() throws Exception { Assert.assertTrue(checkpoint.delete()); - EventQueueBackingStore backingStore = EventQueueBackingStoreFactory. - get(checkpoint, 10, "test"); + EventQueueBackingStore backingStore = + EventQueueBackingStoreFactory.get(checkpoint, 10, "test"); backingStore.close(); EventQueueBackingStoreFactory.get(checkpoint, 9, "test"); Assert.fail(); } - @Test (expected = BadCheckpointException.class) + @Test(expected = BadCheckpointException.class) public void testIncreaseCapacity() throws Exception { Assert.assertTrue(checkpoint.delete()); - EventQueueBackingStore backingStore = EventQueueBackingStoreFactory. - get(checkpoint, 10, "test"); + EventQueueBackingStore backingStore = + EventQueueBackingStoreFactory.get(checkpoint, 10, "test"); backingStore.close(); EventQueueBackingStoreFactory.get(checkpoint, 11, "test"); Assert.fail(); @@ -112,22 +115,21 @@ public void testIncreaseCapacity() throws Exception { public void testNewCheckpoint() throws Exception { Assert.assertTrue(checkpoint.delete()); verify(EventQueueBackingStoreFactory.get(checkpoint, 10, "test", false), - Serialization.VERSION_3, Collections.emptyList()); + Serialization.VERSION_3, Collections.emptyList()); } - @Test (expected = BadCheckpointException.class) + @Test(expected = BadCheckpointException.class) public void testCheckpointBadVersion() throws Exception { - RandomAccessFile writer = new RandomAccessFile(checkpoint, "rw"); + RandomAccessFile writer = new RandomAccessFile(checkpoint, "rw"); try { - EventQueueBackingStore backingStore = EventQueueBackingStoreFactory. - get(checkpoint, 10, "test"); - backingStore.close(); - writer.seek( - EventQueueBackingStoreFile.INDEX_VERSION * Serialization.SIZE_OF_LONG); - writer.writeLong(94L); - writer.getFD().sync(); + EventQueueBackingStore backingStore = + EventQueueBackingStoreFactory.get(checkpoint, 10, "test"); + backingStore.close(); + writer.seek(EventQueueBackingStoreFile.INDEX_VERSION * Serialization.SIZE_OF_LONG); + writer.writeLong(94L); + writer.getFD().sync(); - backingStore = EventQueueBackingStoreFactory.get(checkpoint, 10, "test"); + backingStore = EventQueueBackingStoreFactory.get(checkpoint, 10, "test"); } finally { writer.close(); } @@ -138,15 +140,13 @@ public void testIncompleteCheckpoint() throws Exception { RandomAccessFile writer = new RandomAccessFile(checkpoint, "rw"); try { - EventQueueBackingStore backingStore = EventQueueBackingStoreFactory. - get(checkpoint, 10, "test"); - backingStore.close(); - writer.seek( - EventQueueBackingStoreFile.INDEX_CHECKPOINT_MARKER * - Serialization.SIZE_OF_LONG); - writer.writeLong(EventQueueBackingStoreFile.CHECKPOINT_INCOMPLETE); - writer.getFD().sync(); - backingStore = EventQueueBackingStoreFactory.get(checkpoint, 10, "test"); + EventQueueBackingStore backingStore = + EventQueueBackingStoreFactory.get(checkpoint, 10, "test"); + backingStore.close(); + writer.seek(EventQueueBackingStoreFile.INDEX_CHECKPOINT_MARKER * Serialization.SIZE_OF_LONG); + writer.writeLong(EventQueueBackingStoreFile.CHECKPOINT_INCOMPLETE); + writer.getFD().sync(); + backingStore = EventQueueBackingStoreFactory.get(checkpoint, 10, "test"); } finally { writer.close(); } @@ -156,12 +156,10 @@ public void testIncompleteCheckpoint() throws Exception { public void testCheckpointVersionNotEqualToMeta() throws Exception { RandomAccessFile writer = new RandomAccessFile(checkpoint, "rw"); try { - EventQueueBackingStore backingStore = EventQueueBackingStoreFactory. - get(checkpoint, 10, "test"); + EventQueueBackingStore backingStore = + EventQueueBackingStoreFactory.get(checkpoint, 10, "test"); backingStore.close(); - writer.seek( - EventQueueBackingStoreFile.INDEX_VERSION - * Serialization.SIZE_OF_LONG); + writer.seek(EventQueueBackingStoreFile.INDEX_VERSION * Serialization.SIZE_OF_LONG); writer.writeLong(2L); writer.getFD().sync(); backingStore = EventQueueBackingStoreFactory.get(checkpoint, 10, "test"); @@ -174,8 +172,8 @@ public void testCheckpointVersionNotEqualToMeta() throws Exception { public void testCheckpointVersionNotEqualToMeta2() throws Exception { FileOutputStream os = null; try { - EventQueueBackingStore backingStore = EventQueueBackingStoreFactory. - get(checkpoint, 10, "test"); + EventQueueBackingStore backingStore = + EventQueueBackingStoreFactory.get(checkpoint, 10, "test"); backingStore.close(); Assert.assertTrue(checkpoint.exists()); Assert.assertTrue(Serialization.getMetaDataFile(checkpoint).length() != 0); @@ -183,8 +181,7 @@ public void testCheckpointVersionNotEqualToMeta2() throws Exception { ProtosFactory.Checkpoint meta = ProtosFactory.Checkpoint.parseDelimitedFrom(is); Assert.assertNotNull(meta); is.close(); - os = new FileOutputStream( - Serialization.getMetaDataFile(checkpoint)); + os = new FileOutputStream(Serialization.getMetaDataFile(checkpoint)); meta.toBuilder().setVersion(2).build().writeDelimitedTo(os); os.flush(); backingStore = EventQueueBackingStoreFactory.get(checkpoint, 10, "test"); @@ -197,12 +194,10 @@ public void testCheckpointVersionNotEqualToMeta2() throws Exception { public void testCheckpointOrderIdNotEqualToMeta() throws Exception { RandomAccessFile writer = new RandomAccessFile(checkpoint, "rw"); try { - EventQueueBackingStore backingStore = EventQueueBackingStoreFactory. - get(checkpoint, 10, "test"); + EventQueueBackingStore backingStore = + EventQueueBackingStoreFactory.get(checkpoint, 10, "test"); backingStore.close(); - writer.seek( - EventQueueBackingStoreFile.INDEX_WRITE_ORDER_ID - * Serialization.SIZE_OF_LONG); + writer.seek(EventQueueBackingStoreFile.INDEX_WRITE_ORDER_ID * Serialization.SIZE_OF_LONG); writer.writeLong(2L); writer.getFD().sync(); backingStore = EventQueueBackingStoreFactory.get(checkpoint, 10, "test"); @@ -215,8 +210,8 @@ public void testCheckpointOrderIdNotEqualToMeta() throws Exception { public void testCheckpointOrderIdNotEqualToMeta2() throws Exception { FileOutputStream os = null; try { - EventQueueBackingStore backingStore = EventQueueBackingStoreFactory. - get(checkpoint, 10, "test"); + EventQueueBackingStore backingStore = + EventQueueBackingStoreFactory.get(checkpoint, 10, "test"); backingStore.close(); Assert.assertTrue(checkpoint.exists()); Assert.assertTrue(Serialization.getMetaDataFile(checkpoint).length() != 0); @@ -225,7 +220,7 @@ public void testCheckpointOrderIdNotEqualToMeta2() throws Exception { Assert.assertNotNull(meta); is.close(); os = new FileOutputStream( - Serialization.getMetaDataFile(checkpoint)); + Serialization.getMetaDataFile(checkpoint)); meta.toBuilder().setWriteOrderID(1).build().writeDelimitedTo(os); os.flush(); backingStore = EventQueueBackingStoreFactory.get(checkpoint, 10, "test"); @@ -234,11 +229,10 @@ public void testCheckpointOrderIdNotEqualToMeta2() throws Exception { } } - @Test(expected = BadCheckpointException.class) public void testTruncateMeta() throws Exception { - EventQueueBackingStore backingStore = EventQueueBackingStoreFactory. - get(checkpoint, 10, "test"); + EventQueueBackingStore backingStore = + EventQueueBackingStoreFactory.get(checkpoint, 10, "test"); backingStore.close(); Assert.assertTrue(checkpoint.exists()); File metaFile = Serialization.getMetaDataFile(checkpoint); @@ -250,10 +244,10 @@ public void testTruncateMeta() throws Exception { backingStore = EventQueueBackingStoreFactory.get(checkpoint, 10, "test"); } - @Test (expected = InvalidProtocolBufferException.class) + @Test(expected = InvalidProtocolBufferException.class) public void testCorruptMeta() throws Throwable { - EventQueueBackingStore backingStore = EventQueueBackingStoreFactory. - get(checkpoint, 10, "test"); + EventQueueBackingStore backingStore = + EventQueueBackingStoreFactory.get(checkpoint, 10, "test"); backingStore.close(); Assert.assertTrue(checkpoint.exists()); File metaFile = Serialization.getMetaDataFile(checkpoint); @@ -270,17 +264,13 @@ public void testCorruptMeta() throws Throwable { } } - - - private void verify(EventQueueBackingStore backingStore, long expectedVersion, - List expectedPointers) - throws Exception { - FlumeEventQueue queue = new FlumeEventQueue(backingStore, inflightTakes, - inflightPuts, queueSetDir); + List expectedPointers) throws Exception { + FlumeEventQueue queue = + new FlumeEventQueue(backingStore, inflightTakes, inflightPuts, queueSetDir); List actualPointers = Lists.newArrayList(); FlumeEventPointer ptr; - while((ptr = queue.removeHead(0L)) != null) { + while ((ptr = queue.removeHead(0L)) != null) { actualPointers.add(ptr.toLong()); } Assert.assertEquals(expectedPointers, actualPointers); diff --git a/flume-ng-channels/flume-file-channel/src/test/java/org/apache/flume/channel/file/TestEventUtils.java b/flume-ng-channels/flume-file-channel/src/test/java/org/apache/flume/channel/file/TestEventUtils.java index c72e3f2149..26f9caeb2a 100644 --- a/flume-ng-channels/flume-file-channel/src/test/java/org/apache/flume/channel/file/TestEventUtils.java +++ b/flume-ng-channels/flume-file-channel/src/test/java/org/apache/flume/channel/file/TestEventUtils.java @@ -28,7 +28,7 @@ public class TestEventUtils { @Test public void testPutEvent() { FlumeEvent event = new FlumeEvent(null, new byte[5]); - Put put = new Put(1l, 1l, event); + Put put = new Put(1L, 1L, event); Event returnEvent = EventUtils.getEventFromTransactionEvent(put); Assert.assertNotNull(returnEvent); Assert.assertEquals(5, returnEvent.getBody().length); @@ -36,7 +36,7 @@ public void testPutEvent() { @Test public void testInvalidEvent() { - Take take = new Take(1l, 1l); + Take take = new Take(1L, 1L); Event returnEvent = EventUtils.getEventFromTransactionEvent(take); Assert.assertNull(returnEvent); } diff --git a/flume-ng-channels/flume-file-channel/src/test/java/org/apache/flume/channel/file/TestFileChannel.java b/flume-ng-channels/flume-file-channel/src/test/java/org/apache/flume/channel/file/TestFileChannel.java index bb22e26652..bfc2d0d3ea 100644 --- a/flume-ng-channels/flume-file-channel/src/test/java/org/apache/flume/channel/file/TestFileChannel.java +++ b/flume-ng-channels/flume-file-channel/src/test/java/org/apache/flume/channel/file/TestFileChannel.java @@ -18,8 +18,22 @@ */ package org.apache.flume.channel.file; -import static org.apache.flume.channel.file.TestUtils.*; -import static org.fest.reflect.core.Reflection.*; +import com.google.common.base.Throwables; +import com.google.common.collect.Maps; +import com.google.common.collect.Sets; +import org.apache.flume.ChannelException; +import org.apache.flume.Event; +import org.apache.flume.Transaction; +import org.apache.flume.channel.file.FileChannel.FileBackedTransaction; +import org.apache.flume.channel.file.FlumeEventQueue.InflightEventWrapper; +import org.apache.flume.conf.Configurables; +import org.apache.flume.event.EventBuilder; +import org.junit.After; +import org.junit.Assert; +import org.junit.Before; +import org.junit.Test; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; import java.io.File; import java.io.FilenameFilter; @@ -32,7 +46,6 @@ import java.util.Iterator; import java.util.List; import java.util.Map; -import java.util.Random; import java.util.Set; import java.util.concurrent.Callable; import java.util.concurrent.CountDownLatch; @@ -41,23 +54,15 @@ import java.util.concurrent.Future; import java.util.concurrent.TimeUnit; -import org.apache.flume.ChannelException; -import org.apache.flume.Event; -import org.apache.flume.Transaction; -import org.apache.flume.conf.Configurables; -import org.junit.After; -import org.junit.Assert; -import org.junit.Before; -import org.junit.Test; -import org.slf4j.Logger; -import org.slf4j.LoggerFactory; - -import com.google.common.base.Throwables; -import com.google.common.collect.Maps; -import com.google.common.collect.Sets; -import org.apache.flume.channel.file.FileChannel.FileBackedTransaction; -import org.apache.flume.channel.file.FlumeEventQueue.InflightEventWrapper; -import org.apache.flume.event.EventBuilder; +import static org.apache.flume.channel.file.TestUtils.compareInputAndOut; +import static org.apache.flume.channel.file.TestUtils.consumeChannel; +import static org.apache.flume.channel.file.TestUtils.fillChannel; +import static org.apache.flume.channel.file.TestUtils.forceCheckpoint; +import static org.apache.flume.channel.file.TestUtils.putEvents; +import static org.apache.flume.channel.file.TestUtils.putWithoutCommit; +import static org.apache.flume.channel.file.TestUtils.takeEvents; +import static org.apache.flume.channel.file.TestUtils.takeWithoutCommit; +import static org.fest.reflect.core.Reflection.field; public class TestFileChannel extends TestFileChannelBase { @@ -68,6 +73,7 @@ public class TestFileChannel extends TestFileChannelBase { public void setup() throws Exception { super.setup(); } + @After public void teardown() { super.teardown(); @@ -146,23 +152,22 @@ public void testFailAfterPutCheckpointCommit() throws Throwable { //Simulate multiple sources, so separate thread - txns are thread local, //so a new txn wont be created here unless it is in a different thread. final CountDownLatch latch = new CountDownLatch(1); - Executors.newSingleThreadExecutor().submit( - new Runnable() { - @Override - public void run() { - Transaction tx = channel.getTransaction(); - input.addAll(putWithoutCommit(channel, tx, "failAfterPut", 3)); - try { - latch.await(); - tx.commit(); - } catch (InterruptedException e) { - tx.rollback(); - Throwables.propagate(e); - } finally { - tx.close(); - } - } - }); + Executors.newSingleThreadExecutor().submit(new Runnable() { + @Override + public void run() { + Transaction tx = channel.getTransaction(); + input.addAll(putWithoutCommit(channel, tx, "failAfterPut", 3)); + try { + latch.await(); + tx.commit(); + } catch (InterruptedException e) { + tx.rollback(); + Throwables.propagate(e); + } finally { + tx.close(); + } + } + }); forceCheckpoint(channel); tx.commit(); tx.close(); @@ -198,7 +203,7 @@ public void testReconfigure() throws Exception { Assert.assertTrue(channel.isOpen()); Set in = Sets.newHashSet(); try { - while(true) { + while (true) { in.addAll(putEvents(channel, "reconfig", 1, 1)); } } catch (ChannelException e) { @@ -206,12 +211,13 @@ public void testReconfigure() throws Exception { + "This might be the result of a sink on the channel having too " + "low of batch size, a downstream system running slower than " + "normal, or that the channel capacity is just too low. [channel=" - + channel.getName()+"]", e.getMessage()); + + channel.getName() + "]", e.getMessage()); } Configurables.configure(channel, createContext()); Set out = takeEvents(channel, 1, Integer.MAX_VALUE); compareInputAndOut(in, out); } + @Test public void testPut() throws Exception { channel.start(); @@ -225,6 +231,7 @@ public void testPut() throws Exception { Set actual = takeEvents(channel, 1); compareInputAndOut(expected, actual); } + @Test public void testCommitAfterNoPutTake() throws Exception { channel.start(); @@ -246,6 +253,7 @@ public void testCommitAfterNoPutTake() throws Exception { transaction.commit(); transaction.close(); } + @Test public void testCapacity() throws Exception { Map overrides = Maps.newHashMap(); @@ -270,6 +278,7 @@ public void testCapacity() throws Exception { // ensure we the events back Assert.assertEquals(5, takeEvents(channel, 1, 5).size()); } + /** * This test is here to make sure we can replay a full queue * when we have a PUT with a lower txid than the take which @@ -287,16 +296,14 @@ public void testRaceFoundInFLUME1432() throws Exception { // the idea here is we will fill up the channel Map overrides = Maps.newHashMap(); overrides.put(FileChannelConfiguration.KEEP_ALIVE, String.valueOf(10L)); - overrides.put(FileChannelConfiguration.CAPACITY, String.valueOf(10)); - overrides.put(FileChannelConfiguration.TRANSACTION_CAPACITY, - String.valueOf(10)); + overrides.put(FileChannelConfiguration.CAPACITY, String.valueOf(10L)); + overrides.put(FileChannelConfiguration.TRANSACTION_CAPACITY, String.valueOf(10L)); channel = createFileChannel(overrides); channel.start(); Assert.assertTrue(channel.isOpen()); fillChannel(channel, "fillup"); // then do a put which will block but it will be assigned a tx id - Future put = Executors.newSingleThreadExecutor() - .submit(new Callable() { + Future put = Executors.newSingleThreadExecutor().submit(new Callable() { @Override public String call() throws Exception { Set result = putEvents(channel, "blocked-put", 1, 1); @@ -321,6 +328,7 @@ public String call() throws Exception { channel.start(); Assert.assertTrue(channel.isOpen()); } + @Test public void testThreaded() throws IOException, InterruptedException { channel.start(); @@ -328,12 +336,9 @@ public void testThreaded() throws IOException, InterruptedException { int numThreads = 10; final CountDownLatch producerStopLatch = new CountDownLatch(numThreads); final CountDownLatch consumerStopLatch = new CountDownLatch(numThreads); - final List errors = Collections - .synchronizedList(new ArrayList()); - final Set expected = Collections.synchronizedSet( - new HashSet()); - final Set actual = Collections.synchronizedSet( - new HashSet()); + final List errors = Collections.synchronizedList(new ArrayList()); + final Set expected = Collections.synchronizedSet(new HashSet()); + final Set actual = Collections.synchronizedSet(new HashSet()); for (int i = 0; i < numThreads; i++) { final int id = i; Thread t = new Thread() { @@ -363,15 +368,15 @@ public void run() { @Override public void run() { try { - while(!producerStopLatch.await(1, TimeUnit.SECONDS) || - expected.size() > actual.size()) { + while (!producerStopLatch.await(1, TimeUnit.SECONDS) || + expected.size() > actual.size()) { if (id % 2 == 0) { actual.addAll(takeEvents(channel, 1, Integer.MAX_VALUE)); } else { actual.addAll(takeEvents(channel, 5, Integer.MAX_VALUE)); } } - if(actual.isEmpty()) { + if (actual.isEmpty()) { LOG.error("Found nothing!"); } else { LOG.info("Completed some takes " + actual.size()); @@ -388,12 +393,13 @@ public void run() { t.start(); } Assert.assertTrue("Timed out waiting for producers", - producerStopLatch.await(30, TimeUnit.SECONDS)); + producerStopLatch.await(30, TimeUnit.SECONDS)); Assert.assertTrue("Timed out waiting for consumer", - consumerStopLatch.await(30, TimeUnit.SECONDS)); + consumerStopLatch.await(30, TimeUnit.SECONDS)); Assert.assertEquals(Collections.EMPTY_LIST, errors); compareInputAndOut(expected, actual); } + @Test public void testLocking() throws IOException { channel.start(); @@ -403,7 +409,6 @@ public void testLocking() throws IOException { Assert.assertTrue(!fileChannel.isOpen()); } - /** * Test contributed by Brock Noland during code review. * @throws Exception @@ -437,11 +442,11 @@ public void testTakeTransactionCrossingCheckpoint() throws Exception { } @Test - public void testPutForceCheckpointCommitReplay() throws Exception{ + public void testPutForceCheckpointCommitReplay() throws Exception { Map overrides = Maps.newHashMap(); overrides.put(FileChannelConfiguration.CAPACITY, String.valueOf(2)); overrides.put(FileChannelConfiguration.TRANSACTION_CAPACITY, - String.valueOf(2)); + String.valueOf(2)); overrides.put(FileChannelConfiguration.CHECKPOINT_INTERVAL, "10000"); FileChannel channel = createFileChannel(overrides); channel.start(); @@ -578,28 +583,22 @@ public void testChannelDiesOnCorruptEventFsync() throws Exception { testChannelDiesOnCorruptEvent(true); } - @Test - public void testChannelDiesOnCorruptEventNoFsync() throws - Exception { + public void testChannelDiesOnCorruptEventNoFsync() throws Exception { testChannelDiesOnCorruptEvent(false); } - - - private void testChannelDiesOnCorruptEvent(boolean fsyncPerTxn) - throws Exception { + private void testChannelDiesOnCorruptEvent(boolean fsyncPerTxn) throws Exception { Map overrides = new HashMap(); - overrides.put(FileChannelConfiguration.FSYNC_PER_TXN, - String.valueOf(fsyncPerTxn)); + overrides.put(FileChannelConfiguration.FSYNC_PER_TXN, String.valueOf(fsyncPerTxn)); final FileChannel channel = createFileChannel(overrides); channel.start(); putEvents(channel,"test-corrupt-event",100,100); - for(File dataDir : dataDirs) { + for (File dataDir : dataDirs) { File[] files = dataDir.listFiles(new FilenameFilter() { @Override public boolean accept(File dir, String name) { - if(!name.endsWith("meta") && !name.contains("lock")){ + if (!name.endsWith("meta") && !name.contains("lock")) { return true; } return false; @@ -624,7 +623,7 @@ public boolean accept(File dir, String name) { Assert.assertTrue(ex.getMessage().contains("Log is closed")); throw ex; } - if(fsyncPerTxn) { + if (fsyncPerTxn) { Assert.fail(); } else { // The corrupt event must be missing, the rest should be diff --git a/flume-ng-channels/flume-file-channel/src/test/java/org/apache/flume/channel/file/TestFileChannelFormatRegression.java b/flume-ng-channels/flume-file-channel/src/test/java/org/apache/flume/channel/file/TestFileChannelFormatRegression.java index c95122b40a..f0638f9a67 100644 --- a/flume-ng-channels/flume-file-channel/src/test/java/org/apache/flume/channel/file/TestFileChannelFormatRegression.java +++ b/flume-ng-channels/flume-file-channel/src/test/java/org/apache/flume/channel/file/TestFileChannelFormatRegression.java @@ -18,14 +18,7 @@ */ package org.apache.flume.channel.file; -import static org.apache.flume.channel.file.TestUtils.*; - -import java.io.File; -import java.util.Arrays; -import java.util.HashSet; -import java.util.Map; -import java.util.Set; - +import com.google.common.collect.Maps; import org.junit.After; import org.junit.Assert; import org.junit.Before; @@ -33,8 +26,14 @@ import org.slf4j.Logger; import org.slf4j.LoggerFactory; -import com.google.common.collect.Maps; +import java.io.File; +import java.util.Arrays; +import java.util.HashSet; +import java.util.Map; +import java.util.Set; +import static org.apache.flume.channel.file.TestUtils.compareInputAndOut; +import static org.apache.flume.channel.file.TestUtils.takeEvents; public class TestFileChannelFormatRegression extends TestFileChannelBase { protected static final Logger LOG = LoggerFactory @@ -60,8 +59,8 @@ public void testFileFormatV2postFLUME1432() new File(checkpointDir, "checkpoint")); for (int i = 0; i < dataDirs.length; i++) { int fileIndex = i + 1; - TestUtils.copyDecompressed("fileformat-v2-log-"+fileIndex+".gz", - new File(dataDirs[i], "log-" + fileIndex)); + TestUtils.copyDecompressed("fileformat-v2-log-" + fileIndex + ".gz", + new File(dataDirs[i], "log-" + fileIndex)); } Map overrides = Maps.newHashMap(); overrides.put(FileChannelConfiguration.CAPACITY, String.valueOf(10)); diff --git a/flume-ng-channels/flume-file-channel/src/test/java/org/apache/flume/channel/file/TestFileChannelRestart.java b/flume-ng-channels/flume-file-channel/src/test/java/org/apache/flume/channel/file/TestFileChannelRestart.java index d5fe6fba5c..d21f140679 100644 --- a/flume-ng-channels/flume-file-channel/src/test/java/org/apache/flume/channel/file/TestFileChannelRestart.java +++ b/flume-ng-channels/flume-file-channel/src/test/java/org/apache/flume/channel/file/TestFileChannelRestart.java @@ -55,8 +55,7 @@ import static org.fest.reflect.core.Reflection.*; public class TestFileChannelRestart extends TestFileChannelBase { - protected static final Logger LOG = LoggerFactory - .getLogger(TestFileChannelRestart.class); + protected static final Logger LOG = LoggerFactory.getLogger(TestFileChannelRestart.class); @Before public void setup() throws Exception { @@ -72,8 +71,8 @@ public void teardown() { protected FileChannel createFileChannel(Map overrides) { // FLUME-2482, making sure scheduled checkpoint never gets called overrides.put(FileChannelConfiguration.CHECKPOINT_INTERVAL, "6000000"); - return TestUtils.createFileChannel(checkpointDir.getAbsolutePath(), - dataDir, backupDir.getAbsolutePath(), overrides); + return TestUtils.createFileChannel(checkpointDir.getAbsolutePath(), dataDir, + backupDir.getAbsolutePath(), overrides); } @Test @@ -116,14 +115,14 @@ public void testNormalReplayV2() throws Exception { } public void doTestRestart(boolean useLogReplayV1, - boolean forceCheckpoint, boolean deleteCheckpoint, - boolean useFastReplay) throws Exception { + boolean forceCheckpoint, boolean deleteCheckpoint, + boolean useFastReplay) throws Exception { Map overrides = Maps.newHashMap(); overrides.put(FileChannelConfiguration.USE_LOG_REPLAY_V1, - String.valueOf(useLogReplayV1)); + String.valueOf(useLogReplayV1)); overrides.put( - FileChannelConfiguration.USE_FAST_REPLAY, - String.valueOf(useFastReplay)); + FileChannelConfiguration.USE_FAST_REPLAY, + String.valueOf(useFastReplay)); channel = createFileChannel(overrides); channel.start(); Assert.assertTrue(channel.isOpen()); @@ -132,7 +131,7 @@ public void doTestRestart(boolean useLogReplayV1, forceCheckpoint(channel); } channel.stop(); - if(deleteCheckpoint) { + if (deleteCheckpoint) { File checkpoint = new File(checkpointDir, "checkpoint"); Assert.assertTrue(checkpoint.delete()); File checkpointMetaData = Serialization.getMetaDataFile(checkpoint); @@ -146,19 +145,17 @@ public void doTestRestart(boolean useLogReplayV1, } @Test - public void testRestartWhenMetaDataExistsButCheckpointDoesNot() throws - Exception { + public void testRestartWhenMetaDataExistsButCheckpointDoesNot() throws Exception { doTestRestartWhenMetaDataExistsButCheckpointDoesNot(false); } @Test - public void testRestartWhenMetaDataExistsButCheckpointDoesNotWithBackup() - throws Exception { + public void testRestartWhenMetaDataExistsButCheckpointDoesNotWithBackup() throws Exception { doTestRestartWhenMetaDataExistsButCheckpointDoesNot(true); } - private void doTestRestartWhenMetaDataExistsButCheckpointDoesNot( - boolean backup) throws Exception { + private void doTestRestartWhenMetaDataExistsButCheckpointDoesNot(boolean backup) + throws Exception { Map overrides = Maps.newHashMap(); overrides.put(FileChannelConfiguration.USE_DUAL_CHECKPOINTS, String.valueOf(backup)); channel = createFileChannel(overrides); @@ -167,7 +164,7 @@ private void doTestRestartWhenMetaDataExistsButCheckpointDoesNot( Set in = putEvents(channel, "restart", 10, 100); Assert.assertEquals(100, in.size()); forceCheckpoint(channel); - if(backup) { + if (backup) { Thread.sleep(2000); } channel.stop(); @@ -186,19 +183,16 @@ private void doTestRestartWhenMetaDataExistsButCheckpointDoesNot( } @Test - public void testRestartWhenCheckpointExistsButMetaDoesNot() throws Exception{ + public void testRestartWhenCheckpointExistsButMetaDoesNot() throws Exception { doTestRestartWhenCheckpointExistsButMetaDoesNot(false); } @Test - public void testRestartWhenCheckpointExistsButMetaDoesNotWithBackup() throws - Exception{ + public void testRestartWhenCheckpointExistsButMetaDoesNotWithBackup() throws Exception { doTestRestartWhenCheckpointExistsButMetaDoesNot(true); } - - private void doTestRestartWhenCheckpointExistsButMetaDoesNot(boolean backup) - throws Exception { + private void doTestRestartWhenCheckpointExistsButMetaDoesNot(boolean backup) throws Exception { Map overrides = Maps.newHashMap(); overrides.put(FileChannelConfiguration.USE_DUAL_CHECKPOINTS, String.valueOf(backup)); channel = createFileChannel(overrides); @@ -207,7 +201,7 @@ private void doTestRestartWhenCheckpointExistsButMetaDoesNot(boolean backup) Set in = putEvents(channel, "restart", 10, 100); Assert.assertEquals(100, in.size()); forceCheckpoint(channel); - if(backup) { + if (backup) { Thread.sleep(2000); } channel.stop(); @@ -235,8 +229,7 @@ public void testRestartWhenNoCheckpointExistsWithBackup() throws Exception { doTestRestartWhenNoCheckpointExists(true); } - private void doTestRestartWhenNoCheckpointExists(boolean backup) throws - Exception { + private void doTestRestartWhenNoCheckpointExists(boolean backup) throws Exception { Map overrides = Maps.newHashMap(); overrides.put(FileChannelConfiguration.USE_DUAL_CHECKPOINTS, String.valueOf(backup)); channel = createFileChannel(overrides); @@ -245,7 +238,7 @@ private void doTestRestartWhenNoCheckpointExists(boolean backup) throws Set in = putEvents(channel, "restart", 10, 100); Assert.assertEquals(100, in.size()); forceCheckpoint(channel); - if(backup) { + if (backup) { Thread.sleep(2000); } channel.stop(); @@ -273,7 +266,7 @@ public void testBadCheckpointVersionWithBackup() throws Exception { doTestBadCheckpointVersion(true); } - private void doTestBadCheckpointVersion(boolean backup) throws Exception{ + private void doTestBadCheckpointVersion(boolean backup) throws Exception { Map overrides = Maps.newHashMap(); overrides.put(FileChannelConfiguration.USE_DUAL_CHECKPOINTS, String.valueOf(backup)); channel = createFileChannel(overrides); @@ -282,14 +275,14 @@ private void doTestBadCheckpointVersion(boolean backup) throws Exception{ Set in = putEvents(channel, "restart", 10, 100); Assert.assertEquals(100, in.size()); forceCheckpoint(channel); - if(backup) { + if (backup) { Thread.sleep(2000); } channel.stop(); File checkpoint = new File(checkpointDir, "checkpoint"); RandomAccessFile writer = new RandomAccessFile(checkpoint, "rw"); writer.seek(EventQueueBackingStoreFile.INDEX_VERSION * - Serialization.SIZE_OF_LONG); + Serialization.SIZE_OF_LONG); writer.writeLong(2L); writer.getFD().sync(); writer.close(); @@ -311,8 +304,7 @@ public void testBadCheckpointMetaVersionWithBackup() throws Exception { doTestBadCheckpointMetaVersion(true); } - private void doTestBadCheckpointMetaVersion(boolean backup) throws - Exception { + private void doTestBadCheckpointMetaVersion(boolean backup) throws Exception { Map overrides = Maps.newHashMap(); overrides.put(FileChannelConfiguration.USE_DUAL_CHECKPOINTS, String.valueOf(backup)); channel = createFileChannel(overrides); @@ -321,7 +313,7 @@ private void doTestBadCheckpointMetaVersion(boolean backup) throws Set in = putEvents(channel, "restart", 10, 100); Assert.assertEquals(100, in.size()); forceCheckpoint(channel); - if(backup) { + if (backup) { Thread.sleep(2000); } channel.stop(); @@ -331,7 +323,7 @@ private void doTestBadCheckpointMetaVersion(boolean backup) throws Assert.assertNotNull(meta); is.close(); FileOutputStream os = new FileOutputStream( - Serialization.getMetaDataFile(checkpoint)); + Serialization.getMetaDataFile(checkpoint)); meta.toBuilder().setVersion(2).build().writeDelimitedTo(os); os.flush(); channel = createFileChannel(overrides); @@ -348,13 +340,11 @@ public void testDifferingOrderIDCheckpointAndMetaVersion() throws Exception { } @Test - public void testDifferingOrderIDCheckpointAndMetaVersionWithBackup() throws - Exception { + public void testDifferingOrderIDCheckpointAndMetaVersionWithBackup() throws Exception { doTestDifferingOrderIDCheckpointAndMetaVersion(true); } - private void doTestDifferingOrderIDCheckpointAndMetaVersion(boolean backup) - throws Exception { + private void doTestDifferingOrderIDCheckpointAndMetaVersion(boolean backup) throws Exception { Map overrides = Maps.newHashMap(); overrides.put(FileChannelConfiguration.USE_DUAL_CHECKPOINTS, String.valueOf(backup)); channel = createFileChannel(overrides); @@ -363,7 +353,7 @@ private void doTestDifferingOrderIDCheckpointAndMetaVersion(boolean backup) Set in = putEvents(channel, "restart", 10, 100); Assert.assertEquals(100, in.size()); forceCheckpoint(channel); - if(backup) { + if (backup) { Thread.sleep(2000); } channel.stop(); @@ -373,7 +363,7 @@ private void doTestDifferingOrderIDCheckpointAndMetaVersion(boolean backup) Assert.assertNotNull(meta); is.close(); FileOutputStream os = new FileOutputStream( - Serialization.getMetaDataFile(checkpoint)); + Serialization.getMetaDataFile(checkpoint)); meta.toBuilder().setWriteOrderID(12).build().writeDelimitedTo(os); os.flush(); channel = createFileChannel(overrides); @@ -385,12 +375,12 @@ private void doTestDifferingOrderIDCheckpointAndMetaVersion(boolean backup) } @Test - public void testIncompleteCheckpoint() throws Exception{ + public void testIncompleteCheckpoint() throws Exception { doTestIncompleteCheckpoint(false); } @Test - public void testIncompleteCheckpointWithCheckpoint() throws Exception{ + public void testIncompleteCheckpointWithCheckpoint() throws Exception { doTestIncompleteCheckpoint(true); } @@ -403,14 +393,14 @@ private void doTestIncompleteCheckpoint(boolean backup) throws Exception { Set in = putEvents(channel, "restart", 10, 100); Assert.assertEquals(100, in.size()); forceCheckpoint(channel); - if(backup) { + if (backup) { Thread.sleep(2000); } channel.stop(); File checkpoint = new File(checkpointDir, "checkpoint"); RandomAccessFile writer = new RandomAccessFile(checkpoint, "rw"); writer.seek(EventQueueBackingStoreFile.INDEX_CHECKPOINT_MARKER - * Serialization.SIZE_OF_LONG); + * Serialization.SIZE_OF_LONG); writer.writeLong(EventQueueBackingStoreFile.CHECKPOINT_INCOMPLETE); writer.getFD().sync(); writer.close(); @@ -443,30 +433,30 @@ public void testCorruptInflightTakesWithBackup() throws Exception { } @Test - public void testFastReplayWithCheckpoint() throws Exception{ + public void testFastReplayWithCheckpoint() throws Exception { testFastReplay(false, true); } @Test - public void testFastReplayWithBadCheckpoint() throws Exception{ + public void testFastReplayWithBadCheckpoint() throws Exception { testFastReplay(true, true); } @Test - public void testNoFastReplayWithCheckpoint() throws Exception{ + public void testNoFastReplayWithCheckpoint() throws Exception { testFastReplay(false, false); } @Test - public void testNoFastReplayWithBadCheckpoint() throws Exception{ + public void testNoFastReplayWithBadCheckpoint() throws Exception { testFastReplay(true, false); } - private void testFastReplay(boolean shouldCorruptCheckpoint, - boolean useFastReplay) throws Exception{ + private void testFastReplay(boolean shouldCorruptCheckpoint, boolean useFastReplay) + throws Exception { Map overrides = Maps.newHashMap(); overrides.put(FileChannelConfiguration.USE_FAST_REPLAY, - String.valueOf(useFastReplay)); + String.valueOf(useFastReplay)); channel = createFileChannel(overrides); channel.start(); Assert.assertTrue(channel.isOpen()); @@ -477,7 +467,8 @@ private void testFastReplay(boolean shouldCorruptCheckpoint, if (shouldCorruptCheckpoint) { File checkpoint = new File(checkpointDir, "checkpoint"); RandomAccessFile writer = new RandomAccessFile( - Serialization.getMetaDataFile(checkpoint), "rw"); + Serialization.getMetaDataFile(checkpoint), + "rw"); writer.seek(10); writer.writeLong(new Random().nextLong()); writer.getFD().sync(); @@ -495,14 +486,13 @@ private void testFastReplay(boolean shouldCorruptCheckpoint, compareInputAndOut(in, out); } - private void doTestCorruptInflights(String name, - boolean backup) throws Exception { + private void doTestCorruptInflights(String name, boolean backup) throws Exception { Map overrides = Maps.newHashMap(); overrides.put(FileChannelConfiguration.USE_DUAL_CHECKPOINTS, String.valueOf(backup)); channel = createFileChannel(overrides); channel.start(); Assert.assertTrue(channel.isOpen()); - final Set in1 = putEvents(channel, "restart-",10, 100); + final Set in1 = putEvents(channel, "restart-", 10, 100); Assert.assertEquals(100, in1.size()); Executors.newSingleThreadScheduledExecutor().submit(new Runnable() { @Override @@ -516,7 +506,7 @@ public void run() { Set in2 = putWithoutCommit(channel, tx, "restart", 100); Assert.assertEquals(100, in2.size()); forceCheckpoint(channel); - if(backup) { + if (backup) { Thread.sleep(2000); } tx.commit(); @@ -554,13 +544,12 @@ private void doTestTruncatedCheckpointMeta(boolean backup) throws Exception { Set in = putEvents(channel, "restart", 10, 100); Assert.assertEquals(100, in.size()); forceCheckpoint(channel); - if(backup) { + if (backup) { Thread.sleep(2000); } channel.stop(); File checkpoint = new File(checkpointDir, "checkpoint"); - RandomAccessFile writer = new RandomAccessFile( - Serialization.getMetaDataFile(checkpoint), "rw"); + RandomAccessFile writer = new RandomAccessFile(Serialization.getMetaDataFile(checkpoint), "rw"); writer.setLength(0); writer.getFD().sync(); writer.close(); @@ -591,13 +580,12 @@ private void doTestCorruptCheckpointMeta(boolean backup) throws Exception { Set in = putEvents(channel, "restart", 10, 100); Assert.assertEquals(100, in.size()); forceCheckpoint(channel); - if(backup) { + if (backup) { Thread.sleep(2000); } channel.stop(); File checkpoint = new File(checkpointDir, "checkpoint"); - RandomAccessFile writer = new RandomAccessFile( - Serialization.getMetaDataFile(checkpoint), "rw"); + RandomAccessFile writer = new RandomAccessFile(Serialization.getMetaDataFile(checkpoint), "rw"); writer.seek(10); writer.writeLong(new Random().nextLong()); writer.getFD().sync(); @@ -618,11 +606,10 @@ private void checkIfBackupUsed(boolean backup) { Assert.assertFalse(backupRestored); } } - + //This test will fail without FLUME-1893 @Test - public void testCorruptCheckpointVersionMostSignificant4Bytes() - throws Exception { + public void testCorruptCheckpointVersionMostSignificant4Bytes() throws Exception { Map overrides = Maps.newHashMap(); channel = createFileChannel(overrides); channel.start(); @@ -634,8 +621,8 @@ public void testCorruptCheckpointVersionMostSignificant4Bytes() File checkpoint = new File(checkpointDir, "checkpoint"); RandomAccessFile writer = new RandomAccessFile(checkpoint, "rw"); writer.seek(EventQueueBackingStoreFile.INDEX_VERSION * - Serialization.SIZE_OF_LONG); - writer.write(new byte[]{(byte)1, (byte)5}); + Serialization.SIZE_OF_LONG); + writer.write(new byte[] { (byte) 1, (byte) 5 }); writer.getFD().sync(); writer.close(); channel = createFileChannel(overrides); @@ -648,8 +635,7 @@ public void testCorruptCheckpointVersionMostSignificant4Bytes() //This test will fail without FLUME-1893 @Test - public void testCorruptCheckpointCompleteMarkerMostSignificant4Bytes() - throws Exception { + public void testCorruptCheckpointCompleteMarkerMostSignificant4Bytes() throws Exception { Map overrides = Maps.newHashMap(); channel = createFileChannel(overrides); channel.start(); @@ -661,8 +647,8 @@ public void testCorruptCheckpointCompleteMarkerMostSignificant4Bytes() File checkpoint = new File(checkpointDir, "checkpoint"); RandomAccessFile writer = new RandomAccessFile(checkpoint, "rw"); writer.seek(EventQueueBackingStoreFile.INDEX_CHECKPOINT_MARKER * - Serialization.SIZE_OF_LONG); - writer.write(new byte[]{(byte) 1, (byte) 5}); + Serialization.SIZE_OF_LONG); + writer.write(new byte[] { (byte) 1, (byte) 5 }); writer.getFD().sync(); writer.close(); channel = createFileChannel(overrides); @@ -674,8 +660,7 @@ public void testCorruptCheckpointCompleteMarkerMostSignificant4Bytes() } @Test - public void testWithExtraLogs() - throws Exception { + public void testWithExtraLogs() throws Exception { Map overrides = Maps.newHashMap(); overrides.put(FileChannelConfiguration.CAPACITY, "10"); overrides.put(FileChannelConfiguration.TRANSACTION_CAPACITY, "10"); @@ -702,27 +687,24 @@ public void testWithExtraLogs() // Make sure the entire channel was not replayed, only the events from the // backup. @Test - public void testBackupUsedEnsureNoFullReplayWithoutCompression() throws - Exception { + public void testBackupUsedEnsureNoFullReplayWithoutCompression() throws Exception { testBackupUsedEnsureNoFullReplay(false); } + @Test - public void testBackupUsedEnsureNoFullReplayWithCompression() throws - Exception { + public void testBackupUsedEnsureNoFullReplayWithCompression() throws Exception { testBackupUsedEnsureNoFullReplay(true); } private void testBackupUsedEnsureNoFullReplay(boolean compressedBackup) - throws Exception { + throws Exception { File dataDir = Files.createTempDir(); File tempBackup = Files.createTempDir(); Map overrides = Maps.newHashMap(); - overrides.put(FileChannelConfiguration.DATA_DIRS, - dataDir.getAbsolutePath()); - overrides.put(FileChannelConfiguration.USE_DUAL_CHECKPOINTS, - "true"); + overrides.put(FileChannelConfiguration.DATA_DIRS, dataDir.getAbsolutePath()); + overrides.put(FileChannelConfiguration.USE_DUAL_CHECKPOINTS, "true"); overrides.put(FileChannelConfiguration.COMPRESS_BACKUP_CHECKPOINT, - String.valueOf(compressedBackup)); + String.valueOf(compressedBackup)); channel = createFileChannel(overrides); channel.start(); Assert.assertTrue(channel.isOpen()); @@ -734,8 +716,8 @@ private void testBackupUsedEnsureNoFullReplay(boolean compressedBackup) in = putEvents(channel, "restart", 10, 100); takeEvents(channel, 10, 100); Assert.assertEquals(100, in.size()); - for(File file : backupDir.listFiles()) { - if(file.getName().equals(Log.FILE_LOCK)) { + for (File file : backupDir.listFiles()) { + if (file.getName().equals(Log.FILE_LOCK)) { continue; } Files.copy(file, new File(tempBackup, file.getName())); @@ -749,8 +731,8 @@ private void testBackupUsedEnsureNoFullReplay(boolean compressedBackup) // tests), so throw away the backup and force the use of an older backup by // bringing in the copy of the last backup before the checkpoint. Serialization.deleteAllFiles(backupDir, Log.EXCLUDES); - for(File file : tempBackup.listFiles()) { - if(file.getName().equals(Log.FILE_LOCK)) { + for (File file : tempBackup.listFiles()) { + if (file.getName().equals(Log.FILE_LOCK)) { continue; } Files.copy(file, new File(backupDir, file.getName())); @@ -782,7 +764,7 @@ public void testDataFilesRequiredByBackupNotDeleted() throws Exception { Assert.assertTrue(channel.isOpen()); putEvents(channel, prefix, 10, 100); Set origFiles = Sets.newHashSet(); - for(File dir : dataDirs) { + for (File dir : dataDirs) { origFiles.addAll(Lists.newArrayList(dir.list())); } forceCheckpoint(channel); @@ -792,7 +774,7 @@ public void testDataFilesRequiredByBackupNotDeleted() throws Exception { Set newFiles = Sets.newHashSet(); int olderThanCheckpoint = 0; int totalMetaFiles = 0; - for(File dir : dataDirs) { + for (File dir : dataDirs) { File[] metadataFiles = dir.listFiles(new FilenameFilter() { @Override public boolean accept(File dir, String name) { @@ -803,8 +785,8 @@ public boolean accept(File dir, String name) { } }); totalMetaFiles = metadataFiles.length; - for(File metadataFile : metadataFiles) { - if(metadataFile.lastModified() < beforeSecondCheckpoint) { + for (File metadataFile : metadataFiles) { + if (metadataFile.lastModified() < beforeSecondCheckpoint) { olderThanCheckpoint++; } } @@ -824,13 +806,13 @@ public boolean accept(File dir, String name) { takeEvents(channel, 10, 50); forceCheckpoint(channel); newFiles = Sets.newHashSet(); - for(File dir : dataDirs) { + for (File dir : dataDirs) { newFiles.addAll(Lists.newArrayList(dir.list())); } Assert.assertTrue(!newFiles.containsAll(origFiles)); } - @Test (expected = IOException.class) + @Test(expected = IOException.class) public void testSlowBackup() throws Throwable { Map overrides = Maps.newHashMap(); overrides.put(FileChannelConfiguration.USE_DUAL_CHECKPOINTS, "true"); @@ -858,10 +840,10 @@ public void testSlowBackup() throws Throwable { public void testCompressBackup() throws Throwable { Map overrides = Maps.newHashMap(); overrides.put(FileChannelConfiguration.USE_DUAL_CHECKPOINTS, - "true"); + "true"); overrides.put(FileChannelConfiguration.MAX_FILE_SIZE, "1000"); overrides.put(FileChannelConfiguration.COMPRESS_BACKUP_CHECKPOINT, - "true"); + "true"); channel = createFileChannel(overrides); channel.start(); Assert.assertTrue(channel.isOpen()); @@ -873,36 +855,34 @@ public void testCompressBackup() throws Throwable { Assert.assertTrue(compressedBackupCheckpoint.exists()); - Serialization.decompressFile(compressedBackupCheckpoint, - uncompressedBackupCheckpoint); + Serialization.decompressFile(compressedBackupCheckpoint, uncompressedBackupCheckpoint); File checkpoint = new File(checkpointDir, "checkpoint"); - Assert.assertTrue(FileUtils.contentEquals(checkpoint, - uncompressedBackupCheckpoint)); + Assert.assertTrue(FileUtils.contentEquals(checkpoint, uncompressedBackupCheckpoint)); channel.stop(); } @Test public void testToggleCheckpointCompressionFromTrueToFalse() - throws Exception { + throws Exception { restartToggleCompression(true); } @Test public void testToggleCheckpointCompressionFromFalseToTrue() - throws Exception { + throws Exception { restartToggleCompression(false); } public void restartToggleCompression(boolean originalCheckpointCompressed) - throws Exception { + throws Exception { Map overrides = Maps.newHashMap(); overrides.put(FileChannelConfiguration.USE_DUAL_CHECKPOINTS, - "true"); + "true"); overrides.put(FileChannelConfiguration.MAX_FILE_SIZE, "1000"); overrides.put(FileChannelConfiguration.COMPRESS_BACKUP_CHECKPOINT, - String.valueOf(originalCheckpointCompressed)); + String.valueOf(originalCheckpointCompressed)); channel = createFileChannel(overrides); channel.start(); Assert.assertTrue(channel.isOpen()); @@ -910,17 +890,17 @@ public void restartToggleCompression(boolean originalCheckpointCompressed) forceCheckpoint(channel); Thread.sleep(2000); Assert.assertEquals(compressedBackupCheckpoint.exists(), - originalCheckpointCompressed); + originalCheckpointCompressed); Assert.assertEquals(uncompressedBackupCheckpoint.exists(), - !originalCheckpointCompressed); + !originalCheckpointCompressed); channel.stop(); File checkpoint = new File(checkpointDir, "checkpoint"); Assert.assertTrue(checkpoint.delete()); File checkpointMetaData = Serialization.getMetaDataFile( - checkpoint); + checkpoint); Assert.assertTrue(checkpointMetaData.delete()); overrides.put(FileChannelConfiguration.COMPRESS_BACKUP_CHECKPOINT, - String.valueOf(!originalCheckpointCompressed)); + String.valueOf(!originalCheckpointCompressed)); channel = createFileChannel(overrides); channel.start(); Assert.assertTrue(channel.isOpen()); @@ -929,21 +909,21 @@ public void restartToggleCompression(boolean originalCheckpointCompressed) forceCheckpoint(channel); Thread.sleep(2000); Assert.assertEquals(compressedBackupCheckpoint.exists(), - !originalCheckpointCompressed); + !originalCheckpointCompressed); Assert.assertEquals(uncompressedBackupCheckpoint.exists(), - originalCheckpointCompressed); + originalCheckpointCompressed); } private static void slowdownBackup(FileChannel channel) { Log log = field("log").ofType(Log.class).in(channel).get(); FlumeEventQueue queue = field("queue") - .ofType(FlumeEventQueue.class) - .in(log).get(); + .ofType(FlumeEventQueue.class) + .in(log).get(); EventQueueBackingStore backingStore = field("backingStore") - .ofType(EventQueueBackingStore.class) - .in(queue).get(); + .ofType(EventQueueBackingStore.class) + .in(queue).get(); field("slowdownBackup").ofType(Boolean.class).in(backingStore).set(true); } diff --git a/flume-ng-channels/flume-file-channel/src/test/java/org/apache/flume/channel/file/TestFileChannelRollback.java b/flume-ng-channels/flume-file-channel/src/test/java/org/apache/flume/channel/file/TestFileChannelRollback.java index 23fc64be3a..c06d49807f 100644 --- a/flume-ng-channels/flume-file-channel/src/test/java/org/apache/flume/channel/file/TestFileChannelRollback.java +++ b/flume-ng-channels/flume-file-channel/src/test/java/org/apache/flume/channel/file/TestFileChannelRollback.java @@ -18,11 +18,7 @@ */ package org.apache.flume.channel.file; -import static org.apache.flume.channel.file.TestUtils.*; - -import java.util.Collections; -import java.util.Set; - +import com.google.common.base.Charsets; import org.apache.flume.Transaction; import org.apache.flume.event.EventBuilder; import org.apache.flume.sink.LoggerSink; @@ -33,8 +29,12 @@ import org.slf4j.Logger; import org.slf4j.LoggerFactory; -import com.google.common.base.Charsets; +import java.util.Collections; +import java.util.Set; +import static org.apache.flume.channel.file.TestUtils.compareInputAndOut; +import static org.apache.flume.channel.file.TestUtils.putEvents; +import static org.apache.flume.channel.file.TestUtils.takeEvents; public class TestFileChannelRollback extends TestFileChannelBase { protected static final Logger LOG = LoggerFactory @@ -117,11 +117,11 @@ public void testRollbackSimulatedCrashWithSink() throws Exception { transaction.rollback(); transaction.close(); - while(runner.isAlive()) { + while (runner.isAlive()) { Thread.sleep(10L); } Assert.assertEquals(numEvents - 1, runner.getCount()); - for(Exception ex : runner.getErrors()) { + for (Exception ex : runner.getErrors()) { LOG.warn("Sink had error", ex); } Assert.assertEquals(Collections.EMPTY_LIST, runner.getErrors()); diff --git a/flume-ng-channels/flume-file-channel/src/test/java/org/apache/flume/channel/file/TestFlumeEventQueue.java b/flume-ng-channels/flume-file-channel/src/test/java/org/apache/flume/channel/file/TestFlumeEventQueue.java index 1adb21a63f..f1700f92f5 100644 --- a/flume-ng-channels/flume-file-channel/src/test/java/org/apache/flume/channel/file/TestFlumeEventQueue.java +++ b/flume-ng-channels/flume-file-channel/src/test/java/org/apache/flume/channel/file/TestFlumeEventQueue.java @@ -55,6 +55,7 @@ private abstract static class EventQueueBackingStoreSupplier { File inflightTakes; File inflightPuts; File queueSetDir; + EventQueueBackingStoreSupplier() { baseDir = Files.createTempDir(); checkpoint = new File(baseDir, "checkpoint"); @@ -62,62 +63,73 @@ private abstract static class EventQueueBackingStoreSupplier { inflightPuts = new File(baseDir, "inflighttakes"); queueSetDir = new File(baseDir, "queueset"); } + File getCheckpoint() { return checkpoint; } + File getInflightPuts() { return inflightPuts; } + File getInflightTakes() { return inflightTakes; } + File getQueueSetDir() { return queueSetDir; } + void delete() { FileUtils.deleteQuietly(baseDir); } - abstract EventQueueBackingStore get() throws Exception ; + + abstract EventQueueBackingStore get() throws Exception; } @Parameters public static Collection data() throws Exception { - Object[][] data = new Object[][] { { - new EventQueueBackingStoreSupplier() { - @Override - public EventQueueBackingStore get() throws Exception { - Assert.assertTrue(baseDir.isDirectory() || baseDir.mkdirs()); - return new EventQueueBackingStoreFileV2(getCheckpoint(), 1000, - "test"); + Object[][] data = new Object[][] { + { + new EventQueueBackingStoreSupplier() { + @Override + public EventQueueBackingStore get() throws Exception { + Assert.assertTrue(baseDir.isDirectory() || baseDir.mkdirs()); + return new EventQueueBackingStoreFileV2(getCheckpoint(), 1000, + "test"); + } } - } - }, { - new EventQueueBackingStoreSupplier() { - @Override - public EventQueueBackingStore get() throws Exception { - Assert.assertTrue(baseDir.isDirectory() || baseDir.mkdirs()); - return new EventQueueBackingStoreFileV3(getCheckpoint(), 1000, - "test"); + }, + { + new EventQueueBackingStoreSupplier() { + @Override + public EventQueueBackingStore get() throws Exception { + Assert.assertTrue(baseDir.isDirectory() || baseDir.mkdirs()); + return new EventQueueBackingStoreFileV3(getCheckpoint(), 1000, "test"); + } } } - } }; + }; return Arrays.asList(data); } public TestFlumeEventQueue(EventQueueBackingStoreSupplier backingStoreSupplier) { this.backingStoreSupplier = backingStoreSupplier; } + @Before public void setup() throws Exception { backingStore = backingStoreSupplier.get(); } + @After public void cleanup() throws IOException { - if(backingStore != null) { + if (backingStore != null) { backingStore.close(); } backingStoreSupplier.delete(); } + @Test public void testCapacity() throws Exception { backingStore.close(); @@ -125,70 +137,76 @@ public void testCapacity() throws Exception { Assert.assertTrue(checkpoint.delete()); backingStore = new EventQueueBackingStoreFileV2(checkpoint, 1, "test"); queue = new FlumeEventQueue(backingStore, - backingStoreSupplier.getInflightTakes(), - backingStoreSupplier.getInflightPuts(), - backingStoreSupplier.getQueueSetDir()); + backingStoreSupplier.getInflightTakes(), + backingStoreSupplier.getInflightPuts(), + backingStoreSupplier.getQueueSetDir()); Assert.assertTrue(queue.addTail(pointer1)); Assert.assertFalse(queue.addTail(pointer2)); } - @Test(expected=IllegalArgumentException.class) + + @Test(expected = IllegalArgumentException.class) public void testInvalidCapacityZero() throws Exception { backingStore.close(); File checkpoint = backingStoreSupplier.getCheckpoint(); Assert.assertTrue(checkpoint.delete()); backingStore = new EventQueueBackingStoreFileV2(checkpoint, 0, "test"); queue = new FlumeEventQueue(backingStore, - backingStoreSupplier.getInflightTakes(), - backingStoreSupplier.getInflightPuts(), - backingStoreSupplier.getQueueSetDir()); + backingStoreSupplier.getInflightTakes(), + backingStoreSupplier.getInflightPuts(), + backingStoreSupplier.getQueueSetDir()); } - @Test(expected=IllegalArgumentException.class) + + @Test(expected = IllegalArgumentException.class) public void testInvalidCapacityNegative() throws Exception { backingStore.close(); File checkpoint = backingStoreSupplier.getCheckpoint(); Assert.assertTrue(checkpoint.delete()); backingStore = new EventQueueBackingStoreFileV2(checkpoint, -1, "test"); queue = new FlumeEventQueue(backingStore, - backingStoreSupplier.getInflightTakes(), - backingStoreSupplier.getInflightPuts(), - backingStoreSupplier.getQueueSetDir()); + backingStoreSupplier.getInflightTakes(), + backingStoreSupplier.getInflightPuts(), + backingStoreSupplier.getQueueSetDir()); } + @Test public void testQueueIsEmptyAfterCreation() throws Exception { queue = new FlumeEventQueue(backingStore, - backingStoreSupplier.getInflightTakes(), - backingStoreSupplier.getInflightPuts(), - backingStoreSupplier.getQueueSetDir()); + backingStoreSupplier.getInflightTakes(), + backingStoreSupplier.getInflightPuts(), + backingStoreSupplier.getQueueSetDir()); Assert.assertNull(queue.removeHead(0L)); } + @Test public void addTail1() throws Exception { queue = new FlumeEventQueue(backingStore, - backingStoreSupplier.getInflightTakes(), - backingStoreSupplier.getInflightPuts(), - backingStoreSupplier.getQueueSetDir()); + backingStoreSupplier.getInflightTakes(), + backingStoreSupplier.getInflightPuts(), + backingStoreSupplier.getQueueSetDir()); Assert.assertTrue(queue.addTail(pointer1)); Assert.assertEquals(pointer1, queue.removeHead(0)); Assert.assertEquals(Sets.newHashSet(), queue.getFileIDs()); } + @Test public void addTail2() throws Exception { queue = new FlumeEventQueue(backingStore, - backingStoreSupplier.getInflightTakes(), - backingStoreSupplier.getInflightPuts(), - backingStoreSupplier.getQueueSetDir()); + backingStoreSupplier.getInflightTakes(), + backingStoreSupplier.getInflightPuts(), + backingStoreSupplier.getQueueSetDir()); Assert.assertTrue(queue.addTail(pointer1)); Assert.assertTrue(queue.addTail(pointer2)); Assert.assertEquals(Sets.newHashSet(1, 2), queue.getFileIDs()); Assert.assertEquals(pointer1, queue.removeHead(0)); Assert.assertEquals(Sets.newHashSet(2), queue.getFileIDs()); } + @Test public void addTailLarge() throws Exception { queue = new FlumeEventQueue(backingStore, - backingStoreSupplier.getInflightTakes(), - backingStoreSupplier.getInflightPuts(), - backingStoreSupplier.getQueueSetDir()); + backingStoreSupplier.getInflightTakes(), + backingStoreSupplier.getInflightPuts(), + backingStoreSupplier.getQueueSetDir()); int size = 500; Set fileIDs = Sets.newHashSet(); for (int i = 1; i <= size; i++) { @@ -203,23 +221,25 @@ public void addTailLarge() throws Exception { } Assert.assertEquals(Sets.newHashSet(), queue.getFileIDs()); } + @Test public void addHead1() throws Exception { queue = new FlumeEventQueue(backingStore, - backingStoreSupplier.getInflightTakes(), - backingStoreSupplier.getInflightPuts(), - backingStoreSupplier.getQueueSetDir()); + backingStoreSupplier.getInflightTakes(), + backingStoreSupplier.getInflightPuts(), + backingStoreSupplier.getQueueSetDir()); Assert.assertTrue(queue.addHead(pointer1)); Assert.assertEquals(Sets.newHashSet(1), queue.getFileIDs()); Assert.assertEquals(pointer1, queue.removeHead(0)); Assert.assertEquals(Sets.newHashSet(), queue.getFileIDs()); } + @Test public void addHead2() throws Exception { queue = new FlumeEventQueue(backingStore, - backingStoreSupplier.getInflightTakes(), - backingStoreSupplier.getInflightPuts(), - backingStoreSupplier.getQueueSetDir()); + backingStoreSupplier.getInflightTakes(), + backingStoreSupplier.getInflightPuts(), + backingStoreSupplier.getQueueSetDir()); queue.replayComplete(); Assert.assertTrue(queue.addHead(pointer1)); Assert.assertTrue(queue.addHead(pointer2)); @@ -227,12 +247,13 @@ public void addHead2() throws Exception { Assert.assertEquals(pointer2, queue.removeHead(0)); Assert.assertEquals(Sets.newHashSet(1), queue.getFileIDs()); } + @Test public void addHeadLarge() throws Exception { queue = new FlumeEventQueue(backingStore, - backingStoreSupplier.getInflightTakes(), - backingStoreSupplier.getInflightPuts(), - backingStoreSupplier.getQueueSetDir()); + backingStoreSupplier.getInflightTakes(), + backingStoreSupplier.getInflightPuts(), + backingStoreSupplier.getQueueSetDir()); queue.replayComplete(); int size = 500; Set fileIDs = Sets.newHashSet(); @@ -248,12 +269,13 @@ public void addHeadLarge() throws Exception { } Assert.assertEquals(Sets.newHashSet(), queue.getFileIDs()); } + @Test public void addTailRemove1() throws Exception { queue = new FlumeEventQueue(backingStore, - backingStoreSupplier.getInflightTakes(), - backingStoreSupplier.getInflightPuts(), - backingStoreSupplier.getQueueSetDir()); + backingStoreSupplier.getInflightTakes(), + backingStoreSupplier.getInflightPuts(), + backingStoreSupplier.getQueueSetDir()); Assert.assertTrue(queue.addTail(pointer1)); Assert.assertEquals(Sets.newHashSet(1), queue.getFileIDs()); Assert.assertTrue(queue.remove(pointer1)); @@ -266,9 +288,9 @@ public void addTailRemove1() throws Exception { @Test public void addTailRemove2() throws Exception { queue = new FlumeEventQueue(backingStore, - backingStoreSupplier.getInflightTakes(), - backingStoreSupplier.getInflightPuts(), - backingStoreSupplier.getQueueSetDir()); + backingStoreSupplier.getInflightTakes(), + backingStoreSupplier.getInflightPuts(), + backingStoreSupplier.getQueueSetDir()); Assert.assertTrue(queue.addTail(pointer1)); Assert.assertTrue(queue.addTail(pointer2)); Assert.assertTrue(queue.remove(pointer1)); @@ -279,31 +301,33 @@ public void addTailRemove2() throws Exception { @Test public void addHeadRemove1() throws Exception { queue = new FlumeEventQueue(backingStore, - backingStoreSupplier.getInflightTakes(), - backingStoreSupplier.getInflightPuts(), - backingStoreSupplier.getQueueSetDir()); + backingStoreSupplier.getInflightTakes(), + backingStoreSupplier.getInflightPuts(), + backingStoreSupplier.getQueueSetDir()); queue.addHead(pointer1); Assert.assertTrue(queue.remove(pointer1)); Assert.assertNull(queue.removeHead(0)); } + @Test public void addHeadRemove2() throws Exception { queue = new FlumeEventQueue(backingStore, - backingStoreSupplier.getInflightTakes(), - backingStoreSupplier.getInflightPuts(), - backingStoreSupplier.getQueueSetDir()); + backingStoreSupplier.getInflightTakes(), + backingStoreSupplier.getInflightPuts(), + backingStoreSupplier.getQueueSetDir()); Assert.assertTrue(queue.addHead(pointer1)); Assert.assertTrue(queue.addHead(pointer2)); Assert.assertTrue(queue.remove(pointer1)); queue.replayComplete(); Assert.assertEquals(pointer2, queue.removeHead(0)); } + @Test public void testUnknownPointerDoesNotCauseSearch() throws Exception { queue = new FlumeEventQueue(backingStore, - backingStoreSupplier.getInflightTakes(), - backingStoreSupplier.getInflightPuts(), - backingStoreSupplier.getQueueSetDir()); + backingStoreSupplier.getInflightTakes(), + backingStoreSupplier.getInflightPuts(), + backingStoreSupplier.getQueueSetDir()); Assert.assertTrue(queue.addHead(pointer1)); Assert.assertTrue(queue.addHead(pointer2)); Assert.assertFalse(queue.remove(pointer3)); // does search @@ -312,44 +336,47 @@ public void testUnknownPointerDoesNotCauseSearch() throws Exception { queue.replayComplete(); Assert.assertEquals(2, queue.getSearchCount()); } - @Test(expected=IllegalStateException.class) + + @Test(expected = IllegalStateException.class) public void testRemoveAfterReplayComplete() throws Exception { queue = new FlumeEventQueue(backingStore, - backingStoreSupplier.getInflightTakes(), - backingStoreSupplier.getInflightPuts(), - backingStoreSupplier.getQueueSetDir()); + backingStoreSupplier.getInflightTakes(), + backingStoreSupplier.getInflightPuts(), + backingStoreSupplier.getQueueSetDir()); queue.replayComplete(); queue.remove(pointer1); } + @Test public void testWrappingCorrectly() throws Exception { queue = new FlumeEventQueue(backingStore, - backingStoreSupplier.getInflightTakes(), - backingStoreSupplier.getInflightPuts(), - backingStoreSupplier.getQueueSetDir()); + backingStoreSupplier.getInflightTakes(), + backingStoreSupplier.getInflightPuts(), + backingStoreSupplier.getQueueSetDir()); int size = Integer.MAX_VALUE; for (int i = 1; i <= size; i++) { - if(!queue.addHead(new FlumeEventPointer(i, i))) { + if (!queue.addHead(new FlumeEventPointer(i, i))) { break; } } - for (int i = queue.getSize()/2; i > 0; i--) { + for (int i = queue.getSize() / 2; i > 0; i--) { Assert.assertNotNull(queue.removeHead(0)); } // addHead below would throw an IndexOOBounds with // bad version of FlumeEventQueue.convert for (int i = 1; i <= size; i++) { - if(!queue.addHead(new FlumeEventPointer(i, i))) { + if (!queue.addHead(new FlumeEventPointer(i, i))) { break; } } } + @Test - public void testInflightPuts() throws Exception{ + public void testInflightPuts() throws Exception { queue = new FlumeEventQueue(backingStore, - backingStoreSupplier.getInflightTakes(), - backingStoreSupplier.getInflightPuts(), - backingStoreSupplier.getQueueSetDir()); + backingStoreSupplier.getInflightTakes(), + backingStoreSupplier.getInflightPuts(), + backingStoreSupplier.getQueueSetDir()); long txnID1 = new Random().nextInt(Integer.MAX_VALUE - 1); long txnID2 = txnID1 + 1; queue.addWithoutCommit(new FlumeEventPointer(1, 1), txnID1); @@ -358,16 +385,13 @@ public void testInflightPuts() throws Exception{ queue.checkpoint(true); TimeUnit.SECONDS.sleep(3L); queue = new FlumeEventQueue(backingStore, - backingStoreSupplier.getInflightTakes(), - backingStoreSupplier.getInflightPuts(), - backingStoreSupplier.getQueueSetDir()); + backingStoreSupplier.getInflightTakes(), + backingStoreSupplier.getInflightPuts(), + backingStoreSupplier.getQueueSetDir()); SetMultimap deserializedMap = queue.deserializeInflightPuts(); - Assert.assertTrue(deserializedMap.get( - txnID1).contains(new FlumeEventPointer(1, 1).toLong())); - Assert.assertTrue(deserializedMap.get( - txnID1).contains(new FlumeEventPointer(2, 1).toLong())); - Assert.assertTrue(deserializedMap.get( - txnID2).contains(new FlumeEventPointer(2, 2).toLong())); + Assert.assertTrue(deserializedMap.get(txnID1).contains(new FlumeEventPointer(1, 1).toLong())); + Assert.assertTrue(deserializedMap.get(txnID1).contains(new FlumeEventPointer(2, 1).toLong())); + Assert.assertTrue(deserializedMap.get(txnID2).contains(new FlumeEventPointer(2, 2).toLong())); } @Test diff --git a/flume-ng-channels/flume-file-channel/src/test/java/org/apache/flume/channel/file/TestIntegration.java b/flume-ng-channels/flume-file-channel/src/test/java/org/apache/flume/channel/file/TestIntegration.java index 2fbe116bf7..a138ed4dbb 100644 --- a/flume-ng-channels/flume-file-channel/src/test/java/org/apache/flume/channel/file/TestIntegration.java +++ b/flume-ng-channels/flume-file-channel/src/test/java/org/apache/flume/channel/file/TestIntegration.java @@ -18,13 +18,8 @@ */ package org.apache.flume.channel.file; -import java.io.File; -import java.io.IOException; -import java.util.Collections; -import java.util.List; -import java.util.UUID; -import java.util.concurrent.TimeUnit; - +import com.google.common.collect.Lists; +import com.google.common.io.Files; import org.apache.commons.io.FileUtils; import org.apache.flume.Context; import org.apache.flume.conf.Configurables; @@ -37,8 +32,12 @@ import org.slf4j.Logger; import org.slf4j.LoggerFactory; -import com.google.common.collect.Lists; -import com.google.common.io.Files; +import java.io.File; +import java.io.IOException; +import java.util.Collections; +import java.util.List; +import java.util.UUID; +import java.util.concurrent.TimeUnit; public class TestIntegration { @@ -58,19 +57,21 @@ public void setup() { dataDirs = new File[3]; dataDir = ""; for (int i = 0; i < dataDirs.length; i++) { - dataDirs[i] = new File(baseDir, "data" + (i+1)); + dataDirs[i] = new File(baseDir, "data" + (i + 1)); Assert.assertTrue(dataDirs[i].mkdirs() || dataDirs[i].isDirectory()); dataDir += dataDirs[i].getAbsolutePath() + ","; } dataDir = dataDir.substring(0, dataDir.length() - 1); } + @After public void teardown() { - if(channel != null && channel.isOpen()) { + if (channel != null && channel.isOpen()) { channel.stop(); } FileUtils.deleteQuietly(baseDir); } + @Test public void testIntegration() throws IOException, InterruptedException { // set shorter checkpoint and filesize to ensure @@ -106,11 +107,11 @@ public void testIntegration() throws IOException, InterruptedException { TimeUnit.SECONDS.sleep(30); // shutdown source sourceRunner.shutdown(); - while(sourceRunner.isAlive()) { + while (sourceRunner.isAlive()) { Thread.sleep(10L); } // wait for queue to clear - while(channel.getDepth() > 0) { + while (channel.getDepth() > 0) { Thread.sleep(10L); } // shutdown size @@ -122,15 +123,15 @@ public void testIntegration() throws IOException, InterruptedException { logs.addAll(LogUtils.getLogs(dataDirs[i])); } LOG.info("Total Number of Logs = " + logs.size()); - for(File logFile : logs) { + for (File logFile : logs) { LOG.info("LogFile = " + logFile); } LOG.info("Source processed " + sinkRunner.getCount()); LOG.info("Sink processed " + sourceRunner.getCount()); - for(Exception ex : sourceRunner.getErrors()) { + for (Exception ex : sourceRunner.getErrors()) { LOG.warn("Source had error", ex); } - for(Exception ex : sinkRunner.getErrors()) { + for (Exception ex : sinkRunner.getErrors()) { LOG.warn("Sink had error", ex); } Assert.assertEquals(sinkRunner.getCount(), sinkRunner.getCount()); diff --git a/flume-ng-channels/flume-file-channel/src/test/java/org/apache/flume/channel/file/TestLog.java b/flume-ng-channels/flume-file-channel/src/test/java/org/apache/flume/channel/file/TestLog.java index b1f59cdb22..f7f0950e8e 100644 --- a/flume-ng-channels/flume-file-channel/src/test/java/org/apache/flume/channel/file/TestLog.java +++ b/flume-ng-channels/flume-file-channel/src/test/java/org/apache/flume/channel/file/TestLog.java @@ -18,14 +18,8 @@ */ package org.apache.flume.channel.file; -import static org.mockito.Mockito.*; -import java.io.File; -import java.io.FileOutputStream; -import java.io.IOException; -import java.nio.channels.*; -import java.util.Collection; -import java.util.List; - +import com.google.common.collect.Lists; +import com.google.common.io.Files; import org.apache.commons.io.FileUtils; import org.junit.After; import org.junit.Assert; @@ -34,8 +28,13 @@ import org.slf4j.Logger; import org.slf4j.LoggerFactory; -import com.google.common.collect.Lists; -import com.google.common.io.Files; +import java.io.File; +import java.io.FileOutputStream; +import java.io.IOException; +import java.util.List; + +import static org.mockito.Mockito.mock; +import static org.mockito.Mockito.when; public class TestLog { private static final Logger LOGGER = LoggerFactory.getLogger(TestLog.class); @@ -45,6 +44,7 @@ public class TestLog { private File checkpointDir; private File[] dataDirs; private long transactionID; + @Before public void setup() throws IOException { transactionID = 0; @@ -56,15 +56,20 @@ public void setup() throws IOException { dataDirs[i] = Files.createTempDir(); Assert.assertTrue(dataDirs[i].isDirectory()); } - log = new Log.Builder().setCheckpointInterval(1L).setMaxFileSize( - MAX_FILE_SIZE).setQueueSize(CAPACITY).setCheckpointDir( - checkpointDir).setLogDirs(dataDirs).setCheckpointOnClose(false) - .setChannelName("testlog").build(); + log = new Log.Builder().setCheckpointInterval(1L) + .setMaxFileSize(MAX_FILE_SIZE) + .setQueueSize(CAPACITY) + .setCheckpointDir(checkpointDir) + .setLogDirs(dataDirs) + .setCheckpointOnClose(false) + .setChannelName("testlog") + .build(); log.replay(); } + @After - public void cleanup() throws Exception{ - if(log != null) { + public void cleanup() throws Exception { + if (log != null) { log.close(); } FileUtils.deleteQuietly(checkpointDir); @@ -72,13 +77,14 @@ public void cleanup() throws Exception{ FileUtils.deleteQuietly(dataDirs[i]); } } + /** * Test that we can put, commit and then get. Note that get is * not transactional so the commit is not required. */ @Test public void testPutGet() - throws IOException, InterruptedException, NoopRecordException, CorruptEventException { + throws IOException, InterruptedException, NoopRecordException, CorruptEventException { FlumeEvent eventIn = TestUtils.newPersistableEvent(); long transactionID = ++this.transactionID; FlumeEventPointer eventPointer = log.put(transactionID, eventIn); @@ -89,9 +95,10 @@ public void testPutGet() Assert.assertEquals(eventIn.getHeaders(), eventOut.getHeaders()); Assert.assertArrayEquals(eventIn.getBody(), eventOut.getBody()); } + @Test public void testRoll() - throws IOException, InterruptedException, NoopRecordException, CorruptEventException { + throws IOException, InterruptedException, NoopRecordException, CorruptEventException { log.shutdownWorker(); Thread.sleep(1000); for (int i = 0; i < 1000; i++) { @@ -105,9 +112,9 @@ public void testRoll() Assert.assertArrayEquals(eventIn.getBody(), eventOut.getBody()); } int logCount = 0; - for(File dataDir : dataDirs) { - for(File logFile : dataDir.listFiles()) { - if(logFile.getName().startsWith("log-")) { + for (File dataDir : dataDirs) { + for (File logFile : dataDir.listFiles()) { + if (logFile.getName().startsWith("log-")) { logCount++; } } @@ -115,26 +122,30 @@ public void testRoll() // 93 (*2 for meta) files with TestLog.MAX_FILE_SIZE=1000 Assert.assertEquals(186, logCount); } + /** * After replay of the log, we should find the event because the put * was committed */ @Test public void testPutCommit() - throws IOException, InterruptedException, NoopRecordException, CorruptEventException { + throws IOException, InterruptedException, NoopRecordException, CorruptEventException { FlumeEvent eventIn = TestUtils.newPersistableEvent(); long transactionID = ++this.transactionID; FlumeEventPointer eventPointerIn = log.put(transactionID, eventIn); log.commitPut(transactionID); log.close(); - log = new Log.Builder().setCheckpointInterval( - Long.MAX_VALUE).setMaxFileSize( - FileChannelConfiguration.DEFAULT_MAX_FILE_SIZE).setQueueSize( - CAPACITY).setCheckpointDir(checkpointDir).setLogDirs( - dataDirs).setChannelName("testlog").build(); + log = new Log.Builder().setCheckpointInterval(Long.MAX_VALUE) + .setMaxFileSize(FileChannelConfiguration.DEFAULT_MAX_FILE_SIZE) + .setQueueSize(CAPACITY) + .setCheckpointDir(checkpointDir) + .setLogDirs(dataDirs) + .setChannelName("testlog") + .build(); log.replay(); takeAndVerify(eventPointerIn, eventIn); } + /** * After replay of the log, we should not find the event because the * put was rolled back @@ -146,39 +157,44 @@ public void testPutRollback() throws IOException, InterruptedException { log.put(transactionID, eventIn); log.rollback(transactionID); // rolled back so it should not be replayed log.close(); - log = new Log.Builder().setCheckpointInterval( - Long.MAX_VALUE).setMaxFileSize( - FileChannelConfiguration.DEFAULT_MAX_FILE_SIZE).setQueueSize( - CAPACITY).setCheckpointDir(checkpointDir).setLogDirs( - dataDirs).setChannelName("testlog").build(); + log = new Log.Builder().setCheckpointInterval(Long.MAX_VALUE) + .setMaxFileSize(FileChannelConfiguration.DEFAULT_MAX_FILE_SIZE) + .setQueueSize(CAPACITY) + .setCheckpointDir(checkpointDir) + .setLogDirs(dataDirs) + .setChannelName("testlog") + .build(); log.replay(); FlumeEventQueue queue = log.getFlumeEventQueue(); Assert.assertNull(queue.removeHead(transactionID)); } + @Test public void testMinimumRequiredSpaceTooSmallOnStartup() throws IOException, - InterruptedException { + InterruptedException { log.close(); - log = new Log.Builder().setCheckpointInterval( - Long.MAX_VALUE).setMaxFileSize( - FileChannelConfiguration.DEFAULT_MAX_FILE_SIZE).setQueueSize( - CAPACITY).setCheckpointDir(checkpointDir).setLogDirs( - dataDirs).setChannelName("testlog"). - setMinimumRequiredSpace(Long.MAX_VALUE).build(); + log = new Log.Builder().setCheckpointInterval(Long.MAX_VALUE) + .setMaxFileSize(FileChannelConfiguration.DEFAULT_MAX_FILE_SIZE) + .setQueueSize(CAPACITY) + .setCheckpointDir(checkpointDir) + .setLogDirs(dataDirs) + .setChannelName("testlog") + .setMinimumRequiredSpace(Long.MAX_VALUE) + .build(); try { log.replay(); Assert.fail(); } catch (IOException e) { - Assert.assertTrue(e.getMessage(), e.getMessage() - .startsWith("Usable space exhausted")); + Assert.assertTrue(e.getMessage(), + e.getMessage().startsWith("Usable space exhausted")); } } + /** * There is a race here in that someone could take up some space */ @Test - public void testMinimumRequiredSpaceTooSmallForPut() throws IOException, - InterruptedException { + public void testMinimumRequiredSpaceTooSmallForPut() throws IOException, InterruptedException { try { doTestMinimumRequiredSpaceTooSmallForPut(); } catch (IOException e) { @@ -189,23 +205,26 @@ public void testMinimumRequiredSpaceTooSmallForPut() throws IOException, doTestMinimumRequiredSpaceTooSmallForPut(); } } + public void doTestMinimumRequiredSpaceTooSmallForPut() throws IOException, - InterruptedException { + InterruptedException { long minimumRequiredSpace = checkpointDir.getUsableSpace() - - (10L* 1024L * 1024L); + (10L * 1024L * 1024L); log.close(); - log = new Log.Builder().setCheckpointInterval( - Long.MAX_VALUE).setMaxFileSize( - FileChannelConfiguration.DEFAULT_MAX_FILE_SIZE).setQueueSize( - CAPACITY).setCheckpointDir(checkpointDir).setLogDirs( - dataDirs).setChannelName("testlog"). - setMinimumRequiredSpace(minimumRequiredSpace) - .setUsableSpaceRefreshInterval(1L).build(); + log = new Log.Builder().setCheckpointInterval(Long.MAX_VALUE) + .setMaxFileSize(FileChannelConfiguration.DEFAULT_MAX_FILE_SIZE) + .setQueueSize(CAPACITY) + .setCheckpointDir(checkpointDir) + .setLogDirs(dataDirs) + .setChannelName("testlog") + .setMinimumRequiredSpace(minimumRequiredSpace) + .setUsableSpaceRefreshInterval(1L) + .build(); log.replay(); File filler = new File(checkpointDir, "filler"); byte[] buffer = new byte[64 * 1024]; FileOutputStream out = new FileOutputStream(filler); - while(checkpointDir.getUsableSpace() > minimumRequiredSpace) { + while (checkpointDir.getUsableSpace() > minimumRequiredSpace) { out.write(buffer); } out.close(); @@ -215,10 +234,11 @@ public void doTestMinimumRequiredSpaceTooSmallForPut() throws IOException, log.put(transactionID, eventIn); Assert.fail(); } catch (IOException e) { - Assert.assertTrue(e.getMessage(), e.getMessage() - .startsWith("Usable space exhausted")); + Assert.assertTrue(e.getMessage(), + e.getMessage().startsWith("Usable space exhausted")); } } + /** * After replay of the log, we should not find the event because the take * was committed @@ -233,11 +253,13 @@ public void testPutTakeCommit() throws IOException, InterruptedException { log.take(takeTransactionID, eventPointer); log.commitTake(takeTransactionID); log.close(); - new Log.Builder().setCheckpointInterval( - Long.MAX_VALUE).setMaxFileSize( - FileChannelConfiguration.DEFAULT_MAX_FILE_SIZE).setQueueSize( - 1).setCheckpointDir(checkpointDir).setLogDirs(dataDirs) - .setChannelName("testlog").build(); + new Log.Builder().setCheckpointInterval(Long.MAX_VALUE) + .setMaxFileSize(FileChannelConfiguration.DEFAULT_MAX_FILE_SIZE) + .setQueueSize(1) + .setCheckpointDir(checkpointDir) + .setLogDirs(dataDirs) + .setChannelName("testlog") + .build(); log.replay(); FlumeEventQueue queue = log.getFlumeEventQueue(); Assert.assertNull(queue.removeHead(0)); @@ -249,16 +271,18 @@ public void testPutTakeCommit() throws IOException, InterruptedException { */ @Test public void testPutTakeRollbackLogReplayV1() - throws IOException, InterruptedException, NoopRecordException, CorruptEventException { + throws IOException, InterruptedException, NoopRecordException, CorruptEventException { doPutTakeRollback(true); } + @Test public void testPutTakeRollbackLogReplayV2() - throws IOException, InterruptedException, NoopRecordException, CorruptEventException { + throws IOException, InterruptedException, NoopRecordException, CorruptEventException { doPutTakeRollback(false); } + public void doPutTakeRollback(boolean useLogReplayV1) - throws IOException, InterruptedException, NoopRecordException, CorruptEventException { + throws IOException, InterruptedException, NoopRecordException, CorruptEventException { FlumeEvent eventIn = TestUtils.newPersistableEvent(); long putTransactionID = ++transactionID; FlumeEventPointer eventPointerIn = log.put(putTransactionID, eventIn); @@ -267,11 +291,14 @@ public void doPutTakeRollback(boolean useLogReplayV1) log.take(takeTransactionID, eventPointerIn); log.rollback(takeTransactionID); log.close(); - new Log.Builder().setCheckpointInterval( - Long.MAX_VALUE).setMaxFileSize( - FileChannelConfiguration.DEFAULT_MAX_FILE_SIZE).setQueueSize( - 1).setCheckpointDir(checkpointDir).setLogDirs(dataDirs) - .setChannelName("testlog").setUseLogReplayV1(useLogReplayV1).build(); + new Log.Builder().setCheckpointInterval(Long.MAX_VALUE) + .setMaxFileSize(FileChannelConfiguration.DEFAULT_MAX_FILE_SIZE) + .setQueueSize(1) + .setCheckpointDir(checkpointDir) + .setLogDirs(dataDirs) + .setChannelName("testlog") + .setUseLogReplayV1(useLogReplayV1) + .build(); log.replay(); takeAndVerify(eventPointerIn, eventIn); } @@ -281,11 +308,13 @@ public void testCommitNoPut() throws IOException, InterruptedException { long putTransactionID = ++transactionID; log.commitPut(putTransactionID); log.close(); - new Log.Builder().setCheckpointInterval( - Long.MAX_VALUE).setMaxFileSize( - FileChannelConfiguration.DEFAULT_MAX_FILE_SIZE).setQueueSize( - 1).setCheckpointDir(checkpointDir).setLogDirs(dataDirs) - .setChannelName("testlog").build(); + new Log.Builder().setCheckpointInterval(Long.MAX_VALUE) + .setMaxFileSize(FileChannelConfiguration.DEFAULT_MAX_FILE_SIZE) + .setQueueSize(1) + .setCheckpointDir(checkpointDir) + .setLogDirs(dataDirs) + .setChannelName("testlog") + .build(); log.replay(); FlumeEventQueue queue = log.getFlumeEventQueue(); FlumeEventPointer eventPointerOut = queue.removeHead(0); @@ -297,11 +326,13 @@ public void testCommitNoTake() throws IOException, InterruptedException { long putTransactionID = ++transactionID; log.commitTake(putTransactionID); log.close(); - new Log.Builder().setCheckpointInterval( - Long.MAX_VALUE).setMaxFileSize( - FileChannelConfiguration.DEFAULT_MAX_FILE_SIZE).setQueueSize( - 1).setCheckpointDir(checkpointDir).setLogDirs(dataDirs) - .setChannelName("testlog").build(); + new Log.Builder().setCheckpointInterval(Long.MAX_VALUE) + .setMaxFileSize(FileChannelConfiguration.DEFAULT_MAX_FILE_SIZE) + .setQueueSize(1) + .setCheckpointDir(checkpointDir) + .setLogDirs(dataDirs) + .setChannelName("testlog") + .build(); log.replay(); FlumeEventQueue queue = log.getFlumeEventQueue(); FlumeEventPointer eventPointerOut = queue.removeHead(0); @@ -313,11 +344,13 @@ public void testRollbackNoPutTake() throws IOException, InterruptedException { long putTransactionID = ++transactionID; log.rollback(putTransactionID); log.close(); - new Log.Builder().setCheckpointInterval( - Long.MAX_VALUE).setMaxFileSize( - FileChannelConfiguration.DEFAULT_MAX_FILE_SIZE).setQueueSize( - 1).setCheckpointDir(checkpointDir).setLogDirs(dataDirs) - .setChannelName("testlog").build(); + new Log.Builder().setCheckpointInterval(Long.MAX_VALUE) + .setMaxFileSize(FileChannelConfiguration.DEFAULT_MAX_FILE_SIZE) + .setQueueSize(1) + .setCheckpointDir(checkpointDir) + .setLogDirs(dataDirs) + .setChannelName("testlog") + .build(); log.replay(); FlumeEventQueue queue = log.getFlumeEventQueue(); FlumeEventPointer eventPointerOut = queue.removeHead(0); @@ -337,7 +370,7 @@ public void testGetLogs() throws IOException { File logGzip = new File(logDir, Log.PREFIX + i + ".gz"); Assert.assertTrue(metaDataFile.isFile() || metaDataFile.createNewFile()); Assert.assertTrue(metaDataTempFile.isFile() || - metaDataTempFile.createNewFile()); + metaDataTempFile.createNewFile()); Assert.assertTrue(log.isFile() || logGzip.createNewFile()); } List actual = LogUtils.getLogs(logDir); @@ -345,31 +378,38 @@ public void testGetLogs() throws IOException { LogUtils.sort(expected); Assert.assertEquals(expected, actual); } + @Test public void testReplayFailsWithAllEmptyLogMetaDataNormalReplay() throws IOException, InterruptedException { doTestReplayFailsWithAllEmptyLogMetaData(false); } + @Test public void testReplayFailsWithAllEmptyLogMetaDataFastReplay() throws IOException, InterruptedException { doTestReplayFailsWithAllEmptyLogMetaData(true); } + public void doTestReplayFailsWithAllEmptyLogMetaData(boolean useFastReplay) throws IOException, InterruptedException { // setup log with correct fast replay parameter log.close(); - log = new Log.Builder().setCheckpointInterval(1L).setMaxFileSize( - MAX_FILE_SIZE).setQueueSize(CAPACITY).setCheckpointDir( - checkpointDir).setLogDirs(dataDirs) - .setChannelName("testlog").setUseFastReplay(useFastReplay).build(); + log = new Log.Builder().setCheckpointInterval(1L) + .setMaxFileSize(MAX_FILE_SIZE) + .setQueueSize(CAPACITY) + .setCheckpointDir(checkpointDir) + .setLogDirs(dataDirs) + .setChannelName("testlog") + .setUseFastReplay(useFastReplay) + .build(); log.replay(); FlumeEvent eventIn = TestUtils.newPersistableEvent(); long transactionID = ++this.transactionID; log.put(transactionID, eventIn); log.commitPut(transactionID); log.close(); - if(useFastReplay) { + if (useFastReplay) { FileUtils.deleteQuietly(checkpointDir); Assert.assertTrue(checkpointDir.mkdir()); } @@ -378,41 +418,50 @@ public void doTestReplayFailsWithAllEmptyLogMetaData(boolean useFastReplay) logFiles.addAll(LogUtils.getLogs(dataDirs[i])); } Assert.assertTrue(logFiles.size() > 0); - for(File logFile : logFiles) { + for (File logFile : logFiles) { File logFileMeta = Serialization.getMetaDataFile(logFile); Assert.assertTrue(logFileMeta.delete()); Assert.assertTrue(logFileMeta.createNewFile()); } - log = new Log.Builder().setCheckpointInterval(1L).setMaxFileSize( - MAX_FILE_SIZE).setQueueSize(CAPACITY).setCheckpointDir( - checkpointDir).setLogDirs(dataDirs) - .setChannelName("testlog").setUseFastReplay(useFastReplay).build(); + log = new Log.Builder().setCheckpointInterval(1L) + .setMaxFileSize(MAX_FILE_SIZE) + .setQueueSize(CAPACITY) + .setCheckpointDir(checkpointDir) + .setLogDirs(dataDirs) + .setChannelName("testlog") + .setUseFastReplay(useFastReplay) + .build(); try { log.replay(); Assert.fail(); - } catch(IllegalStateException expected) { + } catch (IllegalStateException expected) { String msg = expected.getMessage(); Assert.assertNotNull(msg); Assert.assertTrue(msg, msg.contains(".meta is empty, but log")); } } + @Test public void testReplaySucceedsWithUnusedEmptyLogMetaDataNormalReplay() - throws IOException, InterruptedException, NoopRecordException, CorruptEventException { + throws IOException, InterruptedException, NoopRecordException, CorruptEventException { FlumeEvent eventIn = TestUtils.newPersistableEvent(); long transactionID = ++this.transactionID; FlumeEventPointer eventPointer = log.put(transactionID, eventIn); log.commitPut(transactionID); // this is not required since log.close(); - log = new Log.Builder().setCheckpointInterval(1L).setMaxFileSize( - MAX_FILE_SIZE).setQueueSize(CAPACITY).setCheckpointDir( - checkpointDir).setLogDirs(dataDirs) - .setChannelName("testlog").build(); + log = new Log.Builder().setCheckpointInterval(1L) + .setMaxFileSize(MAX_FILE_SIZE) + .setQueueSize(CAPACITY) + .setCheckpointDir(checkpointDir) + .setLogDirs(dataDirs) + .setChannelName("testlog") + .build(); doTestReplaySucceedsWithUnusedEmptyLogMetaData(eventIn, eventPointer); } + @Test public void testReplaySucceedsWithUnusedEmptyLogMetaDataFastReplay() - throws IOException, InterruptedException, NoopRecordException, CorruptEventException { + throws IOException, InterruptedException, NoopRecordException, CorruptEventException { FlumeEvent eventIn = TestUtils.newPersistableEvent(); long transactionID = ++this.transactionID; FlumeEventPointer eventPointer = log.put(transactionID, eventIn); @@ -421,18 +470,23 @@ public void testReplaySucceedsWithUnusedEmptyLogMetaDataFastReplay() checkpointDir = Files.createTempDir(); FileUtils.forceDeleteOnExit(checkpointDir); Assert.assertTrue(checkpointDir.isDirectory()); - log = new Log.Builder().setCheckpointInterval(1L).setMaxFileSize( - MAX_FILE_SIZE).setQueueSize(CAPACITY).setCheckpointDir( - checkpointDir).setLogDirs(dataDirs) - .setChannelName("testlog").setUseFastReplay(true).build(); + log = new Log.Builder().setCheckpointInterval(1L) + .setMaxFileSize(MAX_FILE_SIZE) + .setQueueSize(CAPACITY) + .setCheckpointDir(checkpointDir) + .setLogDirs(dataDirs) + .setChannelName("testlog") + .setUseFastReplay(true) + .build(); doTestReplaySucceedsWithUnusedEmptyLogMetaData(eventIn, eventPointer); } + public void doTestReplaySucceedsWithUnusedEmptyLogMetaData(FlumeEvent eventIn, - FlumeEventPointer eventPointer) throws IOException, - InterruptedException, NoopRecordException, CorruptEventException { + FlumeEventPointer eventPointer) + throws IOException, InterruptedException, NoopRecordException, CorruptEventException { for (int i = 0; i < dataDirs.length; i++) { - for(File logFile : LogUtils.getLogs(dataDirs[i])) { - if(logFile.length() == 0L) { + for (File logFile : LogUtils.getLogs(dataDirs[i])) { + if (logFile.length() == 0L) { File logFileMeta = Serialization.getMetaDataFile(logFile); Assert.assertTrue(logFileMeta.delete()); Assert.assertTrue(logFileMeta.createNewFile()); @@ -445,16 +499,15 @@ public void doTestReplaySucceedsWithUnusedEmptyLogMetaData(FlumeEvent eventIn, Assert.assertEquals(eventIn.getHeaders(), eventOut.getHeaders()); Assert.assertArrayEquals(eventIn.getBody(), eventOut.getBody()); } + @Test public void testCachedFSUsableSpace() throws Exception { File fs = mock(File.class); when(fs.getUsableSpace()).thenReturn(Long.MAX_VALUE); - LogFile.CachedFSUsableSpace cachedFS = - new LogFile.CachedFSUsableSpace(fs, 1000L); + LogFile.CachedFSUsableSpace cachedFS = new LogFile.CachedFSUsableSpace(fs, 1000L); Assert.assertEquals(cachedFS.getUsableSpace(), Long.MAX_VALUE); cachedFS.decrement(Integer.MAX_VALUE); - Assert.assertEquals(cachedFS.getUsableSpace(), - Long.MAX_VALUE - Integer.MAX_VALUE); + Assert.assertEquals(cachedFS.getUsableSpace(), Long.MAX_VALUE - Integer.MAX_VALUE); try { cachedFS.decrement(-1); Assert.fail(); @@ -463,20 +516,22 @@ public void testCachedFSUsableSpace() throws Exception { } when(fs.getUsableSpace()).thenReturn(Long.MAX_VALUE - 1L); Thread.sleep(1100); - Assert.assertEquals(cachedFS.getUsableSpace(), - Long.MAX_VALUE - 1L); + Assert.assertEquals(cachedFS.getUsableSpace(), Long.MAX_VALUE - 1L); } @Test public void testCheckpointOnClose() throws Exception { log.close(); - log = new Log.Builder().setCheckpointInterval(1L).setMaxFileSize( - MAX_FILE_SIZE).setQueueSize(CAPACITY).setCheckpointDir( - checkpointDir).setLogDirs(dataDirs).setCheckpointOnClose(true) - .setChannelName("testLog").build(); + log = new Log.Builder().setCheckpointInterval(1L) + .setMaxFileSize(MAX_FILE_SIZE) + .setQueueSize(CAPACITY) + .setCheckpointDir(checkpointDir) + .setLogDirs(dataDirs) + .setCheckpointOnClose(true) + .setChannelName("testLog") + .build(); log.replay(); - // 1 Write One Event FlumeEvent eventIn = TestUtils.newPersistableEvent(); log.put(transactionID, eventIn); @@ -484,20 +539,19 @@ public void testCheckpointOnClose() throws Exception { // 2 Check state of checkpoint before close File checkPointMetaFile = - FileUtils.listFiles(checkpointDir,new String[]{"meta"},false).iterator().next(); - long before = FileUtils.checksumCRC32( checkPointMetaFile ); + FileUtils.listFiles(checkpointDir, new String[] { "meta" }, false).iterator().next(); + long before = FileUtils.checksumCRC32(checkPointMetaFile); // 3 Close Log log.close(); // 4 Verify that checkpoint was modified on close - long after = FileUtils.checksumCRC32( checkPointMetaFile ); - Assert.assertFalse( before == after ); + long after = FileUtils.checksumCRC32(checkPointMetaFile); + Assert.assertFalse(before == after); } - private void takeAndVerify(FlumeEventPointer eventPointerIn, - FlumeEvent eventIn) - throws IOException, InterruptedException, NoopRecordException, CorruptEventException { + private void takeAndVerify(FlumeEventPointer eventPointerIn, FlumeEvent eventIn) + throws IOException, InterruptedException, NoopRecordException, CorruptEventException { FlumeEventQueue queue = log.getFlumeEventQueue(); FlumeEventPointer eventPointerOut = queue.removeHead(0); Assert.assertNotNull(eventPointerOut); diff --git a/flume-ng-channels/flume-file-channel/src/test/java/org/apache/flume/channel/file/TestLogFile.java b/flume-ng-channels/flume-file-channel/src/test/java/org/apache/flume/channel/file/TestLogFile.java index 976a112999..d945c7fec3 100644 --- a/flume-ng-channels/flume-file-channel/src/test/java/org/apache/flume/channel/file/TestLogFile.java +++ b/flume-ng-channels/flume-file-channel/src/test/java/org/apache/flume/channel/file/TestLogFile.java @@ -18,6 +18,16 @@ */ package org.apache.flume.channel.file; +import com.google.common.base.Throwables; +import com.google.common.collect.Maps; +import com.google.common.io.Files; +import org.apache.commons.io.FileUtils; +import org.apache.flume.channel.file.proto.ProtosFactory; +import org.junit.After; +import org.junit.Assert; +import org.junit.Before; +import org.junit.Test; + import java.io.File; import java.io.FileInputStream; import java.io.IOException; @@ -28,33 +38,21 @@ import java.util.Collections; import java.util.List; import java.util.Map; -import java.util.Random; import java.util.concurrent.Callable; import java.util.concurrent.CompletionService; -import java.util.concurrent.CountDownLatch; import java.util.concurrent.CyclicBarrier; import java.util.concurrent.ExecutorCompletionService; import java.util.concurrent.ExecutorService; import java.util.concurrent.Executors; import java.util.concurrent.atomic.AtomicLong; -import org.apache.commons.io.FileUtils; -import org.apache.flume.channel.file.proto.ProtosFactory; -import org.junit.After; -import org.junit.Assert; -import org.junit.Before; -import org.junit.Test; - -import com.google.common.base.Throwables; -import com.google.common.collect.Maps; -import com.google.common.io.Files; - public class TestLogFile { private int fileID; private long transactionID; private LogFile.Writer logFileWriter; private File dataDir; private File dataFile; + @Before public void setup() throws IOException { fileID = 1; @@ -65,28 +63,30 @@ public void setup() throws IOException { logFileWriter = LogFileFactory.getWriter(dataFile, fileID, Integer.MAX_VALUE, null, null, null, Long.MAX_VALUE, true, 0); } + @After public void cleanup() throws IOException { try { - if(logFileWriter != null) { + if (logFileWriter != null) { logFileWriter.close(); } } finally { FileUtils.deleteQuietly(dataDir); } } + @Test public void testWriterRefusesToOverwriteFile() throws IOException { Assert.assertTrue(dataFile.isFile() || dataFile.createNewFile()); try { LogFileFactory.getWriter(dataFile, fileID, Integer.MAX_VALUE, null, null, - null, Long.MAX_VALUE, true, 0); + null, Long.MAX_VALUE, true, 0); Assert.fail(); } catch (IllegalStateException e) { - Assert.assertEquals("File already exists " + dataFile.getAbsolutePath(), - e.getMessage()); + Assert.assertEquals("File already exists " + dataFile.getAbsolutePath(), e.getMessage()); } } + @Test public void testWriterFailsWithDirectory() throws IOException { FileUtils.deleteQuietly(dataFile); @@ -94,30 +94,29 @@ public void testWriterFailsWithDirectory() throws IOException { Assert.assertTrue(dataFile.mkdirs()); try { LogFileFactory.getWriter(dataFile, fileID, Integer.MAX_VALUE, null, null, - null, Long.MAX_VALUE, true, 0); + null, Long.MAX_VALUE, true, 0); Assert.fail(); } catch (IllegalStateException e) { - Assert.assertEquals("File already exists " + dataFile.getAbsolutePath(), - e.getMessage()); + Assert.assertEquals("File already exists " + dataFile.getAbsolutePath(), e.getMessage()); } } + @Test public void testPutGet() throws InterruptedException, IOException { final List errors = Collections.synchronizedList(new ArrayList()); ExecutorService executorService = Executors.newFixedThreadPool(10); CompletionService completionService = new ExecutorCompletionService - (executorService); - final LogFile.RandomReader logFileReader = - LogFileFactory.getRandomReader(dataFile, null, true); + (executorService); + final LogFile.RandomReader logFileReader = LogFileFactory.getRandomReader(dataFile, null, true); for (int i = 0; i < 1000; i++) { // first try and throw failures synchronized (errors) { - for(Throwable throwable : errors) { + for (Throwable throwable : errors) { Throwables.propagateIfInstanceOf(throwable, AssertionError.class); } // then throw errors - for(Throwable throwable : errors) { + for (Throwable throwable : errors) { Throwables.propagate(throwable); } } @@ -134,7 +133,7 @@ public void run() { FlumeEvent eventOut = logFileReader.get(offset); Assert.assertEquals(eventIn.getHeaders(), eventOut.getHeaders()); Assert.assertTrue(Arrays.equals(eventIn.getBody(), eventOut.getBody())); - } catch(Throwable throwable) { + } catch (Throwable throwable) { synchronized (errors) { errors.add(throwable); } @@ -143,26 +142,26 @@ public void run() { }, null); } - for(int i = 0; i < 1000; i++) { + for (int i = 0; i < 1000; i++) { completionService.take(); } // first try and throw failures - for(Throwable throwable : errors) { + for (Throwable throwable : errors) { Throwables.propagateIfInstanceOf(throwable, AssertionError.class); } // then throw errors - for(Throwable throwable : errors) { + for (Throwable throwable : errors) { Throwables.propagate(throwable); } } + @Test public void testReader() throws InterruptedException, IOException, - CorruptEventException { + CorruptEventException { Map puts = Maps.newHashMap(); for (int i = 0; i < 1000; i++) { FlumeEvent eventIn = TestUtils.newPersistableEvent(); - Put put = new Put(++transactionID, WriteOrderOracle.next(), - eventIn); + Put put = new Put(++transactionID, WriteOrderOracle.next(), eventIn); ByteBuffer bytes = TransactionEventRecord.toByteBuffer(put); FlumeEventPointer ptr = logFileWriter.put(bytes); puts.put(ptr.getOffset(), put); @@ -170,14 +169,14 @@ public void testReader() throws InterruptedException, IOException, LogFile.SequentialReader reader = LogFileFactory.getSequentialReader(dataFile, null, true); LogRecord entry; - while((entry = reader.next()) != null) { + while ((entry = reader.next()) != null) { Integer offset = entry.getOffset(); TransactionEventRecord record = entry.getEvent(); Put put = puts.get(offset); FlumeEvent eventIn = put.getEvent(); Assert.assertEquals(put.getTransactionID(), record.getTransactionID()); Assert.assertTrue(record instanceof Put); - FlumeEvent eventOut = ((Put)record).getEvent(); + FlumeEvent eventOut = ((Put) record).getEvent(); Assert.assertEquals(eventIn.getHeaders(), eventOut.getHeaders()); Assert.assertTrue(Arrays.equals(eventIn.getBody(), eventOut.getBody())); } @@ -185,12 +184,12 @@ public void testReader() throws InterruptedException, IOException, @Test public void testReaderOldMetaFile() throws InterruptedException, - IOException, CorruptEventException { + IOException, CorruptEventException { Map puts = Maps.newHashMap(); for (int i = 0; i < 1000; i++) { FlumeEvent eventIn = TestUtils.newPersistableEvent(); Put put = new Put(++transactionID, WriteOrderOracle.next(), - eventIn); + eventIn); ByteBuffer bytes = TransactionEventRecord.toByteBuffer(put); FlumeEventPointer ptr = logFileWriter.put(bytes); puts.put(ptr.getOffset(), put); @@ -202,7 +201,7 @@ public void testReaderOldMetaFile() throws InterruptedException, Assert.fail("Renaming to meta.old failed"); } LogFile.SequentialReader reader = - LogFileFactory.getSequentialReader(dataFile, null, true); + LogFileFactory.getSequentialReader(dataFile, null, true); Assert.assertTrue(metadataFile.exists()); Assert.assertFalse(oldMetadataFile.exists()); LogRecord entry; @@ -219,14 +218,14 @@ public void testReaderOldMetaFile() throws InterruptedException, } } - @Test - public void testReaderTempMetaFile() throws InterruptedException, - IOException, CorruptEventException { + @Test + public void testReaderTempMetaFile() + throws InterruptedException, IOException, CorruptEventException { Map puts = Maps.newHashMap(); for (int i = 0; i < 1000; i++) { FlumeEvent eventIn = TestUtils.newPersistableEvent(); Put put = new Put(++transactionID, WriteOrderOracle.next(), - eventIn); + eventIn); ByteBuffer bytes = TransactionEventRecord.toByteBuffer(put); FlumeEventPointer ptr = logFileWriter.put(bytes); puts.put(ptr.getOffset(), put); @@ -240,7 +239,7 @@ public void testReaderTempMetaFile() throws InterruptedException, Assert.fail("Renaming to meta.temp failed"); } LogFile.SequentialReader reader = - LogFileFactory.getSequentialReader(dataFile, null, true); + LogFileFactory.getSequentialReader(dataFile, null, true); Assert.assertTrue(metadataFile.exists()); Assert.assertFalse(tempMetadataFile.exists()); Assert.assertFalse(oldMetadataFile.exists()); @@ -257,9 +256,10 @@ public void testReaderTempMetaFile() throws InterruptedException, Assert.assertTrue(Arrays.equals(eventIn.getBody(), eventOut.getBody())); } } + @Test public void testWriteDelimitedTo() throws IOException { - if(dataFile.isFile()) { + if (dataFile.isFile()) { Assert.assertTrue(dataFile.delete()); } Assert.assertTrue(dataFile.createNewFile()); @@ -270,25 +270,24 @@ public void testWriteDelimitedTo() throws IOException { metaDataBuilder.setCheckpointPosition(3); metaDataBuilder.setCheckpointWriteOrderID(4); LogFileV3.writeDelimitedTo(metaDataBuilder.build(), dataFile); - ProtosFactory.LogFileMetaData metaData = ProtosFactory.LogFileMetaData. - parseDelimitedFrom(new FileInputStream(dataFile)); + ProtosFactory.LogFileMetaData metaData = + ProtosFactory.LogFileMetaData.parseDelimitedFrom(new FileInputStream(dataFile)); Assert.assertEquals(1, metaData.getVersion()); Assert.assertEquals(2, metaData.getLogFileID()); Assert.assertEquals(3, metaData.getCheckpointPosition()); Assert.assertEquals(4, metaData.getCheckpointWriteOrderID()); } - @Test (expected = CorruptEventException.class) + @Test(expected = CorruptEventException.class) public void testPutGetCorruptEvent() throws Exception { final LogFile.RandomReader logFileReader = - LogFileFactory.getRandomReader(dataFile, null, true); + LogFileFactory.getRandomReader(dataFile, null, true); final FlumeEvent eventIn = TestUtils.newPersistableEvent(2500); - final Put put = new Put(++transactionID, WriteOrderOracle.next(), - eventIn); + final Put put = new Put(++transactionID, WriteOrderOracle.next(), eventIn); ByteBuffer bytes = TransactionEventRecord.toByteBuffer(put); FlumeEventPointer ptr = logFileWriter.put(bytes); - logFileWriter.commit(TransactionEventRecord.toByteBuffer(new Commit - (transactionID, WriteOrderOracle.next()))); + logFileWriter.commit(TransactionEventRecord.toByteBuffer( + new Commit(transactionID, WriteOrderOracle.next()))); logFileWriter.sync(); final int offset = ptr.getOffset(); RandomAccessFile writer = new RandomAccessFile(dataFile, "rw"); @@ -300,24 +299,22 @@ public void testPutGetCorruptEvent() throws Exception { // Should have thrown an exception by now. Assert.fail(); - } - @Test (expected = NoopRecordException.class) + @Test(expected = NoopRecordException.class) public void testPutGetNoopEvent() throws Exception { final LogFile.RandomReader logFileReader = - LogFileFactory.getRandomReader(dataFile, null, true); + LogFileFactory.getRandomReader(dataFile, null, true); final FlumeEvent eventIn = TestUtils.newPersistableEvent(2500); - final Put put = new Put(++transactionID, WriteOrderOracle.next(), - eventIn); + final Put put = new Put(++transactionID, WriteOrderOracle.next(), eventIn); ByteBuffer bytes = TransactionEventRecord.toByteBuffer(put); FlumeEventPointer ptr = logFileWriter.put(bytes); - logFileWriter.commit(TransactionEventRecord.toByteBuffer(new Commit - (transactionID, WriteOrderOracle.next()))); + logFileWriter.commit(TransactionEventRecord.toByteBuffer( + new Commit(transactionID, WriteOrderOracle.next()))); logFileWriter.sync(); final int offset = ptr.getOffset(); - LogFile.OperationRecordUpdater updater = new LogFile - .OperationRecordUpdater(dataFile); + LogFile.OperationRecordUpdater updater = + new LogFile.OperationRecordUpdater(dataFile); updater.markRecordAsNoop(offset); logFileReader.get(offset); @@ -330,40 +327,38 @@ public void testOperationRecordUpdater() throws Exception { File tempDir = Files.createTempDir(); File temp = new File(tempDir, "temp"); final RandomAccessFile tempFile = new RandomAccessFile(temp, "rw"); - for(int i = 0; i < 5000; i++) { + for (int i = 0; i < 5000; i++) { tempFile.write(LogFile.OP_RECORD); } tempFile.seek(0); LogFile.OperationRecordUpdater recordUpdater = new LogFile - .OperationRecordUpdater(temp); + .OperationRecordUpdater(temp); //Convert every 10th byte into a noop byte - for(int i = 0; i < 5000; i+=10) { + for (int i = 0; i < 5000; i += 10) { recordUpdater.markRecordAsNoop(i); } recordUpdater.close(); tempFile.seek(0); // Verify every 10th byte is actually a NOOP - for(int i = 0; i < 5000; i+=10) { + for (int i = 0; i < 5000; i += 10) { tempFile.seek(i); Assert.assertEquals(LogFile.OP_NOOP, tempFile.readByte()); } - } @Test - public void testOpRecordUpdaterWithFlumeEvents() throws Exception{ + public void testOpRecordUpdaterWithFlumeEvents() throws Exception { final FlumeEvent eventIn = TestUtils.newPersistableEvent(2500); - final Put put = new Put(++transactionID, WriteOrderOracle.next(), - eventIn); + final Put put = new Put(++transactionID, WriteOrderOracle.next(), eventIn); ByteBuffer bytes = TransactionEventRecord.toByteBuffer(put); FlumeEventPointer ptr = logFileWriter.put(bytes); - logFileWriter.commit(TransactionEventRecord.toByteBuffer(new Commit - (transactionID, WriteOrderOracle.next()))); + logFileWriter.commit(TransactionEventRecord.toByteBuffer( + new Commit(transactionID, WriteOrderOracle.next()))); logFileWriter.sync(); final int offset = ptr.getOffset(); - LogFile.OperationRecordUpdater updater = new LogFile - .OperationRecordUpdater(dataFile); + LogFile.OperationRecordUpdater updater = + new LogFile.OperationRecordUpdater(dataFile); updater.markRecordAsNoop(offset); RandomAccessFile fileReader = new RandomAccessFile(dataFile, "rw"); Assert.assertEquals(LogFile.OP_NOOP, fileReader.readByte()); @@ -375,7 +370,7 @@ public void testGroupCommit() throws Exception { final CyclicBarrier barrier = new CyclicBarrier(20); ExecutorService executorService = Executors.newFixedThreadPool(20); ExecutorCompletionService completionService = new - ExecutorCompletionService(executorService); + ExecutorCompletionService(executorService); final LogFile.Writer writer = logFileWriter; final AtomicLong txnId = new AtomicLong(++transactionID); for (int i = 0; i < 20; i++) { @@ -384,11 +379,11 @@ public void testGroupCommit() throws Exception { public Void call() { try { Put put = new Put(txnId.incrementAndGet(), - WriteOrderOracle.next(), eventIn); + WriteOrderOracle.next(), eventIn); ByteBuffer bytes = TransactionEventRecord.toByteBuffer(put); writer.put(bytes); writer.commit(TransactionEventRecord.toByteBuffer( - new Commit(txnId.get(), WriteOrderOracle.next()))); + new Commit(txnId.get(), WriteOrderOracle.next()))); barrier.await(); writer.sync(); } catch (Exception ex) { @@ -399,17 +394,15 @@ public Void call() { }); } - for(int i = 0; i < 20; i++) { + for (int i = 0; i < 20; i++) { completionService.take().get(); } - //At least 250*20, but can be higher due to serialization overhead + // At least 250*20, but can be higher due to serialization overhead Assert.assertTrue(logFileWriter.position() >= 5000); Assert.assertEquals(1, writer.getSyncCount()); - Assert.assertTrue(logFileWriter.getLastCommitPosition() == - logFileWriter.getLastSyncPosition()); + Assert.assertTrue(logFileWriter.getLastCommitPosition() == logFileWriter.getLastSyncPosition()); executorService.shutdown(); - } } diff --git a/flume-ng-channels/flume-file-channel/src/test/java/org/apache/flume/channel/file/TestTransactionEventRecordV2.java b/flume-ng-channels/flume-file-channel/src/test/java/org/apache/flume/channel/file/TestTransactionEventRecordV2.java index 2356d9058b..1f07e1f511 100644 --- a/flume-ng-channels/flume-file-channel/src/test/java/org/apache/flume/channel/file/TestTransactionEventRecordV2.java +++ b/flume-ng-channels/flume-file-channel/src/test/java/org/apache/flume/channel/file/TestTransactionEventRecordV2.java @@ -18,7 +18,8 @@ */ package org.apache.flume.channel.file; -import static org.mockito.Mockito.*; +import junit.framework.Assert; +import org.junit.Test; import java.io.ByteArrayInputStream; import java.io.ByteArrayOutputStream; @@ -30,9 +31,8 @@ import java.util.Arrays; import java.util.HashMap; -import junit.framework.Assert; - -import org.junit.Test; +import static org.mockito.Mockito.mock; +import static org.mockito.Mockito.when; @SuppressWarnings("deprecation") public class TestTransactionEventRecordV2 { @@ -127,7 +127,7 @@ public void testBadType() throws IOException { try { TransactionEventRecord.fromDataInputV2(toDataInput(in)); Assert.fail(); - } catch(NullPointerException e) { + } catch (NullPointerException e) { Assert.assertEquals("Unknown action ffff8000", e.getMessage()); } } diff --git a/flume-ng-channels/flume-file-channel/src/test/java/org/apache/flume/channel/file/TestTransactionEventRecordV3.java b/flume-ng-channels/flume-file-channel/src/test/java/org/apache/flume/channel/file/TestTransactionEventRecordV3.java index eb0ce04a69..512d29084f 100644 --- a/flume-ng-channels/flume-file-channel/src/test/java/org/apache/flume/channel/file/TestTransactionEventRecordV3.java +++ b/flume-ng-channels/flume-file-channel/src/test/java/org/apache/flume/channel/file/TestTransactionEventRecordV3.java @@ -18,7 +18,8 @@ */ package org.apache.flume.channel.file; -import static org.mockito.Mockito.*; +import junit.framework.Assert; +import org.junit.Test; import java.io.IOException; import java.nio.ByteBuffer; @@ -26,9 +27,8 @@ import java.util.HashMap; import java.util.Map; -import junit.framework.Assert; - -import org.junit.Test; +import static org.mockito.Mockito.mock; +import static org.mockito.Mockito.when; public class TestTransactionEventRecordV3 { @@ -52,6 +52,7 @@ public void testTypes() throws IOException { Assert.assertEquals(TransactionEventRecord.Type.COMMIT.get(), commit.getRecordType()); } + @Test public void testPutSerialization() throws IOException, CorruptEventException { Map headers = new HashMap(); @@ -69,9 +70,9 @@ public void testPutSerialization() throws IOException, CorruptEventException { Assert.assertEquals(headers, out.getEvent().getHeaders()); Assert.assertTrue(Arrays.equals(in.getEvent().getBody(), out.getEvent().getBody())); } + @Test - public void testPutSerializationNullHeader() throws IOException, - CorruptEventException { + public void testPutSerializationNullHeader() throws IOException, CorruptEventException { Put in = new Put(System.currentTimeMillis(), WriteOrderOracle.next(), new FlumeEvent(null, new byte[0])); @@ -84,11 +85,10 @@ public void testPutSerializationNullHeader() throws IOException, Assert.assertNotNull(out.getEvent().getHeaders()); Assert.assertTrue(Arrays.equals(in.getEvent().getBody(), out.getEvent().getBody())); } + @Test - public void testTakeSerialization() throws IOException, - CorruptEventException { - Take in = new Take(System.currentTimeMillis(), - WriteOrderOracle.next(), 10, 20); + public void testTakeSerialization() throws IOException, CorruptEventException { + Take in = new Take(System.currentTimeMillis(), WriteOrderOracle.next(), 10, 20); Take out = (Take)TransactionEventRecord.fromByteArray(toByteArray(in)); Assert.assertEquals(in.getClass(), out.getClass()); Assert.assertEquals(in.getRecordType(), out.getRecordType()); @@ -99,10 +99,8 @@ public void testTakeSerialization() throws IOException, } @Test - public void testRollbackSerialization() throws IOException, - CorruptEventException { - Rollback in = new Rollback(System.currentTimeMillis(), - WriteOrderOracle.next()); + public void testRollbackSerialization() throws IOException, CorruptEventException { + Rollback in = new Rollback(System.currentTimeMillis(), WriteOrderOracle.next()); Rollback out = (Rollback)TransactionEventRecord.fromByteArray(toByteArray(in)); Assert.assertEquals(in.getClass(), out.getClass()); Assert.assertEquals(in.getRecordType(), out.getRecordType()); @@ -111,10 +109,8 @@ public void testRollbackSerialization() throws IOException, } @Test - public void testCommitSerialization() throws IOException, - CorruptEventException { - Commit in = new Commit(System.currentTimeMillis(), - WriteOrderOracle.next()); + public void testCommitSerialization() throws IOException, CorruptEventException { + Commit in = new Commit(System.currentTimeMillis(), WriteOrderOracle.next()); Commit out = (Commit)TransactionEventRecord.fromByteArray(toByteArray(in)); Assert.assertEquals(in.getClass(), out.getClass()); Assert.assertEquals(in.getRecordType(), out.getRecordType()); @@ -129,7 +125,7 @@ public void testBadType() throws IOException, CorruptEventException { try { TransactionEventRecord.fromByteArray(toByteArray(in)); Assert.fail(); - } catch(NullPointerException e) { + } catch (NullPointerException e) { Assert.assertEquals("Unknown action ffff8000", e.getMessage()); } } diff --git a/flume-ng-channels/flume-file-channel/src/test/java/org/apache/flume/channel/file/TestUtils.java b/flume-ng-channels/flume-file-channel/src/test/java/org/apache/flume/channel/file/TestUtils.java index 61f38d25bb..0ec1831d52 100644 --- a/flume-ng-channels/flume-file-channel/src/test/java/org/apache/flume/channel/file/TestUtils.java +++ b/flume-ng-channels/flume-file-channel/src/test/java/org/apache/flume/channel/file/TestUtils.java @@ -18,7 +18,21 @@ */ package org.apache.flume.channel.file; -import static org.fest.reflect.core.Reflection.*; +import com.google.common.base.Charsets; +import com.google.common.collect.Lists; +import com.google.common.collect.Maps; +import com.google.common.collect.Sets; +import com.google.common.io.ByteStreams; +import com.google.common.io.Files; +import com.google.common.io.Resources; +import org.apache.flume.Channel; +import org.apache.flume.ChannelException; +import org.apache.flume.Context; +import org.apache.flume.Event; +import org.apache.flume.Transaction; +import org.apache.flume.conf.Configurables; +import org.apache.flume.event.EventBuilder; +import org.junit.Assert; import java.io.ByteArrayInputStream; import java.io.ByteArrayOutputStream; @@ -36,22 +50,8 @@ import java.util.UUID; import java.util.zip.GZIPInputStream; -import org.apache.flume.Channel; -import org.apache.flume.ChannelException; -import org.apache.flume.Context; -import org.apache.flume.Event; -import org.apache.flume.Transaction; -import org.apache.flume.conf.Configurables; -import org.apache.flume.event.EventBuilder; -import org.junit.Assert; - -import com.google.common.base.Charsets; -import com.google.common.collect.Lists; -import com.google.common.collect.Maps; -import com.google.common.collect.Sets; -import com.google.common.io.ByteStreams; -import com.google.common.io.Files; -import com.google.common.io.Resources; +import static org.fest.reflect.core.Reflection.field; +import static org.fest.reflect.core.Reflection.method; public class TestUtils { @@ -119,7 +119,7 @@ public static Set takeWithoutCommit(Channel channel, Transaction tx, public static List getAllLogs(File[] dataDirs) { List result = Lists.newArrayList(); - for(File dataDir : dataDirs) { + for (File dataDir : dataDirs) { result.addAll(LogUtils.getLogs(dataDir)); } return result; @@ -139,24 +139,22 @@ public static void forceCheckpoint(FileChannel channel) { .invoke(true)); } - public static Set takeEvents(Channel channel, int batchSize) - throws Exception { + public static Set takeEvents(Channel channel, int batchSize) throws Exception { return takeEvents(channel, batchSize, false); } - public static Set takeEvents(Channel channel, - int batchSize, boolean checkForCorruption) throws Exception { + public static Set takeEvents(Channel channel, int batchSize, boolean checkForCorruption) + throws Exception { return takeEvents(channel, batchSize, Integer.MAX_VALUE, checkForCorruption); } - public static Set takeEvents(Channel channel, - int batchSize, int numEvents) throws Exception { + public static Set takeEvents(Channel channel, int batchSize, int numEvents) + throws Exception { return takeEvents(channel, batchSize, numEvents, false); } - public static Set takeEvents(Channel channel, - int batchSize, int numEvents, boolean checkForCorruption) throws - Exception { + public static Set takeEvents(Channel channel, int batchSize, int numEvents, + boolean checkForCorruption) throws Exception { Set result = Sets.newHashSet(); for (int i = 0; i < numEvents; i += batchSize) { Transaction transaction = channel.getTransaction(); @@ -169,16 +167,15 @@ public static Set takeEvents(Channel channel, } catch (ChannelException ex) { Throwable th = ex; String msg; - if(checkForCorruption) { + if (checkForCorruption) { msg = "Corrupt event found. Please run File Channel"; th = ex.getCause(); } else { msg = "Take list for FileBackedTransaction, capacity"; } - Assert.assertTrue(th.getMessage().startsWith( - msg)); - if(checkForCorruption) { - throw (Exception) th; + Assert.assertTrue(th.getMessage().startsWith(msg)); + if (checkForCorruption) { + throw (Exception)th; } transaction.commit(); return result; @@ -204,16 +201,16 @@ public static Set takeEvents(Channel channel, public static Set consumeChannel(Channel channel) throws Exception { return consumeChannel(channel, false); } - public static Set consumeChannel(Channel channel, - boolean checkForCorruption) throws Exception { + public static Set consumeChannel(Channel channel, boolean checkForCorruption) + throws Exception { Set result = Sets.newHashSet(); int[] batchSizes = new int[] { 1000, 100, 10, 1 }; for (int i = 0; i < batchSizes.length; i++) { - while(true) { + while (true) { Set batch = takeEvents(channel, batchSizes[i], checkForCorruption); - if(batch.isEmpty()) { + if (batch.isEmpty()) { break; } result.addAll(batch); @@ -221,18 +218,16 @@ public static Set consumeChannel(Channel channel, } return result; } - public static Set fillChannel(Channel channel, String prefix) - throws Exception { + public static Set fillChannel(Channel channel, String prefix) throws Exception { Set result = Sets.newHashSet(); int[] batchSizes = new int[] { 1000, 100, 10, 1 }; for (int i = 0; i < batchSizes.length; i++) { try { - while(true) { - Set batch = putEvents(channel, prefix, batchSizes[i], - Integer.MAX_VALUE, true); - if(batch.isEmpty()) { + while (true) { + Set batch = putEvents(channel, prefix, batchSizes[i], Integer.MAX_VALUE, true); + if (batch.isEmpty()) { break; } result.addAll(batch); @@ -243,19 +238,17 @@ public static Set fillChannel(Channel channel, String prefix) + "size, a downstream system running slower than normal, or that " + "the channel capacity is just too low. [channel=" + channel.getName() + "]").equals(e.getMessage()) - || e.getMessage().startsWith("Put queue for FileBackedTransaction " + - "of capacity ")); + || e.getMessage().startsWith("Put queue for FileBackedTransaction of capacity ")); } } return result; } - public static Set putEvents(Channel channel, String prefix, - int batchSize, int numEvents) throws Exception { + public static Set putEvents(Channel channel, String prefix, int batchSize, int numEvents) + throws Exception { return putEvents(channel, prefix, batchSize, numEvents, false); } - public static Set putEvents(Channel channel, String prefix, - int batchSize, int numEvents, boolean untilCapacityIsReached) - throws Exception { + public static Set putEvents(Channel channel, String prefix, int batchSize, int numEvents, + boolean untilCapacityIsReached) throws Exception { Set result = Sets.newHashSet(); for (int i = 0; i < numEvents; i += batchSize) { Transaction transaction = channel.getTransaction(); @@ -272,13 +265,12 @@ public static Set putEvents(Channel channel, String prefix, result.addAll(batch); } catch (Exception ex) { transaction.rollback(); - if(untilCapacityIsReached && ex instanceof ChannelException && + if (untilCapacityIsReached && ex instanceof ChannelException && ("The channel has reached it's capacity. " + "This might be the result of a sink on the channel having too " + "low of batch size, a downstream system running slower than " + "normal, or that the channel capacity is just too low. " - + "[channel=" +channel.getName() + "]"). - equals(ex.getMessage())) { + + "[channel=" + channel.getName() + "]").equals(ex.getMessage())) { break; } throw ex; @@ -288,6 +280,7 @@ public static Set putEvents(Channel channel, String prefix, } return result; } + public static void copyDecompressed(String resource, File output) throws IOException { URL input = Resources.getResource(resource); @@ -298,12 +291,11 @@ public static void copyDecompressed(String resource, File output) gzis.close(); } - public static Context createFileChannelContext(String checkpointDir, - String dataDir, String backupDir, Map overrides) { + public static Context createFileChannelContext(String checkpointDir, String dataDir, + String backupDir, Map overrides) { Context context = new Context(); - context.put(FileChannelConfiguration.CHECKPOINT_DIR, - checkpointDir); - if(backupDir != null) { + context.put(FileChannelConfiguration.CHECKPOINT_DIR, checkpointDir); + if (backupDir != null) { context.put(FileChannelConfiguration.BACKUP_CHECKPOINT_DIR, backupDir); } context.put(FileChannelConfiguration.DATA_DIRS, dataDir); @@ -312,22 +304,22 @@ public static Context createFileChannelContext(String checkpointDir, context.putAll(overrides); return context; } - public static FileChannel createFileChannel(String checkpointDir, - String dataDir, Map overrides) { + + public static FileChannel createFileChannel(String checkpointDir, String dataDir, + Map overrides) { return createFileChannel(checkpointDir, dataDir, null, overrides); } - public static FileChannel createFileChannel(String checkpointDir, - String dataDir, String backupDir, Map overrides) { + public static FileChannel createFileChannel(String checkpointDir, String dataDir, + String backupDir, Map overrides) { FileChannel channel = new FileChannel(); channel.setName("FileChannel-" + UUID.randomUUID()); - Context context = createFileChannelContext(checkpointDir, dataDir, - backupDir, overrides); + Context context = createFileChannelContext(checkpointDir, dataDir, backupDir, overrides); Configurables.configure(channel, context); return channel; } - public static File writeStringToFile(File baseDir, String name, - String text) throws IOException { + + public static File writeStringToFile(File baseDir, String name, String text) throws IOException { File passwordFile = new File(baseDir, name); Files.write(text, passwordFile, Charsets.UTF_8); return passwordFile; diff --git a/flume-ng-channels/flume-file-channel/src/test/java/org/apache/flume/channel/file/encryption/CipherProviderTestSuite.java b/flume-ng-channels/flume-file-channel/src/test/java/org/apache/flume/channel/file/encryption/CipherProviderTestSuite.java index 530ccf68d4..22848d26d9 100644 --- a/flume-ng-channels/flume-file-channel/src/test/java/org/apache/flume/channel/file/encryption/CipherProviderTestSuite.java +++ b/flume-ng-channels/flume-file-channel/src/test/java/org/apache/flume/channel/file/encryption/CipherProviderTestSuite.java @@ -32,24 +32,28 @@ public CipherProviderTestSuite(CipherProvider.Encryptor encryptor, this.encryptor = encryptor; this.decryptor = decryptor; } + public void test() throws Exception { testBasic(); testEmpty(); testNullPlainText(); testNullCipherText(); } + public void testBasic() throws Exception { String expected = "mn state fair is the place to be"; byte[] cipherText = encryptor.encrypt(expected.getBytes(Charsets.UTF_8)); byte[] clearText = decryptor.decrypt(cipherText); Assert.assertEquals(expected, new String(clearText, Charsets.UTF_8)); } + public void testEmpty() throws Exception { String expected = ""; byte[] cipherText = encryptor.encrypt(new byte[]{}); byte[] clearText = decryptor.decrypt(cipherText); Assert.assertEquals(expected, new String(clearText)); } + public void testNullPlainText() throws Exception { try { encryptor.encrypt(null); @@ -58,6 +62,7 @@ public void testNullPlainText() throws Exception { // expected } } + public void testNullCipherText() throws Exception { try { decryptor.decrypt(null); diff --git a/flume-ng-channels/flume-file-channel/src/test/java/org/apache/flume/channel/file/encryption/EncryptionTestUtils.java b/flume-ng-channels/flume-file-channel/src/test/java/org/apache/flume/channel/file/encryption/EncryptionTestUtils.java index 6ca3246512..f290035a12 100644 --- a/flume-ng-channels/flume-file-channel/src/test/java/org/apache/flume/channel/file/encryption/EncryptionTestUtils.java +++ b/flume-ng-channels/flume-file-channel/src/test/java/org/apache/flume/channel/file/encryption/EncryptionTestUtils.java @@ -18,18 +18,6 @@ */ package org.apache.flume.channel.file.encryption; -import java.io.File; -import java.io.FileOutputStream; -import java.io.IOException; -import java.security.Key; -import java.security.KeyStore; -import java.util.List; -import java.util.Map; - -import javax.crypto.KeyGenerator; - -import org.apache.flume.channel.file.TestUtils; - import com.google.common.base.Charsets; import com.google.common.base.Joiner; import com.google.common.base.Throwables; @@ -37,6 +25,16 @@ import com.google.common.collect.Maps; import com.google.common.io.Files; import com.google.common.io.Resources; +import org.apache.flume.channel.file.TestUtils; + +import javax.crypto.KeyGenerator; +import java.io.File; +import java.io.FileOutputStream; +import java.io.IOException; +import java.security.Key; +import java.security.KeyStore; +import java.util.List; +import java.util.Map; public class EncryptionTestUtils { @@ -50,33 +48,32 @@ private static Key newKey() { throw Throwables.propagate(e); } } - public static void createKeyStore(File keyStoreFile, - File keyStorePasswordFile, Map keyAliasPassword) - throws Exception { + + public static void createKeyStore(File keyStoreFile, File keyStorePasswordFile, + Map keyAliasPassword) throws Exception { KeyStore ks = KeyStore.getInstance("jceks"); ks.load(null); List keysWithSeperatePasswords = Lists.newArrayList(); - for(String alias : keyAliasPassword.keySet()) { + for (String alias : keyAliasPassword.keySet()) { Key key = newKey(); char[] password = null; File passwordFile = keyAliasPassword.get(alias); - if(passwordFile == null) { - password = Files.toString(keyStorePasswordFile, Charsets.UTF_8) - .toCharArray(); + if (passwordFile == null) { + password = Files.toString(keyStorePasswordFile, Charsets.UTF_8).toCharArray(); } else { keysWithSeperatePasswords.add(alias); password = Files.toString(passwordFile, Charsets.UTF_8).toCharArray(); } ks.setKeyEntry(alias, key, password, null); } - char[] keyStorePassword = Files. - toString(keyStorePasswordFile, Charsets.UTF_8).toCharArray(); + char[] keyStorePassword = Files.toString(keyStorePasswordFile, Charsets.UTF_8).toCharArray(); FileOutputStream outputStream = new FileOutputStream(keyStoreFile); ks.store(outputStream, keyStorePassword); outputStream.close(); } - public static Map configureTestKeyStore(File baseDir, - File keyStoreFile) throws IOException { + + public static Map configureTestKeyStore(File baseDir, File keyStoreFile) + throws IOException { Map result = Maps.newHashMap(); if (System.getProperty("java.vendor").contains("IBM")) { @@ -86,50 +83,52 @@ public static Map configureTestKeyStore(File baseDir, Resources.copy(Resources.getResource("sun-test.keystore"), new FileOutputStream(keyStoreFile)); } - /* - Commands below: - keytool -genseckey -alias key-0 -keypass keyPassword -keyalg AES \ - -keysize 128 -validity 9000 -keystore src/test/resources/test.keystore \ - -storetype jceks -storepass keyStorePassword - keytool -genseckey -alias key-1 -keyalg AES -keysize 128 -validity 9000 \ - -keystore src/test/resources/test.keystore -storetype jceks \ - -storepass keyStorePassword + + /* Commands below: + * keytool -genseckey -alias key-0 -keypass keyPassword -keyalg AES \ + * -keysize 128 -validity 9000 -keystore src/test/resources/test.keystore \ + * -storetype jceks -storepass keyStorePassword + * keytool -genseckey -alias key-1 -keyalg AES -keysize 128 -validity 9000 \ + * -keystore src/test/resources/test.keystore -storetype jceks \ + * -storepass keyStorePassword */ -// key-0 has own password, key-1 used key store password - result.put("key-0", - TestUtils.writeStringToFile(baseDir, "key-0", "keyPassword")); + // key-0 has own password, key-1 used key store password + result.put("key-0", TestUtils.writeStringToFile(baseDir, "key-0", "keyPassword")); result.put("key-1", null); return result; } - public static Map configureForKeyStore(File keyStoreFile, - File keyStorePasswordFile, Map keyAliasPassword) - throws Exception { + + public static Map configureForKeyStore(File keyStoreFile, + File keyStorePasswordFile, + Map keyAliasPassword) + throws Exception { Map context = Maps.newHashMap(); List keys = Lists.newArrayList(); Joiner joiner = Joiner.on("."); - for(String alias : keyAliasPassword.keySet()) { + for (String alias : keyAliasPassword.keySet()) { File passwordFile = keyAliasPassword.get(alias); - if(passwordFile == null) { + if (passwordFile == null) { keys.add(alias); } else { String propertyName = joiner.join(EncryptionConfiguration.KEY_PROVIDER, - EncryptionConfiguration.JCE_FILE_KEYS, alias, - EncryptionConfiguration.JCE_FILE_KEY_PASSWORD_FILE); + EncryptionConfiguration.JCE_FILE_KEYS, + alias, + EncryptionConfiguration.JCE_FILE_KEY_PASSWORD_FILE); keys.add(alias); context.put(propertyName, passwordFile.getAbsolutePath()); } } context.put(joiner.join(EncryptionConfiguration.KEY_PROVIDER, - EncryptionConfiguration.JCE_FILE_KEY_STORE_FILE), - keyStoreFile.getAbsolutePath()); - if(keyStorePasswordFile != null) { + EncryptionConfiguration.JCE_FILE_KEY_STORE_FILE), + keyStoreFile.getAbsolutePath()); + if (keyStorePasswordFile != null) { context.put(joiner.join(EncryptionConfiguration.KEY_PROVIDER, - EncryptionConfiguration.JCE_FILE_KEY_STORE_PASSWORD_FILE), - keyStorePasswordFile.getAbsolutePath()); + EncryptionConfiguration.JCE_FILE_KEY_STORE_PASSWORD_FILE), + keyStorePasswordFile.getAbsolutePath()); } context.put(joiner.join(EncryptionConfiguration.KEY_PROVIDER, - EncryptionConfiguration.JCE_FILE_KEYS), - Joiner.on(" ").join(keys)); + EncryptionConfiguration.JCE_FILE_KEYS), + Joiner.on(" ").join(keys)); return context; } } diff --git a/flume-ng-channels/flume-file-channel/src/test/java/org/apache/flume/channel/file/encryption/TestAESCTRNoPaddingProvider.java b/flume-ng-channels/flume-file-channel/src/test/java/org/apache/flume/channel/file/encryption/TestAESCTRNoPaddingProvider.java index a7c7cb20ba..d483fcc1fb 100644 --- a/flume-ng-channels/flume-file-channel/src/test/java/org/apache/flume/channel/file/encryption/TestAESCTRNoPaddingProvider.java +++ b/flume-ng-channels/flume-file-channel/src/test/java/org/apache/flume/channel/file/encryption/TestAESCTRNoPaddingProvider.java @@ -35,13 +35,13 @@ public class TestAESCTRNoPaddingProvider { public void setup() throws Exception { KeyGenerator keyGen = KeyGenerator.getInstance("AES"); key = keyGen.generateKey(); - encryptor = CipherProviderFactory. - getEncrypter(CipherProviderType.AESCTRNOPADDING.name(), key); - decryptor = CipherProviderFactory. - getDecrypter(CipherProviderType.AESCTRNOPADDING.name(), key, - encryptor.getParameters()); + encryptor = CipherProviderFactory.getEncrypter( + CipherProviderType.AESCTRNOPADDING.name(), key); + decryptor = CipherProviderFactory.getDecrypter( + CipherProviderType.AESCTRNOPADDING.name(), key, encryptor.getParameters()); cipherProviderTestSuite = new CipherProviderTestSuite(encryptor, decryptor); } + @Test public void test() throws Exception { cipherProviderTestSuite.test(); diff --git a/flume-ng-channels/flume-file-channel/src/test/java/org/apache/flume/channel/file/encryption/TestFileChannelEncryption.java b/flume-ng-channels/flume-file-channel/src/test/java/org/apache/flume/channel/file/encryption/TestFileChannelEncryption.java index d4537a8ac9..4e5ab6f88a 100644 --- a/flume-ng-channels/flume-file-channel/src/test/java/org/apache/flume/channel/file/encryption/TestFileChannelEncryption.java +++ b/flume-ng-channels/flume-file-channel/src/test/java/org/apache/flume/channel/file/encryption/TestFileChannelEncryption.java @@ -18,18 +18,10 @@ */ package org.apache.flume.channel.file.encryption; -import static org.apache.flume.channel.file.TestUtils.*; - -import java.io.File; -import java.util.Collections; -import java.util.HashSet; -import java.util.Map; -import java.util.Set; -import java.util.concurrent.CountDownLatch; -import java.util.concurrent.Executor; -import java.util.concurrent.Executors; -import java.util.concurrent.atomic.AtomicBoolean; - +import com.google.common.base.Charsets; +import com.google.common.base.Joiner; +import com.google.common.collect.Maps; +import com.google.common.io.Files; import org.apache.flume.ChannelException; import org.apache.flume.FlumeException; import org.apache.flume.channel.file.FileChannelConfiguration; @@ -42,10 +34,21 @@ import org.slf4j.Logger; import org.slf4j.LoggerFactory; -import com.google.common.base.Charsets; -import com.google.common.base.Joiner; -import com.google.common.collect.Maps; -import com.google.common.io.Files; +import java.io.File; +import java.util.Collections; +import java.util.HashSet; +import java.util.Map; +import java.util.Set; +import java.util.concurrent.CountDownLatch; +import java.util.concurrent.Executor; +import java.util.concurrent.Executors; +import java.util.concurrent.atomic.AtomicBoolean; + +import static org.apache.flume.channel.file.TestUtils.compareInputAndOut; +import static org.apache.flume.channel.file.TestUtils.consumeChannel; +import static org.apache.flume.channel.file.TestUtils.fillChannel; +import static org.apache.flume.channel.file.TestUtils.putEvents; +import static org.apache.flume.channel.file.TestUtils.takeEvents; public class TestFileChannelEncryption extends TestFileChannelBase { protected static final Logger LOGGER = @@ -62,36 +65,39 @@ public void setup() throws Exception { keyStoreFile = new File(baseDir, "keyStoreFile"); Assert.assertTrue(keyStoreFile.createNewFile()); keyAliasPassword = Maps.newHashMap(); - keyAliasPassword.putAll(EncryptionTestUtils. - configureTestKeyStore(baseDir, keyStoreFile)); + keyAliasPassword.putAll(EncryptionTestUtils.configureTestKeyStore(baseDir, keyStoreFile)); } + @After public void teardown() { super.teardown(); } + private Map getOverrides() throws Exception { Map overrides = Maps.newHashMap(); overrides.put(FileChannelConfiguration.CAPACITY, String.valueOf(100)); - overrides.put(FileChannelConfiguration.TRANSACTION_CAPACITY, - String.valueOf(100)); + overrides.put(FileChannelConfiguration.TRANSACTION_CAPACITY, String.valueOf(100)); return overrides; } + private Map getOverridesForEncryption() throws Exception { Map overrides = getOverrides(); - Map encryptionProps = EncryptionTestUtils. - configureForKeyStore(keyStoreFile, - keyStorePasswordFile, keyAliasPassword); + Map encryptionProps = + EncryptionTestUtils.configureForKeyStore(keyStoreFile, + keyStorePasswordFile, + keyAliasPassword); encryptionProps.put(EncryptionConfiguration.KEY_PROVIDER, - KeyProviderType.JCEKSFILE.name()); + KeyProviderType.JCEKSFILE.name()); encryptionProps.put(EncryptionConfiguration.CIPHER_PROVIDER, - CipherProviderType.AESCTRNOPADDING.name()); + CipherProviderType.AESCTRNOPADDING.name()); encryptionProps.put(EncryptionConfiguration.ACTIVE_KEY, "key-1"); - for(String key : encryptionProps.keySet()) { + for (String key : encryptionProps.keySet()) { overrides.put(EncryptionConfiguration.ENCRYPTION_PREFIX + "." + key, - encryptionProps.get(key)); + encryptionProps.get(key)); } return overrides; } + /** * Test fails without FLUME-1565 */ @@ -222,15 +228,16 @@ public void testEncryptedChannelWithoutEncryptionConfigFails() throws Exception channel = createFileChannel(noEncryptionOverrides); channel.start(); - if(channel.isOpen()) { + if (channel.isOpen()) { try { takeEvents(channel, 1, 1); Assert.fail("Channel was opened and take did not throw exception"); - } catch(ChannelException ex) { + } catch (ChannelException ex) { // expected } } } + @Test public void testUnencyrptedAndEncryptedLogs() throws Exception { Map noEncryptionOverrides = getOverrides(); @@ -252,41 +259,46 @@ public void testUnencyrptedAndEncryptedLogs() throws Exception { Set out = consumeChannel(channel); compareInputAndOut(in, out); } + @Test public void testBadKeyProviderInvalidValue() throws Exception { Map overrides = getOverridesForEncryption(); overrides.put(Joiner.on(".").join(EncryptionConfiguration.ENCRYPTION_PREFIX, - EncryptionConfiguration.KEY_PROVIDER), "invalid"); + EncryptionConfiguration.KEY_PROVIDER), + "invalid"); try { channel = createFileChannel(overrides); Assert.fail(); - } catch(FlumeException ex) { - Assert.assertEquals("java.lang.ClassNotFoundException: invalid", - ex.getMessage()); + } catch (FlumeException ex) { + Assert.assertEquals("java.lang.ClassNotFoundException: invalid", ex.getMessage()); } } + @Test public void testBadKeyProviderInvalidClass() throws Exception { Map overrides = getOverridesForEncryption(); overrides.put(Joiner.on(".").join(EncryptionConfiguration.ENCRYPTION_PREFIX, - EncryptionConfiguration.KEY_PROVIDER), String.class.getName()); + EncryptionConfiguration.KEY_PROVIDER), + String.class.getName()); try { channel = createFileChannel(overrides); Assert.fail(); - } catch(FlumeException ex) { - Assert.assertEquals("Unable to instantiate Builder from java.lang.String", - ex.getMessage()); + } catch (FlumeException ex) { + Assert.assertEquals("Unable to instantiate Builder from java.lang.String", ex.getMessage()); } } + @Test public void testBadCipherProviderInvalidValue() throws Exception { Map overrides = getOverridesForEncryption(); overrides.put(Joiner.on(".").join(EncryptionConfiguration.ENCRYPTION_PREFIX, - EncryptionConfiguration.CIPHER_PROVIDER), "invalid"); + EncryptionConfiguration.CIPHER_PROVIDER), + "invalid"); channel = createFileChannel(overrides); channel.start(); Assert.assertFalse(channel.isOpen()); } + @Test public void testBadCipherProviderInvalidClass() throws Exception { Map overrides = getOverridesForEncryption(); @@ -296,6 +308,7 @@ public void testBadCipherProviderInvalidClass() throws Exception { channel.start(); Assert.assertFalse(channel.isOpen()); } + @Test public void testMissingKeyStoreFile() throws Exception { Map overrides = getOverridesForEncryption(); @@ -306,11 +319,12 @@ public void testMissingKeyStoreFile() throws Exception { try { channel = createFileChannel(overrides); Assert.fail(); - } catch(RuntimeException ex) { + } catch (RuntimeException ex) { Assert.assertTrue("Exception message is incorrect: " + ex.getMessage(), ex.getMessage().startsWith("java.io.FileNotFoundException: /path/does/not/exist ")); } } + @Test public void testMissingKeyStorePasswordFile() throws Exception { Map overrides = getOverridesForEncryption(); @@ -321,11 +335,12 @@ public void testMissingKeyStorePasswordFile() throws Exception { try { channel = createFileChannel(overrides); Assert.fail(); - } catch(RuntimeException ex) { + } catch (RuntimeException ex) { Assert.assertTrue("Exception message is incorrect: " + ex.getMessage(), ex.getMessage().startsWith("java.io.FileNotFoundException: /path/does/not/exist ")); } } + @Test public void testBadKeyStorePassword() throws Exception { Files.write("invalid", keyStorePasswordFile, Charsets.UTF_8); @@ -334,11 +349,12 @@ public void testBadKeyStorePassword() throws Exception { channel = TestUtils.createFileChannel(checkpointDir.getAbsolutePath(), dataDir, overrides); Assert.fail(); - } catch(RuntimeException ex) { + } catch (RuntimeException ex) { Assert.assertEquals("java.io.IOException: Keystore was tampered with, or " + "password was incorrect", ex.getMessage()); } } + @Test public void testBadKeyAlias() throws Exception { Map overrides = getOverridesForEncryption(); diff --git a/flume-ng-channels/flume-file-channel/src/test/java/org/apache/flume/channel/file/encryption/TestJCEFileKeyProvider.java b/flume-ng-channels/flume-file-channel/src/test/java/org/apache/flume/channel/file/encryption/TestJCEFileKeyProvider.java index f33cada220..8214a05d69 100644 --- a/flume-ng-channels/flume-file-channel/src/test/java/org/apache/flume/channel/file/encryption/TestJCEFileKeyProvider.java +++ b/flume-ng-channels/flume-file-channel/src/test/java/org/apache/flume/channel/file/encryption/TestJCEFileKeyProvider.java @@ -18,12 +18,10 @@ */ package org.apache.flume.channel.file.encryption; -import java.io.File; -import java.security.Key; -import java.util.Map; - +import com.google.common.base.Charsets; +import com.google.common.collect.Maps; +import com.google.common.io.Files; import junit.framework.Assert; - import org.apache.commons.io.FileUtils; import org.apache.flume.Context; import org.apache.flume.channel.file.TestUtils; @@ -31,9 +29,9 @@ import org.junit.Before; import org.junit.Test; -import com.google.common.base.Charsets; -import com.google.common.collect.Maps; -import com.google.common.io.Files; +import java.io.File; +import java.security.Key; +import java.util.Map; public class TestJCEFileKeyProvider { private CipherProvider.Encryptor encryptor; @@ -53,62 +51,69 @@ public void setup() throws Exception { Assert.assertTrue(keyStoreFile.createNewFile()); } + @After public void cleanup() { FileUtils.deleteQuietly(baseDir); } + private void initializeForKey(Key key) { - encryptor = - new AESCTRNoPaddingProvider.EncryptorBuilder().setKey(key).build(); - decryptor = - new AESCTRNoPaddingProvider.DecryptorBuilder() - .setKey(key).setParameters(encryptor.getParameters()).build(); + encryptor = new AESCTRNoPaddingProvider.EncryptorBuilder() + .setKey(key) + .build(); + decryptor = new AESCTRNoPaddingProvider.DecryptorBuilder() + .setKey(key) + .setParameters(encryptor.getParameters()) + .build(); } + @Test public void testWithNewKeyStore() throws Exception { createNewKeyStore(); EncryptionTestUtils.createKeyStore(keyStoreFile, keyStorePasswordFile, keyAliasPassword); - Context context = new Context(EncryptionTestUtils. - configureForKeyStore(keyStoreFile, - keyStorePasswordFile, keyAliasPassword)); + Context context = new Context( + EncryptionTestUtils.configureForKeyStore(keyStoreFile, + keyStorePasswordFile, + keyAliasPassword)); Context keyProviderContext = new Context( context.getSubProperties(EncryptionConfiguration.KEY_PROVIDER + ".")); - KeyProvider keyProvider = KeyProviderFactory. - getInstance(KeyProviderType.JCEKSFILE.name(), keyProviderContext); + KeyProvider keyProvider = + KeyProviderFactory.getInstance(KeyProviderType.JCEKSFILE.name(), keyProviderContext); testKeyProvider(keyProvider); } + @Test public void testWithExistingKeyStore() throws Exception { - keyAliasPassword.putAll(EncryptionTestUtils. - configureTestKeyStore(baseDir, keyStoreFile)); - Context context = new Context(EncryptionTestUtils. - configureForKeyStore(keyStoreFile, - keyStorePasswordFile, keyAliasPassword)); + keyAliasPassword.putAll(EncryptionTestUtils.configureTestKeyStore(baseDir, keyStoreFile)); + Context context = new Context( + EncryptionTestUtils.configureForKeyStore(keyStoreFile, + keyStorePasswordFile, + keyAliasPassword)); Context keyProviderContext = new Context( context.getSubProperties(EncryptionConfiguration.KEY_PROVIDER + ".")); - KeyProvider keyProvider = KeyProviderFactory. - getInstance(KeyProviderType.JCEKSFILE.name(), keyProviderContext); + KeyProvider keyProvider = + KeyProviderFactory.getInstance(KeyProviderType.JCEKSFILE.name(), keyProviderContext); testKeyProvider(keyProvider); } + private void createNewKeyStore() throws Exception { - for(int i = 0; i < 10; i++) { + for (int i = 0; i < 10; i++) { // create some with passwords, some without - if(i % 2 == 0) { + if (i % 2 == 0) { String alias = "test-" + i; String password = String.valueOf(i); - keyAliasPassword.put(alias, - TestUtils.writeStringToFile(baseDir, alias, password)); + keyAliasPassword.put(alias, TestUtils.writeStringToFile(baseDir, alias, password)); } } } + private void testKeyProvider(KeyProvider keyProvider) { - for(String alias : keyAliasPassword.keySet()) { + for (String alias : keyAliasPassword.keySet()) { Key key = keyProvider.getKey(alias); initializeForKey(key); String expected = "some text here " + alias; - byte[] cipherText = encryptor. - encrypt(expected.getBytes(Charsets.UTF_8)); + byte[] cipherText = encryptor.encrypt(expected.getBytes(Charsets.UTF_8)); byte[] clearText = decryptor.decrypt(cipherText); Assert.assertEquals(expected, new String(clearText, Charsets.UTF_8)); } diff --git a/flume-ng-channels/flume-jdbc-channel/src/test/java/org/apache/flume/channel/jdbc/BaseJdbcChannelProviderTest.java b/flume-ng-channels/flume-jdbc-channel/src/test/java/org/apache/flume/channel/jdbc/BaseJdbcChannelProviderTest.java index 85ad7fe853..7031eb7201 100644 --- a/flume-ng-channels/flume-jdbc-channel/src/test/java/org/apache/flume/channel/jdbc/BaseJdbcChannelProviderTest.java +++ b/flume-ng-channels/flume-jdbc-channel/src/test/java/org/apache/flume/channel/jdbc/BaseJdbcChannelProviderTest.java @@ -17,6 +17,17 @@ */ package org.apache.flume.channel.jdbc; +import org.apache.flume.Context; +import org.apache.flume.Event; +import org.apache.flume.Transaction; +import org.apache.flume.channel.jdbc.impl.JdbcChannelProviderImpl; +import org.junit.After; +import org.junit.Assert; +import org.junit.Before; +import org.junit.Test; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + import java.io.File; import java.io.IOException; import java.util.ArrayList; @@ -32,17 +43,6 @@ import java.util.concurrent.Future; import java.util.concurrent.TimeUnit; -import org.apache.flume.Context; -import org.apache.flume.Event; -import org.apache.flume.Transaction; -import org.apache.flume.channel.jdbc.impl.JdbcChannelProviderImpl; -import org.junit.After; -import org.junit.Assert; -import org.junit.Before; -import org.junit.Test; -import org.slf4j.Logger; -import org.slf4j.LoggerFactory; - public abstract class BaseJdbcChannelProviderTest { private static final Logger LOGGER = LoggerFactory.getLogger(BaseJdbcChannelProviderTest.class); @@ -103,7 +103,7 @@ public void testDerbyChannelCapacity() { Set events = new HashSet(); for (int i = 1; i < 12; i++) { - events.add(MockEventUtils.generateMockEvent(i, i, i, 61%i, 1)); + events.add(MockEventUtils.generateMockEvent(i, i, i, 61 % i, 1)); } Iterator meIt = events.iterator(); @@ -170,7 +170,7 @@ public void testEventWithSimulatedSourceAndSinks() throws Exception { new HashMap>(); for (int i = 1; i < 121; i++) { - MockEvent me = MockEventUtils.generateMockEvent(i, i, i, 61%i, 10); + MockEvent me = MockEventUtils.generateMockEvent(i, i, i, 61 % i, 10); List meList = eventMap.get(me.getChannel()); if (meList == null) { meList = new ArrayList(); @@ -227,7 +227,7 @@ public void testPeristingEvents() { Set events = new HashSet(); for (int i = 1; i < 81; i++) { - events.add(MockEventUtils.generateMockEvent(i, i, i, 61%i, 5)); + events.add(MockEventUtils.generateMockEvent(i, i, i, 61 % i, 5)); } Iterator meIt = events.iterator(); diff --git a/flume-ng-channels/flume-jdbc-channel/src/test/java/org/apache/flume/channel/jdbc/MockEvent.java b/flume-ng-channels/flume-jdbc-channel/src/test/java/org/apache/flume/channel/jdbc/MockEvent.java index 1e412c5681..6804a9fef6 100644 --- a/flume-ng-channels/flume-jdbc-channel/src/test/java/org/apache/flume/channel/jdbc/MockEvent.java +++ b/flume-ng-channels/flume-jdbc-channel/src/test/java/org/apache/flume/channel/jdbc/MockEvent.java @@ -27,8 +27,7 @@ public class MockEvent implements Event { private final Map headers; private final String channel; - public MockEvent(byte[] payload, Map headers, String channel) - { + public MockEvent(byte[] payload, Map headers, String channel) { this.payload = payload; this.headers = headers; this.channel = channel; diff --git a/flume-ng-channels/flume-jdbc-channel/src/test/java/org/apache/flume/channel/jdbc/MockEventUtils.java b/flume-ng-channels/flume-jdbc-channel/src/test/java/org/apache/flume/channel/jdbc/MockEventUtils.java index 10d8b518af..e5ee324a63 100644 --- a/flume-ng-channels/flume-jdbc-channel/src/test/java/org/apache/flume/channel/jdbc/MockEventUtils.java +++ b/flume-ng-channels/flume-jdbc-channel/src/test/java/org/apache/flume/channel/jdbc/MockEventUtils.java @@ -17,13 +17,13 @@ */ package org.apache.flume.channel.jdbc; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + import java.util.HashMap; import java.util.Map; import java.util.Random; -import org.slf4j.Logger; -import org.slf4j.LoggerFactory; - public final class MockEventUtils { public static final Logger LOGGER = @@ -70,20 +70,20 @@ public static String generateHeaderString(int size) { * @param numChannels * @return */ - public static MockEvent generateMockEvent(int payloadMargin, - int headerNameMargin, int headerValueMargin, int numHeaders, - int numChannels) { + public static MockEvent generateMockEvent(int payloadMargin, int headerNameMargin, + int headerValueMargin, int numHeaders, + int numChannels) { int chIndex = 0; if (numChannels > 1) { - chIndex = Math.abs(RANDOM.nextInt())%numChannels; + chIndex = Math.abs(RANDOM.nextInt()) % numChannels; } - String channel = "test-"+chIndex; + String channel = "test-" + chIndex; StringBuilder sb = new StringBuilder("New Event[payload size:"); int plTh = ConfigurationConstants.PAYLOAD_LENGTH_THRESHOLD; - int plSize = Math.abs(RANDOM.nextInt())%plTh + payloadMargin; + int plSize = Math.abs(RANDOM.nextInt()) % plTh + payloadMargin; sb.append(plSize).append(", numHeaders:").append(numHeaders); sb.append(", channel:").append(channel); @@ -93,8 +93,8 @@ public static MockEvent generateMockEvent(int payloadMargin, Map headers = new HashMap(); for (int i = 0; i < numHeaders; i++) { - int nmSize = Math.abs(RANDOM.nextInt())%nmTh + headerNameMargin; - int vlSize = Math.abs(RANDOM.nextInt())%vlTh + headerValueMargin; + int nmSize = Math.abs(RANDOM.nextInt()) % nmTh + headerNameMargin; + int vlSize = Math.abs(RANDOM.nextInt()) % vlTh + headerValueMargin; String name = generateHeaderString(nmSize); String value = generateHeaderString(vlSize); diff --git a/flume-ng-channels/flume-jdbc-channel/src/test/java/org/apache/flume/channel/jdbc/TestDerbySchemaHandlerQueries.java b/flume-ng-channels/flume-jdbc-channel/src/test/java/org/apache/flume/channel/jdbc/TestDerbySchemaHandlerQueries.java index 362bcfa245..cad972cada 100644 --- a/flume-ng-channels/flume-jdbc-channel/src/test/java/org/apache/flume/channel/jdbc/TestDerbySchemaHandlerQueries.java +++ b/flume-ng-channels/flume-jdbc-channel/src/test/java/org/apache/flume/channel/jdbc/TestDerbySchemaHandlerQueries.java @@ -107,7 +107,6 @@ public class TestDerbySchemaHandlerQueries { = "INSERT INTO FLUME.FL_HEADER (FLH_EVENT, FLH_NAME, FLH_VALUE, " + "FLH_NMSPILL, FLH_VLSPILL) VALUES ( ?, ?, ?, ?, ?)"; - public static final String EXPECTED_STMT_INSERT_HEADER_NAME_SPILL = "INSERT INTO FLUME.FL_NMSPILL (FLN_HEADER, FLN_SPILL) VALUES ( ?, ?)"; @@ -119,7 +118,6 @@ public class TestDerbySchemaHandlerQueries { + "FLE_ID = (SELECT MIN(FLE_ID) FROM FLUME.FL_EVENT WHERE " + "FLE_CHANNEL = ?)"; - public static final String EXPECTED_STMT_FETCH_PAYLOAD_SPILL = "SELECT FLP_SPILL FROM FLUME.FL_PLSPILL WHERE FLP_EVENT = ?"; diff --git a/flume-ng-channels/flume-kafka-channel/src/test/java/org/apache/flume/channel/kafka/TestKafkaChannel.java b/flume-ng-channels/flume-kafka-channel/src/test/java/org/apache/flume/channel/kafka/TestKafkaChannel.java index d01346afd1..b63ac9b8c4 100644 --- a/flume-ng-channels/flume-kafka-channel/src/test/java/org/apache/flume/channel/kafka/TestKafkaChannel.java +++ b/flume-ng-channels/flume-kafka-channel/src/test/java/org/apache/flume/channel/kafka/TestKafkaChannel.java @@ -130,19 +130,19 @@ public void testOldConfig() throws Exception { Properties consumerProps = channel.getConsumerProps(); Properties producerProps = channel.getProducerProps(); - Assert.assertEquals(producerProps.getProperty(CommonClientConfigs.BOOTSTRAP_SERVERS_CONFIG),testUtil.getKafkaServerUrl()); - Assert.assertEquals(consumerProps.getProperty(ConsumerConfig.GROUP_ID_CONFIG), "flume-something"); - Assert.assertEquals(consumerProps.getProperty(ConsumerConfig.AUTO_OFFSET_RESET_CONFIG), "earliest"); - + Assert.assertEquals(producerProps.getProperty(CommonClientConfigs.BOOTSTRAP_SERVERS_CONFIG), + testUtil.getKafkaServerUrl()); + Assert.assertEquals(consumerProps.getProperty(ConsumerConfig.GROUP_ID_CONFIG), + "flume-something"); + Assert.assertEquals(consumerProps.getProperty(ConsumerConfig.AUTO_OFFSET_RESET_CONFIG), + "earliest"); } - @Test public void testSuccess() throws Exception { doTestSuccessRollback(false, false); } - @Test public void testSuccessInterleave() throws Exception { doTestSuccessRollback(false, true); @@ -212,8 +212,10 @@ public void testOffsetsNotCommittedOnStop() throws Exception { KafkaChannel channel = startChannel(false); - KafkaProducer producer = new KafkaProducer(channel.getProducerProps()); - ProducerRecord data = new ProducerRecord(topic, "header-" + message, message.getBytes()); + KafkaProducer producer = + new KafkaProducer(channel.getProducerProps()); + ProducerRecord data = + new ProducerRecord(topic, "header-" + message, message.getBytes()); producer.send(data).get(); producer.flush(); producer.close(); @@ -234,7 +236,7 @@ public void testOffsetsNotCommittedOnStop() throws Exception { } private Event takeEventWithoutCommittingTxn(KafkaChannel channel) { - for (int i=0; i < 5; i++) { + for (int i = 0; i < 5; i++) { Transaction txn = channel.getTransaction(); txn.begin(); @@ -255,17 +257,19 @@ private void doParseAsFlumeEventFalse(Boolean checkHeaders) throws Exception { KafkaProducer producer = new KafkaProducer(props); for (int i = 0; i < 50; i++) { - ProducerRecord data = new ProducerRecord(topic, String.valueOf(i) + "-header", String.valueOf(i).getBytes()); + ProducerRecord data = + new ProducerRecord(topic, String.valueOf(i) + "-header", + String.valueOf(i).getBytes()); producer.send(data).get(); } ExecutorCompletionService submitterSvc = new - ExecutorCompletionService(Executors.newCachedThreadPool()); - List events = pullEvents(channel, submitterSvc, - 50, false, false); + ExecutorCompletionService(Executors.newCachedThreadPool()); + List events = pullEvents(channel, submitterSvc, 50, false, false); wait(submitterSvc, 5); Map finals = new HashMap(); for (int i = 0; i < 50; i++) { - finals.put(Integer.parseInt(new String(events.get(i).getBody())), events.get(i).getHeaders().get(KEY_HEADER)); + finals.put(Integer.parseInt(new String(events.get(i).getBody())), + events.get(i).getHeaders().get(KEY_HEADER)); } for (int i = 0; i < 50; i++) { Assert.assertTrue(finals.keySet().contains(i)); @@ -284,7 +288,8 @@ private void doTestNullKeyNoHeader() throws Exception { KafkaProducer producer = new KafkaProducer(props); for (int i = 0; i < 50; i++) { - ProducerRecord data = new ProducerRecord(topic, null, String.valueOf(i).getBytes()); + ProducerRecord data = + new ProducerRecord(topic, null, String.valueOf(i).getBytes()); producer.send(data).get(); } ExecutorCompletionService submitterSvc = new @@ -323,14 +328,14 @@ public void doParseAsFlumeEventFalseAsSource(Boolean checkHeaders) throws Except channel.put(EventBuilder.withBody(msgs.get(i).getBytes(), headers)); } tx.commit(); - ExecutorCompletionService submitterSvc = new - ExecutorCompletionService(Executors.newCachedThreadPool()); - List events = pullEvents(channel, submitterSvc, - 50, false, false); + ExecutorCompletionService submitterSvc = + new ExecutorCompletionService(Executors.newCachedThreadPool()); + List events = pullEvents(channel, submitterSvc, 50, false, false); wait(submitterSvc, 5); Map finals = new HashMap(); for (int i = 0; i < 50; i++) { - finals.put(Integer.parseInt(new String(events.get(i).getBody())), events.get(i).getHeaders().get(KEY_HEADER)); + finals.put(Integer.parseInt(new String(events.get(i).getBody())), + events.get(i).getHeaders().get(KEY_HEADER)); } for (int i = 0; i < 50; i++) { Assert.assertTrue(finals.keySet().contains(i)); @@ -403,13 +408,13 @@ private void writeAndVerify(final boolean testRollbacks, } private void writeAndVerify(final boolean testRollbacks, - final KafkaChannel channel, final boolean interleave) throws Exception { + final KafkaChannel channel, final boolean interleave) + throws Exception { final List> events = createBaseList(); ExecutorCompletionService submitterSvc = - new ExecutorCompletionService(Executors - .newCachedThreadPool()); + new ExecutorCompletionService(Executors.newCachedThreadPool()); putEvents(channel, events, submitterSvc); @@ -418,11 +423,9 @@ private void writeAndVerify(final boolean testRollbacks, } ExecutorCompletionService submitterSvc2 = - new ExecutorCompletionService(Executors - .newCachedThreadPool()); + new ExecutorCompletionService(Executors.newCachedThreadPool()); - final List eventsPulled = - pullEvents(channel, submitterSvc2, 50, testRollbacks, true); + final List eventsPulled = pullEvents(channel, submitterSvc2, 50, testRollbacks, true); if (!interleave) { wait(submitterSvc, 5); @@ -586,18 +589,18 @@ public static void createTopic(String topicName) { int numPartitions = 5; int sessionTimeoutMs = 10000; int connectionTimeoutMs = 10000; - ZkUtils zkUtils = ZkUtils.apply(testUtil.getZkUrl(), sessionTimeoutMs, connectionTimeoutMs, false); - + ZkUtils zkUtils = + ZkUtils.apply(testUtil.getZkUrl(), sessionTimeoutMs, connectionTimeoutMs, false); int replicationFactor = 1; Properties topicConfig = new Properties(); - AdminUtils.createTopic(zkUtils, topicName, numPartitions, - replicationFactor, topicConfig); + AdminUtils.createTopic(zkUtils, topicName, numPartitions, replicationFactor, topicConfig); } public static void deleteTopic(String topicName) { int sessionTimeoutMs = 10000; int connectionTimeoutMs = 10000; - ZkUtils zkUtils = ZkUtils.apply(testUtil.getZkUrl(), sessionTimeoutMs, connectionTimeoutMs, false); + ZkUtils zkUtils = + ZkUtils.apply(testUtil.getZkUrl(), sessionTimeoutMs, connectionTimeoutMs, false); AdminUtils.deleteTopic(zkUtils, topicName); } } diff --git a/flume-ng-channels/flume-spillable-memory-channel/src/test/java/org/apache/flume/channel/TestSpillableMemoryChannel.java b/flume-ng-channels/flume-spillable-memory-channel/src/test/java/org/apache/flume/channel/TestSpillableMemoryChannel.java index 1e4e81999e..848636b766 100644 --- a/flume-ng-channels/flume-spillable-memory-channel/src/test/java/org/apache/flume/channel/TestSpillableMemoryChannel.java +++ b/flume-ng-channels/flume-spillable-memory-channel/src/test/java/org/apache/flume/channel/TestSpillableMemoryChannel.java @@ -24,10 +24,13 @@ import java.util.ArrayList; import java.util.HashMap; import java.util.Map; - import java.util.UUID; -import org.apache.flume.*; +import org.apache.flume.ChannelException; +import org.apache.flume.ChannelFullException; +import org.apache.flume.Context; +import org.apache.flume.Event; +import org.apache.flume.Transaction; import org.apache.flume.conf.Configurables; import org.apache.flume.event.EventBuilder; import org.apache.flume.channel.file.FileChannelConfiguration; @@ -39,7 +42,6 @@ import org.junit.Rule; import org.junit.rules.TemporaryFolder; - public class TestSpillableMemoryChannel { private SpillableMemoryChannel channel; @@ -51,14 +53,14 @@ private void configureChannel(Map overrides) { Context context = new Context(); File checkPointDir = fileChannelDir.newFolder("checkpoint"); File dataDir = fileChannelDir.newFolder("data"); - context.put(FileChannelConfiguration.CHECKPOINT_DIR - , checkPointDir.getAbsolutePath()); + context.put(FileChannelConfiguration.CHECKPOINT_DIR, checkPointDir.getAbsolutePath()); context.put(FileChannelConfiguration.DATA_DIRS, dataDir.getAbsolutePath()); // Set checkpoint for 5 seconds otherwise test will run out of memory context.put(FileChannelConfiguration.CHECKPOINT_INTERVAL, "5000"); - if (overrides != null) + if (overrides != null) { context.putAll(overrides); + } Configurables.configure(channel, context); } @@ -81,9 +83,9 @@ private void restartChannel(Map params) { startChannel(params); } - static class NullFound extends RuntimeException { public int expectedValue; + public NullFound(int expected) { super("Expected " + expected + ", but null found"); expectedValue = expected; @@ -92,9 +94,10 @@ public NullFound(int expected) { static class TooManyNulls extends RuntimeException { private int nullsFound; + public TooManyNulls(int count) { super("Total nulls found in thread (" - + Thread.currentThread().getName() + ") : " + count); + + Thread.currentThread().getName() + ") : " + count); nullsFound = count; } } @@ -102,7 +105,7 @@ public TooManyNulls(int count) { @Before public void setUp() { channel = new SpillableMemoryChannel(); - channel.setName("spillChannel-" + UUID.randomUUID() ); + channel.setName("spillChannel-" + UUID.randomUUID()); } @After @@ -117,7 +120,7 @@ private static void putN(int first, int count, AbstractChannel channel) { } private static void takeNull(AbstractChannel channel) { - channel.take(); + channel.take(); } private static void takeN(int first, int count, AbstractChannel channel) { @@ -127,7 +130,7 @@ private static void takeN(int first, int count, AbstractChannel channel) { if (e == null) { throw new NullFound(i); } - Event expected = EventBuilder.withBody( String.valueOf(i).getBytes() ); + Event expected = EventBuilder.withBody(String.valueOf(i).getBytes()); Assert.assertArrayEquals(e.getBody(), expected.getBody()); } } @@ -140,16 +143,14 @@ private static int takeN_NoCheck(int batchSize, AbstractChannel channel) { if (e == null) { try { Thread.sleep(0); - } catch (InterruptedException ex) - { /* ignore */ } + } catch (InterruptedException ex) { /* ignore */ } return i; } } return i; } - private static void transactionalPutN(int first, int count, - AbstractChannel channel) { + private static void transactionalPutN(int first, int count, AbstractChannel channel) { Transaction tx = channel.getTransaction(); tx.begin(); try { @@ -163,8 +164,7 @@ private static void transactionalPutN(int first, int count, } } - private static void transactionalTakeN(int first, int count, - AbstractChannel channel) { + private static void transactionalTakeN(int first, int count, AbstractChannel channel) { Transaction tx = channel.getTransaction(); tx.begin(); try { @@ -184,14 +184,13 @@ private static void transactionalTakeN(int first, int count, } } - private static int transactionalTakeN_NoCheck(int count - , AbstractChannel channel) { + private static int transactionalTakeN_NoCheck(int count, AbstractChannel channel) { Transaction tx = channel.getTransaction(); tx.begin(); try { int eventCount = takeN_NoCheck(count, channel); tx.commit(); - return eventCount; + return eventCount; } catch (RuntimeException e) { tx.rollback(); throw e; @@ -204,8 +203,9 @@ private static void transactionalTakeNull(int count, AbstractChannel channel) { Transaction tx = channel.getTransaction(); tx.begin(); try { - for (int i = 0; i < count; ++i) + for (int i = 0; i < count; ++i) { takeNull(channel); + } tx.commit(); } catch (AssertionError e) { tx.rollback(); @@ -218,68 +218,63 @@ private static void transactionalTakeNull(int count, AbstractChannel channel) { } } - private Thread makePutThread(String threadName - , final int first, final int count, final int batchSize - , final AbstractChannel channel) { - return - new Thread(threadName) { - public void run() { - int maxdepth = 0; - StopWatch watch = new StopWatch(); - for (int i = first; i batchSz) ? batchSz : remaining; + int takenCount = transactionalTakeN_NoCheck(batchSz, channel); + if (takenCount < batchSz) { try { - transactionalTakeN(i, batchSize, channel); - i = i + batchSize; - } catch (NullFound e) { - i = e.expectedValue; - } + Thread.sleep(20); + } catch (InterruptedException ex) { /* ignore */ } } - watch.elapsed(); - } - }; - } - - private static Thread makeTakeThread_noCheck(String threadName - , final int totalEvents, final int batchSize, final AbstractChannel channel) { - return - new Thread(threadName) { - public void run() { - int batchSz = batchSize; - StopWatch watch = new StopWatch(); - int i = 0, attempts = 0 ; - while(i < totalEvents) { - int remaining = totalEvents - i; - batchSz = (remaining > batchSz) ? batchSz : remaining; - int takenCount = transactionalTakeN_NoCheck(batchSz, channel); - if(takenCount < batchSz) { - try { - Thread.sleep(20); - } catch (InterruptedException ex) - { /* ignore */ } - } - i += takenCount; - ++attempts; - if(attempts > totalEvents * 3 ) { - throw new TooManyNulls(attempts); - } + i += takenCount; + ++attempts; + if (attempts > totalEvents * 3) { + throw new TooManyNulls(attempts); } - watch.elapsed(" items = " + i + ", attempts = " + attempts); } - }; + watch.elapsed(" items = " + i + ", attempts = " + attempts); + } + }; } @Test @@ -292,37 +287,36 @@ public void testPutTake() { Transaction tx = channel.getTransaction(); tx.begin(); - putN(0,2,channel); + putN(0, 2, channel); tx.commit(); tx.close(); tx = channel.getTransaction(); tx.begin(); - takeN(0,2,channel); + takeN(0, 2, channel); tx.commit(); tx.close(); } - @Test - public void testCapacityDisableOverflow() { + public void testCapacityDisableOverflow() { Map params = new HashMap(); params.put("memoryCapacity", "2"); params.put("overflowCapacity", "0"); // overflow is disabled effectively - params.put("overflowTimeout", "0" ); + params.put("overflowTimeout", "0"); startChannel(params); - transactionalPutN(0,2,channel); + transactionalPutN(0, 2, channel); boolean threw = false; try { - transactionalPutN(2,1,channel); + transactionalPutN(2, 1, channel); } catch (ChannelException e) { threw = true; } Assert.assertTrue("Expecting ChannelFullException to be thrown", threw); - transactionalTakeN(0,2, channel); + transactionalTakeN(0, 2, channel); Transaction tx = channel.getTransaction(); tx.begin(); @@ -332,7 +326,7 @@ public void testCapacityDisableOverflow() { } @Test - public void testCapacityWithOverflow() { + public void testCapacityWithOverflow() { Map params = new HashMap(); params.put("memoryCapacity", "2"); params.put("overflowCapacity", "4"); @@ -346,19 +340,19 @@ public void testCapacityWithOverflow() { boolean threw = false; try { - transactionalPutN(7,2,channel); // cannot fit in channel + transactionalPutN(7, 2, channel); // cannot fit in channel } catch (ChannelFullException e) { threw = true; } Assert.assertTrue("Expecting ChannelFullException to be thrown", threw); - transactionalTakeN(1,2, channel); - transactionalTakeN(3,2, channel); - transactionalTakeN(5,2, channel); + transactionalTakeN(1, 2, channel); + transactionalTakeN(3, 2, channel); + transactionalTakeN(5, 2, channel); } @Test - public void testRestart() { + public void testRestart() { Map params = new HashMap(); params.put("memoryCapacity", "2"); params.put("overflowCapacity", "10"); @@ -372,8 +366,7 @@ public void testRestart() { restartChannel(params); // from overflow, as in memory stuff should be lost - transactionalTakeN(3,2, channel); - + transactionalTakeN(3, 2, channel); } @Test @@ -382,35 +375,34 @@ public void testBasicStart() { params.put("memoryCapacity", "10000000"); params.put("overflowCapacity", "20000000"); params.put(FileChannelConfiguration.TRANSACTION_CAPACITY, "10"); - params.put("overflowTimeout", "1" ); + params.put("overflowTimeout", "1"); startChannel(params); - transactionalPutN( 1,5,channel); - transactionalPutN( 6,5,channel); - transactionalPutN(11,5,channel); // these should go to overflow + transactionalPutN(1, 5, channel); + transactionalPutN(6, 5, channel); + transactionalPutN(11, 5, channel); // these should go to overflow - transactionalTakeN(1,10, channel); - transactionalTakeN(11,5, channel); + transactionalTakeN(1, 10, channel); + transactionalTakeN(11, 5, channel); } @Test public void testOverflow() { - Map params = new HashMap(); params.put("memoryCapacity", "10"); params.put("overflowCapacity", "20"); params.put(FileChannelConfiguration.TRANSACTION_CAPACITY, "10"); - params.put("overflowTimeout", "1" ); + params.put("overflowTimeout", "1"); startChannel(params); - transactionalPutN( 1,5,channel); - transactionalPutN( 6,5,channel); - transactionalPutN(11,5,channel); // these should go to overflow + transactionalPutN(1, 5, channel); + transactionalPutN(6, 5, channel); + transactionalPutN(11, 5, channel); // these should go to overflow - transactionalTakeN(1,10, channel); - transactionalTakeN(11,5, channel); + transactionalTakeN(1, 10, channel); + transactionalTakeN(11, 5, channel); } @Test @@ -419,29 +411,29 @@ public void testDrainOrder() { params.put("memoryCapacity", "10"); params.put("overflowCapacity", "10"); params.put(FileChannelConfiguration.TRANSACTION_CAPACITY, "5"); - params.put("overflowTimeout", "1" ); + params.put("overflowTimeout", "1"); startChannel(params); - transactionalPutN( 1,5,channel); - transactionalPutN( 6,5,channel); - transactionalPutN(11,5,channel); // into overflow - transactionalPutN(16,5,channel); // into overflow + transactionalPutN(1, 5, channel); + transactionalPutN(6, 5, channel); + transactionalPutN(11, 5, channel); // into overflow + transactionalPutN(16, 5, channel); // into overflow transactionalTakeN(1, 1, channel); - transactionalTakeN(2, 5,channel); - transactionalTakeN(7, 4,channel); + transactionalTakeN(2, 5, channel); + transactionalTakeN(7, 4, channel); - transactionalPutN( 20,2,channel); - transactionalPutN( 22,3,channel); + transactionalPutN(20, 2, channel); + transactionalPutN(22, 3, channel); - transactionalTakeN( 11,3,channel); // from overflow - transactionalTakeN( 14,5,channel); // from overflow - transactionalTakeN( 19,2,channel); // from overflow + transactionalTakeN(11, 3, channel); // from overflow + transactionalTakeN(14, 5, channel); // from overflow + transactionalTakeN(19, 2, channel); // from overflow } @Test - public void testByteCapacity() { + public void testByteCapacity() { Map params = new HashMap(); params.put("memoryCapacity", "1000"); // configure to hold 8 events of 10 bytes each (plus 20% event header space) @@ -449,12 +441,12 @@ public void testByteCapacity() { params.put("avgEventSize", "10"); params.put("overflowCapacity", "20"); params.put(FileChannelConfiguration.TRANSACTION_CAPACITY, "10"); - params.put("overflowTimeout", "1" ); + params.put("overflowTimeout", "1"); startChannel(params); transactionalPutN(1, 8, channel); // this wil max the byteCapacity transactionalPutN(9, 10, channel); - transactionalPutN(19,10, channel); // this will fill up the overflow + transactionalPutN(19, 10, channel); // this will fill up the overflow boolean threw = false; try { @@ -463,7 +455,6 @@ public void testByteCapacity() { threw = true; } Assert.assertTrue("byteCapacity did not throw as expected", threw); - } @Test @@ -473,7 +464,7 @@ public void testDrainingOnChannelBoundary() { params.put("memoryCapacity", "5"); params.put("overflowCapacity", "15"); params.put(FileChannelConfiguration.TRANSACTION_CAPACITY, "10"); - params.put("overflowTimeout", "1" ); + params.put("overflowTimeout", "1"); startChannel(params); transactionalPutN(1, 5, channel); @@ -493,13 +484,13 @@ public void testDrainingOnChannelBoundary() { transactionalTakeN(6, 5, channel); // from overflow transactionalTakeN(11, 5, channel); // from overflow - transactionalTakeN(16, 2,channel); // from overflow + transactionalTakeN(16, 2, channel); // from overflow transactionalPutN(21, 5, channel); tx = channel.getTransaction(); tx.begin(); - takeN(18,3, channel); // from overflow + takeN(18, 3, channel); // from overflow takeNull(channel); // expect null since next event is in primary tx.commit(); tx.close(); @@ -516,9 +507,8 @@ public void testRollBack() { params.put("overflowTimeout", "0"); startChannel(params); - //1 Rollback for Puts - transactionalPutN(1,5, channel); + transactionalPutN(1, 5, channel); Transaction tx = channel.getTransaction(); tx.begin(); putN(6, 5, channel); @@ -530,8 +520,7 @@ public void testRollBack() { //2. verify things back to normal after put rollback transactionalPutN(11, 5, channel); - transactionalTakeN(11,5,channel); - + transactionalTakeN(11, 5, channel); //3 Rollback for Takes transactionalPutN(16, 5, channel); @@ -545,13 +534,12 @@ public void testRollBack() { transactionalTakeN_NoCheck(5, channel); //4. verify things back to normal after take rollback - transactionalPutN(21,5, channel); - transactionalTakeN(21,5,channel); + transactionalPutN(21, 5, channel); + transactionalTakeN(21, 5, channel); } - @Test - public void testReconfigure() { + public void testReconfigure() { //1) bring up with small capacity Map params = new HashMap(); params.put("memoryCapacity", "10"); @@ -559,12 +547,12 @@ public void testReconfigure() { params.put("overflowTimeout", "0"); startChannel(params); - Assert.assertTrue("overflowTimeout setting did not reconfigure correctly" - , channel.getOverflowTimeout() == 0); - Assert.assertTrue("memoryCapacity did not reconfigure correctly" - , channel.getMemoryCapacity() == 10); - Assert.assertTrue("overflowCapacity did not reconfigure correctly" - , channel.isOverflowDisabled() ); + Assert.assertTrue("overflowTimeout setting did not reconfigure correctly", + channel.getOverflowTimeout() == 0); + Assert.assertTrue("memoryCapacity did not reconfigure correctly", + channel.getMemoryCapacity() == 10); + Assert.assertTrue("overflowCapacity did not reconfigure correctly", + channel.isOverflowDisabled()); transactionalPutN(1, 10, channel); boolean threw = false; @@ -574,8 +562,7 @@ public void testReconfigure() { threw = true; } Assert.assertTrue("Expected the channel to fill up and throw an exception, " - + "but it did not throw", threw); - + + "but it did not throw", threw); //2) Resize and verify params = new HashMap(); @@ -583,12 +570,13 @@ public void testReconfigure() { params.put("overflowCapacity", "0"); reconfigureChannel(params); - Assert.assertTrue("overflowTimeout setting did not reconfigure correctly" - , channel.getOverflowTimeout() == SpillableMemoryChannel.defaultOverflowTimeout); - Assert.assertTrue("memoryCapacity did not reconfigure correctly" - , channel.getMemoryCapacity() == 20); - Assert.assertTrue("overflowCapacity did not reconfigure correctly" - , channel.isOverflowDisabled() ); + Assert.assertTrue("overflowTimeout setting did not reconfigure correctly", + channel.getOverflowTimeout() == + SpillableMemoryChannel.defaultOverflowTimeout); + Assert.assertTrue("memoryCapacity did not reconfigure correctly", + channel.getMemoryCapacity() == 20); + Assert.assertTrue("overflowCapacity did not reconfigure correctly", + channel.isOverflowDisabled()); // pull out the values inserted prior to reconfiguration transactionalTakeN(1, 10, channel); @@ -603,25 +591,25 @@ public void testReconfigure() { threw = true; } Assert.assertTrue("Expected the channel to fill up and throw an exception, " - + "but it did not throw", threw); + + "but it did not throw", threw); transactionalTakeN(11, 10, channel); transactionalTakeN(21, 10, channel); - // 3) Reconfigure with empty config and verify settings revert to default params = new HashMap(); reconfigureChannel(params); - Assert.assertTrue("overflowTimeout setting did not reconfigure correctly" - , channel.getOverflowTimeout() == SpillableMemoryChannel.defaultOverflowTimeout); - Assert.assertTrue("memoryCapacity did not reconfigure correctly" - , channel.getMemoryCapacity() == SpillableMemoryChannel.defaultMemoryCapacity); - Assert.assertTrue("overflowCapacity did not reconfigure correctly" - , channel.getOverflowCapacity() == SpillableMemoryChannel.defaultOverflowCapacity); - Assert.assertFalse("overflowCapacity did not reconfigure correctly" - , channel.isOverflowDisabled()); - + Assert.assertTrue("overflowTimeout setting did not reconfigure correctly", + channel.getOverflowTimeout() == + SpillableMemoryChannel.defaultOverflowTimeout); + Assert.assertTrue("memoryCapacity did not reconfigure correctly", + channel.getMemoryCapacity() == SpillableMemoryChannel.defaultMemoryCapacity); + Assert.assertTrue("overflowCapacity did not reconfigure correctly", + channel.getOverflowCapacity() == + SpillableMemoryChannel.defaultOverflowCapacity); + Assert.assertFalse("overflowCapacity did not reconfigure correctly", + channel.isOverflowDisabled()); // 4) Reconfiguring of overflow params = new HashMap(); @@ -631,19 +619,18 @@ public void testReconfigure() { params.put("overflowTimeout", "1"); reconfigureChannel(params); - transactionalPutN( 1,5, channel); - transactionalPutN( 6,5, channel); - transactionalPutN(11,5, channel); - transactionalPutN(16,5, channel); - threw=false; + transactionalPutN(1, 5, channel); + transactionalPutN(6, 5, channel); + transactionalPutN(11, 5, channel); + transactionalPutN(16, 5, channel); + threw = false; try { // should error out as both primary & overflow are full - transactionalPutN(21,5, channel); + transactionalPutN(21, 5, channel); } catch (ChannelException e) { threw = true; } - Assert.assertTrue("Expected the last insertion to fail, but it didn't." - , threw); + Assert.assertTrue("Expected the last insertion to fail, but it didn't.", threw); // reconfig the overflow params = new HashMap(); @@ -654,10 +641,10 @@ public void testReconfigure() { reconfigureChannel(params); // should succeed now as we have made room in the overflow - transactionalPutN(21,5, channel); + transactionalPutN(21, 5, channel); - transactionalTakeN(1,10, channel); - transactionalTakeN(11,5, channel); + transactionalTakeN(1, 10, channel); + transactionalTakeN(11, 5, channel); transactionalTakeN(16, 5, channel); transactionalTakeN(21, 5, channel); } @@ -666,13 +653,13 @@ public void testReconfigure() { public void testParallelSingleSourceAndSink() throws InterruptedException { Map params = new HashMap(); params.put("memoryCapacity", "1000020"); - params.put("overflowCapacity", "0"); + params.put("overflowCapacity", "0"); params.put("overflowTimeout", "3"); startChannel(params); // run source and sink concurrently Thread sourceThd = makePutThread("src", 1, 500000, 100, channel); - Thread sinkThd = makeTakeThread("sink", 1, 500000, 100, channel); + Thread sinkThd = makeTakeThread("sink", 1, 500000, 100, channel); StopWatch watch = new StopWatch(); @@ -683,15 +670,15 @@ public void testParallelSingleSourceAndSink() throws InterruptedException { sinkThd.join(); watch.elapsed(); - System.out.println("Max Queue size " + channel.getMaxMemQueueSize() ); + System.out.println("Max Queue size " + channel.getMaxMemQueueSize()); } @Test public void testCounters() throws InterruptedException { Map params = new HashMap(); - params.put("memoryCapacity", "5000"); - params.put("overflowCapacity","5000"); - params.put("transactionCapacity","5000"); + params.put("memoryCapacity", "5000"); + params.put("overflowCapacity", "5000"); + params.put("transactionCapacity", "5000"); params.put("overflowTimeout", "0"); startChannel(params); @@ -706,7 +693,7 @@ public void testCounters() throws InterruptedException { Assert.assertEquals(5000, channel.channelCounter.getEventPutSuccessCount()); //2. empty mem queue - Thread sinkThd = makeTakeThread("sink", 1, 5000, 1000, channel); + Thread sinkThd = makeTakeThread("sink", 1, 5000, 1000, channel); sinkThd.start(); sinkThd.join(); Assert.assertEquals(0, channel.getTotalStored()); @@ -714,7 +701,6 @@ public void testCounters() throws InterruptedException { Assert.assertEquals(5000, channel.channelCounter.getEventTakeAttemptCount()); Assert.assertEquals(5000, channel.channelCounter.getEventTakeSuccessCount()); - //3. fill up mem & overflow sourceThd = makePutThread("src", 1, 10000, 1000, channel); sourceThd.start(); @@ -724,9 +710,8 @@ public void testCounters() throws InterruptedException { Assert.assertEquals(15000, channel.channelCounter.getEventPutAttemptCount()); Assert.assertEquals(15000, channel.channelCounter.getEventPutSuccessCount()); - //4. empty memory - sinkThd = makeTakeThread("sink", 1, 5000, 1000, channel); + sinkThd = makeTakeThread("sink", 1, 5000, 1000, channel); sinkThd.start(); sinkThd.join(); Assert.assertEquals(5000, channel.getTotalStored()); @@ -745,12 +730,10 @@ public void testCounters() throws InterruptedException { Assert.assertEquals(15000, channel.channelCounter.getEventTakeAttemptCount()); Assert.assertEquals(15000, channel.channelCounter.getEventTakeSuccessCount()); - - //6. now do it concurrently sourceThd = makePutThread("src1", 1, 5000, 1000, channel); Thread sourceThd2 = makePutThread("src2", 1, 5000, 500, channel); - sinkThd = makeTakeThread_noCheck("sink1", 5000, 1000, channel); + sinkThd = makeTakeThread_noCheck("sink1", 5000, 1000, channel); sourceThd.start(); sourceThd2.start(); sinkThd.start(); @@ -759,8 +742,8 @@ public void testCounters() throws InterruptedException { sinkThd.join(); Assert.assertEquals(5000, channel.getTotalStored()); Assert.assertEquals(5000, channel.channelCounter.getChannelSize()); - Thread sinkThd2 = makeTakeThread_noCheck("sink2", 2500, 500, channel); - Thread sinkThd3 = makeTakeThread_noCheck("sink3", 2500, 1000, channel); + Thread sinkThd2 = makeTakeThread_noCheck("sink2", 2500, 500, channel); + Thread sinkThd3 = makeTakeThread_noCheck("sink3", 2500, 1000, channel); sinkThd2.start(); sinkThd3.start(); sinkThd2.join(); @@ -769,30 +752,26 @@ public void testCounters() throws InterruptedException { Assert.assertEquals(0, channel.channelCounter.getChannelSize()); Assert.assertEquals(25000, channel.channelCounter.getEventTakeSuccessCount()); Assert.assertEquals(25000, channel.channelCounter.getEventPutSuccessCount()); - Assert.assertTrue("TakeAttempt channel counter value larger than expected" , - 25000 <= channel.channelCounter.getEventTakeAttemptCount()); + Assert.assertTrue("TakeAttempt channel counter value larger than expected", + 25000 <= channel.channelCounter.getEventTakeAttemptCount()); Assert.assertTrue("PutAttempt channel counter value larger than expected", - 25000 <= channel.channelCounter.getEventPutAttemptCount()); + 25000 <= channel.channelCounter.getEventPutAttemptCount()); } - public ArrayList createSourceThreads(int count, int totalEvents - , int batchSize) { + public ArrayList createSourceThreads(int count, int totalEvents, int batchSize) { ArrayList sourceThds = new ArrayList(); for (int i = 0; i < count; ++i) { - sourceThds.add( makePutThread("src" + i, 1, totalEvents/count - , batchSize, channel) ); + sourceThds.add(makePutThread("src" + i, 1, totalEvents / count, batchSize, channel)); } return sourceThds; } - public ArrayList createSinkThreads(int count, int totalEvents - , int batchSize) { + public ArrayList createSinkThreads(int count, int totalEvents, int batchSize) { ArrayList sinkThreads = new ArrayList(count); for (int i = 0; i < count; ++i) { - sinkThreads.add( makeTakeThread_noCheck("sink"+i, totalEvents/count - , batchSize, channel) ); + sinkThreads.add(makeTakeThread_noCheck("sink" + i, totalEvents / count, batchSize, channel)); } return sinkThreads; } @@ -803,13 +782,12 @@ public void startThreads(ArrayList threads) { } } - public void joinThreads(ArrayList threads) - throws InterruptedException { + public void joinThreads(ArrayList threads) throws InterruptedException { for (Thread thread : threads) { try { thread.join(); } catch (InterruptedException e) { - System.out.println("Interrupted while waiting on " + thread.getName() ); + System.out.println("Interrupted while waiting on " + thread.getName()); throw e; } } @@ -824,15 +802,13 @@ public void testParallelMultipleSourcesAndSinks() throws InterruptedException { Map params = new HashMap(); params.put("memoryCapacity", "0"); - params.put("overflowCapacity", "500020"); + params.put("overflowCapacity", "500020"); params.put("overflowTimeout", "3"); startChannel(params); ArrayList sinks = createSinkThreads(sinkCount, eventCount, batchSize); - ArrayList sources = createSourceThreads(sourceCount - , eventCount, batchSize); - + ArrayList sources = createSourceThreads(sourceCount, eventCount, batchSize); StopWatch watch = new StopWatch(); startThreads(sinks); @@ -845,7 +821,7 @@ public void testParallelMultipleSourcesAndSinks() throws InterruptedException { System.out.println("Total puts " + channel.drainOrder.totalPuts); - System.out.println("Max Queue size " + channel.getMaxMemQueueSize() ); + System.out.println("Max Queue size " + channel.getMaxMemQueueSize()); System.out.println(channel.memQueue.size()); System.out.println("done"); @@ -872,10 +848,10 @@ public void elapsed(String suffix) { if (elapsed < 10000) { System.out.println(Thread.currentThread().getName() - + " : [ " + elapsed + " ms ]. " + suffix); + + " : [ " + elapsed + " ms ]. " + suffix); } else { System.out.println(Thread.currentThread().getName() - + " : [ " + elapsed / 1000 + " sec ]. " + suffix); + + " : [ " + elapsed / 1000 + " sec ]. " + suffix); } } } diff --git a/flume-ng-clients/flume-ng-log4jappender/src/test/java/org/apache/flume/clients/log4jappender/TestLoadBalancingLog4jAppender.java b/flume-ng-clients/flume-ng-log4jappender/src/test/java/org/apache/flume/clients/log4jappender/TestLoadBalancingLog4jAppender.java index 267ac1d9a9..53795fb2a4 100644 --- a/flume-ng-clients/flume-ng-log4jappender/src/test/java/org/apache/flume/clients/log4jappender/TestLoadBalancingLog4jAppender.java +++ b/flume-ng-clients/flume-ng-log4jappender/src/test/java/org/apache/flume/clients/log4jappender/TestLoadBalancingLog4jAppender.java @@ -65,7 +65,7 @@ public class TestLoadBalancingLog4jAppender { private boolean slowDown = false; @Before - public void initiate() throws InterruptedException{ + public void initiate() throws InterruptedException { ch = new MemoryChannel(); configureChannel(); @@ -164,9 +164,9 @@ public void testRandomBackoff() throws Exception { @Test public void testRandomBackoffUnsafeMode() throws Exception { File TESTFILE = new File(TestLoadBalancingLog4jAppender.class - .getClassLoader() - .getResource("flume-loadbalancing-backoff-log4jtest.properties") - .getFile()); + .getClassLoader() + .getResource("flume-loadbalancing-backoff-log4jtest.properties") + .getFile()); startSources(TESTFILE, true, new int[]{25430, 25431, 25432}); sources.get(0).setFail(); @@ -179,9 +179,9 @@ public void testRandomBackoffUnsafeMode() throws Exception { @Test (expected = EventDeliveryException.class) public void testTimeout() throws Throwable { File TESTFILE = new File(TestLoadBalancingLog4jAppender.class - .getClassLoader() - .getResource("flume-loadbalancinglog4jtest.properties") - .getFile()); + .getClassLoader() + .getResource("flume-loadbalancinglog4jtest.properties") + .getFile()); ch = new TestLog4jAppender.SlowMemoryChannel(2000); configureChannel(); @@ -200,9 +200,9 @@ public void testTimeout() throws Throwable { @Test(expected = EventDeliveryException.class) public void testRandomBackoffNotUnsafeMode() throws Throwable { File TESTFILE = new File(TestLoadBalancingLog4jAppender.class - .getClassLoader() - .getResource("flume-loadbalancing-backoff-log4jtest.properties") - .getFile()); + .getClassLoader() + .getResource("flume-loadbalancing-backoff-log4jtest.properties") + .getFile()); startSources(TESTFILE, false, new int[]{25430, 25431, 25432}); sources.get(0).setFail(); @@ -224,17 +224,17 @@ private void send(int numberOfMsgs) throws EventDeliveryException { } private void sendAndAssertFail() throws IOException { - int level = 20000; - String msg = "This is log message number" + String.valueOf(level); - fixture.log(Level.toLevel(level), msg); + int level = 20000; + String msg = "This is log message number" + String.valueOf(level); + fixture.log(Level.toLevel(level), msg); - Transaction transaction = ch.getTransaction(); - transaction.begin(); - Event event = ch.take(); - Assert.assertNull(event); + Transaction transaction = ch.getTransaction(); + transaction.begin(); + Event event = ch.take(); + Assert.assertNull(event); - transaction.commit(); - transaction.close(); + transaction.commit(); + transaction.close(); } @@ -271,8 +271,7 @@ private void sendAndAssertMessages(int numberOfMsgs) throws IOException { } private void startSources(File log4jProps, boolean unsafeMode, int... ports) - throws - IOException { + throws IOException { for (int port : ports) { CountingAvroSource source = new CountingAvroSource(port); Context context = new Context(); @@ -297,8 +296,8 @@ private void startSources(File log4jProps, boolean unsafeMode, int... ports) Properties props = new Properties(); props.load(reader); props.setProperty("log4j.appender.out2.UnsafeMode", - String.valueOf(unsafeMode)); - if(slowDown) { + String.valueOf(unsafeMode)); + if (slowDown) { props.setProperty("log4j.appender.out2.Timeout", String.valueOf(1000)); } PropertyConfigurator.configure(props); @@ -308,13 +307,13 @@ private void startSources(File log4jProps, boolean unsafeMode, int... ports) static class CountingAvroSource extends AvroSource { AtomicInteger appendCount = new AtomicInteger(); volatile boolean isFail = false; - private final int port2; + private final int port2; public CountingAvroSource(int port) { - port2 = port; + port2 = port; } - public void setOk() { + public void setOk() { this.isFail = false; } diff --git a/flume-ng-clients/flume-ng-log4jappender/src/test/java/org/apache/flume/clients/log4jappender/TestLog4jAppender.java b/flume-ng-clients/flume-ng-log4jappender/src/test/java/org/apache/flume/clients/log4jappender/TestLog4jAppender.java index 1b840f3ba2..c087b67d24 100644 --- a/flume-ng-clients/flume-ng-log4jappender/src/test/java/org/apache/flume/clients/log4jappender/TestLog4jAppender.java +++ b/flume-ng-clients/flume-ng-log4jappender/src/test/java/org/apache/flume/clients/log4jappender/TestLog4jAppender.java @@ -21,7 +21,10 @@ import java.io.File; import java.io.FileReader; import java.io.IOException; -import java.util.*; +import java.util.ArrayList; +import java.util.List; +import java.util.Map; +import java.util.Properties; import java.util.concurrent.TimeUnit; import junit.framework.Assert; @@ -46,13 +49,13 @@ import org.junit.Before; import org.junit.Test; -public class TestLog4jAppender{ +public class TestLog4jAppender { private AvroSource source; private Channel ch; private Properties props; @Before - public void initiate() throws Exception{ + public void initiate() throws Exception { int port = 25430; source = new AvroSource(); ch = new MemoryChannel(); @@ -88,13 +91,13 @@ public void testLog4jAppender() throws IOException { configureSource(); PropertyConfigurator.configure(props); Logger logger = LogManager.getLogger(TestLog4jAppender.class); - for(int count = 0; count <= 1000; count++){ + for (int count = 0; count <= 1000; count++) { /* * Log4j internally defines levels as multiples of 10000. So if we * create levels directly using count, the level will be set as the * default. */ - int level = ((count % 5)+1)*10000; + int level = ((count % 5) + 1) * 10000; String msg = "This is log message number" + String.valueOf(count); logger.log(Level.toLevel(level), msg); @@ -146,11 +149,11 @@ public void testLog4jAppenderFailureNotUnsafeMode() throws Throwable { } private void sendAndAssertFail(Logger logger) throws Throwable { - /* - * Log4j internally defines levels as multiples of 10000. So if we - * create levels directly using count, the level will be set as the - * default. - */ + /* + * Log4j internally defines levels as multiples of 10000. So if we + * create levels directly using count, the level will be set as the + * default. + */ int level = 20000; try { logger.log(Level.toLevel(level), "Test Msg"); @@ -177,13 +180,13 @@ public void testLayout() throws IOException { PropertyConfigurator.configure(props); Logger logger = LogManager.getLogger(TestLog4jAppender.class); Thread.currentThread().setName("Log4jAppenderTest"); - for(int count = 0; count <= 100; count++){ + for (int count = 0; count <= 100; count++) { /* * Log4j internally defines levels as multiples of 10000. So if we * create levels directly using count, the level will be set as the * default. */ - int level = ((count % 5)+1)*10000; + int level = ((count % 5) + 1) * 10000; String msg = "This is log message number" + String.valueOf(count); logger.log(Level.toLevel(level), msg); @@ -230,7 +233,7 @@ public void testSlowness() throws Throwable { props.put("log4j.appender.out2.Timeout", "1000"); props.put("log4j.appender.out2.layout", "org.apache.log4j.PatternLayout"); props.put("log4j.appender.out2.layout.ConversionPattern", - "%-5p [%t]: %m%n"); + "%-5p [%t]: %m%n"); PropertyConfigurator.configure(props); Logger logger = LogManager.getLogger(TestLog4jAppender.class); Thread.currentThread().setName("Log4jAppenderTest"); @@ -251,13 +254,12 @@ public void testSlownessUnsafeMode() throws Throwable { @After - public void cleanUp(){ + public void cleanUp() { source.stop(); ch.stop(); props.clear(); } - static class SlowMemoryChannel extends MemoryChannel { private final int slowTime; diff --git a/flume-ng-clients/flume-ng-log4jappender/src/test/java/org/apache/flume/clients/log4jappender/TestLog4jAppenderWithAvro.java b/flume-ng-clients/flume-ng-log4jappender/src/test/java/org/apache/flume/clients/log4jappender/TestLog4jAppenderWithAvro.java index 5899c629e7..0607e3a148 100644 --- a/flume-ng-clients/flume-ng-log4jappender/src/test/java/org/apache/flume/clients/log4jappender/TestLog4jAppenderWithAvro.java +++ b/flume-ng-clients/flume-ng-log4jappender/src/test/java/org/apache/flume/clients/log4jappender/TestLog4jAppenderWithAvro.java @@ -125,8 +125,7 @@ public void testAvroGeneric() throws IOException { Assert.assertNull(hdrs.get(Log4jAvroHeaders.MESSAGE_ENCODING.toString())); Assert.assertEquals("Schema URL should be set", - "file:///tmp/myrecord.avsc", hdrs.get(Log4jAvroHeaders.AVRO_SCHEMA_URL.toString - ())); + "file:///tmp/myrecord.avsc", hdrs.get(Log4jAvroHeaders.AVRO_SCHEMA_URL.toString())); Assert.assertNull("Schema string should not be set", hdrs.get(Log4jAvroHeaders.AVRO_SCHEMA_LITERAL.toString())); @@ -174,7 +173,7 @@ public void testAvroReflect() throws IOException { } @After - public void cleanUp(){ + public void cleanUp() { source.stop(); ch.stop(); props.clear(); diff --git a/flume-ng-core/src/test/java/org/apache/flume/channel/AbstractBasicChannelSemanticsTest.java b/flume-ng-core/src/test/java/org/apache/flume/channel/AbstractBasicChannelSemanticsTest.java index 59a804c10a..7600856e8e 100644 --- a/flume-ng-core/src/test/java/org/apache/flume/channel/AbstractBasicChannelSemanticsTest.java +++ b/flume-ng-core/src/test/java/org/apache/flume/channel/AbstractBasicChannelSemanticsTest.java @@ -40,6 +40,7 @@ public abstract class AbstractBasicChannelSemanticsTest { protected static List events; + static { Event[] array = new Event[7]; for (int i = 0; i < array.length; ++i) { @@ -61,7 +62,7 @@ public enum Mode { THROW_RUNTIME, THROW_CHANNEL, SLEEP - }; + } private Mode mode = Mode.NORMAL; private boolean lastTransactionCommitted = false; @@ -158,11 +159,11 @@ protected void doClose() { protected static class TestError extends Error { static final long serialVersionUID = -1; - }; + } protected static class TestRuntimeException extends RuntimeException { static final long serialVersionUID = -1; - }; + } protected void testException(Class exceptionClass, Runnable test) { diff --git a/flume-ng-core/src/test/java/org/apache/flume/channel/TestChannelProcessor.java b/flume-ng-core/src/test/java/org/apache/flume/channel/TestChannelProcessor.java index b37b823036..93bc0cf517 100644 --- a/flume-ng-core/src/test/java/org/apache/flume/channel/TestChannelProcessor.java +++ b/flume-ng-core/src/test/java/org/apache/flume/channel/TestChannelProcessor.java @@ -20,16 +20,22 @@ import com.google.common.base.Charsets; import com.google.common.collect.Lists; - -import java.util.ArrayList; -import java.util.List; - -import org.apache.flume.*; +import org.apache.flume.Channel; +import org.apache.flume.ChannelException; +import org.apache.flume.ChannelSelector; +import org.apache.flume.Context; +import org.apache.flume.Event; +import org.apache.flume.Transaction; import org.apache.flume.conf.Configurables; import org.apache.flume.event.EventBuilder; import org.junit.Assert; import org.junit.Test; -import static org.mockito.Mockito.*; + +import java.util.ArrayList; +import java.util.List; + +import static org.mockito.Mockito.mock; +import static org.mockito.Mockito.when; public class TestChannelProcessor { @@ -88,9 +94,9 @@ public void testNullFromGetTransaction() { public void testRequiredAndOptionalChannels() { Context context = new Context(); ArrayList channels = new ArrayList(); - for(int i = 0; i < 4; i++) { + for (int i = 0; i < 4; i++) { Channel ch = new MemoryChannel(); - ch.setName("ch"+i); + ch.setName("ch" + i); Configurables.configure(ch, context); channels.add(ch); } @@ -114,7 +120,7 @@ public void testRequiredAndOptionalChannels() { } catch (InterruptedException e) { } - for(Channel channel : channels) { + for (Channel channel : channels) { Transaction transaction = channel.getTransaction(); transaction.begin(); Event event_ch = channel.take(); @@ -124,18 +130,18 @@ public void testRequiredAndOptionalChannels() { } List events = Lists.newArrayList(); - for(int i = 0; i < 100; i ++) { - events.add(EventBuilder.withBody("event "+i, Charsets.UTF_8)); + for (int i = 0; i < 100; i++) { + events.add(EventBuilder.withBody("event " + i, Charsets.UTF_8)); } processor.processEventBatch(events); try { Thread.sleep(3000); } catch (InterruptedException e) { } - for(Channel channel : channels) { + for (Channel channel : channels) { Transaction transaction = channel.getTransaction(); transaction.begin(); - for(int i = 0; i < 100; i ++) { + for (int i = 0; i < 100; i++) { Event event_ch = channel.take(); Assert.assertNotNull(event_ch); } diff --git a/flume-ng-core/src/test/java/org/apache/flume/channel/TestMemoryChannel.java b/flume-ng-core/src/test/java/org/apache/flume/channel/TestMemoryChannel.java index 7851536954..8921a19513 100644 --- a/flume-ng-core/src/test/java/org/apache/flume/channel/TestMemoryChannel.java +++ b/flume-ng-core/src/test/java/org/apache/flume/channel/TestMemoryChannel.java @@ -19,10 +19,6 @@ package org.apache.flume.channel; -import java.util.HashMap; -import java.util.Map; -import java.util.concurrent.LinkedBlockingDeque; - import org.apache.flume.Channel; import org.apache.flume.ChannelException; import org.apache.flume.Context; @@ -34,8 +30,12 @@ import org.junit.Assert; import org.junit.Before; import org.junit.Test; -import static org.fest.reflect.core.Reflection.*; +import java.util.HashMap; +import java.util.Map; +import java.util.concurrent.LinkedBlockingDeque; + +import static org.fest.reflect.core.Reflection.field; public class TestMemoryChannel { @@ -81,7 +81,7 @@ public void testChannelResize() { Transaction transaction = channel.getTransaction(); transaction.begin(); - for(int i=0; i < 5; i++) { + for (int i = 0; i < 5; i++) { channel.put(EventBuilder.withBody(String.format("test event %d", i).getBytes())); } transaction.commit(); @@ -124,7 +124,7 @@ public void testChannelResize() { parms.put("transactionCapacity", "2"); context.putAll(parms); Configurables.configure(channel, context); - for(int i=0; i < 6; i++) { + for (int i = 0; i < 6; i++) { transaction = channel.getTransaction(); transaction.begin(); Assert.assertNotNull(channel.take()); @@ -133,7 +133,7 @@ public void testChannelResize() { } } - @Test(expected=ChannelException.class) + @Test(expected = ChannelException.class) public void testTransactionPutCapacityOverload() { Context context = new Context(); Map parms = new HashMap(); @@ -151,7 +151,7 @@ public void testTransactionPutCapacityOverload() { Assert.fail(); } - @Test(expected=ChannelException.class) + @Test(expected = ChannelException.class) public void testCapacityOverload() { Context context = new Context(); Map parms = new HashMap(); @@ -236,7 +236,7 @@ public void testCapacityBufferEmptyingAfterRollback() { tx.close(); } - @Test(expected=ChannelException.class) + @Test(expected = ChannelException.class) public void testByteCapacityOverload() { Context context = new Context(); Map parms = new HashMap(); @@ -284,8 +284,7 @@ public void testByteCapacityBufferEmptyingAfterTakeCommit() { try { channel.put(EventBuilder.withBody(eventBody)); throw new RuntimeException("Put was able to overflow byte capacity."); - } catch (ChannelException ce) - { + } catch (ChannelException ce) { //Do nothing } @@ -306,8 +305,7 @@ public void testByteCapacityBufferEmptyingAfterTakeCommit() { try { channel.put(EventBuilder.withBody(eventBody)); throw new RuntimeException("Put was able to overflow byte capacity."); - } catch (ChannelException ce) - { + } catch (ChannelException ce) { //Do nothing } tx.commit(); @@ -370,7 +368,7 @@ public void testByteCapacityBufferChangeConfig() { channel.put(EventBuilder.withBody(eventBody)); tx.commit(); Assert.fail(); - } catch ( ChannelException e ) { + } catch (ChannelException e) { //success tx.rollback(); } finally { @@ -397,12 +395,12 @@ public void testByteCapacityBufferChangeConfig() { tx = channel.getTransaction(); tx.begin(); try { - for(int i = 0; i < 2; i++) { + for (int i = 0; i < 2; i++) { channel.put(EventBuilder.withBody(eventBody)); } tx.commit(); Assert.fail(); - } catch ( ChannelException e ) { + } catch (ChannelException e) { //success tx.rollback(); } finally { @@ -418,12 +416,12 @@ public void testByteCapacityBufferChangeConfig() { tx.begin(); try { - for(int i = 0; i < 15; i++) { + for (int i = 0; i < 15; i++) { channel.put(EventBuilder.withBody(eventBody)); } tx.commit(); Assert.fail(); - } catch ( ChannelException e ) { + } catch (ChannelException e) { //success tx.rollback(); } finally { @@ -438,12 +436,12 @@ public void testByteCapacityBufferChangeConfig() { tx.begin(); try { - for(int i = 0; i < 25; i++) { + for (int i = 0; i < 25; i++) { channel.put(EventBuilder.withBody(eventBody)); } tx.commit(); Assert.fail(); - } catch ( ChannelException e ) { + } catch (ChannelException e) { //success tx.rollback(); } finally { diff --git a/flume-ng-core/src/test/java/org/apache/flume/channel/TestMemoryChannelConcurrency.java b/flume-ng-core/src/test/java/org/apache/flume/channel/TestMemoryChannelConcurrency.java index d4ba7052dd..68aa117515 100644 --- a/flume-ng-core/src/test/java/org/apache/flume/channel/TestMemoryChannelConcurrency.java +++ b/flume-ng-core/src/test/java/org/apache/flume/channel/TestMemoryChannelConcurrency.java @@ -17,15 +17,6 @@ */ package org.apache.flume.channel; -import java.util.Map.Entry; -import java.util.Random; -import java.util.concurrent.BrokenBarrierException; -import java.util.concurrent.ConcurrentHashMap; -import java.util.concurrent.CountDownLatch; -import java.util.concurrent.CyclicBarrier; -import java.util.concurrent.TimeUnit; -import java.util.concurrent.atomic.AtomicInteger; - import org.apache.flume.Channel; import org.apache.flume.ChannelException; import org.apache.flume.Context; @@ -37,6 +28,15 @@ import org.junit.Before; import org.junit.Test; +import java.util.Map.Entry; +import java.util.Random; +import java.util.concurrent.BrokenBarrierException; +import java.util.concurrent.ConcurrentHashMap; +import java.util.concurrent.CountDownLatch; +import java.util.concurrent.CyclicBarrier; +import java.util.concurrent.TimeUnit; +import java.util.concurrent.atomic.AtomicInteger; + public class TestMemoryChannelConcurrency { private CyclicBarrier barrier; @@ -120,10 +120,10 @@ public void run() { } /** - * Works with a startgate/endgate latches to make sure all threads run at the same time. Threads randomly - * choose to commit or rollback random numbers of actions, tagging them with the thread no. - * The correctness check is made by recording committed entries into a map, and verifying the count - * after the endgate + * Works with a startgate/endgate latches to make sure all threads run at the same time. + * Threads randomly choose to commit or rollback random numbers of actions, tagging them with the + * thread no. The correctness check is made by recording committed entries into a map, and + * verifying the count after the endgate. * Since nothing is taking the puts out, allow for a big capacity * * @throws InterruptedException @@ -135,7 +135,8 @@ public void testManyThreads() throws InterruptedException { context.put("keep-alive", "1"); context.put("capacity", "5000"); // theoretical maximum of 100 threads * 10 * 5 // because we're just grabbing the whole lot in one commit - // normally a transactionCapacity significantly lower than the channel capacity would be recommended + // normally a transactionCapacity significantly lower than the channel capacity would be + // recommended context.put("transactionCapacity", "5000"); Configurables.configure(channel, context); final ConcurrentHashMap committedPuts = @@ -158,17 +159,17 @@ public void run() { } catch (InterruptedException e1) { Thread.currentThread().interrupt(); } - for(int j = 0; j < 10; j++) { + for (int j = 0; j < 10; j++) { int events = rng.nextInt(5) + 1; Transaction tx = channel.getTransaction(); tx.begin(); - for(int k = 0; k < events; k++) { + for (int k = 0; k < events; k++) { channel.put(EventBuilder.withBody(strtid.getBytes())); } - if(rng.nextBoolean()) { + if (rng.nextBoolean()) { tx.commit(); AtomicInteger tcount = committedPuts.get(strtid); - if(tcount == null) { + if (tcount == null) { committedPuts.put(strtid, new AtomicInteger(events)); } else { tcount.addAndGet(events); @@ -186,7 +187,7 @@ public void run() { startGate.countDown(); endGate.await(); - if(committedPuts.isEmpty()) { + if (committedPuts.isEmpty()) { Assert.fail(); } @@ -194,17 +195,17 @@ public void run() { Transaction tx = channel.getTransaction(); tx.begin(); Event e; - while((e = channel.take()) != null) { + while ((e = channel.take()) != null) { String index = new String(e.getBody()); AtomicInteger remain = committedPuts.get(index); int post = remain.decrementAndGet(); - if(post == 0) { + if (post == 0) { committedPuts.remove(index); } } tx.commit(); tx.close(); - if(!committedPuts.isEmpty()) { + if (!committedPuts.isEmpty()) { Assert.fail(); } } @@ -216,10 +217,12 @@ public void testConcurrentSinksAndSources() throws InterruptedException { context.put("keep-alive", "1"); context.put("capacity", "100"); // theoretical maximum of 100 threads * 10 * 5 // because we're just grabbing the whole lot in one commit - // normally a transactionCapacity significantly lower than the channel capacity would be recommended + // normally a transactionCapacity significantly lower than the channel capacity would be + // recommended context.put("transactionCapacity", "100"); Configurables.configure(channel, context); - final ConcurrentHashMap committedPuts = new ConcurrentHashMap(); + final ConcurrentHashMap committedPuts = + new ConcurrentHashMap(); final ConcurrentHashMap committedTakes = new ConcurrentHashMap(); @@ -228,7 +231,7 @@ public void testConcurrentSinksAndSources() throws InterruptedException { final CountDownLatch endGate = new CountDownLatch(threadCount); // start a sink and source for each - for (int i = 0; i < threadCount/2; i++) { + for (int i = 0; i < threadCount / 2; i++) { Thread t = new Thread() { @Override public void run() { @@ -241,23 +244,23 @@ public void run() { } catch (InterruptedException e1) { Thread.currentThread().interrupt(); } - for(int j = 0; j < 10; j++) { + for (int j = 0; j < 10; j++) { int events = rng.nextInt(5) + 1; Transaction tx = channel.getTransaction(); tx.begin(); - for(int k = 0; k < events; k++) { + for (int k = 0; k < events; k++) { channel.put(EventBuilder.withBody(strtid.getBytes())); } - if(rng.nextBoolean()) { + if (rng.nextBoolean()) { try { tx.commit(); AtomicInteger tcount = committedPuts.get(strtid); - if(tcount == null) { + if (tcount == null) { committedPuts.put(strtid, new AtomicInteger(events)); } else { tcount.addAndGet(events); } - } catch(ChannelException e) { + } catch (ChannelException e) { System.out.print("puts commit failed"); tx.rollback(); } @@ -282,25 +285,25 @@ public void run() { } catch (InterruptedException e1) { Thread.currentThread().interrupt(); } - for(int j = 0; j < 10; j++) { + for (int j = 0; j < 10; j++) { int events = rng.nextInt(5) + 1; Transaction tx = channel.getTransaction(); tx.begin(); Event[] taken = new Event[events]; int k; - for(k = 0; k < events; k++) { + for (k = 0; k < events; k++) { taken[k] = channel.take(); - if(taken[k] == null) break; + if (taken[k] == null) break; } - if(rng.nextBoolean()) { + if (rng.nextBoolean()) { try { tx.commit(); - for(Event e : taken) { - if(e == null) break; + for (Event e : taken) { + if (e == null) break; String index = new String(e.getBody()); - synchronized(takeMapLock) { + synchronized (takeMapLock) { AtomicInteger remain = committedTakes.get(index); - if(remain == null) { + if (remain == null) { committedTakes.put(index, new AtomicInteger(1)); } else { remain.incrementAndGet(); @@ -323,7 +326,7 @@ public void run() { t.start(); } startGate.countDown(); - if(!endGate.await(20, TimeUnit.SECONDS)) { + if (!endGate.await(20, TimeUnit.SECONDS)) { Assert.fail("Not all threads ended succesfully"); } @@ -333,11 +336,11 @@ public void run() { Event e; // first pull out what's left in the channel and remove it from the // committed map - while((e = channel.take()) != null) { + while ((e = channel.take()) != null) { String index = new String(e.getBody()); AtomicInteger remain = committedPuts.get(index); int post = remain.decrementAndGet(); - if(post == 0) { + if (post == 0) { committedPuts.remove(index); } } @@ -345,14 +348,19 @@ public void run() { tx.close(); // now just check the committed puts match the committed takes - for(Entry takes : committedTakes.entrySet()) { + for (Entry takes : committedTakes.entrySet()) { AtomicInteger count = committedPuts.get(takes.getKey()); - if(count == null) + if (count == null) { Assert.fail("Putted data doesn't exist"); - if(count.get() != takes.getValue().get()) - Assert.fail(String.format("Mismatched put and take counts expected %d had %d", count.get(), takes.getValue().get())); + } + if (count.get() != takes.getValue().get()) { + Assert.fail(String.format("Mismatched put and take counts expected %d had %d", + count.get(), takes.getValue().get())); + } committedPuts.remove(takes.getKey()); } - if(!committedPuts.isEmpty()) Assert.fail("Puts still has entries remaining"); + if (!committedPuts.isEmpty()) { + Assert.fail("Puts still has entries remaining"); + } } } diff --git a/flume-ng-core/src/test/java/org/apache/flume/channel/TestMemoryChannelTransaction.java b/flume-ng-core/src/test/java/org/apache/flume/channel/TestMemoryChannelTransaction.java index b8e00d81e3..55b81eeb66 100644 --- a/flume-ng-core/src/test/java/org/apache/flume/channel/TestMemoryChannelTransaction.java +++ b/flume-ng-core/src/test/java/org/apache/flume/channel/TestMemoryChannelTransaction.java @@ -43,7 +43,8 @@ public void setUp() { @Test public void testCommit() throws InterruptedException, EventDeliveryException { - Event event, event2; + Event event; + Event event2; Context context = new Context(); int putCounter = 0; @@ -85,7 +86,8 @@ public void testCommit() throws InterruptedException, EventDeliveryException { public void testRollBack() throws InterruptedException, EventDeliveryException { - Event event, event2; + Event event; + Event event2; Context context = new Context(); int putCounter = 0; @@ -158,7 +160,8 @@ public void testRollBack() throws InterruptedException, public void testReEntTxn() throws InterruptedException, EventDeliveryException { - Event event, event2; + Event event; + Event event2; Context context = new Context(); int putCounter = 0; @@ -199,7 +202,8 @@ public void testReEntTxn() throws InterruptedException, @Test public void testReEntTxnRollBack() throws InterruptedException, EventDeliveryException { - Event event, event2; + Event event; + Event event2; Context context = new Context(); int putCounter = 0; diff --git a/flume-ng-core/src/test/java/org/apache/flume/client/avro/TestReliableSpoolingFileEventReader.java b/flume-ng-core/src/test/java/org/apache/flume/client/avro/TestReliableSpoolingFileEventReader.java index 4e90054889..fdc3ce94b0 100644 --- a/flume-ng-core/src/test/java/org/apache/flume/client/avro/TestReliableSpoolingFileEventReader.java +++ b/flume-ng-core/src/test/java/org/apache/flume/client/avro/TestReliableSpoolingFileEventReader.java @@ -37,14 +37,25 @@ import java.io.File; import java.io.FileFilter; import java.io.IOException; -import java.util.*; +import java.util.ArrayList; +import java.util.Collection; +import java.util.Collections; +import java.util.Comparator; +import java.util.List; +import java.util.Map; import java.util.Map.Entry; -import java.util.concurrent.*; +import java.util.Set; +import java.util.TreeMap; +import java.util.concurrent.Callable; +import java.util.concurrent.ExecutorService; +import java.util.concurrent.Executors; +import java.util.concurrent.Future; +import java.util.concurrent.Semaphore; public class TestReliableSpoolingFileEventReader { - private static final Logger logger = LoggerFactory.getLogger - (TestReliableSpoolingFileEventReader.class); + private static final Logger logger = + LoggerFactory.getLogger(TestReliableSpoolingFileEventReader.class); private static final File WORK_DIR = new File("target/test/work/" + TestReliableSpoolingFileEventReader.class.getSimpleName()); @@ -57,7 +68,7 @@ public void setup() throws IOException, InterruptedException { // write out a few files for (int i = 0; i < 4; i++) { - File fileName = new File(WORK_DIR, "file"+i); + File fileName = new File(WORK_DIR, "file" + i); StringBuilder sb = new StringBuilder(); // write as many lines as the index of the file @@ -102,11 +113,12 @@ private void deleteDir(File dir) { @Test public void testIgnorePattern() throws IOException { - ReliableEventReader reader = new ReliableSpoolingFileEventReader.Builder() - .spoolDirectory(WORK_DIR) - .ignorePattern("^file2$") - .deletePolicy(DeletePolicy.IMMEDIATE.toString()) - .build(); + ReliableEventReader reader = + new ReliableSpoolingFileEventReader.Builder() + .spoolDirectory(WORK_DIR) + .ignorePattern("^file2$") + .deletePolicy(DeletePolicy.IMMEDIATE.toString()) + .build(); List before = listFiles(WORK_DIR); Assert.assertEquals("Expected 5, not: " + before, 5, before.size()); @@ -128,8 +140,9 @@ public void testIgnorePattern() throws IOException { @Test public void testRepeatedCallsWithCommitAlways() throws IOException { - ReliableEventReader reader = new ReliableSpoolingFileEventReader.Builder() - .spoolDirectory(WORK_DIR).build(); + ReliableEventReader reader = + new ReliableSpoolingFileEventReader.Builder().spoolDirectory(WORK_DIR) + .build(); final int expectedLines = 0 + 1 + 2 + 3 + 1; int seenLines = 0; @@ -148,8 +161,10 @@ public void testRepeatedCallsWithCommitOnSuccess() throws IOException { SpoolDirectorySourceConfigurationConstants.DEFAULT_TRACKER_DIR; File trackerDir = new File(WORK_DIR, trackerDirPath); - ReliableEventReader reader = new ReliableSpoolingFileEventReader.Builder() - .spoolDirectory(WORK_DIR).trackerDirPath(trackerDirPath).build(); + ReliableEventReader reader = + new ReliableSpoolingFileEventReader.Builder().spoolDirectory(WORK_DIR) + .trackerDirPath(trackerDirPath) + .build(); final int expectedLines = 0 + 1 + 2 + 3 + 1; int seenLines = 0; @@ -173,10 +188,10 @@ public void testRepeatedCallsWithCommitOnSuccess() throws IOException { @Test public void testFileDeletion() throws IOException { - ReliableEventReader reader = new ReliableSpoolingFileEventReader.Builder() - .spoolDirectory(WORK_DIR) - .deletePolicy(DeletePolicy.IMMEDIATE.name()) - .build(); + ReliableEventReader reader = + new ReliableSpoolingFileEventReader.Builder().spoolDirectory(WORK_DIR) + .deletePolicy(DeletePolicy.IMMEDIATE.name()) + .build(); List before = listFiles(WORK_DIR); Assert.assertEquals("Expected 5, not: " + before, 5, before.size()); @@ -197,29 +212,25 @@ public void testFileDeletion() throws IOException { @Test(expected = NullPointerException.class) public void testNullConsumeOrder() throws IOException { - new ReliableSpoolingFileEventReader.Builder() - .spoolDirectory(WORK_DIR) - .consumeOrder(null) - .build(); + new ReliableSpoolingFileEventReader.Builder().spoolDirectory(WORK_DIR) + .consumeOrder(null) + .build(); } @Test public void testConsumeFileRandomly() throws IOException { - ReliableEventReader reader - = new ReliableSpoolingFileEventReader.Builder() - .spoolDirectory(WORK_DIR) - .consumeOrder(ConsumeOrder.RANDOM) - .build(); + ReliableEventReader reader = + new ReliableSpoolingFileEventReader.Builder().spoolDirectory(WORK_DIR) + .consumeOrder(ConsumeOrder.RANDOM) + .build(); File fileName = new File(WORK_DIR, "new-file"); - FileUtils.write(fileName, - "New file created in the end. Shoud be read randomly.\n"); + FileUtils.write(fileName, "New file created in the end. Shoud be read randomly.\n"); Set actual = Sets.newHashSet(); readEventsForFilesInDir(WORK_DIR, reader, actual); Set expected = Sets.newHashSet(); createExpectedFromFilesInSetup(expected); expected.add(""); - expected.add( - "New file created in the end. Shoud be read randomly."); + expected.add("New file created in the end. Shoud be read randomly."); Assert.assertEquals(expected, actual); } @@ -229,54 +240,46 @@ public void testConsumeFileRandomlyNewFile() throws Exception { if (SystemUtils.IS_OS_WINDOWS) { return; } - final ReliableEventReader reader - = new ReliableSpoolingFileEventReader.Builder() - .spoolDirectory(WORK_DIR) - .consumeOrder(ConsumeOrder.RANDOM) - .build(); + final ReliableEventReader reader = + new ReliableSpoolingFileEventReader.Builder().spoolDirectory(WORK_DIR) + .consumeOrder(ConsumeOrder.RANDOM) + .build(); File fileName = new File(WORK_DIR, "new-file"); - FileUtils.write(fileName, - "New file created in the end. Shoud be read randomly.\n"); + FileUtils.write(fileName, "New file created in the end. Shoud be read randomly.\n"); Set expected = Sets.newHashSet(); int totalFiles = WORK_DIR.listFiles().length; final Set actual = Sets.newHashSet(); ExecutorService executor = Executors.newSingleThreadExecutor(); final Semaphore semaphore1 = new Semaphore(0); final Semaphore semaphore2 = new Semaphore(0); - Future wait = executor.submit( - new Callable() { - @Override - public Void call() throws Exception { - readEventsForFilesInDir(WORK_DIR, reader, actual, semaphore1, semaphore2); - return null; - } + Future wait = executor.submit(new Callable() { + @Override + public Void call() throws Exception { + readEventsForFilesInDir(WORK_DIR, reader, actual, semaphore1, semaphore2); + return null; } - ); + }); semaphore1.acquire(); File finalFile = new File(WORK_DIR, "t-file"); FileUtils.write(finalFile, "Last file"); semaphore2.release(); wait.get(); - int listFilesCount = ((ReliableSpoolingFileEventReader)reader) - .getListFilesCount(); + int listFilesCount = ((ReliableSpoolingFileEventReader)reader).getListFilesCount(); finalFile.delete(); createExpectedFromFilesInSetup(expected); expected.add(""); - expected.add( - "New file created in the end. Shoud be read randomly."); + expected.add("New file created in the end. Shoud be read randomly."); expected.add("Last file"); Assert.assertTrue(listFilesCount < (totalFiles + 2)); Assert.assertEquals(expected, actual); } - @Test public void testConsumeFileOldest() throws IOException, InterruptedException { - ReliableEventReader reader - = new ReliableSpoolingFileEventReader.Builder() - .spoolDirectory(WORK_DIR) - .consumeOrder(ConsumeOrder.OLDEST) - .build(); + ReliableEventReader reader = + new ReliableSpoolingFileEventReader.Builder().spoolDirectory(WORK_DIR) + .consumeOrder(ConsumeOrder.OLDEST) + .build(); File file1 = new File(WORK_DIR, "new-file1"); File file2 = new File(WORK_DIR, "new-file2"); File file3 = new File(WORK_DIR, "new-file3"); @@ -299,13 +302,11 @@ public void testConsumeFileOldest() throws IOException, InterruptedException { } @Test - public void testConsumeFileYoungest() - throws IOException, InterruptedException { - ReliableEventReader reader - = new ReliableSpoolingFileEventReader.Builder() - .spoolDirectory(WORK_DIR) - .consumeOrder(ConsumeOrder.YOUNGEST) - .build(); + public void testConsumeFileYoungest() throws IOException, InterruptedException { + ReliableEventReader reader = + new ReliableSpoolingFileEventReader.Builder().spoolDirectory(WORK_DIR) + .consumeOrder(ConsumeOrder.YOUNGEST) + .build(); File file1 = new File(WORK_DIR, "new-file1"); File file2 = new File(WORK_DIR, "new-file2"); File file3 = new File(WORK_DIR, "new-file3"); @@ -332,12 +333,11 @@ public void testConsumeFileYoungest() @Test public void testConsumeFileOldestWithLexicographicalComparision() - throws IOException, InterruptedException { - ReliableEventReader reader - = new ReliableSpoolingFileEventReader.Builder() - .spoolDirectory(WORK_DIR) - .consumeOrder(ConsumeOrder.OLDEST) - .build(); + throws IOException, InterruptedException { + ReliableEventReader reader = + new ReliableSpoolingFileEventReader.Builder().spoolDirectory(WORK_DIR) + .consumeOrder(ConsumeOrder.OLDEST) + .build(); File file1 = new File(WORK_DIR, "new-file1"); File file2 = new File(WORK_DIR, "new-file2"); File file3 = new File(WORK_DIR, "new-file3"); @@ -362,12 +362,11 @@ public void testConsumeFileOldestWithLexicographicalComparision() @Test public void testConsumeFileYoungestWithLexicographicalComparision() - throws IOException, InterruptedException { - ReliableEventReader reader - = new ReliableSpoolingFileEventReader.Builder() - .spoolDirectory(WORK_DIR) - .consumeOrder(ConsumeOrder.YOUNGEST) - .build(); + throws IOException, InterruptedException { + ReliableEventReader reader = + new ReliableSpoolingFileEventReader.Builder().spoolDirectory(WORK_DIR) + .consumeOrder(ConsumeOrder.YOUNGEST) + .build(); File file1 = new File(WORK_DIR, "new-file1"); File file2 = new File(WORK_DIR, "new-file2"); File file3 = new File(WORK_DIR, "new-file3"); @@ -393,6 +392,7 @@ public void testConsumeFileYoungestWithLexicographicalComparision() @Test public void testLargeNumberOfFilesOLDEST() throws IOException { templateTestForLargeNumberOfFiles(ConsumeOrder.OLDEST, null, 1000); } + @Test public void testLargeNumberOfFilesYOUNGEST() throws IOException { templateTestForLargeNumberOfFiles(ConsumeOrder.YOUNGEST, new Comparator() { @@ -402,6 +402,7 @@ public int compare(Long o1, Long o2) { } }, 1000); } + @Test public void testLargeNumberOfFilesRANDOM() throws IOException { templateTestForLargeNumberOfFiles(ConsumeOrder.RANDOM, null, 1000); } @@ -409,19 +410,21 @@ public int compare(Long o1, Long o2) { @Test public void testZeroByteTrackerFile() throws IOException { String trackerDirPath = - SpoolDirectorySourceConfigurationConstants.DEFAULT_TRACKER_DIR; + SpoolDirectorySourceConfigurationConstants.DEFAULT_TRACKER_DIR; File trackerDir = new File(WORK_DIR, trackerDirPath); - if(!trackerDir.exists()) { + if (!trackerDir.exists()) { trackerDir.mkdir(); } File trackerFile = new File(trackerDir, ReliableSpoolingFileEventReader.metaFileName); - if(trackerFile.exists()) { + if (trackerFile.exists()) { trackerFile.delete(); } trackerFile.createNewFile(); - ReliableEventReader reader = new ReliableSpoolingFileEventReader.Builder() - .spoolDirectory(WORK_DIR).trackerDirPath(trackerDirPath).build(); + ReliableEventReader reader = + new ReliableSpoolingFileEventReader.Builder().spoolDirectory(WORK_DIR) + .trackerDirPath(trackerDirPath) + .build(); final int expectedLines = 1; int seenLines = 0; List events = reader.readEvents(10); @@ -434,18 +437,16 @@ public void testZeroByteTrackerFile() throws IOException { Assert.assertEquals(expectedLines, seenLines); } - private void templateTestForLargeNumberOfFiles(ConsumeOrder order, - Comparator comparator, - int N) throws IOException { + private void templateTestForLargeNumberOfFiles(ConsumeOrder order, Comparator comparator, + int N) throws IOException { File dir = null; try { - dir = new File( - "target/test/work/" + this.getClass().getSimpleName() + - "_large"); + dir = new File("target/test/work/" + this.getClass().getSimpleName() + "_large"); Files.createParentDirs(new File(dir, "dummy")); - ReliableEventReader reader - = new ReliableSpoolingFileEventReader.Builder() - .spoolDirectory(dir).consumeOrder(order).build(); + ReliableEventReader reader = + new ReliableSpoolingFileEventReader.Builder().spoolDirectory(dir) + .consumeOrder(order) + .build(); Map> expected; if (comparator == null) { expected = new TreeMap>(); @@ -476,16 +477,14 @@ private void templateTestForLargeNumberOfFiles(ConsumeOrder order, List events; events = reader.readEvents(10); for (Event e : events) { - if (order == ConsumeOrder.RANDOM) { + if (order == ConsumeOrder.RANDOM) { Assert.assertTrue(expectedList.remove(new String(e.getBody()))); } else { - Assert.assertEquals( - ((ArrayList) expectedList).get(0), - new String(e.getBody())); + Assert.assertEquals(((ArrayList) expectedList).get(0), new String(e.getBody())); ((ArrayList) expectedList).remove(0); } } - reader.commit(); + reader.commit(); } } finally { deleteDir(dir); @@ -493,23 +492,24 @@ private void templateTestForLargeNumberOfFiles(ConsumeOrder order, } private void readEventsForFilesInDir(File dir, ReliableEventReader reader, - Collection actual) throws IOException { + Collection actual) throws IOException { readEventsForFilesInDir(dir, reader, actual, null, null); } /* Read events, one for each file in the given directory. */ - private void readEventsForFilesInDir(File dir, ReliableEventReader reader, - Collection actual, Semaphore semaphore1, Semaphore semaphore2) throws IOException { + private void readEventsForFilesInDir(File dir, ReliableEventReader reader, + Collection actual, Semaphore semaphore1, + Semaphore semaphore2) throws IOException { List events; boolean executed = false; - for (int i=0; i < listFiles(dir).size(); i++) { + for (int i = 0; i < listFiles(dir).size(); i++) { events = reader.readEvents(10); for (Event e : events) { actual.add(new String(e.getBody())); } reader.commit(); try { - if(!executed) { + if (!executed) { executed = true; if (semaphore1 != null) { semaphore1.release(); @@ -533,8 +533,7 @@ private void createExpectedFromFilesInSetup(Collection expected) { } private static List listFiles(File dir) { - List files = Lists.newArrayList(dir.listFiles(new FileFilter - () { + List files = Lists.newArrayList(dir.listFiles(new FileFilter() { @Override public boolean accept(File pathname) { return !pathname.isDirectory(); diff --git a/flume-ng-core/src/test/java/org/apache/flume/formatter/output/TestBucketPath.java b/flume-ng-core/src/test/java/org/apache/flume/formatter/output/TestBucketPath.java index 21b972b21f..b1b828a0b3 100644 --- a/flume-ng-core/src/test/java/org/apache/flume/formatter/output/TestBucketPath.java +++ b/flume-ng-core/src/test/java/org/apache/flume/formatter/output/TestBucketPath.java @@ -18,6 +18,12 @@ package org.apache.flume.formatter.output; +import org.apache.flume.Clock; +import org.joda.time.format.DateTimeFormatter; +import org.joda.time.format.ISODateTimeFormat; +import org.junit.Assert; +import org.junit.Before; +import org.junit.Test; import java.text.SimpleDateFormat; import java.util.Calendar; @@ -26,21 +32,15 @@ import java.util.Map; import java.util.TimeZone; -import org.apache.flume.Clock; -import org.joda.time.format.DateTimeFormatter; -import org.joda.time.format.ISODateTimeFormat; -import org.junit.Assert; -import org.junit.Before; -import org.junit.Test; - import static org.mockito.Mockito.mock; import static org.mockito.Mockito.when; public class TestBucketPath { Calendar cal; Map headers; + @Before - public void setUp(){ + public void setUp() { cal = Calendar.getInstance(); cal.set(2012, 5, 23, 13, 46, 33); cal.set(Calendar.MILLISECOND, 234); @@ -49,7 +49,7 @@ public void setUp(){ } @Test - public void testDateFormatCache(){ + public void testDateFormatCache() { TimeZone utcTimeZone = TimeZone.getTimeZone("UTC"); String test = "%c"; BucketPath.escapeString( @@ -60,7 +60,7 @@ public void testDateFormatCache(){ SimpleDateFormat format = new SimpleDateFormat("EEE MMM d HH:mm:ss yyyy"); Date d = new Date(cal.getTimeInMillis()); String expectedString = format.format(d); - System.out.println("Expected String: "+ expectedString); + System.out.println("Expected String: " + expectedString); Assert.assertEquals(expectedString, escapedString); } @@ -76,7 +76,7 @@ public void testDateFormatHours() { SimpleDateFormat format = new SimpleDateFormat("EEE MMM d HH:mm:ss yyyy"); Date d = new Date(cal2.getTimeInMillis()); String expectedString = format.format(d); - System.out.println("Expected String: "+ expectedString); + System.out.println("Expected String: " + expectedString); Assert.assertEquals(expectedString, escapedString); } @@ -89,8 +89,8 @@ public void testDateFormatMinutes() { Calendar cal2 = Calendar.getInstance(); cal2.set(2012, 5, 23, 13, 45, 0); cal2.set(Calendar.MILLISECOND, 0); - String expectedString = String.valueOf(cal2.getTimeInMillis()/1000); - System.out.println("Expected String: "+ expectedString); + String expectedString = String.valueOf(cal2.getTimeInMillis() / 1000); + System.out.println("Expected String: " + expectedString); Assert.assertEquals(expectedString, escapedString); } @@ -103,13 +103,13 @@ public void testDateFormatSeconds() { Calendar cal2 = Calendar.getInstance(); cal2.set(2012, 5, 23, 13, 46, 30); cal2.set(Calendar.MILLISECOND, 0); - String expectedString = String.valueOf(cal2.getTimeInMillis()/1000); - System.out.println("Expected String: "+ expectedString); + String expectedString = String.valueOf(cal2.getTimeInMillis() / 1000); + System.out.println("Expected String: " + expectedString); Assert.assertEquals(expectedString, escapedString); } @Test - public void testNoRounding(){ + public void testNoRounding() { String test = "%c"; String escapedString = BucketPath.escapeString( test, headers, false, Calendar.HOUR_OF_DAY, 12); @@ -117,19 +117,19 @@ public void testNoRounding(){ SimpleDateFormat format = new SimpleDateFormat("EEE MMM d HH:mm:ss yyyy"); Date d = new Date(cal.getTimeInMillis()); String expectedString = format.format(d); - System.out.println("Expected String: "+ expectedString); + System.out.println("Expected String: " + expectedString); Assert.assertEquals(expectedString, escapedString); } @Test - public void testNoPadding(){ + public void testNoPadding() { Calendar calender; Map calender_timestamp; calender = Calendar.getInstance(); - + //Check single digit dates - calender.set(2014, (5-1), 3, 13, 46, 33); + calender.set(2014, (5 - 1), 3, 13, 46, 33); calender_timestamp = new HashMap(); calender_timestamp.put("timestamp", String.valueOf(calender.getTimeInMillis())); SimpleDateFormat format = new SimpleDateFormat("M-d"); @@ -141,19 +141,19 @@ public void testNoPadding(){ String expectedString = format.format(d); //Check two digit dates - calender.set(2014, (11-1), 13, 13, 46, 33); + calender.set(2014, (11 - 1), 13, 13, 46, 33); calender_timestamp.put("timestamp", String.valueOf(calender.getTimeInMillis())); - escapedString += " " + BucketPath.escapeString( + escapedString += " " + BucketPath.escapeString( test, calender_timestamp, false, Calendar.HOUR_OF_DAY, 12); System.out.println("Escaped String: " + escapedString); d = new Date(calender.getTimeInMillis()); - expectedString += " " + format.format(d); - System.out.println("Expected String: "+ expectedString); + expectedString += " " + format.format(d); + System.out.println("Expected String: " + expectedString); Assert.assertEquals(expectedString, escapedString); } @Test - public void testDateFormatTimeZone(){ + public void testDateFormatTimeZone() { TimeZone utcTimeZone = TimeZone.getTimeZone("UTC"); String test = "%c"; String escapedString = BucketPath.escapeString( @@ -163,7 +163,7 @@ public void testDateFormatTimeZone(){ format.setTimeZone(utcTimeZone); Date d = new Date(cal.getTimeInMillis()); String expectedString = format.format(d); - System.out.println("Expected String: "+ expectedString); + System.out.println("Expected String: " + expectedString); Assert.assertEquals(expectedString, escapedString); } diff --git a/flume-ng-core/src/test/java/org/apache/flume/instrumentation/TestMonitoredCounterGroup.java b/flume-ng-core/src/test/java/org/apache/flume/instrumentation/TestMonitoredCounterGroup.java index b1f637f1d2..7db535e2c6 100644 --- a/flume-ng-core/src/test/java/org/apache/flume/instrumentation/TestMonitoredCounterGroup.java +++ b/flume-ng-core/src/test/java/org/apache/flume/instrumentation/TestMonitoredCounterGroup.java @@ -18,22 +18,15 @@ */ package org.apache.flume.instrumentation; -import java.lang.management.ManagementFactory; -import java.util.Random; - -import javax.management.AttributeNotFoundException; -import javax.management.InstanceNotFoundException; -import javax.management.MBeanException; -import javax.management.MBeanInfo; -import javax.management.MBeanServer; -import javax.management.ObjectName; -import javax.management.ReflectionException; - import junit.framework.Assert; - import org.junit.Before; import org.junit.Test; +import javax.management.MBeanServer; +import javax.management.ObjectName; +import java.lang.management.ManagementFactory; +import java.util.Random; + public class TestMonitoredCounterGroup { private static final int MAX_BOUNDS = 1000; @@ -61,7 +54,6 @@ public class TestMonitoredCounterGroup { private static final String SRC_ATTR_APPEND_BATCH_ACCEPTED_COUNT = "AppendBatchAcceptedCount"; - private static final String CH_ATTR_CHANNEL_SIZE = "ChannelSize"; private static final String CH_ATTR_EVENT_PUT_ATTEMPT = "EventPutAttemptCount"; @@ -122,28 +114,28 @@ public void testSinkCounter() throws Exception { int eventDrainAttempt = random.nextInt(MAX_BOUNDS); int eventDrainSuccess = random.nextInt(MAX_BOUNDS); - for (int i = 0; i>>() { - }.getType(); + Type mapType = new TypeToken>>() {}.getType(); Gson gson = new Gson(); @Test @@ -99,7 +97,7 @@ public void testJSON() throws Exception { private void testWithPort(int port) throws Exception { MonitorService srv = new HTTPMetricsServer(); Context context = new Context(); - if(port > 1024){ + if (port > 1024) { context.put(HTTPMetricsServer.CONFIG_PORT, String.valueOf(port)); } else { port = HTTPMetricsServer.DEFAULT_PORT; @@ -139,8 +137,7 @@ public void testOptions() throws Exception { doTestForbiddenMethods(4432,"OPTIONS"); } - public void doTestForbiddenMethods(int port, String method) - throws Exception { + public void doTestForbiddenMethods(int port, String method) throws Exception { MonitorService srv = new HTTPMetricsServer(); Context context = new Context(); if (port > 1024) { @@ -154,8 +151,7 @@ public void doTestForbiddenMethods(int port, String method) URL url = new URL("http://0.0.0.0:" + String.valueOf(port) + "/metrics"); HttpURLConnection conn = (HttpURLConnection) url.openConnection(); conn.setRequestMethod(method); - Assert.assertEquals(HttpServletResponse.SC_FORBIDDEN, - conn.getResponseCode()); + Assert.assertEquals(HttpServletResponse.SC_FORBIDDEN, conn.getResponseCode()); srv.stop(); } } diff --git a/flume-ng-core/src/test/java/org/apache/flume/instrumentation/kafka/KafkaSourceCounterTest.java b/flume-ng-core/src/test/java/org/apache/flume/instrumentation/kafka/KafkaSourceCounterTest.java index 4a712656dc..6d64c53fc4 100644 --- a/flume-ng-core/src/test/java/org/apache/flume/instrumentation/kafka/KafkaSourceCounterTest.java +++ b/flume-ng-core/src/test/java/org/apache/flume/instrumentation/kafka/KafkaSourceCounterTest.java @@ -6,9 +6,9 @@ * to you under the Apache License, Version 2.0 (the * "License"); you may not use this file except in compliance * with the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * + *

    + * http://www.apache.org/licenses/LICENSE-2.0 + *

    * Unless required by applicable law or agreed to in writing, software * distributed under the License is distributed on an "AS IS" BASIS, * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. @@ -23,41 +23,41 @@ public class KafkaSourceCounterTest { - KafkaSourceCounter counter; - - @Before - public void setUp() throws Exception { - counter = new KafkaSourceCounter("test"); - } - - @Test - public void testAddToKafkaEventGetTimer() throws Exception { - Assert.assertEquals(1L, counter.addToKafkaEventGetTimer(1L)); - } - - @Test - public void testAddToKafkaCommitTimer() throws Exception { - Assert.assertEquals(1L, counter.addToKafkaCommitTimer(1L)); - } - - @Test - public void testIncrementKafkaEmptyCount() throws Exception { - Assert.assertEquals(1L, counter.incrementKafkaEmptyCount()); - } - - @Test - public void testGetKafkaCommitTimer() throws Exception { - Assert.assertEquals(0, counter.getKafkaCommitTimer()); - } - - @Test - public void testGetKafkaEventGetTimer() throws Exception { - Assert.assertEquals(0, counter.getKafkaEventGetTimer()); - } - - @Test - public void testGetKafkaEmptyCount() throws Exception { - Assert.assertEquals(0, counter.getKafkaEmptyCount()); - } + KafkaSourceCounter counter; + + @Before + public void setUp() throws Exception { + counter = new KafkaSourceCounter("test"); + } + + @Test + public void testAddToKafkaEventGetTimer() throws Exception { + Assert.assertEquals(1L, counter.addToKafkaEventGetTimer(1L)); + } + + @Test + public void testAddToKafkaCommitTimer() throws Exception { + Assert.assertEquals(1L, counter.addToKafkaCommitTimer(1L)); + } + + @Test + public void testIncrementKafkaEmptyCount() throws Exception { + Assert.assertEquals(1L, counter.incrementKafkaEmptyCount()); + } + + @Test + public void testGetKafkaCommitTimer() throws Exception { + Assert.assertEquals(0, counter.getKafkaCommitTimer()); + } + + @Test + public void testGetKafkaEventGetTimer() throws Exception { + Assert.assertEquals(0, counter.getKafkaEventGetTimer()); + } + + @Test + public void testGetKafkaEmptyCount() throws Exception { + Assert.assertEquals(0, counter.getKafkaEmptyCount()); + } } \ No newline at end of file diff --git a/flume-ng-core/src/test/java/org/apache/flume/interceptor/TestRegexExtractorInterceptorMillisSerializer.java b/flume-ng-core/src/test/java/org/apache/flume/interceptor/TestRegexExtractorInterceptorMillisSerializer.java index ac46131d05..dd42079a3a 100644 --- a/flume-ng-core/src/test/java/org/apache/flume/interceptor/TestRegexExtractorInterceptorMillisSerializer.java +++ b/flume-ng-core/src/test/java/org/apache/flume/interceptor/TestRegexExtractorInterceptorMillisSerializer.java @@ -18,7 +18,6 @@ package org.apache.flume.interceptor; import junit.framework.Assert; - import org.apache.flume.Context; import org.joda.time.format.DateTimeFormat; import org.joda.time.format.DateTimeFormatter; @@ -29,7 +28,8 @@ public class TestRegexExtractorInterceptorMillisSerializer { @Test public void shouldRequirePatternInConfiguration() { try { - RegexExtractorInterceptorMillisSerializer fixture = new RegexExtractorInterceptorMillisSerializer(); + RegexExtractorInterceptorMillisSerializer fixture = + new RegexExtractorInterceptorMillisSerializer(); fixture.configure(new Context()); Assert.fail(); } catch (IllegalArgumentException ex) { @@ -40,7 +40,8 @@ public void shouldRequirePatternInConfiguration() { @Test public void shouldRequireValidPatternInConfiguration() { try { - RegexExtractorInterceptorMillisSerializer fixture = new RegexExtractorInterceptorMillisSerializer(); + RegexExtractorInterceptorMillisSerializer fixture = + new RegexExtractorInterceptorMillisSerializer(); Context context = new Context(); context.put("pattern", "ABCDEFG"); fixture.configure(context); @@ -52,7 +53,8 @@ public void shouldRequireValidPatternInConfiguration() { @Test public void shouldReturnMillisFromPattern() { - RegexExtractorInterceptorMillisSerializer fixture = new RegexExtractorInterceptorMillisSerializer(); + RegexExtractorInterceptorMillisSerializer fixture = + new RegexExtractorInterceptorMillisSerializer(); Context context = new Context(); String pattern = "yyyy-MM-dd HH:mm:ss"; context.put("pattern", pattern); diff --git a/flume-ng-core/src/test/java/org/apache/flume/interceptor/TestRegexExtractorInterceptorPassThroughSerializer.java b/flume-ng-core/src/test/java/org/apache/flume/interceptor/TestRegexExtractorInterceptorPassThroughSerializer.java index 569c274923..33003e63cf 100644 --- a/flume-ng-core/src/test/java/org/apache/flume/interceptor/TestRegexExtractorInterceptorPassThroughSerializer.java +++ b/flume-ng-core/src/test/java/org/apache/flume/interceptor/TestRegexExtractorInterceptorPassThroughSerializer.java @@ -26,7 +26,8 @@ public class TestRegexExtractorInterceptorPassThroughSerializer { @Test public void shouldReturnSameValue() { - RegexExtractorInterceptorPassThroughSerializer fixture = new RegexExtractorInterceptorPassThroughSerializer(); + RegexExtractorInterceptorPassThroughSerializer fixture = + new RegexExtractorInterceptorPassThroughSerializer(); fixture.configure(new Context()); String input = "testing (1,2,3,4)"; Assert.assertEquals(input, fixture.serialize(input)); diff --git a/flume-ng-core/src/test/java/org/apache/flume/interceptor/TestSearchAndReplaceInterceptor.java b/flume-ng-core/src/test/java/org/apache/flume/interceptor/TestSearchAndReplaceInterceptor.java index 2ab15f566a..616b86bf53 100644 --- a/flume-ng-core/src/test/java/org/apache/flume/interceptor/TestSearchAndReplaceInterceptor.java +++ b/flume-ng-core/src/test/java/org/apache/flume/interceptor/TestSearchAndReplaceInterceptor.java @@ -35,7 +35,7 @@ public class TestSearchAndReplaceInterceptor { private void testSearchReplace(Context context, String input, String output) throws Exception { - Interceptor.Builder builder = InterceptorBuilderFactory.newInstance( + Interceptor.Builder builder = InterceptorBuilderFactory.newInstance( InterceptorType.SEARCH_REPLACE.toString()); builder.configure(context); Interceptor interceptor = builder.build(); diff --git a/flume-ng-core/src/test/java/org/apache/flume/serialization/SyslogAvroEventSerializer.java b/flume-ng-core/src/test/java/org/apache/flume/serialization/SyslogAvroEventSerializer.java index 896eced96b..05af3b1e7b 100644 --- a/flume-ng-core/src/test/java/org/apache/flume/serialization/SyslogAvroEventSerializer.java +++ b/flume-ng-core/src/test/java/org/apache/flume/serialization/SyslogAvroEventSerializer.java @@ -231,20 +231,45 @@ public static class SyslogEvent { private String hostname = ""; private String message = ""; - public void setFacility(int f) { facility = f; } - public int getFacility() { return facility; } + public void setFacility(int f) { + facility = f; + } + + public int getFacility() { + return facility; + } + + public void setSeverity(int s) { + severity = s; + } - public void setSeverity(int s) { severity = s; } - public int getSeverity() { return severity; } + public int getSeverity() { + return severity; + } - public void setTimestamp(long t) { timestamp = t; } - public long getTimestamp() { return timestamp; } + public void setTimestamp(long t) { + timestamp = t; + } - public void setHostname(String h) { hostname = h; } - public String getHostname() { return hostname; } + public long getTimestamp() { + return timestamp; + } - public void setMessage(String m) { message = m; } - public String getMessage() { return message; } + public void setHostname(String h) { + hostname = h; + } + + public String getHostname() { + return hostname; + } + + public void setMessage(String m) { + message = m; + } + + public String getMessage() { + return message; + } @Override public String toString() { diff --git a/flume-ng-core/src/test/java/org/apache/flume/serialization/TestAvroEventDeserializer.java b/flume-ng-core/src/test/java/org/apache/flume/serialization/TestAvroEventDeserializer.java index 6f9ddc25b6..b95433fd11 100644 --- a/flume-ng-core/src/test/java/org/apache/flume/serialization/TestAvroEventDeserializer.java +++ b/flume-ng-core/src/test/java/org/apache/flume/serialization/TestAvroEventDeserializer.java @@ -49,6 +49,7 @@ public class TestAvroEventDeserializer { LoggerFactory.getLogger(TestAvroEventDeserializer.class); private static final Schema schema; + static { schema = Schema.createRecord("MyRecord", "", "org.apache.flume", false); Schema.Field field = new Schema.Field("foo", diff --git a/flume-ng-core/src/test/java/org/apache/flume/serialization/TestDurablePositionTracker.java b/flume-ng-core/src/test/java/org/apache/flume/serialization/TestDurablePositionTracker.java index e52affb387..0c76cc93e5 100644 --- a/flume-ng-core/src/test/java/org/apache/flume/serialization/TestDurablePositionTracker.java +++ b/flume-ng-core/src/test/java/org/apache/flume/serialization/TestDurablePositionTracker.java @@ -31,8 +31,7 @@ public class TestDurablePositionTracker { - private static final Logger logger = LoggerFactory.getLogger - (TestDurablePositionTracker.class); + private static final Logger logger = LoggerFactory.getLogger(TestDurablePositionTracker.class); @Test public void testBasicTracker() throws IOException { diff --git a/flume-ng-core/src/test/java/org/apache/flume/serialization/TestFlumeEventAvroEventSerializer.java b/flume-ng-core/src/test/java/org/apache/flume/serialization/TestFlumeEventAvroEventSerializer.java index 3860b5e4dd..ded3b13531 100644 --- a/flume-ng-core/src/test/java/org/apache/flume/serialization/TestFlumeEventAvroEventSerializer.java +++ b/flume-ng-core/src/test/java/org/apache/flume/serialization/TestFlumeEventAvroEventSerializer.java @@ -18,14 +18,7 @@ */ package org.apache.flume.serialization; -import java.io.File; -import java.io.FileNotFoundException; -import java.io.FileOutputStream; -import java.io.IOException; -import java.io.OutputStream; -import java.nio.ByteBuffer; -import java.nio.charset.CharsetDecoder; - +import com.google.common.base.Charsets; import org.apache.avro.file.DataFileReader; import org.apache.avro.generic.GenericData; import org.apache.avro.generic.GenericDatumReader; @@ -38,7 +31,13 @@ import org.junit.Assume; import org.junit.Test; -import com.google.common.base.Charsets; +import java.io.File; +import java.io.FileNotFoundException; +import java.io.FileOutputStream; +import java.io.IOException; +import java.io.OutputStream; +import java.nio.ByteBuffer; +import java.nio.charset.CharsetDecoder; public class TestFlumeEventAvroEventSerializer { @@ -77,17 +76,16 @@ public void testAvroSerializerSnappyCompression() throws FileNotFoundException, IOException { // Snappy currently broken on Mac in OpenJDK 7 per FLUME-2012 Assume.assumeTrue(!"Mac OS X".equals(System.getProperty("os.name")) || - !System.getProperty("java.version").startsWith("1.7.")); + !System.getProperty("java.version").startsWith("1.7.")); createAvroFile(TESTFILE, "snappy"); validateAvroFile(TESTFILE); FileUtils.forceDelete(TESTFILE); } - public void createAvroFile(File file, String codec) - throws FileNotFoundException, IOException { + public void createAvroFile(File file, String codec) throws FileNotFoundException, IOException { - if(file.exists()){ + if (file.exists()) { FileUtils.forceDelete(file); } diff --git a/flume-ng-core/src/test/java/org/apache/flume/serialization/TestResettableFileInputStream.java b/flume-ng-core/src/test/java/org/apache/flume/serialization/TestResettableFileInputStream.java index 631bdfe387..9f336eb9f5 100644 --- a/flume-ng-core/src/test/java/org/apache/flume/serialization/TestResettableFileInputStream.java +++ b/flume-ng-core/src/test/java/org/apache/flume/serialization/TestResettableFileInputStream.java @@ -27,8 +27,6 @@ import org.slf4j.Logger; import org.slf4j.LoggerFactory; -import static org.junit.Assert.*; - import java.io.BufferedOutputStream; import java.io.ByteArrayOutputStream; import java.io.File; @@ -39,15 +37,21 @@ import java.nio.charset.MalformedInputException; import java.util.List; +import static org.junit.Assert.assertEquals; +import static org.junit.Assert.assertNull; +import static org.junit.Assert.assertTrue; +import static org.junit.Assert.fail; + public class TestResettableFileInputStream { private static final boolean CLEANUP = true; private static final File WORK_DIR = new File("target/test/work").getAbsoluteFile(); - private static final Logger logger = LoggerFactory.getLogger - (TestResettableFileInputStream.class); + private static final Logger logger = + LoggerFactory.getLogger(TestResettableFileInputStream.class); - private File file, meta; + private File file; + private File meta; @Before public void setup() throws Exception { @@ -156,7 +160,8 @@ public void testUtf8SurrogatePairRead() throws IOException { public void testUtf16BOMAndSurrogatePairRead() throws IOException { ByteArrayOutputStream out = new ByteArrayOutputStream(); generateUtf16SurrogatePairSequence(out); - // buffer now contains 1 BOM and 2 chars (1 surrogate pair) and 6 bytes total (including 2-byte BOM) + // buffer now contains 1 BOM and 2 chars (1 surrogate pair) and 6 bytes total + // (including 2-byte BOM) Files.write(out.toByteArray(), file); ResettableInputStream in = initInputStream(8, Charsets.UTF_16, DecodeErrorPolicy.FAIL); String result = readLine(in, 2); @@ -176,7 +181,8 @@ public void testShiftJisSurrogateCharRead() throws IOException { generateShiftJis2ByteSequence(out); // buffer now contains 8 chars and 10 bytes total Files.write(out.toByteArray(), file); - ResettableInputStream in = initInputStream(8, Charset.forName("Shift_JIS"), DecodeErrorPolicy.FAIL); + ResettableInputStream in = initInputStream(8, Charset.forName("Shift_JIS"), + DecodeErrorPolicy.FAIL); String result = readLine(in, 8); assertEquals("1234567\u4E9C\n", result); } @@ -215,7 +221,7 @@ public void testUtf8DecodeErrorHandlingReplace() throws IOException { String javaVersionStr = System.getProperty("java.version"); double javaVersion = Double.parseDouble(javaVersionStr.substring(0, 3)); - if(javaVersion < 1.8) { + if (javaVersion < 1.8) { assertTrue(preJdk8ExpectedStr.replaceAll("X", "\ufffd").equals(sb.toString())); } else { assertTrue(expectedStr.replaceAll("X", "\ufffd").equals(sb.toString())); @@ -508,8 +514,8 @@ private ResettableInputStream initInputStream(DecodeErrorPolicy policy) return initInputStream(2048, Charsets.UTF_8, policy); } - private ResettableInputStream initInputStream(int bufferSize, Charset charset, DecodeErrorPolicy policy) - throws IOException { + private ResettableInputStream initInputStream(int bufferSize, Charset charset, + DecodeErrorPolicy policy) throws IOException { PositionTracker tracker = new DurablePositionTracker(meta, file.getPath()); ResettableInputStream in = new ResettableFileInputStream(file, tracker, bufferSize, charset, policy); diff --git a/flume-ng-core/src/test/java/org/apache/flume/serialization/TestSyslogAvroEventSerializer.java b/flume-ng-core/src/test/java/org/apache/flume/serialization/TestSyslogAvroEventSerializer.java index 7bd342a02a..ba9f4ab734 100644 --- a/flume-ng-core/src/test/java/org/apache/flume/serialization/TestSyslogAvroEventSerializer.java +++ b/flume-ng-core/src/test/java/org/apache/flume/serialization/TestSyslogAvroEventSerializer.java @@ -75,7 +75,7 @@ private static List generateSyslogEvents() { public void test() throws FileNotFoundException, IOException { // Snappy currently broken on Mac in OpenJDK 7 per FLUME-2012 Assume.assumeTrue(!"Mac OS X".equals(System.getProperty("os.name")) || - !System.getProperty("java.version").startsWith("1.7.")); + !System.getProperty("java.version").startsWith("1.7.")); //Schema schema = new Schema.Parser().parse(schemaFile); diff --git a/flume-ng-core/src/test/java/org/apache/flume/sink/TestAvroSink.java b/flume-ng-core/src/test/java/org/apache/flume/sink/TestAvroSink.java index 757a53626d..0f3f9eca71 100644 --- a/flume-ng-core/src/test/java/org/apache/flume/sink/TestAvroSink.java +++ b/flume-ng-core/src/test/java/org/apache/flume/sink/TestAvroSink.java @@ -19,28 +19,10 @@ package org.apache.flume.sink; -import java.io.IOException; -import java.net.InetSocketAddress; -import java.nio.ByteBuffer; -import java.nio.charset.Charset; -import java.util.ArrayList; -import java.util.HashMap; -import java.util.List; -import java.util.concurrent.atomic.AtomicLong; - import com.google.common.base.Charsets; -import java.io.FileInputStream; -import java.security.KeyStore; -import java.security.Security; -import java.util.concurrent.Executors; -import javax.net.ssl.KeyManagerFactory; -import javax.net.ssl.SSLContext; -import javax.net.ssl.SSLEngine; import org.apache.avro.AvroRemoteException; import org.apache.avro.ipc.NettyServer; -import org.apache.avro.ipc.NettyTransceiver; import org.apache.avro.ipc.Server; -import org.apache.avro.ipc.specific.SpecificRequestor; import org.apache.avro.ipc.specific.SpecificResponder; import org.apache.flume.Channel; import org.apache.flume.ChannelSelector; @@ -49,8 +31,8 @@ import org.apache.flume.EventDeliveryException; import org.apache.flume.Sink; import org.apache.flume.Transaction; -import org.apache.flume.channel.ChannelProcessor; import org.apache.flume.api.RpcClient; +import org.apache.flume.channel.ChannelProcessor; import org.apache.flume.channel.MemoryChannel; import org.apache.flume.channel.ReplicatingChannelSelector; import org.apache.flume.conf.Configurables; @@ -61,19 +43,30 @@ import org.apache.flume.source.avro.AvroFlumeEvent; import org.apache.flume.source.avro.AvroSourceProtocol; import org.apache.flume.source.avro.Status; -import org.jboss.netty.channel.ChannelException; import org.jboss.netty.channel.ChannelPipeline; import org.jboss.netty.channel.ChannelPipelineFactory; import org.jboss.netty.channel.Channels; import org.jboss.netty.channel.socket.nio.NioServerSocketChannelFactory; import org.jboss.netty.handler.ssl.SslHandler; import org.junit.Assert; -import org.junit.Before; -import org.junit.Ignore; import org.junit.Test; import org.slf4j.Logger; import org.slf4j.LoggerFactory; +import javax.net.ssl.KeyManagerFactory; +import javax.net.ssl.SSLContext; +import javax.net.ssl.SSLEngine; +import java.io.FileInputStream; +import java.io.IOException; +import java.net.InetSocketAddress; +import java.nio.charset.Charset; +import java.security.KeyStore; +import java.security.Security; +import java.util.ArrayList; +import java.util.List; +import java.util.concurrent.Executors; +import java.util.concurrent.atomic.AtomicLong; + public class TestAvroSink { private static final Logger logger = LoggerFactory @@ -90,7 +83,9 @@ public void setUp() { } public void setUp(String compressionType, int compressionLevel) { - if (sink != null) { throw new RuntimeException("double setup");} + if (sink != null) { + throw new RuntimeException("double setup"); + } sink = new AvroSink(); channel = new MemoryChannel(); @@ -607,8 +602,9 @@ public void testSslSinkWithNonSslServer() throws InterruptedException, } @Test - public void testSslSinkWithNonTrustedCert() throws InterruptedException, - EventDeliveryException, InstantiationException, IllegalAccessException { + public void testSslSinkWithNonTrustedCert() + throws InterruptedException, EventDeliveryException, InstantiationException, + IllegalAccessException { setUp(); Event event = EventBuilder.withBody("test event 1", Charsets.UTF_8); Server server = createSslServer(new MockAvroServer()); @@ -662,37 +658,38 @@ public void testSslSinkWithNonTrustedCert() throws InterruptedException, } @Test - public void testRequestWithNoCompression() throws InterruptedException, IOException, EventDeliveryException { - + public void testRequestWithNoCompression() + throws InterruptedException, IOException, EventDeliveryException { doRequest(false, false, 6); } @Test - public void testRequestWithCompressionOnClientAndServerOnLevel0() throws InterruptedException, IOException, EventDeliveryException { - + public void testRequestWithCompressionOnClientAndServerOnLevel0() + throws InterruptedException, IOException, EventDeliveryException { doRequest(true, true, 0); } @Test - public void testRequestWithCompressionOnClientAndServerOnLevel1() throws InterruptedException, IOException, EventDeliveryException { - + public void testRequestWithCompressionOnClientAndServerOnLevel1() + throws InterruptedException, IOException, EventDeliveryException { doRequest(true, true, 1); } @Test - public void testRequestWithCompressionOnClientAndServerOnLevel6() throws InterruptedException, IOException, EventDeliveryException { - + public void testRequestWithCompressionOnClientAndServerOnLevel6() + throws InterruptedException, IOException, EventDeliveryException { doRequest(true, true, 6); } @Test - public void testRequestWithCompressionOnClientAndServerOnLevel9() throws InterruptedException, IOException, EventDeliveryException { - + public void testRequestWithCompressionOnClientAndServerOnLevel9() + throws InterruptedException, IOException, EventDeliveryException { doRequest(true, true, 9); } - private void doRequest(boolean serverEnableCompression, boolean clientEnableCompression, int compressionLevel) throws InterruptedException, IOException, EventDeliveryException { - + private void doRequest(boolean serverEnableCompression, boolean clientEnableCompression, + int compressionLevel) + throws InterruptedException, IOException, EventDeliveryException { if (clientEnableCompression) { setUp("deflate", compressionLevel); } else { @@ -732,15 +729,12 @@ private void doRequest(boolean serverEnableCompression, boolean clientEnableComp source.start(); - Assert - .assertTrue("Reached start or error", LifecycleController.waitForOneOf( - source, LifecycleState.START_OR_ERROR)); - Assert.assertEquals("Server is started", LifecycleState.START, - source.getLifecycleState()); - + Assert.assertTrue("Reached start or error", + LifecycleController.waitForOneOf(source, LifecycleState.START_OR_ERROR)); + Assert.assertEquals("Server is started", + LifecycleState.START, source.getLifecycleState()); - Event event = EventBuilder.withBody("Hello avro", - Charset.forName("UTF8")); + Event event = EventBuilder.withBody("Hello avro", Charset.forName("UTF8")); sink.start(); @@ -858,7 +852,8 @@ private class SSLChannelPipelineFactory public SSLChannelPipelineFactory() { } - public SSLChannelPipelineFactory(String keystore, String keystorePassword, String keystoreType) { + public SSLChannelPipelineFactory(String keystore, String keystorePassword, + String keystoreType) { this.keystore = keystore; this.keystorePassword = keystorePassword; this.keystoreType = keystoreType; diff --git a/flume-ng-core/src/test/java/org/apache/flume/sink/TestDefaultSinkFactory.java b/flume-ng-core/src/test/java/org/apache/flume/sink/TestDefaultSinkFactory.java index 835f5413e4..cf6cbbcc7f 100644 --- a/flume-ng-core/src/test/java/org/apache/flume/sink/TestDefaultSinkFactory.java +++ b/flume-ng-core/src/test/java/org/apache/flume/sink/TestDefaultSinkFactory.java @@ -37,7 +37,6 @@ public void setUp() { @Test public void testDuplicateCreate() { - Sink avroSink1 = sinkFactory.create("avroSink1", "avro"); Sink avroSink2 = sinkFactory.create("avroSink2", "avro"); @@ -55,7 +54,7 @@ public void testDuplicateCreate() { } private void verifySinkCreation(String name, String type, Class typeClass) - throws Exception { + throws Exception { Sink sink = sinkFactory.create(name, type); Assert.assertNotNull(sink); Assert.assertTrue(typeClass.isInstance(sink)); diff --git a/flume-ng-core/src/test/java/org/apache/flume/sink/TestFailoverSinkProcessor.java b/flume-ng-core/src/test/java/org/apache/flume/sink/TestFailoverSinkProcessor.java index 3358cf4f1f..888205672b 100644 --- a/flume-ng-core/src/test/java/org/apache/flume/sink/TestFailoverSinkProcessor.java +++ b/flume-ng-core/src/test/java/org/apache/flume/sink/TestFailoverSinkProcessor.java @@ -94,7 +94,7 @@ public synchronized void setRemaining(int remaining) { @Override public Status process() throws EventDeliveryException { - synchronized(this) { + synchronized (this) { if (remaining <= 0) { throw new EventDeliveryException("can't consume more"); } @@ -107,7 +107,7 @@ public Status process() throws EventDeliveryException { tx.close(); if (e != null) { - synchronized(this) { + synchronized (this) { remaining--; } written++; @@ -167,7 +167,7 @@ public void testFailover() throws InterruptedException { Assert.assertEquals(LifecycleState.START, s1.getLifecycleState()); Assert.assertEquals(LifecycleState.START, s2.getLifecycleState()); Assert.assertEquals(LifecycleState.START, s3.getLifecycleState()); - for(int i = 0; i < 15; i++) { + for (int i = 0; i < 15; i++) { Transaction tx = ch.getTransaction(); tx.begin(); ch.put(EventBuilder.withBody("test".getBytes())); @@ -178,7 +178,7 @@ public void testFailover() throws InterruptedException { Assert.assertEquals(new Integer(10), s1.getWritten()); Assert.assertEquals(new Integer(5), s2.getWritten()); - for(int i = 0; i < 50; i++) { + for (int i = 0; i < 50; i++) { Transaction tx = ch.getTransaction(); tx.begin(); ch.put(EventBuilder.withBody("test".getBytes())); @@ -195,7 +195,7 @@ public void testFailover() throws InterruptedException { // get us past the retry time for the failed sink Thread.sleep(5000); - for(int i = 0; i < 100; i++) { + for (int i = 0; i < 100; i++) { Transaction tx = ch.getTransaction(); tx.begin(); ch.put(EventBuilder.withBody("test".getBytes())); diff --git a/flume-ng-core/src/test/java/org/apache/flume/sink/TestLoadBalancingSinkProcessor.java b/flume-ng-core/src/test/java/org/apache/flume/sink/TestLoadBalancingSinkProcessor.java index 7d95655a0b..011d2d1f53 100644 --- a/flume-ng-core/src/test/java/org/apache/flume/sink/TestLoadBalancingSinkProcessor.java +++ b/flume-ng-core/src/test/java/org/apache/flume/sink/TestLoadBalancingSinkProcessor.java @@ -17,16 +17,7 @@ */ package org.apache.flume.sink; -import java.util.ArrayList; -import java.util.Collections; -import java.util.HashMap; -import java.util.HashSet; -import java.util.List; -import java.util.Map; -import java.util.Set; - import junit.framework.Assert; - import org.apache.flume.Channel; import org.apache.flume.ChannelException; import org.apache.flume.Context; @@ -38,6 +29,14 @@ import org.apache.flume.channel.AbstractChannel; import org.junit.Test; +import java.util.ArrayList; +import java.util.Collections; +import java.util.HashMap; +import java.util.HashSet; +import java.util.List; +import java.util.Map; +import java.util.Set; + public class TestLoadBalancingSinkProcessor { private Context getContext(String selectorType, boolean backoff) { @@ -62,8 +61,7 @@ private LoadBalancingSinkProcessor getProcessor( return getProcessor(sinks, getContext(selectorType, backoff)); } - private LoadBalancingSinkProcessor getProcessor(List sinks, Context ctx) - { + private LoadBalancingSinkProcessor getProcessor(List sinks, Context ctx) { LoadBalancingSinkProcessor lbsp = new LoadBalancingSinkProcessor(); lbsp.setSinks(sinks); lbsp.configure(ctx); @@ -77,7 +75,7 @@ public void testDefaultConfiguration() throws Exception { // If no selector is specified, the round-robin selector should be used Channel ch = new MockChannel(); int n = 100; - int numEvents = 3*n; + int numEvents = 3 * n; for (int i = 0; i < numEvents; i++) { ch.put(new MockEvent("test" + i)); } @@ -185,7 +183,7 @@ public void testRandomBackoff() throws Exception { // TODO: there is a remote possibility that s0 or s2 // never get hit by the random assignment // and thus not backoffed, causing the test to fail - for(int i=0; i < 50; i++) { + for (int i = 0; i < 50; i++) { // a well behaved runner would always check the return. lbsp.process(); } @@ -214,7 +212,7 @@ public void testRandomBackoff() throws Exception { public void testRandomPersistentFailure() throws Exception { Channel ch = new MockChannel(); int n = 100; - int numEvents = 3*n; + int numEvents = 3 * n; for (int i = 0; i < numEvents; i++) { ch.put(new MockEvent("test" + i)); } @@ -244,7 +242,7 @@ public void testRandomPersistentFailure() throws Exception { } Assert.assertTrue(s2.getEvents().size() == 0); - Assert.assertTrue(s1.getEvents().size() + s3.getEvents().size() == 3*n); + Assert.assertTrue(s1.getEvents().size() + s3.getEvents().size() == 3 * n); } @Test @@ -325,8 +323,6 @@ public void testRandomNoFailure() throws Exception { Assert.assertTrue("Miraculous distribution", sizeSet.size() > 1); } - - @Test public void testRoundRobinOneActiveSink() throws Exception { Channel ch = new MockChannel(); @@ -373,7 +369,7 @@ public void testRoundRobinOneActiveSink() throws Exception { public void testRoundRobinPersistentFailure() throws Exception { Channel ch = new MockChannel(); int n = 100; - int numEvents = 3*n; + int numEvents = 3 * n; for (int i = 0; i < numEvents; i++) { ch.put(new MockEvent("test" + i)); } @@ -404,7 +400,7 @@ public void testRoundRobinPersistentFailure() throws Exception { Assert.assertTrue(s1.getEvents().size() == n); Assert.assertTrue(s2.getEvents().size() == 0); - Assert.assertTrue(s3.getEvents().size() == 2*n); + Assert.assertTrue(s3.getEvents().size() == 2 * n); } // test that even if the sink recovers immediately that it is kept out of commission briefly @@ -413,7 +409,7 @@ public void testRoundRobinPersistentFailure() throws Exception { public void testRoundRobinBackoffInitialFailure() throws EventDeliveryException { Channel ch = new MockChannel(); int n = 100; - int numEvents = 3*n; + int numEvents = 3 * n; for (int i = 0; i < numEvents; i++) { ch.put(new MockEvent("test" + i)); } @@ -424,7 +420,7 @@ public void testRoundRobinBackoffInitialFailure() throws EventDeliveryException MockSink s2 = new MockSink(2); s2.setChannel(ch); - MockSink s3 = new MockSink(3); + MockSink s3 = new MockSink(3); s3.setChannel(ch); List sinks = new ArrayList(); @@ -449,14 +445,15 @@ public void testRoundRobinBackoffInitialFailure() throws EventDeliveryException Assert.assertEquals((3 * n) / 2, s1.getEvents().size()); Assert.assertEquals(1, s2.getEvents().size()); - Assert.assertEquals((3 * n) /2 - 1, s3.getEvents().size()); + Assert.assertEquals((3 * n) / 2 - 1, s3.getEvents().size()); } @Test - public void testRoundRobinBackoffIncreasingBackoffs() throws EventDeliveryException, InterruptedException { + public void testRoundRobinBackoffIncreasingBackoffs() + throws EventDeliveryException, InterruptedException { Channel ch = new MockChannel(); int n = 100; - int numEvents = 3*n; + int numEvents = 3 * n; for (int i = 0; i < numEvents; i++) { ch.put(new MockEvent("test" + i)); } @@ -468,7 +465,7 @@ public void testRoundRobinBackoffIncreasingBackoffs() throws EventDeliveryExcept s2.setChannel(ch); s2.setFail(true); - MockSink s3 = new MockSink(3); + MockSink s3 = new MockSink(3); s3.setChannel(ch); List sinks = new ArrayList(); @@ -508,10 +505,11 @@ public void testRoundRobinBackoffIncreasingBackoffs() throws EventDeliveryExcept } @Test - public void testRoundRobinBackoffFailureRecovery() throws EventDeliveryException, InterruptedException { + public void testRoundRobinBackoffFailureRecovery() + throws EventDeliveryException, InterruptedException { Channel ch = new MockChannel(); int n = 100; - int numEvents = 3*n; + int numEvents = 3 * n; for (int i = 0; i < numEvents; i++) { ch.put(new MockEvent("test" + i)); } @@ -523,7 +521,7 @@ public void testRoundRobinBackoffFailureRecovery() throws EventDeliveryException s2.setChannel(ch); s2.setFail(true); - MockSink s3 = new MockSink(3); + MockSink s3 = new MockSink(3); s3.setChannel(ch); List sinks = new ArrayList(); @@ -548,13 +546,12 @@ public void testRoundRobinBackoffFailureRecovery() throws EventDeliveryException Assert.assertEquals(n, s3.getEvents().size()); } - @Test public void testRoundRobinNoFailure() throws Exception { Channel ch = new MockChannel(); int n = 100; - int numEvents = 3*n; + int numEvents = 3 * n; for (int i = 0; i < numEvents; i++) { ch.put(new MockEvent("test" + i)); } @@ -656,8 +653,9 @@ public Status process() throws EventDeliveryException { throw new EventDeliveryException("failed"); } Event e = this.getChannel().take(); - if (e == null) + if (e == null) { return Status.BACKOFF; + } events.add(e); return Status.READY; diff --git a/flume-ng-core/src/test/java/org/apache/flume/sink/TestRollingFileSink.java b/flume-ng-core/src/test/java/org/apache/flume/sink/TestRollingFileSink.java index bf4ed1ff58..e4d2e2e26e 100644 --- a/flume-ng-core/src/test/java/org/apache/flume/sink/TestRollingFileSink.java +++ b/flume-ng-core/src/test/java/org/apache/flume/sink/TestRollingFileSink.java @@ -19,11 +19,6 @@ package org.apache.flume.sink; -import java.io.BufferedReader; -import java.io.File; -import java.io.FileReader; -import java.io.IOException; - import org.apache.flume.Channel; import org.apache.flume.Context; import org.apache.flume.Event; @@ -39,6 +34,11 @@ import org.slf4j.Logger; import org.slf4j.LoggerFactory; +import java.io.BufferedReader; +import java.io.File; +import java.io.FileReader; +import java.io.IOException; + public class TestRollingFileSink { private static final Logger logger = LoggerFactory @@ -108,8 +108,8 @@ public void testAppend() throws InterruptedException, LifecycleException, sink.stop(); for (String file : sink.getDirectory().list()) { - BufferedReader reader = new BufferedReader(new FileReader(new File( - sink.getDirectory(), file))); + BufferedReader reader = + new BufferedReader(new FileReader(new File(sink.getDirectory(), file))); String lastLine = null; String currentLine = null; @@ -157,8 +157,8 @@ public void testAppend2() throws InterruptedException, LifecycleException, sink.stop(); for (String file : sink.getDirectory().list()) { - BufferedReader reader = new BufferedReader(new FileReader(new File( - sink.getDirectory(), file))); + BufferedReader reader = + new BufferedReader(new FileReader(new File(sink.getDirectory(), file))); String lastLine = null; String currentLine = null; @@ -174,7 +174,8 @@ public void testAppend2() throws InterruptedException, LifecycleException, } @Test - public void testAppend3() throws InterruptedException, LifecycleException, EventDeliveryException, IOException { + public void testAppend3() + throws InterruptedException, LifecycleException, EventDeliveryException, IOException { File tmpDir = new File("target/tmpLog"); tmpDir.mkdirs(); cleanDirectory(tmpDir); @@ -208,7 +209,8 @@ public void testAppend3() throws InterruptedException, LifecycleException, Event sink.stop(); for (String file : sink.getDirectory().list()) { - BufferedReader reader = new BufferedReader(new FileReader(new File(sink.getDirectory(), file))); + BufferedReader reader = + new BufferedReader(new FileReader(new File(sink.getDirectory(), file))); String lastLine = null; String currentLine = null; @@ -223,7 +225,8 @@ public void testAppend3() throws InterruptedException, LifecycleException, Event } @Test - public void testRollTime() throws InterruptedException, LifecycleException, EventDeliveryException, IOException { + public void testRollTime() + throws InterruptedException, LifecycleException, EventDeliveryException, IOException { File tmpDir = new File("target/tempLog"); tmpDir.mkdirs(); cleanDirectory(tmpDir); @@ -258,7 +261,8 @@ public void testRollTime() throws InterruptedException, LifecycleException, Even sink.stop(); for (String file : sink.getDirectory().list()) { - BufferedReader reader = new BufferedReader(new FileReader(new File(sink.getDirectory(), file))); + BufferedReader reader = + new BufferedReader(new FileReader(new File(sink.getDirectory(), file))); String lastLine = null; String currentLine = null; diff --git a/flume-ng-core/src/test/java/org/apache/flume/sink/TestThriftSink.java b/flume-ng-core/src/test/java/org/apache/flume/sink/TestThriftSink.java index 1beec76e85..22dcf98d4d 100644 --- a/flume-ng-core/src/test/java/org/apache/flume/sink/TestThriftSink.java +++ b/flume-ng-core/src/test/java/org/apache/flume/sink/TestThriftSink.java @@ -161,8 +161,7 @@ public void testTimeout() throws Exception { @Test public void testFailedConnect() throws Exception { - Event event = EventBuilder.withBody("test event 1", - Charset.forName("UTF8")); + Event event = EventBuilder.withBody("test event 1", Charset.forName("UTF8")); sink.start(); @@ -185,7 +184,7 @@ public void testFailedConnect() throws Exception { threwException = true; } Assert.assertTrue("Must throw EventDeliveryException if disconnected", - threwException); + threwException); } src = new ThriftTestingSource(ThriftTestingSource.HandlerType.OK.name(), @@ -283,7 +282,8 @@ public void testSslSinkWithNonSslServer() throws Exception { Assert.assertTrue(LifecycleController.waitForOneOf(sink, LifecycleState.STOP_OR_ERROR, 5000)); if (failed) { - Assert.fail("SSL-enabled sink successfully connected to a non-SSL-enabled server, that's wrong."); + Assert.fail("SSL-enabled sink successfully connected to a non-SSL-enabled server, " + + "that's wrong."); } } @@ -329,7 +329,8 @@ public void testSslSinkWithNonTrustedCert() throws Exception { Assert.assertTrue(LifecycleController.waitForOneOf(sink, LifecycleState.STOP_OR_ERROR, 5000)); if (failed) { - Assert.fail("SSL-enabled sink successfully connected to a server with an untrusted certificate when it should have failed"); + Assert.fail("SSL-enabled sink successfully connected to a server with an " + + "untrusted certificate when it should have failed"); } } } diff --git a/flume-ng-core/src/test/java/org/apache/flume/source/TestAbstractPollableSource.java b/flume-ng-core/src/test/java/org/apache/flume/source/TestAbstractPollableSource.java index d385abe828..0902db9cd1 100644 --- a/flume-ng-core/src/test/java/org/apache/flume/source/TestAbstractPollableSource.java +++ b/flume-ng-core/src/test/java/org/apache/flume/source/TestAbstractPollableSource.java @@ -90,10 +90,12 @@ protected void doStop() throws FlumeException { Context context = new Context(inputConfigs); source.configure(context); - Assert.assertEquals("BackOffSleepIncrement should equal 42 but it equals " + source.getBackOffSleepIncrement(), - 42l, source.getBackOffSleepIncrement()); - Assert.assertEquals("BackOffSleepIncrement should equal 42 but it equals " + source.getMaxBackOffSleepInterval(), - 4242l, source.getMaxBackOffSleepInterval()); + Assert.assertEquals("BackOffSleepIncrement should equal 42 but it equals " + + source.getBackOffSleepIncrement(), + 42L, source.getBackOffSleepIncrement()); + Assert.assertEquals("BackOffSleepIncrement should equal 42 but it equals " + + source.getMaxBackOffSleepInterval(), + 4242L, source.getMaxBackOffSleepInterval()); } @Test @@ -119,14 +121,16 @@ protected void doStop() throws FlumeException { HashMap inputConfigs = new HashMap(); Assert.assertEquals("BackOffSleepIncrement should equal " + - PollableSourceConstants.DEFAULT_BACKOFF_SLEEP_INCREMENT + - " but it equals " + source.getBackOffSleepIncrement(), - PollableSourceConstants.DEFAULT_BACKOFF_SLEEP_INCREMENT, source.getBackOffSleepIncrement()); + PollableSourceConstants.DEFAULT_BACKOFF_SLEEP_INCREMENT + + " but it equals " + source.getBackOffSleepIncrement(), + PollableSourceConstants.DEFAULT_BACKOFF_SLEEP_INCREMENT, + source.getBackOffSleepIncrement()); Assert.assertEquals("BackOffSleepIncrement should equal " + - PollableSourceConstants.DEFAULT_MAX_BACKOFF_SLEEP + - " but it equals " + source.getMaxBackOffSleepInterval(), - PollableSourceConstants.DEFAULT_MAX_BACKOFF_SLEEP, source.getMaxBackOffSleepInterval()); + PollableSourceConstants.DEFAULT_MAX_BACKOFF_SLEEP + + " but it equals " + source.getMaxBackOffSleepInterval(), + PollableSourceConstants.DEFAULT_MAX_BACKOFF_SLEEP, + source.getMaxBackOffSleepInterval()); } } diff --git a/flume-ng-core/src/test/java/org/apache/flume/source/TestAvroSource.java b/flume-ng-core/src/test/java/org/apache/flume/source/TestAvroSource.java index c75d098141..d73e5adc81 100644 --- a/flume-ng-core/src/test/java/org/apache/flume/source/TestAvroSource.java +++ b/flume-ng-core/src/test/java/org/apache/flume/source/TestAvroSource.java @@ -135,42 +135,47 @@ public void testRequestWithNoCompression() throws InterruptedException, IOExcept } @Test - public void testRequestWithCompressionOnClientAndServerOnLevel0() throws InterruptedException, IOException { + public void testRequestWithCompressionOnClientAndServerOnLevel0() + throws InterruptedException, IOException { doRequest(true, true, 0); } @Test - public void testRequestWithCompressionOnClientAndServerOnLevel1() throws InterruptedException, IOException { + public void testRequestWithCompressionOnClientAndServerOnLevel1() + throws InterruptedException, IOException { doRequest(true, true, 1); } @Test - public void testRequestWithCompressionOnClientAndServerOnLevel6() throws InterruptedException, IOException { + public void testRequestWithCompressionOnClientAndServerOnLevel6() + throws InterruptedException, IOException { doRequest(true, true, 6); } @Test - public void testRequestWithCompressionOnClientAndServerOnLevel9() throws InterruptedException, IOException { + public void testRequestWithCompressionOnClientAndServerOnLevel9() + throws InterruptedException, IOException { doRequest(true, true, 9); } - @Test(expected=org.apache.avro.AvroRemoteException.class) + @Test(expected = org.apache.avro.AvroRemoteException.class) public void testRequestWithCompressionOnServerOnly() throws InterruptedException, IOException { //This will fail because both client and server need compression on doRequest(true, false, 6); } - @Test(expected=org.apache.avro.AvroRemoteException.class) + @Test(expected = org.apache.avro.AvroRemoteException.class) public void testRequestWithCompressionOnClientOnly() throws InterruptedException, IOException { //This will fail because both client and server need compression on doRequest(false, true, 6); } - private void doRequest(boolean serverEnableCompression, boolean clientEnableCompression, int compressionLevel) throws InterruptedException, IOException { + private void doRequest(boolean serverEnableCompression, boolean clientEnableCompression, + int compressionLevel) throws InterruptedException, IOException { boolean bound = false; for (int i = 0; i < 100 && !bound; i++) { @@ -428,8 +433,7 @@ public void testInvalidIpFilter() throws InterruptedException, IOException { false, false); try { doIpFilterTest(localhost, null, false, false); - Assert.fail( - "The null ipFilterRules config should have thrown an exception."); + Assert.fail("The null ipFilterRules config should have thrown an exception."); } catch (FlumeException e) { //Do nothing } @@ -502,15 +506,15 @@ public void doIpFilterTest(InetAddress dest, String ruleDefinition, try { if (testWithSSL) { nettyTransceiver = new NettyTransceiver( - new InetSocketAddress (dest, selectedPort), - new SSLChannelFactory()); + new InetSocketAddress(dest, selectedPort), + new SSLChannelFactory()); client = SpecificRequestor.getClient( - AvroSourceProtocol.class, nettyTransceiver); + AvroSourceProtocol.class, nettyTransceiver); } else { nettyTransceiver = new NettyTransceiver( - new InetSocketAddress (dest, selectedPort)); + new InetSocketAddress(dest, selectedPort)); client = SpecificRequestor.getClient( - AvroSourceProtocol.class, nettyTransceiver); + AvroSourceProtocol.class, nettyTransceiver); } AvroFlumeEvent avroEvent = new AvroFlumeEvent(); @@ -523,7 +527,7 @@ public void doIpFilterTest(InetAddress dest, String ruleDefinition, Assert.assertEquals(Status.OK, status); } catch (IOException e) { Assert.assertTrue("Should have been allowed: " + ruleDefinition, - !eventShouldBeAllowed); + !eventShouldBeAllowed); return; } finally { if (nettyTransceiver != null) { diff --git a/flume-ng-core/src/test/java/org/apache/flume/source/TestExecSource.java b/flume-ng-core/src/test/java/org/apache/flume/source/TestExecSource.java index afa93bfcf0..6204d134a5 100644 --- a/flume-ng-core/src/test/java/org/apache/flume/source/TestExecSource.java +++ b/flume-ng-core/src/test/java/org/apache/flume/source/TestExecSource.java @@ -19,20 +19,8 @@ package org.apache.flume.source; - -import static org.junit.Assert.*; - -import java.io.*; -import java.lang.management.ManagementFactory; -import java.nio.charset.Charset; -import java.util.*; -import java.util.regex.Pattern; - -import javax.management.Attribute; -import javax.management.AttributeList; -import javax.management.MBeanServer; -import javax.management.ObjectName; - +import com.google.common.base.Charsets; +import com.google.common.collect.Lists; import org.apache.commons.io.FileUtils; import org.apache.commons.lang.RandomStringUtils; import org.apache.commons.lang.SystemUtils; @@ -47,22 +35,37 @@ import org.apache.flume.channel.ReplicatingChannelSelector; import org.apache.flume.conf.Configurables; import org.apache.flume.lifecycle.LifecycleException; -import org.junit.*; +import org.junit.After; +import org.junit.Assert; +import org.junit.Before; +import org.junit.Test; -import com.google.common.base.Charsets; -import com.google.common.collect.Lists; -import com.google.common.io.Files; +import javax.management.Attribute; +import javax.management.AttributeList; +import javax.management.MBeanServer; +import javax.management.ObjectName; +import java.io.BufferedReader; +import java.io.File; +import java.io.FileOutputStream; +import java.io.FileReader; +import java.io.IOException; +import java.io.InputStreamReader; +import java.lang.management.ManagementFactory; +import java.nio.charset.Charset; +import java.util.List; +import java.util.regex.Pattern; + +import static org.junit.Assert.assertEquals; +import static org.junit.Assert.assertNotNull; +import static org.junit.Assert.assertTrue; public class TestExecSource { private AbstractSource source; private Channel channel = new MemoryChannel(); - private Context context = new Context(); - private ChannelSelector rcs = new ReplicatingChannelSelector(); - @Before public void setUp() { context.put("keep-alive", "1"); @@ -82,19 +85,19 @@ public void tearDown() { // Remove the MBean registered for Monitoring ObjectName objName = null; try { - objName = new ObjectName("org.apache.flume.source" - + ":type=" + source.getName()); + objName = new ObjectName("org.apache.flume.source" + + ":type=" + source.getName()); - ManagementFactory.getPlatformMBeanServer().unregisterMBean(objName); + ManagementFactory.getPlatformMBeanServer().unregisterMBean(objName); } catch (Exception ex) { System.out.println("Failed to unregister the monitored counter: " - + objName + ex.getMessage()); + + objName + ex.getMessage()); } } @Test public void testProcess() throws InterruptedException, LifecycleException, - EventDeliveryException, IOException { + EventDeliveryException, IOException { // Generates a random files for input\output File inputFile = File.createTempFile("input", null); @@ -104,16 +107,15 @@ public void testProcess() throws InterruptedException, LifecycleException, // Generates input file with a random data set (10 lines, 200 characters each) FileOutputStream outputStream1 = new FileOutputStream(inputFile); - for (int i=0; i<10; i++) { - outputStream1.write( - RandomStringUtils.randomAlphanumeric(200).getBytes()); - outputStream1.write('\n'); + for (int i = 0; i < 10; i++) { + outputStream1.write(RandomStringUtils.randomAlphanumeric(200).getBytes()); + outputStream1.write('\n'); } outputStream1.close(); String command = SystemUtils.IS_OS_WINDOWS ? - String.format("cmd /c type %s", inputFile.getAbsolutePath()) : - String.format("cat %s", inputFile.getAbsolutePath()); + String.format("cmd /c type %s", inputFile.getAbsolutePath()) : + String.format("cat %s", inputFile.getAbsolutePath()); context.put("command", command); context.put("keep-alive", "1"); context.put("capacity", "1000"); @@ -139,155 +141,150 @@ public void testProcess() throws InterruptedException, LifecycleException, transaction.close(); Assert.assertEquals(FileUtils.checksumCRC32(inputFile), - FileUtils.checksumCRC32(ouputFile)); + FileUtils.checksumCRC32(ouputFile)); } @Test public void testShellCommandSimple() throws InterruptedException, LifecycleException, - EventDeliveryException, IOException { + EventDeliveryException, IOException { if (SystemUtils.IS_OS_WINDOWS) { runTestShellCmdHelper("powershell -ExecutionPolicy Unrestricted -command", - "1..5", new String[]{"1", "2", "3", "4", "5"}); + "1..5", new String[] { "1", "2", "3", "4", "5" }); } else { runTestShellCmdHelper("/bin/bash -c", "seq 5", - new String[]{"1", "2", "3", "4", "5"}); + new String[] { "1", "2", "3", "4", "5" }); } } @Test - public void testShellCommandBackTicks() - throws InterruptedException, LifecycleException, EventDeliveryException, - IOException { + public void testShellCommandBackTicks() throws InterruptedException, LifecycleException, + EventDeliveryException, IOException { // command with backticks if (SystemUtils.IS_OS_WINDOWS) { - runTestShellCmdHelper( - "powershell -ExecutionPolicy Unrestricted -command", "$(1..5)", - new String[]{"1", "2", "3", "4", "5"}); + runTestShellCmdHelper("powershell -ExecutionPolicy Unrestricted -command", "$(1..5)", + new String[] { "1", "2", "3", "4", "5" }); } else { runTestShellCmdHelper("/bin/bash -c", "echo `seq 5`", - new String[]{"1 2 3 4 5"}); + new String[] { "1 2 3 4 5" }); runTestShellCmdHelper("/bin/bash -c", "echo $(seq 5)", - new String[]{"1 2 3 4 5"}); + new String[] { "1 2 3 4 5" }); } } @Test public void testShellCommandComplex() - throws InterruptedException, LifecycleException, EventDeliveryException, - IOException { + throws InterruptedException, LifecycleException, EventDeliveryException, IOException { // command with wildcards & pipes String[] expected = {"1234", "abcd", "ijk", "xyz", "zzz"}; // pipes if (SystemUtils.IS_OS_WINDOWS) { - runTestShellCmdHelper( - "powershell -ExecutionPolicy Unrestricted -command", - "'zzz','1234','xyz','abcd','ijk' | sort", expected); + runTestShellCmdHelper("powershell -ExecutionPolicy Unrestricted -command", + "'zzz','1234','xyz','abcd','ijk' | sort", expected); } else { runTestShellCmdHelper("/bin/bash -c", - "echo zzz 1234 xyz abcd ijk | xargs -n1 echo | sort -f", expected); + "echo zzz 1234 xyz abcd ijk | xargs -n1 echo | sort -f", expected); } } @Test - public void testShellCommandScript() - throws InterruptedException, LifecycleException, EventDeliveryException, - IOException { + public void testShellCommandScript() throws InterruptedException, LifecycleException, + EventDeliveryException, IOException { // mini script if (SystemUtils.IS_OS_WINDOWS) { runTestShellCmdHelper("powershell -ExecutionPolicy Unrestricted -command", - "foreach ($i in 1..5) { $i }", new String[]{"1", "2", "3", "4", "5"}); + "foreach ($i in 1..5) { $i }", + new String[] { "1", "2", "3", "4", "5" }); // shell arithmetic runTestShellCmdHelper("powershell -ExecutionPolicy Unrestricted -command", - "if(2+2 -gt 3) { 'good' } else { 'not good' } ", new String[]{"good"}); + "if(2+2 -gt 3) { 'good' } else { 'not good' } ", + new String[] { "good" }); } else { - runTestShellCmdHelper("/bin/bash -c", "for i in {1..5}; do echo $i;done" - , new String[]{"1", "2", "3", "4", "5"}); + runTestShellCmdHelper("/bin/bash -c", "for i in {1..5}; do echo $i;done", + new String[] { "1", "2", "3", "4", "5" }); // shell arithmetic - runTestShellCmdHelper("/bin/bash -c", "if ((2+2>3)); " + - "then echo good; else echo not good; fi", new String[]{"good"}); + runTestShellCmdHelper("/bin/bash -c", + "if ((2+2>3)); " + "then echo good; else echo not good; fi", + new String[] { "good" }); } } @Test public void testShellCommandEmbeddingAndEscaping() - throws InterruptedException, LifecycleException, EventDeliveryException, - IOException { + throws InterruptedException, LifecycleException, EventDeliveryException, IOException { // mini script String fileName = SystemUtils.IS_OS_WINDOWS ? - "src\\test\\resources\\test_command.ps1" : - "src/test/resources/test_command.txt"; + "src\\test\\resources\\test_command.ps1" : + "src/test/resources/test_command.txt"; BufferedReader reader = new BufferedReader(new FileReader(fileName)); - try { - String shell = SystemUtils.IS_OS_WINDOWS ? - "powershell -ExecutionPolicy Unrestricted -command" : - "/bin/bash -c"; - String command1 = reader.readLine(); - Assert.assertNotNull(command1); - String[] output1 = new String[] {"'1'", "\"2\"", "\\3", "\\4"}; - runTestShellCmdHelper( shell, command1 , output1); - String command2 = reader.readLine(); - Assert.assertNotNull(command2); - String[] output2 = new String[]{"1","2","3","4","5" }; - runTestShellCmdHelper(shell, command2 , output2); - String command3 = reader.readLine(); - Assert.assertNotNull(command3); - String[] output3 = new String[]{"2","3","4","5","6" }; - runTestShellCmdHelper(shell, command3 , output3); - } finally { - reader.close(); - } + try { + String shell = SystemUtils.IS_OS_WINDOWS ? + "powershell -ExecutionPolicy Unrestricted -command" : + "/bin/bash -c"; + String command1 = reader.readLine(); + Assert.assertNotNull(command1); + String[] output1 = new String[] { "'1'", "\"2\"", "\\3", "\\4" }; + runTestShellCmdHelper(shell, command1, output1); + String command2 = reader.readLine(); + Assert.assertNotNull(command2); + String[] output2 = new String[] { "1", "2", "3", "4", "5" }; + runTestShellCmdHelper(shell, command2, output2); + String command3 = reader.readLine(); + Assert.assertNotNull(command3); + String[] output3 = new String[] { "2", "3", "4", "5", "6" }; + runTestShellCmdHelper(shell, command3, output3); + } finally { + reader.close(); } + } @Test - public void testMonitoredCounterGroup() throws InterruptedException, LifecycleException, - EventDeliveryException, IOException { + public void testMonitoredCounterGroup() + throws InterruptedException, LifecycleException, EventDeliveryException, IOException { // mini script if (SystemUtils.IS_OS_WINDOWS) { runTestShellCmdHelper("powershell -ExecutionPolicy Unrestricted -command", - "foreach ($i in 1..5) { $i }" - , new String[]{"1", "2", "3", "4", "5"}); + "foreach ($i in 1..5) { $i }", + new String[] { "1", "2", "3", "4", "5" }); } else { - runTestShellCmdHelper("/bin/bash -c", "for i in {1..5}; do echo $i;done" - , new String[]{"1", "2", "3", "4", "5"}); + runTestShellCmdHelper("/bin/bash -c", "for i in {1..5}; do echo $i;done", + new String[] { "1", "2", "3", "4", "5" }); } ObjectName objName = null; try { - objName = new ObjectName("org.apache.flume.source" - + ":type=" + source.getName()); - - MBeanServer mbeanServer = ManagementFactory.getPlatformMBeanServer(); - String strAtts[] = {"Type", "EventReceivedCount", "EventAcceptedCount"}; - AttributeList attrList = mbeanServer.getAttributes(objName, strAtts); - - Assert.assertNotNull(attrList.get(0)); - Assert.assertEquals("Expected Value: Type", "Type", - ((Attribute) attrList.get(0)).getName()); - Assert.assertEquals("Expected Value: SOURCE", "SOURCE", - ((Attribute) attrList.get(0)).getValue()); - - Assert.assertNotNull(attrList.get(1)); - Assert.assertEquals("Expected Value: EventReceivedCount", "EventReceivedCount", - ((Attribute) attrList.get(1)).getName()); - Assert.assertEquals("Expected Value: 5", "5", - ((Attribute) attrList.get(1)).getValue().toString()); - - Assert.assertNotNull(attrList.get(2)); - Assert.assertEquals("Expected Value: EventAcceptedCount", "EventAcceptedCount", - ((Attribute) attrList.get(2)).getName()); - Assert.assertEquals("Expected Value: 5", "5", - ((Attribute) attrList.get(2)).getValue().toString()); + objName = new ObjectName("org.apache.flume.source" + ":type=" + source.getName()); + + MBeanServer mbeanServer = ManagementFactory.getPlatformMBeanServer(); + String[] strAtts = { "Type", "EventReceivedCount", "EventAcceptedCount" }; + AttributeList attrList = mbeanServer.getAttributes(objName, strAtts); + + Assert.assertNotNull(attrList.get(0)); + Assert.assertEquals("Expected Value: Type", "Type", + ((Attribute) attrList.get(0)).getName()); + Assert.assertEquals("Expected Value: SOURCE", "SOURCE", + ((Attribute) attrList.get(0)).getValue()); + + Assert.assertNotNull(attrList.get(1)); + Assert.assertEquals("Expected Value: EventReceivedCount", "EventReceivedCount", + ((Attribute) attrList.get(1)).getName()); + Assert.assertEquals("Expected Value: 5", "5", + ((Attribute) attrList.get(1)).getValue().toString()); + + Assert.assertNotNull(attrList.get(2)); + Assert.assertEquals("Expected Value: EventAcceptedCount", "EventAcceptedCount", + ((Attribute) attrList.get(2)).getName()); + Assert.assertEquals("Expected Value: 5", "5", + ((Attribute) attrList.get(2)).getValue().toString()); } catch (Exception ex) { - System.out.println("Unable to retreive the monitored counter: " - + objName + ex.getMessage()); + System.out.println("Unable to retreive the monitored counter: " + objName + ex.getMessage()); } } @Test - public void testBatchTimeout() throws InterruptedException, LifecycleException, - EventDeliveryException, IOException { + public void testBatchTimeout() + throws InterruptedException, LifecycleException, EventDeliveryException, IOException { String filePath = "/tmp/flume-execsource." + Thread.currentThread().getId(); String eventBody = "TestMessage"; @@ -296,12 +293,12 @@ public void testBatchTimeout() throws InterruptedException, LifecycleException, context.put(ExecSourceConfigurationConstants.CONFIG_BATCH_SIZE, "50000"); context.put(ExecSourceConfigurationConstants.CONFIG_BATCH_TIME_OUT, "750"); context.put("shell", SystemUtils.IS_OS_WINDOWS ? - "powershell -ExecutionPolicy Unrestricted -command" : - "/bin/bash -c"); + "powershell -ExecutionPolicy Unrestricted -command" : + "/bin/bash -c"); context.put("command", SystemUtils.IS_OS_WINDOWS ? - "Get-Content " + filePath + - " | Select-Object -Last 10" : - ("tail -f " + filePath)); + "Get-Content " + filePath + + " | Select-Object -Last 10" : + ("tail -f " + filePath)); Configurables.configure(source, context); source.start(); @@ -310,15 +307,15 @@ public void testBatchTimeout() throws InterruptedException, LifecycleException, transaction.begin(); for (int lineNumber = 0; lineNumber < 3; lineNumber++) { - outputStream.write((eventBody).getBytes()); - outputStream.write(String.valueOf(lineNumber).getBytes()); - outputStream.write('\n'); - outputStream.flush(); + outputStream.write((eventBody).getBytes()); + outputStream.write(String.valueOf(lineNumber).getBytes()); + outputStream.write('\n'); + outputStream.flush(); } outputStream.close(); Thread.sleep(1500); - for(int i = 0; i < 3; i++) { + for (int i = 0; i < 3; i++) { Event event = channel.take(); assertNotNull(event); assertNotNull(event.getBody()); @@ -332,45 +329,37 @@ public void testBatchTimeout() throws InterruptedException, LifecycleException, FileUtils.forceDelete(file); } - private void runTestShellCmdHelper(String shell, String command, String[] expectedOutput) - throws InterruptedException, LifecycleException, EventDeliveryException, IOException { - context.put("shell", shell); - context.put("command", command); - Configurables.configure(source, context); - source.start(); - // Some commands might take longer to complete, specially on Windows - // or on slow environments (e.g. Travis CI). - Thread.sleep(2500); - Transaction transaction = channel.getTransaction(); - transaction.begin(); - try { - List output = Lists.newArrayList(); - Event event; - while ((event = channel.take()) != null) { - output.add(new String(event.getBody(), Charset.defaultCharset())); - } - transaction.commit(); -// System.out.println("command : " + command); -// System.out.println("output : "); -// for( String line : output ) -// System.out.println(line); - Assert.assertArrayEquals(expectedOutput, output.toArray(new String[]{})); - } finally { - transaction.close(); - source.stop(); + private void runTestShellCmdHelper(String shell, String command, String[] expectedOutput) + throws InterruptedException, LifecycleException, EventDeliveryException, IOException { + context.put("shell", shell); + context.put("command", command); + Configurables.configure(source, context); + source.start(); + // Some commands might take longer to complete, specially on Windows + // or on slow environments (e.g. Travis CI). + Thread.sleep(2500); + Transaction transaction = channel.getTransaction(); + transaction.begin(); + try { + List output = Lists.newArrayList(); + Event event; + while ((event = channel.take()) != null) { + output.add(new String(event.getBody(), Charset.defaultCharset())); } + transaction.commit(); + Assert.assertArrayEquals(expectedOutput, output.toArray(new String[] {})); + } finally { + transaction.close(); + source.stop(); } - + } @Test public void testRestart() throws InterruptedException, LifecycleException, - EventDeliveryException, IOException { - + EventDeliveryException, IOException { context.put(ExecSourceConfigurationConstants.CONFIG_RESTART_THROTTLE, "10"); context.put(ExecSourceConfigurationConstants.CONFIG_RESTART, "true"); - - context.put("command", - SystemUtils.IS_OS_WINDOWS ? "cmd /c echo flume" : "echo flume"); + context.put("command", SystemUtils.IS_OS_WINDOWS ? "cmd /c echo flume" : "echo flume"); Configurables.configure(source, context); source.start(); @@ -380,7 +369,7 @@ public void testRestart() throws InterruptedException, LifecycleException, long start = System.currentTimeMillis(); - for(int i = 0; i < 5; i++) { + for (int i = 0; i < 5; i++) { Event event = channel.take(); assertNotNull(event); assertNotNull(event.getBody()); @@ -396,7 +385,6 @@ public void testRestart() throws InterruptedException, LifecycleException, source.stop(); } - /** * Tests to make sure that the shutdown mechanism works. There are races * in this test if the system has another sleep command running with the @@ -412,14 +400,12 @@ public void testShutdown() throws Exception { boolean searchForCommand = true; while (searchForCommand) { searchForCommand = false; - String command = SystemUtils.IS_OS_WINDOWS ? ("cmd /c sleep " + seconds) : - ("sleep " + seconds); - String searchTxt = SystemUtils.IS_OS_WINDOWS ? ("sleep.exe") : - ("\b" + command + "\b"); + String command = SystemUtils.IS_OS_WINDOWS ? "cmd /c sleep " + seconds : "sleep " + seconds; + String searchTxt = SystemUtils.IS_OS_WINDOWS ? "sleep.exe" : "\b" + command + "\b"; Pattern pattern = Pattern.compile(searchTxt); for (String line : exec(SystemUtils.IS_OS_WINDOWS ? - "cmd /c tasklist /FI \"SESSIONNAME eq Console\"" : - "ps -ef")) { + "cmd /c tasklist /FI \"SESSIONNAME eq Console\"" : + "ps -ef")) { if (pattern.matcher(line).find()) { seconds++; searchForCommand = true; @@ -444,9 +430,9 @@ public void testShutdown() throws Exception { source.stop(); Thread.sleep(1000L); for (String line : exec(SystemUtils.IS_OS_WINDOWS ? - "cmd /c tasklist /FI \"SESSIONNAME eq Console\"" : - "ps -ef")) { - if(pattern.matcher(line).find()) { + "cmd /c tasklist /FI \"SESSIONNAME eq Console\"" : + "ps -ef")) { + if (pattern.matcher(line).find()) { Assert.fail("Found [" + line + "]"); } } @@ -457,23 +443,21 @@ private static List exec(String command) throws Exception { Process process = new ProcessBuilder(commandArgs).start(); BufferedReader reader = null; try { - reader = new BufferedReader( - new InputStreamReader(process.getInputStream())); + reader = new BufferedReader(new InputStreamReader(process.getInputStream())); List result = Lists.newArrayList(); String line; - while((line = reader.readLine()) != null) { + while ((line = reader.readLine()) != null) { result.add(line); } return result; } finally { process.destroy(); - if(reader != null) { + if (reader != null) { reader.close(); } int exit = process.waitFor(); - if(exit != 0) { - throw new IllegalStateException("Command [" + command + "] exited with " - + exit); + if (exit != 0) { + throw new IllegalStateException("Command [" + command + "] exited with " + exit); } } } diff --git a/flume-ng-core/src/test/java/org/apache/flume/source/TestMultiportSyslogTCPSource.java b/flume-ng-core/src/test/java/org/apache/flume/source/TestMultiportSyslogTCPSource.java index c3dc241175..56c78812ed 100644 --- a/flume-ng-core/src/test/java/org/apache/flume/source/TestMultiportSyslogTCPSource.java +++ b/flume-ng-core/src/test/java/org/apache/flume/source/TestMultiportSyslogTCPSource.java @@ -58,6 +58,7 @@ import org.junit.Test; import org.slf4j.Logger; import org.slf4j.LoggerFactory; + import static org.mockito.Mockito.*; public class TestMultiportSyslogTCPSource { diff --git a/flume-ng-core/src/test/java/org/apache/flume/source/TestNetcatSource.java b/flume-ng-core/src/test/java/org/apache/flume/source/TestNetcatSource.java index e11b4b61a6..99d413af3e 100644 --- a/flume-ng-core/src/test/java/org/apache/flume/source/TestNetcatSource.java +++ b/flume-ng-core/src/test/java/org/apache/flume/source/TestNetcatSource.java @@ -21,7 +21,11 @@ import org.apache.commons.io.IOUtils; import org.apache.commons.io.LineIterator; -import org.apache.flume.*; +import org.apache.flume.Channel; +import org.apache.flume.ChannelSelector; +import org.apache.flume.Context; +import org.apache.flume.Event; +import org.apache.flume.Transaction; import org.apache.flume.channel.ChannelProcessor; import org.apache.flume.channel.MemoryChannel; import org.apache.flume.channel.ReplicatingChannelSelector; @@ -45,26 +49,27 @@ import java.util.List; public class TestNetcatSource { - private static final Logger logger = LoggerFactory - .getLogger(TestAvroSource.class); + private static final Logger logger = + LoggerFactory.getLogger(TestAvroSource.class); /** * Five first sentences of the Fables "The Crow and the Fox" * written by Jean de La Fontaine, French poet. * - * @see Jean de La Fontaine on wikipedia + * @see Jean de La Fontaine on + * wikipedia */ private final String french = "Maître Corbeau, sur un arbre perché, " + - "Tenait en son bec un fromage. " + - "Maître Renard, par l'odeur alléché, " + - "Lui tint à peu près ce langage : " + - "Et bonjour, Monsieur du Corbeau,"; + "Tenait en son bec un fromage. " + + "Maître Renard, par l'odeur alléché, " + + "Lui tint à peu près ce langage : " + + "Et bonjour, Monsieur du Corbeau,"; private final String english = "At the top of a tree perched Master Crow; " + - "In his beak he was holding a cheese. " + - "Drawn by the smell, Master Fox spoke, below. " + - "The words, more or less, were these: " + - "\"Hey, now, Sir Crow! Good day, good day!"; + "In his beak he was holding a cheese. " + + "Drawn by the smell, Master Fox spoke, below. " + + "The words, more or less, were these: " + + "\"Hey, now, Sir Crow! Good day, good day!"; private int selectedPort; private NetcatSource source; @@ -109,12 +114,14 @@ public void testUTF16BEencoding() throws InterruptedException, IOException { // Test on english text snippet for (int i = 0; i < 20; i++) { sendEvent(netcatSocket, english, encoding); - Assert.assertArrayEquals("Channel contained our event", english.getBytes(defaultCharset), getFlumeEvent()); + Assert.assertArrayEquals("Channel contained our event", english.getBytes(defaultCharset), + getFlumeEvent()); } // Test on french text snippet for (int i = 0; i < 20; i++) { sendEvent(netcatSocket, french, encoding); - Assert.assertArrayEquals("Channel contained our event", french.getBytes(defaultCharset), getFlumeEvent()); + Assert.assertArrayEquals("Channel contained our event", french.getBytes(defaultCharset), + getFlumeEvent()); } } finally { netcatSocket.close(); @@ -137,12 +144,14 @@ public void testUTF16LEencoding() throws InterruptedException, IOException { // Test on english text snippet for (int i = 0; i < 20; i++) { sendEvent(netcatSocket, english, encoding); - Assert.assertArrayEquals("Channel contained our event", english.getBytes(defaultCharset), getFlumeEvent()); + Assert.assertArrayEquals("Channel contained our event", english.getBytes(defaultCharset), + getFlumeEvent()); } // Test on french text snippet for (int i = 0; i < 20; i++) { sendEvent(netcatSocket, french, encoding); - Assert.assertArrayEquals("Channel contained our event", french.getBytes(defaultCharset), getFlumeEvent()); + Assert.assertArrayEquals("Channel contained our event", french.getBytes(defaultCharset), + getFlumeEvent()); } } finally { netcatSocket.close(); @@ -165,12 +174,14 @@ public void testUTF8encoding() throws InterruptedException, IOException { // Test on english text snippet for (int i = 0; i < 20; i++) { sendEvent(netcatSocket, english, encoding); - Assert.assertArrayEquals("Channel contained our event", english.getBytes(defaultCharset), getFlumeEvent()); + Assert.assertArrayEquals("Channel contained our event", english.getBytes(defaultCharset), + getFlumeEvent()); } // Test on french text snippet for (int i = 0; i < 20; i++) { sendEvent(netcatSocket, french, encoding); - Assert.assertArrayEquals("Channel contained our event", french.getBytes(defaultCharset), getFlumeEvent()); + Assert.assertArrayEquals("Channel contained our event", french.getBytes(defaultCharset), + getFlumeEvent()); } } finally { netcatSocket.close(); @@ -193,12 +204,14 @@ public void testIS88591encoding() throws InterruptedException, IOException { // Test on english text snippet for (int i = 0; i < 20; i++) { sendEvent(netcatSocket, english, encoding); - Assert.assertArrayEquals("Channel contained our event", english.getBytes(defaultCharset), getFlumeEvent()); + Assert.assertArrayEquals("Channel contained our event", english.getBytes(defaultCharset), + getFlumeEvent()); } // Test on french text snippet for (int i = 0; i < 20; i++) { sendEvent(netcatSocket, french, encoding); - Assert.assertArrayEquals("Channel contained our event", french.getBytes(defaultCharset), getFlumeEvent()); + Assert.assertArrayEquals("Channel contained our event", french.getBytes(defaultCharset), + getFlumeEvent()); } } finally { netcatSocket.close(); @@ -223,13 +236,15 @@ public void testAck() throws InterruptedException, IOException { // Test on english text snippet for (int i = 0; i < 20; i++) { sendEvent(netcatSocket, english, encoding); - Assert.assertArrayEquals("Channel contained our event", english.getBytes(defaultCharset), getFlumeEvent()); + Assert.assertArrayEquals("Channel contained our event", english.getBytes(defaultCharset), + getFlumeEvent()); Assert.assertEquals("Socket contained the Ack", ackEvent, inputLineIterator.nextLine()); } // Test on french text snippet for (int i = 0; i < 20; i++) { sendEvent(netcatSocket, french, encoding); - Assert.assertArrayEquals("Channel contained our event", french.getBytes(defaultCharset), getFlumeEvent()); + Assert.assertArrayEquals("Channel contained our event", french.getBytes(defaultCharset), + getFlumeEvent()); Assert.assertEquals("Socket contained the Ack", ackEvent, inputLineIterator.nextLine()); } } finally { @@ -251,7 +266,8 @@ public void testMaxLineLength() throws InterruptedException, IOException { Socket netcatSocket = new Socket(localhost, selectedPort); try { sendEvent(netcatSocket, "123456789", encoding); - Assert.assertArrayEquals("Channel contained our event", "123456789".getBytes(defaultCharset), getFlumeEvent()); + Assert.assertArrayEquals("Channel contained our event", + "123456789".getBytes(defaultCharset), getFlumeEvent()); sendEvent(netcatSocket, english, encoding); Assert.assertEquals("Channel does not contain an event", null, getRawFlumeEvent()); } finally { @@ -276,18 +292,21 @@ public void testMaxLineLengthwithAck() throws InterruptedException, IOException LineIterator inputLineIterator = IOUtils.lineIterator(netcatSocket.getInputStream(), encoding); try { sendEvent(netcatSocket, "123456789", encoding); - Assert.assertArrayEquals("Channel contained our event", "123456789".getBytes(defaultCharset), getFlumeEvent()); + Assert.assertArrayEquals("Channel contained our event", + "123456789".getBytes(defaultCharset), getFlumeEvent()); Assert.assertEquals("Socket contained the Ack", ackEvent, inputLineIterator.nextLine()); sendEvent(netcatSocket, english, encoding); Assert.assertEquals("Channel does not contain an event", null, getRawFlumeEvent()); - Assert.assertEquals("Socket contained the Error Ack", ackErrorEvent, inputLineIterator.nextLine()); + Assert.assertEquals("Socket contained the Error Ack", ackErrorEvent, inputLineIterator + .nextLine()); } finally { netcatSocket.close(); stopSource(); } } - private void startSource(String encoding, String ack, String batchSize, String maxLineLength) throws InterruptedException { + private void startSource(String encoding, String ack, String batchSize, String maxLineLength) + throws InterruptedException { boolean bound = false; for (int i = 0; i < 100 && !bound; i++) { @@ -313,9 +332,9 @@ private void startSource(String encoding, String ack, String batchSize, String m } Assert.assertTrue("Reached start or error", - LifecycleController.waitForOneOf(source, LifecycleState.START_OR_ERROR)); + LifecycleController.waitForOneOf(source, LifecycleState.START_OR_ERROR)); Assert.assertEquals("Server is started", LifecycleState.START, - source.getLifecycleState()); + source.getLifecycleState()); } private void sendEvent(Socket socket, String content, String encoding) throws IOException { @@ -366,9 +385,9 @@ private Event getRawFlumeEvent() { private void stopSource() throws InterruptedException { source.stop(); Assert.assertTrue("Reached stop or error", - LifecycleController.waitForOneOf(source, LifecycleState.STOP_OR_ERROR)); + LifecycleController.waitForOneOf(source, LifecycleState.STOP_OR_ERROR)); Assert.assertEquals("Server is stopped", LifecycleState.STOP, - source.getLifecycleState()); + source.getLifecycleState()); logger.info("Source stopped"); } } diff --git a/flume-ng-core/src/test/java/org/apache/flume/source/TestSequenceGeneratorSource.java b/flume-ng-core/src/test/java/org/apache/flume/source/TestSequenceGeneratorSource.java index 2bbcdaf3c2..5d6cc29dc3 100644 --- a/flume-ng-core/src/test/java/org/apache/flume/source/TestSequenceGeneratorSource.java +++ b/flume-ng-core/src/test/java/org/apache/flume/source/TestSequenceGeneratorSource.java @@ -18,9 +18,6 @@ */ package org.apache.flume.source; -import java.util.ArrayList; -import java.util.List; - import org.apache.flume.Channel; import org.apache.flume.ChannelSelector; import org.apache.flume.Context; @@ -36,6 +33,9 @@ import org.junit.Before; import org.junit.Test; +import java.util.ArrayList; +import java.util.List; + public class TestSequenceGeneratorSource { private PollableSource source; @@ -105,7 +105,7 @@ public void testBatchProcessWithLifeCycle() throws InterruptedException, Lifecyc for (long j = batchSize; j > 0; j--) { Event event = channel.take(); - String expectedVal = String.valueOf(((i+1)*batchSize)-j); + String expectedVal = String.valueOf(((i + 1) * batchSize) - j); String resultedVal = new String(event.getBody()); Assert.assertTrue("Expected " + expectedVal + " is not equals to " + resultedVal, expectedVal.equals(resultedVal)); diff --git a/flume-ng-core/src/test/java/org/apache/flume/source/TestSpoolDirectorySource.java b/flume-ng-core/src/test/java/org/apache/flume/source/TestSpoolDirectorySource.java index 47fdc7aa64..82c53519c3 100644 --- a/flume-ng-core/src/test/java/org/apache/flume/source/TestSpoolDirectorySource.java +++ b/flume-ng-core/src/test/java/org/apache/flume/source/TestSpoolDirectorySource.java @@ -17,15 +17,9 @@ package org.apache.flume.source; -import java.io.ByteArrayOutputStream; -import java.io.File; -import java.io.IOException; -import java.util.ArrayList; -import java.util.Arrays; -import java.util.List; -import java.util.concurrent.TimeUnit; - +import com.google.common.base.Charsets; import com.google.common.collect.Lists; +import com.google.common.io.Files; import org.apache.flume.Channel; import org.apache.flume.ChannelSelector; import org.apache.flume.Context; @@ -42,8 +36,13 @@ import org.junit.Before; import org.junit.Test; -import com.google.common.base.Charsets; -import com.google.common.io.Files; +import java.io.ByteArrayOutputStream; +import java.io.File; +import java.io.IOException; +import java.util.ArrayList; +import java.util.Arrays; +import java.util.List; +import java.util.concurrent.TimeUnit; public class TestSpoolDirectorySource { static SpoolDirectorySource source; @@ -151,14 +150,13 @@ public void testPutFilenameHeader() throws IOException, InterruptedException { * Tests if SpoolDirectorySource sets basename headers on events correctly */ @Test - public void testPutBasenameHeader() throws IOException, - InterruptedException { + public void testPutBasenameHeader() throws IOException, InterruptedException { Context context = new Context(); File f1 = new File(tmpDir.getAbsolutePath() + "/file1"); Files.write("file1line1\nfile1line2\nfile1line3\nfile1line4\n" + - "file1line5\nfile1line6\nfile1line7\nfile1line8\n", - f1, Charsets.UTF_8); + "file1line5\nfile1line6\nfile1line7\nfile1line8\n", + f1, Charsets.UTF_8); context.put(SpoolDirectorySourceConfigurationConstants.SPOOL_DIRECTORY, tmpDir.getAbsolutePath()); @@ -179,7 +177,7 @@ public void testPutBasenameHeader() throws IOException, Assert.assertNotNull("Event headers must not be null", e.getHeaders()); Assert.assertNotNull(e.getHeaders().get("basenameHeaderKeyTest")); Assert.assertEquals(f1.getName(), - e.getHeaders().get("basenameHeaderKeyTest")); + e.getHeaders().get("basenameHeaderKeyTest")); txn.commit(); txn.close(); } @@ -233,7 +231,7 @@ public void testRecursion_SetToTrue() throws IOException, InterruptedException { baos.write(e.getBody()); baos.write('\n'); // newline characters are consumed in the process e = channel.take(); - } while(e != null); + } while (e != null); Assert.assertEquals("Event body is correct", Arrays.toString(origBody.getBytes()), @@ -371,7 +369,7 @@ public void testSourceDoesNotDieOnFullChannel() throws Exception { context.put(SpoolDirectorySourceConfigurationConstants.SPOOL_DIRECTORY, - tmpDir.getAbsolutePath()); + tmpDir.getAbsolutePath()); context.put(SpoolDirectorySourceConfigurationConstants.BATCH_SIZE, "2"); Configurables.configure(source, context); @@ -379,7 +377,7 @@ public void testSourceDoesNotDieOnFullChannel() throws Exception { source.start(); // Wait for the source to read enough events to fill up the channel. - while(!source.hitChannelException()) { + while (!source.hitChannelException()) { Thread.sleep(50); } @@ -402,7 +400,7 @@ public void testSourceDoesNotDieOnFullChannel() throws Exception { tx.close(); } Assert.assertTrue("Expected to hit ChannelException, but did not!", - source.hitChannelException()); + source.hitChannelException()); Assert.assertEquals(8, dataOut.size()); source.stop(); } @@ -424,7 +422,7 @@ public void testEndWithZeroByteFiles() throws IOException, InterruptedException Files.touch(f4); context.put(SpoolDirectorySourceConfigurationConstants.SPOOL_DIRECTORY, - tmpDir.getAbsolutePath()); + tmpDir.getAbsolutePath()); Configurables.configure(source, context); source.start(); @@ -432,8 +430,8 @@ public void testEndWithZeroByteFiles() throws IOException, InterruptedException Thread.sleep(5000); Assert.assertFalse("Server did not error", source.hasFatalError()); - Assert.assertEquals("One message was read", 1, - source.getSourceCounter().getEventAcceptedCount()); + Assert.assertEquals("One message was read", + 1, source.getSourceCounter().getEventAcceptedCount()); source.stop(); } } diff --git a/flume-ng-core/src/test/java/org/apache/flume/source/TestStressSource.java b/flume-ng-core/src/test/java/org/apache/flume/source/TestStressSource.java index a6512818dc..06e663c4c5 100644 --- a/flume-ng-core/src/test/java/org/apache/flume/source/TestStressSource.java +++ b/flume-ng-core/src/test/java/org/apache/flume/source/TestStressSource.java @@ -101,7 +101,7 @@ public void testBatchEvents() throws InterruptedException, } if (i < 3) { verify(mockProcessor, - times(i+1)).processEventBatch(getLastProcessedEventList(source)); + times(i + 1)).processEventBatch(getLastProcessedEventList(source)); } else { verify(mockProcessor, times(1)).processEventBatch(getLastProcessedEventList(source)); diff --git a/flume-ng-core/src/test/java/org/apache/flume/source/TestSyslogParser.java b/flume-ng-core/src/test/java/org/apache/flume/source/TestSyslogParser.java index 265157eb5f..7edb9b737d 100644 --- a/flume-ng-core/src/test/java/org/apache/flume/source/TestSyslogParser.java +++ b/flume-ng-core/src/test/java/org/apache/flume/source/TestSyslogParser.java @@ -20,17 +20,17 @@ import com.google.common.base.Charsets; import com.google.common.collect.Lists; -import java.nio.charset.Charset; -import java.util.HashSet; -import java.util.List; -import java.util.Set; - import org.apache.flume.Event; import org.joda.time.format.DateTimeFormatter; import org.joda.time.format.ISODateTimeFormat; import org.junit.Assert; import org.junit.Test; +import java.nio.charset.Charset; +import java.util.HashSet; +import java.util.List; +import java.util.Set; + public class TestSyslogParser { @Test public void testRfc5424DateParsing() { @@ -84,7 +84,7 @@ public void testMessageParsing() { Set keepFields = new HashSet(); Event event = parser.parseMessage(msg, charset, keepFields); Assert.assertNull("Failure to parse known-good syslog message", - event.getHeaders().get(SyslogUtils.EVENT_STATUS)); + event.getHeaders().get(SyslogUtils.EVENT_STATUS)); } // test that priority, timestamp and hostname are preserved in event body diff --git a/flume-ng-core/src/test/java/org/apache/flume/source/TestSyslogTcpSource.java b/flume-ng-core/src/test/java/org/apache/flume/source/TestSyslogTcpSource.java index 239ba51bcd..10ef8d874c 100644 --- a/flume-ng-core/src/test/java/org/apache/flume/source/TestSyslogTcpSource.java +++ b/flume-ng-core/src/test/java/org/apache/flume/source/TestSyslogTcpSource.java @@ -41,7 +41,7 @@ public class TestSyslogTcpSource { private static final org.slf4j.Logger logger = - LoggerFactory.getLogger(TestSyslogTcpSource.class); + LoggerFactory.getLogger(TestSyslogTcpSource.class); private SyslogTcpSource source; private Channel channel; private static final int TEST_SYSLOG_PORT = 0; @@ -56,7 +56,7 @@ public class TestSyslogTcpSource { private final String bodyWithTandH = "<10>" + stamp1 + " " + host1 + " " + data1 + "\n"; - private void init(String keepFields){ + private void init(String keepFields) { source = new SyslogTcpSource(); channel = new MemoryChannel(); @@ -116,7 +116,7 @@ private void runKeepFieldsTest(String keepFields) throws IOException { logger.info(str); if (keepFields.equals("true") || keepFields.equals("all")) { Assert.assertArrayEquals(bodyWithTandH.trim().getBytes(), - e.getBody()); + e.getBody()); } else if (keepFields.equals("false") || keepFields.equals("none")) { Assert.assertArrayEquals(data1.getBytes(), e.getBody()); } else if (keepFields.equals("hostname")) { @@ -136,7 +136,7 @@ public void testKeepFields() throws IOException { } @Test - public void testRemoveFields() throws IOException{ + public void testRemoveFields() throws IOException { runKeepFieldsTest("none"); // Backwards compatibility @@ -144,12 +144,12 @@ public void testRemoveFields() throws IOException{ } @Test - public void testKeepHostname() throws IOException{ + public void testKeepHostname() throws IOException { runKeepFieldsTest("hostname"); } @Test - public void testKeepTimestamp() throws IOException{ + public void testKeepTimestamp() throws IOException { runKeepFieldsTest("timestamp"); } } diff --git a/flume-ng-core/src/test/java/org/apache/flume/source/TestSyslogUdpSource.java b/flume-ng-core/src/test/java/org/apache/flume/source/TestSyslogUdpSource.java index 8fc80be5da..e5b7a066a6 100644 --- a/flume-ng-core/src/test/java/org/apache/flume/source/TestSyslogUdpSource.java +++ b/flume-ng-core/src/test/java/org/apache/flume/source/TestSyslogUdpSource.java @@ -18,14 +18,7 @@ */ package org.apache.flume.source; -import java.util.ArrayList; -import java.util.List; -import java.io.IOException; -import java.net.DatagramPacket; -import java.net.InetAddress; -import java.net.DatagramSocket; import com.google.common.base.Charsets; -import com.google.common.base.Strings; import org.apache.flume.Channel; import org.apache.flume.ChannelSelector; import org.apache.flume.Context; @@ -40,10 +33,16 @@ import org.junit.Test; import org.slf4j.LoggerFactory; +import java.io.IOException; +import java.net.DatagramPacket; +import java.net.DatagramSocket; +import java.net.InetAddress; +import java.util.ArrayList; +import java.util.List; public class TestSyslogUdpSource { private static final org.slf4j.Logger logger = - LoggerFactory.getLogger(TestSyslogUdpSource.class); + LoggerFactory.getLogger(TestSyslogUdpSource.class); private SyslogUDPSource source; private Channel channel; private static final int TEST_SYSLOG_PORT = 0; @@ -192,12 +191,12 @@ public void testRemoveFields() throws IOException { } @Test - public void testKeepHostname() throws IOException{ + public void testKeepHostname() throws IOException { runKeepFieldsTest("hostname"); } @Test - public void testKeepTimestamp() throws IOException{ + public void testKeepTimestamp() throws IOException { runKeepFieldsTest("timestamp"); } diff --git a/flume-ng-core/src/test/java/org/apache/flume/source/TestSyslogUtils.java b/flume-ng-core/src/test/java/org/apache/flume/source/TestSyslogUtils.java index 1c005ffffa..80d8dac992 100644 --- a/flume-ng-core/src/test/java/org/apache/flume/source/TestSyslogUtils.java +++ b/flume-ng-core/src/test/java/org/apache/flume/source/TestSyslogUtils.java @@ -18,7 +18,6 @@ */ package org.apache.flume.source; - import org.apache.flume.Event; import org.jboss.netty.buffer.ChannelBuffer; import org.jboss.netty.buffer.ChannelBuffers; @@ -41,7 +40,7 @@ public void TestHeader0() throws ParseException { String host1 = "ubuntu-11.cloudera.com"; String data1 = "some msg"; // timestamp with hh:mm format timezone with no version - String msg1 = "<10>" + stamp1+ "+08:00" + " " + host1 + " " + data1 + "\n"; + String msg1 = "<10>" + stamp1 + "+08:00" + " " + host1 + " " + data1 + "\n"; checkHeader(msg1, stamp1 + "+0800", format1, host1, data1); } @@ -62,7 +61,7 @@ public void TestHeader2() throws ParseException { String host1 = "ubuntu-11.cloudera.com"; String data1 = "some msg"; // timestamp with 'Z' appended, translates to UTC - String msg1 = "<10>1 " + stamp1+ "Z" + " " + host1 + " " + data1 + "\n"; + String msg1 = "<10>1 " + stamp1 + "Z" + " " + host1 + " " + data1 + "\n"; checkHeader(msg1, stamp1 + "+0000", format1, host1, data1); } @@ -73,13 +72,13 @@ public void TestHeader3() throws ParseException { String host1 = "ubuntu-11.cloudera.com"; String data1 = "some msg"; // timestamp with hh:mm format timezone - String msg1 = "<10>1 " + stamp1+ "+08:00" + " " + host1 + " " + data1 + "\n"; + String msg1 = "<10>1 " + stamp1 + "+08:00" + " " + host1 + " " + data1 + "\n"; checkHeader(msg1, stamp1 + "+0800", format1, host1, data1); } @Test public void TestHeader4() throws ParseException { - String host1 = "ubuntu-11.cloudera.com"; + String host1 = "ubuntu-11.cloudera.com"; String data1 = "some msg"; // null format timestamp (-) String msg1 = "<10>1 " + "-" + " " + host1 + " " + data1 + "\n"; @@ -104,7 +103,7 @@ public void TestHeader6() throws ParseException { String host1 = "-"; String data1 = "some msg"; // null host - String msg1 = "<10>1 " + stamp1+ "Z" + " " + host1 + " " + data1 + "\n"; + String msg1 = "<10>1 " + stamp1 + "Z" + " " + host1 + " " + data1 + "\n"; checkHeader(msg1, stamp1 + "+0000", format1, null, data1); } @@ -115,7 +114,7 @@ public void TestHeader7() throws ParseException { String host1 = "-"; String data1 = "some msg"; // null host - String msg1 = "<10>1 " + stamp1+ "+08:00" + " " + host1 + " " + data1 + "\n"; + String msg1 = "<10>1 " + stamp1 + "+08:00" + " " + host1 + " " + data1 + "\n"; checkHeader(msg1, stamp1 + "+0800", format1, null, data1); } @@ -141,8 +140,7 @@ public void TestHeader9() throws ParseException { String data1 = "some msg"; // timestamp with 'Z' appended, translates to UTC String msg1 = "<10>" + stamp1 + " " + host1 + " " + data1 + "\n"; - checkHeader(msg1, year + stamp1, - format1, host1, data1); + checkHeader(msg1, year + stamp1, format1, host1, data1); } @Test @@ -157,8 +155,7 @@ public void TestHeader10() throws ParseException { String data1 = "some msg"; // timestamp with 'Z' appended, translates to UTC String msg1 = "<10>" + stamp1 + " " + host1 + " " + data1 + "\n"; - checkHeader(msg1, year + stamp1, - format1, host1, data1); + checkHeader(msg1, year + stamp1, format1, host1, data1); } @Test @@ -187,21 +184,19 @@ public void TestRfc3164HeaderApacheLogWithNulls() throws ParseException { String host1 = "ubuntu-11.cloudera.com"; String data1 = "- hyphen_null_breaks_5424_pattern [07/Jun/2012:14:46:44 -0600]"; String msg1 = "<10>" + stamp1 + " " + host1 + " " + data1 + "\n"; - checkHeader(msg1, year + stamp1, - format1, host1, data1); + checkHeader(msg1, year + stamp1, format1, host1, data1); } + /* This test creates a series of dates that range from 10 months in the past to (5 days short + * of) one month in the future. This tests that the year addition code is clever enough to + * handle scenarios where the event received was generated in a different year to what flume + * considers to be "current" (e.g. where there has been some lag somewhere, especially when + * flicking over on New Year's eve, or when you are about to flick over and the flume's + * system clock is slightly slower than the Syslog source's clock). + */ @Test public void TestRfc3164Dates() throws ParseException { - /* - * This test creates a series of dates that range from 10 months in the past to (5 days short of) - * one month in the future. This tests that the year addition code is clever enough to handle scenarios - * where the event received was generated in a different year to what flume considers to be "current" - * (e.g. where there has been some lag somewhere, especially when flicking over on New Year's eve, or - * when you are about to flick over and the flume's system clock is slightly slower than the Syslog - * source's clock). - */ - for (int i=-10; i<=1; i++) { + for (int i = -10; i <= 1; i++) { SimpleDateFormat sdf = new SimpleDateFormat("MMM d hh:MM:ss"); Date date = new Date(System.currentTimeMillis()); Calendar cal = Calendar.getInstance(); @@ -210,7 +205,7 @@ public void TestRfc3164Dates() throws ParseException { //Small tweak to avoid the 1 month in the future ticking over by a few seconds between now //and when the checkHeader actually runs - if (i==1) { + if (i == 1) { cal.add(Calendar.DAY_OF_MONTH, -1); } @@ -223,8 +218,7 @@ public void TestRfc3164Dates() throws ParseException { // timestamp with 'Z' appended, translates to UTC String msg1 = "<10>" + stamp1 + " " + host1 + " " + data1 + "\n"; - checkHeader(msg1, year + stamp1, - format1, host1, data1); + checkHeader(msg1, year + stamp1, format1, host1, data1); } } @@ -234,16 +228,15 @@ public static void checkHeader(String keepFields, String msg1, String stamp1, St if (keepFields == null || keepFields.isEmpty()) { util = new SyslogUtils(SyslogUtils.DEFAULT_SIZE, new HashSet(), false); } else { - util = new SyslogUtils( - SyslogUtils.DEFAULT_SIZE, - SyslogUtils.chooseFieldsToKeep(keepFields), - false); + util = new SyslogUtils(SyslogUtils.DEFAULT_SIZE, + SyslogUtils.chooseFieldsToKeep(keepFields), + false); } ChannelBuffer buff = ChannelBuffers.buffer(200); buff.writeBytes(msg1.getBytes()); Event e = util.extractEvent(buff); - if(e == null){ + if (e == null) { throw new NullPointerException("Event is null"); } Map headers2 = e.getHeaders(); @@ -251,13 +244,14 @@ public static void checkHeader(String keepFields, String msg1, String stamp1, St Assert.assertFalse(headers2.containsKey("timestamp")); } else { SimpleDateFormat formater = new SimpleDateFormat(format1, Locale.ENGLISH); - Assert.assertEquals(String.valueOf(formater.parse(stamp1).getTime()), headers2.get("timestamp")); + Assert.assertEquals(String.valueOf(formater.parse(stamp1).getTime()), + headers2.get("timestamp")); } if (host1 == null) { Assert.assertFalse(headers2.containsKey("host")); } else { String host2 = headers2.get("host"); - Assert.assertEquals(host2,host1); + Assert.assertEquals(host2, host1); } Assert.assertEquals(data1, new String(e.getBody())); } @@ -278,22 +272,20 @@ public void testExtractBadEvent1() { ChannelBuffer buff = ChannelBuffers.buffer(100); buff.writeBytes(badData1.getBytes()); Event e = util.extractEvent(buff); - if(e == null){ + if (e == null) { throw new NullPointerException("Event is null"); } Map headers = e.getHeaders(); Assert.assertEquals("0", headers.get(SyslogUtils.SYSLOG_FACILITY)); Assert.assertEquals("0", headers.get(SyslogUtils.SYSLOG_SEVERITY)); Assert.assertEquals(SyslogUtils.SyslogStatus.INVALID.getSyslogStatus(), - headers.get(SyslogUtils.EVENT_STATUS)); + headers.get(SyslogUtils.EVENT_STATUS)); Assert.assertEquals(badData1.trim(), new String(e.getBody()).trim()); - } /** * Test bad event format 2: The first char is not < */ - @Test public void testExtractBadEvent2() { String badData1 = "hi guys! <10> bad bad data\n"; @@ -301,22 +293,20 @@ public void testExtractBadEvent2() { ChannelBuffer buff = ChannelBuffers.buffer(100); buff.writeBytes(badData1.getBytes()); Event e = util.extractEvent(buff); - if(e == null){ + if (e == null) { throw new NullPointerException("Event is null"); } Map headers = e.getHeaders(); Assert.assertEquals("0", headers.get(SyslogUtils.SYSLOG_FACILITY)); Assert.assertEquals("0", headers.get(SyslogUtils.SYSLOG_SEVERITY)); Assert.assertEquals(SyslogUtils.SyslogStatus.INVALID.getSyslogStatus(), - headers.get(SyslogUtils.EVENT_STATUS)); + headers.get(SyslogUtils.EVENT_STATUS)); Assert.assertEquals(badData1.trim(), new String(e.getBody()).trim()); - } /** * Test bad event format 3: Empty priority - <> */ - @Test public void testExtractBadEvent3() { String badData1 = "<> bad bad data\n"; @@ -324,22 +314,20 @@ public void testExtractBadEvent3() { ChannelBuffer buff = ChannelBuffers.buffer(100); buff.writeBytes(badData1.getBytes()); Event e = util.extractEvent(buff); - if(e == null){ + if (e == null) { throw new NullPointerException("Event is null"); } Map headers = e.getHeaders(); Assert.assertEquals("0", headers.get(SyslogUtils.SYSLOG_FACILITY)); Assert.assertEquals("0", headers.get(SyslogUtils.SYSLOG_SEVERITY)); Assert.assertEquals(SyslogUtils.SyslogStatus.INVALID.getSyslogStatus(), - headers.get(SyslogUtils.EVENT_STATUS)); + headers.get(SyslogUtils.EVENT_STATUS)); Assert.assertEquals(badData1.trim(), new String(e.getBody()).trim()); - } /** * Test bad event format 4: Priority too long */ - @Test public void testExtractBadEvent4() { String badData1 = "<123123123123123123123123123123> bad bad data\n"; @@ -347,16 +335,15 @@ public void testExtractBadEvent4() { ChannelBuffer buff = ChannelBuffers.buffer(100); buff.writeBytes(badData1.getBytes()); Event e = util.extractEvent(buff); - if(e == null){ + if (e == null) { throw new NullPointerException("Event is null"); } Map headers = e.getHeaders(); Assert.assertEquals("0", headers.get(SyslogUtils.SYSLOG_FACILITY)); Assert.assertEquals("0", headers.get(SyslogUtils.SYSLOG_SEVERITY)); Assert.assertEquals(SyslogUtils.SyslogStatus.INVALID.getSyslogStatus(), - headers.get(SyslogUtils.EVENT_STATUS)); + headers.get(SyslogUtils.EVENT_STATUS)); Assert.assertEquals(badData1.trim(), new String(e.getBody()).trim()); - } /** @@ -368,9 +355,9 @@ public void testExtractGoodEvent() { String goodData1 = "Good good good data\n"; SyslogUtils util = new SyslogUtils(false); ChannelBuffer buff = ChannelBuffers.buffer(100); - buff.writeBytes((priority+goodData1).getBytes()); + buff.writeBytes((priority + goodData1).getBytes()); Event e = util.extractEvent(buff); - if(e == null){ + if (e == null) { throw new NullPointerException("Event is null"); } Map headers = e.getHeaders(); @@ -378,86 +365,81 @@ public void testExtractGoodEvent() { Assert.assertEquals("2", headers.get(SyslogUtils.SYSLOG_SEVERITY)); Assert.assertEquals(null, headers.get(SyslogUtils.EVENT_STATUS)); Assert.assertEquals(priority + goodData1.trim(), - new String(e.getBody()).trim()); - + new String(e.getBody()).trim()); } /** * Bad event immediately followed by a good event */ @Test - public void testBadEventGoodEvent(){ + public void testBadEventGoodEvent() { String badData1 = "hi guys! <10F> bad bad data\n"; SyslogUtils util = new SyslogUtils(false); ChannelBuffer buff = ChannelBuffers.buffer(100); buff.writeBytes(badData1.getBytes()); String priority = "<10>"; String goodData1 = "Good good good data\n"; - buff.writeBytes((priority+goodData1).getBytes()); + buff.writeBytes((priority + goodData1).getBytes()); Event e = util.extractEvent(buff); - if(e == null){ + if (e == null) { throw new NullPointerException("Event is null"); } Map headers = e.getHeaders(); Assert.assertEquals("0", headers.get(SyslogUtils.SYSLOG_FACILITY)); Assert.assertEquals("0", headers.get(SyslogUtils.SYSLOG_SEVERITY)); Assert.assertEquals(SyslogUtils.SyslogStatus.INVALID.getSyslogStatus(), - headers.get(SyslogUtils.EVENT_STATUS)); - Assert.assertEquals(badData1.trim(), new String(e.getBody()) - .trim()); + headers.get(SyslogUtils.EVENT_STATUS)); + Assert.assertEquals(badData1.trim(), new String(e.getBody()).trim()); Event e2 = util.extractEvent(buff); - if(e2 == null){ + if (e2 == null) { throw new NullPointerException("Event is null"); } Map headers2 = e2.getHeaders(); Assert.assertEquals("1", headers2.get(SyslogUtils.SYSLOG_FACILITY)); Assert.assertEquals("2", headers2.get(SyslogUtils.SYSLOG_SEVERITY)); - Assert.assertEquals(null, - headers2.get(SyslogUtils.EVENT_STATUS)); - Assert.assertEquals(priority + goodData1.trim(), - new String(e2.getBody()).trim()); + Assert.assertEquals(null, headers2.get(SyslogUtils.EVENT_STATUS)); + Assert.assertEquals(priority + goodData1.trim(), new String(e2.getBody()).trim()); } @Test - public void testGoodEventBadEvent(){ + public void testGoodEventBadEvent() { String badData1 = "hi guys! <10F> bad bad data\n"; String priority = "<10>"; String goodData1 = "Good good good data\n"; SyslogUtils util = new SyslogUtils(false); ChannelBuffer buff = ChannelBuffers.buffer(100); - buff.writeBytes((priority+goodData1).getBytes()); + buff.writeBytes((priority + goodData1).getBytes()); buff.writeBytes(badData1.getBytes()); Event e2 = util.extractEvent(buff); - if(e2 == null){ + if (e2 == null) { throw new NullPointerException("Event is null"); } Map headers2 = e2.getHeaders(); Assert.assertEquals("1", headers2.get(SyslogUtils.SYSLOG_FACILITY)); Assert.assertEquals("2", headers2.get(SyslogUtils.SYSLOG_SEVERITY)); Assert.assertEquals(null, - headers2.get(SyslogUtils.EVENT_STATUS)); + headers2.get(SyslogUtils.EVENT_STATUS)); Assert.assertEquals(priority + goodData1.trim(), - new String(e2.getBody()).trim()); + new String(e2.getBody()).trim()); Event e = util.extractEvent(buff); - if(e == null){ + if (e == null) { throw new NullPointerException("Event is null"); } Map headers = e.getHeaders(); Assert.assertEquals("0", headers.get(SyslogUtils.SYSLOG_FACILITY)); Assert.assertEquals("0", headers.get(SyslogUtils.SYSLOG_SEVERITY)); Assert.assertEquals(SyslogUtils.SyslogStatus.INVALID.getSyslogStatus(), - headers.get(SyslogUtils.EVENT_STATUS)); + headers.get(SyslogUtils.EVENT_STATUS)); Assert.assertEquals(badData1.trim(), new String(e.getBody()).trim()); - } @Test - public void testBadEventBadEvent(){ + public void testBadEventBadEvent() { String badData1 = "hi guys! <10F> bad bad data\n"; SyslogUtils util = new SyslogUtils(false); ChannelBuffer buff = ChannelBuffers.buffer(100); @@ -466,65 +448,63 @@ public void testBadEventBadEvent(){ buff.writeBytes((badData2).getBytes()); Event e = util.extractEvent(buff); - if(e == null){ + if (e == null) { throw new NullPointerException("Event is null"); } Map headers = e.getHeaders(); Assert.assertEquals("0", headers.get(SyslogUtils.SYSLOG_FACILITY)); Assert.assertEquals("0", headers.get(SyslogUtils.SYSLOG_SEVERITY)); Assert.assertEquals(SyslogUtils.SyslogStatus.INVALID.getSyslogStatus(), - headers.get(SyslogUtils.EVENT_STATUS)); + headers.get(SyslogUtils.EVENT_STATUS)); Assert.assertEquals(badData1.trim(), new String(e.getBody()).trim()); Event e2 = util.extractEvent(buff); - if(e2 == null){ + if (e2 == null) { throw new NullPointerException("Event is null"); } Map headers2 = e2.getHeaders(); Assert.assertEquals("0", headers2.get(SyslogUtils.SYSLOG_FACILITY)); Assert.assertEquals("0", headers2.get(SyslogUtils.SYSLOG_SEVERITY)); Assert.assertEquals(SyslogUtils.SyslogStatus.INVALID.getSyslogStatus(), - headers2.get(SyslogUtils.EVENT_STATUS)); + headers2.get(SyslogUtils.EVENT_STATUS)); Assert.assertEquals(badData2.trim(), new String(e2.getBody()).trim()); } @Test public void testGoodEventGoodEvent() { - String priority = "<10>"; String goodData1 = "Good good good data\n"; SyslogUtils util = new SyslogUtils(false); ChannelBuffer buff = ChannelBuffers.buffer(100); - buff.writeBytes((priority+goodData1).getBytes()); + buff.writeBytes((priority + goodData1).getBytes()); String priority2 = "<20>"; String goodData2 = "Good really good data\n"; - buff.writeBytes((priority2+goodData2).getBytes()); + buff.writeBytes((priority2 + goodData2).getBytes()); Event e = util.extractEvent(buff); - if(e == null){ + if (e == null) { throw new NullPointerException("Event is null"); } Map headers = e.getHeaders(); Assert.assertEquals("1", headers.get(SyslogUtils.SYSLOG_FACILITY)); Assert.assertEquals("2", headers.get(SyslogUtils.SYSLOG_SEVERITY)); Assert.assertEquals(null, - headers.get(SyslogUtils.EVENT_STATUS)); + headers.get(SyslogUtils.EVENT_STATUS)); Assert.assertEquals(priority + goodData1.trim(), - new String(e.getBody()).trim()); + new String(e.getBody()).trim()); Event e2 = util.extractEvent(buff); - if(e2 == null){ + if (e2 == null) { throw new NullPointerException("Event is null"); } Map headers2 = e2.getHeaders(); Assert.assertEquals("2", headers2.get(SyslogUtils.SYSLOG_FACILITY)); Assert.assertEquals("4", headers2.get(SyslogUtils.SYSLOG_SEVERITY)); Assert.assertEquals(null, - headers.get(SyslogUtils.EVENT_STATUS)); + headers.get(SyslogUtils.EVENT_STATUS)); Assert.assertEquals(priority2 + goodData2.trim(), - new String(e2.getBody()).trim()); - + new String(e2.getBody()).trim()); } @Test @@ -535,28 +515,27 @@ public void testExtractBadEventLarge() { ChannelBuffer buff = ChannelBuffers.buffer(100); buff.writeBytes(badData1.getBytes()); Event e = util.extractEvent(buff); - if(e == null){ + if (e == null) { throw new NullPointerException("Event is null"); } Map headers = e.getHeaders(); Assert.assertEquals("1", headers.get(SyslogUtils.SYSLOG_FACILITY)); Assert.assertEquals("2", headers.get(SyslogUtils.SYSLOG_SEVERITY)); Assert.assertEquals(SyslogUtils.SyslogStatus.INCOMPLETE.getSyslogStatus(), - headers.get(SyslogUtils.EVENT_STATUS)); + headers.get(SyslogUtils.EVENT_STATUS)); Assert.assertEquals("<10> bad b".trim(), new String(e.getBody()).trim()); Event e2 = util.extractEvent(buff); - if(e2 == null){ + if (e2 == null) { throw new NullPointerException("Event is null"); } Map headers2 = e2.getHeaders(); Assert.assertEquals("0", headers2.get(SyslogUtils.SYSLOG_FACILITY)); Assert.assertEquals("0", headers2.get(SyslogUtils.SYSLOG_SEVERITY)); Assert.assertEquals(SyslogUtils.SyslogStatus.INVALID.getSyslogStatus(), - headers2.get(SyslogUtils.EVENT_STATUS)); + headers2.get(SyslogUtils.EVENT_STATUS)); Assert.assertEquals("ad data ba".trim(), new String(e2.getBody()).trim()); - } @Test @@ -580,7 +559,8 @@ public void testKeepFields() throws Exception { checkHeader("priority timestamp hostname", msg1, stamp1 + "+0800", format1, host1, data4); String data5 = "<10>1 2012-04-13T11:11:11+08:00 ubuntu-11.cloudera.com some msg"; - checkHeader("priority version timestamp hostname", msg1, stamp1 + "+0800", format1, host1, data5); + checkHeader("priority version timestamp hostname", msg1, stamp1 + "+0800", + format1, host1, data5); checkHeader("all", msg1, stamp1 + "+0800", format1, host1, data5); checkHeader("true", msg1, stamp1 + "+0800", format1, host1, data5); } diff --git a/flume-ng-core/src/test/java/org/apache/flume/source/TestThriftSource.java b/flume-ng-core/src/test/java/org/apache/flume/source/TestThriftSource.java index 3d2901a574..cdaefafc08 100644 --- a/flume-ng-core/src/test/java/org/apache/flume/source/TestThriftSource.java +++ b/flume-ng-core/src/test/java/org/apache/flume/source/TestThriftSource.java @@ -67,10 +67,9 @@ public void setUp() { port = random.nextInt(50000) + 1024; props.clear(); props.setProperty("hosts", "h1"); - props.setProperty("hosts.h1", "0.0.0.0:"+ String.valueOf(port)); + props.setProperty("hosts.h1", "0.0.0.0:" + String.valueOf(port)); props.setProperty(RpcClientConfigurationConstants.CONFIG_BATCH_SIZE, "10"); - props.setProperty(RpcClientConfigurationConstants.CONFIG_REQUEST_TIMEOUT, - "2000"); + props.setProperty(RpcClientConfigurationConstants.CONFIG_REQUEST_TIMEOUT, "2000"); channel = new MemoryChannel(); source = new ThriftSource(); } @@ -110,7 +109,7 @@ public void testAppendSSL() throws Exception { context.put("keymanager-type", KeyManagerFactory.getDefaultAlgorithm()); Configurables.configure(source, context); source.start(); - for(int i = 0; i < 30; i++) { + for (int i = 0; i < 30; i++) { client.append(EventBuilder.withBody(String.valueOf(i).getBytes())); } Transaction transaction = channel.getTransaction(); @@ -135,7 +134,7 @@ public void testAppend() throws Exception { context.put(ThriftSource.CONFIG_PORT, String.valueOf(port)); Configurables.configure(source, context); source.start(); - for(int i = 0; i < 30; i++) { + for (int i = 0; i < 30; i++) { client.append(EventBuilder.withBody(String.valueOf(i).getBytes())); } Transaction transaction = channel.getTransaction(); @@ -188,8 +187,8 @@ public void testAppendBatch() throws Exception { int index = 0; //30 batches of 10 - for(int i = 0; i < 30; i++) { - for(int j = 0; j < 10; j++) { + for (int i = 0; i < 30; i++) { + for (int j = 0; j < 10; j++) { Assert.assertEquals(i, events.get(index++).intValue()); } } @@ -233,8 +232,8 @@ public void testAppendBigBatch() throws Exception { int index = 0; //10 batches of 500 - for(int i = 0; i < 5; i++) { - for(int j = 0; j < 500; j++) { + for (int i = 0; i < 5; i++) { + for (int j = 0; j < 500; j++) { Assert.assertEquals(i, events.get(index++).intValue()); } } @@ -253,15 +252,14 @@ public void testMultipleClients() throws Exception { context.put(ThriftSource.CONFIG_PORT, String.valueOf(port)); Configurables.configure(source, context); source.start(); - ExecutorCompletionService completionService = new - ExecutorCompletionService(submitter); + ExecutorCompletionService completionService = new ExecutorCompletionService(submitter); for (int i = 0; i < 30; i++) { completionService.submit(new SubmitHelper(i), null); } //wait for all threads to be done - for(int i = 0; i < 30; i++) { + for (int i = 0; i < 30; i++) { completionService.take(); } @@ -282,19 +280,20 @@ public void testMultipleClients() throws Exception { int index = 0; //30 batches of 10 - for(int i = 0; i < 30; i++) { - for(int j = 0; j < 10; j++) { + for (int i = 0; i < 30; i++) { + for (int j = 0; j < 10; j++) { Assert.assertEquals(i, events.get(index++).intValue()); } } } private class SubmitHelper implements Runnable { - private final int i; + public SubmitHelper(int i) { this.i = i; } + @Override public void run() { List events = Lists.newArrayList(); diff --git a/flume-ng-core/src/test/java/org/apache/flume/source/http/FlumeHttpServletRequestWrapper.java b/flume-ng-core/src/test/java/org/apache/flume/source/http/FlumeHttpServletRequestWrapper.java index 6b94b2ea3c..475d92fef7 100644 --- a/flume-ng-core/src/test/java/org/apache/flume/source/http/FlumeHttpServletRequestWrapper.java +++ b/flume-ng-core/src/test/java/org/apache/flume/source/http/FlumeHttpServletRequestWrapper.java @@ -40,7 +40,9 @@ class FlumeHttpServletRequestWrapper implements HttpServletRequest { private BufferedReader reader; String charset; - public FlumeHttpServletRequestWrapper(String data, String charset) throws UnsupportedEncodingException { + + public FlumeHttpServletRequestWrapper(String data, String charset) + throws UnsupportedEncodingException { reader = new BufferedReader(new InputStreamReader( new ByteArrayInputStream(data.getBytes(charset)), charset)); this.charset = charset; diff --git a/flume-ng-core/src/test/java/org/apache/flume/source/http/TestHTTPSource.java b/flume-ng-core/src/test/java/org/apache/flume/source/http/TestHTTPSource.java index c59fdd4b3b..3ad82829a5 100644 --- a/flume-ng-core/src/test/java/org/apache/flume/source/http/TestHTTPSource.java +++ b/flume-ng-core/src/test/java/org/apache/flume/source/http/TestHTTPSource.java @@ -22,7 +22,11 @@ import com.google.gson.Gson; import com.google.gson.reflect.TypeToken; import junit.framework.Assert; -import org.apache.flume.*; +import org.apache.flume.Channel; +import org.apache.flume.ChannelSelector; +import org.apache.flume.Context; +import org.apache.flume.Event; +import org.apache.flume.Transaction; import org.apache.flume.channel.ChannelProcessor; import org.apache.flume.channel.MemoryChannel; import org.apache.flume.channel.ReplicatingChannelSelector; @@ -41,11 +45,22 @@ import org.junit.BeforeClass; import org.junit.Test; -import javax.net.ssl.*; +import javax.net.ssl.HostnameVerifier; +import javax.net.ssl.HttpsURLConnection; +import javax.net.ssl.SSLContext; +import javax.net.ssl.SSLSession; +import javax.net.ssl.SSLSocket; +import javax.net.ssl.TrustManager; +import javax.net.ssl.X509TrustManager; import javax.servlet.http.HttpServletResponse; import java.io.IOException; import java.lang.reflect.Type; -import java.net.*; +import java.net.HttpURLConnection; +import java.net.InetAddress; +import java.net.ServerSocket; +import java.net.Socket; +import java.net.URL; +import java.net.UnknownHostException; import java.security.SecureRandom; import java.security.cert.CertificateException; import java.util.ArrayList; @@ -114,9 +129,10 @@ public static void setUpClass() throws Exception { sslContext.put(HTTPSourceConfigurationConstants.SSL_ENABLED, "true"); sslPort = findFreePort(); sslContext.put(HTTPSourceConfigurationConstants.CONFIG_PORT, - String.valueOf(sslPort)); + String.valueOf(sslPort)); sslContext.put(HTTPSourceConfigurationConstants.SSL_KEYSTORE_PASSWORD, "password"); - sslContext.put(HTTPSourceConfigurationConstants.SSL_KEYSTORE, "src/test/resources/jettykeystore"); + sslContext.put(HTTPSourceConfigurationConstants.SSL_KEYSTORE, + "src/test/resources/jettykeystore"); Configurables.configure(source, context); Configurables.configure(httpsSource, sslContext); @@ -180,7 +196,7 @@ public void testOptions() throws Exception { private void doTestForbidden(HttpRequestBase request) throws Exception { HttpResponse response = httpClient.execute(request); Assert.assertEquals(HttpServletResponse.SC_FORBIDDEN, - response.getStatusLine().getStatusCode()); + response.getStatusLine().getStatusCode()); } @Test @@ -286,10 +302,8 @@ public void testHandlerThrowingException() throws Exception { } - private ResultWrapper putWithEncoding(String encoding, int n) - throws Exception{ - Type listType = new TypeToken>() { - }.getType(); + private ResultWrapper putWithEncoding(String encoding, int n) throws Exception { + Type listType = new TypeToken>() {}.getType(); List events = Lists.newArrayList(); Random rand = new Random(); for (int i = 0; i < n; i++) { @@ -341,25 +355,25 @@ public void doTestHttps(String protocol) throws Exception { String json = gson.toJson(events, listType); HttpsURLConnection httpsURLConnection = null; try { - TrustManager[] trustAllCerts = {new X509TrustManager() { - @Override - public void checkClientTrusted( - java.security.cert.X509Certificate[] x509Certificates, String s) - throws CertificateException { - // noop + TrustManager[] trustAllCerts = { + new X509TrustManager() { + @Override + public void checkClientTrusted(java.security.cert.X509Certificate[] x509Certificates, + String s) throws CertificateException { + // noop + } + + @Override + public void checkServerTrusted(java.security.cert.X509Certificate[] x509Certificates, + String s) throws CertificateException { + // noop + } + + public java.security.cert.X509Certificate[] getAcceptedIssuers() { + return null; + } } - - @Override - public void checkServerTrusted( - java.security.cert.X509Certificate[] x509Certificates, String s) - throws CertificateException { - // noop - } - - public java.security.cert.X509Certificate[] getAcceptedIssuers() { - return null; - } - }}; + }; SSLContext sc = null; javax.net.ssl.SSLSocketFactory factory = null; @@ -376,14 +390,13 @@ public boolean verify(String arg0, SSLSession arg1) { }; sc.init(null, trustAllCerts, new SecureRandom()); - if(protocol != null) { + if (protocol != null) { factory = new DisabledProtocolsSocketFactory(sc.getSocketFactory(), protocol); } else { factory = sc.getSocketFactory(); } HttpsURLConnection.setDefaultSSLSocketFactory(factory); - HttpsURLConnection.setDefaultHostnameVerifier( - SSLSocketFactory.ALLOW_ALL_HOSTNAME_VERIFIER); + HttpsURLConnection.setDefaultHostnameVerifier(SSLSocketFactory.ALLOW_ALL_HOSTNAME_VERIFIER); URL sslUrl = new URL("https://0.0.0.0:" + sslPort); httpsURLConnection = (HttpsURLConnection) sslUrl.openConnection(); httpsURLConnection.setDoInput(true); @@ -396,14 +409,14 @@ public boolean verify(String arg0, SSLSession arg1) { Transaction transaction = channel.getTransaction(); transaction.begin(); - for(int i = 0; i < 10; i++) { + for (int i = 0; i < 10; i++) { Event e = channel.take(); Assert.assertNotNull(e); Assert.assertEquals(String.valueOf(i), e.getHeaders().get("MsgNum")); } - transaction.commit(); - transaction.close(); + transaction.commit(); + transaction.close(); } finally { httpsURLConnection.disconnect(); } @@ -416,38 +429,37 @@ public void testHttpsSourceNonHttpsClient() throws Exception { List events = Lists.newArrayList(); Random rand = new Random(); for (int i = 0; i < 10; i++) { - Map input = Maps.newHashMap(); - for (int j = 0; j < 10; j++) { - input.put(String.valueOf(i) + String.valueOf(j), String.valueOf(i)); - } - input.put("MsgNum", String.valueOf(i)); - JSONEvent e = new JSONEvent(); - e.setHeaders(input); - e.setBody(String.valueOf(rand.nextGaussian()).getBytes("UTF-8")); - events.add(e); + Map input = Maps.newHashMap(); + for (int j = 0; j < 10; j++) { + input.put(String.valueOf(i) + String.valueOf(j), String.valueOf(i)); + } + input.put("MsgNum", String.valueOf(i)); + JSONEvent e = new JSONEvent(); + e.setHeaders(input); + e.setBody(String.valueOf(rand.nextGaussian()).getBytes("UTF-8")); + events.add(e); } Gson gson = new Gson(); String json = gson.toJson(events, listType); HttpURLConnection httpURLConnection = null; try { - URL url = new URL("http://0.0.0.0:" + sslPort); - httpURLConnection = (HttpURLConnection) url.openConnection(); - httpURLConnection.setDoInput(true); - httpURLConnection.setDoOutput(true); - httpURLConnection.setRequestMethod("POST"); - httpURLConnection.getOutputStream().write(json.getBytes()); - httpURLConnection.getResponseCode(); - - Assert.fail("HTTP Client cannot connect to HTTPS source"); + URL url = new URL("http://0.0.0.0:" + sslPort); + httpURLConnection = (HttpURLConnection) url.openConnection(); + httpURLConnection.setDoInput(true); + httpURLConnection.setDoOutput(true); + httpURLConnection.setRequestMethod("POST"); + httpURLConnection.getOutputStream().write(json.getBytes()); + httpURLConnection.getResponseCode(); + + Assert.fail("HTTP Client cannot connect to HTTPS source"); } catch (Exception exception) { - Assert.assertTrue("Exception expected", true); + Assert.assertTrue("Exception expected", true); } finally { - httpURLConnection.disconnect(); + httpURLConnection.disconnect(); } } - private void takeWithEncoding(String encoding, int n, List events) - throws Exception{ + private void takeWithEncoding(String encoding, int n, List events) throws Exception { Transaction tx = channel.getTransaction(); tx.begin(); Event e = null; @@ -459,7 +471,7 @@ private void takeWithEncoding(String encoding, int n, List events) } Event current = events.get(i++); Assert.assertEquals(new String(current.getBody(), encoding), - new String(e.getBody(), encoding)); + new String(e.getBody(), encoding)); Assert.assertEquals(current.getHeaders(), e.getHeaders()); } Assert.assertEquals(n, events.size()); @@ -480,7 +492,8 @@ private void testBatchWithVariousEncoding(String encoding, int n) private class ResultWrapper { public final HttpResponse response; public final List events; - public ResultWrapper(HttpResponse resp, List events){ + + public ResultWrapper(HttpResponse resp, List events) { this.response = resp; this.events = events; } @@ -508,43 +521,39 @@ public String[] getSupportedCipherSuites() { } @Override - public Socket createSocket(Socket socket, String s, int i, boolean b) - throws IOException { + public Socket createSocket(Socket socket, String s, int i, boolean b) throws IOException { SSLSocket sc = (SSLSocket) socketFactory.createSocket(socket, s, i, b); sc.setEnabledProtocols(protocols); return sc; } @Override - public Socket createSocket(String s, int i) - throws IOException, UnknownHostException { - SSLSocket sc = (SSLSocket)socketFactory.createSocket(s, i); + public Socket createSocket(String s, int i) throws IOException, UnknownHostException { + SSLSocket sc = (SSLSocket) socketFactory.createSocket(s, i); sc.setEnabledProtocols(protocols); return sc; } @Override public Socket createSocket(String s, int i, InetAddress inetAddress, int i2) - throws IOException, UnknownHostException { - SSLSocket sc = (SSLSocket)socketFactory.createSocket(s, i, inetAddress, - i2); + throws IOException, UnknownHostException { + SSLSocket sc = (SSLSocket) socketFactory.createSocket(s, i, inetAddress, i2); sc.setEnabledProtocols(protocols); return sc; } @Override - public Socket createSocket(InetAddress inetAddress, int i) - throws IOException { - SSLSocket sc = (SSLSocket)socketFactory.createSocket(inetAddress, i); + public Socket createSocket(InetAddress inetAddress, int i) throws IOException { + SSLSocket sc = (SSLSocket) socketFactory.createSocket(inetAddress, i); sc.setEnabledProtocols(protocols); return sc; } @Override public Socket createSocket(InetAddress inetAddress, int i, - InetAddress inetAddress2, int i2) throws IOException { - SSLSocket sc = (SSLSocket)socketFactory.createSocket(inetAddress, i, - inetAddress2, i2); + InetAddress inetAddress2, int i2) throws IOException { + SSLSocket sc = (SSLSocket) socketFactory.createSocket(inetAddress, i, + inetAddress2, i2); sc.setEnabledProtocols(protocols); return sc; } diff --git a/flume-ng-core/src/test/java/org/apache/flume/tools/TestTimestampRoundDownUtil.java b/flume-ng-core/src/test/java/org/apache/flume/tools/TestTimestampRoundDownUtil.java index cc7eac015f..1ac11ab05f 100644 --- a/flume-ng-core/src/test/java/org/apache/flume/tools/TestTimestampRoundDownUtil.java +++ b/flume-ng-core/src/test/java/org/apache/flume/tools/TestTimestampRoundDownUtil.java @@ -18,7 +18,6 @@ package org.apache.flume.tools; - import java.util.Calendar; import junit.framework.Assert; @@ -38,8 +37,7 @@ public void testRoundDownTimeStampSeconds() { cal2.set(2012, 5, 15, 15, 12, 0); cal2.set(Calendar.MILLISECOND, 0); long timeToVerify = cal2.getTimeInMillis(); - long ret = TimestampRoundDownUtil. - roundDownTimeStampSeconds(cal.getTimeInMillis(), 60); + long ret = TimestampRoundDownUtil.roundDownTimeStampSeconds(cal.getTimeInMillis(), 60); System.out.println("Cal 1: " + cal.toString()); System.out.println("Cal 2: " + cal2.toString()); Assert.assertEquals(timeToVerify, ret); @@ -56,8 +54,7 @@ public void testRoundDownTimeStampMinutes() { cal2.set(2012, 5, 15, 15, 10, 0); cal2.set(Calendar.MILLISECOND, 0); long timeToVerify = cal2.getTimeInMillis(); - long ret = TimestampRoundDownUtil. - roundDownTimeStampMinutes(cal.getTimeInMillis(), 5); + long ret = TimestampRoundDownUtil.roundDownTimeStampMinutes(cal.getTimeInMillis(), 5); System.out.println("Cal 1: " + cal.toString()); System.out.println("Cal 2: " + cal2.toString()); Assert.assertEquals(timeToVerify, ret); @@ -74,8 +71,7 @@ public void testRoundDownTimeStampHours() { cal2.set(2012, 5, 15, 14, 0, 0); cal2.set(Calendar.MILLISECOND, 0); long timeToVerify = cal2.getTimeInMillis(); - long ret = TimestampRoundDownUtil. - roundDownTimeStampHours(cal.getTimeInMillis(), 2); + long ret = TimestampRoundDownUtil.roundDownTimeStampHours(cal.getTimeInMillis(), 2); System.out.println("Cal 1: " + ret); System.out.println("Cal 2: " + cal2.toString()); Assert.assertEquals(timeToVerify, ret); diff --git a/flume-ng-core/src/test/java/org/apache/flume/tools/TestVersionInfo.java b/flume-ng-core/src/test/java/org/apache/flume/tools/TestVersionInfo.java index b46389995c..0bdc82065e 100644 --- a/flume-ng-core/src/test/java/org/apache/flume/tools/TestVersionInfo.java +++ b/flume-ng-core/src/test/java/org/apache/flume/tools/TestVersionInfo.java @@ -51,8 +51,8 @@ public void testVersionInfoUnknown() { !VersionInfo.getSrcChecksum().equals("Unknown")); // check getBuildVersion() return format - assertTrue("getBuildVersion returned unexpected format",VersionInfo. - getBuildVersion().matches(".+from.+by.+on.+source checksum.+")); + assertTrue("getBuildVersion returned unexpected format", + VersionInfo.getBuildVersion().matches(".+from.+by.+on.+source checksum.+")); //"Unknown" when build without svn or git assertNotNull("getRevision returned null", VersionInfo.getRevision()); diff --git a/flume-ng-embedded-agent/src/test/java/org/apache/flume/agent/embedded/TestEmbeddedAgent.java b/flume-ng-embedded-agent/src/test/java/org/apache/flume/agent/embedded/TestEmbeddedAgent.java index 241e2b5264..032a4f8768 100644 --- a/flume-ng-embedded-agent/src/test/java/org/apache/flume/agent/embedded/TestEmbeddedAgent.java +++ b/flume-ng-embedded-agent/src/test/java/org/apache/flume/agent/embedded/TestEmbeddedAgent.java @@ -95,14 +95,14 @@ public void setUp() throws Exception { @After public void tearDown() throws Exception { - if(agent != null) { + if (agent != null) { try { agent.stop(); } catch (Exception e) { LOGGER.debug("Error shutting down agent", e); } } - if(nettyServer != null) { + if (nettyServer != null) { try { nettyServer.close(); } catch (Exception e) { @@ -118,7 +118,7 @@ public void testPut() throws Exception { agent.put(EventBuilder.withBody(body, headers)); Event event; - while((event = eventCollector.poll()) == null) { + while ((event = eventCollector.poll()) == null) { Thread.sleep(500L); } Assert.assertNotNull(event); @@ -135,7 +135,7 @@ public void testPutAll() throws Exception { agent.putAll(events); Event event; - while((event = eventCollector.poll()) == null) { + while ((event = eventCollector.poll()) == null) { Thread.sleep(500L); } Assert.assertNotNull(event); @@ -155,7 +155,7 @@ public void testPutWithInterceptors() throws Exception { agent.put(EventBuilder.withBody(body, headers)); Event event; - while((event = eventCollector.poll()) == null) { + while ((event = eventCollector.poll()) == null) { Thread.sleep(500L); } Assert.assertNotNull(event); @@ -176,13 +176,13 @@ public void testEmbeddedAgentName() throws Exception { embedAgent.putAll(events); Event event; - while((event = eventCollector.poll()) == null) { + while ((event = eventCollector.poll()) == null) { Thread.sleep(500L); } Assert.assertNotNull(event); Assert.assertArrayEquals(body, event.getBody()); Assert.assertEquals(headers, event.getHeaders()); - if(embedAgent != null) { + if (embedAgent != null) { try { embedAgent.stop(); } catch (Exception e) { @@ -191,14 +191,13 @@ public void testEmbeddedAgentName() throws Exception { } } - static class EventCollector implements AvroSourceProtocol { private final Queue eventQueue = new LinkedBlockingQueue(); public Event poll() { AvroFlumeEvent avroEvent = eventQueue.poll(); - if(avroEvent != null) { + if (avroEvent != null) { return EventBuilder.withBody(avroEvent.getBody().array(), toStringMap(avroEvent.getHeaders())); } @@ -216,10 +215,9 @@ public Status appendBatch(List events) return Status.OK; } } - private static Map toStringMap( - Map charSeqMap) { - Map stringMap = - new HashMap(); + + private static Map toStringMap(Map charSeqMap) { + Map stringMap = new HashMap(); for (Map.Entry entry : charSeqMap.entrySet()) { stringMap.put(entry.getKey().toString(), entry.getValue().toString()); } diff --git a/flume-ng-embedded-agent/src/test/java/org/apache/flume/agent/embedded/TestEmbeddedAgentConfiguration.java b/flume-ng-embedded-agent/src/test/java/org/apache/flume/agent/embedded/TestEmbeddedAgentConfiguration.java index f4a9a58cc9..ed26294288 100644 --- a/flume-ng-embedded-agent/src/test/java/org/apache/flume/agent/embedded/TestEmbeddedAgentConfiguration.java +++ b/flume-ng-embedded-agent/src/test/java/org/apache/flume/agent/embedded/TestEmbeddedAgentConfiguration.java @@ -34,8 +34,7 @@ public class TestEmbeddedAgentConfiguration { @Before public void setUp() throws Exception { properties = Maps.newHashMap(); - properties.put("source.type", EmbeddedAgentConfiguration. - SOURCE_TYPE_EMBEDDED); + properties.put("source.type", EmbeddedAgentConfiguration.SOURCE_TYPE_EMBEDDED); properties.put("channel.type", "memory"); properties.put("channel.capacity", "200"); properties.put("sinks", "sink1 sink2"); @@ -50,28 +49,23 @@ public void setUp() throws Exception { properties.put("source.interceptors.i1.type", "timestamp"); } - @Test public void testFullSourceType() throws Exception { - doTestExcepted(EmbeddedAgentConfiguration. - configure("test1", properties)); + doTestExcepted(EmbeddedAgentConfiguration.configure("test1", properties)); } @Test public void testMissingSourceType() throws Exception { Assert.assertNotNull(properties.remove("source.type")); - doTestExcepted(EmbeddedAgentConfiguration. - configure("test1", properties)); + doTestExcepted(EmbeddedAgentConfiguration.configure("test1", properties)); } @Test public void testShortSourceType() throws Exception { properties.put("source.type", "EMBEDDED"); - doTestExcepted(EmbeddedAgentConfiguration. - configure("test1", properties)); + doTestExcepted(EmbeddedAgentConfiguration.configure("test1", properties)); } - public void doTestExcepted(Map actual) throws Exception { Map expected = Maps.newHashMap(); expected.put("test1.channels", "channel-test1"); @@ -91,8 +85,8 @@ public void doTestExcepted(Map actual) throws Exception { expected.put("test1.sinks.sink2.type", "avro"); expected.put("test1.sources", "source-test1"); expected.put("test1.sources.source-test1.channels", "channel-test1"); - expected.put("test1.sources.source-test1.type", EmbeddedAgentConfiguration. - SOURCE_TYPE_EMBEDDED); + expected.put("test1.sources.source-test1.type", + EmbeddedAgentConfiguration.SOURCE_TYPE_EMBEDDED); expected.put("test1.sources.source-test1.interceptors", "i1"); expected.put("test1.sources.source-test1.interceptors.i1.type", "timestamp"); Assert.assertEquals(expected, actual); diff --git a/flume-ng-embedded-agent/src/test/java/org/apache/flume/agent/embedded/TestEmbeddedAgentEmbeddedSource.java b/flume-ng-embedded-agent/src/test/java/org/apache/flume/agent/embedded/TestEmbeddedAgentEmbeddedSource.java index 9d85e6e5c3..c122a12c4d 100644 --- a/flume-ng-embedded-agent/src/test/java/org/apache/flume/agent/embedded/TestEmbeddedAgentEmbeddedSource.java +++ b/flume-ng-embedded-agent/src/test/java/org/apache/flume/agent/embedded/TestEmbeddedAgentEmbeddedSource.java @@ -51,7 +51,6 @@ public class TestEmbeddedAgentEmbeddedSource { private Channel channel; private SinkRunner sinkRunner; - @Before public void setUp() throws Exception { @@ -81,26 +80,31 @@ public ImmutableMap getSourceRunners() { result.put("source", sourceRunner); return ImmutableMap.copyOf(result); } + @Override public ImmutableMap getSinkRunners() { Map result = Maps.newHashMap(); result.put("sink", sinkRunner); return ImmutableMap.copyOf(result); } + @Override public ImmutableMap getChannels() { Map result = Maps.newHashMap(); result.put("channel", channel); return ImmutableMap.copyOf(result); } + @Override public void addSourceRunner(String name, SourceRunner sourceRunner) { throw new UnsupportedOperationException(); } + @Override public void addSinkRunner(String name, SinkRunner sinkRunner) { throw new UnsupportedOperationException(); } + @Override public void addChannel(String name, Channel channel) { throw new UnsupportedOperationException(); @@ -122,7 +126,6 @@ public void testStart() { verify(sinkRunner, times(1)).start(); } - @Test public void testStop() { agent.configure(properties); @@ -138,16 +141,19 @@ public void testStartSourceThrowsException() { doThrow(new LocalRuntimeException()).when(sourceRunner).start(); startExpectingLocalRuntimeException(); } + @Test public void testStartChannelThrowsException() { doThrow(new LocalRuntimeException()).when(channel).start(); startExpectingLocalRuntimeException(); } + @Test public void testStartSinkThrowsException() { doThrow(new LocalRuntimeException()).when(sinkRunner).start(); startExpectingLocalRuntimeException(); } + private void startExpectingLocalRuntimeException() { agent.configure(properties); try { @@ -160,9 +166,11 @@ private void startExpectingLocalRuntimeException() { verify(channel, times(1)).stop(); verify(sinkRunner, times(1)).stop(); } + private static class LocalRuntimeException extends RuntimeException { private static final long serialVersionUID = 116546244849853151L; } + @Test public void testPut() throws EventDeliveryException { Event event = new SimpleEvent(); @@ -171,6 +179,7 @@ public void testPut() throws EventDeliveryException { agent.put(event); verify(source, times(1)).put(event); } + @Test public void testPutAll() throws EventDeliveryException { Event event = new SimpleEvent(); @@ -181,12 +190,14 @@ public void testPutAll() throws EventDeliveryException { agent.putAll(events); verify(source, times(1)).putAll(events); } + @Test(expected = IllegalStateException.class) public void testPutNotStarted() throws EventDeliveryException { Event event = new SimpleEvent(); agent.configure(properties); agent.put(event); } + @Test(expected = IllegalStateException.class) public void testPutAllNotStarted() throws EventDeliveryException { Event event = new SimpleEvent(); diff --git a/flume-ng-embedded-agent/src/test/java/org/apache/flume/agent/embedded/TestEmbeddedAgentState.java b/flume-ng-embedded-agent/src/test/java/org/apache/flume/agent/embedded/TestEmbeddedAgentState.java index a14a87ebd8..0f0ad23070 100644 --- a/flume-ng-embedded-agent/src/test/java/org/apache/flume/agent/embedded/TestEmbeddedAgentState.java +++ b/flume-ng-embedded-agent/src/test/java/org/apache/flume/agent/embedded/TestEmbeddedAgentState.java @@ -18,19 +18,19 @@ */ package org.apache.flume.agent.embedded; -import java.util.Map; - +import com.google.common.base.Throwables; +import com.google.common.collect.Maps; import org.apache.flume.FlumeException; import org.junit.Before; import org.junit.Test; -import com.google.common.base.Throwables; -import com.google.common.collect.Maps; +import java.util.Map; public class TestEmbeddedAgentState { private static final String HOSTNAME = "localhost"; private EmbeddedAgent agent; private Map properties; + @Before public void setUp() throws Exception { agent = new EmbeddedAgent("dummy"); @@ -47,13 +47,13 @@ public void setUp() throws Exception { properties.put("processor.type", "load_balance"); } - @Test(expected=FlumeException.class) + @Test(expected = FlumeException.class) public void testConfigureWithBadSourceType() { properties.put(EmbeddedAgentConfiguration.SOURCE_TYPE, "bad"); agent.configure(properties); } - @Test(expected=IllegalStateException.class) + @Test(expected = IllegalStateException.class) public void testConfigureWhileStarted() { try { agent.configure(properties); @@ -63,13 +63,14 @@ public void testConfigureWhileStarted() { } agent.configure(properties); } + @Test public void testConfigureMultipleTimes() { agent.configure(properties); agent.configure(properties); } - @Test(expected=IllegalStateException.class) + @Test(expected = IllegalStateException.class) public void testStartWhileStarted() { try { agent.configure(properties); @@ -79,15 +80,18 @@ public void testStartWhileStarted() { } agent.start(); } - @Test(expected=IllegalStateException.class) + + @Test(expected = IllegalStateException.class) public void testStartUnconfigured() { agent.start(); } - @Test(expected=IllegalStateException.class) + + @Test(expected = IllegalStateException.class) public void testStopBeforeConfigure() { agent.stop(); } - @Test(expected=IllegalStateException.class) + + @Test(expected = IllegalStateException.class) public void testStoppedWhileStopped() { try { agent.configure(properties); @@ -96,7 +100,8 @@ public void testStoppedWhileStopped() { } agent.stop(); } - @Test(expected=IllegalStateException.class) + + @Test(expected = IllegalStateException.class) public void testStopAfterStop() { try { agent.configure(properties); @@ -107,7 +112,8 @@ public void testStopAfterStop() { } agent.stop(); } - @Test(expected=IllegalStateException.class) + + @Test(expected = IllegalStateException.class) public void testStopAfterConfigure() { try { agent.configure(properties); diff --git a/flume-ng-legacy-sources/flume-avro-source/src/test/java/org/apache/flume/source/avroLegacy/TestLegacyAvroSource.java b/flume-ng-legacy-sources/flume-avro-source/src/test/java/org/apache/flume/source/avroLegacy/TestLegacyAvroSource.java index 6e3eb53a39..610aa64e64 100644 --- a/flume-ng-legacy-sources/flume-avro-source/src/test/java/org/apache/flume/source/avroLegacy/TestLegacyAvroSource.java +++ b/flume-ng-legacy-sources/flume-avro-source/src/test/java/org/apache/flume/source/avroLegacy/TestLegacyAvroSource.java @@ -19,13 +19,9 @@ package org.apache.flume.source.avroLegacy; -import java.io.IOException; -import java.net.URL; -import java.nio.ByteBuffer; -import java.util.ArrayList; -import java.util.HashMap; -import java.util.List; - +import com.cloudera.flume.handlers.avro.AvroFlumeOGEvent; +import com.cloudera.flume.handlers.avro.FlumeOGEventAvroServer; +import com.cloudera.flume.handlers.avro.Priority; import org.apache.avro.ipc.HttpTransceiver; import org.apache.avro.ipc.Transceiver; import org.apache.avro.ipc.specific.SpecificRequestor; @@ -40,9 +36,6 @@ import org.apache.flume.conf.Configurables; import org.apache.flume.lifecycle.LifecycleController; import org.apache.flume.lifecycle.LifecycleState; -import com.cloudera.flume.handlers.avro.AvroFlumeOGEvent; -import com.cloudera.flume.handlers.avro.FlumeOGEventAvroServer; -import com.cloudera.flume.handlers.avro.Priority; import org.jboss.netty.channel.ChannelException; import org.junit.Assert; import org.junit.Before; @@ -50,6 +43,12 @@ import org.slf4j.Logger; import org.slf4j.LoggerFactory; +import java.io.IOException; +import java.net.URL; +import java.nio.ByteBuffer; +import java.util.ArrayList; +import java.util.HashMap; +import java.util.List; public class TestLegacyAvroSource { @@ -143,10 +142,10 @@ public void testRequest() throws InterruptedException, IOException { FlumeOGEventAvroServer client = SpecificRequestor.getClient( FlumeOGEventAvroServer.class, http); - AvroFlumeOGEvent avroEvent = AvroFlumeOGEvent.newBuilder().setHost("foo"). - setPriority(Priority.INFO).setNanos(0).setTimestamp(1). - setFields(new HashMap ()). - setBody(ByteBuffer.wrap("foo".getBytes())).build(); + AvroFlumeOGEvent avroEvent = AvroFlumeOGEvent.newBuilder().setHost("foo") + .setPriority(Priority.INFO).setNanos(0).setTimestamp(1) + .setFields(new HashMap()) + .setBody(ByteBuffer.wrap("foo".getBytes())).build(); client.append(avroEvent); diff --git a/flume-ng-legacy-sources/flume-thrift-source/src/test/java/org/apache/flume/source/thriftLegacy/TestThriftLegacySource.java b/flume-ng-legacy-sources/flume-thrift-source/src/test/java/org/apache/flume/source/thriftLegacy/TestThriftLegacySource.java index d8a68721dd..f228ddefd9 100644 --- a/flume-ng-legacy-sources/flume-thrift-source/src/test/java/org/apache/flume/source/thriftLegacy/TestThriftLegacySource.java +++ b/flume-ng-legacy-sources/flume-thrift-source/src/test/java/org/apache/flume/source/thriftLegacy/TestThriftLegacySource.java @@ -19,15 +19,10 @@ package org.apache.flume.source.thriftLegacy; -import java.io.IOException; -import java.nio.ByteBuffer; -import java.util.ArrayList; -import java.util.HashMap; -import java.util.List; -import java.util.Map; - +import com.cloudera.flume.handlers.thrift.Priority; +import com.cloudera.flume.handlers.thrift.ThriftFlumeEvent; +import com.cloudera.flume.handlers.thrift.ThriftFlumeEventServer.Client; import org.apache.flume.Channel; -import org.apache.flume.ChannelException; import org.apache.flume.ChannelSelector; import org.apache.flume.Context; import org.apache.flume.Event; @@ -39,25 +34,27 @@ import org.apache.flume.conf.Configurables; import org.apache.flume.lifecycle.LifecycleController; import org.apache.flume.lifecycle.LifecycleState; - -import com.cloudera.flume.handlers.thrift.Priority; -import com.cloudera.flume.handlers.thrift.ThriftFlumeEvent; -import com.cloudera.flume.handlers.thrift.ThriftFlumeEventServer.Client; -//EventStatus.java Priority.java ThriftFlumeEvent.java ThriftFlumeEventServer.java - import org.apache.thrift.TException; import org.apache.thrift.protocol.TBinaryProtocol; import org.apache.thrift.protocol.TProtocol; import org.apache.thrift.transport.TSocket; import org.apache.thrift.transport.TTransport; import org.apache.thrift.transport.TTransportException; - import org.junit.Assert; import org.junit.Before; import org.junit.Test; import org.slf4j.Logger; import org.slf4j.LoggerFactory; +import java.io.IOException; +import java.nio.ByteBuffer; +import java.util.ArrayList; +import java.util.HashMap; +import java.util.List; +import java.util.Map; + +//EventStatus.java Priority.java ThriftFlumeEvent.java ThriftFlumeEventServer.java + public class TestThriftLegacySource { private static final Logger logger = LoggerFactory @@ -75,7 +72,8 @@ public FlumeClient(String host, int port) { this.host = host; this.port = port; } - public void append(ThriftFlumeEvent evt){ + + public void append(ThriftFlumeEvent evt) { TTransport transport; try { transport = new TSocket(host, port); diff --git a/flume-ng-node/src/test/java/org/apache/flume/node/TestAbstractConfigurationProvider.java b/flume-ng-node/src/test/java/org/apache/flume/node/TestAbstractConfigurationProvider.java index 15a478da1c..e27d8f7d53 100644 --- a/flume-ng-node/src/test/java/org/apache/flume/node/TestAbstractConfigurationProvider.java +++ b/flume-ng-node/src/test/java/org/apache/flume/node/TestAbstractConfigurationProvider.java @@ -6,9 +6,9 @@ * to you under the Apache License, Version 2.0 (the * "License"); you may not use this file except in compliance * with the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * + *

    + * http://www.apache.org/licenses/LICENSE-2.0 + *

    * Unless required by applicable law or agreed to in writing, software * distributed under the License is distributed on an "AS IS" BASIS, * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. @@ -17,10 +17,8 @@ */ package org.apache.flume.node; -import java.util.Map; - +import com.google.common.collect.Maps; import junit.framework.Assert; - import org.apache.flume.Channel; import org.apache.flume.ChannelException; import org.apache.flume.Context; @@ -36,7 +34,7 @@ import org.apache.flume.source.AbstractSource; import org.junit.Test; -import com.google.common.collect.Maps; +import java.util.Map; public class TestAbstractConfigurationProvider { @@ -44,7 +42,7 @@ public class TestAbstractConfigurationProvider { public void testDispoableChannel() throws Exception { String agentName = "agent1"; Map properties = getPropertiesForChannel(agentName, - DisposableChannel.class.getName()); + DisposableChannel.class.getName()); MemoryConfigurationProvider provider = new MemoryConfigurationProvider(agentName, properties); MaterializedConfiguration config1 = provider.getConfiguration(); @@ -60,7 +58,7 @@ public void testDispoableChannel() throws Exception { public void testReusableChannel() throws Exception { String agentName = "agent1"; Map properties = getPropertiesForChannel(agentName, - RecyclableChannel.class.getName()); + RecyclableChannel.class.getName()); MemoryConfigurationProvider provider = new MemoryConfigurationProvider(agentName, properties); @@ -79,7 +77,7 @@ public void testReusableChannel() throws Exception { public void testUnspecifiedChannel() throws Exception { String agentName = "agent1"; Map properties = getPropertiesForChannel(agentName, - UnspecifiedChannel.class.getName()); + UnspecifiedChannel.class.getName()); MemoryConfigurationProvider provider = new MemoryConfigurationProvider(agentName, properties); @@ -98,9 +96,11 @@ public void testUnspecifiedChannel() throws Exception { public void testReusableChannelNotReusedLater() throws Exception { String agentName = "agent1"; Map propertiesReusable = getPropertiesForChannel(agentName, - RecyclableChannel.class.getName()); + RecyclableChannel.class + .getName()); Map propertiesDispoable = getPropertiesForChannel(agentName, - DisposableChannel.class.getName()); + DisposableChannel.class + .getName()); MemoryConfigurationProvider provider = new MemoryConfigurationProvider(agentName, propertiesReusable); MaterializedConfiguration config1 = provider.getConfiguration(); @@ -127,7 +127,7 @@ public void testSourceThrowsExceptionDuringConfiguration() throws Exception { String channelType = "memory"; String sinkType = "null"; Map properties = getProperties(agentName, sourceType, - channelType, sinkType); + channelType, sinkType); MemoryConfigurationProvider provider = new MemoryConfigurationProvider(agentName, properties); MaterializedConfiguration config = provider.getConfiguration(); @@ -135,6 +135,7 @@ public void testSourceThrowsExceptionDuringConfiguration() throws Exception { Assert.assertTrue(config.getChannels().size() == 1); Assert.assertTrue(config.getSinkRunners().size() == 1); } + @Test public void testChannelThrowsExceptionDuringConfiguration() throws Exception { String agentName = "agent1"; @@ -142,7 +143,7 @@ public void testChannelThrowsExceptionDuringConfiguration() throws Exception { String channelType = UnconfigurableChannel.class.getName(); String sinkType = "null"; Map properties = getProperties(agentName, sourceType, - channelType, sinkType); + channelType, sinkType); MemoryConfigurationProvider provider = new MemoryConfigurationProvider(agentName, properties); MaterializedConfiguration config = provider.getConfiguration(); @@ -150,6 +151,7 @@ public void testChannelThrowsExceptionDuringConfiguration() throws Exception { Assert.assertTrue(config.getChannels().size() == 0); Assert.assertTrue(config.getSinkRunners().size() == 0); } + @Test public void testSinkThrowsExceptionDuringConfiguration() throws Exception { String agentName = "agent1"; @@ -157,7 +159,7 @@ public void testSinkThrowsExceptionDuringConfiguration() throws Exception { String channelType = "memory"; String sinkType = UnconfigurableSink.class.getName(); Map properties = getProperties(agentName, sourceType, - channelType, sinkType); + channelType, sinkType); MemoryConfigurationProvider provider = new MemoryConfigurationProvider(agentName, properties); MaterializedConfiguration config = provider.getConfiguration(); @@ -165,6 +167,7 @@ public void testSinkThrowsExceptionDuringConfiguration() throws Exception { Assert.assertTrue(config.getChannels().size() == 1); Assert.assertTrue(config.getSinkRunners().size() == 0); } + @Test public void testSourceAndSinkThrowExceptionDuringConfiguration() throws Exception { @@ -173,7 +176,7 @@ public void testSourceAndSinkThrowExceptionDuringConfiguration() String channelType = "memory"; String sinkType = UnconfigurableSink.class.getName(); Map properties = getProperties(agentName, sourceType, - channelType, sinkType); + channelType, sinkType); MemoryConfigurationProvider provider = new MemoryConfigurationProvider(agentName, properties); MaterializedConfiguration config = provider.getConfiguration(); @@ -181,8 +184,10 @@ public void testSourceAndSinkThrowExceptionDuringConfiguration() Assert.assertTrue(config.getChannels().size() == 0); Assert.assertTrue(config.getSinkRunners().size() == 0); } + private Map getProperties(String agentName, - String sourceType, String channelType, String sinkType) { + String sourceType, String channelType, + String sinkType) { Map properties = Maps.newHashMap(); properties.put(agentName + ".sources", "source1"); properties.put(agentName + ".channels", "channel1"); @@ -195,12 +200,14 @@ private Map getProperties(String agentName, properties.put(agentName + ".sinks.sink1.channel", "channel1"); return properties; } + private Map getPropertiesForChannel(String agentName, String channelType) { return getProperties(agentName, "seq", channelType, "null"); } public static class MemoryConfigurationProvider extends AbstractConfigurationProvider { private Map properties; + public MemoryConfigurationProvider(String agentName, Map properties) { super(agentName); this.properties = properties; @@ -215,81 +222,95 @@ protected FlumeConfiguration getFlumeConfiguration() { return new FlumeConfiguration(properties); } } + @Disposable public static class DisposableChannel extends AbstractChannel { @Override public void put(Event event) throws ChannelException { throw new UnsupportedOperationException(); } + @Override public Event take() throws ChannelException { throw new UnsupportedOperationException(); - } + } + @Override public Transaction getTransaction() { throw new UnsupportedOperationException(); } } + @Recyclable public static class RecyclableChannel extends AbstractChannel { @Override public void put(Event event) throws ChannelException { throw new UnsupportedOperationException(); } + @Override public Event take() throws ChannelException { throw new UnsupportedOperationException(); - } + } + @Override public Transaction getTransaction() { throw new UnsupportedOperationException(); } } + public static class UnspecifiedChannel extends AbstractChannel { @Override public void put(Event event) throws ChannelException { throw new UnsupportedOperationException(); } + @Override public Event take() throws ChannelException { throw new UnsupportedOperationException(); - } + } + @Override public Transaction getTransaction() { throw new UnsupportedOperationException(); } } + public static class UnconfigurableChannel extends AbstractChannel { @Override public void configure(Context context) { throw new RuntimeException("expected"); } + @Override public void put(Event event) throws ChannelException { throw new UnsupportedOperationException(); } + @Override public Event take() throws ChannelException { throw new UnsupportedOperationException(); - } + } + @Override public Transaction getTransaction() { throw new UnsupportedOperationException(); } } - public static class UnconfigurableSource extends AbstractSource - implements Configurable { + + public static class UnconfigurableSource extends AbstractSource implements Configurable { @Override public void configure(Context context) { throw new RuntimeException("expected"); } } - public static class UnconfigurableSink extends AbstractSink - implements Configurable { + + public static class UnconfigurableSink extends AbstractSink implements Configurable { @Override public void configure(Context context) { throw new RuntimeException("expected"); } + @Override public Status process() throws EventDeliveryException { throw new UnsupportedOperationException(); diff --git a/flume-ng-node/src/test/java/org/apache/flume/node/TestAbstractZooKeeperConfigurationProvider.java b/flume-ng-node/src/test/java/org/apache/flume/node/TestAbstractZooKeeperConfigurationProvider.java index 1ab412753e..2e306349bf 100644 --- a/flume-ng-node/src/test/java/org/apache/flume/node/TestAbstractZooKeeperConfigurationProvider.java +++ b/flume-ng-node/src/test/java/org/apache/flume/node/TestAbstractZooKeeperConfigurationProvider.java @@ -18,15 +18,10 @@ package org.apache.flume.node; -import java.io.InputStreamReader; -import java.io.Reader; -import java.util.Collections; -import java.util.List; -import java.util.Set; - import com.google.common.base.Charsets; +import com.google.common.collect.Lists; +import com.google.common.collect.Sets; import junit.framework.Assert; - import org.apache.commons.io.IOUtils; import org.apache.curator.framework.CuratorFramework; import org.apache.curator.framework.CuratorFrameworkFactory; @@ -38,8 +33,11 @@ import org.junit.After; import org.junit.Before; -import com.google.common.collect.Lists; -import com.google.common.collect.Sets; +import java.io.InputStreamReader; +import java.io.Reader; +import java.util.Collections; +import java.util.List; +import java.util.Set; public abstract class TestAbstractZooKeeperConfigurationProvider { @@ -48,8 +46,7 @@ public abstract class TestAbstractZooKeeperConfigurationProvider { protected static final String AGENT_NAME = "a1"; protected static final String AGENT_PATH = - AbstractZooKeeperConfigurationProvider.DEFAULT_ZK_BASE_PATH - + "/" + AGENT_NAME; + AbstractZooKeeperConfigurationProvider.DEFAULT_ZK_BASE_PATH + "/" + AGENT_NAME; protected TestingServer zkServer; protected CuratorFramework client; @@ -112,10 +109,8 @@ protected void verifyProperties(AbstractConfigurationProvider cp) { expected.add("host2 PROPERTY_VALUE_NULL"); expected.add("host2 AGENT_CONFIGURATION_INVALID"); List actual = Lists.newArrayList(); - for (FlumeConfigurationError error : configuration - .getConfigurationErrors()) { - actual.add(error.getComponentName() + " " - + error.getErrorType().toString()); + for (FlumeConfigurationError error : configuration.getConfigurationErrors()) { + actual.add(error.getComponentName() + " " + error.getErrorType().toString()); } Collections.sort(expected); Collections.sort(actual); diff --git a/flume-ng-node/src/test/java/org/apache/flume/node/TestApplication.java b/flume-ng-node/src/test/java/org/apache/flume/node/TestApplication.java index 930f2a288e..affbd8c131 100644 --- a/flume-ng-node/src/test/java/org/apache/flume/node/TestApplication.java +++ b/flume-ng-node/src/test/java/org/apache/flume/node/TestApplication.java @@ -44,13 +44,13 @@ public class TestApplication { - private File baseDir; @Before public void setup() throws Exception { baseDir = Files.createTempDir(); } + @After public void tearDown() throws Exception { FileUtils.deleteDirectory(baseDir); diff --git a/flume-ng-node/src/test/java/org/apache/flume/node/TestPollingPropertiesFileConfigurationProvider.java b/flume-ng-node/src/test/java/org/apache/flume/node/TestPollingPropertiesFileConfigurationProvider.java index eed22eebeb..480f6a5320 100644 --- a/flume-ng-node/src/test/java/org/apache/flume/node/TestPollingPropertiesFileConfigurationProvider.java +++ b/flume-ng-node/src/test/java/org/apache/flume/node/TestPollingPropertiesFileConfigurationProvider.java @@ -36,7 +36,6 @@ public class TestPollingPropertiesFileConfigurationProvider { - private static final File TESTFILE = new File( TestPollingPropertiesFileConfigurationProvider.class.getClassLoader() .getResource("flume-conf.properties").getFile()); diff --git a/flume-ng-node/src/test/java/org/apache/flume/node/TestPropertiesFileConfigurationProvider.java b/flume-ng-node/src/test/java/org/apache/flume/node/TestPropertiesFileConfigurationProvider.java index 84a8cfdcad..4875c56890 100644 --- a/flume-ng-node/src/test/java/org/apache/flume/node/TestPropertiesFileConfigurationProvider.java +++ b/flume-ng-node/src/test/java/org/apache/flume/node/TestPropertiesFileConfigurationProvider.java @@ -17,13 +17,9 @@ */ package org.apache.flume.node; -import java.io.File; -import java.util.Collections; -import java.util.List; -import java.util.Set; - +import com.google.common.collect.Lists; +import com.google.common.collect.Sets; import junit.framework.Assert; - import org.apache.flume.conf.FlumeConfiguration; import org.apache.flume.conf.FlumeConfiguration.AgentConfiguration; import org.apache.flume.conf.FlumeConfigurationError; @@ -33,14 +29,15 @@ import org.slf4j.Logger; import org.slf4j.LoggerFactory; -import com.google.common.collect.Lists; -import com.google.common.collect.Sets; +import java.io.File; +import java.util.Collections; +import java.util.List; +import java.util.Set; public class TestPropertiesFileConfigurationProvider { - - private static final Logger LOGGER = LoggerFactory - .getLogger(TestPropertiesFileConfigurationProvider.class); + private static final Logger LOGGER = + LoggerFactory.getLogger(TestPropertiesFileConfigurationProvider.class); private static final File TESTFILE = new File( TestPropertiesFileConfigurationProvider.class.getClassLoader() @@ -83,23 +80,20 @@ public void testPropertyRead() throws Exception { expected.add("host2 PROPERTY_VALUE_NULL"); expected.add("host2 AGENT_CONFIGURATION_INVALID"); List actual = Lists.newArrayList(); - for(FlumeConfigurationError error : configuration.getConfigurationErrors()) { + for (FlumeConfigurationError error : configuration.getConfigurationErrors()) { actual.add(error.getComponentName() + " " + error.getErrorType().toString()); } Collections.sort(expected); Collections.sort(actual); Assert.assertEquals(expected, actual); - AgentConfiguration agentConfiguration = configuration.getConfigurationFor("host1"); Assert.assertNotNull(agentConfiguration); - LOGGER.info(agentConfiguration.getPrevalidationConfig()); LOGGER.info(agentConfiguration.getPostvalidationConfig()); - Set sources = Sets.newHashSet("source1"); Set sinks = Sets.newHashSet("sink1"); Set channels = Sets.newHashSet("channel1"); diff --git a/flume-ng-node/src/test/java/org/apache/flume/source/TestNetcatSource.java b/flume-ng-node/src/test/java/org/apache/flume/source/TestNetcatSource.java index 91fbf6321b..a597a31e7b 100644 --- a/flume-ng-node/src/test/java/org/apache/flume/source/TestNetcatSource.java +++ b/flume-ng-node/src/test/java/org/apache/flume/source/TestNetcatSource.java @@ -19,19 +19,6 @@ package org.apache.flume.source; -import java.io.BufferedReader; -import java.io.IOException; -import java.io.Writer; -import java.net.InetSocketAddress; -import java.nio.channels.Channels; -import java.nio.channels.SocketChannel; -import java.util.List; -import java.util.concurrent.ExecutorService; -import java.util.concurrent.Executors; -import java.util.concurrent.TimeUnit; -import java.util.Collection; -import java.util.Arrays; - import com.google.common.collect.Lists; import org.apache.flume.Channel; import org.apache.flume.ChannelSelector; @@ -49,12 +36,25 @@ import org.junit.Assert; import org.junit.Before; import org.junit.Test; +import org.junit.runner.RunWith; import org.junit.runners.Parameterized; import org.junit.runners.Parameterized.Parameters; -import org.junit.runner.RunWith; import org.slf4j.Logger; import org.slf4j.LoggerFactory; +import java.io.BufferedReader; +import java.io.IOException; +import java.io.Writer; +import java.net.InetSocketAddress; +import java.nio.channels.Channels; +import java.nio.channels.SocketChannel; +import java.util.Arrays; +import java.util.Collection; +import java.util.List; +import java.util.concurrent.ExecutorService; +import java.util.concurrent.Executors; +import java.util.concurrent.TimeUnit; + @RunWith(value = Parameterized.class) public class TestNetcatSource { @@ -72,7 +72,7 @@ public TestNetcatSource(boolean ackForEveryEvent) { @Parameters public static Collection data() { Object[][] data = new Object[][] { { true }, { false } }; - return Arrays.asList(data); + return Arrays.asList(data); } @Before @@ -99,7 +99,7 @@ public void testLifecycle() throws InterruptedException, LifecycleException, ExecutorService executor = Executors.newFixedThreadPool(3); boolean bound = false; - for(int i = 0; i < 100 && !bound; i++) { + for (int i = 0; i < 100 && !bound; i++) { try { Context context = new Context(); context.put("bind", "0.0.0.0"); @@ -131,10 +131,10 @@ public void run() { writer.flush(); if (ackEveryEvent) { - String response = reader.readLine(); - Assert.assertEquals("Server should return OK", "OK", response); + String response = reader.readLine(); + Assert.assertEquals("Server should return OK", "OK", response); } else { - Assert.assertFalse("Server should not return anything", reader.ready()); + Assert.assertFalse("Server should not return anything", reader.ready()); } clientChannel.close(); } catch (IOException e) { diff --git a/flume-ng-sdk/src/test/java/org/apache/flume/api/RpcTestUtils.java b/flume-ng-sdk/src/test/java/org/apache/flume/api/RpcTestUtils.java index 88068605b8..d9355f7f74 100644 --- a/flume-ng-sdk/src/test/java/org/apache/flume/api/RpcTestUtils.java +++ b/flume-ng-sdk/src/test/java/org/apache/flume/api/RpcTestUtils.java @@ -18,13 +18,6 @@ */ package org.apache.flume.api; -import java.net.InetSocketAddress; -import java.nio.charset.Charset; -import java.util.ArrayList; -import java.util.List; -import java.util.Properties; -import java.util.concurrent.Executors; - import junit.framework.Assert; import org.apache.avro.AvroRemoteException; import org.apache.avro.ipc.NettyServer; @@ -47,6 +40,13 @@ import org.slf4j.Logger; import org.slf4j.LoggerFactory; +import java.net.InetSocketAddress; +import java.nio.charset.Charset; +import java.util.ArrayList; +import java.util.List; +import java.util.Properties; +import java.util.concurrent.Executors; + /** * Helpers for Netty Avro RPC testing */ @@ -75,7 +75,9 @@ public static void handlerSimpleAppendTest(AvroSourceProtocol handler) * @throws FlumeException * @throws EventDeliveryException */ - public static void handlerSimpleAppendTest(AvroSourceProtocol handler, boolean enableServerCompression, boolean enableClientCompression, int compressionLevel) + public static void handlerSimpleAppendTest(AvroSourceProtocol handler, + boolean enableServerCompression, + boolean enableClientCompression, int compressionLevel) throws FlumeException, EventDeliveryException { NettyAvroRpcClient client = null; Server server = startServer(handler, 0, enableServerCompression); @@ -83,7 +85,8 @@ public static void handlerSimpleAppendTest(AvroSourceProtocol handler, boolean e Properties starterProp = new Properties(); if (enableClientCompression) { starterProp.setProperty(RpcClientConfigurationConstants.CONFIG_COMPRESSION_TYPE, "deflate"); - starterProp.setProperty(RpcClientConfigurationConstants.CONFIG_COMPRESSION_LEVEL, "" + compressionLevel); + starterProp.setProperty(RpcClientConfigurationConstants.CONFIG_COMPRESSION_LEVEL, + "" + compressionLevel); } else { starterProp.setProperty(RpcClientConfigurationConstants.CONFIG_COMPRESSION_TYPE, "none"); } @@ -108,7 +111,9 @@ public static void handlerBatchAppendTest(AvroSourceProtocol handler) * @throws FlumeException * @throws EventDeliveryException */ - public static void handlerBatchAppendTest(AvroSourceProtocol handler, boolean enableServerCompression, boolean enableClientCompression, int compressionLevel) + public static void handlerBatchAppendTest(AvroSourceProtocol handler, + boolean enableServerCompression, + boolean enableClientCompression, int compressionLevel) throws FlumeException, EventDeliveryException { NettyAvroRpcClient client = null; Server server = startServer(handler, 0 , enableServerCompression); @@ -117,7 +122,8 @@ public static void handlerBatchAppendTest(AvroSourceProtocol handler, boolean en Properties starterProp = new Properties(); if (enableClientCompression) { starterProp.setProperty(RpcClientConfigurationConstants.CONFIG_COMPRESSION_TYPE, "deflate"); - starterProp.setProperty(RpcClientConfigurationConstants.CONFIG_COMPRESSION_LEVEL, "" + compressionLevel); + starterProp.setProperty(RpcClientConfigurationConstants.CONFIG_COMPRESSION_LEVEL, + "" + compressionLevel); } else { starterProp.setProperty(RpcClientConfigurationConstants.CONFIG_COMPRESSION_TYPE, "none"); } @@ -161,28 +167,24 @@ public static NettyAvroRpcClient getStockLocalClient(int port, Properties starte /** * Start a NettyServer, wait a moment for it to spin up, and return it. */ - public static Server startServer(AvroSourceProtocol handler, int port, boolean enableCompression) { - Responder responder = new SpecificResponder(AvroSourceProtocol.class, - handler); + public static Server startServer(AvroSourceProtocol handler, int port, + boolean enableCompression) { + Responder responder = new SpecificResponder(AvroSourceProtocol.class, handler); Server server; if (enableCompression) { - server = new NettyServer(responder, - new InetSocketAddress(localhost, port), - new NioServerSocketChannelFactory - (Executors .newCachedThreadPool(), Executors.newCachedThreadPool()), - new CompressionChannelPipelineFactory(), null); + server = new NettyServer(responder, new InetSocketAddress(localhost, port), + new NioServerSocketChannelFactory(Executors.newCachedThreadPool(), + Executors.newCachedThreadPool()), + new CompressionChannelPipelineFactory(), null); } else { - server = new NettyServer(responder, - new InetSocketAddress(localhost, port)); + server = new NettyServer(responder, new InetSocketAddress(localhost, port)); } server.start(); logger.info("Server started on hostname: {}, port: {}", - new Object[] { localhost, Integer.toString(server.getPort()) }); + new Object[] { localhost, Integer.toString(server.getPort()) }); try { - Thread.sleep(300L); - } catch (InterruptedException ex) { logger.error("Thread interrupted. Exception follows.", ex); Thread.currentThread().interrupt(); @@ -298,15 +300,13 @@ public static class FailedAvroHandler implements AvroSourceProtocol { @Override public Status append(AvroFlumeEvent event) throws AvroRemoteException { logger.info("Failed: Received event from append(): {}", - new String(event.getBody().array(), Charset.forName("UTF8"))); + new String(event.getBody().array(), Charset.forName("UTF8"))); return Status.FAILED; } @Override - public Status appendBatch(List events) throws - AvroRemoteException { - logger.info("Failed: Received {} events from appendBatch()", - events.size()); + public Status appendBatch(List events) throws AvroRemoteException { + logger.info("Failed: Received {} events from appendBatch()", events.size()); return Status.FAILED; } @@ -320,15 +320,14 @@ public static class UnknownAvroHandler implements AvroSourceProtocol { @Override public Status append(AvroFlumeEvent event) throws AvroRemoteException { logger.info("Unknown: Received event from append(): {}", - new String(event.getBody().array(), Charset.forName("UTF8"))); + new String(event.getBody().array(), Charset.forName("UTF8"))); return Status.UNKNOWN; } @Override - public Status appendBatch(List events) throws - AvroRemoteException { + public Status appendBatch(List events) throws AvroRemoteException { logger.info("Unknown: Received {} events from appendBatch()", - events.size()); + events.size()); return Status.UNKNOWN; } @@ -342,22 +341,18 @@ public static class ThrowingAvroHandler implements AvroSourceProtocol { @Override public Status append(AvroFlumeEvent event) throws AvroRemoteException { logger.info("Throwing: Received event from append(): {}", - new String(event.getBody().array(), Charset.forName("UTF8"))); + new String(event.getBody().array(), Charset.forName("UTF8"))); throw new AvroRemoteException("Handler smash!"); } @Override - public Status appendBatch(List events) throws - AvroRemoteException { - logger.info("Throwing: Received {} events from appendBatch()", - events.size()); + public Status appendBatch(List events) throws AvroRemoteException { + logger.info("Throwing: Received {} events from appendBatch()", events.size()); throw new AvroRemoteException("Handler smash!"); } - } - private static class CompressionChannelPipelineFactory implements - ChannelPipelineFactory { + private static class CompressionChannelPipelineFactory implements ChannelPipelineFactory { @Override public ChannelPipeline getPipeline() throws Exception { diff --git a/flume-ng-sdk/src/test/java/org/apache/flume/api/TestFailoverRpcClient.java b/flume-ng-sdk/src/test/java/org/apache/flume/api/TestFailoverRpcClient.java index 64dc181bfa..c3eb205aba 100644 --- a/flume-ng-sdk/src/test/java/org/apache/flume/api/TestFailoverRpcClient.java +++ b/flume-ng-sdk/src/test/java/org/apache/flume/api/TestFailoverRpcClient.java @@ -113,9 +113,7 @@ public void testFailover() server5.close(); Thread.sleep(1000L); // wait a second for the close to occur Server server6 = RpcTestUtils.startServer(new OKAvroHandler(), s1Port); - client - .append(EventBuilder.withBody("Had a whole watermelon?", - Charset.forName("UTF8"))); + client.append(EventBuilder.withBody("Had a whole watermelon?", Charset.forName("UTF8"))); Assert.assertEquals(new InetSocketAddress("localhost", s1Port), client.getLastConnectedServerAddress()); diff --git a/flume-ng-sdk/src/test/java/org/apache/flume/api/TestLoadBalancingRpcClient.java b/flume-ng-sdk/src/test/java/org/apache/flume/api/TestLoadBalancingRpcClient.java index 5d6828b065..dc53d3fe7f 100644 --- a/flume-ng-sdk/src/test/java/org/apache/flume/api/TestLoadBalancingRpcClient.java +++ b/flume-ng-sdk/src/test/java/org/apache/flume/api/TestLoadBalancingRpcClient.java @@ -18,14 +18,7 @@ */ package org.apache.flume.api; -import java.util.ArrayList; -import java.util.HashSet; -import java.util.List; -import java.util.Properties; -import java.util.Set; - import junit.framework.Assert; - import org.apache.avro.ipc.Server; import org.apache.flume.Event; import org.apache.flume.EventDeliveryException; @@ -37,12 +30,16 @@ import org.slf4j.Logger; import org.slf4j.LoggerFactory; -public class TestLoadBalancingRpcClient { - private static final Logger LOGGER = LoggerFactory - .getLogger(TestLoadBalancingRpcClient.class); +import java.util.ArrayList; +import java.util.HashSet; +import java.util.List; +import java.util.Properties; +import java.util.Set; +public class TestLoadBalancingRpcClient { + private static final Logger LOGGER = LoggerFactory.getLogger(TestLoadBalancingRpcClient.class); - @Test(expected=FlumeException.class) + @Test(expected = FlumeException.class) public void testCreatingLbClientSingleHost() { Server server1 = null; RpcClient c = null; @@ -61,9 +58,10 @@ public void testCreatingLbClientSingleHost() { @Test public void testTwoHostFailover() throws Exception { - Server s1 = null, s2 = null; + Server s1 = null; + Server s2 = null; RpcClient c = null; - try{ + try { LoadBalancedAvroHandler h1 = new LoadBalancedAvroHandler(); LoadBalancedAvroHandler h2 = new LoadBalancedAvroHandler(); @@ -100,9 +98,10 @@ public void testTwoHostFailover() throws Exception { // This will fail without FLUME-1823 @Test(expected = EventDeliveryException.class) public void testTwoHostFailoverThrowAfterClose() throws Exception { - Server s1 = null, s2 = null; + Server s1 = null; + Server s2 = null; RpcClient c = null; - try{ + try { LoadBalancedAvroHandler h1 = new LoadBalancedAvroHandler(); LoadBalancedAvroHandler h2 = new LoadBalancedAvroHandler(); @@ -140,13 +139,15 @@ public void testTwoHostFailoverThrowAfterClose() throws Exception { /** * Ensure that we can tolerate a host that is completely down. + * * @throws Exception */ @Test public void testTwoHostsOneDead() throws Exception { LOGGER.info("Running testTwoHostsOneDead..."); Server s1 = null; - RpcClient c1 = null, c2 = null; + RpcClient c1 = null; + RpcClient c2 = null; try { LoadBalancedAvroHandler h1 = new LoadBalancedAvroHandler(); s1 = RpcTestUtils.startServer(h1); @@ -186,9 +187,10 @@ public void testTwoHostsOneDead() throws Exception { @Test public void testTwoHostFailoverBatch() throws Exception { - Server s1 = null, s2 = null; + Server s1 = null; + Server s2 = null; RpcClient c = null; - try{ + try { LoadBalancedAvroHandler h1 = new LoadBalancedAvroHandler(); LoadBalancedAvroHandler h2 = new LoadBalancedAvroHandler(); @@ -225,9 +227,10 @@ public void testTwoHostFailoverBatch() throws Exception { @Test public void testLbDefaultClientTwoHosts() throws Exception { - Server s1 = null, s2 = null; + Server s1 = null; + Server s2 = null; RpcClient c = null; - try{ + try { LoadBalancedAvroHandler h1 = new LoadBalancedAvroHandler(); LoadBalancedAvroHandler h2 = new LoadBalancedAvroHandler(); @@ -258,9 +261,10 @@ public void testLbDefaultClientTwoHosts() throws Exception { @Test public void testLbDefaultClientTwoHostsBatch() throws Exception { - Server s1 = null, s2 = null; + Server s1 = null; + Server s2 = null; RpcClient c = null; - try{ + try { LoadBalancedAvroHandler h1 = new LoadBalancedAvroHandler(); LoadBalancedAvroHandler h2 = new LoadBalancedAvroHandler(); @@ -296,10 +300,10 @@ public void testLbClientTenHostRandomDistribution() throws Exception { Server[] s = new Server[NUM_HOSTS]; LoadBalancedAvroHandler[] h = new LoadBalancedAvroHandler[NUM_HOSTS]; RpcClient c = null; - try{ + try { Properties p = new Properties(); StringBuilder hostList = new StringBuilder(""); - for (int i = 0; i 2); Assert.assertTrue("Missing events", total == NUM_EVENTS); } finally { - for (int i = 0; i 2); Assert.assertTrue("Missing events", total == NUM_EVENTS); } finally { - for (int i = 0; i hosts = - new ArrayList(); + new ArrayList(); List servers = new ArrayList(); StringBuilder hostList = new StringBuilder(""); - for(int i = 0; i < 3;i++){ + for (int i = 0; i < 3; i++) { LoadBalancedAvroHandler s = new LoadBalancedAvroHandler(); hosts.add(s); Server srv = RpcTestUtils.startServer(s); @@ -499,7 +502,7 @@ public void testRandomBackoff() throws Exception { // TODO: there is a remote possibility that s0 or s2 // never get hit by the random assignment // and thus not backoffed, causing the test to fail - for(int i=0; i < 50; i++) { + for (int i = 0; i < 50; i++) { // a well behaved runner would always check the return. c.append(EventBuilder.withBody(("test" + String.valueOf(i)).getBytes())); } @@ -525,11 +528,12 @@ public void testRandomBackoff() throws Exception { Assert.assertEquals(50, hosts.get(1).getAppendCount()); Assert.assertEquals(0, hosts.get(2).getAppendCount()); } + @Test public void testRoundRobinBackoffInitialFailure() throws EventDeliveryException { Properties p = new Properties(); List hosts = - new ArrayList(); + new ArrayList(); List servers = new ArrayList(); StringBuilder hostList = new StringBuilder(""); for (int i = 0; i < 3; i++) { @@ -572,13 +576,13 @@ public void testRoundRobinBackoffInitialFailure() throws EventDeliveryException public void testRoundRobinBackoffIncreasingBackoffs() throws Exception { Properties p = new Properties(); List hosts = - new ArrayList(); + new ArrayList(); List servers = new ArrayList(); StringBuilder hostList = new StringBuilder(""); for (int i = 0; i < 3; i++) { LoadBalancedAvroHandler s = new LoadBalancedAvroHandler(); hosts.add(s); - if(i == 1) { + if (i == 1) { s.setFailed(); } Server srv = RpcTestUtils.startServer(s); @@ -620,16 +624,17 @@ public void testRoundRobinBackoffIncreasingBackoffs() throws Exception { c.append(EventBuilder.withBody("testing".getBytes())); } - Assert.assertEquals( 2 + 2 + 1 + (numEvents/3), hosts.get(0).getAppendCount()); - Assert.assertEquals((numEvents/3), hosts.get(1).getAppendCount()); - Assert.assertEquals(1 + 1 + 2 + (numEvents/3), hosts.get(2).getAppendCount()); + Assert.assertEquals(2 + 2 + 1 + (numEvents / 3), hosts.get(0).getAppendCount()); + Assert.assertEquals((numEvents / 3), hosts.get(1).getAppendCount()); + Assert.assertEquals(1 + 1 + 2 + (numEvents / 3), hosts.get(2).getAppendCount()); } @Test - public void testRoundRobinBackoffFailureRecovery() throws EventDeliveryException, InterruptedException { + public void testRoundRobinBackoffFailureRecovery() + throws EventDeliveryException, InterruptedException { Properties p = new Properties(); List hosts = - new ArrayList(); + new ArrayList(); List servers = new ArrayList(); StringBuilder hostList = new StringBuilder(""); for (int i = 0; i < 3; i++) { @@ -660,13 +665,13 @@ public void testRoundRobinBackoffFailureRecovery() throws EventDeliveryException Thread.sleep(3000); int numEvents = 60; - for(int i = 0; i < numEvents; i++){ + for (int i = 0; i < numEvents; i++) { c.append(EventBuilder.withBody("testing".getBytes())); } - Assert.assertEquals(2 + (numEvents/3) , hosts.get(0).getAppendCount()); - Assert.assertEquals(0 + (numEvents/3), hosts.get(1).getAppendCount()); - Assert.assertEquals(1 + (numEvents/3), hosts.get(2).getAppendCount()); + Assert.assertEquals(2 + (numEvents / 3), hosts.get(0).getAppendCount()); + Assert.assertEquals(0 + (numEvents / 3), hosts.get(1).getAppendCount()); + Assert.assertEquals(1 + (numEvents / 3), hosts.get(2).getAppendCount()); } private List getBatchedEvent(int index) { diff --git a/flume-ng-sdk/src/test/java/org/apache/flume/api/TestNettyAvroRpcClient.java b/flume-ng-sdk/src/test/java/org/apache/flume/api/TestNettyAvroRpcClient.java index cf4f41532e..6cd14548ca 100644 --- a/flume-ng-sdk/src/test/java/org/apache/flume/api/TestNettyAvroRpcClient.java +++ b/flume-ng-sdk/src/test/java/org/apache/flume/api/TestNettyAvroRpcClient.java @@ -87,7 +87,7 @@ public void testOKServerSimpleCompressionLevel0() throws FlumeException, * @throws FlumeException * @throws EventDeliveryException */ - @Test(expected=org.apache.flume.EventDeliveryException.class) + @Test(expected = org.apache.flume.EventDeliveryException.class) public void testOKServerSimpleCompressionClientOnly() throws FlumeException, EventDeliveryException { RpcTestUtils.handlerSimpleAppendTest(new OKAvroHandler(), false, true, 6); @@ -98,7 +98,7 @@ public void testOKServerSimpleCompressionClientOnly() throws FlumeException, * @throws FlumeException * @throws EventDeliveryException */ - @Test(expected=org.apache.flume.EventDeliveryException.class) + @Test(expected = org.apache.flume.EventDeliveryException.class) public void testOKServerSimpleCompressionServerOnly() throws FlumeException, EventDeliveryException { RpcTestUtils.handlerSimpleAppendTest(new OKAvroHandler(), true, false, 6); @@ -142,7 +142,7 @@ public void testOKServerBatchCompressionLevel6() throws FlumeException, * @throws FlumeException * @throws EventDeliveryException */ - @Test(expected=org.apache.flume.EventDeliveryException.class) + @Test(expected = org.apache.flume.EventDeliveryException.class) public void testOKServerBatchCompressionServerOnly() throws FlumeException, EventDeliveryException { RpcTestUtils.handlerBatchAppendTest(new OKAvroHandler(), true, false, 6); @@ -153,7 +153,7 @@ public void testOKServerBatchCompressionServerOnly() throws FlumeException, * @throws FlumeException * @throws EventDeliveryException */ - @Test(expected=org.apache.flume.EventDeliveryException.class) + @Test(expected = org.apache.flume.EventDeliveryException.class) public void testOKServerBatchCompressionClientOnly() throws FlumeException, EventDeliveryException { RpcTestUtils.handlerBatchAppendTest(new OKAvroHandler(), false, true, 6); @@ -164,7 +164,7 @@ public void testOKServerBatchCompressionClientOnly() throws FlumeException, * Note: this test tries to connect to port 1 on localhost. * @throws FlumeException */ - @Test(expected=FlumeException.class) + @Test(expected = FlumeException.class) public void testUnableToConnect() throws FlumeException { @SuppressWarnings("unused") NettyAvroRpcClient client = new NettyAvroRpcClient(); @@ -214,7 +214,7 @@ public void testBatchOverrun() throws FlumeException, EventDeliveryException { * @throws EventDeliveryException * @throws InterruptedException */ - @Test(expected=EventDeliveryException.class) + @Test(expected = EventDeliveryException.class) public void testServerDisconnect() throws FlumeException, EventDeliveryException, InterruptedException { NettyAvroRpcClient client = null; @@ -245,7 +245,7 @@ public void testServerDisconnect() throws FlumeException, * @throws FlumeException * @throws EventDeliveryException */ - @Test(expected=EventDeliveryException.class) + @Test(expected = EventDeliveryException.class) public void testClientClosedRequest() throws FlumeException, EventDeliveryException { NettyAvroRpcClient client = null; @@ -265,7 +265,7 @@ public void testClientClosedRequest() throws FlumeException, /** * Send an event to an online server that returns FAILED. */ - @Test(expected=EventDeliveryException.class) + @Test(expected = EventDeliveryException.class) public void testFailedServerSimple() throws FlumeException, EventDeliveryException { @@ -276,7 +276,7 @@ public void testFailedServerSimple() throws FlumeException, /** * Send an event to an online server that returns UNKNOWN. */ - @Test(expected=EventDeliveryException.class) + @Test(expected = EventDeliveryException.class) public void testUnknownServerSimple() throws FlumeException, EventDeliveryException { @@ -287,7 +287,7 @@ public void testUnknownServerSimple() throws FlumeException, /** * Send an event to an online server that throws an exception. */ - @Test(expected=EventDeliveryException.class) + @Test(expected = EventDeliveryException.class) public void testThrowingServerSimple() throws FlumeException, EventDeliveryException { @@ -298,7 +298,7 @@ public void testThrowingServerSimple() throws FlumeException, /** * Send a batch of events to a server that returns FAILED. */ - @Test(expected=EventDeliveryException.class) + @Test(expected = EventDeliveryException.class) public void testFailedServerBatch() throws FlumeException, EventDeliveryException { @@ -309,7 +309,7 @@ public void testFailedServerBatch() throws FlumeException, /** * Send a batch of events to a server that returns UNKNOWN. */ - @Test(expected=EventDeliveryException.class) + @Test(expected = EventDeliveryException.class) public void testUnknownServerBatch() throws FlumeException, EventDeliveryException { @@ -320,7 +320,7 @@ public void testUnknownServerBatch() throws FlumeException, /** * Send a batch of events to a server that always throws exceptions. */ - @Test(expected=EventDeliveryException.class) + @Test(expected = EventDeliveryException.class) public void testThrowingServerBatch() throws FlumeException, EventDeliveryException { diff --git a/flume-ng-sdk/src/test/java/org/apache/flume/api/TestThriftRpcClient.java b/flume-ng-sdk/src/test/java/org/apache/flume/api/TestThriftRpcClient.java index a8baaa838d..b03fc8d125 100644 --- a/flume-ng-sdk/src/test/java/org/apache/flume/api/TestThriftRpcClient.java +++ b/flume-ng-sdk/src/test/java/org/apache/flume/api/TestThriftRpcClient.java @@ -50,12 +50,10 @@ public class TestThriftRpcClient { public void setUp() throws Exception { props.setProperty("hosts", "h1"); port = random.nextInt(40000) + 1024; - props.setProperty(RpcClientConfigurationConstants.CONFIG_CLIENT_TYPE, - "thrift"); - props.setProperty("hosts.h1", "0.0.0.0:"+ String.valueOf(port)); + props.setProperty(RpcClientConfigurationConstants.CONFIG_CLIENT_TYPE, "thrift"); + props.setProperty("hosts.h1", "0.0.0.0:" + String.valueOf(port)); props.setProperty(RpcClientConfigurationConstants.CONFIG_BATCH_SIZE, "10"); - props.setProperty(RpcClientConfigurationConstants.CONFIG_REQUEST_TIMEOUT, - "2000"); + props.setProperty(RpcClientConfigurationConstants.CONFIG_REQUEST_TIMEOUT, "2000"); props.setProperty(ThriftRpcClient.CONFIG_PROTOCOL, ThriftRpcClient.COMPACT_PROTOCOL); } @@ -71,13 +69,11 @@ public void tearDown() throws Exception { * @param count * @throws Exception */ - public static void insertEvents(RpcClient client, - int count) throws Exception { + public static void insertEvents(RpcClient client, int count) throws Exception { for (int i = 0; i < count; i++) { Map header = new HashMap(); header.put(SEQ, String.valueOf(i)); - client.append(EventBuilder.withBody(String.valueOf(i).getBytes(), - header)); + client.append(EventBuilder.withBody(String.valueOf(i).getBytes(), header)); } } @@ -149,22 +145,20 @@ public void testFail() throws Exception { @Test public void testError() throws Throwable { try { - src = new ThriftTestingSource(ThriftTestingSource.HandlerType.ERROR - .name(), port, ThriftRpcClient.COMPACT_PROTOCOL); - client = (ThriftRpcClient) RpcClientFactory.getThriftInstance("0.0.0" + - ".0", port); + src = new ThriftTestingSource(ThriftTestingSource.HandlerType.ERROR.name(), port, + ThriftRpcClient.COMPACT_PROTOCOL); + client = (ThriftRpcClient) RpcClientFactory.getThriftInstance("0.0.0" + ".0", port); insertEvents(client, 2); //2 events } catch (EventDeliveryException ex) { - Assert.assertEquals("Failed to send event. ", - ex.getMessage()); + Assert.assertEquals("Failed to send event. ", ex.getMessage()); } } @Test (expected = TimeoutException.class) public void testTimeout() throws Throwable { try { - src = new ThriftTestingSource(ThriftTestingSource.HandlerType.TIMEOUT - .name(), port, ThriftRpcClient.COMPACT_PROTOCOL); + src = new ThriftTestingSource(ThriftTestingSource.HandlerType.TIMEOUT.name(), port, + ThriftRpcClient.COMPACT_PROTOCOL); client = (ThriftRpcClient) RpcClientFactory.getThriftInstance(props); insertEvents(client, 2); //2 events } catch (EventDeliveryException ex) { @@ -174,10 +168,9 @@ public void testTimeout() throws Throwable { @Test public void testMultipleThreads() throws Throwable { - src = new ThriftTestingSource(ThriftTestingSource.HandlerType.OK.name(), - port, ThriftRpcClient.COMPACT_PROTOCOL); - client = (ThriftRpcClient) RpcClientFactory.getThriftInstance("0.0.0" + - ".0", port, 10); + src = new ThriftTestingSource(ThriftTestingSource.HandlerType.OK.name(), port, + ThriftRpcClient.COMPACT_PROTOCOL); + client = (ThriftRpcClient) RpcClientFactory.getThriftInstance("0.0.0" + ".0", port, 10); int threadCount = 100; ExecutorService submissionSvc = Executors.newFixedThreadPool(threadCount); ArrayList> futures = new ArrayList>(threadCount); @@ -194,18 +187,18 @@ public void run() { } })); } - for(int i = 0; i < threadCount; i++) { + for (int i = 0; i < threadCount; i++) { futures.get(i).get(); } ArrayList events = new ArrayList(); - for(Event e: src.flumeEvents) { + for (Event e : src.flumeEvents) { events.add(new String(e.getBody())); } int count = 0; Collections.sort(events); for (int i = 0; i < events.size();) { - for(int j = 0; j < threadCount; j++) { + for (int j = 0; j < threadCount; j++) { Assert.assertEquals(String.valueOf(count), events.get(i++)); } count++; diff --git a/flume-ng-sdk/src/test/java/org/apache/flume/api/ThriftTestingSource.java b/flume-ng-sdk/src/test/java/org/apache/flume/api/ThriftTestingSource.java index 70d2c1b5d4..724f0939fe 100644 --- a/flume-ng-sdk/src/test/java/org/apache/flume/api/ThriftTestingSource.java +++ b/flume-ng-sdk/src/test/java/org/apache/flume/api/ThriftTestingSource.java @@ -67,8 +67,7 @@ public ThriftOKHandler() { @Override public Status append(ThriftFlumeEvent event) throws TException { - flumeEvents.add(EventBuilder.withBody(event.getBody(), - event.getHeaders())); + flumeEvents.add(EventBuilder.withBody(event.getBody(), event.getHeaders())); individualCount++; return Status.OK; } @@ -80,8 +79,7 @@ public Status appendBatch(List events) throws TException { incompleteBatches++; } for (ThriftFlumeEvent event : events) { - flumeEvents.add(EventBuilder.withBody(event.getBody(), - event.getHeaders())); + flumeEvents.add(EventBuilder.withBody(event.getBody(), event.getHeaders())); } return Status.OK; } @@ -175,8 +173,7 @@ public Status append(ThriftFlumeEvent event) throws TException { } @Override - public Status appendBatch(List events) throws - TException { + public Status appendBatch(List events) throws TException { try { if (delay != null) { TimeUnit.MILLISECONDS.sleep(delay.get()); @@ -207,8 +204,8 @@ private ThriftSourceProtocol.Iface getHandler(String handlerName) { } public ThriftTestingSource(String handlerName, int port, String protocol) throws Exception { - TNonblockingServerTransport serverTransport = new TNonblockingServerSocket(new - InetSocketAddress("0.0.0.0", port)); + TNonblockingServerTransport serverTransport = + new TNonblockingServerSocket(new InetSocketAddress("0.0.0.0", port)); ThriftSourceProtocol.Iface handler = getHandler(handlerName); TProtocolFactory transportProtocolFactory = null; @@ -217,10 +214,9 @@ public ThriftTestingSource(String handlerName, int port, String protocol) throws } else { transportProtocolFactory = new TCompactProtocol.Factory(); } - server = new THsHaServer(new THsHaServer.Args - (serverTransport).processor( - new ThriftSourceProtocol.Processor(handler)).protocolFactory( - transportProtocolFactory)); + server = new THsHaServer(new THsHaServer.Args(serverTransport).processor( + new ThriftSourceProtocol.Processor(handler)).protocolFactory( + transportProtocolFactory)); Executors.newSingleThreadExecutor().submit(new Runnable() { @Override public void run() { @@ -260,10 +256,8 @@ public ThriftTestingSource(String handlerName, int port, args.protocolFactory(transportProtocolFactory); args.inputTransportFactory(new TFastFramedTransport.Factory()); args.outputTransportFactory(new TFastFramedTransport.Factory()); - args.processor(new ThriftSourceProtocol - .Processor(handler)); - server = (TServer) serverClass.getConstructor(argsClass).newInstance - (args); + args.processor(new ThriftSourceProtocol.Processor(handler)); + server = (TServer) serverClass.getConstructor(argsClass).newInstance(args); Executors.newSingleThreadExecutor().submit(new Runnable() { @Override public void run() { @@ -285,5 +279,4 @@ public void stop() { server.stop(); } - } diff --git a/flume-ng-sinks/flume-dataset-sink/src/test/java/org/apache/flume/sink/kite/TestDatasetSink.java b/flume-ng-sinks/flume-dataset-sink/src/test/java/org/apache/flume/sink/kite/TestDatasetSink.java index 621920d0b1..370957789b 100644 --- a/flume-ng-sinks/flume-dataset-sink/src/test/java/org/apache/flume/sink/kite/TestDatasetSink.java +++ b/flume-ng-sinks/flume-dataset-sink/src/test/java/org/apache/flume/sink/kite/TestDatasetSink.java @@ -18,27 +18,12 @@ package org.apache.flume.sink.kite; -import org.apache.flume.sink.kite.parser.EntityParser; -import org.apache.flume.sink.kite.policy.FailurePolicy; import com.google.common.base.Function; import com.google.common.base.Throwables; import com.google.common.collect.Iterables; import com.google.common.collect.Lists; import com.google.common.collect.Maps; import com.google.common.collect.Sets; -import java.io.ByteArrayOutputStream; -import java.io.File; -import java.io.FileWriter; -import java.io.IOException; -import java.net.URI; -import java.nio.ByteBuffer; -import java.util.Arrays; -import java.util.HashSet; -import java.util.List; -import java.util.Map; -import java.util.Set; -import java.util.concurrent.Callable; -import javax.annotation.Nullable; import org.apache.avro.Schema; import org.apache.avro.file.DataFileWriter; import org.apache.avro.generic.GenericData; @@ -57,6 +42,8 @@ import org.apache.flume.channel.MemoryChannel; import org.apache.flume.conf.Configurables; import org.apache.flume.event.SimpleEvent; +import org.apache.flume.sink.kite.parser.EntityParser; +import org.apache.flume.sink.kite.policy.FailurePolicy; import org.apache.flume.source.avro.AvroFlumeEvent; import org.apache.hadoop.conf.Configuration; import org.apache.hadoop.fs.FileSystem; @@ -74,7 +61,28 @@ import org.kitesdk.data.Datasets; import org.kitesdk.data.PartitionStrategy; import org.kitesdk.data.View; -import static org.mockito.Mockito.*; + +import javax.annotation.Nullable; +import java.io.ByteArrayOutputStream; +import java.io.File; +import java.io.FileWriter; +import java.io.IOException; +import java.net.URI; +import java.nio.ByteBuffer; +import java.util.Arrays; +import java.util.HashSet; +import java.util.List; +import java.util.Map; +import java.util.Set; +import java.util.concurrent.Callable; + +import static org.mockito.Mockito.any; +import static org.mockito.Mockito.doThrow; +import static org.mockito.Mockito.eq; +import static org.mockito.Mockito.mock; +import static org.mockito.Mockito.never; +import static org.mockito.Mockito.verify; +import static org.mockito.Mockito.when; public class TestDatasetSink { @@ -206,7 +214,8 @@ public void testDatasetUriOverridesOldConfig() throws EventDeliveryException { } @Test - public void testFileStore() throws EventDeliveryException, NonRecoverableEventException, NonRecoverableEventException { + public void testFileStore() + throws EventDeliveryException, NonRecoverableEventException, NonRecoverableEventException { DatasetSink sink = sink(in, config); // run the sink @@ -307,31 +316,28 @@ public void testStartBeforeDatasetCreated() throws EventDeliveryException { sink.process(); sink.stop(); - Assert.assertEquals( - Sets.newHashSet(expected), - read(Datasets.load(FILE_DATASET_URI))); + Assert.assertEquals(Sets.newHashSet(expected), read(Datasets.load(FILE_DATASET_URI))); Assert.assertEquals("Should have committed", 0, remaining(in)); } @Test public void testDatasetUpdate() throws EventDeliveryException { // add an updated record that is missing the msg field - GenericRecordBuilder updatedBuilder = new GenericRecordBuilder( - UPDATED_SCHEMA); + GenericRecordBuilder updatedBuilder = new GenericRecordBuilder(UPDATED_SCHEMA); GenericData.Record updatedRecord = updatedBuilder - .set("id", "0") - .set("priority", 1) - .set("msg", "Priority 1 message!") - .build(); + .set("id", "0") + .set("priority", 1) + .set("msg", "Priority 1 message!") + .build(); // make a set of the expected records with the new schema Set expectedAsUpdated = Sets.newHashSet(); for (GenericRecord record : expected) { expectedAsUpdated.add(updatedBuilder - .clear("priority") - .set("id", record.get("id")) - .set("msg", record.get("msg")) - .build()); + .clear("priority") + .set("id", record.get("id")) + .set("msg", record.get("msg")) + .build()); } expectedAsUpdated.add(updatedRecord); @@ -343,9 +349,9 @@ public void testDatasetUpdate() throws EventDeliveryException { // update the dataset's schema DatasetDescriptor updated = new DatasetDescriptor - .Builder(Datasets.load(FILE_DATASET_URI).getDataset().getDescriptor()) - .schema(UPDATED_SCHEMA) - .build(); + .Builder(Datasets.load(FILE_DATASET_URI).getDataset().getDescriptor()) + .schema(UPDATED_SCHEMA) + .build(); Datasets.update(FILE_DATASET_URI, updated); // trigger a roll on the next process call to refresh the writer @@ -358,15 +364,12 @@ public void testDatasetUpdate() throws EventDeliveryException { sink.process(); sink.stop(); - Assert.assertEquals( - expectedAsUpdated, - read(Datasets.load(FILE_DATASET_URI))); + Assert.assertEquals(expectedAsUpdated, read(Datasets.load(FILE_DATASET_URI))); Assert.assertEquals("Should have committed", 0, remaining(in)); } @Test - public void testMiniClusterStore() - throws EventDeliveryException, IOException { + public void testMiniClusterStore() throws EventDeliveryException, IOException { // setup a minicluster MiniDFSCluster cluster = new MiniDFSCluster .Builder(new Configuration()) diff --git a/flume-ng-sinks/flume-hdfs-sink/src/test/java/org/apache/flume/sink/hdfs/HDFSTestSeqWriter.java b/flume-ng-sinks/flume-hdfs-sink/src/test/java/org/apache/flume/sink/hdfs/HDFSTestSeqWriter.java index 9c1cd0998d..f1dadf1f28 100644 --- a/flume-ng-sinks/flume-hdfs-sink/src/test/java/org/apache/flume/sink/hdfs/HDFSTestSeqWriter.java +++ b/flume-ng-sinks/flume-hdfs-sink/src/test/java/org/apache/flume/sink/hdfs/HDFSTestSeqWriter.java @@ -19,25 +19,27 @@ package org.apache.flume.sink.hdfs; -import java.io.IOException; - import org.apache.flume.Event; import org.apache.hadoop.io.SequenceFile.CompressionType; import org.apache.hadoop.io.compress.CompressionCodec; +import java.io.IOException; + public class HDFSTestSeqWriter extends HDFSSequenceFile { - protected volatile boolean closed, opened; + protected volatile boolean closed; + protected volatile boolean opened; private int openCount = 0; + HDFSTestSeqWriter(int openCount) { this.openCount = openCount; } @Override - public void open(String filePath, CompressionCodec codeC, - CompressionType compType) throws IOException { + public void open(String filePath, CompressionCodec codeC, CompressionType compType) + throws IOException { super.open(filePath, codeC, compType); - if(closed) { + if (closed) { opened = true; } } @@ -52,7 +54,7 @@ public void append(Event e) throws IOException { throw new IOException("Injected fault"); } else if (e.getHeaders().containsKey("fault-until-reopen")) { // opening first time. - if(openCount == 1) { + if (openCount == 1) { throw new IOException("Injected fault-until-reopen"); } } else if (e.getHeaders().containsKey("slow")) { diff --git a/flume-ng-sinks/flume-hdfs-sink/src/test/java/org/apache/flume/sink/hdfs/MockDataStream.java b/flume-ng-sinks/flume-hdfs-sink/src/test/java/org/apache/flume/sink/hdfs/MockDataStream.java index f0c6e7e005..a85a99f1c8 100644 --- a/flume-ng-sinks/flume-hdfs-sink/src/test/java/org/apache/flume/sink/hdfs/MockDataStream.java +++ b/flume-ng-sinks/flume-hdfs-sink/src/test/java/org/apache/flume/sink/hdfs/MockDataStream.java @@ -30,9 +30,9 @@ class MockDataStream extends HDFSDataStream { MockDataStream(FileSystem fs) { this.fs = fs; } + @Override - protected FileSystem getDfs(Configuration conf, - Path dstPath) throws IOException{ + protected FileSystem getDfs(Configuration conf, Path dstPath) throws IOException { return fs; } diff --git a/flume-ng-sinks/flume-hdfs-sink/src/test/java/org/apache/flume/sink/hdfs/MockFileSystem.java b/flume-ng-sinks/flume-hdfs-sink/src/test/java/org/apache/flume/sink/hdfs/MockFileSystem.java index 444333534b..a079b83164 100644 --- a/flume-ng-sinks/flume-hdfs-sink/src/test/java/org/apache/flume/sink/hdfs/MockFileSystem.java +++ b/flume-ng-sinks/flume-hdfs-sink/src/test/java/org/apache/flume/sink/hdfs/MockFileSystem.java @@ -42,8 +42,7 @@ public class MockFileSystem extends FileSystem { int currentRenameAttempts; boolean closeSucceed = true; - public MockFileSystem(FileSystem fs, - int numberOfRetriesRequired) { + public MockFileSystem(FileSystem fs, int numberOfRetriesRequired) { this.fs = fs; this.numberOfRetriesRequired = numberOfRetriesRequired; } @@ -67,17 +66,14 @@ public FSDataOutputStream append(Path arg0, int arg1, Progressable arg2) @Override public FSDataOutputStream create(Path arg0) throws IOException { - //throw new IOException ("HI there2"); - latestOutputStream = new MockFsDataOutputStream( - fs.create(arg0), closeSucceed); - + latestOutputStream = new MockFsDataOutputStream(fs.create(arg0), closeSucceed); return latestOutputStream; } @Override - public FSDataOutputStream create(Path arg0, FsPermission arg1, - boolean arg2, int arg3, short arg4, long arg5, Progressable arg6) - throws IOException { + public FSDataOutputStream create(Path arg0, FsPermission arg1, boolean arg2, int arg3, + short arg4, long arg5, Progressable arg6) + throws IOException { throw new IOException("Not a real file system"); } @@ -126,11 +122,9 @@ public FSDataInputStream open(Path arg0, int arg1) throws IOException { @Override public boolean rename(Path arg0, Path arg1) throws IOException { currentRenameAttempts++; - logger.info( - "Attempting to Rename: '" + currentRenameAttempts + "' of '" + - numberOfRetriesRequired + "'"); - if (currentRenameAttempts >= numberOfRetriesRequired || - numberOfRetriesRequired == 0) { + logger.info("Attempting to Rename: '" + currentRenameAttempts + "' of '" + + numberOfRetriesRequired + "'"); + if (currentRenameAttempts >= numberOfRetriesRequired || numberOfRetriesRequired == 0) { logger.info("Renaming file"); return fs.rename(arg0, arg1); } else { @@ -141,6 +135,5 @@ public boolean rename(Path arg0, Path arg1) throws IOException { @Override public void setWorkingDirectory(Path arg0) { fs.setWorkingDirectory(arg0); - } } diff --git a/flume-ng-sinks/flume-hdfs-sink/src/test/java/org/apache/flume/sink/hdfs/MockFsDataOutputStream.java b/flume-ng-sinks/flume-hdfs-sink/src/test/java/org/apache/flume/sink/hdfs/MockFsDataOutputStream.java index 35b034ef7b..f5d579c708 100644 --- a/flume-ng-sinks/flume-hdfs-sink/src/test/java/org/apache/flume/sink/hdfs/MockFsDataOutputStream.java +++ b/flume-ng-sinks/flume-hdfs-sink/src/test/java/org/apache/flume/sink/hdfs/MockFsDataOutputStream.java @@ -17,21 +17,20 @@ + */ package org.apache.flume.sink.hdfs; -import java.io.IOException; - import org.apache.hadoop.fs.FSDataOutputStream; import org.slf4j.Logger; import org.slf4j.LoggerFactory; -public class MockFsDataOutputStream extends FSDataOutputStream{ +import java.io.IOException; + +public class MockFsDataOutputStream extends FSDataOutputStream { private static final Logger logger = LoggerFactory.getLogger(MockFsDataOutputStream.class); boolean closeSucceed; - public MockFsDataOutputStream(FSDataOutputStream wrapMe, - boolean closeSucceed) + public MockFsDataOutputStream(FSDataOutputStream wrapMe, boolean closeSucceed) throws IOException { super(wrapMe.getWrappedStream(), null); this.closeSucceed = closeSucceed; @@ -39,8 +38,7 @@ public MockFsDataOutputStream(FSDataOutputStream wrapMe, @Override public void close() throws IOException { - logger.info( - "Close Succeeded - " + closeSucceed); + logger.info("Close Succeeded - " + closeSucceed); if (closeSucceed) { logger.info("closing file"); super.close(); diff --git a/flume-ng-sinks/flume-hdfs-sink/src/test/java/org/apache/flume/sink/hdfs/MockHDFSWriter.java b/flume-ng-sinks/flume-hdfs-sink/src/test/java/org/apache/flume/sink/hdfs/MockHDFSWriter.java index ec49b97dde..05c4316e29 100644 --- a/flume-ng-sinks/flume-hdfs-sink/src/test/java/org/apache/flume/sink/hdfs/MockHDFSWriter.java +++ b/flume-ng-sinks/flume-hdfs-sink/src/test/java/org/apache/flume/sink/hdfs/MockHDFSWriter.java @@ -68,7 +68,8 @@ public void open(String filePath) throws IOException { filesOpened++; } - public void open(String filePath, CompressionCodec codec, CompressionType cType) throws IOException { + public void open(String filePath, CompressionCodec codec, CompressionType cType) + throws IOException { this.filePath = filePath; filesOpened++; } diff --git a/flume-ng-sinks/flume-hdfs-sink/src/test/java/org/apache/flume/sink/hdfs/TestBucketWriter.java b/flume-ng-sinks/flume-hdfs-sink/src/test/java/org/apache/flume/sink/hdfs/TestBucketWriter.java index 2581f73992..742deb0bc7 100644 --- a/flume-ng-sinks/flume-hdfs-sink/src/test/java/org/apache/flume/sink/hdfs/TestBucketWriter.java +++ b/flume-ng-sinks/flume-hdfs-sink/src/test/java/org/apache/flume/sink/hdfs/TestBucketWriter.java @@ -18,14 +18,7 @@ */ package org.apache.flume.sink.hdfs; -import java.io.File; -import java.io.IOException; -import java.util.Calendar; -import java.util.concurrent.Executors; -import java.util.concurrent.ScheduledExecutorService; -import java.util.concurrent.TimeUnit; -import java.util.concurrent.atomic.AtomicBoolean; - +import com.google.common.base.Charsets; import org.apache.flume.Clock; import org.apache.flume.Context; import org.apache.flume.Event; @@ -46,12 +39,17 @@ import org.slf4j.Logger; import org.slf4j.LoggerFactory; -import com.google.common.base.Charsets; +import java.io.File; +import java.io.IOException; +import java.util.Calendar; +import java.util.concurrent.Executors; +import java.util.concurrent.ScheduledExecutorService; +import java.util.concurrent.TimeUnit; +import java.util.concurrent.atomic.AtomicBoolean; public class TestBucketWriter { - private static Logger logger = - LoggerFactory.getLogger(TestBucketWriter.class); + private static Logger logger = LoggerFactory.getLogger(TestBucketWriter.class); private Context ctx = new Context(); private static ScheduledExecutorService timedRollerPool; @@ -74,11 +72,11 @@ public static void teardown() throws InterruptedException { public void testEventCountingRoller() throws IOException, InterruptedException { int maxEvents = 100; MockHDFSWriter hdfsWriter = new MockHDFSWriter(); - BucketWriter bucketWriter = new BucketWriter(0, 0, maxEvents, 0, ctx, - "/tmp", "file", "", ".tmp", null, null, SequenceFile.CompressionType.NONE, - hdfsWriter, timedRollerPool, proxy, - new SinkCounter("test-bucket-writer-" + System.currentTimeMillis()), 0, - null, null, 30000, Executors.newSingleThreadExecutor(), 0, 0); + BucketWriter bucketWriter = new BucketWriter( + 0, 0, maxEvents, 0, ctx, "/tmp", "file", "", ".tmp", null, null, + SequenceFile.CompressionType.NONE, hdfsWriter, timedRollerPool, proxy, + new SinkCounter("test-bucket-writer-" + System.currentTimeMillis()), 0, null, null, 30000, + Executors.newSingleThreadExecutor(), 0, 0); Event e = EventBuilder.withBody("foo", Charsets.UTF_8); for (int i = 0; i < 1000; i++) { @@ -98,12 +96,11 @@ public void testEventCountingRoller() throws IOException, InterruptedException { public void testSizeRoller() throws IOException, InterruptedException { int maxBytes = 300; MockHDFSWriter hdfsWriter = new MockHDFSWriter(); - BucketWriter bucketWriter = new BucketWriter(0, maxBytes, 0, 0, - ctx, "/tmp", "file", "", ".tmp", null, null, - SequenceFile.CompressionType.NONE, hdfsWriter,timedRollerPool, - proxy, new SinkCounter("test-bucket-writer-" + - System.currentTimeMillis()),0, null, null, 30000, - Executors.newSingleThreadExecutor(), 0, 0); + BucketWriter bucketWriter = new BucketWriter( + 0, maxBytes, 0, 0, ctx, "/tmp", "file", "", ".tmp", null, null, + SequenceFile.CompressionType.NONE, hdfsWriter, timedRollerPool, proxy, + new SinkCounter("test-bucket-writer-" + System.currentTimeMillis()), 0, null, null, 30000, + Executors.newSingleThreadExecutor(), 0, 0); Event e = EventBuilder.withBody("foo", Charsets.UTF_8); for (int i = 0; i < 1000; i++) { @@ -126,16 +123,16 @@ public void testIntervalRoller() throws IOException, InterruptedException { final AtomicBoolean calledBack = new AtomicBoolean(false); MockHDFSWriter hdfsWriter = new MockHDFSWriter(); - BucketWriter bucketWriter = new BucketWriter(ROLL_INTERVAL, 0, 0, 0, ctx, - "/tmp", "file", "", ".tmp", null, null, SequenceFile.CompressionType.NONE, - hdfsWriter, timedRollerPool, proxy, - new SinkCounter("test-bucket-writer-" + System.currentTimeMillis()), - 0, new HDFSEventSink.WriterCallback() { - @Override - public void run(String filePath) { - calledBack.set(true); - } - }, null, 30000, Executors.newSingleThreadExecutor(), 0, 0); + BucketWriter bucketWriter = new BucketWriter( + ROLL_INTERVAL, 0, 0, 0, ctx, "/tmp", "file", "", ".tmp", null, null, + SequenceFile.CompressionType.NONE, hdfsWriter, timedRollerPool, proxy, + new SinkCounter("test-bucket-writer-" + System.currentTimeMillis()), 0, + new HDFSEventSink.WriterCallback() { + @Override + public void run(String filePath) { + calledBack.set(true); + } + }, null, 30000, Executors.newSingleThreadExecutor(), 0, 0); Event e = EventBuilder.withBody("foo", Charsets.UTF_8); long startNanos = System.nanoTime(); @@ -149,12 +146,11 @@ public void run(String filePath) { Assert.assertTrue(bucketWriter.closed); Assert.assertTrue(calledBack.get()); - bucketWriter = new BucketWriter(ROLL_INTERVAL, 0, 0, 0, ctx, - "/tmp", "file", "", ".tmp", null, null, SequenceFile.CompressionType.NONE, - hdfsWriter, timedRollerPool, proxy, - new SinkCounter("test-bucket-writer-" - + System.currentTimeMillis()), 0, null, null, 30000, - Executors.newSingleThreadExecutor(), 0, 0); + bucketWriter = new BucketWriter( + ROLL_INTERVAL, 0, 0, 0, ctx, "/tmp", "file", "", ".tmp", null, null, + SequenceFile.CompressionType.NONE, hdfsWriter, timedRollerPool, proxy, + new SinkCounter("test-bucket-writer-" + System.currentTimeMillis()), 0, null, null, 30000, + Executors.newSingleThreadExecutor(), 0, 0); // write one more event (to reopen a new file so we will roll again later) bucketWriter.append(e); @@ -193,17 +189,16 @@ public void testIntervalRollerBug() throws IOException, InterruptedException { private volatile boolean open = false; public void configure(Context context) { - } public void sync() throws IOException { - if(!open) { + if (!open) { throw new IOException("closed"); } } - public void open(String filePath, CompressionCodec codec, - CompressionType cType) throws IOException { + public void open(String filePath, CompressionCodec codec, CompressionType cType) + throws IOException { open = true; } @@ -225,19 +220,18 @@ public void append(Event e) throws IOException { open = true; } }; + HDFSTextSerializer serializer = new HDFSTextSerializer(); File tmpFile = File.createTempFile("flume", "test"); tmpFile.deleteOnExit(); String path = tmpFile.getParent(); String name = tmpFile.getName(); - BucketWriter bucketWriter = new BucketWriter(ROLL_INTERVAL, 0, 0, - 0, ctx, path, name, "", ".tmp", null, null, - SequenceFile.CompressionType.NONE, hdfsWriter, - timedRollerPool, proxy, new SinkCounter("test-bucket-writer-" - + System.currentTimeMillis()), - 0, null, null, 30000, Executors.newSingleThreadExecutor(), - 0, 0); + BucketWriter bucketWriter = new BucketWriter( + ROLL_INTERVAL, 0, 0, 0, ctx, path, name, "", ".tmp", null, null, + SequenceFile.CompressionType.NONE, hdfsWriter, timedRollerPool, proxy, + new SinkCounter("test-bucket-writer-" + System.currentTimeMillis()), 0, null, null, 30000, + Executors.newSingleThreadExecutor(), 0, 0); Event e = EventBuilder.withBody("foo", Charsets.UTF_8); for (int i = 0; i < NUM_EVENTS - 1; i++) { @@ -252,62 +246,61 @@ timedRollerPool, proxy, new SinkCounter("test-bucket-writer-" @Test public void testFileSuffixNotGiven() throws IOException, InterruptedException { - final int ROLL_INTERVAL = 1000; // seconds. Make sure it doesn't change in course of test - final String suffix = null; - - MockHDFSWriter hdfsWriter = new MockHDFSWriter(); - BucketWriter bucketWriter = new BucketWriter(ROLL_INTERVAL, 0, - 0, 0, ctx, "/tmp", "file", "", ".tmp", suffix, null, - SequenceFile.CompressionType.NONE, hdfsWriter, - timedRollerPool, proxy, new SinkCounter("test-bucket-writer-" - + System.currentTimeMillis()), 0, null, null, 30000, + final int ROLL_INTERVAL = 1000; // seconds. Make sure it doesn't change in course of test + final String suffix = null; + + MockHDFSWriter hdfsWriter = new MockHDFSWriter(); + BucketWriter bucketWriter = new BucketWriter( + ROLL_INTERVAL, 0, 0, 0, ctx, "/tmp", "file", "", ".tmp", suffix, null, + SequenceFile.CompressionType.NONE, hdfsWriter, timedRollerPool, proxy, + new SinkCounter("test-bucket-writer-" + System.currentTimeMillis()), 0, null, null, 30000, Executors.newSingleThreadExecutor(), 0, 0); - // Need to override system time use for test so we know what to expect - final long testTime = System.currentTimeMillis(); - Clock testClock = new Clock() { - public long currentTimeMillis() { - return testTime; - } - }; - bucketWriter.setClock(testClock); + // Need to override system time use for test so we know what to expect + final long testTime = System.currentTimeMillis(); + Clock testClock = new Clock() { + public long currentTimeMillis() { + return testTime; + } + }; + bucketWriter.setClock(testClock); - Event e = EventBuilder.withBody("foo", Charsets.UTF_8); - bucketWriter.append(e); + Event e = EventBuilder.withBody("foo", Charsets.UTF_8); + bucketWriter.append(e); - Assert.assertTrue("Incorrect suffix", hdfsWriter.getOpenedFilePath().endsWith(Long.toString(testTime+1) + ".tmp")); + Assert.assertTrue("Incorrect suffix", hdfsWriter.getOpenedFilePath().endsWith( + Long.toString(testTime + 1) + ".tmp")); } - @Test - public void testFileSuffixGiven() throws IOException, InterruptedException { - final int ROLL_INTERVAL = 1000; // seconds. Make sure it doesn't change in course of test - final String suffix = ".avro"; + @Test + public void testFileSuffixGiven() throws IOException, InterruptedException { + final int ROLL_INTERVAL = 1000; // seconds. Make sure it doesn't change in course of test + final String suffix = ".avro"; - MockHDFSWriter hdfsWriter = new MockHDFSWriter(); - BucketWriter bucketWriter = new BucketWriter(ROLL_INTERVAL, 0, - 0, 0, ctx, "/tmp", "file", "", ".tmp", suffix, null, - SequenceFile.CompressionType.NONE, hdfsWriter, - timedRollerPool, proxy, new SinkCounter( - "test-bucket-writer-" + System.currentTimeMillis()), 0, - null, null, 30000, Executors.newSingleThreadExecutor(), 0, 0); + MockHDFSWriter hdfsWriter = new MockHDFSWriter(); + BucketWriter bucketWriter = new BucketWriter( + ROLL_INTERVAL, 0, 0, 0, ctx, "/tmp", "file", "", ".tmp", suffix, null, + SequenceFile.CompressionType.NONE, hdfsWriter, timedRollerPool, proxy, + new SinkCounter("test-bucket-writer-" + System.currentTimeMillis()), 0, null, null, 30000, + Executors.newSingleThreadExecutor(), 0, 0); - // Need to override system time use for test so we know what to expect + // Need to override system time use for test so we know what to expect - final long testTime = System.currentTimeMillis(); + final long testTime = System.currentTimeMillis(); - Clock testClock = new Clock() { - public long currentTimeMillis() { - return testTime; - } - }; - bucketWriter.setClock(testClock); + Clock testClock = new Clock() { + public long currentTimeMillis() { + return testTime; + } + }; + bucketWriter.setClock(testClock); - Event e = EventBuilder.withBody("foo", Charsets.UTF_8); - bucketWriter.append(e); + Event e = EventBuilder.withBody("foo", Charsets.UTF_8); + bucketWriter.append(e); - Assert.assertTrue("Incorrect suffix", hdfsWriter.getOpenedFilePath().endsWith( - Long.toString(testTime + 1) + suffix + ".tmp")); - } + Assert.assertTrue("Incorrect suffix",hdfsWriter.getOpenedFilePath().endsWith( + Long.toString(testTime + 1) + suffix + ".tmp")); + } @Test public void testFileSuffixCompressed() @@ -316,13 +309,11 @@ public void testFileSuffixCompressed() final String suffix = ".foo"; MockHDFSWriter hdfsWriter = new MockHDFSWriter(); - BucketWriter bucketWriter = new BucketWriter(ROLL_INTERVAL, 0, 0, - 0, ctx, "/tmp", "file", "", ".tmp", suffix, - HDFSEventSink.getCodec("gzip"), - SequenceFile.CompressionType.BLOCK, hdfsWriter, - timedRollerPool, proxy, new SinkCounter("test-bucket-writer-" - + System.currentTimeMillis()), 0, null, null, 30000, - Executors.newSingleThreadExecutor(), 0, 0 + BucketWriter bucketWriter = new BucketWriter( + ROLL_INTERVAL, 0, 0, 0, ctx, "/tmp", "file", "", ".tmp", suffix, + HDFSEventSink.getCodec("gzip"), SequenceFile.CompressionType.BLOCK, hdfsWriter, + timedRollerPool, proxy, new SinkCounter("test-bucket-writer-" + System.currentTimeMillis()), + 0, null, null, 30000, Executors.newSingleThreadExecutor(), 0, 0 ); // Need to override system time use for test so we know what to expect @@ -338,8 +329,8 @@ public long currentTimeMillis() { Event e = EventBuilder.withBody("foo", Charsets.UTF_8); bucketWriter.append(e); - Assert.assertTrue("Incorrect suffix",hdfsWriter.getOpenedFilePath() - .endsWith(Long.toString(testTime+1) + suffix + ".tmp")); + Assert.assertTrue("Incorrect suffix", hdfsWriter.getOpenedFilePath().endsWith( + Long.toString(testTime + 1) + suffix + ".tmp")); } @Test @@ -349,12 +340,11 @@ public void testInUsePrefix() throws IOException, InterruptedException { MockHDFSWriter hdfsWriter = new MockHDFSWriter(); HDFSTextSerializer formatter = new HDFSTextSerializer(); - BucketWriter bucketWriter = new BucketWriter(ROLL_INTERVAL, 0, 0, - 0, ctx, "/tmp", "file", PREFIX, ".tmp", null, null, - SequenceFile.CompressionType.NONE, hdfsWriter, - timedRollerPool, proxy, new SinkCounter( - "test-bucket-writer-" + System.currentTimeMillis()), 0, - null, null, 30000, Executors.newSingleThreadExecutor(), 0, 0); + BucketWriter bucketWriter = new BucketWriter( + ROLL_INTERVAL, 0, 0, 0, ctx, "/tmp", "file", PREFIX, ".tmp", null, null, + SequenceFile.CompressionType.NONE, hdfsWriter, timedRollerPool, proxy, + new SinkCounter("test-bucket-writer-" + System.currentTimeMillis()), 0, null, null, 30000, + Executors.newSingleThreadExecutor(), 0, 0); Event e = EventBuilder.withBody("foo", Charsets.UTF_8); bucketWriter.append(e); @@ -369,12 +359,11 @@ public void testInUseSuffix() throws IOException, InterruptedException { MockHDFSWriter hdfsWriter = new MockHDFSWriter(); HDFSTextSerializer serializer = new HDFSTextSerializer(); - BucketWriter bucketWriter = new BucketWriter(ROLL_INTERVAL, 0, 0, - 0, ctx, "/tmp", "file", "", SUFFIX, null, null, - SequenceFile.CompressionType.NONE, hdfsWriter, - timedRollerPool, proxy, new SinkCounter( - "test-bucket-writer-" + System.currentTimeMillis()), 0, - null, null, 30000, Executors.newSingleThreadExecutor(), 0, 0); + BucketWriter bucketWriter = new BucketWriter( + ROLL_INTERVAL, 0, 0, 0, ctx, "/tmp", "file", "", SUFFIX, null, null, + SequenceFile.CompressionType.NONE, hdfsWriter, timedRollerPool, proxy, + new SinkCounter("test-bucket-writer-" + System.currentTimeMillis()), 0, null, null, 30000, + Executors.newSingleThreadExecutor(), 0, 0); Event e = EventBuilder.withBody("foo", Charsets.UTF_8); bucketWriter.append(e); @@ -389,18 +378,16 @@ public void testCallbackOnClose() throws IOException, InterruptedException { final AtomicBoolean callbackCalled = new AtomicBoolean(false); MockHDFSWriter hdfsWriter = new MockHDFSWriter(); - BucketWriter bucketWriter = new BucketWriter(ROLL_INTERVAL, 0, 0, - 0, ctx, "/tmp", "file", "", SUFFIX, null, null, - SequenceFile.CompressionType.NONE, - hdfsWriter, timedRollerPool, proxy, - new SinkCounter( - "test-bucket-writer-" + System.currentTimeMillis()), 0, - new HDFSEventSink.WriterCallback() { - @Override - public void run(String filePath) { - callbackCalled.set(true); - } - }, "blah", 30000, Executors.newSingleThreadExecutor(), 0, 0); + BucketWriter bucketWriter = new BucketWriter( + ROLL_INTERVAL, 0, 0, 0, ctx, "/tmp", "file", "", SUFFIX, null, null, + SequenceFile.CompressionType.NONE, hdfsWriter, timedRollerPool, proxy, + new SinkCounter("test-bucket-writer-" + System.currentTimeMillis()), 0, + new HDFSEventSink.WriterCallback() { + @Override + public void run(String filePath) { + callbackCalled.set(true); + } + }, "blah", 30000, Executors.newSingleThreadExecutor(), 0, 0); Event e = EventBuilder.withBody("foo", Charsets.UTF_8); bucketWriter.append(e); @@ -420,13 +407,13 @@ public void testSequenceFileRenameRetries() throws Exception { SequenceFileRenameRetryCoreTest(1, false); SequenceFileRenameRetryCoreTest(5, false); SequenceFileRenameRetryCoreTest(2, false); - } - public void SequenceFileRenameRetryCoreTest(int numberOfRetriesRequired, boolean closeSucceed) throws Exception { - String hdfsPath = "file:///tmp/flume-test." - + Calendar.getInstance().getTimeInMillis() + "." - + Thread.currentThread().getId(); + public void SequenceFileRenameRetryCoreTest(int numberOfRetriesRequired, boolean closeSucceed) + throws Exception { + String hdfsPath = "file:///tmp/flume-test." + + Calendar.getInstance().getTimeInMillis() + + "." + Thread.currentThread().getId(); Context context = new Context(); Configuration conf = new Configuration(); @@ -435,22 +422,16 @@ public void SequenceFileRenameRetryCoreTest(int numberOfRetriesRequired, boolean fs.delete(dirPath, true); fs.mkdirs(dirPath); context.put("hdfs.path", hdfsPath); - context.put("hdfs.closeTries", - String.valueOf(numberOfRetriesRequired)); + context.put("hdfs.closeTries", String.valueOf(numberOfRetriesRequired)); context.put("hdfs.rollCount", "1"); context.put("hdfs.retryInterval", "1"); context.put("hdfs.callTimeout", Long.toString(1000)); - MockFileSystem mockFs = new - MockFileSystem(fs, - numberOfRetriesRequired, closeSucceed); - BucketWriter bucketWriter = new BucketWriter(0, 0, 1, 1, ctx, - hdfsPath, hdfsPath, "singleBucket", ".tmp", null, null, - null, new MockDataStream(mockFs), - timedRollerPool, proxy, - new SinkCounter( - "test-bucket-writer-" + System.currentTimeMillis()), - 0, null, null, 30000, Executors.newSingleThreadExecutor(), 1, - numberOfRetriesRequired); + MockFileSystem mockFs = new MockFileSystem(fs, numberOfRetriesRequired, closeSucceed); + BucketWriter bucketWriter = new BucketWriter( + 0, 0, 1, 1, ctx, hdfsPath, hdfsPath, "singleBucket", ".tmp", null, null, + null, new MockDataStream(mockFs), timedRollerPool, proxy, + new SinkCounter("test-bucket-writer-" + System.currentTimeMillis()), 0, null, null, 30000, + Executors.newSingleThreadExecutor(), 1, numberOfRetriesRequired); bucketWriter.setFileSystem(mockFs); // At this point, we checked if isFileClosed is available in @@ -463,8 +444,7 @@ null, new MockDataStream(mockFs), TimeUnit.SECONDS.sleep(numberOfRetriesRequired + 2); Assert.assertTrue("Expected " + numberOfRetriesRequired + " " + - "but got " + bucketWriter.renameTries.get(), - bucketWriter.renameTries.get() == - numberOfRetriesRequired); + "but got " + bucketWriter.renameTries.get(), + bucketWriter.renameTries.get() == numberOfRetriesRequired); } } diff --git a/flume-ng-sinks/flume-hdfs-sink/src/test/java/org/apache/flume/sink/hdfs/TestHDFSEventSink.java b/flume-ng-sinks/flume-hdfs-sink/src/test/java/org/apache/flume/sink/hdfs/TestHDFSEventSink.java index 23862eb44e..73f016bf3f 100644 --- a/flume-ng-sinks/flume-hdfs-sink/src/test/java/org/apache/flume/sink/hdfs/TestHDFSEventSink.java +++ b/flume-ng-sinks/flume-hdfs-sink/src/test/java/org/apache/flume/sink/hdfs/TestHDFSEventSink.java @@ -71,7 +71,6 @@ import com.google.common.base.Charsets; import com.google.common.collect.Lists; - public class TestHDFSEventSink { private HDFSEventSink sink; @@ -118,8 +117,7 @@ public void setUp() { @After public void tearDown() { - if (System.getenv("hdfs_keepFiles") == null) - dirCleanup(); + if (System.getenv("hdfs_keepFiles") == null) dirCleanup(); } @Test @@ -176,7 +174,7 @@ public void doTestTextBatchAppend(boolean useRawLocalFileSystem) List bodies = Lists.newArrayList(); // push the event batches into channel to roll twice - for (i = 1; i <= (rollCount*10)/batchSize; i++) { + for (i = 1; i <= (rollCount * 10) / batchSize; i++) { Transaction txn = channel.getTransaction(); txn.begin(); for (j = 1; j <= batchSize; j++) { @@ -200,7 +198,7 @@ public void doTestTextBatchAppend(boolean useRawLocalFileSystem) // loop through all the files generated and check their contains FileStatus[] dirStat = fs.listStatus(dirPath); - Path fList[] = FileUtil.stat2Paths(dirStat); + Path[] fList = FileUtil.stat2Paths(dirStat); // check that the roll happened correctly for the given data long expectedFiles = totalEvents / rollCount; @@ -353,7 +351,7 @@ public void testTextAppend() throws InterruptedException, LifecycleException, // loop through all the files generated and check their contains FileStatus[] dirStat = fs.listStatus(dirPath); - Path fList[] = FileUtil.stat2Paths(dirStat); + Path[] fList = FileUtil.stat2Paths(dirStat); // check that the roll happened correctly for the given data long expectedFiles = totalEvents / rollCount; @@ -432,7 +430,7 @@ public void testAvroAppend() throws InterruptedException, LifecycleException, // loop through all the files generated and check their contains FileStatus[] dirStat = fs.listStatus(dirPath); - Path fList[] = FileUtil.stat2Paths(dirStat); + Path[] fList = FileUtil.stat2Paths(dirStat); // check that the roll happened correctly for the given data long expectedFiles = totalEvents / rollCount; @@ -508,7 +506,7 @@ public void testSimpleAppend() throws InterruptedException, // loop through all the files generated and check their contains FileStatus[] dirStat = fs.listStatus(dirPath); - Path fList[] = FileUtil.stat2Paths(dirStat); + Path[] fList = FileUtil.stat2Paths(dirStat); // check that the roll happened correctly for the given data long expectedFiles = totalEvents / rollCount; @@ -519,8 +517,8 @@ public void testSimpleAppend() throws InterruptedException, } @Test - public void testSimpleAppendLocalTime() throws InterruptedException, - LifecycleException, EventDeliveryException, IOException { + public void testSimpleAppendLocalTime() + throws InterruptedException, LifecycleException, EventDeliveryException, IOException { final long currentTime = System.currentTimeMillis(); Clock clk = new Clock() { @Override @@ -536,7 +534,7 @@ public long currentTimeMillis() { final int numBatches = 4; String newPath = testPath + "/singleBucket/%s" ; String expectedPath = testPath + "/singleBucket/" + - String.valueOf(currentTime/1000); + String.valueOf(currentTime / 1000); int totalEvents = 0; int i = 1, j = 1; @@ -576,7 +574,7 @@ public long currentTimeMillis() { eventDate.clear(); eventDate.set(2011, i, i, i, 0); // yy mm dd event.getHeaders().put("timestamp", - String.valueOf(eventDate.getTimeInMillis())); + String.valueOf(eventDate.getTimeInMillis())); event.getHeaders().put("hostname", "Host" + i); String body = "Test." + i + "." + j; event.setBody(body.getBytes()); @@ -595,13 +593,13 @@ public long currentTimeMillis() { // loop through all the files generated and check their contains FileStatus[] dirStat = fs.listStatus(dirPath); - Path fList[] = FileUtil.stat2Paths(dirStat); + Path[] fList = FileUtil.stat2Paths(dirStat); // check that the roll happened correctly for the given data long expectedFiles = totalEvents / rollCount; if (totalEvents % rollCount > 0) expectedFiles++; Assert.assertEquals("num files wrong, found: " + - Lists.newArrayList(fList), expectedFiles, fList.length); + Lists.newArrayList(fList), expectedFiles, fList.length); verifyOutputSequenceFiles(fs, conf, dirPath.toUri().getPath(), fileName, bodies); // The clock in bucketpath is static, so restore the real clock sink.setBucketClock(new SystemClock()); @@ -750,10 +748,10 @@ public void testBadSimpleAppend() throws InterruptedException, private List getAllFiles(String input) { List output = Lists.newArrayList(); File dir = new File(input); - if(dir.isFile()) { + if (dir.isFile()) { output.add(dir.getAbsolutePath()); - } else if(dir.isDirectory()) { - for(String file : dir.list()) { + } else if (dir.isDirectory()) { + for (String file : dir.list()) { File subDir = new File(dir, file); output.addAll(getAllFiles(subDir.getAbsolutePath())); } @@ -761,16 +759,17 @@ private List getAllFiles(String input) { return output; } - private void verifyOutputSequenceFiles(FileSystem fs, Configuration conf, String dir, String prefix, List bodies) throws IOException { + private void verifyOutputSequenceFiles(FileSystem fs, Configuration conf, String dir, + String prefix, List bodies) throws IOException { int found = 0; int expected = bodies.size(); - for(String outputFile : getAllFiles(dir)) { + for (String outputFile : getAllFiles(dir)) { String name = (new File(outputFile)).getName(); - if(name.startsWith(prefix)) { + if (name.startsWith(prefix)) { SequenceFile.Reader reader = new SequenceFile.Reader(fs, new Path(outputFile), conf); LongWritable key = new LongWritable(); BytesWritable value = new BytesWritable(); - while(reader.next(key, value)) { + while (reader.next(key, value)) { String body = new String(value.getBytes(), 0, value.getLength()); if (bodies.contains(body)) { LOG.debug("Found event body: {}", body); @@ -792,16 +791,17 @@ private void verifyOutputSequenceFiles(FileSystem fs, Configuration conf, String } - private void verifyOutputTextFiles(FileSystem fs, Configuration conf, String dir, String prefix, List bodies) throws IOException { + private void verifyOutputTextFiles(FileSystem fs, Configuration conf, String dir, String prefix, + List bodies) throws IOException { int found = 0; int expected = bodies.size(); - for(String outputFile : getAllFiles(dir)) { + for (String outputFile : getAllFiles(dir)) { String name = (new File(outputFile)).getName(); - if(name.startsWith(prefix)) { + if (name.startsWith(prefix)) { FSDataInputStream input = fs.open(new Path(outputFile)); BufferedReader reader = new BufferedReader(new InputStreamReader(input)); String body = null; - while((body = reader.readLine()) != null) { + while ((body = reader.readLine()) != null) { bodies.remove(body); found++; } @@ -814,12 +814,13 @@ private void verifyOutputTextFiles(FileSystem fs, Configuration conf, String dir } - private void verifyOutputAvroFiles(FileSystem fs, Configuration conf, String dir, String prefix, List bodies) throws IOException { + private void verifyOutputAvroFiles(FileSystem fs, Configuration conf, String dir, String prefix, + List bodies) throws IOException { int found = 0; int expected = bodies.size(); - for(String outputFile : getAllFiles(dir)) { + for (String outputFile : getAllFiles(dir)) { String name = (new File(outputFile)).getName(); - if(name.startsWith(prefix)) { + if (name.startsWith(prefix)) { FSDataInputStream input = fs.open(new Path(outputFile)); DatumReader reader = new GenericDatumReader(); DataFileStream avroStream = @@ -840,7 +841,7 @@ private void verifyOutputAvroFiles(FileSystem fs, Configuration conf, String dir } Assert.assertTrue("Found = " + found + ", Expected = " + expected + ", Left = " + bodies.size() + " " + bodies, - bodies.size() == 0); + bodies.size() == 0); } /** @@ -849,9 +850,9 @@ private void verifyOutputAvroFiles(FileSystem fs, Configuration conf, String dir * This relies on Transactional rollback semantics for durability and * the behavior of the BucketWriter class of close()ing upon IOException. */ - @Test - public void testCloseReopen() throws InterruptedException, - LifecycleException, EventDeliveryException, IOException { + @Test + public void testCloseReopen() + throws InterruptedException, LifecycleException, EventDeliveryException, IOException { LOG.debug("Starting..."); final int numBatches = 4; @@ -924,8 +925,8 @@ public void testCloseReopen() throws InterruptedException, * a new one is used for the next set of events. */ @Test - public void testCloseReopenOnRollTime() throws InterruptedException, - LifecycleException, EventDeliveryException, IOException { + public void testCloseReopenOnRollTime() + throws InterruptedException, LifecycleException, EventDeliveryException, IOException { LOG.debug("Starting..."); final int numBatches = 4; @@ -973,7 +974,7 @@ public void testCloseReopenOnRollTime() throws InterruptedException, eventDate.clear(); eventDate.set(2011, i, i, i, 0); // yy mm dd event.getHeaders().put("timestamp", - String.valueOf(eventDate.getTimeInMillis())); + String.valueOf(eventDate.getTimeInMillis())); event.getHeaders().put("hostname", "Host" + i); String body = "Test." + i + "." + j; event.setBody(body.getBytes()); @@ -997,9 +998,9 @@ public void testCloseReopenOnRollTime() throws InterruptedException, Assert.assertTrue(badWriterFactory.openCount.get() >= 2); LOG.info("Total number of bucket writers opened: {}", - badWriterFactory.openCount.get()); + badWriterFactory.openCount.get()); verifyOutputSequenceFiles(fs, conf, dirPath.toUri().getPath(), fileName, - bodies); + bodies); } /** @@ -1007,8 +1008,8 @@ public void testCloseReopenOnRollTime() throws InterruptedException, * sfWriters map. */ @Test - public void testCloseRemovesFromSFWriters() throws InterruptedException, - LifecycleException, EventDeliveryException, IOException { + public void testCloseRemovesFromSFWriters() + throws InterruptedException, LifecycleException, EventDeliveryException, IOException { LOG.debug("Starting..."); final String fileName = "FlumeData"; @@ -1055,7 +1056,7 @@ public void testCloseRemovesFromSFWriters() throws InterruptedException, eventDate.clear(); eventDate.set(2011, i, i, i, 0); // yy mm dd event.getHeaders().put("timestamp", - String.valueOf(eventDate.getTimeInMillis())); + String.valueOf(eventDate.getTimeInMillis())); event.getHeaders().put("hostname", "Host" + i); String body = "Test." + i + "." + j; event.setBody(body.getBytes()); @@ -1080,9 +1081,9 @@ public void testCloseRemovesFromSFWriters() throws InterruptedException, sink.stop(); LOG.info("Total number of bucket writers opened: {}", - badWriterFactory.openCount.get()); + badWriterFactory.openCount.get()); verifyOutputSequenceFiles(fs, conf, dirPath.toUri().getPath(), fileName, - bodies); + bodies); } @@ -1163,8 +1164,9 @@ public void testSlowAppendFailure() throws InterruptedException, * append using slow sink writer with specified append timeout * verify that the data is written correctly to files */ - private void slowAppendTestHelper (long appendTimeout) throws InterruptedException, IOException, - LifecycleException, EventDeliveryException, IOException { + private void slowAppendTestHelper(long appendTimeout) + throws InterruptedException, IOException, LifecycleException, EventDeliveryException, + IOException { final String fileName = "FlumeData"; final long rollCount = 5; final long batchSize = 2; @@ -1230,7 +1232,7 @@ private void slowAppendTestHelper (long appendTimeout) throws InterruptedExcept // loop through all the files generated and check their contains FileStatus[] dirStat = fs.listStatus(dirPath); - Path fList[] = FileUtil.stat2Paths(dirStat); + Path[] fList = FileUtil.stat2Paths(dirStat); // check that the roll happened correctly for the given data // Note that we'll end up with two files with only a head @@ -1292,7 +1294,7 @@ public void testCloseOnIdle() throws IOException, EventDeliveryException, Interr Transaction txn = channel.getTransaction(); txn.begin(); - for(int i=0; i < 10; i++) { + for (int i = 0; i < 10; i++) { Event event = new SimpleEvent(); event.setBody(("test event " + i).getBytes()); channel.put(event); @@ -1317,9 +1319,9 @@ public void testCloseOnIdle() throws IOException, EventDeliveryException, Interr FileStatus[] dirStat = fs.listStatus(dirPath); Path[] fList = FileUtil.stat2Paths(dirStat); Assert.assertEquals("Incorrect content of the directory " + StringUtils.join(fList, ","), - 2, fList.length); + 2, fList.length); Assert.assertTrue(!fList[0].getName().endsWith(".tmp") && - !fList[1].getName().endsWith(".tmp")); + !fList[1].getName().endsWith(".tmp")); fs.close(); } @@ -1338,8 +1340,7 @@ private Context getContextForRetryTests() { } @Test - public void testBadConfigurationForRetryIntervalZero() throws - Exception { + public void testBadConfigurationForRetryIntervalZero() throws Exception { Context context = getContextForRetryTests(); context.put("hdfs.retryInterval", "0"); @@ -1348,43 +1349,41 @@ public void testBadConfigurationForRetryIntervalZero() throws } @Test - public void testBadConfigurationForRetryIntervalNegative() throws - Exception { + public void testBadConfigurationForRetryIntervalNegative() throws Exception { Context context = getContextForRetryTests(); context.put("hdfs.retryInterval", "-1"); Configurables.configure(sink, context); Assert.assertEquals(1, sink.getTryCount()); } + @Test - public void testBadConfigurationForRetryCountZero() throws - Exception { + public void testBadConfigurationForRetryCountZero() throws Exception { Context context = getContextForRetryTests(); context.put("hdfs.closeTries" ,"0"); Configurables.configure(sink, context); Assert.assertEquals(Integer.MAX_VALUE, sink.getTryCount()); } + @Test - public void testBadConfigurationForRetryCountNegative() throws - Exception { + public void testBadConfigurationForRetryCountNegative() throws Exception { Context context = getContextForRetryTests(); context.put("hdfs.closeTries" ,"-4"); Configurables.configure(sink, context); Assert.assertEquals(Integer.MAX_VALUE, sink.getTryCount()); } + @Test - public void testRetryRename() throws InterruptedException, - LifecycleException, - EventDeliveryException, IOException { + public void testRetryRename() + throws InterruptedException, LifecycleException, EventDeliveryException, IOException { testRetryRename(true); testRetryRename(false); } - private void testRetryRename(boolean closeSucceed) throws InterruptedException, - LifecycleException, - EventDeliveryException, IOException { + private void testRetryRename(boolean closeSucceed) + throws InterruptedException, LifecycleException, EventDeliveryException, IOException { LOG.debug("Starting..."); String newPath = testPath + "/retryBucket"; @@ -1441,8 +1440,8 @@ private void testRetryRename(boolean closeSucceed) throws InterruptedException, Collection writers = sink.getSfWriters().values(); int totalRenameAttempts = 0; - for(BucketWriter writer: writers) { - LOG.info("Rename tries = "+ writer.renameTries.get()); + for (BucketWriter writer : writers) { + LOG.info("Rename tries = " + writer.renameTries.get()); totalRenameAttempts += writer.renameTries.get(); } // stop clears the sfWriters map, so we need to compute the diff --git a/flume-ng-sinks/flume-hdfs-sink/src/test/java/org/apache/flume/sink/hdfs/TestSequenceFileSerializerFactory.java b/flume-ng-sinks/flume-hdfs-sink/src/test/java/org/apache/flume/sink/hdfs/TestSequenceFileSerializerFactory.java index 6381edcc61..974e857c15 100644 --- a/flume-ng-sinks/flume-hdfs-sink/src/test/java/org/apache/flume/sink/hdfs/TestSequenceFileSerializerFactory.java +++ b/flume-ng-sinks/flume-hdfs-sink/src/test/java/org/apache/flume/sink/hdfs/TestSequenceFileSerializerFactory.java @@ -47,8 +47,7 @@ public void getWritableFormatter() { @Test public void getCustomFormatter() { - SequenceFileSerializer formatter = SequenceFileSerializerFactory - .getSerializer( + SequenceFileSerializer formatter = SequenceFileSerializerFactory.getSerializer( "org.apache.flume.sink.hdfs.MyCustomSerializer$Builder", new Context()); assertTrue(formatter != null); diff --git a/flume-ng-sinks/flume-hive-sink/src/test/java/org/apache/flume/sink/hive/TestHiveSink.java b/flume-ng-sinks/flume-hive-sink/src/test/java/org/apache/flume/sink/hive/TestHiveSink.java index 46724f29be..c4174049a6 100644 --- a/flume-ng-sinks/flume-hive-sink/src/test/java/org/apache/flume/sink/hive/TestHiveSink.java +++ b/flume-ng-sinks/flume-hive-sink/src/test/java/org/apache/flume/sink/hive/TestHiveSink.java @@ -55,8 +55,8 @@ public class TestHiveSink { // 1) partitioned table - final static String dbName = "testing"; - final static String tblName = "alerts"; + static final String dbName = "testing"; + static final String tblName = "alerts"; public static final String PART1_NAME = "continent"; public static final String PART2_NAME = "country"; @@ -72,8 +72,8 @@ public class TestHiveSink { private final ArrayList partitionVals; // 2) un-partitioned table - final static String dbName2 = "testing2"; - final static String tblName2 = "alerts2"; + static final String dbName2 = "testing2"; + static final String tblName2 = "alerts2"; final String[] colNames2 = {COL1,COL2}; private String[] colTypes2 = { "int", "string" }; @@ -88,7 +88,6 @@ public class TestHiveSink { @Rule public TemporaryFolder dbFolder = new TemporaryFolder(); - private static final Logger LOG = LoggerFactory.getLogger(HiveSink.class); public TestHiveSink() throws Exception { @@ -182,8 +181,8 @@ public void testSingleWriterSimpleUnPartitionedTable() TestUtil.dropDB(conf, dbName2); String dbLocation = dbFolder.newFolder(dbName2).getCanonicalPath() + ".db"; dbLocation = dbLocation.replaceAll("\\\\","/"); // for windows paths - TestUtil.createDbAndTable(driver, dbName2, tblName2, null, colNames2, colTypes2 - , null, dbLocation); + TestUtil.createDbAndTable(driver, dbName2, tblName2, null, colNames2, colTypes2, + null, dbLocation); try { int totalRecords = 4; @@ -282,7 +281,7 @@ public void testSingleWriterUseHeaders() String body = j + ",blah,This is a log message,other stuff"; event.setBody(body.getBytes()); eventDate.clear(); - eventDate.set(2014, 03, 03, j%batchCount, 1); // yy mm dd hh mm + eventDate.set(2014, 03, 03, j % batchCount, 1); // yy mm dd hh mm event.getHeaders().put( "timestamp", String.valueOf(eventDate.getTimeInMillis()) ); event.getHeaders().put( PART1_NAME, "Asia" ); @@ -317,7 +316,7 @@ public void testHeartBeat() throws EventDeliveryException, IOException, CommandNeedRetryException { int batchSize = 2; int batchCount = 3; - int totalRecords = batchCount*batchSize; + int totalRecords = batchCount * batchSize; Context context = new Context(); context.put("hive.metastore", metaStoreURI); context.put("hive.database", dbName); @@ -340,7 +339,7 @@ public void testHeartBeat() txn.begin(); for (int j = 1; j <= batchSize; j++) { Event event = new SimpleEvent(); - String body = i*j + ",blah,This is a log message,other stuff"; + String body = i * j + ",blah,This is a log message,other stuff"; event.setBody(body.getBytes()); bodies.add(body); channel.put(event); @@ -361,7 +360,7 @@ public void testHeartBeat() public void testJsonSerializer() throws Exception { int batchSize = 2; int batchCount = 2; - int totalRecords = batchCount*batchSize; + int totalRecords = batchCount * batchSize; Context context = new Context(); context.put("hive.metastore",metaStoreURI); context.put("hive.database",dbName); diff --git a/flume-ng-sinks/flume-hive-sink/src/test/java/org/apache/flume/sink/hive/TestHiveWriter.java b/flume-ng-sinks/flume-hive-sink/src/test/java/org/apache/flume/sink/hive/TestHiveWriter.java index 41bf0f6813..4d7c9bb8ca 100644 --- a/flume-ng-sinks/flume-hive-sink/src/test/java/org/apache/flume/sink/hive/TestHiveWriter.java +++ b/flume-ng-sinks/flume-hive-sink/src/test/java/org/apache/flume/sink/hive/TestHiveWriter.java @@ -42,8 +42,8 @@ import java.util.concurrent.Executors; public class TestHiveWriter { - final static String dbName = "testing"; - final static String tblName = "alerts"; + static final String dbName = "testing"; + static final String tblName = "alerts"; public static final String PART1_NAME = "continent"; public static final String PART2_NAME = "country"; @@ -106,8 +106,8 @@ public void setUp() throws Exception { TestUtil.dropDB(conf, dbName); String dbLocation = dbFolder.newFolder(dbName).getCanonicalPath() + ".db"; dbLocation = dbLocation.replaceAll("\\\\","/"); // for windows paths - TestUtil.createDbAndTable(driver, dbName, tblName, partVals, colNames, colTypes - , partNames, dbLocation); + TestUtil.createDbAndTable(driver, dbName, tblName, partVals, colNames, colTypes, partNames, + dbLocation); // 2) Setup serializer Context ctx = new Context(); @@ -120,8 +120,8 @@ public void setUp() throws Exception { public void testInstantiate() throws Exception { HiveEndPoint endPoint = new HiveEndPoint(metaStoreURI, dbName, tblName, partVals); SinkCounter sinkCounter = new SinkCounter(this.getClass().getName()); - HiveWriter writer = new HiveWriter(endPoint, 10, true, timeout - , callTimeoutPool, "flumetest", serializer, sinkCounter); + HiveWriter writer = new HiveWriter(endPoint, 10, true, timeout, callTimeoutPool, "flumetest", + serializer, sinkCounter); writer.close(); } @@ -130,8 +130,8 @@ public void testInstantiate() throws Exception { public void testWriteBasic() throws Exception { HiveEndPoint endPoint = new HiveEndPoint(metaStoreURI, dbName, tblName, partVals); SinkCounter sinkCounter = new SinkCounter(this.getClass().getName()); - HiveWriter writer = new HiveWriter(endPoint, 10, true, timeout - , callTimeoutPool, "flumetest", serializer, sinkCounter); + HiveWriter writer = new HiveWriter(endPoint, 10, true, timeout, callTimeoutPool, "flumetest", + serializer, sinkCounter); writeEvents(writer,3); writer.flush(false); @@ -144,8 +144,8 @@ public void testWriteMultiFlush() throws Exception { HiveEndPoint endPoint = new HiveEndPoint(metaStoreURI, dbName, tblName, partVals); SinkCounter sinkCounter = new SinkCounter(this.getClass().getName()); - HiveWriter writer = new HiveWriter(endPoint, 10, true, timeout - , callTimeoutPool, "flumetest", serializer, sinkCounter); + HiveWriter writer = new HiveWriter(endPoint, 10, true, timeout, callTimeoutPool, "flumetest", + serializer, sinkCounter); checkRecordCountInTable(0); SimpleEvent event = new SimpleEvent(); @@ -184,8 +184,8 @@ public void testTxnBatchConsumption() throws Exception { int txnPerBatch = 3; - HiveWriter writer = new HiveWriter(endPoint, txnPerBatch, true, timeout - , callTimeoutPool, "flumetest", serializer, sinkCounter); + HiveWriter writer = new HiveWriter(endPoint, txnPerBatch, true, timeout, callTimeoutPool, + "flumetest", serializer, sinkCounter); Assert.assertEquals(writer.getRemainingTxns(),2); writer.flush(true); @@ -275,14 +275,13 @@ public void testSerdeSeparatorCharParsing() throws Exception { ctx.put("serializer.serdeSeparator", "ab"); try { serializer3.configure(ctx); - Assert.assertTrue("Bad serdeSeparator character was accepted" ,false); - } catch (Exception e){ + Assert.assertTrue("Bad serdeSeparator character was accepted", false); + } catch (Exception e) { // expect an exception } } - @Test public void testSecondWriterBeforeFirstCommits() throws Exception { // here we open a new writer while the first is still writing (not committed) @@ -295,13 +294,13 @@ public void testSecondWriterBeforeFirstCommits() throws Exception { SinkCounter sinkCounter1 = new SinkCounter(this.getClass().getName()); SinkCounter sinkCounter2 = new SinkCounter(this.getClass().getName()); - HiveWriter writer1 = new HiveWriter(endPoint1, 10, true, timeout - , callTimeoutPool, "flumetest", serializer, sinkCounter1); + HiveWriter writer1 = new HiveWriter(endPoint1, 10, true, timeout, callTimeoutPool, "flumetest", + serializer, sinkCounter1); writeEvents(writer1, 3); - HiveWriter writer2 = new HiveWriter(endPoint2, 10, true, timeout - , callTimeoutPool, "flumetest", serializer, sinkCounter2); + HiveWriter writer2 = new HiveWriter(endPoint2, 10, true, timeout, callTimeoutPool, "flumetest", + serializer, sinkCounter2); writeEvents(writer2, 3); writer2.flush(false); // commit @@ -311,7 +310,6 @@ public void testSecondWriterBeforeFirstCommits() throws Exception { writer2.close(); } - @Test public void testSecondWriterAfterFirstCommits() throws Exception { // here we open a new writer after the first writer has committed one txn @@ -324,16 +322,16 @@ public void testSecondWriterAfterFirstCommits() throws Exception { SinkCounter sinkCounter1 = new SinkCounter(this.getClass().getName()); SinkCounter sinkCounter2 = new SinkCounter(this.getClass().getName()); - HiveWriter writer1 = new HiveWriter(endPoint1, 10, true, timeout - , callTimeoutPool, "flumetest", serializer, sinkCounter1); + HiveWriter writer1 = new HiveWriter(endPoint1, 10, true, timeout, callTimeoutPool, "flumetest", + serializer, sinkCounter1); writeEvents(writer1, 3); writer1.flush(false); // commit - HiveWriter writer2 = new HiveWriter(endPoint2, 10, true, timeout - , callTimeoutPool, "flumetest", serializer, sinkCounter2); + HiveWriter writer2 = new HiveWriter(endPoint2, 10, true, timeout, callTimeoutPool, "flumetest", + serializer, sinkCounter2); writeEvents(writer2, 3); writer2.flush(false); // commit @@ -342,8 +340,8 @@ public void testSecondWriterAfterFirstCommits() throws Exception { writer2.close(); } - - private void writeEvents(HiveWriter writer, int count) throws InterruptedException, HiveWriter.WriteException { + private void writeEvents(HiveWriter writer, int count) + throws InterruptedException, HiveWriter.WriteException { SimpleEvent event = new SimpleEvent(); for (int i = 1; i <= count; i++) { event.setBody((i + ",xyz,Hello world,abc").getBytes()); diff --git a/flume-ng-sinks/flume-hive-sink/src/test/java/org/apache/flume/sink/hive/TestUtil.java b/flume-ng-sinks/flume-hive-sink/src/test/java/org/apache/flume/sink/hive/TestUtil.java index 107789f168..1fcb4eb46f 100644 --- a/flume-ng-sinks/flume-hive-sink/src/test/java/org/apache/flume/sink/hive/TestUtil.java +++ b/flume-ng-sinks/flume-hive-sink/src/test/java/org/apache/flume/sink/hive/TestUtil.java @@ -46,7 +46,7 @@ public class TestUtil { - private final static String txnMgr = "org.apache.hadoop.hive.ql.lockmgr.DbTxnManager"; + private static final String txnMgr = "org.apache.hadoop.hive.ql.lockmgr.DbTxnManager"; /** * Set up the configuration so it will use the DbTxnManager, concurrency will be set to true, @@ -81,7 +81,7 @@ public static void createDbAndTable(Driver driver, String databaseName, runDDL(driver, crtTbl); System.out.println("crtTbl = " + crtTbl); - if (partNames!=null && partNames.length!=0) { + if (partNames != null && partNames.length != 0) { String addPart = "alter table " + tableName + " add partition ( " + getTablePartsStr2(partNames, partVals) + " )"; runDDL(driver, addPart); @@ -96,7 +96,8 @@ private static String getPartitionStmtStr(String[] partNames) { } // delete db and all tables in it - public static void dropDB(HiveConf conf, String databaseName) throws HiveException, MetaException { + public static void dropDB(HiveConf conf, String databaseName) + throws HiveException, MetaException { IMetaStoreClient client = new HiveMetaStoreClient(conf); try { for (String table : client.listTableNamesByFilter(databaseName, "", (short)-1)) { @@ -110,9 +111,9 @@ public static void dropDB(HiveConf conf, String databaseName) throws HiveExcepti private static String getTableColumnsStr(String[] colNames, String[] colTypes) { StringBuffer sb = new StringBuffer(); - for (int i=0; i < colNames.length; ++i) { + for (int i = 0; i < colNames.length; ++i) { sb.append(colNames[i] + " " + colTypes[i]); - if (i partVals) { StringBuffer sb = new StringBuffer(); - for (int i=0; i < partVals.size(); ++i) { + for (int i = 0; i < partVals.size(); ++i) { sb.append(partNames[i] + " = '" + partVals.get(i) + "'"); - if (i < partVals.size()-1) { + if (i < partVals.size() - 1) { sb.append(","); } } @@ -147,7 +148,7 @@ private static String getTablePartsStr2(String[] partNames, List partVal } public static ArrayList listRecordsInTable(Driver driver, String dbName, String tblName) - throws CommandNeedRetryException, IOException { + throws CommandNeedRetryException, IOException { driver.run("select * from " + dbName + "." + tblName); ArrayList res = new ArrayList(); driver.getResults(res); @@ -155,8 +156,9 @@ public static ArrayList listRecordsInTable(Driver driver, String dbName, } public static ArrayList listRecordsInPartition(Driver driver, String dbName, - String tblName, String continent, String country) - throws CommandNeedRetryException, IOException { + String tblName, String continent, + String country) + throws CommandNeedRetryException, IOException { driver.run("select * from " + dbName + "." + tblName + " where continent='" + continent + "' and country='" + country + "'"); ArrayList res = new ArrayList(); @@ -164,9 +166,9 @@ public static ArrayList listRecordsInPartition(Driver driver, String dbN return res; } - public static class RawFileSystem extends RawLocalFileSystem { private static final URI NAME; + static { try { NAME = new URI("raw:///"); @@ -211,9 +213,10 @@ public FileStatus getFileStatus(Path path) throws IOException { FsPermission.createImmutable(mod), "owen", "users", path); } } + private static boolean runDDL(Driver driver, String sql) throws QueryFailedException { int retryCount = 1; // # of times to retry if first attempt fails - for (int attempt=0; attempt <= retryCount; ++attempt) { + for (int attempt = 0; attempt <= retryCount; ++attempt) { try { driver.run(sql); return true; diff --git a/flume-ng-sinks/flume-irc-sink/src/test/java/org/apache/flume/sink/irc/TestIRCSink.java b/flume-ng-sinks/flume-irc-sink/src/test/java/org/apache/flume/sink/irc/TestIRCSink.java index e6c065ee0d..32517d19f1 100644 --- a/flume-ng-sinks/flume-irc-sink/src/test/java/org/apache/flume/sink/irc/TestIRCSink.java +++ b/flume-ng-sinks/flume-irc-sink/src/test/java/org/apache/flume/sink/irc/TestIRCSink.java @@ -19,7 +19,12 @@ import org.apache.commons.io.FileUtils; import org.apache.commons.io.IOUtils; -import org.apache.flume.*; +import org.apache.flume.Channel; +import org.apache.flume.Context; +import org.apache.flume.Event; +import org.apache.flume.EventDeliveryException; +import org.apache.flume.Sink; +import org.apache.flume.Transaction; import org.apache.flume.channel.MemoryChannel; import org.apache.flume.conf.Configurables; import org.apache.flume.event.EventBuilder; @@ -116,7 +121,9 @@ public void run() { try { Socket socket = ss.accept(); process(socket); - } catch (Exception ex) {/* noop */ } + } catch (Exception ex) { + /* noop */ + } } } catch (IOException e) { // noop diff --git a/flume-ng-sinks/flume-ng-elasticsearch-sink/src/test/java/org/apache/flume/sink/elasticsearch/AbstractElasticSearchSinkTest.java b/flume-ng-sinks/flume-ng-elasticsearch-sink/src/test/java/org/apache/flume/sink/elasticsearch/AbstractElasticSearchSinkTest.java index f9272fa78d..9fbd7474b4 100644 --- a/flume-ng-sinks/flume-ng-elasticsearch-sink/src/test/java/org/apache/flume/sink/elasticsearch/AbstractElasticSearchSinkTest.java +++ b/flume-ng-sinks/flume-ng-elasticsearch-sink/src/test/java/org/apache/flume/sink/elasticsearch/AbstractElasticSearchSinkTest.java @@ -18,17 +18,6 @@ */ package org.apache.flume.sink.elasticsearch; -import static org.apache.flume.sink.elasticsearch.ElasticSearchSinkConstants.BATCH_SIZE; -import static org.apache.flume.sink.elasticsearch.ElasticSearchSinkConstants.CLUSTER_NAME; -import static org.apache.flume.sink.elasticsearch.ElasticSearchSinkConstants.INDEX_NAME; -import static org.apache.flume.sink.elasticsearch.ElasticSearchSinkConstants.INDEX_TYPE; -import static org.apache.flume.sink.elasticsearch.ElasticSearchSinkConstants.TTL; -import static org.junit.Assert.assertEquals; - -import java.util.Arrays; -import java.util.Comparator; -import java.util.Map; - import org.apache.flume.Channel; import org.apache.flume.Context; import org.apache.flume.Event; @@ -51,6 +40,17 @@ import org.junit.After; import org.junit.Before; +import java.util.Arrays; +import java.util.Comparator; +import java.util.Map; + +import static org.apache.flume.sink.elasticsearch.ElasticSearchSinkConstants.BATCH_SIZE; +import static org.apache.flume.sink.elasticsearch.ElasticSearchSinkConstants.CLUSTER_NAME; +import static org.apache.flume.sink.elasticsearch.ElasticSearchSinkConstants.INDEX_NAME; +import static org.apache.flume.sink.elasticsearch.ElasticSearchSinkConstants.INDEX_TYPE; +import static org.apache.flume.sink.elasticsearch.ElasticSearchSinkConstants.TTL; +import static org.junit.Assert.assertEquals; + public abstract class AbstractElasticSearchSinkTest { static final String DEFAULT_INDEX_NAME = "flume"; @@ -136,7 +136,8 @@ SearchResponse performSearch(QueryBuilder query) { .setTypes(DEFAULT_INDEX_TYPE).setQuery(query).execute().actionGet(); } - void assertSearch(int expectedHits, SearchResponse response, Map expectedBody, Event... events) { + void assertSearch(int expectedHits, SearchResponse response, Map expectedBody, + Event... events) { SearchHits hitResponse = response.getHits(); assertEquals(expectedHits, hitResponse.getTotalHits()); diff --git a/flume-ng-sinks/flume-ng-elasticsearch-sink/src/test/java/org/apache/flume/sink/elasticsearch/TestElasticSearchIndexRequestBuilderFactory.java b/flume-ng-sinks/flume-ng-elasticsearch-sink/src/test/java/org/apache/flume/sink/elasticsearch/TestElasticSearchIndexRequestBuilderFactory.java index 8022111312..b62254e9cc 100644 --- a/flume-ng-sinks/flume-ng-elasticsearch-sink/src/test/java/org/apache/flume/sink/elasticsearch/TestElasticSearchIndexRequestBuilderFactory.java +++ b/flume-ng-sinks/flume-ng-elasticsearch-sink/src/test/java/org/apache/flume/sink/elasticsearch/TestElasticSearchIndexRequestBuilderFactory.java @@ -18,15 +18,7 @@ */ package org.apache.flume.sink.elasticsearch; -import static org.junit.Assert.assertArrayEquals; -import static org.junit.Assert.assertEquals; -import static org.junit.Assert.assertFalse; -import static org.junit.Assert.assertNull; -import static org.junit.Assert.assertTrue; - -import java.io.IOException; -import java.util.Map; - +import com.google.common.collect.Maps; import org.apache.flume.Context; import org.apache.flume.Event; import org.apache.flume.conf.ComponentConfiguration; @@ -39,7 +31,14 @@ import org.junit.Before; import org.junit.Test; -import com.google.common.collect.Maps; +import java.io.IOException; +import java.util.Map; + +import static org.junit.Assert.assertArrayEquals; +import static org.junit.Assert.assertEquals; +import static org.junit.Assert.assertFalse; +import static org.junit.Assert.assertNull; +import static org.junit.Assert.assertTrue; public class TestElasticSearchIndexRequestBuilderFactory extends AbstractElasticSearchSinkTest { @@ -70,7 +69,7 @@ public void shouldUseUtcAsBasisForDateFormat() { @Test public void indexNameShouldBePrefixDashFormattedTimestamp() { long millis = 987654321L; - assertEquals("prefix-"+factory.fastDateFormat.format(millis), + assertEquals("prefix-" + factory.fastDateFormat.format(millis), factory.getIndexName("prefix", millis)); } @@ -135,7 +134,7 @@ public void shouldSetIndexNameTypeAndSerializedEventIntoIndexRequest() assertEquals(indexPrefix + '-' + ElasticSearchIndexRequestBuilderFactory.df.format(FIXED_TIME_MILLIS), - indexRequestBuilder.request().index()); + indexRequestBuilder.request().index()); assertEquals(indexType, indexRequestBuilder.request().type()); assertArrayEquals(FakeEventSerializer.FAKE_BYTES, indexRequestBuilder.request().source().array()); @@ -154,7 +153,7 @@ public void shouldSetIndexNameFromTimestampHeaderWhenPresent() assertEquals(indexPrefix + '-' + ElasticSearchIndexRequestBuilderFactory.df.format(1213141516L), - indexRequestBuilder.request().index()); + indexRequestBuilder.request().index()); } @Test @@ -174,7 +173,7 @@ public void shouldSetIndexNameTypeFromHeaderWhenPresent() assertEquals(indexValue + '-' + ElasticSearchIndexRequestBuilderFactory.df.format(FIXED_TIME_MILLIS), - indexRequestBuilder.request().index()); + indexRequestBuilder.request().index()); assertEquals(typeValue, indexRequestBuilder.request().type()); } @@ -192,7 +191,8 @@ public void shouldConfigureEventSerializer() throws Exception { static class FakeEventSerializer implements ElasticSearchEventSerializer { static final byte[] FAKE_BYTES = new byte[]{9, 8, 7, 6}; - boolean configuredWithContext, configuredWithComponentConfiguration; + boolean configuredWithContext; + boolean configuredWithComponentConfiguration; @Override public BytesStream getContentBuilder(Event event) throws IOException { diff --git a/flume-ng-sinks/flume-ng-elasticsearch-sink/src/test/java/org/apache/flume/sink/elasticsearch/TestElasticSearchLogStashEventSerializer.java b/flume-ng-sinks/flume-ng-elasticsearch-sink/src/test/java/org/apache/flume/sink/elasticsearch/TestElasticSearchLogStashEventSerializer.java index ab9587d282..65b4dababb 100644 --- a/flume-ng-sinks/flume-ng-elasticsearch-sink/src/test/java/org/apache/flume/sink/elasticsearch/TestElasticSearchLogStashEventSerializer.java +++ b/flume-ng-sinks/flume-ng-elasticsearch-sink/src/test/java/org/apache/flume/sink/elasticsearch/TestElasticSearchLogStashEventSerializer.java @@ -18,6 +18,7 @@ */ package org.apache.flume.sink.elasticsearch; +import com.google.gson.JsonParser; import org.apache.flume.Context; import org.apache.flume.Event; import org.apache.flume.event.EventBuilder; @@ -28,9 +29,6 @@ import java.util.Date; import java.util.Map; -import com.google.gson.JsonParser; -import com.google.gson.JsonElement; - import static org.apache.flume.sink.elasticsearch.ElasticSearchEventSerializer.charset; import static org.elasticsearch.common.xcontent.XContentFactory.jsonBuilder; import static org.junit.Assert.assertEquals; @@ -56,26 +54,25 @@ public void testRoundTrip() throws Exception { Event event = EventBuilder.withBody(message.getBytes(charset)); event.setHeaders(headers); - XContentBuilder expected = jsonBuilder() - .startObject(); - expected.field("@message", new String(message.getBytes(), charset)); - expected.field("@timestamp", new Date(timestamp)); - expected.field("@source", "flume_tail_src"); - expected.field("@type", "sometype"); - expected.field("@source_host", "test@localhost"); - expected.field("@source_path", "/tmp/test"); - - expected.startObject("@fields"); - expected.field("timestamp", String.valueOf(timestamp)); - expected.field("src_path", "/tmp/test"); - expected.field("host", "test@localhost"); - expected.field("headerNameTwo", "headerValueTwo"); - expected.field("source", "flume_tail_src"); - expected.field("headerNameOne", "headerValueOne"); - expected.field("type", "sometype"); - expected.endObject(); - - expected.endObject(); + XContentBuilder expected = jsonBuilder().startObject(); + expected.field("@message", new String(message.getBytes(), charset)); + expected.field("@timestamp", new Date(timestamp)); + expected.field("@source", "flume_tail_src"); + expected.field("@type", "sometype"); + expected.field("@source_host", "test@localhost"); + expected.field("@source_path", "/tmp/test"); + + expected.startObject("@fields"); + expected.field("timestamp", String.valueOf(timestamp)); + expected.field("src_path", "/tmp/test"); + expected.field("host", "test@localhost"); + expected.field("headerNameTwo", "headerValueTwo"); + expected.field("source", "flume_tail_src"); + expected.field("headerNameOne", "headerValueOne"); + expected.field("type", "sometype"); + expected.endObject(); + + expected.endObject(); XContentBuilder actual = fixture.getContentBuilder(event); @@ -102,26 +99,25 @@ public void shouldHandleInvalidJSONDuringComplexParsing() throws Exception { Event event = EventBuilder.withBody(message.getBytes(charset)); event.setHeaders(headers); - XContentBuilder expected = jsonBuilder(). - startObject(); - expected.field("@message", new String(message.getBytes(), charset)); - expected.field("@timestamp", new Date(timestamp)); - expected.field("@source", "flume_tail_src"); - expected.field("@type", "sometype"); - expected.field("@source_host", "test@localhost"); - expected.field("@source_path", "/tmp/test"); - - expected.startObject("@fields"); - expected.field("timestamp", String.valueOf(timestamp)); - expected.field("src_path", "/tmp/test"); - expected.field("host", "test@localhost"); - expected.field("headerNameTwo", "headerValueTwo"); - expected.field("source", "flume_tail_src"); - expected.field("headerNameOne", "headerValueOne"); - expected.field("type", "sometype"); - expected.endObject(); - - expected.endObject(); + XContentBuilder expected = jsonBuilder().startObject(); + expected.field("@message", new String(message.getBytes(), charset)); + expected.field("@timestamp", new Date(timestamp)); + expected.field("@source", "flume_tail_src"); + expected.field("@type", "sometype"); + expected.field("@source_host", "test@localhost"); + expected.field("@source_path", "/tmp/test"); + + expected.startObject("@fields"); + expected.field("timestamp", String.valueOf(timestamp)); + expected.field("src_path", "/tmp/test"); + expected.field("host", "test@localhost"); + expected.field("headerNameTwo", "headerValueTwo"); + expected.field("source", "flume_tail_src"); + expected.field("headerNameOne", "headerValueOne"); + expected.field("type", "sometype"); + expected.endObject(); + + expected.endObject(); XContentBuilder actual = fixture.getContentBuilder(event); diff --git a/flume-ng-sinks/flume-ng-elasticsearch-sink/src/test/java/org/apache/flume/sink/elasticsearch/TestElasticSearchSink.java b/flume-ng-sinks/flume-ng-elasticsearch-sink/src/test/java/org/apache/flume/sink/elasticsearch/TestElasticSearchSink.java index a58f344669..69acc06993 100644 --- a/flume-ng-sinks/flume-ng-elasticsearch-sink/src/test/java/org/apache/flume/sink/elasticsearch/TestElasticSearchSink.java +++ b/flume-ng-sinks/flume-ng-elasticsearch-sink/src/test/java/org/apache/flume/sink/elasticsearch/TestElasticSearchSink.java @@ -18,25 +18,7 @@ */ package org.apache.flume.sink.elasticsearch; -import static org.apache.flume.sink.elasticsearch.ElasticSearchSinkConstants.BATCH_SIZE; -import static org.apache.flume.sink.elasticsearch.ElasticSearchSinkConstants.CLUSTER_NAME; -import static org.apache.flume.sink.elasticsearch.ElasticSearchSinkConstants.HOSTNAMES; -import static org.apache.flume.sink.elasticsearch.ElasticSearchSinkConstants.INDEX_NAME; -import static org.apache.flume.sink.elasticsearch.ElasticSearchSinkConstants.INDEX_TYPE; -import static org.apache.flume.sink.elasticsearch.ElasticSearchSinkConstants.SERIALIZER; -import static org.apache.flume.sink.elasticsearch.ElasticSearchSinkConstants.TTL; -import static org.junit.Assert.assertArrayEquals; -import static org.junit.Assert.assertEquals; -import static org.junit.Assert.assertTrue; -import static org.junit.Assert.assertNull; - -import java.io.IOException; -import java.util.Map; -import java.util.HashMap; -import java.util.TimeZone; -import java.util.concurrent.TimeUnit; import org.apache.commons.lang.time.FastDateFormat; - import org.apache.flume.Channel; import org.apache.flume.Context; import org.apache.flume.Event; @@ -56,6 +38,24 @@ import org.junit.Before; import org.junit.Test; +import java.io.IOException; +import java.util.HashMap; +import java.util.Map; +import java.util.TimeZone; +import java.util.concurrent.TimeUnit; + +import static org.apache.flume.sink.elasticsearch.ElasticSearchSinkConstants.BATCH_SIZE; +import static org.apache.flume.sink.elasticsearch.ElasticSearchSinkConstants.CLUSTER_NAME; +import static org.apache.flume.sink.elasticsearch.ElasticSearchSinkConstants.HOSTNAMES; +import static org.apache.flume.sink.elasticsearch.ElasticSearchSinkConstants.INDEX_NAME; +import static org.apache.flume.sink.elasticsearch.ElasticSearchSinkConstants.INDEX_TYPE; +import static org.apache.flume.sink.elasticsearch.ElasticSearchSinkConstants.SERIALIZER; +import static org.apache.flume.sink.elasticsearch.ElasticSearchSinkConstants.TTL; +import static org.junit.Assert.assertArrayEquals; +import static org.junit.Assert.assertEquals; +import static org.junit.Assert.assertNull; +import static org.junit.Assert.assertTrue; + public class TestElasticSearchSink extends AbstractElasticSearchSinkTest { private ElasticSearchSink fixture; @@ -373,23 +373,22 @@ public void shouldParseFullyQualifiedTTLs() { public static final class CustomElasticSearchIndexRequestBuilderFactory extends AbstractElasticSearchIndexRequestBuilderFactory { - static String actualIndexName, actualIndexType; + static String actualIndexName; + static String actualIndexType; static byte[] actualEventBody; static boolean hasContext; public CustomElasticSearchIndexRequestBuilderFactory() { - super(FastDateFormat.getInstance("HH_mm_ss_SSS", - TimeZone.getTimeZone("EST5EDT"))); + super(FastDateFormat.getInstance("HH_mm_ss_SSS", TimeZone.getTimeZone("EST5EDT"))); } @Override - protected void prepareIndexRequest(IndexRequestBuilder indexRequest, - String indexName, String indexType, Event event) throws IOException { + protected void prepareIndexRequest(IndexRequestBuilder indexRequest, String indexName, + String indexType, Event event) throws IOException { actualIndexName = indexName; actualIndexType = indexType; actualEventBody = event.getBody(); - indexRequest.setIndex(indexName).setType(indexType) - .setSource(event.getBody()); + indexRequest.setIndex(indexName).setType(indexType).setSource(event.getBody()); } @Override @@ -458,7 +457,8 @@ public void configure(Context arg0) { class FakeEventSerializer implements ElasticSearchEventSerializer { static final byte[] FAKE_BYTES = new byte[] { 9, 8, 7, 6 }; - boolean configuredWithContext, configuredWithComponentConfiguration; + boolean configuredWithContext; + boolean configuredWithComponentConfiguration; @Override public BytesStream getContentBuilder(Event event) throws IOException { diff --git a/flume-ng-sinks/flume-ng-elasticsearch-sink/src/test/java/org/apache/flume/sink/elasticsearch/TestElasticSearchSinkCreation.java b/flume-ng-sinks/flume-ng-elasticsearch-sink/src/test/java/org/apache/flume/sink/elasticsearch/TestElasticSearchSinkCreation.java index b5a4d2f87b..2a3643939d 100644 --- a/flume-ng-sinks/flume-ng-elasticsearch-sink/src/test/java/org/apache/flume/sink/elasticsearch/TestElasticSearchSinkCreation.java +++ b/flume-ng-sinks/flume-ng-elasticsearch-sink/src/test/java/org/apache/flume/sink/elasticsearch/TestElasticSearchSinkCreation.java @@ -28,7 +28,7 @@ public class TestElasticSearchSinkCreation { -private SinkFactory sinkFactory; + private SinkFactory sinkFactory; @Before public void setUp() { diff --git a/flume-ng-sinks/flume-ng-elasticsearch-sink/src/test/java/org/apache/flume/sink/elasticsearch/client/RoundRobinListTest.java b/flume-ng-sinks/flume-ng-elasticsearch-sink/src/test/java/org/apache/flume/sink/elasticsearch/client/RoundRobinListTest.java index 38e7399a3b..0d1d0921dc 100644 --- a/flume-ng-sinks/flume-ng-elasticsearch-sink/src/test/java/org/apache/flume/sink/elasticsearch/client/RoundRobinListTest.java +++ b/flume-ng-sinks/flume-ng-elasticsearch-sink/src/test/java/org/apache/flume/sink/elasticsearch/client/RoundRobinListTest.java @@ -17,10 +17,11 @@ package org.apache.flume.sink.elasticsearch.client; import java.util.Arrays; -import static org.junit.Assert.assertEquals; import org.junit.Before; import org.junit.Test; +import static org.junit.Assert.assertEquals; + public class RoundRobinListTest { private RoundRobinList fixture; diff --git a/flume-ng-sinks/flume-ng-elasticsearch-sink/src/test/java/org/apache/flume/sink/elasticsearch/client/TestElasticSearchClientFactory.java b/flume-ng-sinks/flume-ng-elasticsearch-sink/src/test/java/org/apache/flume/sink/elasticsearch/client/TestElasticSearchClientFactory.java index 4b70b65740..c3f07b05ef 100644 --- a/flume-ng-sinks/flume-ng-elasticsearch-sink/src/test/java/org/apache/flume/sink/elasticsearch/client/TestElasticSearchClientFactory.java +++ b/flume-ng-sinks/flume-ng-elasticsearch-sink/src/test/java/org/apache/flume/sink/elasticsearch/client/TestElasticSearchClientFactory.java @@ -21,10 +21,10 @@ import org.apache.flume.sink.elasticsearch.ElasticSearchEventSerializer; import org.junit.Before; import org.junit.Test; +import org.mockito.Mock; import static org.hamcrest.core.IsInstanceOf.instanceOf; import static org.junit.Assert.assertThat; -import org.mockito.Mock; import static org.mockito.MockitoAnnotations.initMocks; public class TestElasticSearchClientFactory { @@ -44,7 +44,7 @@ public void setUp() { public void shouldReturnTransportClient() throws Exception { String[] hostNames = { "127.0.0.1" }; Object o = factory.getClient(ElasticSearchClientFactory.TransportClient, - hostNames, "test", serializer, null); + hostNames, "test", serializer, null); assertThat(o, instanceOf(ElasticSearchTransportClient.class)); } @@ -52,13 +52,13 @@ public void shouldReturnTransportClient() throws Exception { public void shouldReturnRestClient() throws NoSuchClientTypeException { String[] hostNames = { "127.0.0.1" }; Object o = factory.getClient(ElasticSearchClientFactory.RestClient, - hostNames, "test", serializer, null); + hostNames, "test", serializer, null); assertThat(o, instanceOf(ElasticSearchRestClient.class)); } - @Test(expected=NoSuchClientTypeException.class) + @Test(expected = NoSuchClientTypeException.class) public void shouldThrowNoSuchClientTypeException() throws NoSuchClientTypeException { - String[] hostNames = {"127.0.0.1"}; + String[] hostNames = { "127.0.0.1" }; factory.getClient("not_existing_client", hostNames, "test", null, null); } } diff --git a/flume-ng-sinks/flume-ng-elasticsearch-sink/src/test/java/org/apache/flume/sink/elasticsearch/client/TestElasticSearchRestClient.java b/flume-ng-sinks/flume-ng-elasticsearch-sink/src/test/java/org/apache/flume/sink/elasticsearch/client/TestElasticSearchRestClient.java index 1fe983a8da..9551c81634 100644 --- a/flume-ng-sinks/flume-ng-elasticsearch-sink/src/test/java/org/apache/flume/sink/elasticsearch/client/TestElasticSearchRestClient.java +++ b/flume-ng-sinks/flume-ng-elasticsearch-sink/src/test/java/org/apache/flume/sink/elasticsearch/client/TestElasticSearchRestClient.java @@ -25,6 +25,15 @@ import org.apache.flume.EventDeliveryException; import org.apache.flume.sink.elasticsearch.ElasticSearchEventSerializer; import org.apache.flume.sink.elasticsearch.IndexNameBuilder; +import org.apache.http.HttpEntity; +import org.apache.http.HttpResponse; +import org.apache.http.HttpStatus; +import org.apache.http.StatusLine; +import org.apache.http.client.HttpClient; +import org.apache.http.client.methods.HttpPost; +import org.apache.http.client.methods.HttpUriRequest; +import org.apache.http.util.EntityUtils; +import org.elasticsearch.common.bytes.BytesArray; import org.elasticsearch.common.bytes.BytesReference; import org.elasticsearch.common.io.BytesStream; import org.junit.Before; @@ -38,16 +47,12 @@ import static junit.framework.Assert.assertEquals; import static junit.framework.Assert.assertTrue; -import org.apache.http.HttpEntity; -import org.apache.http.HttpResponse; -import org.apache.http.HttpStatus; -import org.apache.http.StatusLine; -import org.apache.http.client.HttpClient; -import org.apache.http.client.methods.HttpPost; -import org.apache.http.client.methods.HttpUriRequest; -import org.apache.http.util.EntityUtils; -import org.elasticsearch.common.bytes.BytesArray; -import static org.mockito.Mockito.*; +import static org.mockito.Mockito.any; +import static org.mockito.Mockito.isA; +import static org.mockito.Mockito.mock; +import static org.mockito.Mockito.times; +import static org.mockito.Mockito.verify; +import static org.mockito.Mockito.when; import static org.mockito.MockitoAnnotations.initMocks; public class TestElasticSearchRestClient { @@ -126,15 +131,16 @@ public void shouldAddNewEventWithTTL() throws Exception { verify(httpClient).execute(argument.capture()); assertEquals("http://host1/_bulk", argument.getValue().getURI().toString()); - assertTrue(verifyJsonEvents("{\"index\":{\"_type\":\"bar_type\",\"_index\":\"foo_index\",\"_ttl\":\"123\"}}\n", - MESSAGE_CONTENT, EntityUtils.toString(argument.getValue().getEntity()))); + assertTrue(verifyJsonEvents( + "{\"index\":{\"_type\":\"bar_type\",\"_index\":\"foo_index\",\"_ttl\":\"123\"}}\n", + MESSAGE_CONTENT, EntityUtils.toString(argument.getValue().getEntity()))); } private boolean verifyJsonEvents(String expectedIndex, String expectedBody, String actual) { Iterator it = Splitter.on("\n").split(actual).iterator(); JsonParser parser = new JsonParser(); JsonObject[] arr = new JsonObject[2]; - for(int i = 0; i < 2; i++) { + for (int i = 0; i < 2; i++) { arr[i] = (JsonObject) parser.parse(it.next()); } return arr[0].equals(parser.parse(expectedIndex)) && arr[1].equals(parser.parse(expectedBody)); @@ -156,7 +162,8 @@ public void shouldThrowEventDeliveryException() throws Exception { public void shouldRetryBulkOperation() throws Exception { ArgumentCaptor argument = ArgumentCaptor.forClass(HttpPost.class); - when(httpStatus.getStatusCode()).thenReturn(HttpStatus.SC_INTERNAL_SERVER_ERROR, HttpStatus.SC_OK); + when(httpStatus.getStatusCode()).thenReturn(HttpStatus.SC_INTERNAL_SERVER_ERROR, + HttpStatus.SC_OK); when(httpResponse.getStatusLine()).thenReturn(httpStatus); when(httpClient.execute(any(HttpUriRequest.class))).thenReturn(httpResponse); diff --git a/flume-ng-sinks/flume-ng-hbase-sink/src/test/java/org/apache/flume/sink/hbase/IncrementAsyncHBaseSerializer.java b/flume-ng-sinks/flume-ng-hbase-sink/src/test/java/org/apache/flume/sink/hbase/IncrementAsyncHBaseSerializer.java index b8aefe8d4f..9a2be5a014 100644 --- a/flume-ng-sinks/flume-ng-hbase-sink/src/test/java/org/apache/flume/sink/hbase/IncrementAsyncHBaseSerializer.java +++ b/flume-ng-sinks/flume-ng-hbase-sink/src/test/java/org/apache/flume/sink/hbase/IncrementAsyncHBaseSerializer.java @@ -37,6 +37,7 @@ public class IncrementAsyncHBaseSerializer implements AsyncHbaseEventSerializer private byte[] cf; private byte[] column; private Event currentEvent; + @Override public void initialize(byte[] table, byte[] cf) { this.table = table; @@ -55,10 +56,9 @@ public List getActions() { @Override public List getIncrements() { - List incrs - = new ArrayList(); + List incrs = new ArrayList(); AtomicIncrementRequest incr = new AtomicIncrementRequest(table, - currentEvent.getBody(), cf, column, 1); + currentEvent.getBody(), cf, column, 1); incrs.add(incr); return incrs; } diff --git a/flume-ng-sinks/flume-ng-hbase-sink/src/test/java/org/apache/flume/sink/hbase/TestAsyncHBaseSink.java b/flume-ng-sinks/flume-ng-hbase-sink/src/test/java/org/apache/flume/sink/hbase/TestAsyncHBaseSink.java index b4bbd6b3c7..f8faa1e9e2 100644 --- a/flume-ng-sinks/flume-ng-hbase-sink/src/test/java/org/apache/flume/sink/hbase/TestAsyncHBaseSink.java +++ b/flume-ng-sinks/flume-ng-hbase-sink/src/test/java/org/apache/flume/sink/hbase/TestAsyncHBaseSink.java @@ -176,7 +176,7 @@ public void testThreeEvents() throws Exception { sink.start(); Transaction tx = channel.getTransaction(); tx.begin(); - for(int i = 0; i < 3; i++){ + for (int i = 0; i < 3; i++) { Event e = EventBuilder.withBody(Bytes.toBytes(valBase + "-" + i)); channel.put(e); } @@ -189,9 +189,9 @@ public void testThreeEvents() throws Exception { byte[][] results = getResults(table, 3); byte[] out; int found = 0; - for(int i = 0; i < 3; i++){ - for(int j = 0; j < 3; j++){ - if(Arrays.equals(results[j],Bytes.toBytes(valBase + "-" + i))){ + for (int i = 0; i < 3; i++) { + for (int j = 0; j < 3; j++) { + if (Arrays.equals(results[j], Bytes.toBytes(valBase + "-" + i))) { found++; break; } @@ -209,8 +209,7 @@ public void testThreeEvents() throws Exception { public void testTimeOut() throws Exception { testUtility.createTable(tableName.getBytes(), columnFamily.getBytes()); deleteTable = true; - AsyncHBaseSink sink = new AsyncHBaseSink(testUtility.getConfiguration(), - true, false); + AsyncHBaseSink sink = new AsyncHBaseSink(testUtility.getConfiguration(), true, false); Configurables.configure(sink, ctx); Channel channel = new MemoryChannel(); Configurables.configure(channel, ctx); @@ -219,7 +218,7 @@ public void testTimeOut() throws Exception { sink.start(); Transaction tx = channel.getTransaction(); tx.begin(); - for(int i = 0; i < 3; i++){ + for (int i = 0; i < 3; i++) { Event e = EventBuilder.withBody(Bytes.toBytes(valBase + "-" + i)); channel.put(e); } @@ -245,7 +244,7 @@ public void testMultipleBatches() throws Exception { sink.start(); Transaction tx = channel.getTransaction(); tx.begin(); - for(int i = 0; i < 3; i++){ + for (int i = 0; i < 3; i++) { Event e = EventBuilder.withBody(Bytes.toBytes(valBase + "-" + i)); channel.put(e); } @@ -253,7 +252,7 @@ public void testMultipleBatches() throws Exception { tx.close(); int count = 0; Status status = Status.READY; - while(status != Status.BACKOFF){ + while (status != Status.BACKOFF) { count++; status = sink.process(); } @@ -264,9 +263,9 @@ public void testMultipleBatches() throws Exception { byte[][] results = getResults(table, 3); byte[] out; int found = 0; - for(int i = 0; i < 3; i++){ - for(int j = 0; j < 3; j++){ - if(Arrays.equals(results[j],Bytes.toBytes(valBase + "-" + i))){ + for (int i = 0; i < 3; i++) { + for (int j = 0; j < 3; j++) { + if (Arrays.equals(results[j], Bytes.toBytes(valBase + "-" + i))) { found++; break; } @@ -278,26 +277,21 @@ public void testMultipleBatches() throws Exception { } @Test - public void testMultipleBatchesBatchIncrementsWithCoalescing() - throws Exception { + public void testMultipleBatchesBatchIncrementsWithCoalescing() throws Exception { doTestMultipleBatchesBatchIncrements(true); } @Test - public void testMultipleBatchesBatchIncrementsNoCoalescing() - throws Exception { + public void testMultipleBatchesBatchIncrementsNoCoalescing() throws Exception { doTestMultipleBatchesBatchIncrements(false); } - public void doTestMultipleBatchesBatchIncrements(boolean coalesce) throws - Exception { + public void doTestMultipleBatchesBatchIncrements(boolean coalesce) throws Exception { testUtility.createTable(tableName.getBytes(), columnFamily.getBytes()); deleteTable = true; - AsyncHBaseSink sink = new AsyncHBaseSink(testUtility.getConfiguration(), - false, true); + AsyncHBaseSink sink = new AsyncHBaseSink(testUtility.getConfiguration(), false, true); if (coalesce) { - ctx.put(HBaseSinkConfigurationConstants.CONFIG_COALESCE_INCREMENTS, - "true"); + ctx.put(HBaseSinkConfigurationConstants.CONFIG_COALESCE_INCREMENTS, "true"); } ctx.put("batchSize", "2"); ctx.put("serializer", IncrementAsyncHBaseSerializer.class.getName()); @@ -309,7 +303,7 @@ public void doTestMultipleBatchesBatchIncrements(boolean coalesce) throws ctx.put("serializer", SimpleAsyncHbaseEventSerializer.class.getName()); //Restore the no coalescing behavior ctx.put(HBaseSinkConfigurationConstants.CONFIG_COALESCE_INCREMENTS, - "false"); + "false"); Channel channel = new MemoryChannel(); Configurables.configure(channel, ctx); sink.setChannel(channel); @@ -335,7 +329,7 @@ public void doTestMultipleBatchesBatchIncrements(boolean coalesce) throws Assert.assertEquals(7, count); HTable table = new HTable(testUtility.getConfiguration(), tableName); Scan scan = new Scan(); - scan.addColumn(columnFamily.getBytes(),"test".getBytes()); + scan.addColumn(columnFamily.getBytes(), "test".getBytes()); scan.setStartRow(Bytes.toBytes(valBase)); ResultScanner rs = table.getScanner(scan); int i = 0; @@ -358,19 +352,19 @@ public void doTestMultipleBatchesBatchIncrements(boolean coalesce) throws } @Test - public void testWithoutConfigurationObject() throws Exception{ + public void testWithoutConfigurationObject() throws Exception { testUtility.createTable(tableName.getBytes(), columnFamily.getBytes()); deleteTable = true; ctx.put("batchSize", "2"); ctx.put(HBaseSinkConfigurationConstants.ZK_QUORUM, - ZKConfig.getZKQuorumServersString(testUtility.getConfiguration()) ); + ZKConfig.getZKQuorumServersString(testUtility.getConfiguration())); ctx.put(HBaseSinkConfigurationConstants.ZK_ZNODE_PARENT, - testUtility.getConfiguration().get(HConstants.ZOOKEEPER_ZNODE_PARENT)); + testUtility.getConfiguration().get(HConstants.ZOOKEEPER_ZNODE_PARENT)); AsyncHBaseSink sink = new AsyncHBaseSink(); Configurables.configure(sink, ctx); // Reset context to values usable by other tests. ctx.put(HBaseSinkConfigurationConstants.ZK_QUORUM, null); - ctx.put(HBaseSinkConfigurationConstants.ZK_ZNODE_PARENT,null); + ctx.put(HBaseSinkConfigurationConstants.ZK_ZNODE_PARENT, null); ctx.put("batchSize", "100"); Channel channel = new MemoryChannel(); Configurables.configure(channel, ctx); @@ -378,7 +372,7 @@ public void testWithoutConfigurationObject() throws Exception{ sink.start(); Transaction tx = channel.getTransaction(); tx.begin(); - for(int i = 0; i < 3; i++){ + for (int i = 0; i < 3; i++) { Event e = EventBuilder.withBody(Bytes.toBytes(valBase + "-" + i)); channel.put(e); } @@ -386,7 +380,7 @@ public void testWithoutConfigurationObject() throws Exception{ tx.close(); int count = 0; Status status = Status.READY; - while(status != Status.BACKOFF){ + while (status != Status.BACKOFF) { count++; status = sink.process(); } @@ -401,9 +395,9 @@ public void testWithoutConfigurationObject() throws Exception{ byte[][] results = getResults(table, 3); byte[] out; int found = 0; - for(int i = 0; i < 3; i++){ - for(int j = 0; j < 3; j++){ - if(Arrays.equals(results[j],Bytes.toBytes(valBase + "-" + i))){ + for (int i = 0; i < 3; i++) { + for (int j = 0; j < 3; j++) { + if (Arrays.equals(results[j], Bytes.toBytes(valBase + "-" + i))) { found++; break; } @@ -428,7 +422,7 @@ public void testMissingTable() throws Exception { sink.start(); Transaction tx = channel.getTransaction(); tx.begin(); - for(int i = 0; i < 3; i++){ + for (int i = 0; i < 3; i++) { Event e = EventBuilder.withBody(Bytes.toBytes(valBase + "-" + i)); channel.put(e); } @@ -440,9 +434,9 @@ public void testMissingTable() throws Exception { byte[][] results = getResults(table, 2); byte[] out; int found = 0; - for(int i = 0; i < 2; i++){ - for(int j = 0; j < 2; j++){ - if(Arrays.equals(results[j],Bytes.toBytes(valBase + "-" + i))){ + for (int i = 0; i < 2; i++) { + for (int j = 0; j < 2; j++) { + if (Arrays.equals(results[j], Bytes.toBytes(valBase + "-" + i))) { found++; break; } @@ -457,7 +451,7 @@ public void testMissingTable() throws Exception { // We only have support for getting File Descriptor count for Unix from the JDK private long getOpenFileDescriptorCount() { - if(os instanceof UnixOperatingSystemMXBean){ + if (os instanceof UnixOperatingSystemMXBean) { return ((UnixOperatingSystemMXBean) os).getOpenFileDescriptorCount(); } else { return -1; @@ -476,13 +470,13 @@ private long getOpenFileDescriptorCount() { */ @Test public void testFDLeakOnShutdown() throws Exception { - if(getOpenFileDescriptorCount() < 0) { + if (getOpenFileDescriptorCount() < 0) { return; } testUtility.createTable(tableName.getBytes(), columnFamily.getBytes()); deleteTable = true; AsyncHBaseSink sink = new AsyncHBaseSink(testUtility.getConfiguration(), - true, false); + true, false); ctx.put("maxConsecutiveFails", "1"); Configurables.configure(sink, ctx); Channel channel = new MemoryChannel(); @@ -492,7 +486,7 @@ public void testFDLeakOnShutdown() throws Exception { sink.start(); Transaction tx = channel.getTransaction(); tx.begin(); - for(int i = 0; i < 3; i++){ + for (int i = 0; i < 3; i++) { Event e = EventBuilder.withBody(Bytes.toBytes(valBase + "-" + i)); channel.put(e); } @@ -503,7 +497,7 @@ public void testFDLeakOnShutdown() throws Exception { // Since the isTimeOutTest is set to true, transaction will fail // with EventDeliveryException - for(int i = 0; i < 10; i ++) { + for (int i = 0; i < 10; i++) { try { sink.process(); } catch (EventDeliveryException ex) { @@ -511,18 +505,20 @@ public void testFDLeakOnShutdown() throws Exception { } long increaseInFD = getOpenFileDescriptorCount() - initialFDCount; Assert.assertTrue("File Descriptor leak detected. FDs have increased by " + - increaseInFD + " from an initial FD count of " + initialFDCount, increaseInFD < 50); + increaseInFD + " from an initial FD count of " + initialFDCount, + increaseInFD < 50); } /** * This test must run last - it shuts down the minicluster :D + * * @throws Exception */ @Ignore("For dev builds only:" + - "This test takes too long, and this has to be run after all other" + - "tests, since it shuts down the minicluster. " + - "Comment out all other tests" + - "and uncomment this annotation to run this test.") + "This test takes too long, and this has to be run after all other" + + "tests, since it shuts down the minicluster. " + + "Comment out all other tests" + + "and uncomment this annotation to run this test.") @Test(expected = EventDeliveryException.class) public void testHBaseFailure() throws Exception { ctx.put("batchSize", "2"); @@ -538,7 +534,7 @@ public void testHBaseFailure() throws Exception { sink.start(); Transaction tx = channel.getTransaction(); tx.begin(); - for(int i = 0; i < 3; i++){ + for (int i = 0; i < 3; i++) { Event e = EventBuilder.withBody(Bytes.toBytes(valBase + "-" + i)); channel.put(e); } @@ -550,9 +546,9 @@ public void testHBaseFailure() throws Exception { byte[][] results = getResults(table, 2); byte[] out; int found = 0; - for(int i = 0; i < 2; i++){ - for(int j = 0; j < 2; j++){ - if(Arrays.equals(results[j],Bytes.toBytes(valBase + "-" + i))){ + for (int i = 0; i < 2; i++) { + for (int j = 0; j < 2; j++) { + if (Arrays.equals(results[j], Bytes.toBytes(valBase + "-" + i))) { found++; break; } @@ -565,21 +561,23 @@ public void testHBaseFailure() throws Exception { sink.process(); sink.stop(); } + /** * Makes Hbase scans to get rows in the payload column and increment column * in the table given. Expensive, so tread lightly. * Calling this function multiple times for the same result set is a bad * idea. Cache the result set once it is returned by this function. + * * @param table * @param numEvents Number of events inserted into the table * @return * @throws IOException */ - private byte[][] getResults(HTable table, int numEvents) throws IOException{ - byte[][] results = new byte[numEvents+1][]; + private byte[][] getResults(HTable table, int numEvents) throws IOException { + byte[][] results = new byte[numEvents + 1][]; Scan scan = new Scan(); - scan.addColumn(columnFamily.getBytes(),plCol.getBytes()); - scan.setStartRow( Bytes.toBytes("default")); + scan.addColumn(columnFamily.getBytes(), plCol.getBytes()); + scan.setStartRow(Bytes.toBytes("default")); ResultScanner rs = table.getScanner(scan); byte[] out = null; int i = 0; @@ -587,10 +585,10 @@ private byte[][] getResults(HTable table, int numEvents) throws IOException{ for (Result r = rs.next(); r != null; r = rs.next()) { out = r.getValue(columnFamily.getBytes(), plCol.getBytes()); - if(i >= results.length - 1){ + if (i >= results.length - 1) { rs.close(); throw new FlumeException("More results than expected in the table." + - "Expected = " + numEvents +". Found = " + i); + "Expected = " + numEvents + ". Found = " + i); } results[i++] = out; System.out.println(out); @@ -601,7 +599,7 @@ private byte[][] getResults(HTable table, int numEvents) throws IOException{ Assert.assertEquals(i, results.length - 1); scan = new Scan(); - scan.addColumn(columnFamily.getBytes(),inColumn.getBytes()); + scan.addColumn(columnFamily.getBytes(), inColumn.getBytes()); scan.setStartRow(Bytes.toBytes("incRow")); rs = table.getScanner(scan); out = null; diff --git a/flume-ng-sinks/flume-ng-hbase-sink/src/test/java/org/apache/flume/sink/hbase/TestHBaseSink.java b/flume-ng-sinks/flume-ng-hbase-sink/src/test/java/org/apache/flume/sink/hbase/TestHBaseSink.java index ab65a3898b..217913b20d 100644 --- a/flume-ng-sinks/flume-ng-hbase-sink/src/test/java/org/apache/flume/sink/hbase/TestHBaseSink.java +++ b/flume-ng-sinks/flume-ng-hbase-sink/src/test/java/org/apache/flume/sink/hbase/TestHBaseSink.java @@ -18,14 +18,6 @@ */ package org.apache.flume.sink.hbase; -import static org.mockito.Mockito.*; - -import java.io.IOException; -import java.lang.reflect.Method; -import java.util.Arrays; -import java.util.List; -import java.util.Map; -import java.util.NavigableMap; import com.google.common.base.Charsets; import com.google.common.base.Throwables; import com.google.common.collect.Lists; @@ -54,14 +46,25 @@ import org.apache.hadoop.hbase.zookeeper.ZKConfig; import org.junit.After; import org.junit.AfterClass; +import org.junit.Assert; import org.junit.Before; import org.junit.BeforeClass; import org.junit.Ignore; import org.junit.Test; -import org.junit.Assert; import org.slf4j.Logger; import org.slf4j.LoggerFactory; +import java.io.IOException; +import java.lang.reflect.Method; +import java.util.Arrays; +import java.util.List; +import java.util.Map; +import java.util.NavigableMap; + +import static org.mockito.Mockito.doReturn; +import static org.mockito.Mockito.doThrow; +import static org.mockito.Mockito.spy; + public class TestHBaseSink { private static final Logger logger = LoggerFactory.getLogger(TestHBaseSink.class); @@ -140,8 +143,7 @@ public void testOneEventWithDefaults() throws Exception { sink.start(); Transaction tx = channel.getTransaction(); tx.begin(); - Event e = EventBuilder.withBody( - Bytes.toBytes(valBase)); + Event e = EventBuilder.withBody(Bytes.toBytes(valBase)); channel.put(e); tx.commit(); tx.close(); @@ -195,7 +197,7 @@ public void testThreeEvents() throws Exception { sink.start(); Transaction tx = channel.getTransaction(); tx.begin(); - for(int i = 0; i < 3; i++){ + for (int i = 0; i < 3; i++) { Event e = EventBuilder.withBody(Bytes.toBytes(valBase + "-" + i)); channel.put(e); } @@ -207,9 +209,9 @@ public void testThreeEvents() throws Exception { byte[][] results = getResults(table, 3); byte[] out; int found = 0; - for(int i = 0; i < 3; i++){ - for(int j = 0; j < 3; j++){ - if(Arrays.equals(results[j],Bytes.toBytes(valBase + "-" + i))){ + for (int i = 0; i < 3; i++) { + for (int j = 0; j < 3; j++) { + if (Arrays.equals(results[j], Bytes.toBytes(valBase + "-" + i))) { found++; break; } @@ -234,14 +236,14 @@ public void testMultipleBatches() throws Exception { sink.start(); Transaction tx = channel.getTransaction(); tx.begin(); - for(int i = 0; i < 3; i++){ + for (int i = 0; i < 3; i++) { Event e = EventBuilder.withBody(Bytes.toBytes(valBase + "-" + i)); channel.put(e); } tx.commit(); tx.close(); int count = 0; - while(sink.process() != Status.BACKOFF){ + while (sink.process() != Status.BACKOFF) { count++; } sink.stop(); @@ -250,9 +252,9 @@ public void testMultipleBatches() throws Exception { byte[][] results = getResults(table, 3); byte[] out; int found = 0; - for(int i = 0; i < 3; i++){ - for(int j = 0; j < 3; j++){ - if(Arrays.equals(results[j],Bytes.toBytes(valBase + "-" + i))){ + for (int i = 0; i < 3; i++) { + for (int j = 0; j < 3; j++) { + if (Arrays.equals(results[j], Bytes.toBytes(valBase + "-" + i))) { found++; break; } @@ -284,7 +286,7 @@ public void testMissingTable() throws Exception { logger.info("Writing data into channel"); Transaction tx = channel.getTransaction(); tx.begin(); - for(int i = 0; i < 3; i++){ + for (int i = 0; i < 3; i++) { Event e = EventBuilder.withBody(Bytes.toBytes(valBase + "-" + i)); channel.put(e); } @@ -313,9 +315,9 @@ public void testMissingTable() throws Exception { byte[][] results = getResults(table, 2); byte[] out; int found = 0; - for(int i = 0; i < 2; i++){ - for(int j = 0; j < 2; j++){ - if(Arrays.equals(results[j],Bytes.toBytes(valBase + "-" + i))){ + for (int i = 0; i < 2; i++) { + for (int j = 0; j < 2; j++) { + if (Arrays.equals(results[j], Bytes.toBytes(valBase + "-" + i))) { found++; break; } @@ -328,15 +330,17 @@ public void testMissingTable() throws Exception { } // TODO: Move this test to a different class and run it stand-alone. + /** * This test must run last - it shuts down the minicluster :D + * * @throws Exception */ @Ignore("For dev builds only:" + - "This test takes too long, and this has to be run after all other" + - "tests, since it shuts down the minicluster. " + - "Comment out all other tests" + - "and uncomment this annotation to run this test.") + "This test takes too long, and this has to be run after all other" + + "tests, since it shuts down the minicluster. " + + "Comment out all other tests" + + "and uncomment this annotation to run this test.") @Test(expected = EventDeliveryException.class) public void testHBaseFailure() throws Exception { initContextForSimpleHbaseEventSerializer(); @@ -351,7 +355,7 @@ public void testHBaseFailure() throws Exception { sink.start(); Transaction tx = channel.getTransaction(); tx.begin(); - for(int i = 0; i < 3; i++){ + for (int i = 0; i < 3; i++) { Event e = EventBuilder.withBody(Bytes.toBytes(valBase + "-" + i)); channel.put(e); } @@ -362,9 +366,9 @@ public void testHBaseFailure() throws Exception { byte[][] results = getResults(table, 2); byte[] out; int found = 0; - for(int i = 0; i < 2; i++){ - for(int j = 0; j < 2; j++){ - if(Arrays.equals(results[j],Bytes.toBytes(valBase + "-" + i))){ + for (int i = 0; i < 2; i++) { + for (int j = 0; j < 2; j++) { + if (Arrays.equals(results[j], Bytes.toBytes(valBase + "-" + i))) { found++; break; } @@ -378,22 +382,22 @@ public void testHBaseFailure() throws Exception { sink.stop(); } - /** * Makes Hbase scans to get rows in the payload column and increment column * in the table given. Expensive, so tread lightly. * Calling this function multiple times for the same result set is a bad * idea. Cache the result set once it is returned by this function. + * * @param table * @param numEvents Number of events inserted into the table * @return * @throws IOException */ - private byte[][] getResults(HTable table, int numEvents) throws IOException{ - byte[][] results = new byte[numEvents+1][]; + private byte[][] getResults(HTable table, int numEvents) throws IOException { + byte[][] results = new byte[numEvents + 1][]; Scan scan = new Scan(); - scan.addColumn(columnFamily.getBytes(),plCol.getBytes()); - scan.setStartRow( Bytes.toBytes("default")); + scan.addColumn(columnFamily.getBytes(), plCol.getBytes()); + scan.setStartRow(Bytes.toBytes("default")); ResultScanner rs = table.getScanner(scan); byte[] out = null; int i = 0; @@ -401,10 +405,10 @@ private byte[][] getResults(HTable table, int numEvents) throws IOException{ for (Result r = rs.next(); r != null; r = rs.next()) { out = r.getValue(columnFamily.getBytes(), plCol.getBytes()); - if(i >= results.length - 1){ + if (i >= results.length - 1) { rs.close(); throw new FlumeException("More results than expected in the table." + - "Expected = " + numEvents +". Found = " + i); + "Expected = " + numEvents + ". Found = " + i); } results[i++] = out; System.out.println(out); @@ -415,7 +419,7 @@ private byte[][] getResults(HTable table, int numEvents) throws IOException{ Assert.assertEquals(i, results.length - 1); scan = new Scan(); - scan.addColumn(columnFamily.getBytes(),inColumn.getBytes()); + scan.addColumn(columnFamily.getBytes(), inColumn.getBytes()); scan.setStartRow(Bytes.toBytes("incRow")); rs = table.getScanner(scan); out = null; @@ -472,7 +476,7 @@ public void testTransactionStateOnSerializationException() throws Exception { initContextForSimpleHbaseEventSerializer(); ctx.put("batchSize", "1"); ctx.put(HBaseSinkConfigurationConstants.CONFIG_SERIALIZER, - "org.apache.flume.sink.hbase.MockSimpleHbaseEventSerializer"); + "org.apache.flume.sink.hbase.MockSimpleHbaseEventSerializer"); HBaseSink sink = new HBaseSink(conf); Configurables.configure(sink, ctx); @@ -507,16 +511,16 @@ public void testTransactionStateOnSerializationException() throws Exception { } @Test - public void testWithoutConfigurationObject() throws Exception{ + public void testWithoutConfigurationObject() throws Exception { initContextForSimpleHbaseEventSerializer(); Context tmpContext = new Context(ctx.getParameters()); tmpContext.put("batchSize", "2"); tmpContext.put(HBaseSinkConfigurationConstants.ZK_QUORUM, - ZKConfig.getZKQuorumServersString(conf) ); + ZKConfig.getZKQuorumServersString(conf)); System.out.print(ctx.getString(HBaseSinkConfigurationConstants.ZK_QUORUM)); tmpContext.put(HBaseSinkConfigurationConstants.ZK_ZNODE_PARENT, - conf.get(HConstants.ZOOKEEPER_ZNODE_PARENT, - HConstants.DEFAULT_ZOOKEEPER_ZNODE_PARENT)); + conf.get(HConstants.ZOOKEEPER_ZNODE_PARENT, + HConstants.DEFAULT_ZOOKEEPER_ZNODE_PARENT)); HBaseSink sink = new HBaseSink(); Configurables.configure(sink, tmpContext); @@ -526,14 +530,14 @@ public void testWithoutConfigurationObject() throws Exception{ sink.start(); Transaction tx = channel.getTransaction(); tx.begin(); - for(int i = 0; i < 3; i++){ + for (int i = 0; i < 3; i++) { Event e = EventBuilder.withBody(Bytes.toBytes(valBase + "-" + i)); channel.put(e); } tx.commit(); tx.close(); Status status = Status.READY; - while(status != Status.BACKOFF){ + while (status != Status.BACKOFF) { status = sink.process(); } sink.stop(); @@ -541,9 +545,9 @@ public void testWithoutConfigurationObject() throws Exception{ byte[][] results = getResults(table, 3); byte[] out; int found = 0; - for(int i = 0; i < 3; i++){ - for(int j = 0; j < 3; j++){ - if(Arrays.equals(results[j],Bytes.toBytes(valBase + "-" + i))){ + for (int i = 0; i < 3; i++) { + for (int j = 0; j < 3; j++) { + if (Arrays.equals(results[j], Bytes.toBytes(valBase + "-" + i))) { found++; break; } @@ -555,37 +559,36 @@ public void testWithoutConfigurationObject() throws Exception{ } @Test - public void testZKQuorum() throws Exception{ + public void testZKQuorum() throws Exception { initContextForSimpleHbaseEventSerializer(); Context tmpContext = new Context(ctx.getParameters()); String zkQuorum = "zk1.flume.apache.org:3342, zk2.flume.apache.org:3342, " + - "zk3.flume.apache.org:3342"; + "zk3.flume.apache.org:3342"; tmpContext.put("batchSize", "2"); tmpContext.put(HBaseSinkConfigurationConstants.ZK_QUORUM, zkQuorum); tmpContext.put(HBaseSinkConfigurationConstants.ZK_ZNODE_PARENT, - conf.get(HConstants.ZOOKEEPER_ZNODE_PARENT, - HConstants.DEFAULT_ZOOKEEPER_ZNODE_PARENT)); + conf.get(HConstants.ZOOKEEPER_ZNODE_PARENT, + HConstants.DEFAULT_ZOOKEEPER_ZNODE_PARENT)); HBaseSink sink = new HBaseSink(); Configurables.configure(sink, tmpContext); Assert.assertEquals("zk1.flume.apache.org,zk2.flume.apache.org," + - "zk3.flume.apache.org", sink.getConfig().get(HConstants - .ZOOKEEPER_QUORUM)); - Assert.assertEquals(String.valueOf(3342), sink.getConfig().get(HConstants - .ZOOKEEPER_CLIENT_PORT)); + "zk3.flume.apache.org", sink.getConfig().get(HConstants.ZOOKEEPER_QUORUM)); + Assert.assertEquals(String.valueOf(3342), + sink.getConfig().get(HConstants.ZOOKEEPER_CLIENT_PORT)); } - @Test (expected = FlumeException.class) - public void testZKQuorumIncorrectPorts() throws Exception{ + @Test(expected = FlumeException.class) + public void testZKQuorumIncorrectPorts() throws Exception { initContextForSimpleHbaseEventSerializer(); Context tmpContext = new Context(ctx.getParameters()); String zkQuorum = "zk1.flume.apache.org:3345, zk2.flume.apache.org:3342, " + - "zk3.flume.apache.org:3342"; + "zk3.flume.apache.org:3342"; tmpContext.put("batchSize", "2"); tmpContext.put(HBaseSinkConfigurationConstants.ZK_QUORUM, zkQuorum); tmpContext.put(HBaseSinkConfigurationConstants.ZK_ZNODE_PARENT, - conf.get(HConstants.ZOOKEEPER_ZNODE_PARENT, - HConstants.DEFAULT_ZOOKEEPER_ZNODE_PARENT)); + conf.get(HConstants.ZOOKEEPER_ZNODE_PARENT, + HConstants.DEFAULT_ZOOKEEPER_ZNODE_PARENT)); HBaseSink sink = new HBaseSink(); Configurables.configure(sink, tmpContext); Assert.fail(); diff --git a/flume-ng-sinks/flume-ng-hbase-sink/src/test/java/org/apache/flume/sink/hbase/TestRegexHbaseEventSerializer.java b/flume-ng-sinks/flume-ng-hbase-sink/src/test/java/org/apache/flume/sink/hbase/TestRegexHbaseEventSerializer.java index b102b49543..24bcf37cba 100644 --- a/flume-ng-sinks/flume-ng-hbase-sink/src/test/java/org/apache/flume/sink/hbase/TestRegexHbaseEventSerializer.java +++ b/flume-ng-sinks/flume-ng-hbase-sink/src/test/java/org/apache/flume/sink/hbase/TestRegexHbaseEventSerializer.java @@ -18,17 +18,7 @@ */ package org.apache.flume.sink.hbase; -import static org.junit.Assert.assertArrayEquals; -import static org.junit.Assert.assertEquals; -import static org.junit.Assert.assertTrue; -import static org.mockito.Mockito.mock; -import static org.mockito.Mockito.when; - -import java.nio.charset.Charset; -import java.util.Calendar; -import java.util.List; -import java.util.Map; - +import com.google.common.collect.Maps; import org.apache.flume.Context; import org.apache.flume.Event; import org.apache.flume.event.EventBuilder; @@ -39,7 +29,16 @@ import org.apache.hadoop.hbase.util.Bytes; import org.junit.Test; -import com.google.common.collect.Maps; +import java.nio.charset.Charset; +import java.util.Calendar; +import java.util.List; +import java.util.Map; + +import static org.junit.Assert.assertArrayEquals; +import static org.junit.Assert.assertEquals; +import static org.junit.Assert.assertTrue; +import static org.mockito.Mockito.mock; +import static org.mockito.Mockito.when; public class TestRegexHbaseEventSerializer { @@ -76,8 +75,7 @@ public void testDefaultBehavior() throws Exception { public void testRowIndexKey() throws Exception { RegexHbaseEventSerializer s = new RegexHbaseEventSerializer(); Context context = new Context(); - context.put(RegexHbaseEventSerializer.REGEX_CONFIG,"^([^\t]+)\t([^\t]+)\t" + - "([^\t]+)$"); + context.put(RegexHbaseEventSerializer.REGEX_CONFIG,"^([^\t]+)\t([^\t]+)\t" + "([^\t]+)$"); context.put(RegexHbaseEventSerializer.COL_NAME_CONFIG, "col1,col2,ROW_KEY"); context.put("rowKeyIndex", "2"); s.configure(context); @@ -115,9 +113,9 @@ public void testApacheRegex() throws Exception { "referer,agent"); s.configure(context); String logMsg = "33.22.11.00 - - [20/May/2011:07:01:19 +0000] " + - "\"GET /wp-admin/css/install.css HTTP/1.0\" 200 813 " + - "\"http://www.cloudera.com/wp-admin/install.php\" \"Mozilla/5.0 (comp" + - "atible; Yahoo! Slurp; http://help.yahoo.com/help/us/ysearch/slurp)\""; + "\"GET /wp-admin/css/install.css HTTP/1.0\" 200 813 " + + "\"http://www.cloudera.com/wp-admin/install.php\" \"Mozilla/5.0 (comp" + + "atible; Yahoo! Slurp; http://help.yahoo.com/help/us/ysearch/slurp)\""; Event e = EventBuilder.withBody(Bytes.toBytes(logMsg)); s.initialize(e, "CF".getBytes()); @@ -189,7 +187,7 @@ public void testRowKeyGeneration() { } - @Test + @Test /** Test depositing of the header information. */ public void testDepositHeaders() throws Exception { Charset charset = Charset.forName("KOI8-R"); @@ -222,7 +220,8 @@ public void testDepositHeaders() throws Exception { resultMap.put(new String(kv.getQualifier(), charset), kv.getValue()); } - assertEquals(body, new String(resultMap.get(RegexHbaseEventSerializer.COLUMN_NAME_DEFAULT), charset)); + assertEquals(body, + new String(resultMap.get(RegexHbaseEventSerializer.COLUMN_NAME_DEFAULT), charset)); assertEquals("value1", new String(resultMap.get("header1"), charset)); assertArrayEquals("значение2".getBytes(charset), resultMap.get("заголовок2")); assertEquals("значение2".length(), resultMap.get("заголовок2").length); diff --git a/flume-ng-sinks/flume-ng-kafka-sink/src/test/java/org/apache/flume/sink/kafka/TestKafkaSink.java b/flume-ng-sinks/flume-ng-kafka-sink/src/test/java/org/apache/flume/sink/kafka/TestKafkaSink.java index f577e98d9f..76eca37160 100644 --- a/flume-ng-sinks/flume-ng-kafka-sink/src/test/java/org/apache/flume/sink/kafka/TestKafkaSink.java +++ b/flume-ng-sinks/flume-ng-kafka-sink/src/test/java/org/apache/flume/sink/kafka/TestKafkaSink.java @@ -18,8 +18,8 @@ Licensed to the Apache Software Foundation (ASF) under one or more package org.apache.flume.sink.kafka; +import com.google.common.base.Charsets; import kafka.message.MessageAndMetadata; - import org.apache.avro.io.BinaryDecoder; import org.apache.avro.io.DecoderFactory; import org.apache.avro.specific.SpecificDatumReader; @@ -41,8 +41,6 @@ Licensed to the Apache Software Foundation (ASF) under one or more import org.junit.BeforeClass; import org.junit.Test; -import com.google.common.base.Charsets; - import java.io.ByteArrayInputStream; import java.io.IOException; import java.io.UnsupportedEncodingException; @@ -52,12 +50,21 @@ Licensed to the Apache Software Foundation (ASF) under one or more import java.util.Map; import java.util.Properties; +import static org.apache.flume.sink.kafka.KafkaSinkConstants.AVRO_EVENT; +import static org.apache.flume.sink.kafka.KafkaSinkConstants.BATCH_SIZE; +import static org.apache.flume.sink.kafka.KafkaSinkConstants.BOOTSTRAP_SERVERS_CONFIG; +import static org.apache.flume.sink.kafka.KafkaSinkConstants.BROKER_LIST_FLUME_KEY; +import static org.apache.flume.sink.kafka.KafkaSinkConstants.DEFAULT_KEY_SERIALIZER; +import static org.apache.flume.sink.kafka.KafkaSinkConstants.DEFAULT_TOPIC; +import static org.apache.flume.sink.kafka.KafkaSinkConstants.KAFKA_PREFIX; +import static org.apache.flume.sink.kafka.KafkaSinkConstants.KAFKA_PRODUCER_PREFIX; +import static org.apache.flume.sink.kafka.KafkaSinkConstants.OLD_BATCH_SIZE; +import static org.apache.flume.sink.kafka.KafkaSinkConstants.REQUIRED_ACKS_FLUME_KEY; +import static org.apache.flume.sink.kafka.KafkaSinkConstants.TOPIC_CONFIG; import static org.junit.Assert.assertEquals; import static org.junit.Assert.assertNull; import static org.junit.Assert.fail; -import static org.apache.flume.sink.kafka.KafkaSinkConstants.*; - /** * Unit tests for Kafka Sink */ @@ -86,40 +93,45 @@ public void testKafkaProperties() { KafkaSink kafkaSink = new KafkaSink(); Context context = new Context(); context.put(KAFKA_PREFIX + TOPIC_CONFIG, ""); - context.put(KAFKA_PRODUCER_PREFIX + ProducerConfig.VALUE_SERIALIZER_CLASS_CONFIG, "override.default.serializer"); + context.put(KAFKA_PRODUCER_PREFIX + ProducerConfig.VALUE_SERIALIZER_CLASS_CONFIG, + "override.default.serializer"); context.put("kafka.producer.fake.property", "kafka.property.value"); context.put("kafka.bootstrap.servers", "localhost:9092,localhost:9092"); - context.put("brokerList","real-broker-list"); - Configurables.configure(kafkaSink,context); + context.put("brokerList", "real-broker-list"); + Configurables.configure(kafkaSink, context); Properties kafkaProps = kafkaSink.getKafkaProps(); //check that we have defaults set - assertEquals( - kafkaProps.getProperty(ProducerConfig.KEY_SERIALIZER_CLASS_CONFIG), DEFAULT_KEY_SERIALIZER); + assertEquals(kafkaProps.getProperty(ProducerConfig.KEY_SERIALIZER_CLASS_CONFIG), + DEFAULT_KEY_SERIALIZER); //check that kafka properties override the default and get correct name - assertEquals(kafkaProps.getProperty(ProducerConfig.VALUE_SERIALIZER_CLASS_CONFIG), "override.default.serializer"); + assertEquals(kafkaProps.getProperty(ProducerConfig.VALUE_SERIALIZER_CLASS_CONFIG), + "override.default.serializer"); //check that any kafka-producer property gets in - assertEquals(kafkaProps.getProperty("fake.property"), "kafka.property.value"); + assertEquals(kafkaProps.getProperty("fake.property"), + "kafka.property.value"); //check that documented property overrides defaults - assertEquals(kafkaProps.getProperty("bootstrap.servers") ,"localhost:9092,localhost:9092"); + assertEquals(kafkaProps.getProperty("bootstrap.servers"), + "localhost:9092,localhost:9092"); } @Test public void testOldProperties() { KafkaSink kafkaSink = new KafkaSink(); Context context = new Context(); - context.put("topic","test-topic"); + context.put("topic", "test-topic"); context.put(OLD_BATCH_SIZE, "300"); - context.put(BROKER_LIST_FLUME_KEY,"localhost:9092,localhost:9092"); + context.put(BROKER_LIST_FLUME_KEY, "localhost:9092,localhost:9092"); context.put(REQUIRED_ACKS_FLUME_KEY, "all"); - Configurables.configure(kafkaSink,context); + Configurables.configure(kafkaSink, context); Properties kafkaProps = kafkaSink.getKafkaProps(); assertEquals(kafkaSink.getTopic(), "test-topic"); - assertEquals(kafkaSink.getBatchSize(),300); - assertEquals(kafkaProps.getProperty(CommonClientConfigs.BOOTSTRAP_SERVERS_CONFIG),"localhost:9092,localhost:9092"); + assertEquals(kafkaSink.getBatchSize(), 300); + assertEquals(kafkaProps.getProperty(CommonClientConfigs.BOOTSTRAP_SERVERS_CONFIG), + "localhost:9092,localhost:9092"); assertEquals(kafkaProps.getProperty(ProducerConfig.ACKS_CONFIG), "all"); } @@ -151,9 +163,8 @@ public void testDefaultTopic() { // ignore } - String fetchedMsg = new String((byte[]) - testUtil.getNextMessageFromConsumer(DEFAULT_TOPIC) - .message()); + String fetchedMsg = new String((byte[]) testUtil.getNextMessageFromConsumer(DEFAULT_TOPIC) + .message()); assertEquals(msg, fetchedMsg); } @@ -173,14 +184,13 @@ public void testStaticTopic() { // ignore } - String fetchedMsg = new String((byte[]) testUtil.getNextMessageFromConsumer(TestConstants.STATIC_TOPIC).message()); + String fetchedMsg = new String((byte[]) testUtil.getNextMessageFromConsumer( + TestConstants.STATIC_TOPIC).message()); assertEquals(msg, fetchedMsg); } @Test public void testTopicAndKeyFromHeader() throws UnsupportedEncodingException { - - Sink kafkaSink = new KafkaSink(); Context context = prepareDefaultContext(); Configurables.configure(kafkaSink, context); @@ -210,19 +220,16 @@ public void testTopicAndKeyFromHeader() throws UnsupportedEncodingException { } MessageAndMetadata fetchedMsg = - testUtil.getNextMessageFromConsumer(TestConstants.CUSTOM_TOPIC); + testUtil.getNextMessageFromConsumer(TestConstants.CUSTOM_TOPIC); assertEquals(msg, new String((byte[]) fetchedMsg.message(), "UTF-8")); assertEquals(TestConstants.CUSTOM_KEY, - new String((byte[]) fetchedMsg.key(), "UTF-8")); - + new String((byte[]) fetchedMsg.key(), "UTF-8")); } @SuppressWarnings("rawtypes") @Test public void testAvroEvent() throws IOException { - - Sink kafkaSink = new KafkaSink(); Context context = prepareDefaultContext(); context.put(AVRO_EVENT, "true"); @@ -254,13 +261,12 @@ public void testAvroEvent() throws IOException { // ignore } - MessageAndMetadata fetchedMsg = - testUtil.getNextMessageFromConsumer(TestConstants.CUSTOM_TOPIC); + MessageAndMetadata fetchedMsg = testUtil.getNextMessageFromConsumer(TestConstants.CUSTOM_TOPIC); - ByteArrayInputStream in = - new ByteArrayInputStream((byte[])fetchedMsg.message()); + ByteArrayInputStream in = new ByteArrayInputStream((byte[]) fetchedMsg.message()); BinaryDecoder decoder = DecoderFactory.get().directBinaryDecoder(in, null); - SpecificDatumReader reader = new SpecificDatumReader(AvroFlumeEvent.class); + SpecificDatumReader reader = + new SpecificDatumReader(AvroFlumeEvent.class); AvroFlumeEvent avroevent = reader.read(null, decoder); @@ -268,17 +274,15 @@ public void testAvroEvent() throws IOException { Map eventHeaders = avroevent.getHeaders(); assertEquals(msg, eventBody); - assertEquals(TestConstants.CUSTOM_KEY, - new String((byte[]) fetchedMsg.key(), "UTF-8")); + assertEquals(TestConstants.CUSTOM_KEY, new String((byte[]) fetchedMsg.key(), "UTF-8")); - assertEquals(TestConstants.HEADER_1_VALUE, eventHeaders.get(new Utf8(TestConstants.HEADER_1_KEY)).toString()); + assertEquals(TestConstants.HEADER_1_VALUE, + eventHeaders.get(new Utf8(TestConstants.HEADER_1_KEY)).toString()); assertEquals(TestConstants.CUSTOM_KEY, eventHeaders.get(new Utf8("key")).toString()); - } @Test - public void testEmptyChannel() throws UnsupportedEncodingException, - EventDeliveryException { + public void testEmptyChannel() throws UnsupportedEncodingException, EventDeliveryException { Sink kafkaSink = new KafkaSink(); Context context = prepareDefaultContext(); Configurables.configure(kafkaSink, context); @@ -291,8 +295,7 @@ public void testEmptyChannel() throws UnsupportedEncodingException, if (status != Sink.Status.BACKOFF) { fail("Error Occurred"); } - assertNull( - testUtil.getNextMessageFromConsumer(DEFAULT_TOPIC)); + assertNull(testUtil.getNextMessageFromConsumer(DEFAULT_TOPIC)); } private Context prepareDefaultContext() { @@ -304,7 +307,7 @@ private Context prepareDefaultContext() { } private Sink.Status prepareAndSend(Context context, String msg) - throws EventDeliveryException { + throws EventDeliveryException { Sink kafkaSink = new KafkaSink(); Configurables.configure(kafkaSink, context); Channel memoryChannel = new MemoryChannel(); diff --git a/flume-ng-sinks/flume-ng-kafka-sink/src/test/java/org/apache/flume/sink/kafka/util/KafkaLocal.java b/flume-ng-sinks/flume-ng-kafka-sink/src/test/java/org/apache/flume/sink/kafka/util/KafkaLocal.java index d8a45ef73c..6d89bd33d4 100644 --- a/flume-ng-sinks/flume-ng-kafka-sink/src/test/java/org/apache/flume/sink/kafka/util/KafkaLocal.java +++ b/flume-ng-sinks/flume-ng-kafka-sink/src/test/java/org/apache/flume/sink/kafka/util/KafkaLocal.java @@ -1,19 +1,19 @@ /** - Licensed to the Apache Software Foundation (ASF) under one or more - contributor license agreements. See the NOTICE file distributed with - this work for additional information regarding copyright ownership. - The ASF licenses this file to You under the Apache License, Version 2.0 - (the "License"); you may not use this file except in compliance with - the License. You may obtain a copy of the License at - - http://www.apache.org/licenses/LICENSE-2.0 - - Unless required by applicable law or agreed to in writing, software - distributed under the License is distributed on an "AS IS" BASIS, - WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - See the License for the specific language governing permissions and - limitations under the License. - limitations under the License. + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + *

    + * http://www.apache.org/licenses/LICENSE-2.0 + *

    + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + * limitations under the License. */ package org.apache.flume.sink.kafka.util; @@ -30,23 +30,22 @@ Licensed to the Apache Software Foundation (ASF) under one or more */ public class KafkaLocal { - public KafkaServerStartable kafka; - public ZooKeeperLocal zookeeper; + public KafkaServerStartable kafka; + public ZooKeeperLocal zookeeper; - public KafkaLocal(Properties kafkaProperties) throws IOException, - InterruptedException{ - KafkaConfig kafkaConfig = KafkaConfig.fromProps(kafkaProperties); + public KafkaLocal(Properties kafkaProperties) throws IOException, InterruptedException { + KafkaConfig kafkaConfig = KafkaConfig.fromProps(kafkaProperties); - //start local kafka broker - kafka = new KafkaServerStartable(kafkaConfig); - } + // start local kafka broker + kafka = new KafkaServerStartable(kafkaConfig); + } - public void start() throws Exception{ - kafka.startup(); - } + public void start() throws Exception { + kafka.startup(); + } - public void stop(){ - kafka.shutdown(); - } + public void stop() { + kafka.shutdown(); + } } \ No newline at end of file diff --git a/flume-ng-sinks/flume-ng-kafka-sink/src/test/java/org/apache/flume/sink/kafka/util/ZooKeeperLocal.java b/flume-ng-sinks/flume-ng-kafka-sink/src/test/java/org/apache/flume/sink/kafka/util/ZooKeeperLocal.java index 1a5728fcf0..35c1e47d54 100644 --- a/flume-ng-sinks/flume-ng-kafka-sink/src/test/java/org/apache/flume/sink/kafka/util/ZooKeeperLocal.java +++ b/flume-ng-sinks/flume-ng-kafka-sink/src/test/java/org/apache/flume/sink/kafka/util/ZooKeeperLocal.java @@ -1,19 +1,19 @@ /** - Licensed to the Apache Software Foundation (ASF) under one or more - contributor license agreements. See the NOTICE file distributed with - this work for additional information regarding copyright ownership. - The ASF licenses this file to You under the Apache License, Version 2.0 - (the "License"); you may not use this file except in compliance with - the License. You may obtain a copy of the License at - - http://www.apache.org/licenses/LICENSE-2.0 - - Unless required by applicable law or agreed to in writing, software - distributed under the License is distributed on an "AS IS" BASIS, - WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - See the License for the specific language governing permissions and - limitations under the License. - limitations under the License. + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + *

    + * http://www.apache.org/licenses/LICENSE-2.0 + *

    + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + * limitations under the License. */ package org.apache.flume.sink.kafka.util; @@ -33,30 +33,29 @@ Licensed to the Apache Software Foundation (ASF) under one or more */ public class ZooKeeperLocal { - private static final Logger logger = - LoggerFactory.getLogger(ZooKeeperLocal.class); - private ZooKeeperServerMain zooKeeperServer; + private static final Logger logger = LoggerFactory.getLogger(ZooKeeperLocal.class); + private ZooKeeperServerMain zooKeeperServer; - public ZooKeeperLocal(Properties zkProperties) throws IOException{ - QuorumPeerConfig quorumConfiguration = new QuorumPeerConfig(); + public ZooKeeperLocal(Properties zkProperties) throws IOException { + QuorumPeerConfig quorumConfiguration = new QuorumPeerConfig(); + try { + quorumConfiguration.parseProperties(zkProperties); + } catch (Exception e) { + throw new RuntimeException(e); + } + + zooKeeperServer = new ZooKeeperServerMain(); + final ServerConfig configuration = new ServerConfig(); + configuration.readFrom(quorumConfiguration); + + new Thread() { + public void run() { try { - quorumConfiguration.parseProperties(zkProperties); - } catch(Exception e) { - throw new RuntimeException(e); + zooKeeperServer.runFromConfig(configuration); + } catch (IOException e) { + logger.error("Zookeeper startup failed.", e); } - - zooKeeperServer = new ZooKeeperServerMain(); - final ServerConfig configuration = new ServerConfig(); - configuration.readFrom(quorumConfiguration); - - new Thread() { - public void run() { - try { - zooKeeperServer.runFromConfig(configuration); - } catch (IOException e) { - logger.error("Zookeeper startup failed.", e); - } - } - }.start(); - } + } + }.start(); + } } diff --git a/flume-ng-sinks/flume-ng-morphline-solr-sink/src/test/java/org/apache/flume/sink/solr/morphline/TestBlobDeserializer.java b/flume-ng-sinks/flume-ng-morphline-solr-sink/src/test/java/org/apache/flume/sink/solr/morphline/TestBlobDeserializer.java index 6172c6810d..be377baada 100644 --- a/flume-ng-sinks/flume-ng-morphline-solr-sink/src/test/java/org/apache/flume/sink/solr/morphline/TestBlobDeserializer.java +++ b/flume-ng-sinks/flume-ng-morphline-solr-sink/src/test/java/org/apache/flume/sink/solr/morphline/TestBlobDeserializer.java @@ -16,9 +16,7 @@ */ package org.apache.flume.sink.solr.morphline; -import java.io.IOException; -import java.util.List; - +import com.google.common.base.Charsets; import org.apache.flume.Context; import org.apache.flume.Event; import org.apache.flume.serialization.EventDeserializer; @@ -28,7 +26,8 @@ import org.junit.Before; import org.junit.Test; -import com.google.common.base.Charsets; +import java.io.IOException; +import java.util.List; public class TestBlobDeserializer extends Assert { @@ -61,7 +60,8 @@ public void testSimpleViaBuilder() throws IOException { public void testSimpleViaFactory() throws IOException { ResettableInputStream in = new ResettableTestStringInputStream(mini); EventDeserializer des; - des = EventDeserializerFactory.getInstance(BlobDeserializer.Builder.class.getName(), new Context(), in); + des = EventDeserializerFactory.getInstance(BlobDeserializer.Builder.class.getName(), + new Context(), in); validateMiniParse(des); } diff --git a/flume-ng-sinks/flume-ng-morphline-solr-sink/src/test/java/org/apache/flume/sink/solr/morphline/TestMorphlineInterceptor.java b/flume-ng-sinks/flume-ng-morphline-solr-sink/src/test/java/org/apache/flume/sink/solr/morphline/TestMorphlineInterceptor.java index 22cfe96a2d..8d62d38f64 100644 --- a/flume-ng-sinks/flume-ng-morphline-solr-sink/src/test/java/org/apache/flume/sink/solr/morphline/TestMorphlineInterceptor.java +++ b/flume-ng-sinks/flume-ng-morphline-solr-sink/src/test/java/org/apache/flume/sink/solr/morphline/TestMorphlineInterceptor.java @@ -16,22 +16,21 @@ */ package org.apache.flume.sink.solr.morphline; -import java.io.File; -import java.util.Collections; -import java.util.HashMap; -import java.util.List; -import java.util.Map; - +import com.google.common.base.Charsets; +import com.google.common.collect.ImmutableMap; +import com.google.common.io.Files; import org.apache.flume.Context; import org.apache.flume.Event; import org.apache.flume.event.EventBuilder; import org.junit.Assert; import org.junit.Test; - import org.kitesdk.morphline.base.Fields; -import com.google.common.base.Charsets; -import com.google.common.collect.ImmutableMap; -import com.google.common.io.Files; + +import java.io.File; +import java.util.Collections; +import java.util.HashMap; +import java.util.List; +import java.util.Map; public class TestMorphlineInterceptor extends Assert { @@ -40,13 +39,15 @@ public class TestMorphlineInterceptor extends Assert { @Test public void testNoOperation() throws Exception { Context context = new Context(); - context.put(MorphlineHandlerImpl.MORPHLINE_FILE_PARAM, RESOURCES_DIR + "/test-morphlines/noOperation.conf"); + context.put(MorphlineHandlerImpl.MORPHLINE_FILE_PARAM, + RESOURCES_DIR + "/test-morphlines/noOperation.conf"); Event input = EventBuilder.withBody("foo", Charsets.UTF_8); input.getHeaders().put("name", "nadja"); MorphlineInterceptor interceptor = build(context); Event actual = interceptor.intercept(input); interceptor.close(); - Event expected = EventBuilder.withBody("foo".getBytes("UTF-8"), ImmutableMap.of("name", "nadja")); + Event expected = EventBuilder.withBody("foo".getBytes("UTF-8"), + ImmutableMap.of("name", "nadja")); assertEqualsEvent(expected, actual); List actualList = build(context).intercept(Collections.singletonList(input)); @@ -57,11 +58,13 @@ public void testNoOperation() throws Exception { @Test public void testReadClob() throws Exception { Context context = new Context(); - context.put(MorphlineHandlerImpl.MORPHLINE_FILE_PARAM, RESOURCES_DIR + "/test-morphlines/readClob.conf"); + context.put(MorphlineHandlerImpl.MORPHLINE_FILE_PARAM, + RESOURCES_DIR + "/test-morphlines/readClob.conf"); Event input = EventBuilder.withBody("foo", Charsets.UTF_8); input.getHeaders().put("name", "nadja"); Event actual = build(context).intercept(input); - Event expected = EventBuilder.withBody(null, ImmutableMap.of("name", "nadja", Fields.MESSAGE, "foo")); + Event expected = EventBuilder.withBody(null, + ImmutableMap.of("name", "nadja", Fields.MESSAGE, "foo")); assertEqualsEvent(expected, actual); List actualList = build(context).intercept(Collections.singletonList(input)); @@ -72,7 +75,8 @@ public void testReadClob() throws Exception { @Test public void testGrokIfNotMatchDropEventRetain() throws Exception { Context context = new Context(); - context.put(MorphlineHandlerImpl.MORPHLINE_FILE_PARAM, RESOURCES_DIR + "/test-morphlines/grokIfNotMatchDropRecord.conf"); + context.put(MorphlineHandlerImpl.MORPHLINE_FILE_PARAM, + RESOURCES_DIR + "/test-morphlines/grokIfNotMatchDropRecord.conf"); String msg = "<164>Feb 4 10:46:14 syslog sshd[607]: Server listening on 0.0.0.0 port 22."; Event input = EventBuilder.withBody(null, ImmutableMap.of(Fields.MESSAGE, msg)); @@ -94,8 +98,10 @@ public void testGrokIfNotMatchDropEventRetain() throws Exception { /* leading XXXXX does not match regex, thus we expect the event to be dropped */ public void testGrokIfNotMatchDropEventDrop() throws Exception { Context context = new Context(); - context.put(MorphlineHandlerImpl.MORPHLINE_FILE_PARAM, RESOURCES_DIR + "/test-morphlines/grokIfNotMatchDropRecord.conf"); - String msg = "Feb 4 10:46:14 syslog sshd[607]: Server listening on 0.0.0.0 port 22."; + context.put(MorphlineHandlerImpl.MORPHLINE_FILE_PARAM, + RESOURCES_DIR + "/test-morphlines/grokIfNotMatchDropRecord.conf"); + String msg = "Feb 4 10:46:14 syslog sshd[607]: Server listening on 0.0.0.0" + + " port 22."; Event input = EventBuilder.withBody(null, ImmutableMap.of(Fields.MESSAGE, msg)); Event actual = build(context).intercept(input); assertNull(actual); @@ -105,10 +111,12 @@ public void testGrokIfNotMatchDropEventDrop() throws Exception { /** morphline says route to southpole if it's an avro file, otherwise route to northpole */ public void testIfDetectMimeTypeRouteToSouthPole() throws Exception { Context context = new Context(); - context.put(MorphlineHandlerImpl.MORPHLINE_FILE_PARAM, RESOURCES_DIR + "/test-morphlines/ifDetectMimeType.conf"); + context.put(MorphlineHandlerImpl.MORPHLINE_FILE_PARAM, + RESOURCES_DIR + "/test-morphlines/ifDetectMimeType.conf"); context.put(MorphlineHandlerImpl.MORPHLINE_VARIABLE_PARAM + ".MY.MIME_TYPE", "avro/binary"); - Event input = EventBuilder.withBody(Files.toByteArray(new File(RESOURCES_DIR + "/test-documents/sample-statuses-20120906-141433.avro"))); + Event input = EventBuilder.withBody(Files.toByteArray( + new File(RESOURCES_DIR + "/test-documents/sample-statuses-20120906-141433.avro"))); Event actual = build(context).intercept(input); Map expected = new HashMap(); @@ -122,10 +130,12 @@ public void testIfDetectMimeTypeRouteToSouthPole() throws Exception { /** morphline says route to southpole if it's an avro file, otherwise route to northpole */ public void testIfDetectMimeTypeRouteToNorthPole() throws Exception { Context context = new Context(); - context.put(MorphlineHandlerImpl.MORPHLINE_FILE_PARAM, RESOURCES_DIR + "/test-morphlines/ifDetectMimeType.conf"); + context.put(MorphlineHandlerImpl.MORPHLINE_FILE_PARAM, + RESOURCES_DIR + "/test-morphlines/ifDetectMimeType.conf"); context.put(MorphlineHandlerImpl.MORPHLINE_VARIABLE_PARAM + ".MY.MIME_TYPE", "avro/binary"); - Event input = EventBuilder.withBody(Files.toByteArray(new File(RESOURCES_DIR + "/test-documents/testPDF.pdf"))); + Event input = EventBuilder.withBody( + Files.toByteArray(new File(RESOURCES_DIR + "/test-documents/testPDF.pdf"))); Event actual = build(context).intercept(input); Map expected = new HashMap(); @@ -140,8 +150,9 @@ private MorphlineInterceptor build(Context context) { builder.configure(context); return builder.build(); } - - private void assertEqualsEvent(Event x, Event y) { // b/c SimpleEvent doesn't implement equals() method :-( + + // b/c SimpleEvent doesn't implement equals() method :-( + private void assertEqualsEvent(Event x, Event y) { assertEquals(x.getHeaders(), y.getHeaders()); assertArrayEquals(x.getBody(), y.getBody()); } diff --git a/flume-ng-sinks/flume-ng-morphline-solr-sink/src/test/java/org/apache/flume/sink/solr/morphline/TestMorphlineSolrSink.java b/flume-ng-sinks/flume-ng-morphline-solr-sink/src/test/java/org/apache/flume/sink/solr/morphline/TestMorphlineSolrSink.java index 232c092883..1bfae952ee 100644 --- a/flume-ng-sinks/flume-ng-morphline-solr-sink/src/test/java/org/apache/flume/sink/solr/morphline/TestMorphlineSolrSink.java +++ b/flume-ng-sinks/flume-ng-morphline-solr-sink/src/test/java/org/apache/flume/sink/solr/morphline/TestMorphlineSolrSink.java @@ -87,8 +87,7 @@ public static void beforeClass() throws Exception { initCore( RESOURCES_DIR + "/solr/collection1/conf/solrconfig.xml", RESOURCES_DIR + "/solr/collection1/conf/schema.xml", - RESOURCES_DIR + "/solr" - ); + RESOURCES_DIR + "/solr"); } @Before @@ -139,9 +138,9 @@ public MySolrSink(MorphlineHandlerImpl indexer) { int batchSize = SEQ_NUM2.incrementAndGet() % 2 == 0 ? 100 : 1; DocumentLoader testServer = new SolrServerDocumentLoader(solrServer, batchSize); MorphlineContext solrMorphlineContext = new SolrMorphlineContext.Builder() - .setDocumentLoader(testServer) - .setExceptionHandler(new FaultTolerance(false, false, SolrServerException.class.getName())) - .setMetricRegistry(new MetricRegistry()).build(); + .setDocumentLoader(testServer) + .setExceptionHandler(new FaultTolerance(false, false, SolrServerException.class.getName())) + .setMetricRegistry(new MetricRegistry()).build(); MorphlineHandlerImpl impl = new MorphlineHandlerImpl(); impl.setMorphlineContext(solrMorphlineContext); @@ -302,9 +301,11 @@ public void testAvroRoundTrip() throws Exception { QueryResponse rsp = query("*:*"); Iterator iter = rsp.getResults().iterator(); ListMultimap expectedFieldValues; - expectedFieldValues = ImmutableListMultimap.of("id", "1234567890", "text", "sample tweet one", "user_screen_name", "fake_user1"); + expectedFieldValues = ImmutableListMultimap.of("id", "1234567890", "text", "sample tweet one", + "user_screen_name", "fake_user1"); assertEquals(expectedFieldValues, next(iter)); - expectedFieldValues = ImmutableListMultimap.of("id", "2345678901", "text", "sample tweet two", "user_screen_name", "fake_user2"); + expectedFieldValues = ImmutableListMultimap.of("id", "2345678901", "text", "sample tweet two", + "user_screen_name", "fake_user2"); assertEquals(expectedFieldValues, next(iter)); assertFalse(iter.hasNext()); } @@ -398,8 +399,8 @@ public void benchmarkDocumentTypes() throws Exception { float secs = (System.currentTimeMillis() - startTime) / 1000.0f; long numDocs = queryResultSetSize("*:*"); - LOGGER.info("Took secs: " + secs + ", iters/sec: " + (iters/secs)); - LOGGER.info("Took secs: " + secs + ", docs/sec: " + (numDocs/secs)); + LOGGER.info("Took secs: " + secs + ", iters/sec: " + (iters / secs)); + LOGGER.info("Took secs: " + secs + ", docs/sec: " + (numDocs / secs)); LOGGER.info("Iterations: " + iters + ", numDocs: " + numDocs); LOGGER.info("sink: ", sink); } diff --git a/flume-ng-sources/flume-jms-source/src/test/java/org/apache/flume/source/jms/JMSMessageConsumerTestBase.java b/flume-ng-sources/flume-jms-source/src/test/java/org/apache/flume/source/jms/JMSMessageConsumerTestBase.java index 68819677b3..b8466f7949 100644 --- a/flume-ng-sources/flume-jms-source/src/test/java/org/apache/flume/source/jms/JMSMessageConsumerTestBase.java +++ b/flume-ng-sources/flume-jms-source/src/test/java/org/apache/flume/source/jms/JMSMessageConsumerTestBase.java @@ -92,13 +92,10 @@ public Object nextElement() { } }); when(message.getText()).thenReturn(TEXT); - when(connectionFactory.createConnection(USERNAME, PASSWORD)). - thenReturn(connection); - when(connection.createSession(true, Session.SESSION_TRANSACTED)). - thenReturn(session); + when(connectionFactory.createConnection(USERNAME, PASSWORD)).thenReturn(connection); + when(connection.createSession(true, Session.SESSION_TRANSACTED)).thenReturn(session); when(session.createQueue(destinationName)).thenReturn(queue); - when(session.createConsumer(any(Destination.class), anyString())) - .thenReturn(messageConsumer); + when(session.createConsumer(any(Destination.class), anyString())).thenReturn(messageConsumer); when(messageConsumer.receiveNoWait()).thenReturn(message); when(messageConsumer.receive(anyLong())).thenReturn(message); destinationName = DESTINATION_NAME; @@ -127,7 +124,7 @@ void afterTearDown() throws Exception { } void assertBodyIsExpected(List events) { - for(Event event : events) { + for (Event event : events) { assertEquals(TEXT, new String(event.getBody(), Charsets.UTF_8)); } } @@ -140,7 +137,7 @@ JMSMessageConsumer create() { @After public void tearDown() throws Exception { beforeTearDown(); - if(consumer != null) { + if (consumer != null) { consumer.close(); } afterTearDown(); diff --git a/flume-ng-sources/flume-jms-source/src/test/java/org/apache/flume/source/jms/TestDefaultJMSMessageConverter.java b/flume-ng-sources/flume-jms-source/src/test/java/org/apache/flume/source/jms/TestDefaultJMSMessageConverter.java index 8d413f7b8c..0b2193c938 100644 --- a/flume-ng-sources/flume-jms-source/src/test/java/org/apache/flume/source/jms/TestDefaultJMSMessageConverter.java +++ b/flume-ng-sources/flume-jms-source/src/test/java/org/apache/flume/source/jms/TestDefaultJMSMessageConverter.java @@ -73,7 +73,7 @@ void createBytesMessage() throws Exception { @Override public Integer answer(InvocationOnMock invocation) throws Throwable { byte[] buffer = (byte[])invocation.getArguments()[0]; - if(buffer != null) { + if (buffer != null) { assertEquals(buffer.length, BYTES.length); System.arraycopy(BYTES, 0, buffer, 0, BYTES.length); } diff --git a/flume-ng-sources/flume-jms-source/src/test/java/org/apache/flume/source/jms/TestIntegrationActiveMQ.java b/flume-ng-sources/flume-jms-source/src/test/java/org/apache/flume/source/jms/TestIntegrationActiveMQ.java index e28e02ab4f..53cc09a59f 100644 --- a/flume-ng-sources/flume-jms-source/src/test/java/org/apache/flume/source/jms/TestIntegrationActiveMQ.java +++ b/flume-ng-sources/flume-jms-source/src/test/java/org/apache/flume/source/jms/TestIntegrationActiveMQ.java @@ -56,9 +56,10 @@ public class TestIntegrationActiveMQ { - private final static String INITIAL_CONTEXT_FACTORY = "org.apache.activemq.jndi.ActiveMQInitialContextFactory"; + private static final String INITIAL_CONTEXT_FACTORY = + "org.apache.activemq.jndi.ActiveMQInitialContextFactory"; public static final String BROKER_BIND_URL = "tcp://localhost:61516"; - private final static String DESTINATION_NAME = "test"; + private static final String DESTINATION_NAME = "test"; private static final String USERNAME = "user"; private static final String PASSWORD = "pass"; // specific for dynamic queues on ActiveMq @@ -115,15 +116,14 @@ public Void answer(InvocationOnMock invocation) throws Throwable { } }).when(channelProcessor).processEventBatch(any(List.class)); source.setChannelProcessor(channelProcessor); - - } + @After public void tearDown() throws Exception { - if(source != null) { + if (source != null) { source.stop(); } - if(broker != null) { + if (broker != null) { broker.stop(); } FileUtils.deleteDirectory(baseDir); @@ -140,8 +140,7 @@ private void putQueue(List events) throws Exception { Destination destination = session.createQueue(DESTINATION_NAME); MessageProducer producer = session.createProducer(destination); - - for(String event : events) { + for (String event : events) { TextMessage message = session.createTextMessage(); message.setText(event); producer.send(message); @@ -162,8 +161,7 @@ private void putTopic(List events) throws Exception { Destination destination = session.createTopic(DESTINATION_NAME); MessageProducer producer = session.createProducer(destination); - - for(String event : events) { + for (String event : events) { TextMessage message = session.createTextMessage(); message.setText(event); producer.send(message); @@ -202,13 +200,14 @@ public void testQueue() throws Exception { Assert.assertEquals(Status.BACKOFF, source.process()); Assert.assertEquals(expected.size(), events.size()); List actual = Lists.newArrayList(); - for(Event event : events) { + for (Event event : events) { actual.add(new String(event.getBody(), Charsets.UTF_8)); } Collections.sort(expected); Collections.sort(actual); Assert.assertEquals(expected, actual); } + @Test public void testTopic() throws Exception { context.put(JMSSourceConfiguration.DESTINATION_TYPE, @@ -229,7 +228,7 @@ public void testTopic() throws Exception { Assert.assertEquals(Status.BACKOFF, source.process()); Assert.assertEquals(expected.size(), events.size()); List actual = Lists.newArrayList(); - for(Event event : events) { + for (Event event : events) { actual.add(new String(event.getBody(), Charsets.UTF_8)); } Collections.sort(expected); diff --git a/flume-ng-sources/flume-jms-source/src/test/java/org/apache/flume/source/jms/TestJMSMessageConsumer.java b/flume-ng-sources/flume-jms-source/src/test/java/org/apache/flume/source/jms/TestJMSMessageConsumer.java index 9bace82dc3..dcb47d9d6f 100644 --- a/flume-ng-sources/flume-jms-source/src/test/java/org/apache/flume/source/jms/TestJMSMessageConsumer.java +++ b/flume-ng-sources/flume-jms-source/src/test/java/org/apache/flume/source/jms/TestJMSMessageConsumer.java @@ -105,8 +105,7 @@ public void testUserPass() throws Exception { @Test public void testNoUserPass() throws Exception { userName = Optional.absent(); - when(connectionFactory.createConnection(USERNAME, PASSWORD)). - thenThrow(new AssertionError()); + when(connectionFactory.createConnection(USERNAME, PASSWORD)).thenThrow(new AssertionError()); when(connectionFactory.createConnection()).thenReturn(connection); consumer = create(); List events = consumer.take(); diff --git a/flume-ng-sources/flume-jms-source/src/test/java/org/apache/flume/source/jms/TestJMSSource.java b/flume-ng-sources/flume-jms-source/src/test/java/org/apache/flume/source/jms/TestJMSSource.java index 5423f8f369..cdc09b5ea2 100644 --- a/flume-ng-sources/flume-jms-source/src/test/java/org/apache/flume/source/jms/TestJMSSource.java +++ b/flume-ng-sources/flume-jms-source/src/test/java/org/apache/flume/source/jms/TestJMSSource.java @@ -61,6 +61,7 @@ public class TestJMSSource extends JMSMessageConsumerTestBase { private InitialContextFactory contextFactory; private File baseDir; private File passwordFile; + @SuppressWarnings("unchecked") @Override void afterSetup() throws Exception { @@ -79,10 +80,11 @@ public Void answer(InvocationOnMock invocation) throws Throwable { }).when(channelProcessor).processEventBatch(any(List.class)); consumerFactory = mock(JMSMessageConsumerFactory.class); consumer = spy(create()); - when(consumerFactory.create(any(InitialContext.class), any(ConnectionFactory.class), anyString(), - any(JMSDestinationType.class), any(JMSDestinationLocator.class), anyString(), anyInt(), anyLong(), - any(JMSMessageConverter.class), any(Optional.class), - any(Optional.class))).thenReturn(consumer); + when(consumerFactory.create(any(InitialContext.class), any(ConnectionFactory.class), + anyString(), any(JMSDestinationType.class), + any(JMSDestinationLocator.class), anyString(), anyInt(), anyLong(), + any(JMSMessageConverter.class), any(Optional.class), + any(Optional.class))).thenReturn(consumer); when(initialContext.lookup(anyString())).thenReturn(connectionFactory); contextFactory = mock(InitialContextFactory.class); when(contextFactory.create(any(Properties.class))).thenReturn(initialContext); @@ -97,10 +99,12 @@ public Void answer(InvocationOnMock invocation) throws Throwable { context.put(JMSSourceConfiguration.PROVIDER_URL, "dummy:1414"); context.put(JMSSourceConfiguration.INITIAL_CONTEXT_FACTORY, "ldap://dummy:389"); } + @Override void afterTearDown() throws Exception { FileUtils.deleteDirectory(baseDir); } + @Test public void testStop() throws Exception { source.configure(context); @@ -108,38 +112,45 @@ public void testStop() throws Exception { source.stop(); verify(consumer).close(); } + @Test(expected = IllegalArgumentException.class) public void testConfigureWithoutInitialContextFactory() throws Exception { context.put(JMSSourceConfiguration.INITIAL_CONTEXT_FACTORY, ""); source.configure(context); } + @Test(expected = IllegalArgumentException.class) public void testConfigureWithoutProviderURL() throws Exception { context.put(JMSSourceConfiguration.PROVIDER_URL, ""); source.configure(context); } + @Test(expected = IllegalArgumentException.class) public void testConfigureWithoutDestinationName() throws Exception { context.put(JMSSourceConfiguration.DESTINATION_NAME, ""); source.configure(context); } + @Test(expected = FlumeException.class) public void testConfigureWithBadDestinationType() throws Exception { context.put(JMSSourceConfiguration.DESTINATION_TYPE, "DUMMY"); source.configure(context); } + @Test(expected = IllegalArgumentException.class) public void testConfigureWithEmptyDestinationType() throws Exception { context.put(JMSSourceConfiguration.DESTINATION_TYPE, ""); source.configure(context); } + @SuppressWarnings("unchecked") @Test public void testStartConsumerCreateThrowsException() throws Exception { - when(consumerFactory.create(any(InitialContext.class), any(ConnectionFactory.class), anyString(), - any(JMSDestinationType.class), any(JMSDestinationLocator.class), anyString(), anyInt(), anyLong(), - any(JMSMessageConverter.class), any(Optional.class), - any(Optional.class))).thenThrow(new RuntimeException()); + when(consumerFactory.create(any(InitialContext.class), any(ConnectionFactory.class), + anyString(), any(JMSDestinationType.class), + any(JMSDestinationLocator.class), anyString(), anyInt(), anyLong(), + any(JMSMessageConverter.class), any(Optional.class), + any(Optional.class))).thenThrow(new RuntimeException()); source.configure(context); source.start(); try { @@ -149,28 +160,33 @@ public void testStartConsumerCreateThrowsException() throws Exception { } } + @Test(expected = FlumeException.class) public void testConfigureWithContextLookupThrowsException() throws Exception { when(initialContext.lookup(anyString())).thenThrow(new NamingException()); source.configure(context); } + @Test(expected = FlumeException.class) public void testConfigureWithContextCreateThrowsException() throws Exception { when(contextFactory.create(any(Properties.class))) .thenThrow(new NamingException()); source.configure(context); } + @Test(expected = IllegalArgumentException.class) public void testConfigureWithInvalidBatchSize() throws Exception { context.put(JMSSourceConfiguration.BATCH_SIZE, "0"); source.configure(context); } + @Test(expected = FlumeException.class) public void testConfigureWithInvalidPasswordFile() throws Exception { context.put(JMSSourceConfiguration.PASSWORD_FILE, "/dev/does/not/exist/nor/will/ever/exist"); source.configure(context); } + @Test public void testConfigureWithUserNameButNoPasswordFile() throws Exception { context.put(JMSSourceConfiguration.USERNAME, "dummy"); @@ -180,6 +196,7 @@ public void testConfigureWithUserNameButNoPasswordFile() throws Exception { Assert.assertEquals(batchSize, events.size()); assertBodyIsExpected(events); } + @Test public void testConfigureWithUserNameAndPasswordFile() throws Exception { context.put(JMSSourceConfiguration.USERNAME, "dummy"); @@ -191,11 +208,13 @@ public void testConfigureWithUserNameAndPasswordFile() throws Exception { Assert.assertEquals(batchSize, events.size()); assertBodyIsExpected(events); } + @Test(expected = FlumeException.class) public void testConfigureWithInvalidConverterClass() throws Exception { context.put(JMSSourceConfiguration.CONVERTER_TYPE, "not a valid classname"); source.configure(context); } + @Test public void testProcessNoStart() throws Exception { try { @@ -205,6 +224,7 @@ public void testProcessNoStart() throws Exception { } } + @Test public void testNonDefaultConverter() throws Exception { // tests that a classname can be specified @@ -217,24 +237,26 @@ public void testNonDefaultConverter() throws Exception { assertBodyIsExpected(events); verify(consumer).commit(); } - public static class NonBuilderNonConfigurableConverter - implements JMSMessageConverter { + + public static class NonBuilderNonConfigurableConverter implements JMSMessageConverter { @Override public List convert(Message message) throws JMSException { throw new UnsupportedOperationException(); } } - public static class NonBuilderConfigurableConverter - implements JMSMessageConverter, Configurable { + + public static class NonBuilderConfigurableConverter implements JMSMessageConverter, Configurable { @Override public List convert(Message message) throws JMSException { throw new UnsupportedOperationException(); } + @Override public void configure(Context context) { } } + @Test public void testNonBuilderConfigurableConverter() throws Exception { // tests that a non builder by configurable converter works @@ -247,6 +269,7 @@ public void testNonBuilderConfigurableConverter() throws Exception { assertBodyIsExpected(events); verify(consumer).commit(); } + @Test public void testNonBuilderNonConfigurableConverter() throws Exception { // tests that a non builder non configurable converter @@ -259,6 +282,7 @@ public void testNonBuilderNonConfigurableConverter() throws Exception { assertBodyIsExpected(events); verify(consumer).commit(); } + @Test public void testProcessFullBatch() throws Exception { source.configure(context); @@ -268,6 +292,7 @@ public void testProcessFullBatch() throws Exception { assertBodyIsExpected(events); verify(consumer).commit(); } + @Test public void testProcessNoEvents() throws Exception { when(messageConsumer.receive(anyLong())).thenReturn(null); @@ -277,6 +302,7 @@ public void testProcessNoEvents() throws Exception { Assert.assertEquals(0, events.size()); verify(consumer).commit(); } + @Test public void testProcessPartialBatch() throws Exception { when(messageConsumer.receiveNoWait()).thenReturn(message, (Message)null); @@ -287,6 +313,7 @@ public void testProcessPartialBatch() throws Exception { assertBodyIsExpected(events); verify(consumer).commit(); } + @SuppressWarnings("unchecked") @Test public void testProcessChannelProcessorThrowsChannelException() throws Exception { @@ -297,6 +324,7 @@ public void testProcessChannelProcessorThrowsChannelException() throws Exception Assert.assertEquals(Status.BACKOFF, source.process()); verify(consumer).rollback(); } + @SuppressWarnings("unchecked") @Test public void testProcessChannelProcessorThrowsError() throws Exception { @@ -312,6 +340,7 @@ public void testProcessChannelProcessorThrowsError() throws Exception { } verify(consumer).rollback(); } + @Test public void testProcessReconnect() throws Exception { source.configure(context); diff --git a/flume-ng-sources/flume-kafka-source/src/test/java/org/apache/flume/source/kafka/KafkaSourceEmbeddedKafka.java b/flume-ng-sources/flume-kafka-source/src/test/java/org/apache/flume/source/kafka/KafkaSourceEmbeddedKafka.java index affac03b6c..b72c532ee6 100644 --- a/flume-ng-sources/flume-kafka-source/src/test/java/org/apache/flume/source/kafka/KafkaSourceEmbeddedKafka.java +++ b/flume-ng-sources/flume-kafka-source/src/test/java/org/apache/flume/source/kafka/KafkaSourceEmbeddedKafka.java @@ -16,9 +16,9 @@ */ package org.apache.flume.source.kafka; +import kafka.admin.AdminUtils; import kafka.server.KafkaConfig; import kafka.server.KafkaServerStartable; -import kafka.admin.AdminUtils; import kafka.utils.ZkUtils; import org.I0Itec.zkclient.ZkClient; import org.apache.commons.io.FileUtils; @@ -70,8 +70,9 @@ public KafkaSourceEmbeddedKafka(Properties properties) { props.put("host.name", "localhost"); props.put("port", String.valueOf(serverPort)); props.put("log.dir", dir.getAbsolutePath()); - if (properties != null) + if (properties != null) { props.putAll(props); + } KafkaConfig config = new KafkaConfig(props); kafkaServer = new KafkaServerStartable(config); kafkaServer.startup(); @@ -134,12 +135,12 @@ public void createTopic(String topicName, int numPartitions) { // Create a ZooKeeper client int sessionTimeoutMs = 10000; int connectionTimeoutMs = 10000; - ZkClient zkClient = ZkUtils.createZkClient(HOST + ":" + zkPort, sessionTimeoutMs, connectionTimeoutMs); + ZkClient zkClient = + ZkUtils.createZkClient(HOST + ":" + zkPort, sessionTimeoutMs, connectionTimeoutMs); ZkUtils zkUtils = ZkUtils.apply(zkClient, false); int replicationFactor = 1; Properties topicConfig = new Properties(); - AdminUtils.createTopic(zkUtils, topicName, numPartitions, - replicationFactor, topicConfig); + AdminUtils.createTopic(zkUtils, topicName, numPartitions, replicationFactor, topicConfig); } } diff --git a/flume-ng-sources/flume-kafka-source/src/test/java/org/apache/flume/source/kafka/KafkaSourceEmbeddedZookeeper.java b/flume-ng-sources/flume-kafka-source/src/test/java/org/apache/flume/source/kafka/KafkaSourceEmbeddedZookeeper.java index db144c2b8e..f04fc64b29 100644 --- a/flume-ng-sources/flume-kafka-source/src/test/java/org/apache/flume/source/kafka/KafkaSourceEmbeddedZookeeper.java +++ b/flume-ng-sources/flume-kafka-source/src/test/java/org/apache/flume/source/kafka/KafkaSourceEmbeddedZookeeper.java @@ -31,7 +31,6 @@ public class KafkaSourceEmbeddedZookeeper { private NIOServerCnxnFactory factory; File dir; - public KafkaSourceEmbeddedZookeeper(int zkPort) { int tickTime = 2000; diff --git a/flume-ng-sources/flume-kafka-source/src/test/java/org/apache/flume/source/kafka/TestKafkaSource.java b/flume-ng-sources/flume-kafka-source/src/test/java/org/apache/flume/source/kafka/TestKafkaSource.java index b4250debca..1598741cd9 100644 --- a/flume-ng-sources/flume-kafka-source/src/test/java/org/apache/flume/source/kafka/TestKafkaSource.java +++ b/flume-ng-sources/flume-kafka-source/src/test/java/org/apache/flume/source/kafka/TestKafkaSource.java @@ -17,29 +17,18 @@ package org.apache.flume.source.kafka; -import static org.junit.Assert.assertEquals; -import static org.junit.Assert.assertTrue; -import static org.mockito.Matchers.any; -import static org.mockito.Mockito.*; - -import java.io.ByteArrayOutputStream; -import java.io.IOException; -import java.nio.ByteBuffer; -import java.util.HashMap; -import java.util.List; -import java.util.Map; -import java.util.Properties; -import java.util.regex.Pattern; - import com.google.common.base.Charsets; import com.google.common.collect.Lists; import junit.framework.Assert; import kafka.common.TopicExistsException; - import org.apache.avro.io.BinaryEncoder; import org.apache.avro.io.EncoderFactory; import org.apache.avro.specific.SpecificDatumWriter; -import org.apache.flume.*; +import org.apache.flume.ChannelException; +import org.apache.flume.Context; +import org.apache.flume.Event; +import org.apache.flume.EventDeliveryException; +import org.apache.flume.FlumeException; import org.apache.flume.PollableSource.Status; import org.apache.flume.channel.ChannelProcessor; import org.apache.flume.source.avro.AvroFlumeEvent; @@ -52,11 +41,36 @@ import org.slf4j.Logger; import org.slf4j.LoggerFactory; -import static org.apache.flume.source.kafka.KafkaSourceConstants.*; +import java.io.ByteArrayOutputStream; +import java.io.IOException; +import java.nio.ByteBuffer; +import java.util.HashMap; +import java.util.List; +import java.util.Map; +import java.util.Properties; +import java.util.regex.Pattern; + +import static org.apache.flume.source.kafka.KafkaSourceConstants.AVRO_EVENT; +import static org.apache.flume.source.kafka.KafkaSourceConstants.BATCH_DURATION_MS; +import static org.apache.flume.source.kafka.KafkaSourceConstants.BATCH_SIZE; +import static org.apache.flume.source.kafka.KafkaSourceConstants.BOOTSTRAP_SERVERS; +import static org.apache.flume.source.kafka.KafkaSourceConstants.DEFAULT_AUTO_COMMIT; +import static org.apache.flume.source.kafka.KafkaSourceConstants.KAFKA_CONSUMER_PREFIX; +import static org.apache.flume.source.kafka.KafkaSourceConstants.OLD_GROUP_ID; +import static org.apache.flume.source.kafka.KafkaSourceConstants.PARTITION_HEADER; +import static org.apache.flume.source.kafka.KafkaSourceConstants.TIMESTAMP_HEADER; +import static org.apache.flume.source.kafka.KafkaSourceConstants.TOPIC; +import static org.apache.flume.source.kafka.KafkaSourceConstants.TOPICS; +import static org.apache.flume.source.kafka.KafkaSourceConstants.TOPICS_REGEX; +import static org.apache.flume.source.kafka.KafkaSourceConstants.TOPIC_HEADER; +import static org.junit.Assert.assertEquals; +import static org.junit.Assert.assertTrue; +import static org.mockito.Matchers.any; +import static org.mockito.Mockito.doAnswer; +import static org.mockito.Mockito.mock; public class TestKafkaSource { - private static final Logger log = - LoggerFactory.getLogger(TestKafkaSource.class); + private static final Logger log = LoggerFactory.getLogger(TestKafkaSource.class); private KafkaSource kafkaSource; private KafkaSourceEmbeddedKafka kafkaServer; @@ -243,10 +257,10 @@ public void testNonExistingTopic() throws EventDeliveryException, } @SuppressWarnings("unchecked") - @Test(expected= FlumeException.class) - public void testNonExistingKafkaServer() throws EventDeliveryException, - SecurityException, NoSuchFieldException, IllegalArgumentException, - IllegalAccessException, InterruptedException { + @Test(expected = FlumeException.class) + public void testNonExistingKafkaServer() throws EventDeliveryException, SecurityException, + NoSuchFieldException, IllegalArgumentException, + IllegalAccessException, InterruptedException { context.put(TOPICS, topic0); context.put(BOOTSTRAP_SERVERS,"blabla:666"); kafkaSource.configure(context); @@ -258,8 +272,7 @@ public void testNonExistingKafkaServer() throws EventDeliveryException, } @Test - public void testBatchTime() throws InterruptedException, - EventDeliveryException { + public void testBatchTime() throws InterruptedException, EventDeliveryException { context.put(TOPICS, topic0); context.put(BATCH_DURATION_MS, "250"); kafkaSource.configure(context); @@ -267,7 +280,7 @@ public void testBatchTime() throws InterruptedException, Thread.sleep(500L); - for (int i=1; i<5000; i++) { + for (int i = 1; i < 5000; i++) { kafkaServer.produce(topic0, "", "hello, world " + i); } Thread.sleep(500L); @@ -277,8 +290,7 @@ public void testBatchTime() throws InterruptedException, Status status = kafkaSource.process(); long endTime = System.currentTimeMillis(); assertEquals(Status.READY, status); - assertTrue(endTime - startTime < - (context.getLong(BATCH_DURATION_MS) + error)); + assertTrue(endTime - startTime < (context.getLong(BATCH_DURATION_MS) + error)); } // Consume event, stop source, start again and make sure we are not @@ -302,13 +314,11 @@ public void testCommit() throws InterruptedException, EventDeliveryException { kafkaSource.start(); Thread.sleep(500L); Assert.assertEquals(Status.BACKOFF, kafkaSource.process()); - } // Remove channel processor and test if we can consume events again @Test - public void testNonCommit() throws EventDeliveryException, - InterruptedException { + public void testNonCommit() throws EventDeliveryException, InterruptedException { context.put(TOPICS, topic0); context.put(BATCH_SIZE,"1"); context.put(BATCH_DURATION_MS,"30000"); @@ -328,15 +338,11 @@ public void testNonCommit() throws EventDeliveryException, log.debug("re-process to good channel - this should work"); kafkaSource.process(); - Assert.assertEquals("hello, world", new String(events.get(0).getBody(), - Charsets.UTF_8)); - - + Assert.assertEquals("hello, world", new String(events.get(0).getBody(), Charsets.UTF_8)); } @Test - public void testTwoBatches() throws InterruptedException, - EventDeliveryException { + public void testTwoBatches() throws InterruptedException, EventDeliveryException { context.put(TOPICS, topic0); context.put(BATCH_SIZE,"1"); context.put(BATCH_DURATION_MS, "30000"); @@ -348,20 +354,17 @@ public void testTwoBatches() throws InterruptedException, Thread.sleep(500L); kafkaSource.process(); - Assert.assertEquals("event 1", new String(events.get(0).getBody(), - Charsets.UTF_8)); + Assert.assertEquals("event 1", new String(events.get(0).getBody(), Charsets.UTF_8)); events.clear(); kafkaServer.produce(topic0, "", "event 2"); Thread.sleep(500L); kafkaSource.process(); - Assert.assertEquals("event 2", new String(events.get(0).getBody(), - Charsets.UTF_8)); + Assert.assertEquals("event 2", new String(events.get(0).getBody(), Charsets.UTF_8)); } @Test - public void testTwoBatchesWithAutocommit() throws InterruptedException, - EventDeliveryException { + public void testTwoBatchesWithAutocommit() throws InterruptedException, EventDeliveryException { context.put(TOPICS, topic0); context.put(BATCH_SIZE,"1"); context.put(BATCH_DURATION_MS,"30000"); @@ -374,23 +377,20 @@ public void testTwoBatchesWithAutocommit() throws InterruptedException, Thread.sleep(500L); kafkaSource.process(); - Assert.assertEquals("event 1", new String(events.get(0).getBody(), - Charsets.UTF_8)); + Assert.assertEquals("event 1", new String(events.get(0).getBody(), Charsets.UTF_8)); events.clear(); kafkaServer.produce(topic0, "", "event 2"); Thread.sleep(500L); kafkaSource.process(); - Assert.assertEquals("event 2", new String(events.get(0).getBody(), - Charsets.UTF_8)); - + Assert.assertEquals("event 2", new String(events.get(0).getBody(), Charsets.UTF_8)); } @SuppressWarnings("unchecked") @Test - public void testNullKey() throws EventDeliveryException, - SecurityException, NoSuchFieldException, IllegalArgumentException, - IllegalAccessException, InterruptedException { + public void testNullKey() throws EventDeliveryException, SecurityException, NoSuchFieldException, + IllegalArgumentException, IllegalAccessException, + InterruptedException { context.put(TOPICS, topic0); context.put(BATCH_SIZE, "1"); kafkaSource.configure(context); @@ -406,8 +406,7 @@ public void testNullKey() throws EventDeliveryException, Assert.assertEquals(Status.BACKOFF, kafkaSource.process()); Assert.assertEquals(1, events.size()); - Assert.assertEquals("hello, world", new String(events.get(0).getBody(), - Charsets.UTF_8)); + Assert.assertEquals("hello, world", new String(events.get(0).getBody(), Charsets.UTF_8)); } @Test @@ -430,7 +429,8 @@ public void testSourceProperties() { public void testKafkaProperties() { Context context = new Context(); context.put(TOPICS, "test1, test2"); - context.put(KAFKA_CONSUMER_PREFIX + ConsumerConfig.GROUP_ID_CONFIG, "override.default.group.id"); + context.put(KAFKA_CONSUMER_PREFIX + ConsumerConfig.GROUP_ID_CONFIG, + "override.default.group.id"); context.put(KAFKA_CONSUMER_PREFIX + "fake.property", "kafka.property.value"); context.put(BOOTSTRAP_SERVERS, "real-bootstrap-servers-list"); context.put(KAFKA_CONSUMER_PREFIX + "bootstrap.servers", "bad-bootstrap-servers-list"); @@ -439,21 +439,17 @@ public void testKafkaProperties() { Properties kafkaProps = source.getConsumerProps(); //check that we have defaults set - assertEquals( - String.valueOf(DEFAULT_AUTO_COMMIT), - kafkaProps.getProperty(ConsumerConfig.ENABLE_AUTO_COMMIT_CONFIG)); + assertEquals(String.valueOf(DEFAULT_AUTO_COMMIT), + kafkaProps.getProperty(ConsumerConfig.ENABLE_AUTO_COMMIT_CONFIG)); //check that kafka properties override the default and get correct name - assertEquals( - "override.default.group.id", - kafkaProps.getProperty(ConsumerConfig.GROUP_ID_CONFIG)); + assertEquals("override.default.group.id", + kafkaProps.getProperty(ConsumerConfig.GROUP_ID_CONFIG)); //check that any kafka property gets in - assertEquals( - "kafka.property.value", - kafkaProps.getProperty("fake.property")); + assertEquals("kafka.property.value", + kafkaProps.getProperty("fake.property")); //check that documented property overrides defaults - assertEquals( - "real-bootstrap-servers-list", - kafkaProps.getProperty("bootstrap.servers")); + assertEquals("real-bootstrap-servers-list", + kafkaProps.getProperty("bootstrap.servers")); } @Test @@ -469,22 +465,16 @@ public void testOldProperties() { KafkaSource.Subscriber> subscriber = source.getSubscriber(); //check topic was set - assertEquals( - "old.topic", - subscriber.get().get(0)); + assertEquals("old.topic", subscriber.get().get(0)); //check that kafka old properties override the default and get correct name - assertEquals( - "old.groupId", - kafkaProps.getProperty(ConsumerConfig.GROUP_ID_CONFIG)); + assertEquals("old.groupId", kafkaProps.getProperty(ConsumerConfig.GROUP_ID_CONFIG)); source = new KafkaSource(); context.put(KAFKA_CONSUMER_PREFIX + ConsumerConfig.GROUP_ID_CONFIG, "override.old.group.id"); source.doConfigure(context); kafkaProps = source.getConsumerProps(); //check that kafka new properties override old - assertEquals( - "override.old.group.id", - kafkaProps.getProperty(ConsumerConfig.GROUP_ID_CONFIG)); + assertEquals("override.old.group.id", kafkaProps.getProperty(ConsumerConfig.GROUP_ID_CONFIG)); context.clear(); context.put(BOOTSTRAP_SERVERS, "real-bootstrap-servers-list"); @@ -493,9 +483,8 @@ public void testOldProperties() { source.doConfigure(context); kafkaProps = source.getConsumerProps(); //check defaults set - assertEquals( - KafkaSourceConstants.DEFAULT_GROUP_ID, - kafkaProps.getProperty(ConsumerConfig.GROUP_ID_CONFIG)); + assertEquals(KafkaSourceConstants.DEFAULT_GROUP_ID, + kafkaProps.getProperty(ConsumerConfig.GROUP_ID_CONFIG)); } @Test @@ -568,8 +557,7 @@ public void testAvroEvent() throws InterruptedException, EventDeliveryException, Event event = events.get(0); - Assert.assertEquals("hello, world", new String(event.getBody(), - Charsets.UTF_8)); + Assert.assertEquals("hello, world", new String(event.getBody(), Charsets.UTF_8)); Assert.assertEquals("value1", e.getHeaders().get("header1")); Assert.assertEquals("value2", e.getHeaders().get("header2")); @@ -577,8 +565,7 @@ public void testAvroEvent() throws InterruptedException, EventDeliveryException, event = events.get(1); - Assert.assertEquals("hello, world2", new String(event.getBody(), - Charsets.UTF_8)); + Assert.assertEquals("hello, world2", new String(event.getBody(), Charsets.UTF_8)); Assert.assertEquals("value1", e.getHeaders().get("header1")); Assert.assertEquals("value2", e.getHeaders().get("header2")); @@ -603,7 +590,6 @@ public Void answer(InvocationOnMock invocation) throws Throwable { }).when(channelProcessor).processEventBatch(any(List.class)); return channelProcessor; - } ChannelProcessor createBadChannel() { diff --git a/flume-ng-sources/flume-taildir-source/src/test/java/org/apache/flume/source/taildir/TestTaildirEventReader.java b/flume-ng-sources/flume-taildir-source/src/test/java/org/apache/flume/source/taildir/TestTaildirEventReader.java index 18968838ab..bcfe4bbf61 100644 --- a/flume-ng-sources/flume-taildir-source/src/test/java/org/apache/flume/source/taildir/TestTaildirEventReader.java +++ b/flume-ng-sources/flume-taildir-source/src/test/java/org/apache/flume/source/taildir/TestTaildirEventReader.java @@ -19,20 +19,6 @@ package org.apache.flume.source.taildir; -import static org.apache.flume.source.taildir.TaildirSourceConfigurationConstants.*; -import static org.junit.Assert.*; - -import java.io.File; -import java.io.IOException; -import java.util.List; -import java.util.Map; -import java.util.Set; - -import org.apache.flume.Event; -import org.junit.After; -import org.junit.Before; -import org.junit.Test; - import com.google.common.base.Charsets; import com.google.common.base.Throwables; import com.google.common.collect.HashBasedTable; @@ -41,6 +27,22 @@ import com.google.common.collect.Sets; import com.google.common.collect.Table; import com.google.common.io.Files; +import org.apache.flume.Event; +import org.junit.After; +import org.junit.Before; +import org.junit.Test; + +import java.io.File; +import java.io.IOException; +import java.util.List; +import java.util.Map; +import java.util.Set; + +import static org.apache.flume.source.taildir.TaildirSourceConfigurationConstants + .BYTE_OFFSET_HEADER_KEY; +import static org.junit.Assert.assertEquals; +import static org.junit.Assert.assertNull; +import static org.junit.Assert.assertTrue; public class TestTaildirEventReader { private File tmpDir; @@ -85,7 +87,8 @@ private ReliableTaildirEventReader getReader(Map filePaths, } private ReliableTaildirEventReader getReader(boolean addByteOffset) { - Map filePaths = ImmutableMap.of("testFiles", tmpDir.getAbsolutePath() + "/file.*"); + Map filePaths = ImmutableMap.of("testFiles", + tmpDir.getAbsolutePath() + "/file.*"); Table headerTable = HashBasedTable.create(); return getReader(filePaths, headerTable, addByteOffset); } @@ -472,7 +475,8 @@ public void testByteOffsetHeader() throws IOException { @Test public void testNewLineBoundaries() throws IOException { File f1 = new File(tmpDir, "file1"); - Files.write("file1line1\nfile1line2\rfile1line2\nfile1line3\r\nfile1line4\n", f1, Charsets.UTF_8); + Files.write("file1line1\nfile1line2\rfile1line2\nfile1line3\r\nfile1line4\n", + f1, Charsets.UTF_8); ReliableTaildirEventReader reader = getReader(); List out = Lists.newArrayList(); diff --git a/flume-ng-sources/flume-taildir-source/src/test/java/org/apache/flume/source/taildir/TestTaildirMatcher.java b/flume-ng-sources/flume-taildir-source/src/test/java/org/apache/flume/source/taildir/TestTaildirMatcher.java index 4bff8418ce..c3410548eb 100644 --- a/flume-ng-sources/flume-taildir-source/src/test/java/org/apache/flume/source/taildir/TestTaildirMatcher.java +++ b/flume-ng-sources/flume-taildir-source/src/test/java/org/apache/flume/source/taildir/TestTaildirMatcher.java @@ -33,7 +33,10 @@ import java.util.List; import java.util.Map; -import static org.junit.Assert.*; +import static org.junit.Assert.assertEquals; +import static org.junit.Assert.assertFalse; +import static org.junit.Assert.assertNotNull; +import static org.junit.Assert.assertTrue; public class TestTaildirMatcher { private File tmpDir; @@ -55,10 +58,10 @@ public class TestTaildirMatcher { */ private void append(String fileName) throws IOException { File f; - if(!files.containsKey(fileName)){ + if (!files.containsKey(fileName)) { f = new File(tmpDir, fileName); files.put(fileName, f); - }else{ + } else { f = files.get(fileName); } Files.append(fileName + "line\n", f, Charsets.UTF_8); @@ -67,7 +70,7 @@ private void append(String fileName) throws IOException { /** * Translate a list of files to list of filename strings. */ - private static List filesToNames(List origList){ + private static List filesToNames(List origList) { Function file2nameFn = new Function() { @Override public String apply(File input) { @@ -102,7 +105,9 @@ public void getMatchingFiles() throws Exception { append("file0"); append("file1"); - TaildirMatcher tm = new TaildirMatcher("f1", tmpDir.getAbsolutePath() + File.separator + "file.*", isCachingNeeded); + TaildirMatcher tm = new TaildirMatcher("f1", + tmpDir.getAbsolutePath() + File.separator + "file.*", + isCachingNeeded); List files = filesToNames(tm.getMatchingFiles()); assertEquals(msgAlreadyExistingFile, 2, files.size()); assertTrue(msgAlreadyExistingFile, files.contains("file1")); @@ -136,7 +141,9 @@ public void getMatchingFilesNoCache() throws Exception { append("file0"); append("file1"); - TaildirMatcher tm = new TaildirMatcher("f1", tmpDir.getAbsolutePath() + File.separator + "file.*", false); + TaildirMatcher tm = new TaildirMatcher("f1", + tmpDir.getAbsolutePath() + File.separator + "file.*", + false); List files = filesToNames(tm.getMatchingFiles()); assertEquals(msgAlreadyExistingFile, 2, files.size()); assertTrue(msgAlreadyExistingFile, files.contains("file1")); @@ -167,7 +174,9 @@ public void getMatchingFilesNoCache() throws Exception { @Test public void testEmtpyDirMatching() throws Exception { - TaildirMatcher tm = new TaildirMatcher("empty", tmpDir.getAbsolutePath() + File.separator + ".*", isCachingNeeded); + TaildirMatcher tm = new TaildirMatcher("empty", + tmpDir.getAbsolutePath() + File.separator + ".*", + isCachingNeeded); List files = tm.getMatchingFiles(); assertNotNull(msgEmptyDir, files); assertTrue(msgEmptyDir, files.isEmpty()); @@ -175,7 +184,10 @@ public void testEmtpyDirMatching() throws Exception { @Test public void testNoMatching() throws Exception { - TaildirMatcher tm = new TaildirMatcher("nomatch", tmpDir.getAbsolutePath() + File.separator + "abracadabra_nonexisting", isCachingNeeded); + TaildirMatcher tm = new TaildirMatcher( + "nomatch", + tmpDir.getAbsolutePath() + File.separator + "abracadabra_nonexisting", + isCachingNeeded); List files = tm.getMatchingFiles(); assertNotNull(msgNoMatch, files); assertTrue(msgNoMatch, files.isEmpty()); @@ -183,7 +195,8 @@ public void testNoMatching() throws Exception { @Test(expected = IllegalStateException.class) public void testNonExistingDir() { - TaildirMatcher tm = new TaildirMatcher("exception", "/abracadabra/doesntexist/.*", isCachingNeeded); + TaildirMatcher tm = new TaildirMatcher("exception", "/abracadabra/doesntexist/.*", + isCachingNeeded); } @Test @@ -191,7 +204,8 @@ public void testDirectoriesAreNotListed() throws Exception { new File(tmpDir, "outerFile").createNewFile(); new File(tmpDir, "recursiveDir").mkdir(); new File(tmpDir + File.separator + "recursiveDir", "innerFile").createNewFile(); - TaildirMatcher tm = new TaildirMatcher("f1", tmpDir.getAbsolutePath() + File.separator + ".*", isCachingNeeded); + TaildirMatcher tm = new TaildirMatcher("f1", tmpDir.getAbsolutePath() + File.separator + ".*", + isCachingNeeded); List files = filesToNames(tm.getMatchingFiles()); assertEquals(msgSubDirs, 1, files.size()); @@ -207,9 +221,13 @@ public void testRegexFileNameFiltering() throws IOException { append("c.log.yyyy.MM-02"); // Tail a.log and b.log - TaildirMatcher tm1 = new TaildirMatcher("ab", tmpDir.getAbsolutePath() + File.separator + "[ab].log", isCachingNeeded); + TaildirMatcher tm1 = new TaildirMatcher("ab", + tmpDir.getAbsolutePath() + File.separator + "[ab].log", + isCachingNeeded); // Tail files that starts with c.log - TaildirMatcher tm2 = new TaildirMatcher("c", tmpDir.getAbsolutePath() + File.separator + "c.log.*", isCachingNeeded); + TaildirMatcher tm2 = new TaildirMatcher("c", + tmpDir.getAbsolutePath() + File.separator + "c.log.*", + isCachingNeeded); List files1 = filesToNames(tm1.getMatchingFiles()); List files2 = filesToNames(tm2.getMatchingFiles()); @@ -217,11 +235,16 @@ public void testRegexFileNameFiltering() throws IOException { assertEquals(2, files1.size()); assertEquals(2, files2.size()); // Make sure we got every file - assertTrue("Regex pattern for ab should have matched a.log file", files1.contains("a.log")); - assertFalse("Regex pattern for ab should NOT have matched a.log.1 file", files1.contains("a.log.1")); - assertTrue("Regex pattern for ab should have matched b.log file", files1.contains("b.log")); - assertTrue("Regex pattern for c should have matched c.log.yyyy-MM-01 file", files2.contains("c.log.yyyy.MM-01")); - assertTrue("Regex pattern for c should have matched c.log.yyyy-MM-02 file", files2.contains("c.log.yyyy.MM-02")); + assertTrue("Regex pattern for ab should have matched a.log file", + files1.contains("a.log")); + assertFalse("Regex pattern for ab should NOT have matched a.log.1 file", + files1.contains("a.log.1")); + assertTrue("Regex pattern for ab should have matched b.log file", + files1.contains("b.log")); + assertTrue("Regex pattern for c should have matched c.log.yyyy-MM-01 file", + files2.contains("c.log.yyyy.MM-01")); + assertTrue("Regex pattern for c should have matched c.log.yyyy-MM-02 file", + files2.contains("c.log.yyyy.MM-02")); } } \ No newline at end of file diff --git a/flume-ng-sources/flume-taildir-source/src/test/java/org/apache/flume/source/taildir/TestTaildirSource.java b/flume-ng-sources/flume-taildir-source/src/test/java/org/apache/flume/source/taildir/TestTaildirSource.java index f6289cd15b..e090b74461 100644 --- a/flume-ng-sources/flume-taildir-source/src/test/java/org/apache/flume/source/taildir/TestTaildirSource.java +++ b/flume-ng-sources/flume-taildir-source/src/test/java/org/apache/flume/source/taildir/TestTaildirSource.java @@ -17,14 +17,9 @@ package org.apache.flume.source.taildir; -import static org.apache.flume.source.taildir.TaildirSourceConfigurationConstants.*; -import static org.junit.Assert.*; - -import java.io.File; -import java.io.IOException; -import java.util.ArrayList; -import java.util.List; - +import com.google.common.base.Charsets; +import com.google.common.collect.Lists; +import com.google.common.io.Files; import org.apache.flume.Channel; import org.apache.flume.ChannelSelector; import org.apache.flume.Context; @@ -40,9 +35,21 @@ import org.junit.Before; import org.junit.Test; -import com.google.common.base.Charsets; -import com.google.common.collect.Lists; -import com.google.common.io.Files; +import java.io.File; +import java.io.IOException; +import java.util.ArrayList; +import java.util.List; + +import static org.apache.flume.source.taildir.TaildirSourceConfigurationConstants.FILE_GROUPS; +import static org.apache.flume.source.taildir.TaildirSourceConfigurationConstants + .FILE_GROUPS_PREFIX; +import static org.apache.flume.source.taildir.TaildirSourceConfigurationConstants.HEADERS_PREFIX; +import static org.apache.flume.source.taildir.TaildirSourceConfigurationConstants.POSITION_FILE; +import static org.junit.Assert.assertArrayEquals; +import static org.junit.Assert.assertEquals; +import static org.junit.Assert.assertFalse; +import static org.junit.Assert.assertNull; +import static org.junit.Assert.assertTrue; public class TestTaildirSource { static TaildirSource source; @@ -274,10 +281,11 @@ public void testFileConsumeOrder() throws IOException { line1b, line2b, line3b, // file2 line1d, line2d, line3d, // file4 line1c, line2c, line3c // file3 - ); - for(int i =0; i!=expected.size(); ++i) { - expected.set(i, expected.get(i).trim() ); + ); + for (int i = 0; i != expected.size(); ++i) { + expected.set(i, expected.get(i).trim()); } - assertArrayEquals("Files not consumed in expected order", expected.toArray(), consumedOrder.toArray()); + assertArrayEquals("Files not consumed in expected order", expected.toArray(), + consumedOrder.toArray()); } } diff --git a/flume-ng-tests/src/test/java/org/apache/flume/test/agent/TestFileChannel.java b/flume-ng-tests/src/test/java/org/apache/flume/test/agent/TestFileChannel.java index 4a80b8cca4..6a98292847 100644 --- a/flume-ng-tests/src/test/java/org/apache/flume/test/agent/TestFileChannel.java +++ b/flume-ng-tests/src/test/java/org/apache/flume/test/agent/TestFileChannel.java @@ -18,14 +18,8 @@ */ package org.apache.flume.test.agent; -import java.io.File; -import java.nio.charset.Charset; -import java.util.ArrayList; -import java.util.Collection; -import java.util.Properties; -import java.util.concurrent.TimeUnit; -import java.util.regex.Pattern; - +import com.google.common.base.Charsets; +import com.google.common.io.Files; import org.apache.flume.test.util.StagedInstall; import org.apache.log4j.Logger; import org.junit.After; @@ -33,9 +27,11 @@ import org.junit.Before; import org.junit.Test; -import com.google.common.base.Charsets; -import com.google.common.base.Splitter; -import com.google.common.io.Files; +import java.io.File; +import java.util.ArrayList; +import java.util.Collection; +import java.util.Properties; +import java.util.concurrent.TimeUnit; public class TestFileChannel { @@ -48,61 +44,57 @@ public class TestFileChannel { @Before public void setUp() throws Exception { - /* Create 3 temp dirs, each used as value within agentProps */ - - final File sinkOutputDir = Files.createTempDir(); - tempResources.add(sinkOutputDir); - final String sinkOutputDirPath = sinkOutputDir.getCanonicalPath(); - LOGGER.info("Created rolling file sink's output dir: " - + sinkOutputDirPath); - - final File channelCheckpointDir = Files.createTempDir(); - tempResources.add(channelCheckpointDir); - final String channelCheckpointDirPath = channelCheckpointDir - .getCanonicalPath(); - LOGGER.info("Created file channel's checkpoint dir: " - + channelCheckpointDirPath); - - final File channelDataDir = Files.createTempDir(); - tempResources.add(channelDataDir); - final String channelDataDirPath = channelDataDir.getCanonicalPath(); - LOGGER.info("Created file channel's data dir: " - + channelDataDirPath); - - /* Build props to pass to flume agent */ - - Properties agentProps = new Properties(); - - // Active sets - agentProps.put("a1.channels", "c1"); - agentProps.put("a1.sources", "r1"); - agentProps.put("a1.sinks", "k1"); - - // c1 - agentProps.put("a1.channels.c1.type", "FILE"); - agentProps.put("a1.channels.c1.checkpointDir", channelCheckpointDirPath); - agentProps.put("a1.channels.c1.dataDirs", channelDataDirPath); - - // r1 - agentProps.put("a1.sources.r1.channels", "c1"); - agentProps.put("a1.sources.r1.type", "EXEC"); - agentProps.put("a1.sources.r1.command", "seq 1 100"); - - // k1 - agentProps.put("a1.sinks.k1.channel", "c1"); - agentProps.put("a1.sinks.k1.type", "FILE_ROLL"); - agentProps.put("a1.sinks.k1.sink.directory", sinkOutputDirPath); - agentProps.put("a1.sinks.k1.sink.rollInterval", "0"); - - this.agentProps = agentProps; - this.sinkOutputDir = sinkOutputDir; + /* Create 3 temp dirs, each used as value within agentProps */ + + final File sinkOutputDir = Files.createTempDir(); + tempResources.add(sinkOutputDir); + final String sinkOutputDirPath = sinkOutputDir.getCanonicalPath(); + LOGGER.info("Created rolling file sink's output dir: " + sinkOutputDirPath); + + final File channelCheckpointDir = Files.createTempDir(); + tempResources.add(channelCheckpointDir); + final String channelCheckpointDirPath = channelCheckpointDir.getCanonicalPath(); + LOGGER.info("Created file channel's checkpoint dir: " + channelCheckpointDirPath); + + final File channelDataDir = Files.createTempDir(); + tempResources.add(channelDataDir); + final String channelDataDirPath = channelDataDir.getCanonicalPath(); + LOGGER.info("Created file channel's data dir: " + channelDataDirPath); + + /* Build props to pass to flume agent */ + + Properties agentProps = new Properties(); + + // Active sets + agentProps.put("a1.channels", "c1"); + agentProps.put("a1.sources", "r1"); + agentProps.put("a1.sinks", "k1"); + + // c1 + agentProps.put("a1.channels.c1.type", "FILE"); + agentProps.put("a1.channels.c1.checkpointDir", channelCheckpointDirPath); + agentProps.put("a1.channels.c1.dataDirs", channelDataDirPath); + + // r1 + agentProps.put("a1.sources.r1.channels", "c1"); + agentProps.put("a1.sources.r1.type", "EXEC"); + agentProps.put("a1.sources.r1.command", "seq 1 100"); + + // k1 + agentProps.put("a1.sinks.k1.channel", "c1"); + agentProps.put("a1.sinks.k1.type", "FILE_ROLL"); + agentProps.put("a1.sinks.k1.sink.directory", sinkOutputDirPath); + agentProps.put("a1.sinks.k1.sink.rollInterval", "0"); + + this.agentProps = agentProps; + this.sinkOutputDir = sinkOutputDir; } @After public void tearDown() throws Exception { StagedInstall.getInstance().stopAgent(); for (File tempResource : tempResources) { - tempResource.delete(); + tempResource.delete(); } agentProps = null; } @@ -110,7 +102,7 @@ public void tearDown() throws Exception { /** * File channel in/out test. Verifies that all events inserted into the * file channel are received by the sink in order. - * + *

    * The EXEC source creates 100 events where the event bodies have * sequential numbers. The source puts those events into the file channel, * and the FILE_ROLL The sink is expected to take all 100 events in FIFO @@ -119,38 +111,36 @@ public void tearDown() throws Exception { * @throws Exception */ @Test - public void testInOut() throws Exception { - LOGGER.debug("testInOut() started."); - - StagedInstall.getInstance().startAgent("a1", agentProps); - TimeUnit.SECONDS.sleep(10); // Wait for source and sink to finish - // TODO make this more deterministic - - /* Create expected output */ - - StringBuffer sb = new StringBuffer(); - for (int i = 1; i <= 100; i++) { - sb.append(i).append("\n"); - } - String expectedOutput = sb.toString(); - LOGGER.info("Created expected output: " + expectedOutput); - - /* Create actual output file */ - - File[] sinkOutputDirChildren = sinkOutputDir.listFiles(); - // Only 1 file should be in FILE_ROLL sink's dir (rolling is disabled) - Assert.assertEquals("Expected FILE_ROLL sink's dir to have only 1 child," + - " but found " + sinkOutputDirChildren.length + " children.", - 1, sinkOutputDirChildren.length); - File actualOutput = sinkOutputDirChildren[0]; - - if (!Files.toString(actualOutput, Charsets.UTF_8).equals(expectedOutput)) { - LOGGER.error("Actual output doesn't match expected output.\n"); - throw new AssertionError("FILE_ROLL sink's actual output doesn't " + - "match expected output."); - } - - LOGGER.debug("testInOut() ended."); - } + public void testInOut() throws Exception { + LOGGER.debug("testInOut() started."); + StagedInstall.getInstance().startAgent("a1", agentProps); + TimeUnit.SECONDS.sleep(10); // Wait for source and sink to finish + // TODO make this more deterministic + + /* Create expected output */ + StringBuffer sb = new StringBuffer(); + for (int i = 1; i <= 100; i++) { + sb.append(i).append("\n"); + } + String expectedOutput = sb.toString(); + LOGGER.info("Created expected output: " + expectedOutput); + + /* Create actual output file */ + + File[] sinkOutputDirChildren = sinkOutputDir.listFiles(); + // Only 1 file should be in FILE_ROLL sink's dir (rolling is disabled) + Assert.assertEquals("Expected FILE_ROLL sink's dir to have only 1 child," + + " but found " + sinkOutputDirChildren.length + " children.", + 1, sinkOutputDirChildren.length); + File actualOutput = sinkOutputDirChildren[0]; + + if (!Files.toString(actualOutput, Charsets.UTF_8).equals(expectedOutput)) { + LOGGER.error("Actual output doesn't match expected output.\n"); + throw new AssertionError("FILE_ROLL sink's actual output doesn't " + + "match expected output."); + } + + LOGGER.debug("testInOut() ended."); + } } diff --git a/flume-ng-tests/src/test/java/org/apache/flume/test/util/StagedInstall.java b/flume-ng-tests/src/test/java/org/apache/flume/test/util/StagedInstall.java index 973ff4a327..51194b6ac4 100644 --- a/flume-ng-tests/src/test/java/org/apache/flume/test/util/StagedInstall.java +++ b/flume-ng-tests/src/test/java/org/apache/flume/test/util/StagedInstall.java @@ -18,6 +18,14 @@ */ package org.apache.flume.test.util; +import com.google.common.base.Joiner; +import com.google.common.base.Preconditions; +import com.google.common.collect.ImmutableList; +import com.google.common.io.Files; +import org.apache.commons.compress.archivers.tar.TarArchiveEntry; +import org.apache.commons.compress.archivers.tar.TarArchiveInputStream; +import org.apache.log4j.Logger; + import java.io.File; import java.io.FileFilter; import java.io.FileInputStream; @@ -29,18 +37,8 @@ import java.util.List; import java.util.Map; import java.util.Properties; -import java.util.concurrent.TimeUnit; import java.util.zip.GZIPInputStream; -import com.google.common.base.Preconditions; -import com.google.common.io.Files; -import org.apache.commons.compress.archivers.tar.TarArchiveEntry; -import org.apache.commons.compress.archivers.tar.TarArchiveInputStream; -import org.apache.log4j.Logger; - -import com.google.common.base.Joiner; -import com.google.common.collect.ImmutableList; - /** * Attempts to setup a staged install using explicitly specified tar-ball @@ -73,7 +71,7 @@ public class StagedInstall { private static StagedInstall INSTANCE; - public synchronized static StagedInstall getInstance() throws Exception { + public static synchronized StagedInstall getInstance() throws Exception { if (INSTANCE == null) { INSTANCE = new StagedInstall(); } @@ -124,8 +122,7 @@ public synchronized void startAgent(String name, Properties properties) if (process != null) { throw new Exception("A process is already running"); } - LOGGER.info("Starting process for agent: " + agentName + " using config: " - + properties); + LOGGER.info("Starting process for agent: " + agentName + " using config: " + properties); File configFile = createConfigurationFile(agentName, properties); configFilePath = configFile.getCanonicalPath(); @@ -252,7 +249,7 @@ private StagedInstall() throws Exception { File[] listBaseDirs = stageDir.listFiles(); if (listBaseDirs != null && listBaseDirs.length == 1 && listBaseDirs[0].isDirectory()) { - rootDir =listBaseDirs[0]; + rootDir = listBaseDirs[0]; } baseDir = rootDir; @@ -417,7 +414,6 @@ private String getRelativeTarballPath() throws Exception { if (testFile.exists() && testFile.isDirectory()) { LOGGER.info("Found candidate dir: " + testFile.getCanonicalPath()); File[] candidateFiles = testFile.listFiles(new FileFilter() { - @Override public boolean accept(File pathname) { String name = pathname.getName(); @@ -426,7 +422,8 @@ public boolean accept(File pathname) { return true; } return false; - }}); + } + }); // There should be at most one if (candidateFiles != null && candidateFiles.length > 0) { @@ -466,22 +463,22 @@ public boolean accept(File pathname) { } public static void waitUntilPortOpens(String host, int port, long timeout) - throws IOException, InterruptedException{ + throws IOException, InterruptedException { long startTime = System.currentTimeMillis(); Socket socket; boolean connected = false; //See if port has opened for timeout. - while(System.currentTimeMillis() - startTime < timeout){ - try{ + while (System.currentTimeMillis() - startTime < timeout) { + try { socket = new Socket(host, port); socket.close(); connected = true; break; - } catch (IOException e){ + } catch (IOException e) { Thread.sleep(2000); } } - if(!connected) { + if (!connected) { throw new IOException("Port not opened within specified timeout."); } } diff --git a/flume-ng-tests/src/test/java/org/apache/flume/test/util/SyslogAgent.java b/flume-ng-tests/src/test/java/org/apache/flume/test/util/SyslogAgent.java index 7159549b46..c908fc167d 100644 --- a/flume-ng-tests/src/test/java/org/apache/flume/test/util/SyslogAgent.java +++ b/flume-ng-tests/src/test/java/org/apache/flume/test/util/SyslogAgent.java @@ -60,7 +60,7 @@ private SyslogSourceType(String syslogSourceType) { public String toString() { return syslogSourceType; } - }; + } private Properties agentProps; private File sinkOutputDir; @@ -73,7 +73,6 @@ public String toString() { public SyslogAgent() throws IOException { hostname = "localhost"; - setRandomPort(); } @@ -141,7 +140,7 @@ public void start(String keepFields) throws Exception { while (client == null) { try { client = new BufferedOutputStream(new Socket(hostname, port).getOutputStream()); - } catch(IOException e) { + } catch (IOException e) { if (++numberOfAttempts >= DEFAULT_ATTEMPTS) { throw new AssertionError("Could not connect to source after " + DEFAULT_ATTEMPTS + " attempts with " + DEFAULT_TIMEOUT + " ms timeout."); @@ -206,8 +205,8 @@ public void runKeepFieldsTest() throws Exception { // Only 1 file should be in FILE_ROLL sink's dir (rolling is disabled) File[] sinkOutputDirChildren = sinkOutputDir.listFiles(); Assert.assertEquals("Expected FILE_ROLL sink's dir to have only 1 child," + - " but found " + sinkOutputDirChildren.length + " children.", - 1, sinkOutputDirChildren.length); + " but found " + sinkOutputDirChildren.length + " children.", + 1, sinkOutputDirChildren.length); /* Wait for output file stats to be as expected. */ File outputDirChild = sinkOutputDirChildren[0]; diff --git a/flume-tools/src/test/java/org/apache/flume/tools/TestFileChannelIntegrityTool.java b/flume-tools/src/test/java/org/apache/flume/tools/TestFileChannelIntegrityTool.java index a11126dfdd..9fc5f2c34f 100644 --- a/flume-tools/src/test/java/org/apache/flume/tools/TestFileChannelIntegrityTool.java +++ b/flume-tools/src/test/java/org/apache/flume/tools/TestFileChannelIntegrityTool.java @@ -34,7 +34,6 @@ import org.apache.flume.event.EventBuilder; import org.junit.After; import org.junit.AfterClass; -import static org.fest.reflect.core.Reflection.*; import org.junit.Assert; import org.junit.Before; import org.junit.BeforeClass; @@ -47,6 +46,8 @@ import java.util.Random; import java.util.Set; +import static org.fest.reflect.core.Reflection.field; +import static org.fest.reflect.core.Reflection.method; public class TestFileChannelIntegrityTool { private static File baseDir; @@ -61,7 +62,7 @@ public class TestFileChannelIntegrityTool { private static int invalidEvent = 0; @BeforeClass - public static void setUpClass() throws Exception{ + public static void setUpClass() throws Exception { createDataFiles(); } @@ -74,13 +75,13 @@ public void setUp() throws Exception { File[] dataFiles = origDataDir.listFiles(new FilenameFilter() { @Override public boolean accept(File dir, String name) { - if(name.contains("lock")) { + if (name.contains("lock")) { return false; } return true; } }); - for(File dataFile : dataFiles) { + for (File dataFile : dataFiles) { Serialization.copyFile(dataFile, new File(dataDir, dataFile.getName())); } } @@ -146,7 +147,7 @@ public boolean accept(File dir, String name) { Transaction tx = channel.getTransaction(); tx.begin(); int i = 0; - while(channel.take() != null) { + while (channel.take() != null) { i++; } tx.commit(); @@ -161,7 +162,7 @@ public void doTestFixCorruptEvents(boolean withCheckpoint) throws Exception { File[] files = dataDir.listFiles(new FilenameFilter() { @Override public boolean accept(File dir, String name) { - if(name.contains("lock") || name.contains("meta")) { + if (name.contains("lock") || name.contains("meta")) { return false; } return true; @@ -170,15 +171,13 @@ public boolean accept(File dir, String name) { Random random = new Random(); int corrupted = 0; for (File dataFile : files) { - LogFile.SequentialReader reader = - new LogFileV3.SequentialReader(dataFile, null, true); + LogFile.SequentialReader reader = new LogFileV3.SequentialReader(dataFile, null, true); RandomAccessFile handle = new RandomAccessFile(dataFile, "rw"); long eventPosition1 = reader.getPosition(); LogRecord rec = reader.next(); //No point corrupting commits, so ignore them - if(rec == null || - rec.getEvent().getClass().getName(). - equals("org.apache.flume.channel.file.Commit")) { + if (rec == null || + rec.getEvent().getClass().getName().equals("org.apache.flume.channel.file.Commit")) { handle.close(); reader.close(); continue; @@ -190,8 +189,7 @@ public boolean accept(File dir, String name) { corrupted++; corruptFiles.add(dataFile.getName()); if (rec == null || - rec.getEvent().getClass().getName(). - equals("org.apache.flume.channel.file.Commit")) { + rec.getEvent().getClass().getName().equals("org.apache.flume.channel.file.Commit")) { handle.close(); reader.close(); continue; @@ -231,7 +229,7 @@ public boolean accept(File dir, String name) { Transaction tx = channel.getTransaction(); tx.begin(); int i = 0; - while(channel.take() != null) { + while (channel.take() != null) { i++; } tx.commit(); @@ -241,14 +239,14 @@ public boolean accept(File dir, String name) { files = dataDir.listFiles(new FilenameFilter() { @Override public boolean accept(File dir, String name) { - if(name.contains(".bak")) { + if (name.contains(".bak")) { return true; } return false; } }); Assert.assertEquals(corruptFiles.size(), files.length); - for(File file : files) { + for (File file : files) { String name = file.getName(); name = name.replaceAll(".bak", ""); Assert.assertTrue(corruptFiles.remove(name)); @@ -258,13 +256,13 @@ public boolean accept(File dir, String name) { private static void createDataFiles() throws Exception { final byte[] eventData = new byte[2000]; - for(int i = 0; i < 2000; i++) { + for (int i = 0; i < 2000; i++) { eventData[i] = 1; } WriteOrderOracle.setSeed(System.currentTimeMillis()); event = EventBuilder.withBody(eventData); baseDir = Files.createTempDir(); - if(baseDir.exists()) { + if (baseDir.exists()) { FileUtils.deleteDirectory(baseDir); } baseDir = Files.createTempDir(); @@ -286,7 +284,7 @@ private static void createDataFiles() throws Exception { Transaction tx = channel.getTransaction(); tx.begin(); for (int i = 0; i < 5; i++) { - if(i % 3 == 0) { + if (i % 3 == 0) { event.getBody()[0] = 0; invalidEvent++; } else { @@ -297,22 +295,19 @@ private static void createDataFiles() throws Exception { tx.commit(); tx.close(); } - Log log = field("log") - .ofType(Log.class) - .in(channel) - .get(); + Log log = field("log").ofType(Log.class) + .in(channel) + .get(); Assert.assertTrue("writeCheckpoint returned false", - method("writeCheckpoint") - .withReturnType(Boolean.class) - .withParameterTypes(Boolean.class) - .in(log) - .invoke(true)); + method("writeCheckpoint").withReturnType(Boolean.class) + .withParameterTypes(Boolean.class) + .in(log) + .invoke(true)); channel.stop(); } public static class DummyEventVerifier implements EventValidator { - private int value = 0; private DummyEventVerifier(int val) { @@ -325,7 +320,6 @@ public boolean validateEvent(Event event) { } public static class Builder implements EventValidator.Builder { - private int binaryValidator = 0; @Override diff --git a/pom.xml b/pom.xml index 85c0dc8a64..b50693ed90 100644 --- a/pom.xml +++ b/pom.xml @@ -672,9 +672,7 @@ limitations under the License. flume/checkstyle-suppressions.xml checkstyle.suppressions.file UTF-8 - true - true - false + true false @@ -1474,7 +1472,7 @@ limitations under the License. flume/checkstyle-suppressions.xml checkstyle.suppressions.file UTF-8 - false + true false From 7013708baddc8ed7d861797d1fd8280a94b6025c Mon Sep 17 00:00:00 2001 From: Mike Percy Date: Fri, 8 Jul 2016 17:32:09 -0700 Subject: [PATCH 298/341] FLUME-2939. Update recursive SpoolDir source to use Java 7 APIs MIME-Version: 1.0 Content-Type: text/plain; charset=UTF-8 Content-Transfer-Encoding: 8bit (Bessenyei Balázs Donát via Mike Percy) --- .../avro/ReliableSpoolingFileEventReader.java | 95 ++++++++++--------- 1 file changed, 49 insertions(+), 46 deletions(-) diff --git a/flume-ng-core/src/main/java/org/apache/flume/client/avro/ReliableSpoolingFileEventReader.java b/flume-ng-core/src/main/java/org/apache/flume/client/avro/ReliableSpoolingFileEventReader.java index 4dc0207723..ca5308c5af 100644 --- a/flume-ng-core/src/main/java/org/apache/flume/client/avro/ReliableSpoolingFileEventReader.java +++ b/flume-ng-core/src/main/java/org/apache/flume/client/avro/ReliableSpoolingFileEventReader.java @@ -23,7 +23,6 @@ import com.google.common.base.Charsets; import com.google.common.base.Optional; import com.google.common.base.Preconditions; -import com.google.common.io.Files; import org.apache.flume.Context; import org.apache.flume.Event; import org.apache.flume.FlumeException; @@ -43,10 +42,14 @@ import org.slf4j.LoggerFactory; import java.io.File; -import java.io.FileFilter; import java.io.FileNotFoundException; import java.io.IOException; import java.nio.charset.Charset; +import java.nio.file.FileVisitResult; +import java.nio.file.Files; +import java.nio.file.Path; +import java.nio.file.SimpleFileVisitor; +import java.nio.file.attribute.BasicFileAttributes; import java.util.ArrayList; import java.util.Collections; import java.util.Iterator; @@ -160,8 +163,8 @@ private ReliableSpoolingFileEventReader(File spoolDirectory, try { File canary = File.createTempFile("flume-spooldir-perm-check-", ".canary", spoolDirectory); - Files.write("testing flume file permissions\n", canary, Charsets.UTF_8); - List lines = Files.readLines(canary, Charsets.UTF_8); + Files.write(canary.toPath(), "testing flume file permissions\n".getBytes()); + List lines = Files.readAllLines(canary.toPath(), Charsets.UTF_8); Preconditions.checkState(!lines.isEmpty(), "Empty canary file %s", canary); if (!canary.delete()) { throw new IOException("Unable to delete canary file " + canary); @@ -215,50 +218,47 @@ private ReliableSpoolingFileEventReader(File spoolDirectory, } } - /** - * Filter to exclude files/directories either hidden, finished, or names matching the ignore pattern - */ - final FileFilter filter = new FileFilter() { - public boolean accept(File candidate) { - if (candidate.isDirectory()) { - String directoryName = candidate.getName(); - if (!recursiveDirectorySearch || - directoryName.startsWith(".") || - ignorePattern.matcher(directoryName).matches()) { - - return false; - } - return true; - } - String fileName = candidate.getName(); - if (fileName.endsWith(completedSuffix) || - fileName.startsWith(".") || - ignorePattern.matcher(fileName).matches()) { - return false; - } - - return true; - } - }; - /** * Recursively gather candidate files * @param directory the directory to gather files from * @return list of files within the passed in directory */ - private List getCandidateFiles(File directory) { + private List getCandidateFiles(final Path directory) { Preconditions.checkNotNull(directory); - List candidateFiles = new ArrayList(); - if (!directory.isDirectory()) { - return candidateFiles; - } + final List candidateFiles = new ArrayList<>(); + try { + Files.walkFileTree(directory, new SimpleFileVisitor() { + @Override + public FileVisitResult preVisitDirectory(Path dir, BasicFileAttributes attrs) + throws IOException { + if (directory.equals(dir)) { // The top directory should always be listed + return FileVisitResult.CONTINUE; + } + String directoryName = dir.getFileName().toString(); + if (!recursiveDirectorySearch || + directoryName.startsWith(".") || + ignorePattern.matcher(directoryName).matches()) { + return FileVisitResult.SKIP_SUBTREE; + } + return FileVisitResult.CONTINUE; + } - for (File file : directory.listFiles(filter)) { - if (file.isDirectory()) { - candidateFiles.addAll(getCandidateFiles(file)); - } else { - candidateFiles.add(file); - } + @Override + public FileVisitResult visitFile(Path candidate, BasicFileAttributes attrs) + throws IOException { + String fileName = candidate.getFileName().toString(); + if (!fileName.endsWith(completedSuffix) && + !fileName.startsWith(".") && + !ignorePattern.matcher(fileName).matches()) { + candidateFiles.add(candidate.toFile()); + } + + return FileVisitResult.CONTINUE; + } + }); + } catch (IOException e) { + logger.error("I/O exception occurred while listing directories. " + + "Files already matched will be returned. " + directory, e); } return candidateFiles; @@ -315,7 +315,8 @@ public List readEvents(int numEvents) throws IOException { * If so, try to roll to the next file, if there is one. * Loop until events is not empty or there is no next file in case of 0 byte files */ while (events.isEmpty()) { - logger.info("Last read took us just up to a file boundary. Rolling to the next file, if there is one."); + logger.info("Last read took us just up to a file boundary. " + + "Rolling to the next file, if there is one."); retireCurrentFile(); currentFile = getNextFile(); if (!currentFile.isPresent()) { @@ -417,7 +418,7 @@ private void rollCurrentFile(File fileToRoll) throws IOException { * file was already rolled but the rename was not atomic. If that seems * likely, we let it pass with only a warning. */ - if (Files.equal(currentFile.get().getFile(), dest)) { + if (Files.isSameFile(currentFile.get().getFile().toPath(), dest.toPath())) { logger.warn("Completed file " + dest + " already exists, but files match, so continuing."); boolean deleted = fileToRoll.delete(); @@ -494,7 +495,7 @@ private Optional getNextFile() { if (consumeOrder != ConsumeOrder.RANDOM || candidateFileIter == null || !candidateFileIter.hasNext()) { - candidateFiles = getCandidateFiles(spoolDirectory); + candidateFiles = getCandidateFiles(spoolDirectory.toPath()); listFilesCount++; candidateFileIter = candidateFiles.iterator(); } @@ -540,7 +541,7 @@ private File smallerLexicographical(File f1, File f2) { /** * Opens a file for consuming * @param file - * @return {@link #FileInfo} for the file to consume or absent option if the + * @return {@link FileInfo} for the file to consume or absent option if the * file does not exists or readable. */ private Optional openFile(File file) { @@ -584,7 +585,9 @@ private void deleteMetaFile() throws IOException { } } - /** An immutable class with information about a file being processed. */ + /** + * An immutable class with information about a file being processed. + */ private static class FileInfo { private final File file; private final long length; From 19664c3c559c7d0360f7bb5a64eff88b8f25a80d Mon Sep 17 00:00:00 2001 From: Niccolo Becchi Date: Fri, 8 Jul 2016 11:18:47 +0100 Subject: [PATCH 299/341] FLUME-2948. docs: Fix parameters on Replicating Channel Selector example This closes #55 (Niccolo Becchi via Mike Percy) --- flume-ng-doc/sphinx/FlumeUserGuide.rst | 6 +++--- 1 file changed, 3 insertions(+), 3 deletions(-) diff --git a/flume-ng-doc/sphinx/FlumeUserGuide.rst b/flume-ng-doc/sphinx/FlumeUserGuide.rst index c4d7c6c747..f9ca1b24d7 100644 --- a/flume-ng-doc/sphinx/FlumeUserGuide.rst +++ b/flume-ng-doc/sphinx/FlumeUserGuide.rst @@ -3010,9 +3010,9 @@ Example for agent named a1 and it's source called r1: a1.sources = r1 a1.channels = c1 c2 c3 - a1.source.r1.selector.type = replicating - a1.source.r1.channels = c1 c2 c3 - a1.source.r1.selector.optional = c3 + a1.sources.r1.selector.type = replicating + a1.sources.r1.channels = c1 c2 c3 + a1.sources.r1.selector.optional = c3 In the above configuration, c3 is an optional channel. Failure to write to c3 is simply ignored. Since c1 and c2 are not marked optional, failure to write to From 188c3104ab6030c40d652595a2274527a4ad4105 Mon Sep 17 00:00:00 2001 From: Mike Percy Date: Sat, 9 Jul 2016 14:54:07 -0700 Subject: [PATCH 300/341] FLUME-2949. checkstyle: Make suppressions compatible with Windows File paths in checkstyle-suppressions.xml are written with non-portable separators. Switching to "[/\\]" as a path separator should fix this issue. (Lior Zeno via Mike Percy) --- .../resources/flume/checkstyle-suppressions.xml | 14 +++++++++----- 1 file changed, 9 insertions(+), 5 deletions(-) diff --git a/flume-checkstyle/src/main/resources/flume/checkstyle-suppressions.xml b/flume-checkstyle/src/main/resources/flume/checkstyle-suppressions.xml index 2642baa873..78b9ee7aa4 100644 --- a/flume-checkstyle/src/main/resources/flume/checkstyle-suppressions.xml +++ b/flume-checkstyle/src/main/resources/flume/checkstyle-suppressions.xml @@ -2,15 +2,19 @@ + + + + files="generated-sources|com[/\\]cloudera[/\\]flume[/\\]handlers[/\\]thrift|org[/\\]apache[/\\]flume[/\\]thrift[/\\]|org[/\\]apache[/\\]flume[/\\]source[/\\]scribe|ProtosFactory.java"/> + files="org[/\\]apache[/\\]flume[/\\]source[/\\]avroLegacy|org[/\\]apache[/\\]flume[/\\]source[/\\]thriftLegacy"/> + files="channel[/\\]file|RpcClientFactory\.java|BucketPath\.java|SinkGroup\.java|DefaultSinkProcessor\.java|RegexExtractorInterceptorMillisSerializer\.java|SimpleAsyncHbaseEventSerializer\.java|hdfs[/\\]BucketWriter\.java|AbstractBasicChannelSemanticsTest\.java"/> + files="channel[/\\]file[/\\]LogFile\.java|TestDatasetSink\.java|CountingSourceRunner\.java|CountingSinkRunner\.java|TestKafkaChannel\.java|TestTaildirSource\.java|TestChannelProcessor\.java|TestHiveSink\.java|AbstractBasicChannelSemanticsTest\.java|TestJMSSource\.java|TestEmbeddedAgent\.java|TestAsyncHBaseSink\.java"/> + files="channel[/\\]MemoryChannel\.java|ReliableSpoolingFileEventReader\.java|TestAvroSink\.java"/> Date: Sat, 9 Jul 2016 15:05:33 -0700 Subject: [PATCH 301/341] FLUME-2936. KafkaSource tests: Use loopback instead of host address TestKafkaSource is flaky because it tries to run Kafka on the host address instead of the loopback address. This patch fixes that problem. It also introduces a bug fix related to passing properties to KafkaSourceEmbeddedKafka. (Lior Zeno via Mike Percy) --- .../source/kafka/KafkaSourceEmbeddedKafka.java | 13 ++----------- 1 file changed, 2 insertions(+), 11 deletions(-) diff --git a/flume-ng-sources/flume-kafka-source/src/test/java/org/apache/flume/source/kafka/KafkaSourceEmbeddedKafka.java b/flume-ng-sources/flume-kafka-source/src/test/java/org/apache/flume/source/kafka/KafkaSourceEmbeddedKafka.java index b72c532ee6..a3a2f9267b 100644 --- a/flume-ng-sources/flume-kafka-source/src/test/java/org/apache/flume/source/kafka/KafkaSourceEmbeddedKafka.java +++ b/flume-ng-sources/flume-kafka-source/src/test/java/org/apache/flume/source/kafka/KafkaSourceEmbeddedKafka.java @@ -30,22 +30,13 @@ import java.io.File; import java.io.IOException; import java.net.InetAddress; -import java.net.UnknownHostException; import java.util.Properties; import java.util.UUID; import java.util.concurrent.ExecutionException; public class KafkaSourceEmbeddedKafka { - public static String HOST; - - static { - try { - HOST = InetAddress.getLocalHost().getHostAddress(); - } catch (UnknownHostException e) { - throw new RuntimeException("Host address can not be obtained", e); - } - } + public static String HOST = InetAddress.getLoopbackAddress().getHostAddress(); KafkaServerStartable kafkaServer; KafkaSourceEmbeddedZookeeper zookeeper; @@ -71,7 +62,7 @@ public KafkaSourceEmbeddedKafka(Properties properties) { props.put("port", String.valueOf(serverPort)); props.put("log.dir", dir.getAbsolutePath()); if (properties != null) { - props.putAll(props); + props.putAll(properties); } KafkaConfig config = new KafkaConfig(props); kafkaServer = new KafkaServerStartable(config); From ba64b12670f7b2c9ce079f8898f4601689a51def Mon Sep 17 00:00:00 2001 From: Lior Zeno Date: Sun, 10 Jul 2016 15:03:08 -0700 Subject: [PATCH 302/341] FLUME-2950. ReliableSpoolingFileEventReader.rollCurrentFile is broken FLUME-2939 introduced a Java 7 API (nio.file) to ReliableSpoolingFileEventReader. However, the committed patch breaks rollCurrentFile() on Windows machines. File rename might be not atomic, in such cases, and under the assumption that the rolled file and the current file store the same data, Flume only logs a warning and does not assume that the user broke the contract that the spooling directory source demands. In order to determine whether the files store the same data, we used the com.google.common.io.Files.equal() method. However, FLUME-2939 uses java.nio.file.Files.isSameFile() which has different semantics. Consequently, this relaxation does not hold anymore. In addition, TestSpoolingFileLineReader.testDestinationExistsAndSameFileWindows fails. This patch brings back the old check using Files.equal(). (Lior Zeno via Mike Percy) --- .../flume/client/avro/ReliableSpoolingFileEventReader.java | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/flume-ng-core/src/main/java/org/apache/flume/client/avro/ReliableSpoolingFileEventReader.java b/flume-ng-core/src/main/java/org/apache/flume/client/avro/ReliableSpoolingFileEventReader.java index ca5308c5af..01381a520c 100644 --- a/flume-ng-core/src/main/java/org/apache/flume/client/avro/ReliableSpoolingFileEventReader.java +++ b/flume-ng-core/src/main/java/org/apache/flume/client/avro/ReliableSpoolingFileEventReader.java @@ -418,7 +418,7 @@ private void rollCurrentFile(File fileToRoll) throws IOException { * file was already rolled but the rename was not atomic. If that seems * likely, we let it pass with only a warning. */ - if (Files.isSameFile(currentFile.get().getFile().toPath(), dest.toPath())) { + if (com.google.common.io.Files.equal(currentFile.get().getFile(), dest)) { logger.warn("Completed file " + dest + " already exists, but files match, so continuing."); boolean deleted = fileToRoll.delete(); From ec28b66246f1f165ccaf01abf7fb27adebc9e4bb Mon Sep 17 00:00:00 2001 From: Denes Arvay Date: Wed, 13 Jul 2016 12:07:24 -0700 Subject: [PATCH 303/341] FLUME-2725. HDFS Sink does not use configured timezone for rounding (Denes Arvay via Mike Percy) --- .../flume/formatter/output/BucketPath.java | 10 +- .../flume/tools/TimestampRoundDownUtil.java | 72 +++++++++- .../formatter/output/TestBucketPath.java | 119 ++++++++++++++--- .../tools/TestTimestampRoundDownUtil.java | 124 ++++++++++++++---- 4 files changed, 275 insertions(+), 50 deletions(-) diff --git a/flume-ng-core/src/main/java/org/apache/flume/formatter/output/BucketPath.java b/flume-ng-core/src/main/java/org/apache/flume/formatter/output/BucketPath.java index b2fe3f0693..167b5425ea 100644 --- a/flume-ng-core/src/main/java/org/apache/flume/formatter/output/BucketPath.java +++ b/flume-ng-core/src/main/java/org/apache/flume/formatter/output/BucketPath.java @@ -236,7 +236,7 @@ protected static String replaceShorthand(char c, Map headers, } if (needRounding) { - ts = roundDown(roundDown, unit, ts); + ts = roundDown(roundDown, unit, ts, timeZone); } // It's a date @@ -329,7 +329,7 @@ protected static String replaceShorthand(char c, Map headers, return format.format(date); } - private static long roundDown(int roundDown, int unit, long ts) { + private static long roundDown(int roundDown, int unit, long ts, TimeZone timeZone) { long timestamp = ts; if (roundDown <= 0) { roundDown = 1; @@ -337,15 +337,15 @@ private static long roundDown(int roundDown, int unit, long ts) { switch (unit) { case Calendar.SECOND: timestamp = TimestampRoundDownUtil.roundDownTimeStampSeconds( - ts, roundDown); + ts, roundDown, timeZone); break; case Calendar.MINUTE: timestamp = TimestampRoundDownUtil.roundDownTimeStampMinutes( - ts, roundDown); + ts, roundDown, timeZone); break; case Calendar.HOUR_OF_DAY: timestamp = TimestampRoundDownUtil.roundDownTimeStampHours( - ts, roundDown); + ts, roundDown, timeZone); break; default: timestamp = ts; diff --git a/flume-ng-core/src/main/java/org/apache/flume/tools/TimestampRoundDownUtil.java b/flume-ng-core/src/main/java/org/apache/flume/tools/TimestampRoundDownUtil.java index daa9606ea9..6ce0fb93e5 100644 --- a/flume-ng-core/src/main/java/org/apache/flume/tools/TimestampRoundDownUtil.java +++ b/flume-ng-core/src/main/java/org/apache/flume/tools/TimestampRoundDownUtil.java @@ -19,25 +19,47 @@ package org.apache.flume.tools; import java.util.Calendar; +import java.util.TimeZone; import com.google.common.base.Preconditions; +import org.apache.flume.annotations.InterfaceAudience; +import org.apache.flume.annotations.InterfaceStability; +@InterfaceAudience.Private +@InterfaceStability.Evolving public class TimestampRoundDownUtil { /** * * @param timestamp - The time stamp to be rounded down. + * For parsing the timestamp the system default timezone will be used. * @param roundDownSec - The timestamp is rounded down to the largest * multiple of roundDownSec seconds * less than or equal to timestamp. Should be between 0 and 60. * @return - Rounded down timestamp * @throws IllegalStateException + * @see TimestampRoundDownUtil#roundDownTimeStampSeconds(long, int, TimeZone) */ public static long roundDownTimeStampSeconds(long timestamp, int roundDownSec) throws IllegalStateException { + return roundDownTimeStampSeconds(timestamp, roundDownSec, null); + } + + /** + * + * @param timestamp - The time stamp to be rounded down. + * @param roundDownSec - The timestamp is rounded down to the largest + * multiple of roundDownSec seconds + * less than or equal to timestamp. Should be between 0 and 60. + * @param timeZone - The timezone to use for parsing the timestamp. + * @return - Rounded down timestamp + * @throws IllegalStateException + */ + public static long roundDownTimeStampSeconds(long timestamp, int roundDownSec, TimeZone timeZone) + throws IllegalStateException { Preconditions.checkArgument(roundDownSec > 0 && roundDownSec <= 60, "RoundDownSec must be > 0 and <=60"); - Calendar cal = roundDownField(timestamp, Calendar.SECOND, roundDownSec); + Calendar cal = roundDownField(timestamp, Calendar.SECOND, roundDownSec, timeZone); cal.set(Calendar.MILLISECOND, 0); return cal.getTimeInMillis(); } @@ -45,17 +67,35 @@ public static long roundDownTimeStampSeconds(long timestamp, /** * * @param timestamp - The time stamp to be rounded down. + * For parsing the timestamp the system default timezone will be used. * @param roundDownMins - The timestamp is rounded down to the * largest multiple of roundDownMins minutes less than * or equal to timestamp. Should be between 0 and 60. * @return - Rounded down timestamp * @throws IllegalStateException + * @see TimestampRoundDownUtil#roundDownTimeStampMinutes(long, int, TimeZone) */ public static long roundDownTimeStampMinutes(long timestamp, int roundDownMins) throws IllegalStateException { + return roundDownTimeStampMinutes(timestamp, roundDownMins, null); + } + + /** + * + * @param timestamp - The time stamp to be rounded down. + * @param roundDownMins - The timestamp is rounded down to the + * largest multiple of roundDownMins minutes less than + * or equal to timestamp. Should be between 0 and 60. + * @param timeZone - The timezone to use for parsing the timestamp. + * If null the system default will be used. + * @return - Rounded down timestamp + * @throws IllegalStateException + */ + public static long roundDownTimeStampMinutes(long timestamp, int roundDownMins, TimeZone timeZone) + throws IllegalStateException { Preconditions.checkArgument(roundDownMins > 0 && roundDownMins <= 60, "RoundDown must be > 0 and <=60"); - Calendar cal = roundDownField(timestamp, Calendar.MINUTE, roundDownMins); + Calendar cal = roundDownField(timestamp, Calendar.MINUTE, roundDownMins, timeZone); cal.set(Calendar.SECOND, 0); cal.set(Calendar.MILLISECOND, 0); return cal.getTimeInMillis(); @@ -65,28 +105,48 @@ public static long roundDownTimeStampMinutes(long timestamp, /** * * @param timestamp - The time stamp to be rounded down. + * For parsing the timestamp the system default timezone will be used. * @param roundDownHours - The timestamp is rounded down to the * largest multiple of roundDownHours hours less than * or equal to timestamp. Should be between 0 and 24. * @return - Rounded down timestamp * @throws IllegalStateException + * @see TimestampRoundDownUtil#roundDownTimeStampHours(long, int, TimeZone) */ public static long roundDownTimeStampHours(long timestamp, int roundDownHours) throws IllegalStateException { + return roundDownTimeStampHours(timestamp, roundDownHours, null); + } + + /** + * + * @param timestamp - The time stamp to be rounded down. + * @param roundDownHours - The timestamp is rounded down to the + * largest multiple of roundDownHours hours less than + * or equal to timestamp. Should be between 0 and 24. + * @param timeZone - The timezone to use for parsing the timestamp. + * If null the system default will be used. + * @return - Rounded down timestamp + * @throws IllegalStateException + */ + public static long roundDownTimeStampHours(long timestamp, int roundDownHours, TimeZone timeZone) + throws IllegalStateException { Preconditions.checkArgument(roundDownHours > 0 && roundDownHours <= 24, "RoundDown must be > 0 and <=24"); - Calendar cal = roundDownField(timestamp, - Calendar.HOUR_OF_DAY, roundDownHours); + Calendar cal = roundDownField(timestamp, Calendar.HOUR_OF_DAY, roundDownHours, timeZone); cal.set(Calendar.MINUTE, 0); cal.set(Calendar.SECOND, 0); cal.set(Calendar.MILLISECOND, 0); return cal.getTimeInMillis(); } - private static Calendar roundDownField(long timestamp, int field, int roundDown) { + private static Calendar roundDownField(long timestamp, int field, int roundDown, + TimeZone timeZone) { Preconditions.checkArgument(timestamp > 0, "Timestamp must be positive"); - Calendar cal = Calendar.getInstance(); + + Calendar cal = (timeZone == null) ? Calendar.getInstance() : Calendar.getInstance(timeZone); cal.setTimeInMillis(timestamp); + int fieldVal = cal.get(field); int remainder = (fieldVal % roundDown); cal.set(field, fieldVal - remainder); diff --git a/flume-ng-core/src/test/java/org/apache/flume/formatter/output/TestBucketPath.java b/flume-ng-core/src/test/java/org/apache/flume/formatter/output/TestBucketPath.java index b1b828a0b3..ccc746066e 100644 --- a/flume-ng-core/src/test/java/org/apache/flume/formatter/output/TestBucketPath.java +++ b/flume-ng-core/src/test/java/org/apache/flume/formatter/output/TestBucketPath.java @@ -25,29 +25,40 @@ import org.junit.Before; import org.junit.Test; +import javax.annotation.Nullable; import java.text.SimpleDateFormat; import java.util.Calendar; import java.util.Date; import java.util.HashMap; import java.util.Map; +import java.util.SimpleTimeZone; import java.util.TimeZone; import static org.mockito.Mockito.mock; import static org.mockito.Mockito.when; public class TestBucketPath { - Calendar cal; - Map headers; + private static final TimeZone CUSTOM_TIMEZONE = new SimpleTimeZone(1, "custom-timezone"); + + private Calendar cal; + private Map headers; + private Map headersWithTimeZone; @Before public void setUp() { - cal = Calendar.getInstance(); - cal.set(2012, 5, 23, 13, 46, 33); - cal.set(Calendar.MILLISECOND, 234); - headers = new HashMap(); + cal = createCalendar(2012, 5, 23, 13, 46, 33, 234, null); + headers = new HashMap<>(); headers.put("timestamp", String.valueOf(cal.getTimeInMillis())); + + Calendar calWithTimeZone = createCalendar(2012, 5, 23, 13, 46, 33, 234, CUSTOM_TIMEZONE); + headersWithTimeZone = new HashMap<>(); + headersWithTimeZone.put("timestamp", String.valueOf(calWithTimeZone.getTimeInMillis())); } + /** + * Tests if the internally cached SimpleDateFormat instances can be reused with different + * TimeZone without interference. + */ @Test public void testDateFormatCache() { TimeZone utcTimeZone = TimeZone.getTimeZone("UTC"); @@ -64,15 +75,19 @@ public void testDateFormatCache() { Assert.assertEquals(expectedString, escapedString); } + /** + * Tests if the timestamp with the default timezone is properly rounded down + * to 12 hours using "%c" ("EEE MMM d HH:mm:ss yyyy") formatting. + */ @Test public void testDateFormatHours() { String test = "%c"; String escapedString = BucketPath.escapeString( test, headers, true, Calendar.HOUR_OF_DAY, 12); System.out.println("Escaped String: " + escapedString); - Calendar cal2 = Calendar.getInstance(); - cal2.set(2012, 5, 23, 12, 0, 0); - cal2.set(Calendar.MILLISECOND, 0); + + Calendar cal2 = createCalendar(2012, 5, 23, 12, 0, 0, 0, null); + SimpleDateFormat format = new SimpleDateFormat("EEE MMM d HH:mm:ss yyyy"); Date d = new Date(cal2.getTimeInMillis()); String expectedString = format.format(d); @@ -80,34 +95,99 @@ public void testDateFormatHours() { Assert.assertEquals(expectedString, escapedString); } + /** + * Tests if the timestamp with the custom timezone is properly rounded down + * to 12 hours using "%c" ("EEE MMM d HH:mm:ss yyyy") formatting. + */ + @Test + public void testDateFormatHoursTimeZone() { + String test = "%c"; + String escapedString = BucketPath.escapeString( + test, headersWithTimeZone, CUSTOM_TIMEZONE, true, Calendar.HOUR_OF_DAY, 12, false); + System.out.println("Escaped String: " + escapedString); + + Calendar cal2 = createCalendar(2012, 5, 23, 12, 0, 0, 0, CUSTOM_TIMEZONE); + + SimpleDateFormat format = new SimpleDateFormat("EEE MMM d HH:mm:ss yyyy"); + format.setTimeZone(CUSTOM_TIMEZONE); + + Date d = new Date(cal2.getTimeInMillis()); + String expectedString = format.format(d); + System.out.println("Expected String: " + expectedString); + Assert.assertEquals(expectedString, escapedString); + } + + /** + * Tests if the timestamp with the default timezone is properly rounded down + * to 5 minutes using "%s" (seconds) formatting + */ @Test public void testDateFormatMinutes() { String test = "%s"; String escapedString = BucketPath.escapeString( test, headers, true, Calendar.MINUTE, 5); System.out.println("Escaped String: " + escapedString); - Calendar cal2 = Calendar.getInstance(); - cal2.set(2012, 5, 23, 13, 45, 0); - cal2.set(Calendar.MILLISECOND, 0); + + Calendar cal2 = createCalendar(2012, 5, 23, 13, 45, 0, 0, null); + String expectedString = String.valueOf(cal2.getTimeInMillis() / 1000); + System.out.println("Expected String: " + expectedString); + Assert.assertEquals(expectedString, escapedString); + } + + /** + * Tests if the timestamp with the custom timezone is properly rounded down + * to 5 minutes using "%s" (seconds) formatting + */ + @Test + public void testDateFormatMinutesTimeZone() { + String test = "%s"; + String escapedString = BucketPath.escapeString( + test, headersWithTimeZone, CUSTOM_TIMEZONE, true, Calendar.MINUTE, 5, false); + System.out.println("Escaped String: " + escapedString); + + Calendar cal2 = createCalendar(2012, 5, 23, 13, 45, 0, 0, CUSTOM_TIMEZONE); String expectedString = String.valueOf(cal2.getTimeInMillis() / 1000); System.out.println("Expected String: " + expectedString); Assert.assertEquals(expectedString, escapedString); } + /** + * Tests if the timestamp with the default timezone is properly rounded down + * to 5 seconds using "%s" (seconds) formatting + */ @Test public void testDateFormatSeconds() { String test = "%s"; String escapedString = BucketPath.escapeString( test, headers, true, Calendar.SECOND, 5); System.out.println("Escaped String: " + escapedString); - Calendar cal2 = Calendar.getInstance(); - cal2.set(2012, 5, 23, 13, 46, 30); - cal2.set(Calendar.MILLISECOND, 0); + + Calendar cal2 = createCalendar(2012, 5, 23, 13, 46, 30, 0, null); String expectedString = String.valueOf(cal2.getTimeInMillis() / 1000); System.out.println("Expected String: " + expectedString); Assert.assertEquals(expectedString, escapedString); } + /** + * Tests if the timestamp with the custom timezone is properly rounded down + * to 5 seconds using "%s" (seconds) formatting + */ + @Test + public void testDateFormatSecondsTimeZone() { + String test = "%s"; + String escapedString = BucketPath.escapeString( + test, headersWithTimeZone, CUSTOM_TIMEZONE, true, Calendar.SECOND, 5, false); + System.out.println("Escaped String: " + escapedString); + + Calendar cal2 = createCalendar(2012, 5, 23, 13, 46, 30, 0, CUSTOM_TIMEZONE); + String expectedString = String.valueOf(cal2.getTimeInMillis() / 1000); + System.out.println("Expected String: " + expectedString); + Assert.assertEquals(expectedString, escapedString); + } + + /** + * Tests if the timestamp is properly formatted without rounding it down. + */ @Test public void testNoRounding() { String test = "%c"; @@ -189,4 +269,13 @@ public void testDateRace() { Assert.assertEquals("Race condition detected", "02:50", escaped); } + + private static Calendar createCalendar(int year, int month, int day, + int hour, int minute, int second, int ms, + @Nullable TimeZone timeZone) { + Calendar cal = (timeZone == null) ? Calendar.getInstance() : Calendar.getInstance(timeZone); + cal.set(year, month, day, hour, minute, second); + cal.set(Calendar.MILLISECOND, ms); + return cal; + } } diff --git a/flume-ng-core/src/test/java/org/apache/flume/tools/TestTimestampRoundDownUtil.java b/flume-ng-core/src/test/java/org/apache/flume/tools/TestTimestampRoundDownUtil.java index 1ac11ab05f..384a9dc7fc 100644 --- a/flume-ng-core/src/test/java/org/apache/flume/tools/TestTimestampRoundDownUtil.java +++ b/flume-ng-core/src/test/java/org/apache/flume/tools/TestTimestampRoundDownUtil.java @@ -19,23 +19,36 @@ package org.apache.flume.tools; import java.util.Calendar; +import java.util.SimpleTimeZone; +import java.util.TimeZone; import junit.framework.Assert; import org.junit.Test; +import javax.annotation.Nullable; + +import static org.hamcrest.core.IsEqual.equalTo; +import static org.hamcrest.core.IsNot.not; +import static org.junit.Assert.assertThat; + public class TestTimestampRoundDownUtil { + private static final TimeZone CUSTOM_TIMEZONE = new SimpleTimeZone(1, "custom-timezone"); + private static final Calendar BASE_CALENDAR_WITH_DEFAULT_TIMEZONE = + createCalendar(2012, 5, 15, 15, 12, 54, 0, null); + private static final Calendar BASE_CALENDAR_WITH_CUSTOM_TIMEZONE = + createCalendar(2012, 5, 15, 15, 12, 54, 0, CUSTOM_TIMEZONE); + + /** + * Tests if the timestamp with the default timezone is properly rounded down + * to 60 seconds. + */ @Test public void testRoundDownTimeStampSeconds() { - Calendar cal = Calendar.getInstance(); - cal.clear(); - cal.set(2012, 5, 15, 15, 12, 54); - cal.set(Calendar.MILLISECOND, 20); - Calendar cal2 = Calendar.getInstance(); - cal2.clear(); - cal2.set(2012, 5, 15, 15, 12, 0); - cal2.set(Calendar.MILLISECOND, 0); + Calendar cal = BASE_CALENDAR_WITH_DEFAULT_TIMEZONE; + Calendar cal2 = createCalendar(2012, 5, 15, 15, 12, 0, 0, null); + long timeToVerify = cal2.getTimeInMillis(); long ret = TimestampRoundDownUtil.roundDownTimeStampSeconds(cal.getTimeInMillis(), 60); System.out.println("Cal 1: " + cal.toString()); @@ -43,16 +56,34 @@ public void testRoundDownTimeStampSeconds() { Assert.assertEquals(timeToVerify, ret); } + /** + * Tests if the timestamp with the custom timezone is properly rounded down + * to 60 seconds. + */ + @Test + public void testRoundDownTimeStampSecondsWithTimeZone() { + Calendar cal = BASE_CALENDAR_WITH_CUSTOM_TIMEZONE; + Calendar cal2 = createCalendar(2012, 5, 15, 15, 12, 0, 0, CUSTOM_TIMEZONE); + + long timeToVerify = cal2.getTimeInMillis(); + long withoutTimeZone = TimestampRoundDownUtil.roundDownTimeStampSeconds( + cal.getTimeInMillis(), 60); + long withTimeZone = TimestampRoundDownUtil.roundDownTimeStampSeconds( + cal.getTimeInMillis(), 60, CUSTOM_TIMEZONE); + + assertThat(withoutTimeZone, not(equalTo(timeToVerify))); + Assert.assertEquals(withTimeZone, timeToVerify); + } + + /** + * Tests if the timestamp with the default timezone is properly rounded down + * to 5 minutes. + */ @Test public void testRoundDownTimeStampMinutes() { - Calendar cal = Calendar.getInstance(); - cal.clear(); - cal.set(2012, 5, 15, 15, 12, 54); - cal.set(Calendar.MILLISECOND, 20); - Calendar cal2 = Calendar.getInstance(); - cal2.clear(); - cal2.set(2012, 5, 15, 15, 10, 0); - cal2.set(Calendar.MILLISECOND, 0); + Calendar cal = BASE_CALENDAR_WITH_DEFAULT_TIMEZONE; + Calendar cal2 = createCalendar(2012, 5, 15, 15, 10, 0, 0, null); + long timeToVerify = cal2.getTimeInMillis(); long ret = TimestampRoundDownUtil.roundDownTimeStampMinutes(cal.getTimeInMillis(), 5); System.out.println("Cal 1: " + cal.toString()); @@ -60,16 +91,34 @@ public void testRoundDownTimeStampMinutes() { Assert.assertEquals(timeToVerify, ret); } + /** + * Tests if the timestamp with the custom timezone is properly rounded down + * to 5 minutes. + */ + @Test + public void testRoundDownTimeStampMinutesWithTimeZone() { + Calendar cal = BASE_CALENDAR_WITH_CUSTOM_TIMEZONE; + Calendar cal2 = createCalendar(2012, 5, 15, 15, 10, 0, 0, CUSTOM_TIMEZONE); + + long timeToVerify = cal2.getTimeInMillis(); + long withoutTimeZone = TimestampRoundDownUtil.roundDownTimeStampMinutes( + cal.getTimeInMillis(), 5); + long withTimeZone = TimestampRoundDownUtil.roundDownTimeStampMinutes( + cal.getTimeInMillis(), 5, CUSTOM_TIMEZONE); + + assertThat(withoutTimeZone, not(equalTo(timeToVerify))); + Assert.assertEquals(withTimeZone, timeToVerify); + } + + /** + * Tests if the timestamp with the default timezone is properly rounded down + * to 2 hours. + */ @Test public void testRoundDownTimeStampHours() { - Calendar cal = Calendar.getInstance(); - cal.clear(); - cal.set(2012, 5, 15, 15, 12, 54); - cal.set(Calendar.MILLISECOND, 20); - Calendar cal2 = Calendar.getInstance(); - cal2.clear(); - cal2.set(2012, 5, 15, 14, 0, 0); - cal2.set(Calendar.MILLISECOND, 0); + Calendar cal = BASE_CALENDAR_WITH_DEFAULT_TIMEZONE; + Calendar cal2 = createCalendar(2012, 5, 15, 14, 0, 0, 0, null); + long timeToVerify = cal2.getTimeInMillis(); long ret = TimestampRoundDownUtil.roundDownTimeStampHours(cal.getTimeInMillis(), 2); System.out.println("Cal 1: " + ret); @@ -77,4 +126,31 @@ public void testRoundDownTimeStampHours() { Assert.assertEquals(timeToVerify, ret); } + /** + * Tests if the timestamp with the custom timezone is properly rounded down + * to 2 hours. + */ + @Test + public void testRoundDownTimeStampHoursWithTimeZone() { + Calendar cal = BASE_CALENDAR_WITH_CUSTOM_TIMEZONE; + Calendar cal2 = createCalendar(2012, 5, 15, 14, 0, 0, 0, CUSTOM_TIMEZONE); + + long timeToVerify = cal2.getTimeInMillis(); + long withoutTimeZone = TimestampRoundDownUtil.roundDownTimeStampHours( + cal.getTimeInMillis(), 2); + long withTimeZone = TimestampRoundDownUtil.roundDownTimeStampHours( + cal.getTimeInMillis(), 2, CUSTOM_TIMEZONE); + + assertThat(withoutTimeZone, not(equalTo(timeToVerify))); + Assert.assertEquals(withTimeZone, timeToVerify); + } + + private static Calendar createCalendar(int year, int month, int day, + int hour, int minute, int second, int ms, + @Nullable TimeZone timeZone) { + Calendar cal = (timeZone == null) ? Calendar.getInstance() : Calendar.getInstance(timeZone); + cal.set(year, month, day, hour, minute, second); + cal.set(Calendar.MILLISECOND, ms); + return cal; + } } From c7de4ba5ca70c42099a2f9bf4ad2de3fdcbb7d30 Mon Sep 17 00:00:00 2001 From: Mike Percy Date: Mon, 11 Jul 2016 19:53:50 -0700 Subject: [PATCH 304/341] FLUME-2952. SyslogAgent: Avoid NPE on stop() Minor test bug fix Reviewed by Hari. --- .../src/test/java/org/apache/flume/test/util/SyslogAgent.java | 4 +++- 1 file changed, 3 insertions(+), 1 deletion(-) diff --git a/flume-ng-tests/src/test/java/org/apache/flume/test/util/SyslogAgent.java b/flume-ng-tests/src/test/java/org/apache/flume/test/util/SyslogAgent.java index c908fc167d..a2b0f3bdcb 100644 --- a/flume-ng-tests/src/test/java/org/apache/flume/test/util/SyslogAgent.java +++ b/flume-ng-tests/src/test/java/org/apache/flume/test/util/SyslogAgent.java @@ -156,7 +156,9 @@ public boolean isRunning() throws Exception { } public void stop() throws Exception { - client.close(); + if (client != null) { + client.close(); + } client = null; StagedInstall.getInstance().stopAgent(); From dbf2e989744a6b2921076be017102f75323a69f4 Mon Sep 17 00:00:00 2001 From: Jeff Holoman Date: Tue, 19 Jul 2016 12:29:08 -0700 Subject: [PATCH 305/341] FLUME-2810. Add static Schema URL to AvroEventSerializer configuration Currently the only way to pass a schema to the avro event serializer is via header. This introduces an option to specify the location directly in the Flume configuration. (Jeff Holoman via Mike Percy) --- ...EventSerializerConfigurationConstants.java | 5 ++ flume-ng-doc/sphinx/FlumeUserGuide.rst | 51 +++++++++++++++++-- .../flume/sink/hdfs/AvroEventSerializer.java | 45 ++++++++++------ .../sink/hdfs/TestAvroEventSerializer.java | 47 +++++++++++++---- 4 files changed, 117 insertions(+), 31 deletions(-) diff --git a/flume-ng-core/src/main/java/org/apache/flume/serialization/AvroEventSerializerConfigurationConstants.java b/flume-ng-core/src/main/java/org/apache/flume/serialization/AvroEventSerializerConfigurationConstants.java index cce67166f2..470fcea7b0 100644 --- a/flume-ng-core/src/main/java/org/apache/flume/serialization/AvroEventSerializerConfigurationConstants.java +++ b/flume-ng-core/src/main/java/org/apache/flume/serialization/AvroEventSerializerConfigurationConstants.java @@ -35,4 +35,9 @@ public class AvroEventSerializerConfigurationConstants { public static final String COMPRESSION_CODEC = "compressionCodec"; public static final String DEFAULT_COMPRESSION_CODEC = "null"; // no codec + /** + * Avro static Schema URL + */ + public static final String STATIC_SCHEMA_URL = "schemaURL"; + public static final String DEFAULT_STATIC_SCHEMA_URL = null; } diff --git a/flume-ng-doc/sphinx/FlumeUserGuide.rst b/flume-ng-doc/sphinx/FlumeUserGuide.rst index f9ca1b24d7..3937514a71 100644 --- a/flume-ng-doc/sphinx/FlumeUserGuide.rst +++ b/flume-ng-doc/sphinx/FlumeUserGuide.rst @@ -3235,19 +3235,59 @@ Example for agent named a1: a1.sinks.k1.sink.serializer = text a1.sinks.k1.sink.serializer.appendNewline = false +"Flume Event" Avro Event Serializer +~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~ + +Alias: ``avro_event``. + +This interceptor serializes Flume events into an Avro container file. The schema used is the same schema used for +Flume events in the Avro RPC mechanism. + +This serializer inherits from the ``AbstractAvroEventSerializer`` class. + +Configuration options are as follows: + +========================== ================ =========================================================================== +Property Name Default Description +========================== ================ =========================================================================== +syncIntervalBytes 2048000 Avro sync interval, in approximate bytes. +compressionCodec null Avro compression codec. For supported codecs, see Avro's CodecFactory docs. +========================== ================ =========================================================================== + +Example for agent named a1: + +.. code-block:: properties + + a1.sinks.k1.type = hdfs + a1.sinks.k1.channel = c1 + a1.sinks.k1.hdfs.path = /flume/events/%y-%m-%d/%H%M/%S + a1.sinks.k1.serializer = avro_event + a1.sinks.k1.serializer.compressionCodec = snappy + Avro Event Serializer ~~~~~~~~~~~~~~~~~~~~~ -Alias: ``avro_event``. This interceptor serializes Flume events into an Avro -container file. The schema used is the same schema used for Flume events -in the Avro RPC mechanism. This serializers inherits from the -``AbstractAvroEventSerializer`` class. Configuration options are as follows: +Alias: This serializer does not have an alias, and must be specified using the fully-qualified class name class name. + +This serializes Flume events into an Avro container file like the "Flume Event" Avro Event Serializer, however the +record schema is configurable. The record schema may be specified either as a Flume configuration property or passed in an event header. + +To pass the record schema as part of the Flume configuration, use the property ``schemaURL`` as listed below. + +To pass the record schema in an event header, specify either the event header ``flume.avro.schema.literal`` +containing a JSON-format representation of the schema or ``flume.avro.schema.url`` with a URL where +the schema may be found (``hdfs:/...`` URIs are supported). + +This serializer inherits from the ``AbstractAvroEventSerializer`` class. + +Configuration options are as follows: ========================== ================ =========================================================================== Property Name Default Description ========================== ================ =========================================================================== syncIntervalBytes 2048000 Avro sync interval, in approximate bytes. compressionCodec null Avro compression codec. For supported codecs, see Avro's CodecFactory docs. +schemaURL null Avro schema URL. Schemas specified in the header ovverride this option. ========================== ================ =========================================================================== Example for agent named a1: @@ -3257,8 +3297,9 @@ Example for agent named a1: a1.sinks.k1.type = hdfs a1.sinks.k1.channel = c1 a1.sinks.k1.hdfs.path = /flume/events/%y-%m-%d/%H%M/%S - a1.sinks.k1.serializer = avro_event + a1.sinks.k1.serializer = org.apache.flume.sink.hdfs.AvroEventSerializer$Builder a1.sinks.k1.serializer.compressionCodec = snappy + a1.sinks.k1.serializer.schemaURL = hdfs://namenode/path/to/schema.avsc Flume Interceptors diff --git a/flume-ng-sinks/flume-hdfs-sink/src/main/java/org/apache/flume/sink/hdfs/AvroEventSerializer.java b/flume-ng-sinks/flume-hdfs-sink/src/main/java/org/apache/flume/sink/hdfs/AvroEventSerializer.java index fea62188a9..3231742404 100644 --- a/flume-ng-sinks/flume-hdfs-sink/src/main/java/org/apache/flume/sink/hdfs/AvroEventSerializer.java +++ b/flume-ng-sinks/flume-hdfs-sink/src/main/java/org/apache/flume/sink/hdfs/AvroEventSerializer.java @@ -18,14 +18,6 @@ */ package org.apache.flume.sink.hdfs; -import java.io.IOException; -import java.io.InputStream; -import java.io.OutputStream; -import java.net.URL; -import java.nio.ByteBuffer; -import java.util.HashMap; -import java.util.Locale; -import java.util.Map; import org.apache.avro.AvroRuntimeException; import org.apache.avro.Schema; import org.apache.avro.file.CodecFactory; @@ -44,7 +36,21 @@ import org.slf4j.Logger; import org.slf4j.LoggerFactory; -import static org.apache.flume.serialization.AvroEventSerializerConfigurationConstants.*; +import java.io.IOException; +import java.io.InputStream; +import java.io.OutputStream; +import java.net.URL; +import java.nio.ByteBuffer; +import java.util.HashMap; +import java.util.Locale; +import java.util.Map; + +import static org.apache.flume.serialization.AvroEventSerializerConfigurationConstants.COMPRESSION_CODEC; +import static org.apache.flume.serialization.AvroEventSerializerConfigurationConstants.DEFAULT_COMPRESSION_CODEC; +import static org.apache.flume.serialization.AvroEventSerializerConfigurationConstants.DEFAULT_STATIC_SCHEMA_URL; +import static org.apache.flume.serialization.AvroEventSerializerConfigurationConstants.DEFAULT_SYNC_INTERVAL_BYTES; +import static org.apache.flume.serialization.AvroEventSerializerConfigurationConstants.STATIC_SCHEMA_URL; +import static org.apache.flume.serialization.AvroEventSerializerConfigurationConstants.SYNC_INTERVAL_BYTES; /** *

    @@ -76,6 +82,7 @@ public class AvroEventSerializer implements EventSerializer, Configurable { private int syncIntervalBytes; private String compressionCodec; private Map schemaCache = new HashMap(); + private String staticSchemaURL; private AvroEventSerializer(OutputStream out) { this.out = out; @@ -87,6 +94,7 @@ public void configure(Context context) { context.getInteger(SYNC_INTERVAL_BYTES, DEFAULT_SYNC_INTERVAL_BYTES); compressionCodec = context.getString(COMPRESSION_CODEC, DEFAULT_COMPRESSION_CODEC); + staticSchemaURL = context.getString(STATIC_SCHEMA_URL, DEFAULT_STATIC_SCHEMA_URL); } @Override @@ -111,19 +119,24 @@ public void write(Event event) throws IOException { private void initialize(Event event) throws IOException { Schema schema = null; String schemaUrl = event.getHeaders().get(AVRO_SCHEMA_URL_HEADER); - if (schemaUrl != null) { + String schemaString = event.getHeaders().get(AVRO_SCHEMA_LITERAL_HEADER); + + if (schemaUrl != null) { // if URL_HEADER is there then use it schema = schemaCache.get(schemaUrl); if (schema == null) { schema = loadFromUrl(schemaUrl); schemaCache.put(schemaUrl, schema); } - } - if (schema == null) { - String schemaString = event.getHeaders().get(AVRO_SCHEMA_LITERAL_HEADER); - if (schemaString == null) { - throw new FlumeException("Could not find schema for event " + event); - } + } else if (schemaString != null) { // fallback to LITERAL_HEADER if it was there schema = new Schema.Parser().parse(schemaString); + } else if (staticSchemaURL != null) { // fallback to static url if it was there + schema = schemaCache.get(staticSchemaURL); + if (schema == null) { + schema = loadFromUrl(staticSchemaURL); + schemaCache.put(staticSchemaURL, schema); + } + } else { // no other options so giving up + throw new FlumeException("Could not find schema for event " + event); } writer = new GenericDatumWriter(schema); diff --git a/flume-ng-sinks/flume-hdfs-sink/src/test/java/org/apache/flume/sink/hdfs/TestAvroEventSerializer.java b/flume-ng-sinks/flume-hdfs-sink/src/test/java/org/apache/flume/sink/hdfs/TestAvroEventSerializer.java index 38af74d1d1..6b38da2a0d 100644 --- a/flume-ng-sinks/flume-hdfs-sink/src/test/java/org/apache/flume/sink/hdfs/TestAvroEventSerializer.java +++ b/flume-ng-sinks/flume-hdfs-sink/src/test/java/org/apache/flume/sink/hdfs/TestAvroEventSerializer.java @@ -39,10 +39,12 @@ import org.apache.flume.Context; import org.apache.flume.Event; import org.apache.flume.event.EventBuilder; +import org.apache.flume.serialization.AvroEventSerializerConfigurationConstants; import org.apache.flume.serialization.EventSerializer; import org.junit.Assert; import org.junit.Before; import org.junit.Test; +import org.junit.After; public class TestAvroEventSerializer { @@ -53,39 +55,55 @@ public void setUp() throws Exception { file = File.createTempFile(getClass().getSimpleName(), ""); } + @After + public void tearDown() throws Exception { + file.delete(); + } + @Test public void testNoCompression() throws IOException { - createAvroFile(file, null, false); + createAvroFile(file, null, false, false); validateAvroFile(file); } @Test public void testNullCompression() throws IOException { - createAvroFile(file, "null", false); + createAvroFile(file, "null", false, false); validateAvroFile(file); } @Test public void testDeflateCompression() throws IOException { - createAvroFile(file, "deflate", false); + createAvroFile(file, "deflate", false, false); validateAvroFile(file); } @Test public void testSnappyCompression() throws IOException { - createAvroFile(file, "snappy", false); + createAvroFile(file, "snappy", false, false); validateAvroFile(file); } @Test public void testSchemaUrl() throws IOException { - createAvroFile(file, null, true); + createAvroFile(file, null, true, false); validateAvroFile(file); } - public void createAvroFile(File file, String codec, boolean useSchemaUrl) throws - IOException { + @Test + public void testStaticSchemaUrl() throws IOException { + createAvroFile(file,null,false, true); + validateAvroFile(file); + } + @Test + public void testBothUrls() throws IOException { + createAvroFile(file,null,true,true); + validateAvroFile(file); + } + + public void createAvroFile(File file, String codec, boolean useSchemaUrl, + boolean useStaticSchemaUrl) throws IOException { // serialize a few events using the reflection-based avro serializer OutputStream out = new FileOutputStream(file); @@ -100,11 +118,16 @@ public void createAvroFile(File file, String codec, boolean useSchemaUrl) throws })); GenericRecordBuilder recordBuilder = new GenericRecordBuilder(schema); File schemaFile = null; - if (useSchemaUrl) { + if (useSchemaUrl || useStaticSchemaUrl) { schemaFile = File.createTempFile(getClass().getSimpleName(), ".avsc"); Files.write(schema.toString(), schemaFile, Charsets.UTF_8); } + if (useStaticSchemaUrl) { + ctx.put(AvroEventSerializerConfigurationConstants.STATIC_SCHEMA_URL, + schemaFile.toURI().toURL().toExternalForm()); + } + EventSerializer.Builder builder = new AvroEventSerializer.Builder(); EventSerializer serializer = builder.build(ctx, out); @@ -112,10 +135,10 @@ public void createAvroFile(File file, String codec, boolean useSchemaUrl) throws for (int i = 0; i < 3; i++) { GenericRecord record = recordBuilder.set("message", "Hello " + i).build(); Event event = EventBuilder.withBody(serializeAvro(record, schema)); - if (schemaFile == null) { + if (schemaFile == null && !useSchemaUrl) { event.getHeaders().put(AvroEventSerializer.AVRO_SCHEMA_LITERAL_HEADER, schema.toString()); - } else { + } else if (useSchemaUrl) { event.getHeaders().put(AvroEventSerializer.AVRO_SCHEMA_URL_HEADER, schemaFile.toURI().toURL().toExternalForm()); } @@ -125,6 +148,10 @@ public void createAvroFile(File file, String codec, boolean useSchemaUrl) throws serializer.beforeClose(); out.flush(); out.close(); + if (schemaFile != null ) { + schemaFile.delete(); + } + } private byte[] serializeAvro(Object datum, Schema schema) throws IOException { From 0259d302730f94b99c1a84f8311d1abc0f635eb3 Mon Sep 17 00:00:00 2001 From: Mike Percy Date: Tue, 19 Jul 2016 12:42:10 -0700 Subject: [PATCH 306/341] Fix broken link in README --- README.md | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/README.md b/README.md index 7198a65bc6..9ebb2a3830 100644 --- a/README.md +++ b/README.md @@ -39,7 +39,7 @@ In source form, it can be found in the flume-ng-doc directory. The Flume 1.x guide and FAQ are available here: -* https://cwiki.apache.org/FLUME/flume-ng.html +* https://cwiki.apache.org/FLUME * https://cwiki.apache.org/confluence/display/FLUME/Getting+Started ## Contact us! From 9965dae7bd384abd2e25fd1756dbe516e8acdb9b Mon Sep 17 00:00:00 2001 From: Mike Percy Date: Tue, 19 Jul 2016 13:33:58 -0700 Subject: [PATCH 307/341] Fix sphinx layout errors Minor syntax error fixes --- flume-ng-doc/sphinx/FlumeUserGuide.rst | 92 +++++++++++++------------- 1 file changed, 46 insertions(+), 46 deletions(-) diff --git a/flume-ng-doc/sphinx/FlumeUserGuide.rst b/flume-ng-doc/sphinx/FlumeUserGuide.rst index 3937514a71..d8bfebfe61 100644 --- a/flume-ng-doc/sphinx/FlumeUserGuide.rst +++ b/flume-ng-doc/sphinx/FlumeUserGuide.rst @@ -976,53 +976,53 @@ Despite the reliability guarantees of this source, there are still cases in which events may be duplicated if certain downstream failures occur. This is consistent with the guarantees offered by other Flume components. -====================== ============== ========================================================== -Property Name Default Description -====================== ============== ========================================================== -**channels** -- -**type** -- The component type name, needs to be ``spooldir``. -**spoolDir** -- The directory from which to read files from. -fileSuffix .COMPLETED Suffix to append to completely ingested files -deletePolicy never When to delete completed files: ``never`` or ``immediate`` -fileHeader false Whether to add a header storing the absolute path filename. -fileHeaderKey file Header key to use when appending absolute path filename to event header. -basenameHeader false Whether to add a header storing the basename of the file. -basenameHeaderKey basename Header Key to use when appending basename of file to event header. -ignorePattern ^$ Regular expression specifying which files to ignore (skip) -trackerDir .flumespool Directory to store metadata related to processing of files. - If this path is not an absolute path, then it is interpreted as relative to the spoolDir. -consumeOrder oldest In which order files in the spooling directory will be consumed ``oldest``, - ``youngest`` and ``random``. In case of ``oldest`` and ``youngest``, the last modified - time of the files will be used to compare the files. In case of a tie, the file - with smallest lexicographical order will be consumed first. In case of ``random`` any - file will be picked randomly. When using ``oldest`` and ``youngest`` the whole - directory will be scanned to pick the oldest/youngest file, which might be slow if there - are a large number of files, while using ``random`` may cause old files to be consumed - very late if new files keep coming in the spooling directory. -pollDelay 500 Delay (in milliseconds) used when polling for new files. -recursiveDirectorySearch false Whether to monitor sub directories for new files to read. -maxBackoff 4000 The maximum time (in millis) to wait between consecutive attempts to - write to the channel(s) if the channel is full. The source will start at - a low backoff and increase it exponentially each time the channel throws a - ChannelException, upto the value specified by this parameter. -batchSize 100 Granularity at which to batch transfer to the channel -inputCharset UTF-8 Character set used by deserializers that treat the input file as text. -decodeErrorPolicy ``FAIL`` What to do when we see a non-decodable character in the input file. - ``FAIL``: Throw an exception and fail to parse the file. - ``REPLACE``: Replace the unparseable character with the "replacement character" char, - typically Unicode U+FFFD. - ``IGNORE``: Drop the unparseable character sequence. -deserializer ``LINE`` Specify the deserializer used to parse the file into events. - Defaults to parsing each line as an event. The class specified must implement - ``EventDeserializer.Builder``. -deserializer.* Varies per event deserializer. -bufferMaxLines -- (Obselete) This option is now ignored. -bufferMaxLineLength 5000 (Deprecated) Maximum length of a line in the commit buffer. Use deserializer.maxLineLength instead. -selector.type replicating replicating or multiplexing -selector.* Depends on the selector.type value -interceptors -- Space-separated list of interceptors +======================== ============== ========================================================== +Property Name Default Description +======================== ============== ========================================================== +**channels** -- +**type** -- The component type name, needs to be ``spooldir``. +**spoolDir** -- The directory from which to read files from. +fileSuffix .COMPLETED Suffix to append to completely ingested files +deletePolicy never When to delete completed files: ``never`` or ``immediate`` +fileHeader false Whether to add a header storing the absolute path filename. +fileHeaderKey file Header key to use when appending absolute path filename to event header. +basenameHeader false Whether to add a header storing the basename of the file. +basenameHeaderKey basename Header Key to use when appending basename of file to event header. +ignorePattern ^$ Regular expression specifying which files to ignore (skip) +trackerDir .flumespool Directory to store metadata related to processing of files. + If this path is not an absolute path, then it is interpreted as relative to the spoolDir. +consumeOrder oldest In which order files in the spooling directory will be consumed ``oldest``, + ``youngest`` and ``random``. In case of ``oldest`` and ``youngest``, the last modified + time of the files will be used to compare the files. In case of a tie, the file + with smallest lexicographical order will be consumed first. In case of ``random`` any + file will be picked randomly. When using ``oldest`` and ``youngest`` the whole + directory will be scanned to pick the oldest/youngest file, which might be slow if there + are a large number of files, while using ``random`` may cause old files to be consumed + very late if new files keep coming in the spooling directory. +pollDelay 500 Delay (in milliseconds) used when polling for new files. +recursiveDirectorySearch false Whether to monitor sub directories for new files to read. +maxBackoff 4000 The maximum time (in millis) to wait between consecutive attempts to + write to the channel(s) if the channel is full. The source will start at + a low backoff and increase it exponentially each time the channel throws a + ChannelException, upto the value specified by this parameter. +batchSize 100 Granularity at which to batch transfer to the channel +inputCharset UTF-8 Character set used by deserializers that treat the input file as text. +decodeErrorPolicy ``FAIL`` What to do when we see a non-decodable character in the input file. + ``FAIL``: Throw an exception and fail to parse the file. + ``REPLACE``: Replace the unparseable character with the "replacement character" char, + typically Unicode U+FFFD. + ``IGNORE``: Drop the unparseable character sequence. +deserializer ``LINE`` Specify the deserializer used to parse the file into events. + Defaults to parsing each line as an event. The class specified must implement + ``EventDeserializer.Builder``. +deserializer.* Varies per event deserializer. +bufferMaxLines -- (Obselete) This option is now ignored. +bufferMaxLineLength 5000 (Deprecated) Maximum length of a line in the commit buffer. Use deserializer.maxLineLength instead. +selector.type replicating replicating or multiplexing +selector.* Depends on the selector.type value +interceptors -- Space-separated list of interceptors interceptors.* -======================= ============== ========================================================== +======================== ============== ========================================================== Example for an agent named agent-1: From 358bb670029549ed4cff192c79307fd3e4d69972 Mon Sep 17 00:00:00 2001 From: Kevin Conaway Date: Thu, 9 Jun 2016 15:50:13 -0400 Subject: [PATCH 308/341] FLUME-2922. Sync SequenceFile.Writer before calling hflush This closes #52 (Kevin Conaway via Mike Percy) --- .../flume/sink/hdfs/HDFSSequenceFile.java | 1 + .../flume/sink/hdfs/TestHDFSEventSink.java | 95 +++++++++++++++++++ 2 files changed, 96 insertions(+) diff --git a/flume-ng-sinks/flume-hdfs-sink/src/main/java/org/apache/flume/sink/hdfs/HDFSSequenceFile.java b/flume-ng-sinks/flume-hdfs-sink/src/main/java/org/apache/flume/sink/hdfs/HDFSSequenceFile.java index ba8b30d7e0..c5430ba658 100644 --- a/flume-ng-sinks/flume-hdfs-sink/src/main/java/org/apache/flume/sink/hdfs/HDFSSequenceFile.java +++ b/flume-ng-sinks/flume-hdfs-sink/src/main/java/org/apache/flume/sink/hdfs/HDFSSequenceFile.java @@ -109,6 +109,7 @@ public void append(Event e) throws IOException { @Override public void sync() throws IOException { + writer.sync(); hflushOrSync(outStream); } diff --git a/flume-ng-sinks/flume-hdfs-sink/src/test/java/org/apache/flume/sink/hdfs/TestHDFSEventSink.java b/flume-ng-sinks/flume-hdfs-sink/src/test/java/org/apache/flume/sink/hdfs/TestHDFSEventSink.java index 73f016bf3f..782cf47cc6 100644 --- a/flume-ng-sinks/flume-hdfs-sink/src/test/java/org/apache/flume/sink/hdfs/TestHDFSEventSink.java +++ b/flume-ng-sinks/flume-hdfs-sink/src/test/java/org/apache/flume/sink/hdfs/TestHDFSEventSink.java @@ -23,8 +23,10 @@ import java.io.InputStreamReader; import java.nio.ByteBuffer; import java.nio.charset.CharsetDecoder; +import java.util.Arrays; import java.util.Calendar; import java.util.Collection; +import java.util.Collections; import java.util.List; import java.util.Map; import java.util.UUID; @@ -1325,6 +1327,99 @@ public void testCloseOnIdle() throws IOException, EventDeliveryException, Interr fs.close(); } + /** + * This test simulates what happens when a batch of events is written to a compressed sequence + * file (and thus hsync'd to hdfs) but the file is not yet closed. + * + * When this happens, the data that we wrote should still be readable. + */ + @Test + public void testBlockCompressSequenceFileWriterSync() throws IOException, EventDeliveryException { + String hdfsPath = testPath + "/sequenceFileWriterSync"; + FileSystem fs = FileSystem.get(new Configuration()); + // Since we are reading a partial file we don't want to use checksums + fs.setVerifyChecksum(false); + fs.setWriteChecksum(false); + + // Compression codecs that don't require native hadoop libraries + String [] codecs = {"BZip2Codec", "DeflateCodec"}; + + for (String codec : codecs) { + sequenceFileWriteAndVerifyEvents(fs, hdfsPath, codec, Collections.singletonList( + "single-event" + )); + + sequenceFileWriteAndVerifyEvents(fs, hdfsPath, codec, Arrays.asList( + "multiple-events-1", + "multiple-events-2", + "multiple-events-3", + "multiple-events-4", + "multiple-events-5" + )); + } + + fs.close(); + } + + private void sequenceFileWriteAndVerifyEvents(FileSystem fs, String hdfsPath, String codec, + Collection eventBodies) + throws IOException, EventDeliveryException { + Path dirPath = new Path(hdfsPath); + fs.delete(dirPath, true); + fs.mkdirs(dirPath); + + Context context = new Context(); + context.put("hdfs.path", hdfsPath); + // Ensure the file isn't closed and rolled + context.put("hdfs.rollCount", String.valueOf(eventBodies.size() + 1)); + context.put("hdfs.rollSize", "0"); + context.put("hdfs.rollInterval", "0"); + context.put("hdfs.batchSize", "1"); + context.put("hdfs.fileType", "SequenceFile"); + context.put("hdfs.codeC", codec); + context.put("hdfs.writeFormat", "Writable"); + Configurables.configure(sink, context); + + Channel channel = new MemoryChannel(); + Configurables.configure(channel, context); + + sink.setChannel(channel); + sink.start(); + + for (String eventBody : eventBodies) { + Transaction txn = channel.getTransaction(); + txn.begin(); + + Event event = new SimpleEvent(); + event.setBody(eventBody.getBytes()); + channel.put(event); + + txn.commit(); + txn.close(); + + sink.process(); + } + + // Sink is _not_ closed. The file should remain open but + // the data written should be visible to readers via sync + hflush + FileStatus[] dirStat = fs.listStatus(dirPath); + Path[] paths = FileUtil.stat2Paths(dirStat); + + Assert.assertEquals(1, paths.length); + + SequenceFile.Reader reader = + new SequenceFile.Reader(fs.getConf(), SequenceFile.Reader.stream(fs.open(paths[0]))); + LongWritable key = new LongWritable(); + BytesWritable value = new BytesWritable(); + + for (String eventBody : eventBodies) { + Assert.assertTrue(reader.next(key, value)); + Assert.assertArrayEquals(eventBody.getBytes(), value.copyBytes()); + } + + Assert.assertFalse(reader.next(key, value)); + } + private Context getContextForRetryTests() { Context context = new Context(); From 1ca0765aae795a41a43e39324f5f1c8bae57b751 Mon Sep 17 00:00:00 2001 From: wenqiao <315524513@qq.com> Date: Wed, 20 Jul 2016 11:12:40 -0700 Subject: [PATCH 309/341] FLUME-2955. Add file path to the header in TaildirSource Allow for adding a file path to the header dynamically. This is particularly useful when the filegroup path contains a regex expression. (tinawenqiao via Mike Percy) --- flume-ng-doc/sphinx/FlumeUserGuide.rst | 3 ++ .../taildir/ReliableTaildirEventReader.java | 33 ++++++++++++++++--- .../flume/source/taildir/TaildirSource.java | 8 +++++ .../TaildirSourceConfigurationConstants.java | 8 +++++ .../source/taildir/TestTaildirSource.java | 32 ++++++++++++++++-- 5 files changed, 78 insertions(+), 6 deletions(-) diff --git a/flume-ng-doc/sphinx/FlumeUserGuide.rst b/flume-ng-doc/sphinx/FlumeUserGuide.rst index d8bfebfe61..105a036537 100644 --- a/flume-ng-doc/sphinx/FlumeUserGuide.rst +++ b/flume-ng-doc/sphinx/FlumeUserGuide.rst @@ -1134,6 +1134,8 @@ cachePatternMatching true Listing direc containing thousands of files. Caching the list of matching files can improve performance. The order in which files are consumed will also be cached. Requires that the file system keeps track of modification times with at least a 1-second granularity. +fileHeader false Whether to add a header storing the absolute path filename. +fileHeaderKey file Header key to use when appending absolute path filename to event header. =================================== ============================== =================================================== Example for agent named a1: @@ -1151,6 +1153,7 @@ Example for agent named a1: a1.sources.r1.filegroups.f2 = /var/log/test2/.*log.* a1.sources.r1.headers.f2.headerKey1 = value2 a1.sources.r1.headers.f2.headerKey2 = value2-2 + a1.sources.r1.fileHeader = true Twitter 1% firehose Source (experimental) ~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~ diff --git a/flume-ng-sources/flume-taildir-source/src/main/java/org/apache/flume/source/taildir/ReliableTaildirEventReader.java b/flume-ng-sources/flume-taildir-source/src/main/java/org/apache/flume/source/taildir/ReliableTaildirEventReader.java index 1409f2522a..8838320cbe 100644 --- a/flume-ng-sources/flume-taildir-source/src/main/java/org/apache/flume/source/taildir/ReliableTaildirEventReader.java +++ b/flume-ng-sources/flume-taildir-source/src/main/java/org/apache/flume/source/taildir/ReliableTaildirEventReader.java @@ -57,13 +57,16 @@ public class ReliableTaildirEventReader implements ReliableEventReader { private boolean addByteOffset; private boolean cachePatternMatching; private boolean committed = true; + private final boolean annotateFileName; + private final String fileNameHeader; /** * Create a ReliableTaildirEventReader to watch the given directory. */ private ReliableTaildirEventReader(Map filePaths, Table headerTable, String positionFilePath, - boolean skipToEnd, boolean addByteOffset, boolean cachePatternMatching) throws IOException { + boolean skipToEnd, boolean addByteOffset, boolean cachePatternMatching, + boolean annotateFileName, String fileNameHeader) throws IOException { // Sanity checks Preconditions.checkNotNull(filePaths); Preconditions.checkNotNull(positionFilePath); @@ -84,6 +87,8 @@ private ReliableTaildirEventReader(Map filePaths, this.headerTable = headerTable; this.addByteOffset = addByteOffset; this.cachePatternMatching = cachePatternMatching; + this.annotateFileName = annotateFileName; + this.fileNameHeader = fileNameHeader; updateTailFiles(skipToEnd); logger.info("Updating position from position file: " + positionFilePath); @@ -193,9 +198,14 @@ public List readEvents(int numEvents, boolean backoffWithoutNL) } Map headers = currentFile.getHeaders(); - if (headers != null && !headers.isEmpty()) { + if (annotateFileName || (headers != null && !headers.isEmpty())) { for (Event event : events) { - event.getHeaders().putAll(headers); + if (headers != null && !headers.isEmpty()) { + event.getHeaders().putAll(headers); + } + if (annotateFileName) { + event.getHeaders().put(fileNameHeader, currentFile.getPath()); + } } } committed = false; @@ -287,6 +297,10 @@ public static class Builder { private boolean skipToEnd; private boolean addByteOffset; private boolean cachePatternMatching; + private Boolean annotateFileName = + TaildirSourceConfigurationConstants.DEFAULT_FILE_HEADER; + private String fileNameHeader = + TaildirSourceConfigurationConstants.DEFAULT_FILENAME_HEADER_KEY; public Builder filePaths(Map filePaths) { this.filePaths = filePaths; @@ -318,9 +332,20 @@ public Builder cachePatternMatching(boolean cachePatternMatching) { return this; } + public Builder annotateFileName(boolean annotateFileName) { + this.annotateFileName = annotateFileName; + return this; + } + + public Builder fileNameHeader(String fileNameHeader) { + this.fileNameHeader = fileNameHeader; + return this; + } + public ReliableTaildirEventReader build() throws IOException { return new ReliableTaildirEventReader(filePaths, headerTable, positionFilePath, skipToEnd, - addByteOffset, cachePatternMatching); + addByteOffset, cachePatternMatching, + annotateFileName, fileNameHeader); } } diff --git a/flume-ng-sources/flume-taildir-source/src/main/java/org/apache/flume/source/taildir/TaildirSource.java b/flume-ng-sources/flume-taildir-source/src/main/java/org/apache/flume/source/taildir/TaildirSource.java index eae1b1a40f..a107a01721 100644 --- a/flume-ng-sources/flume-taildir-source/src/main/java/org/apache/flume/source/taildir/TaildirSource.java +++ b/flume-ng-sources/flume-taildir-source/src/main/java/org/apache/flume/source/taildir/TaildirSource.java @@ -84,6 +84,8 @@ public class TaildirSource extends AbstractSource implements private List idleInodes = new CopyOnWriteArrayList(); private Long backoffSleepIncrement; private Long maxBackOffSleepInterval; + private boolean fileHeader; + private String fileHeaderKey; @Override public synchronized void start() { @@ -96,6 +98,8 @@ public synchronized void start() { .skipToEnd(skipToEnd) .addByteOffset(byteOffsetHeader) .cachePatternMatching(cachePatternMatching) + .annotateFileName(fileHeader) + .fileNameHeader(fileHeaderKey) .build(); } catch (IOException e) { throw new FlumeException("Error instantiating ReliableTaildirEventReader", e); @@ -176,6 +180,10 @@ public synchronized void configure(Context context) { PollableSourceConstants.DEFAULT_BACKOFF_SLEEP_INCREMENT); maxBackOffSleepInterval = context.getLong(PollableSourceConstants.MAX_BACKOFF_SLEEP, PollableSourceConstants.DEFAULT_MAX_BACKOFF_SLEEP); + fileHeader = context.getBoolean(FILENAME_HEADER, + DEFAULT_FILE_HEADER); + fileHeaderKey = context.getString(FILENAME_HEADER_KEY, + DEFAULT_FILENAME_HEADER_KEY); if (sourceCounter == null) { sourceCounter = new SourceCounter(getName()); diff --git a/flume-ng-sources/flume-taildir-source/src/main/java/org/apache/flume/source/taildir/TaildirSourceConfigurationConstants.java b/flume-ng-sources/flume-taildir-source/src/main/java/org/apache/flume/source/taildir/TaildirSourceConfigurationConstants.java index 2c49540862..f2347f38c5 100644 --- a/flume-ng-sources/flume-taildir-source/src/main/java/org/apache/flume/source/taildir/TaildirSourceConfigurationConstants.java +++ b/flume-ng-sources/flume-taildir-source/src/main/java/org/apache/flume/source/taildir/TaildirSourceConfigurationConstants.java @@ -55,4 +55,12 @@ public class TaildirSourceConfigurationConstants { */ public static final String CACHE_PATTERN_MATCHING = "cachePatternMatching"; public static final boolean DEFAULT_CACHE_PATTERN_MATCHING = true; + + /** Header in which to put absolute path filename. */ + public static final String FILENAME_HEADER_KEY = "fileHeaderKey"; + public static final String DEFAULT_FILENAME_HEADER_KEY = "file"; + + /** Whether to include absolute path filename in a header. */ + public static final String FILENAME_HEADER = "fileHeader"; + public static final boolean DEFAULT_FILE_HEADER = false; } diff --git a/flume-ng-sources/flume-taildir-source/src/test/java/org/apache/flume/source/taildir/TestTaildirSource.java b/flume-ng-sources/flume-taildir-source/src/test/java/org/apache/flume/source/taildir/TestTaildirSource.java index e090b74461..097ee0ba1f 100644 --- a/flume-ng-sources/flume-taildir-source/src/test/java/org/apache/flume/source/taildir/TestTaildirSource.java +++ b/flume-ng-sources/flume-taildir-source/src/test/java/org/apache/flume/source/taildir/TestTaildirSource.java @@ -41,13 +41,15 @@ import java.util.List; import static org.apache.flume.source.taildir.TaildirSourceConfigurationConstants.FILE_GROUPS; -import static org.apache.flume.source.taildir.TaildirSourceConfigurationConstants - .FILE_GROUPS_PREFIX; +import static org.apache.flume.source.taildir.TaildirSourceConfigurationConstants.FILE_GROUPS_PREFIX; import static org.apache.flume.source.taildir.TaildirSourceConfigurationConstants.HEADERS_PREFIX; import static org.apache.flume.source.taildir.TaildirSourceConfigurationConstants.POSITION_FILE; +import static org.apache.flume.source.taildir.TaildirSourceConfigurationConstants.FILENAME_HEADER; +import static org.apache.flume.source.taildir.TaildirSourceConfigurationConstants.FILENAME_HEADER_KEY; import static org.junit.Assert.assertArrayEquals; import static org.junit.Assert.assertEquals; import static org.junit.Assert.assertFalse; +import static org.junit.Assert.assertNotNull; import static org.junit.Assert.assertNull; import static org.junit.Assert.assertTrue; @@ -288,4 +290,30 @@ public void testFileConsumeOrder() throws IOException { assertArrayEquals("Files not consumed in expected order", expected.toArray(), consumedOrder.toArray()); } + + @Test + public void testPutFilenameHeader() throws IOException { + File f1 = new File(tmpDir, "file1"); + Files.write("f1\n", f1, Charsets.UTF_8); + + Context context = new Context(); + context.put(POSITION_FILE, posFilePath); + context.put(FILE_GROUPS, "fg"); + context.put(FILE_GROUPS_PREFIX + "fg", tmpDir.getAbsolutePath() + "/file.*"); + context.put(FILENAME_HEADER, "true"); + context.put(FILENAME_HEADER_KEY, "path"); + + Configurables.configure(source, context); + source.start(); + source.process(); + Transaction txn = channel.getTransaction(); + txn.begin(); + Event e = channel.take(); + txn.commit(); + txn.close(); + + assertNotNull(e.getHeaders().get("path")); + assertEquals(f1.getAbsolutePath(), + e.getHeaders().get("path")); + } } From c554ba3d9b8234b37ba0b5dde3657034d11acb0b Mon Sep 17 00:00:00 2001 From: "Santiago M. Mola" Date: Wed, 20 Jul 2016 11:36:57 -0700 Subject: [PATCH 310/341] FLUME-2514. Speed up TestFileChannelRestart tests MIME-Version: 1.0 Content-Type: text/plain; charset=UTF-8 Content-Transfer-Encoding: 8bit TestFileChannelRestart tests are really slow. For example, testToggleCheckpointCompressionFromFalseToTrue and testToggleCheckpointCompressionFromTrueToFalse take ~4 minutes each. Some of them could be made faster by using channels with lower capacity. Reviewed by Bessenyei Balázs Donát (Santiago M. Mola via Mike Percy) --- .../org/apache/flume/channel/file/TestFileChannelRestart.java | 2 ++ 1 file changed, 2 insertions(+) diff --git a/flume-ng-channels/flume-file-channel/src/test/java/org/apache/flume/channel/file/TestFileChannelRestart.java b/flume-ng-channels/flume-file-channel/src/test/java/org/apache/flume/channel/file/TestFileChannelRestart.java index d21f140679..80c3013dd3 100644 --- a/flume-ng-channels/flume-file-channel/src/test/java/org/apache/flume/channel/file/TestFileChannelRestart.java +++ b/flume-ng-channels/flume-file-channel/src/test/java/org/apache/flume/channel/file/TestFileChannelRestart.java @@ -881,6 +881,8 @@ public void restartToggleCompression(boolean originalCheckpointCompressed) overrides.put(FileChannelConfiguration.USE_DUAL_CHECKPOINTS, "true"); overrides.put(FileChannelConfiguration.MAX_FILE_SIZE, "1000"); + overrides.put(FileChannelConfiguration.TRANSACTION_CAPACITY, "1000"); + overrides.put(FileChannelConfiguration.CAPACITY, "1000"); overrides.put(FileChannelConfiguration.COMPRESS_BACKUP_CHECKPOINT, String.valueOf(originalCheckpointCompressed)); channel = createFileChannel(overrides); From 10639e80cc83c14213dcc78692ab8f78945ddf8e Mon Sep 17 00:00:00 2001 From: Roshan Naik Date: Wed, 20 Jul 2016 16:36:14 -0700 Subject: [PATCH 311/341] FLUME-2761. Move Hive sink out of preview mode MIME-Version: 1.0 Content-Type: text/plain; charset=UTF-8 Content-Transfer-Encoding: 8bit Reviewed by Bessenyei Balázs Donát (Roshan Naik via Mike Percy) --- flume-ng-doc/sphinx/FlumeUserGuide.rst | 3 +-- 1 file changed, 1 insertion(+), 2 deletions(-) diff --git a/flume-ng-doc/sphinx/FlumeUserGuide.rst b/flume-ng-doc/sphinx/FlumeUserGuide.rst index 105a036537..1334500faa 100644 --- a/flume-ng-doc/sphinx/FlumeUserGuide.rst +++ b/flume-ng-doc/sphinx/FlumeUserGuide.rst @@ -1863,8 +1863,7 @@ This sink streams events containing delimited text or JSON data directly into a Events are written using Hive transactions. As soon as a set of events are committed to Hive, they become immediately visible to Hive queries. Partitions to which flume will stream to can either be pre-created or, optionally, Flume can create them if they are missing. Fields from incoming event data are mapped to -corresponding columns in the Hive table. **This sink is provided as a preview feature and not recommended -for use in production.** +corresponding columns in the Hive table. ====================== ============ ====================================================================== Name Default Description From 5a083a30b603d7213b1d44a174400b6cd44db70c Mon Sep 17 00:00:00 2001 From: Daniel Templeton Date: Tue, 26 Jul 2016 01:09:31 -0700 Subject: [PATCH 312/341] FLUME-2890. Typo in Twitter source warning Reviewed by Lior Zeno. (Daniel Templeton via Mike Percy) --- flume-ng-doc/sphinx/FlumeUserGuide.rst | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/flume-ng-doc/sphinx/FlumeUserGuide.rst b/flume-ng-doc/sphinx/FlumeUserGuide.rst index 1334500faa..b390e71129 100644 --- a/flume-ng-doc/sphinx/FlumeUserGuide.rst +++ b/flume-ng-doc/sphinx/FlumeUserGuide.rst @@ -1159,7 +1159,7 @@ Twitter 1% firehose Source (experimental) ~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~ .. warning:: - This source is hightly experimental and may change between minor versions of Flume. + This source is highly experimental and may change between minor versions of Flume. Use at your own risk. Experimental source that connects via Streaming API to the 1% sample twitter From 988ede948ffaf6526c226323a6808922f38b625c Mon Sep 17 00:00:00 2001 From: Lior Zeno Date: Wed, 27 Jul 2016 11:49:43 -0700 Subject: [PATCH 313/341] FLUME-2959. Fix issues with flume-checkstyle module * The lack of a parent POM caused warnings related to UTF-8 encoding * The XML files should have an ASL 2.0 license header (Lior Zeno via Mike Percy) --- flume-checkstyle/pom.xml | 6 ++++++ .../flume/checkstyle-suppressions.xml | 19 +++++++++++++++++++ .../src/main/resources/flume/checkstyle.xml | 19 +++++++++++++++++++ 3 files changed, 44 insertions(+) diff --git a/flume-checkstyle/pom.xml b/flume-checkstyle/pom.xml index 74ebf6b0d1..10d74f3b9b 100644 --- a/flume-checkstyle/pom.xml +++ b/flume-checkstyle/pom.xml @@ -25,4 +25,10 @@ flume-checkstyle Flume checkstyle project 1.7.0-SNAPSHOT + + + + UTF-8 + + diff --git a/flume-checkstyle/src/main/resources/flume/checkstyle-suppressions.xml b/flume-checkstyle/src/main/resources/flume/checkstyle-suppressions.xml index 78b9ee7aa4..598146870f 100644 --- a/flume-checkstyle/src/main/resources/flume/checkstyle-suppressions.xml +++ b/flume-checkstyle/src/main/resources/flume/checkstyle-suppressions.xml @@ -1,4 +1,23 @@ + + diff --git a/flume-checkstyle/src/main/resources/flume/checkstyle.xml b/flume-checkstyle/src/main/resources/flume/checkstyle.xml index fdbcb5dae0..7035e091c0 100644 --- a/flume-checkstyle/src/main/resources/flume/checkstyle.xml +++ b/flume-checkstyle/src/main/resources/flume/checkstyle.xml @@ -1,4 +1,23 @@ + + From dff1505c1ed76358d3f6e84df95118e1d92ece8d Mon Sep 17 00:00:00 2001 From: lfzCarlosC Date: Thu, 28 Jul 2016 22:04:19 -0700 Subject: [PATCH 314/341] Fix various typos This closes #49 (Fonso Li via Mike Percy) --- CHANGELOG | 2 +- .../java/org/apache/flume/channel/file/FlumeEventQueue.java | 2 +- .../src/main/java/org/apache/flume/channel/file/Log.java | 2 +- .../org/apache/flume/clients/log4jappender/Log4jAppender.java | 2 +- .../apache/flume/conf/sink/SinkProcessorConfiguration.java | 2 +- .../org/apache/flume/conf/source/SourceConfiguration.java | 2 +- .../src/main/java/org/apache/flume/conf/Configurable.java | 2 +- .../java/org/apache/flume/formatter/output/BucketPath.java | 4 ++-- .../apache/flume/serialization/ResettableFileInputStream.java | 2 +- .../java/org/apache/flume/sink/FailoverSinkProcessor.java | 2 +- .../src/test/java/org/apache/flume/source/TestExecSource.java | 2 +- .../test/java/org/apache/flume/source/TestStressSource.java | 2 +- flume-ng-doc/sphinx/FlumeUserGuide.rst | 4 ++-- .../flume/node/PropertiesFileConfigurationProvider.java | 2 +- .../java/org/apache/flume/api/TestLoadBalancingRpcClient.java | 2 +- .../src/main/java/org/apache/flume/sink/kite/DatasetSink.java | 2 +- .../sink/elasticsearch/client/ElasticSearchClientFactory.java | 2 +- .../main/java/org/apache/flume/sink/hbase/AsyncHBaseSink.java | 4 ++-- .../java/org/apache/flume/source/twitter/TwitterSource.java | 2 +- 19 files changed, 22 insertions(+), 22 deletions(-) diff --git a/CHANGELOG b/CHANGELOG index 53ea45cbd4..6e6a668151 100644 --- a/CHANGELOG +++ b/CHANGELOG @@ -720,7 +720,7 @@ Release Notes - Flume - Version v1.2.0 * [FLUME-1017] - syslog classes missing * [FLUME-1026] - Document Thread Safety Guarantees * [FLUME-1027] - Missing log4j library in Flume distribution - * [FLUME-1031] - Depricate code generated by Thrift and Avro OG sources that is under com.cloudera package + * [FLUME-1031] - Deprecate code generated by Thrift and Avro OG sources that is under com.cloudera package * [FLUME-1035] - slf4j error in flume sdk unit tests * [FLUME-1036] - Reconfiguration of AVRO or NETCAT source causes port bind exception * [FLUME-1037] - NETCAT handler theads terminate under stress test diff --git a/flume-ng-channels/flume-file-channel/src/main/java/org/apache/flume/channel/file/FlumeEventQueue.java b/flume-ng-channels/flume-file-channel/src/main/java/org/apache/flume/channel/file/FlumeEventQueue.java index 4311c7fb73..4bb4f7264e 100644 --- a/flume-ng-channels/flume-file-channel/src/main/java/org/apache/flume/channel/file/FlumeEventQueue.java +++ b/flume-ng-channels/flume-file-channel/src/main/java/org/apache/flume/channel/file/FlumeEventQueue.java @@ -514,7 +514,7 @@ public void serializeAndWrite() throws Exception { + values.size()) * 8) //Event pointers + 16; //Checksum //There is no real need of filling the channel with 0s, since we - //will write the exact nummber of bytes as expected file size. + //will write the exact number of bytes as expected file size. file.setLength(expectedFileSize); Preconditions.checkState(file.length() == expectedFileSize, "Expected File size of inflight events file does not match the " diff --git a/flume-ng-channels/flume-file-channel/src/main/java/org/apache/flume/channel/file/Log.java b/flume-ng-channels/flume-file-channel/src/main/java/org/apache/flume/channel/file/Log.java index 02d8e7f6bf..5f59d97ff9 100644 --- a/flume-ng-channels/flume-file-channel/src/main/java/org/apache/flume/channel/file/Log.java +++ b/flume-ng-channels/flume-file-channel/src/main/java/org/apache/flume/channel/file/Log.java @@ -1141,7 +1141,7 @@ private void removeOldLogs(SortedSet fileIDs) { *

    Locking is not supported by all file systems. * E.g., NFS does not consistently support exclusive locks. *

    - *

    If locking is supported we guarantee exculsive access to the + *

    If locking is supported we guarantee exclusive access to the * storage directory. Otherwise, no guarantee is given. * * @throws IOException if locking fails diff --git a/flume-ng-clients/flume-ng-log4jappender/src/main/java/org/apache/flume/clients/log4jappender/Log4jAppender.java b/flume-ng-clients/flume-ng-log4jappender/src/main/java/org/apache/flume/clients/log4jappender/Log4jAppender.java index f9803e424c..117542d481 100644 --- a/flume-ng-clients/flume-ng-log4jappender/src/main/java/org/apache/flume/clients/log4jappender/Log4jAppender.java +++ b/flume-ng-clients/flume-ng-log4jappender/src/main/java/org/apache/flume/clients/log4jappender/Log4jAppender.java @@ -47,7 +47,7 @@ /** * - * Appends Log4j Events to an external Flume client which is decribed by + * Appends Log4j Events to an external Flume client which is described by * the Log4j configuration file. The appender takes two required parameters: *

    *Hostname : This is the hostname of the first hop diff --git a/flume-ng-configuration/src/main/java/org/apache/flume/conf/sink/SinkProcessorConfiguration.java b/flume-ng-configuration/src/main/java/org/apache/flume/conf/sink/SinkProcessorConfiguration.java index 9ce005879b..d0c9802929 100644 --- a/flume-ng-configuration/src/main/java/org/apache/flume/conf/sink/SinkProcessorConfiguration.java +++ b/flume-ng-configuration/src/main/java/org/apache/flume/conf/sink/SinkProcessorConfiguration.java @@ -46,7 +46,7 @@ public void setSinks(Set sinks) { public enum SinkProcessorConfigurationType { /** - * Load balacing channel selector + * Load balancing channel selector */ LOAD_BALANCE("org.apache.flume.conf.sink.LoadBalancingSinkProcessorConfiguration"), /** diff --git a/flume-ng-configuration/src/main/java/org/apache/flume/conf/source/SourceConfiguration.java b/flume-ng-configuration/src/main/java/org/apache/flume/conf/source/SourceConfiguration.java index 8d7318e25c..6bd148969f 100644 --- a/flume-ng-configuration/src/main/java/org/apache/flume/conf/source/SourceConfiguration.java +++ b/flume-ng-configuration/src/main/java/org/apache/flume/conf/source/SourceConfiguration.java @@ -153,7 +153,7 @@ public enum SourceConfigurationType { EXEC("org.apache.flume.conf.source.ExecSourceConfiguration"), /** - * Avro soruce. + * Avro source. * * @see AvroSource */ diff --git a/flume-ng-core/src/main/java/org/apache/flume/conf/Configurable.java b/flume-ng-core/src/main/java/org/apache/flume/conf/Configurable.java index 5b7862619e..f5f92859f7 100644 --- a/flume-ng-core/src/main/java/org/apache/flume/conf/Configurable.java +++ b/flume-ng-core/src/main/java/org/apache/flume/conf/Configurable.java @@ -41,7 +41,7 @@ public interface Configurable { * reflected by the component asap. *

    *

    - * There are no thread safety guarrantees on when configure might be called. + * There are no thread safety guarantees on when configure might be called. *

    * @param context */ diff --git a/flume-ng-core/src/main/java/org/apache/flume/formatter/output/BucketPath.java b/flume-ng-core/src/main/java/org/apache/flume/formatter/output/BucketPath.java index 167b5425ea..f640ec9791 100644 --- a/flume-ng-core/src/main/java/org/apache/flume/formatter/output/BucketPath.java +++ b/flume-ng-core/src/main/java/org/apache/flume/formatter/output/BucketPath.java @@ -433,10 +433,10 @@ public static String escapeString(String in, Map headers, // The replacement string must have '$' and '\' chars escaped. This // replacement string is pretty arcane. // - // replacee : '$' -> for java '\$' -> for regex "\\$" + // replace : '$' -> for java '\$' -> for regex "\\$" // replacement: '\$' -> for regex '\\\$' -> for java "\\\\\\$" // - // replacee : '\' -> for java "\\" -> for regex "\\\\" + // replace : '\' -> for java "\\" -> for regex "\\\\" // replacement: '\\' -> for regex "\\\\" -> for java "\\\\\\\\" // note: order matters diff --git a/flume-ng-core/src/main/java/org/apache/flume/serialization/ResettableFileInputStream.java b/flume-ng-core/src/main/java/org/apache/flume/serialization/ResettableFileInputStream.java index 7d6d95c8d0..b263f98e68 100644 --- a/flume-ng-core/src/main/java/org/apache/flume/serialization/ResettableFileInputStream.java +++ b/flume-ng-core/src/main/java/org/apache/flume/serialization/ResettableFileInputStream.java @@ -119,7 +119,7 @@ public class ResettableFileInputStream extends ResettableInputStream private boolean hasLowSurrogate = false; /** - * A low surrrgate character read from a surrogate pair. + * A low surrogate character read from a surrogate pair. * When a surrogate pair is found, the high (first) surrogate pair * is returned upon a call to {@link #read()}, * while the low (second) surrogate remains stored in memory, diff --git a/flume-ng-core/src/main/java/org/apache/flume/sink/FailoverSinkProcessor.java b/flume-ng-core/src/main/java/org/apache/flume/sink/FailoverSinkProcessor.java index 69541e6feb..fd6dc2f352 100644 --- a/flume-ng-core/src/main/java/org/apache/flume/sink/FailoverSinkProcessor.java +++ b/flume-ng-core/src/main/java/org/apache/flume/sink/FailoverSinkProcessor.java @@ -39,7 +39,7 @@ * * The failover mechanism works by relegating failed sinks to a pool * where they are assigned a cooldown period, increasing with sequential - * failures before they are retried. Once a sink succesfully sends an + * failures before they are retried. Once a sink successfully sends an * event it is restored to the live pool. * * FailoverSinkProcessor is in no way thread safe and expects to be run via diff --git a/flume-ng-core/src/test/java/org/apache/flume/source/TestExecSource.java b/flume-ng-core/src/test/java/org/apache/flume/source/TestExecSource.java index 6204d134a5..36a292122f 100644 --- a/flume-ng-core/src/test/java/org/apache/flume/source/TestExecSource.java +++ b/flume-ng-core/src/test/java/org/apache/flume/source/TestExecSource.java @@ -415,7 +415,7 @@ public void testShutdown() throws Exception { } // yes in the mean time someone could use our sleep time - // but this should be a fairly rare scenerio + // but this should be a fairly rare scenario String command = "sleep " + seconds; Pattern pattern = Pattern.compile("\b" + command + "\b"); diff --git a/flume-ng-core/src/test/java/org/apache/flume/source/TestStressSource.java b/flume-ng-core/src/test/java/org/apache/flume/source/TestStressSource.java index 06e663c4c5..5bf3805569 100644 --- a/flume-ng-core/src/test/java/org/apache/flume/source/TestStressSource.java +++ b/flume-ng-core/src/test/java/org/apache/flume/source/TestStressSource.java @@ -164,7 +164,7 @@ public void testMaxSuccessfulEvents() throws InterruptedException, source.process(); } - // 1 failed call, 50 succesful + // 1 failed call, 50 successful doThrow(new ChannelException("stub")).when( mockProcessor).processEvent(getEvent(source)); source.process(); diff --git a/flume-ng-doc/sphinx/FlumeUserGuide.rst b/flume-ng-doc/sphinx/FlumeUserGuide.rst index b390e71129..3f08d8b610 100644 --- a/flume-ng-doc/sphinx/FlumeUserGuide.rst +++ b/flume-ng-doc/sphinx/FlumeUserGuide.rst @@ -900,7 +900,7 @@ Property Name Default Description **channels** -- **type** -- The component type name, needs to be ``jms`` **initialContextFactory** -- Inital Context Factory, e.g: org.apache.activemq.jndi.ActiveMQInitialContextFactory -**connectionFactory** -- The JNDI name the connection factory shoulld appear as +**connectionFactory** -- The JNDI name the connection factory should appear as **providerURL** -- The JMS provider URL **destinationName** -- Destination name **destinationType** -- Destination type (queue or topic) @@ -1176,7 +1176,7 @@ Property Name Default Description **consumerKey** -- OAuth consumer key **consumerSecret** -- OAuth consumer secret **accessToken** -- OAuth access token -**accessTokenSecret** -- OAuth toekn secret +**accessTokenSecret** -- OAuth token secret maxBatchSize 1000 Maximum number of twitter messages to put in a single batch maxBatchDurationMillis 1000 Maximum number of milliseconds to wait before closing a batch ====================== =========== =================================================== diff --git a/flume-ng-node/src/main/java/org/apache/flume/node/PropertiesFileConfigurationProvider.java b/flume-ng-node/src/main/java/org/apache/flume/node/PropertiesFileConfigurationProvider.java index bc5438ad3b..b428c9ef99 100644 --- a/flume-ng-node/src/main/java/org/apache/flume/node/PropertiesFileConfigurationProvider.java +++ b/flume-ng-node/src/main/java/org/apache/flume/node/PropertiesFileConfigurationProvider.java @@ -96,7 +96,7 @@ * belonging to it. These cannot be shared by multiple groups. * Further, one can set a processor and behavioral parameters to determine * how sink selection is made via <agent name>.sinkgroups.< - * group name<.processor. For further detail refer to inividual processor + * group name<.processor. For further detail refer to individual processor * documentation *
  • Sinks not assigned to a group will be assigned to default single sink * groups.
  • diff --git a/flume-ng-sdk/src/test/java/org/apache/flume/api/TestLoadBalancingRpcClient.java b/flume-ng-sdk/src/test/java/org/apache/flume/api/TestLoadBalancingRpcClient.java index dc53d3fe7f..ba7a735cd5 100644 --- a/flume-ng-sdk/src/test/java/org/apache/flume/api/TestLoadBalancingRpcClient.java +++ b/flume-ng-sdk/src/test/java/org/apache/flume/api/TestLoadBalancingRpcClient.java @@ -617,7 +617,7 @@ public void testRoundRobinBackoffIncreasingBackoffs() throws Exception { } Assert.assertEquals(0, hosts.get(1).getAppendCount()); - // after this s2 should be receiving events agains + // after this s2 should be receiving events again Thread.sleep(2500); int numEvents = 60; for (int i = 0; i < numEvents; i++) { diff --git a/flume-ng-sinks/flume-dataset-sink/src/main/java/org/apache/flume/sink/kite/DatasetSink.java b/flume-ng-sinks/flume-dataset-sink/src/main/java/org/apache/flume/sink/kite/DatasetSink.java index e3b57c323f..fa312628df 100644 --- a/flume-ng-sinks/flume-dataset-sink/src/main/java/org/apache/flume/sink/kite/DatasetSink.java +++ b/flume-ng-sinks/flume-dataset-sink/src/main/java/org/apache/flume/sink/kite/DatasetSink.java @@ -391,7 +391,7 @@ void write(Event event) throws EventDeliveryException { */ @VisibleForTesting void createWriter() throws EventDeliveryException { - // reset the commited flag whenver a new writer is created + // reset the commited flag whenever a new writer is created committedBatch = false; try { View view; diff --git a/flume-ng-sinks/flume-ng-elasticsearch-sink/src/main/java/org/apache/flume/sink/elasticsearch/client/ElasticSearchClientFactory.java b/flume-ng-sinks/flume-ng-elasticsearch-sink/src/main/java/org/apache/flume/sink/elasticsearch/client/ElasticSearchClientFactory.java index cb34394eba..986fb2b8f0 100644 --- a/flume-ng-sinks/flume-ng-elasticsearch-sink/src/main/java/org/apache/flume/sink/elasticsearch/client/ElasticSearchClientFactory.java +++ b/flume-ng-sinks/flume-ng-elasticsearch-sink/src/main/java/org/apache/flume/sink/elasticsearch/client/ElasticSearchClientFactory.java @@ -34,7 +34,7 @@ public class ElasticSearchClientFactory { * @param clientType * String representation of client type * @param hostNames - * Array of strings that represents hosntames with ports (hostname:port) + * Array of strings that represents hostnames with ports (hostname:port) * @param clusterName * Elasticsearch cluster name used only by Transport Client * @param serializer diff --git a/flume-ng-sinks/flume-ng-hbase-sink/src/main/java/org/apache/flume/sink/hbase/AsyncHBaseSink.java b/flume-ng-sinks/flume-ng-hbase-sink/src/main/java/org/apache/flume/sink/hbase/AsyncHBaseSink.java index 280d0b0ffd..f120f59f35 100644 --- a/flume-ng-sinks/flume-ng-hbase-sink/src/main/java/org/apache/flume/sink/hbase/AsyncHBaseSink.java +++ b/flume-ng-sinks/flume-ng-hbase-sink/src/main/java/org/apache/flume/sink/hbase/AsyncHBaseSink.java @@ -64,9 +64,9 @@ /** * A simple sink which reads events from a channel and writes them to HBase. - * This Sink uses an aysnchronous API internally and is likely to + * This Sink uses an asynchronous API internally and is likely to * perform better. - * The Hbase configution is picked up from the first hbase-site.xml + * The Hbase configuration is picked up from the first hbase-site.xml * encountered in the classpath. This sink supports batch reading of * events from the channel, and writing them to Hbase, to minimize the number * of flushes on the hbase tables. To use this sink, it has to be configured diff --git a/flume-ng-sources/flume-twitter-source/src/main/java/org/apache/flume/source/twitter/TwitterSource.java b/flume-ng-sources/flume-twitter-source/src/main/java/org/apache/flume/source/twitter/TwitterSource.java index 27b2c3ff3d..f5c832807d 100644 --- a/flume-ng-sources/flume-twitter-source/src/main/java/org/apache/flume/source/twitter/TwitterSource.java +++ b/flume-ng-sources/flume-twitter-source/src/main/java/org/apache/flume/source/twitter/TwitterSource.java @@ -57,7 +57,7 @@ /** * Demo Flume source that connects via Streaming API to the 1% sample twitter - * firehose, continously downloads tweets, converts them to Avro format and + * firehose, continuously downloads tweets, converts them to Avro format and * sends Avro events to a downstream Flume sink. * * Requires the consumer and access tokens and secrets of a Twitter developer From 4b74aa286b16916900854ccea95cf906d3fdda5d Mon Sep 17 00:00:00 2001 From: Denes Arvay Date: Fri, 29 Jul 2016 15:16:54 -0700 Subject: [PATCH 315/341] FLUME-2963. FlumeUserGuide: Fix error in Kafka Source properties table (Denes Arvay via Mike Percy) --- flume-ng-doc/sphinx/FlumeUserGuide.rst | 8 ++++---- 1 file changed, 4 insertions(+), 4 deletions(-) diff --git a/flume-ng-doc/sphinx/FlumeUserGuide.rst b/flume-ng-doc/sphinx/FlumeUserGuide.rst index 3f08d8b610..fde9ff7b04 100644 --- a/flume-ng-doc/sphinx/FlumeUserGuide.rst +++ b/flume-ng-doc/sphinx/FlumeUserGuide.rst @@ -1230,11 +1230,11 @@ useFlumeEventFormat false By default events are taken as byt true to read events as the Flume Avro binary format. Used in conjunction with the same property on the KafkaSink or with the parseAsFlumeEvent property on the Kafka Channel this will preserve any Flume headers sent on the producing side. - Other Kafka Consumer Properties -- These properties are used to configure the Kafka Consumer. Any producer property supported - by Kafka can be used. The only requirement is to prepend the property name with the prefix ``kafka.consumer``. - For example: kafka.consumer.auto.offset.reset - Check `Kafka documentation ` for details + by Kafka can be used. The only requirement is to prepend the property name with the prefix + ``kafka.consumer``. + For example: ``kafka.consumer.auto.offset.reset`` + Check `Kafka documentation `_ for details =============================== =========== =================================================== .. note:: The Kafka Source overrides two Kafka consumer parameters: From 1422f733007dbb78caae7e5135bc33470e88502a Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?Bessenyei=20Bal=C3=A1zs=20Don=C3=A1t?= Date: Mon, 8 Aug 2016 18:09:44 -0700 Subject: [PATCH 316/341] FLUME-2619. Spooldir source should log channel exceptions MIME-Version: 1.0 Content-Type: text/plain; charset=UTF-8 Content-Transfer-Encoding: 8bit Reviewed by Denes Arvay and Mike Percy. (Bessenyei Balázs Donát via Mike Percy) --- .../flume/channel/ChannelProcessor.java | 24 ++++++---- .../flume/source/SpoolDirectorySource.java | 45 ++++++++++++------ .../source/TestSpoolDirectorySource.java | 46 ++++++++++--------- 3 files changed, 71 insertions(+), 44 deletions(-) diff --git a/flume-ng-core/src/main/java/org/apache/flume/channel/ChannelProcessor.java b/flume-ng-core/src/main/java/org/apache/flume/channel/ChannelProcessor.java index 1cce137f78..69878609ce 100644 --- a/flume-ng-core/src/main/java/org/apache/flume/channel/ChannelProcessor.java +++ b/flume-ng-core/src/main/java/org/apache/flume/channel/ChannelProcessor.java @@ -20,6 +20,7 @@ import com.google.common.base.Preconditions; import com.google.common.collect.Lists; + import java.util.ArrayList; import java.util.LinkedHashMap; import java.util.List; @@ -43,7 +44,7 @@ * A channel processor exposes operations to put {@link Event}s into * {@link Channel}s. These operations will propagate a {@link ChannelException} * if any errors occur while attempting to write to {@code required} channels. - * + *

    * Each channel processor instance is configured with a {@link ChannelSelector} * instance that specifies which channels are * {@linkplain ChannelSelector#getRequiredChannels(Event) required} and which @@ -73,6 +74,7 @@ public void close() { /** * The Context of the associated Source is passed. + * * @param context */ @Override @@ -103,7 +105,7 @@ private void configureInterceptors(Context context) { if (type == null) { LOG.error("Type not specified for interceptor " + interceptorName); throw new FlumeException("Interceptor.Type not specified for " + - interceptorName); + interceptorName); } try { Interceptor.Builder builder = factory.newInstance(type); @@ -132,7 +134,7 @@ public ChannelSelector getSelector() { * Attempts to {@linkplain Channel#put(Event) put} the given events into each * configured channel. If any {@code required} channel throws a * {@link ChannelException}, that exception will be propagated. - * + *

    *

    Note that if multiple channels are configured, some {@link Transaction}s * may have already been committed while others may be rolled back in the * case of an exception. @@ -165,7 +167,7 @@ public void processEventBatch(List events) { List optChannels = selector.getOptionalChannels(event); - for (Channel ch: optChannels) { + for (Channel ch : optChannels) { List eventQueue = optChannelQueue.get(ch); if (eventQueue == null) { eventQueue = new ArrayList(); @@ -193,9 +195,10 @@ public void processEventBatch(List events) { } catch (Throwable t) { tx.rollback(); if (t instanceof Error) { - LOG.error("Error while writing to required channel: " + - reqChannel, t); + LOG.error("Error while writing to required channel: " + reqChannel, t); throw (Error) t; + } else if (t instanceof ChannelException) { + throw (ChannelException) t; } else { throw new ChannelException("Unable to put batch on required " + "channel: " + reqChannel, t); @@ -216,7 +219,7 @@ public void processEventBatch(List events) { List batch = optChannelQueue.get(optChannel); - for (Event event : batch ) { + for (Event event : batch) { optChannel.put(event); } @@ -239,7 +242,7 @@ public void processEventBatch(List events) { * Attempts to {@linkplain Channel#put(Event) put} the given event into each * configured channel. If any {@code required} channel throws a * {@link ChannelException}, that exception will be propagated. - * + *

    *

    Note that if multiple channels are configured, some {@link Transaction}s * may have already been committed while others may be rolled back in the * case of an exception. @@ -268,9 +271,10 @@ public void processEvent(Event event) { } catch (Throwable t) { tx.rollback(); if (t instanceof Error) { - LOG.error("Error while writing to required channel: " + - reqChannel, t); + LOG.error("Error while writing to required channel: " + reqChannel, t); throw (Error) t; + } else if (t instanceof ChannelException) { + throw (ChannelException) t; } else { throw new ChannelException("Unable to put event on required " + "channel: " + reqChannel, t); diff --git a/flume-ng-core/src/main/java/org/apache/flume/source/SpoolDirectorySource.java b/flume-ng-core/src/main/java/org/apache/flume/source/SpoolDirectorySource.java index d88cc1d8b7..c8c7cda674 100644 --- a/flume-ng-core/src/main/java/org/apache/flume/source/SpoolDirectorySource.java +++ b/flume-ng-core/src/main/java/org/apache/flume/source/SpoolDirectorySource.java @@ -21,6 +21,7 @@ import com.google.common.base.Preconditions; import com.google.common.base.Throwables; import org.apache.flume.ChannelException; +import org.apache.flume.ChannelFullException; import org.apache.flume.Context; import org.apache.flume.Event; import org.apache.flume.EventDrivenSource; @@ -44,7 +45,7 @@ import static org.apache.flume.source.SpoolDirectorySourceConfigurationConstants.*; public class SpoolDirectorySource extends AbstractSource - implements Configurable, EventDrivenSource { + implements Configurable, EventDrivenSource { private static final Logger logger = LoggerFactory.getLogger(SpoolDirectorySource.class); @@ -70,6 +71,7 @@ public class SpoolDirectorySource extends AbstractSource private ScheduledExecutorService executor; private boolean backoff = true; private boolean hitChannelException = false; + private boolean hitChannelFullException = false; private int maxBackoff; private ConsumeOrder consumeOrder; private int pollDelay; @@ -158,7 +160,7 @@ public synchronized void configure(Context context) { inputCharset = context.getString(INPUT_CHARSET, DEFAULT_INPUT_CHARSET); decodeErrorPolicy = DecodeErrorPolicy.valueOf( context.getString(DECODE_ERROR_POLICY, DEFAULT_DECODE_ERROR_POLICY) - .toUpperCase(Locale.ENGLISH)); + .toUpperCase(Locale.ENGLISH)); ignorePattern = context.getString(IGNORE_PAT, DEFAULT_IGNORE_PAT); trackerDirPath = context.getString(TRACKER_DIR, DEFAULT_TRACKER_DIR); @@ -196,10 +198,10 @@ protected boolean hasFatalError() { } - /** * The class always backs off, this exists only so that we can test without * taking a really long time. + * * @param backoff - whether the source should backoff if the channel is full */ @VisibleForTesting @@ -208,10 +210,15 @@ protected void setBackOff(boolean backoff) { } @VisibleForTesting - protected boolean hitChannelException() { + protected boolean didHitChannelException() { return hitChannelException; } + @VisibleForTesting + protected boolean didHitChannelFullException() { + return hitChannelFullException; + } + @VisibleForTesting protected SourceCounter getSourceCounter() { return sourceCounter; @@ -227,7 +234,7 @@ private class SpoolDirectoryRunnable implements Runnable { private SourceCounter sourceCounter; public SpoolDirectoryRunnable(ReliableSpoolingFileEventReader reader, - SourceCounter sourceCounter) { + SourceCounter sourceCounter) { this.reader = reader; this.sourceCounter = sourceCounter; } @@ -247,17 +254,19 @@ public void run() { try { getChannelProcessor().processEventBatch(events); reader.commit(); - } catch (ChannelException ex) { + } catch (ChannelFullException ex) { logger.warn("The channel is full, and cannot write data now. The " + - "source will try again after " + String.valueOf(backoffInterval) + + "source will try again after " + backoffInterval + + " milliseconds"); + hitChannelFullException = true; + backoffInterval = waitAndGetNewBackoffInterval(backoffInterval); + continue; + } catch (ChannelException ex) { + logger.warn("The channel threw an exception, and cannot write data now. The " + + "source will try again after " + backoffInterval + " milliseconds"); hitChannelException = true; - if (backoff) { - TimeUnit.MILLISECONDS.sleep(backoffInterval); - backoffInterval = backoffInterval << 1; - backoffInterval = backoffInterval >= maxBackoff ? maxBackoff : - backoffInterval; - } + backoffInterval = waitAndGetNewBackoffInterval(backoffInterval); continue; } backoffInterval = 250; @@ -272,5 +281,15 @@ public void run() { Throwables.propagate(t); } } + + private int waitAndGetNewBackoffInterval(int backoffInterval) throws InterruptedException { + if (backoff) { + TimeUnit.MILLISECONDS.sleep(backoffInterval); + backoffInterval = backoffInterval << 1; + backoffInterval = backoffInterval >= maxBackoff ? maxBackoff : + backoffInterval; + } + return backoffInterval; + } } } diff --git a/flume-ng-core/src/test/java/org/apache/flume/source/TestSpoolDirectorySource.java b/flume-ng-core/src/test/java/org/apache/flume/source/TestSpoolDirectorySource.java index 82c53519c3..0182d2120f 100644 --- a/flume-ng-core/src/test/java/org/apache/flume/source/TestSpoolDirectorySource.java +++ b/flume-ng-core/src/test/java/org/apache/flume/source/TestSpoolDirectorySource.java @@ -74,6 +74,7 @@ public void tearDown() { /** * Helper method to recursively clean up testing directory + * * @param directory the directory to clean up */ private void deleteFiles(File directory) { @@ -87,7 +88,7 @@ private void deleteFiles(File directory) { } } - @Test (expected = IllegalArgumentException.class) + @Test(expected = IllegalArgumentException.class) public void testInvalidSortOrder() { Context context = new Context(); context.put(SpoolDirectorySourceConfigurationConstants.SPOOL_DIRECTORY, @@ -102,7 +103,7 @@ public void testValidSortOrder() { Context context = new Context(); context.put(SpoolDirectorySourceConfigurationConstants.SPOOL_DIRECTORY, tmpDir.getAbsolutePath()); - context.put(SpoolDirectorySourceConfigurationConstants.CONSUME_ORDER, + context.put(SpoolDirectorySourceConfigurationConstants.CONSUME_ORDER, "oLdESt"); Configurables.configure(source, context); context.put(SpoolDirectorySourceConfigurationConstants.CONSUME_ORDER, @@ -110,17 +111,17 @@ public void testValidSortOrder() { Configurables.configure(source, context); context.put(SpoolDirectorySourceConfigurationConstants.CONSUME_ORDER, "rAnDom"); - Configurables.configure(source, context); + Configurables.configure(source, context); } - + @Test public void testPutFilenameHeader() throws IOException, InterruptedException { Context context = new Context(); File f1 = new File(tmpDir.getAbsolutePath() + "/file1"); Files.write("file1line1\nfile1line2\nfile1line3\nfile1line4\n" + - "file1line5\nfile1line6\nfile1line7\nfile1line8\n", - f1, Charsets.UTF_8); + "file1line5\nfile1line6\nfile1line7\nfile1line8\n", + f1, Charsets.UTF_8); context.put(SpoolDirectorySourceConfigurationConstants.SPOOL_DIRECTORY, tmpDir.getAbsolutePath()); @@ -155,7 +156,7 @@ public void testPutBasenameHeader() throws IOException, InterruptedException { File f1 = new File(tmpDir.getAbsolutePath() + "/file1"); Files.write("file1line1\nfile1line2\nfile1line3\nfile1line4\n" + - "file1line5\nfile1line6\nfile1line7\nfile1line8\n", + "file1line5\nfile1line6\nfile1line7\nfile1line8\n", f1, Charsets.UTF_8); context.put(SpoolDirectorySourceConfigurationConstants.SPOOL_DIRECTORY, @@ -192,18 +193,18 @@ public void testRecursion_SetToTrue() throws IOException, InterruptedException { Assert.assertTrue("source directories must be created", directoriesCreated); final String FILE_NAME = "recursion_file.txt"; - File f1 = new File(subDir, FILE_NAME); + File f1 = new File(subDir, FILE_NAME); String origBody = "file1line1\nfile1line2\nfile1line3\nfile1line4\n" + "file1line5\nfile1line6\nfile1line7\nfile1line8\n"; Files.write(origBody, f1, Charsets.UTF_8); Context context = new Context(); context.put(SpoolDirectorySourceConfigurationConstants.RECURSIVE_DIRECTORY_SEARCH, - "true"); // enable recursion, so we should find the file we created above + "true"); // enable recursion, so we should find the file we created above context.put(SpoolDirectorySourceConfigurationConstants.SPOOL_DIRECTORY, - tmpDir.getAbsolutePath()); // spool set to root dir + tmpDir.getAbsolutePath()); // spool set to root dir context.put(SpoolDirectorySourceConfigurationConstants.FILENAME_HEADER, - "true"); // put the file name in the "file" header + "true"); // put the file name in the "file" header Configurables.configure(source, context); source.start(); @@ -224,7 +225,7 @@ public void testRecursion_SetToTrue() throws IOException, InterruptedException { Assert.assertNotNull("Event headers must not be null", e.getHeaders()); Assert.assertTrue("File header value did not end with expected filename", - e.getHeaders().get("file").endsWith(FILE_NAME)); + e.getHeaders().get("file").endsWith(FILE_NAME)); ByteArrayOutputStream baos = new ByteArrayOutputStream(); do { // collecting the whole body @@ -256,7 +257,6 @@ public void testRecursion_SetToFalse() throws IOException, InterruptedException Assert.assertTrue("source directories must be created", directoriesCreated); - File f1 = new File(subDir.getAbsolutePath() + "/file1.txt"); Files.write("file1line1\nfile1line2\nfile1line3\nfile1line4\n" + @@ -364,12 +364,12 @@ public void testSourceDoesNotDieOnFullChannel() throws Exception { File f1 = new File(tmpDir.getAbsolutePath() + "/file1"); Files.write("file1line1\nfile1line2\nfile1line3\nfile1line4\n" + - "file1line5\nfile1line6\nfile1line7\nfile1line8\n", + "file1line5\nfile1line6\nfile1line7\nfile1line8\n", f1, Charsets.UTF_8); context.put(SpoolDirectorySourceConfigurationConstants.SPOOL_DIRECTORY, - tmpDir.getAbsolutePath()); + tmpDir.getAbsolutePath()); context.put(SpoolDirectorySourceConfigurationConstants.BATCH_SIZE, "2"); Configurables.configure(source, context); @@ -377,10 +377,16 @@ public void testSourceDoesNotDieOnFullChannel() throws Exception { source.start(); // Wait for the source to read enough events to fill up the channel. - while (!source.hitChannelException()) { - Thread.sleep(50); + + long startTime = System.currentTimeMillis(); + while (System.currentTimeMillis() - startTime < 5000 && !source.didHitChannelFullException()) { + Thread.sleep(10); } + Assert.assertTrue("Expected to hit ChannelFullException, but did not!", + source.didHitChannelFullException()); + + List dataOut = Lists.newArrayList(); for (int i = 0; i < 8; ) { @@ -399,8 +405,6 @@ public void testSourceDoesNotDieOnFullChannel() throws Exception { tx.commit(); tx.close(); } - Assert.assertTrue("Expected to hit ChannelException, but did not!", - source.hitChannelException()); Assert.assertEquals(8, dataOut.size()); source.stop(); } @@ -422,7 +426,7 @@ public void testEndWithZeroByteFiles() throws IOException, InterruptedException Files.touch(f4); context.put(SpoolDirectorySourceConfigurationConstants.SPOOL_DIRECTORY, - tmpDir.getAbsolutePath()); + tmpDir.getAbsolutePath()); Configurables.configure(source, context); source.start(); @@ -431,7 +435,7 @@ public void testEndWithZeroByteFiles() throws IOException, InterruptedException Assert.assertFalse("Server did not error", source.hasFatalError()); Assert.assertEquals("One message was read", - 1, source.getSourceCounter().getEventAcceptedCount()); + 1, source.getSourceCounter().getEventAcceptedCount()); source.stop(); } } From c1fae53bf2c254f5cf1282f32116a7aedc5340ed Mon Sep 17 00:00:00 2001 From: Roshan Naik Date: Tue, 9 Aug 2016 15:56:39 -0700 Subject: [PATCH 317/341] FLUME-2901. Document Kerberos setup for Kafka channel Add details about configuring Kafka channel to work with a Kerberized Kafka cluster. (Roshan Naik via Mike Percy) --- flume-ng-doc/sphinx/FlumeUserGuide.rst | 95 ++++++++++++++++++-------- 1 file changed, 68 insertions(+), 27 deletions(-) diff --git a/flume-ng-doc/sphinx/FlumeUserGuide.rst b/flume-ng-doc/sphinx/FlumeUserGuide.rst index fde9ff7b04..1c15f1e303 100644 --- a/flume-ng-doc/sphinx/FlumeUserGuide.rst +++ b/flume-ng-doc/sphinx/FlumeUserGuide.rst @@ -2715,31 +2715,35 @@ is logged on startup when they are present in the configuration file. Required properties are in **bold**. -================================ ========================== =============================================================================================================== -Property Name Default Description -================================ ========================== =============================================================================================================== -**type** -- The component type name, needs to be ``org.apache.flume.channel.kafka.KafkaChannel`` -**kafka.bootstrap.servers** -- List of brokers in the Kafka cluster used by the channel - This can be a partial list of brokers, but we recommend at least two for HA. - The format is comma separated list of hostname:port -kafka.topic flume-channel Kafka topic which the channel will use -kafka.consumer.group.id flume Consumer group ID the channel uses to register with Kafka. - Multiple channels must use the same topic and group to ensure that when one agent fails another can get the data - Note that having non-channel consumers with the same ID can lead to data loss. - -parseAsFlumeEvent true Expecting Avro datums with FlumeEvent schema in the channel. - This should be true if Flume source is writing to the channel and false if other producers are - writing into the topic that the channel is using. Flume source messages to Kafka can be parsed outside of Flume by using - org.apache.flume.source.avro.AvroFlumeEvent provided by the flume-ng-sdk artifact -pollTimeout 500 The amount of time(in milliseconds) to wait in the "poll()" call of the conumer. - https://kafka.apache.org/090/javadoc/org/apache/kafka/clients/consumer/KafkaConsumer.html#poll(long) -kafka.consumer.auto.offset.reset latest What to do when there is no initial offset in Kafka or if the current offset does not exist any more on the server - (e.g. because that data has been deleted): - earliest: automatically reset the offset to the earliest offset - latest: automatically reset the offset to the latest offset - none: throw exception to the consumer if no previous offset is found for the consumer\'s group - anything else: throw exception to the consumer. -================================ ========================== =============================================================================================================== +======================================= ========================== =============================================================================================================== +Property Name Default Description +======================================= ========================== =============================================================================================================== +**type** -- The component type name, needs to be ``org.apache.flume.channel.kafka.KafkaChannel`` +**kafka.bootstrap.servers** -- List of brokers in the Kafka cluster used by the channel + This can be a partial list of brokers, but we recommend at least two for HA. + The format is comma separated list of hostname:port +kafka.topic flume-channel Kafka topic which the channel will use +kafka.consumer.group.id flume Consumer group ID the channel uses to register with Kafka. + Multiple channels must use the same topic and group to ensure that when one agent fails another can get the data + Note that having non-channel consumers with the same ID can lead to data loss. + +parseAsFlumeEvent true Expecting Avro datums with FlumeEvent schema in the channel. + This should be true if Flume source is writing to the channel and false if other producers are + writing into the topic that the channel is using. Flume source messages to Kafka can be parsed outside of Flume by using + org.apache.flume.source.avro.AvroFlumeEvent provided by the flume-ng-sdk artifact +pollTimeout 500 The amount of time(in milliseconds) to wait in the "poll()" call of the conumer. + https://kafka.apache.org/090/javadoc/org/apache/kafka/clients/consumer/KafkaConsumer.html#poll(long) +kafka.consumer.auto.offset.reset latest What to do when there is no initial offset in Kafka or if the current offset does not exist any more on the server + (e.g. because that data has been deleted): + earliest: automatically reset the offset to the earliest offset + latest: automatically reset the offset to the latest offset + none: throw exception to the consumer if no previous offset is found for the consumer\'s group + anything else: throw exception to the consumer. +kafka.producer.security.protocol PLAINTEXT Set to SASL_PLAINTEXT, SASL_SSL or SSL if writing to Kafka using Kerberos. See below for additional info on Kerberos setup. +kafka.consumer.security.protocol PLAINTEXT Same as kafka.producer.security.protocol but for reading/consuming from Kafka. +*more producer/consumer security props* If using SASL_SSL or SSL, refer to `Kafka security `_ for additional + properties that need to be set on producer/consumer. +======================================= ========================== =============================================================================================================== Deprecated Properties @@ -2762,12 +2766,49 @@ Example for agent named a1: .. code-block:: properties a1.channels.channel1.type = org.apache.flume.channel.kafka.KafkaChannel - a1.channels.channel1.capacity = 10000 - a1.channels.channel1.transactionCapacity = 1000 a1.channels.channel1.kafka.bootstrap.servers = kafka-1:9092,kafka-2:9092,kafka-3:9092 a1.channels.channel1.kafka.topic = channel1 a1.channels.channel1.kafka.consumer.group.id = flume-consumer +**Kerberos and Kafka Channel:** + +To use Kafka channel with a Kafka cluster secured with Kerberos, set the producer/consumer.security.protocol properties noted above for producer and/or consumer. +The Kerberos keytab and principal to be used is specified in a JAAS file's "KafkaClient" section. See `Kafka doc `_ +for info on the JAAS file contents. The location of this JAAS file is specified via JAVA_OPTS using -Djava.security.auth.login.config=/path/to/kafka_jaas.conf (in flume-env.sh) + + +Sample secure configuration using SASL_PLAINTEXT. + +.. code-block:: properties + + a1.channels.channel1.type = org.apache.flume.channel.kafka.KafkaChannel + a1.channels.channel1.kafka.bootstrap.servers = kafka-1:9092,kafka-2:9092,kafka-3:9092 + a1.channels.channel1.kafka.topic = channel1 + a1.channels.channel1.kafka.consumer.group.id = flume-consumer + a1.channels.channel1.kafka.producer.security.protocol = SASL_PLAINTEXT + a1.channels.channel1.kafka.consumer.security.protocol = SASL_PLAINTEXT + +Sample JAAS file + +.. code-block:: javascript + + KafkaClient { + com.sun.security.auth.module.Krb5LoginModule required + useKeyTab=true + storeKey=true + serviceName="kafka" + keyTab="/path/to/keytabs/testuser1.keytab" + principal="testuser1/kafka1.example.com"; + }; + +Sample flume-env.sh + +.. code-block:: properties + + export JAVA_HOME=/path/java-home/ + export JAVA_OPTS="-Djava.security.auth.login.config=/path/to/kafka_jaas.conf" + + File Channel ~~~~~~~~~~~~ From b6dede8c108e0d8e3516767de31e3a847c69cfe0 Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?Bessenyei=20Bal=C3=A1zs=20Don=C3=A1t?= Date: Wed, 17 Aug 2016 09:52:21 -0700 Subject: [PATCH 318/341] FLUME-2318: Make SpoolingDirectorySource able to handle empty files MIME-Version: 1.0 Content-Type: text/plain; charset=UTF-8 Content-Transfer-Encoding: 8bit (Muhammad Ehsan ul Haque and Bessenyei Balázs Donát via Mike Percy) --- .../avro/ReliableSpoolingFileEventReader.java | 73 ++++++++++------ .../source/TestSpoolDirectorySource.java | 83 ++++++++++++++++--- 2 files changed, 120 insertions(+), 36 deletions(-) diff --git a/flume-ng-core/src/main/java/org/apache/flume/client/avro/ReliableSpoolingFileEventReader.java b/flume-ng-core/src/main/java/org/apache/flume/client/avro/ReliableSpoolingFileEventReader.java index 01381a520c..a0f929cd9e 100644 --- a/flume-ng-core/src/main/java/org/apache/flume/client/avro/ReliableSpoolingFileEventReader.java +++ b/flume-ng-core/src/main/java/org/apache/flume/client/avro/ReliableSpoolingFileEventReader.java @@ -28,6 +28,7 @@ import org.apache.flume.FlumeException; import org.apache.flume.annotations.InterfaceAudience; import org.apache.flume.annotations.InterfaceStability; +import org.apache.flume.event.EventBuilder; import org.apache.flume.serialization.DecodeErrorPolicy; import org.apache.flume.serialization.DurablePositionTracker; import org.apache.flume.serialization.EventDeserializer; @@ -58,20 +59,18 @@ import java.util.regex.Pattern; /** - *

    A {@link ReliableEventReader} which reads log data from files stored + *

    A {@link ReliableEventReader} which reads log data from files stored * in a spooling directory and renames each file once all of its data has been * read (through {@link EventDeserializer#readEvent()} calls). The user must * {@link #commit()} each read, to indicate that the lines have been fully * processed. - *

    Read calls will return no data if there are no files left to read. This + *

    Read calls will return no data if there are no files left to read. This * class, in general, is not thread safe. - * - *

    This reader assumes that files with unique file names are left in the + *

    This reader assumes that files with unique file names are left in the * spooling directory and not modified once they are placed there. Any user * behavior which violates these assumptions, when detected, will result in a * FlumeException being thrown. - * - *

    This class makes the following guarantees, if above assumptions are met: + *

    This class makes the following guarantees, if above assumptions are met: *

      *
    • Once a log file has been renamed with the {@link #completedSuffix}, * all of its records have been read through the @@ -106,11 +105,12 @@ public class ReliableSpoolingFileEventReader implements ReliableEventReader { private final boolean recursiveDirectorySearch; private Optional currentFile = Optional.absent(); - /** Always contains the last file from which lines have been read. **/ + /** Always contains the last file from which lines have been read. */ private Optional lastFileRead = Optional.absent(); private boolean committed = true; + private boolean firstTimeRead = true; - /** Instance var to Cache directory listing **/ + /** Instance var to Cache directory listing */ private Iterator candidateFileIter = null; private int listFilesCount = 0; @@ -220,6 +220,7 @@ private ReliableSpoolingFileEventReader(File spoolDirectory, /** * Recursively gather candidate files + * * @param directory the directory to gather files from * @return list of files within the passed in directory */ @@ -269,9 +270,11 @@ int getListFilesCount() { return listFilesCount; } - /** Return the filename which generated the data from the last successful + /** + * Return the filename which generated the data from the last successful * {@link #readEvents(int)} call. Returns null if called before any file - * contents are read. */ + * contents are read. + */ public String getLastFileRead() { if (!lastFileRead.isPresent()) { return null; @@ -308,8 +311,7 @@ public List readEvents(int numEvents) throws IOException { } } - EventDeserializer des = currentFile.get().getDeserializer(); - List events = des.readEvents(numEvents); + List events = readDeserializerEvents(numEvents); /* It's possible that the last read took us just up to a file boundary. * If so, try to roll to the next file, if there is one. @@ -322,9 +324,27 @@ public List readEvents(int numEvents) throws IOException { if (!currentFile.isPresent()) { return Collections.emptyList(); } - events = currentFile.get().getDeserializer().readEvents(numEvents); + events = readDeserializerEvents(numEvents); + } + + fillHeader(events); + + committed = false; + lastFileRead = currentFile; + return events; + } + + private List readDeserializerEvents(int numEvents) throws IOException { + EventDeserializer des = currentFile.get().getDeserializer(); + List events = des.readEvents(numEvents); + if (events.isEmpty() && firstTimeRead) { + events.add(EventBuilder.withBody(new byte[0])); } + firstTimeRead = false; + return events; + } + private void fillHeader(List events) { if (annotateFileName) { String filename = currentFile.get().getFile().getAbsolutePath(); for (Event event : events) { @@ -338,10 +358,6 @@ public List readEvents(int numEvents) throws IOException { event.getHeaders().put(baseNameHeader, basename); } } - - committed = false; - lastFileRead = currentFile; - return events; } @Override @@ -352,7 +368,9 @@ public void close() throws IOException { } } - /** Commit the last lines which were read. */ + /** + * Commit the last lines which were read. + */ @Override public void commit() throws IOException { if (!committed && currentFile.isPresent()) { @@ -363,11 +381,12 @@ public void commit() throws IOException { /** * Closes currentFile and attempt to rename it. - * + *

      * If these operations fail in a way that may cause duplicate log entries, * an error is logged but no exceptions are thrown. If these operations fail * in a way that indicates potential misuse of the spooling directory, a * FlumeException will be thrown. + * * @throws FlumeException if files do not conform to spooling assumptions */ private void retireCurrentFile() throws IOException { @@ -400,6 +419,7 @@ private void retireCurrentFile() throws IOException { /** * Rename the given spooled file + * * @param fileToRoll * @throws IOException */ @@ -432,13 +452,13 @@ private void rollCurrentFile(File fileToRoll) throws IOException { throw new IllegalStateException(message); } - // Dest file exists and not on windows + // Dest file exists and not on windows } else if (dest.exists()) { String message = "File name has been re-used with different" + " files. Spooling assumptions violated for " + dest; throw new IllegalStateException(message); - // Destination file does not already exist. We are good to go! + // Destination file does not already exist. We are good to go! } else { boolean renamed = fileToRoll.renameTo(dest); if (renamed) { @@ -460,6 +480,7 @@ private void rollCurrentFile(File fileToRoll) throws IOException { /** * Delete the given spooled file + * * @param fileToDelete * @throws IOException */ @@ -508,7 +529,7 @@ private Optional getNextFile() { if (consumeOrder == ConsumeOrder.RANDOM) { // Selected file is random. return openFile(selectedFile); } else if (consumeOrder == ConsumeOrder.YOUNGEST) { - for (File candidateFile: candidateFiles) { + for (File candidateFile : candidateFiles) { long compare = selectedFile.lastModified() - candidateFile.lastModified(); if (compare == 0) { // ts is same pick smallest lexicographically. @@ -518,7 +539,7 @@ private Optional getNextFile() { } } } else { // default order is OLDEST - for (File candidateFile: candidateFiles) { + for (File candidateFile : candidateFiles) { long compare = selectedFile.lastModified() - candidateFile.lastModified(); if (compare == 0) { // ts is same pick smallest lexicographically. @@ -529,6 +550,8 @@ private Optional getNextFile() { } } + firstTimeRead = true; + return openFile(selectedFile); } @@ -538,13 +561,15 @@ private File smallerLexicographical(File f1, File f2) { } return f2; } + /** * Opens a file for consuming + * * @param file * @return {@link FileInfo} for the file to consume or absent option if the * file does not exists or readable. */ - private Optional openFile(File file) { + private Optional openFile(File file) { try { // roll the meta file, if needed String nextPath = file.getPath(); diff --git a/flume-ng-core/src/test/java/org/apache/flume/source/TestSpoolDirectorySource.java b/flume-ng-core/src/test/java/org/apache/flume/source/TestSpoolDirectorySource.java index 0182d2120f..92a698d287 100644 --- a/flume-ng-core/src/test/java/org/apache/flume/source/TestSpoolDirectorySource.java +++ b/flume-ng-core/src/test/java/org/apache/flume/source/TestSpoolDirectorySource.java @@ -120,8 +120,8 @@ public void testPutFilenameHeader() throws IOException, InterruptedException { File f1 = new File(tmpDir.getAbsolutePath() + "/file1"); Files.write("file1line1\nfile1line2\nfile1line3\nfile1line4\n" + - "file1line5\nfile1line6\nfile1line7\nfile1line8\n", - f1, Charsets.UTF_8); + "file1line5\nfile1line6\nfile1line7\nfile1line8\n", + f1, Charsets.UTF_8); context.put(SpoolDirectorySourceConfigurationConstants.SPOOL_DIRECTORY, tmpDir.getAbsolutePath()); @@ -156,8 +156,8 @@ public void testPutBasenameHeader() throws IOException, InterruptedException { File f1 = new File(tmpDir.getAbsolutePath() + "/file1"); Files.write("file1line1\nfile1line2\nfile1line3\nfile1line4\n" + - "file1line5\nfile1line6\nfile1line7\nfile1line8\n", - f1, Charsets.UTF_8); + "file1line5\nfile1line6\nfile1line7\nfile1line8\n", + f1, Charsets.UTF_8); context.put(SpoolDirectorySourceConfigurationConstants.SPOOL_DIRECTORY, tmpDir.getAbsolutePath()); @@ -256,7 +256,6 @@ public void testRecursion_SetToFalse() throws IOException, InterruptedException boolean directoriesCreated = subDir.mkdirs(); Assert.assertTrue("source directories must be created", directoriesCreated); - File f1 = new File(subDir.getAbsolutePath() + "/file1.txt"); Files.write("file1line1\nfile1line2\nfile1line3\nfile1line4\n" + @@ -364,9 +363,8 @@ public void testSourceDoesNotDieOnFullChannel() throws Exception { File f1 = new File(tmpDir.getAbsolutePath() + "/file1"); Files.write("file1line1\nfile1line2\nfile1line3\nfile1line4\n" + - "file1line5\nfile1line6\nfile1line7\nfile1line8\n", - f1, Charsets.UTF_8); - + "file1line5\nfile1line6\nfile1line7\nfile1line8\n", + f1, Charsets.UTF_8); context.put(SpoolDirectorySourceConfigurationConstants.SPOOL_DIRECTORY, tmpDir.getAbsolutePath()); @@ -384,8 +382,7 @@ public void testSourceDoesNotDieOnFullChannel() throws Exception { } Assert.assertTrue("Expected to hit ChannelFullException, but did not!", - source.didHitChannelFullException()); - + source.didHitChannelFullException()); List dataOut = Lists.newArrayList(); @@ -434,8 +431,70 @@ public void testEndWithZeroByteFiles() throws IOException, InterruptedException Thread.sleep(5000); Assert.assertFalse("Server did not error", source.hasFatalError()); - Assert.assertEquals("One message was read", - 1, source.getSourceCounter().getEventAcceptedCount()); + Assert.assertEquals("Four messages were read", + 4, source.getSourceCounter().getEventAcceptedCount()); + source.stop(); + } + + @Test + public void testWithAllEmptyFiles() + throws InterruptedException, IOException { + Context context = new Context(); + File[] f = new File[10]; + for (int i = 0; i < 10; i++) { + f[i] = new File(tmpDir.getAbsolutePath() + "/file" + i); + Files.write(new byte[0], f[i]); + } + context.put(SpoolDirectorySourceConfigurationConstants.SPOOL_DIRECTORY, + tmpDir.getAbsolutePath()); + context.put(SpoolDirectorySourceConfigurationConstants.FILENAME_HEADER, + "true"); + context.put(SpoolDirectorySourceConfigurationConstants.FILENAME_HEADER_KEY, + "fileHeaderKeyTest"); + Configurables.configure(source, context); + source.start(); + Thread.sleep(10); + for (int i = 0; i < 10; i++) { + Transaction txn = channel.getTransaction(); + txn.begin(); + Event e = channel.take(); + Assert.assertNotNull("Event must not be null", e); + Assert.assertNotNull("Event headers must not be null", e.getHeaders()); + Assert.assertNotNull(e.getHeaders().get("fileHeaderKeyTest")); + Assert.assertEquals(f[i].getAbsolutePath(), + e.getHeaders().get("fileHeaderKeyTest")); + Assert.assertArrayEquals(new byte[0], e.getBody()); + txn.commit(); + txn.close(); + } + source.stop(); + } + + @Test + public void testWithEmptyAndDataFiles() + throws InterruptedException, IOException { + Context context = new Context(); + File f1 = new File(tmpDir.getAbsolutePath() + "/file1"); + Files.write("some data".getBytes(), f1); + File f2 = new File(tmpDir.getAbsolutePath() + "/file2"); + Files.write(new byte[0], f2); + context.put(SpoolDirectorySourceConfigurationConstants.SPOOL_DIRECTORY, + tmpDir.getAbsolutePath()); + Configurables.configure(source, context); + source.start(); + Thread.sleep(10); + for (int i = 0; i < 2; i++) { + Transaction txn = channel.getTransaction(); + txn.begin(); + Event e = channel.take(); + txn.commit(); + txn.close(); + } + Transaction txn = channel.getTransaction(); + txn.begin(); + Assert.assertNull(channel.take()); + txn.commit(); + txn.close(); source.stop(); } } From 118d5a4bcf5371fb215e0f7393fadf26073643f8 Mon Sep 17 00:00:00 2001 From: Denes Arvay Date: Wed, 17 Aug 2016 11:00:16 -0700 Subject: [PATCH 319/341] FLUME-2844. SpillableMemoryChannel must start ChannelCounter MIME-Version: 1.0 Content-Type: text/plain; charset=UTF-8 Content-Transfer-Encoding: 8bit Reviewed by Bessenyei Balázs Donát (Denes Arvay via Mike Percy) --- .../java/org/apache/flume/channel/SpillableMemoryChannel.java | 1 + .../org/apache/flume/channel/TestSpillableMemoryChannel.java | 3 +++ 2 files changed, 4 insertions(+) diff --git a/flume-ng-channels/flume-spillable-memory-channel/src/main/java/org/apache/flume/channel/SpillableMemoryChannel.java b/flume-ng-channels/flume-spillable-memory-channel/src/main/java/org/apache/flume/channel/SpillableMemoryChannel.java index b46d6469fc..09d7f295b2 100644 --- a/flume-ng-channels/flume-spillable-memory-channel/src/main/java/org/apache/flume/channel/SpillableMemoryChannel.java +++ b/flume-ng-channels/flume-spillable-memory-channel/src/main/java/org/apache/flume/channel/SpillableMemoryChannel.java @@ -790,6 +790,7 @@ public synchronized void start() { drainOrder.putOverflow(overFlowCount); totalStored.release(overFlowCount); } + channelCounter.start(); int totalCount = overFlowCount + memQueue.size(); channelCounter.setChannelCapacity(memoryCapacity + getOverflowCapacity()); channelCounter.setChannelSize(totalCount); diff --git a/flume-ng-channels/flume-spillable-memory-channel/src/test/java/org/apache/flume/channel/TestSpillableMemoryChannel.java b/flume-ng-channels/flume-spillable-memory-channel/src/test/java/org/apache/flume/channel/TestSpillableMemoryChannel.java index 848636b766..ab90c3d3ef 100644 --- a/flume-ng-channels/flume-spillable-memory-channel/src/test/java/org/apache/flume/channel/TestSpillableMemoryChannel.java +++ b/flume-ng-channels/flume-spillable-memory-channel/src/test/java/org/apache/flume/channel/TestSpillableMemoryChannel.java @@ -682,6 +682,9 @@ public void testCounters() throws InterruptedException { params.put("overflowTimeout", "0"); startChannel(params); + Assert.assertTrue("channel.channelCounter should have started", + channel.channelCounter.getStartTime() > 0); + //1. fill up mem queue Thread sourceThd = makePutThread("src", 1, 5000, 2500, channel); sourceThd.start(); From e0deb7281fce0f6937f20da305b58d52d9a67a94 Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?Bessenyei=20Bal=C3=A1zs=20Don=C3=A1t?= Date: Sun, 21 Aug 2016 12:34:52 -0700 Subject: [PATCH 320/341] FLUME-2975. docs: Fix NetcatSource example MIME-Version: 1.0 Content-Type: text/plain; charset=UTF-8 Content-Transfer-Encoding: 8bit (Bessenyei Balázs Donát via Mike Percy) --- flume-ng-doc/sphinx/FlumeUserGuide.rst | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/flume-ng-doc/sphinx/FlumeUserGuide.rst b/flume-ng-doc/sphinx/FlumeUserGuide.rst index 1c15f1e303..5e677c6c0b 100644 --- a/flume-ng-doc/sphinx/FlumeUserGuide.rst +++ b/flume-ng-doc/sphinx/FlumeUserGuide.rst @@ -1311,7 +1311,7 @@ Example for agent named a1: a1.channels = c1 a1.sources.r1.type = netcat a1.sources.r1.bind = 0.0.0.0 - a1.sources.r1.bind = 6666 + a1.sources.r1.port = 6666 a1.sources.r1.channels = c1 Sequence Generator Source From 9eb92dab0d24813e9a3871f0c7038920797c0466 Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?Bessenyei=20Bal=C3=A1zs=20Don=C3=A1t?= Date: Mon, 22 Aug 2016 13:29:17 -0700 Subject: [PATCH 321/341] FLUME-2974. Fix some broken SpoolDirSource unit tests MIME-Version: 1.0 Content-Type: text/plain; charset=UTF-8 Content-Transfer-Encoding: 8bit TestReliableSpoolingFileEventReader and TestSpoolingFileLineReader were broken as part of commit b6dede8c108e0d8e3516767de31e3a847c69cfe0. This patch fixes them again. (Bessenyei Balázs Donát via Mike Percy) --- .../client/avro/TestReliableSpoolingFileEventReader.java | 5 +++-- .../apache/flume/client/avro/TestSpoolingFileLineReader.java | 5 ++++- 2 files changed, 7 insertions(+), 3 deletions(-) diff --git a/flume-ng-core/src/test/java/org/apache/flume/client/avro/TestReliableSpoolingFileEventReader.java b/flume-ng-core/src/test/java/org/apache/flume/client/avro/TestReliableSpoolingFileEventReader.java index fdc3ce94b0..59fb1c39a3 100644 --- a/flume-ng-core/src/test/java/org/apache/flume/client/avro/TestReliableSpoolingFileEventReader.java +++ b/flume-ng-core/src/test/java/org/apache/flume/client/avro/TestReliableSpoolingFileEventReader.java @@ -144,7 +144,7 @@ public void testRepeatedCallsWithCommitAlways() throws IOException { new ReliableSpoolingFileEventReader.Builder().spoolDirectory(WORK_DIR) .build(); - final int expectedLines = 0 + 1 + 2 + 3 + 1; + final int expectedLines = 1 + 1 + 2 + 3 + 1; int seenLines = 0; for (int i = 0; i < 10; i++) { List events = reader.readEvents(10); @@ -166,7 +166,7 @@ public void testRepeatedCallsWithCommitOnSuccess() throws IOException { .trackerDirPath(trackerDirPath) .build(); - final int expectedLines = 0 + 1 + 2 + 3 + 1; + final int expectedLines = 1 + 1 + 2 + 3 + 1; int seenLines = 0; for (int i = 0; i < 10; i++) { List events = reader.readEvents(10); @@ -525,6 +525,7 @@ private void readEventsForFilesInDir(File dir, ReliableEventReader reader, } /* Create expected results out of the files created in the setup method. */ private void createExpectedFromFilesInSetup(Collection expected) { + expected.add(""); for (int i = 0; i < 4; i++) { for (int j = 0; j < i; j++) { expected.add("file" + i + "line" + j); diff --git a/flume-ng-core/src/test/java/org/apache/flume/client/avro/TestSpoolingFileLineReader.java b/flume-ng-core/src/test/java/org/apache/flume/client/avro/TestSpoolingFileLineReader.java index ac046a93b0..bc3aa82400 100644 --- a/flume-ng-core/src/test/java/org/apache/flume/client/avro/TestSpoolingFileLineReader.java +++ b/flume-ng-core/src/test/java/org/apache/flume/client/avro/TestSpoolingFileLineReader.java @@ -420,7 +420,10 @@ public void testBehaviorWithEmptyFile() throws IOException { "file1line5\nfile1line6\nfile1line7\nfile1line8\n", f2, Charsets.UTF_8); - // Expect to skip over first file + // Skip over first file, which is empty, and will return an empty event. + Event event = parser.readEvent(); + assertEquals(0, event.getBody().length); + List out = bodiesAsStrings(parser.readEvents(8)); parser.commit(); From 1e8f2651dacf5daef55d75c7b9b12492962e7921 Mon Sep 17 00:00:00 2001 From: Grant Henke Date: Thu, 25 Aug 2016 16:48:25 -0700 Subject: [PATCH 322/341] FLUME-2972. Handle offset migration in the new Kafka Channel MIME-Version: 1.0 Content-Type: text/plain; charset=UTF-8 Content-Transfer-Encoding: 8bit Offsets tracking the position in Kafka consumers change from using Zookeeper for offset storage to Kafka when moving from Kafka 0.8.x to 0.9.x. FLUME-2823 makes the client change in the Kafka Channel but does not ensure existing offsets get migrated in order to continue consuming where it left off. Flume should have some automated logic on startup to check if Kafka offsets exist, if not and migration is enabled (by default) then copy the offsets from Zookeeper and commit them to Kafka. Reviewers: Balázs Donát Bessenyei, Denes Arvay, Mike Percy (Grant Henke via Mike Percy) --- flume-ng-channels/flume-kafka-channel/pom.xml | 2 +- .../flume/channel/kafka/KafkaChannel.java | 103 +++++++++++- .../kafka/KafkaChannelConfiguration.java | 3 + .../flume/channel/kafka/TestKafkaChannel.java | 148 +++++++++++++++--- flume-ng-doc/sphinx/FlumeUserGuide.rst | 4 + 5 files changed, 234 insertions(+), 26 deletions(-) diff --git a/flume-ng-channels/flume-kafka-channel/pom.xml b/flume-ng-channels/flume-kafka-channel/pom.xml index 587b4b4f37..c1cc844fad 100644 --- a/flume-ng-channels/flume-kafka-channel/pom.xml +++ b/flume-ng-channels/flume-kafka-channel/pom.xml @@ -40,7 +40,7 @@ limitations under the License. org.apache.kafka kafka_2.10 - test + ${kafka.version} org.apache.kafka diff --git a/flume-ng-channels/flume-kafka-channel/src/main/java/org/apache/flume/channel/kafka/KafkaChannel.java b/flume-ng-channels/flume-kafka-channel/src/main/java/org/apache/flume/channel/kafka/KafkaChannel.java index 90e3288a5e..684120f55d 100644 --- a/flume-ng-channels/flume-kafka-channel/src/main/java/org/apache/flume/channel/kafka/KafkaChannel.java +++ b/flume-ng-channels/flume-kafka-channel/src/main/java/org/apache/flume/channel/kafka/KafkaChannel.java @@ -20,6 +20,8 @@ import com.google.common.annotations.VisibleForTesting; import com.google.common.base.Optional; +import kafka.utils.ZKGroupTopicDirs; +import kafka.utils.ZkUtils; import org.apache.avro.io.BinaryDecoder; import org.apache.avro.io.BinaryEncoder; import org.apache.avro.io.DecoderFactory; @@ -47,9 +49,12 @@ import org.apache.kafka.clients.producer.ProducerConfig; import org.apache.kafka.clients.producer.ProducerRecord; import org.apache.kafka.clients.producer.RecordMetadata; +import org.apache.kafka.common.PartitionInfo; import org.apache.kafka.common.TopicPartition; +import org.apache.kafka.common.security.JaasUtils; import org.slf4j.Logger; import org.slf4j.LoggerFactory; +import scala.Option; import java.io.ByteArrayInputStream; import java.io.ByteArrayOutputStream; @@ -70,12 +75,17 @@ import java.util.concurrent.atomic.AtomicReference; import static org.apache.flume.channel.kafka.KafkaChannelConfiguration.*; +import static scala.collection.JavaConverters.asJavaListConverter; public class KafkaChannel extends BasicChannelSemantics { private static final Logger logger = LoggerFactory.getLogger(KafkaChannel.class); + // Constants used only for offset migration zookeeper connections + private static final int ZK_SESSION_TIMEOUT = 30000; + private static final int ZK_CONNECTION_TIMEOUT = 30000; + private final Properties consumerProps = new Properties(); private final Properties producerProps = new Properties(); @@ -84,6 +94,10 @@ public class KafkaChannel extends BasicChannelSemantics { private AtomicReference topic = new AtomicReference(); private boolean parseAsFlumeEvent = DEFAULT_PARSE_AS_FLUME_EVENT; + private String zookeeperConnect = null; + private String topicStr = DEFAULT_TOPIC; + private String groupId = DEFAULT_GROUP_ID; + private boolean migrateZookeeperOffsets = DEFAULT_MIGRATE_ZOOKEEPER_OFFSETS; //used to indicate if a rebalance has occurred during the current transaction AtomicBoolean rebalanceFlag = new AtomicBoolean(); @@ -113,6 +127,11 @@ public ConsumerAndRecords initialValue() { @Override public void start() { logger.info("Starting Kafka Channel: {}", getName()); + // As a migration step check if there are any offsets from the group stored in kafka + // If not read them from Zookeeper and commit them to Kafka + if (migrateZookeeperOffsets && zookeeperConnect != null && !zookeeperConnect.isEmpty()) { + migrateOffsets(); + } producer = new KafkaProducer(producerProps); // We always have just one topic being read by one thread logger.info("Topic = {}", topic.get()); @@ -147,12 +166,19 @@ public void configure(Context ctx) { //Can remove in the next release translateOldProps(ctx); - String topicStr = ctx.getString(TOPIC_CONFIG); + topicStr = ctx.getString(TOPIC_CONFIG); if (topicStr == null || topicStr.isEmpty()) { topicStr = DEFAULT_TOPIC; logger.info("Topic was not specified. Using {} as the topic.", topicStr); } topic.set(topicStr); + + groupId = ctx.getString(KAFKA_CONSUMER_PREFIX + ConsumerConfig.GROUP_ID_CONFIG); + if (groupId == null || groupId.isEmpty()) { + groupId = DEFAULT_GROUP_ID; + logger.info("Group ID was not specified. Using {} as the group id.", groupId); + } + String bootStrapServers = ctx.getString(BOOTSTRAP_SERVERS_CONFIG); if (bootStrapServers == null || bootStrapServers.isEmpty()) { throw new ConfigurationException("Bootstrap Servers must be specified"); @@ -164,6 +190,10 @@ public void configure(Context ctx) { parseAsFlumeEvent = ctx.getBoolean(PARSE_AS_FLUME_EVENT, DEFAULT_PARSE_AS_FLUME_EVENT); pollTimeout = ctx.getLong(POLL_TIMEOUT, DEFAULT_POLL_TIMEOUT); + migrateZookeeperOffsets = ctx.getBoolean(MIGRATE_ZOOKEEPER_OFFSETS, + DEFAULT_MIGRATE_ZOOKEEPER_OFFSETS); + zookeeperConnect = ctx.getString(ZOOKEEPER_CONNECT); + if (counter == null) { counter = new KafkaChannelCounter(getName()); } @@ -235,11 +265,6 @@ protected Properties getProducerProps() { } private void setConsumerProps(Context ctx, String bootStrapServers) { - String groupId = ctx.getString(KAFKA_CONSUMER_PREFIX + ConsumerConfig.GROUP_ID_CONFIG); - if (groupId == null || groupId.isEmpty()) { - groupId = DEFAULT_GROUP_ID; - logger.info("Group ID was not specified. Using {} as the group id.", groupId); - } consumerProps.put(ConsumerConfig.KEY_DESERIALIZER_CLASS_CONFIG, DEFAULT_KEY_DESERIALIZER); consumerProps.put(ConsumerConfig.VALUE_DESERIALIZER_CLASS_CONFIG, DEFAULT_VALUE_DESERIAIZER); consumerProps.put(ConsumerConfig.AUTO_OFFSET_RESET_CONFIG, DEFAULT_AUTO_OFFSET_RESET); @@ -272,6 +297,72 @@ private synchronized ConsumerAndRecords createConsumerAndRecords() { } } + private void migrateOffsets() { + ZkUtils zkUtils = ZkUtils.apply(zookeeperConnect, ZK_SESSION_TIMEOUT, ZK_CONNECTION_TIMEOUT, + JaasUtils.isZkSecurityEnabled()); + KafkaConsumer consumer = new KafkaConsumer<>(consumerProps); + try { + Map kafkaOffsets = getKafkaOffsets(consumer); + if (!kafkaOffsets.isEmpty()) { + logger.info("Found Kafka offsets for topic " + topicStr + + ". Will not migrate from zookeeper"); + logger.debug("Offsets found: {}", kafkaOffsets); + return; + } + + logger.info("No Kafka offsets found. Migrating zookeeper offsets"); + Map zookeeperOffsets = getZookeeperOffsets(zkUtils); + if (zookeeperOffsets.isEmpty()) { + logger.warn("No offsets to migrate found in Zookeeper"); + return; + } + + logger.info("Committing Zookeeper offsets to Kafka"); + logger.debug("Offsets to commit: {}", zookeeperOffsets); + consumer.commitSync(zookeeperOffsets); + // Read the offsets to verify they were committed + Map newKafkaOffsets = getKafkaOffsets(consumer); + logger.debug("Offsets committed: {}", newKafkaOffsets); + if (!newKafkaOffsets.keySet().containsAll(zookeeperOffsets.keySet())) { + throw new FlumeException("Offsets could not be committed"); + } + } finally { + zkUtils.close(); + consumer.close(); + } + } + + private Map getKafkaOffsets( + KafkaConsumer client) { + Map offsets = new HashMap<>(); + List partitions = client.partitionsFor(topicStr); + for (PartitionInfo partition : partitions) { + TopicPartition key = new TopicPartition(topicStr, partition.partition()); + OffsetAndMetadata offsetAndMetadata = client.committed(key); + if (offsetAndMetadata != null) { + offsets.put(key, offsetAndMetadata); + } + } + return offsets; + } + + private Map getZookeeperOffsets(ZkUtils client) { + Map offsets = new HashMap<>(); + ZKGroupTopicDirs topicDirs = new ZKGroupTopicDirs(groupId, topicStr); + List partitions = asJavaListConverter( + client.getChildrenParentMayNotExist(topicDirs.consumerOffsetDir())).asJava(); + for (String partition : partitions) { + TopicPartition key = new TopicPartition(topicStr, Integer.valueOf(partition)); + Option data = client.readDataMaybeNull( + topicDirs.consumerOffsetDir() + "/" + partition)._1(); + if (data.isDefined()) { + Long offset = Long.valueOf(data.get()); + offsets.put(key, new OffsetAndMetadata(offset)); + } + } + return offsets; + } + private void decommissionConsumerAndRecords(ConsumerAndRecords c) { c.consumer.close(); } diff --git a/flume-ng-channels/flume-kafka-channel/src/main/java/org/apache/flume/channel/kafka/KafkaChannelConfiguration.java b/flume-ng-channels/flume-kafka-channel/src/main/java/org/apache/flume/channel/kafka/KafkaChannelConfiguration.java index ccf46d9132..3ab807b53c 100644 --- a/flume-ng-channels/flume-kafka-channel/src/main/java/org/apache/flume/channel/kafka/KafkaChannelConfiguration.java +++ b/flume-ng-channels/flume-kafka-channel/src/main/java/org/apache/flume/channel/kafka/KafkaChannelConfiguration.java @@ -49,6 +49,9 @@ public class KafkaChannelConfiguration { public static final String PARSE_AS_FLUME_EVENT = "parseAsFlumeEvent"; public static final boolean DEFAULT_PARSE_AS_FLUME_EVENT = true; + public static final String MIGRATE_ZOOKEEPER_OFFSETS = "migrateZookeeperOffsets"; + public static final boolean DEFAULT_MIGRATE_ZOOKEEPER_OFFSETS = true; + /*** Old Configuration Parameters ****/ public static final String BROKER_LIST_KEY = "metadata.broker.list"; public static final String REQUIRED_ACKS_KEY = "request.required.acks"; diff --git a/flume-ng-channels/flume-kafka-channel/src/test/java/org/apache/flume/channel/kafka/TestKafkaChannel.java b/flume-ng-channels/flume-kafka-channel/src/test/java/org/apache/flume/channel/kafka/TestKafkaChannel.java index b63ac9b8c4..e7ae68f3a5 100644 --- a/flume-ng-channels/flume-kafka-channel/src/test/java/org/apache/flume/channel/kafka/TestKafkaChannel.java +++ b/flume-ng-channels/flume-kafka-channel/src/test/java/org/apache/flume/channel/kafka/TestKafkaChannel.java @@ -20,6 +20,7 @@ import com.google.common.collect.Lists; import kafka.admin.AdminUtils; +import kafka.utils.ZKGroupTopicDirs; import kafka.utils.ZkUtils; import org.apache.commons.lang.RandomStringUtils; import org.apache.flume.Context; @@ -30,8 +31,13 @@ import org.apache.flume.sink.kafka.util.TestUtil; import org.apache.kafka.clients.CommonClientConfigs; import org.apache.kafka.clients.consumer.ConsumerConfig; +import org.apache.kafka.clients.consumer.KafkaConsumer; +import org.apache.kafka.clients.consumer.OffsetAndMetadata; import org.apache.kafka.clients.producer.KafkaProducer; import org.apache.kafka.clients.producer.ProducerRecord; +import org.apache.kafka.clients.producer.RecordMetadata; +import org.apache.kafka.common.TopicPartition; +import org.apache.kafka.common.security.JaasUtils; import org.junit.AfterClass; import org.junit.Assert; import org.junit.Before; @@ -55,13 +61,7 @@ import java.util.concurrent.atomic.AtomicBoolean; import java.util.concurrent.atomic.AtomicInteger; -import static org.apache.flume.channel.kafka.KafkaChannelConfiguration.BOOTSTRAP_SERVERS_CONFIG; -import static org.apache.flume.channel.kafka.KafkaChannelConfiguration.BROKER_LIST_FLUME_KEY; -import static org.apache.flume.channel.kafka.KafkaChannelConfiguration.GROUP_ID_FLUME; -import static org.apache.flume.channel.kafka.KafkaChannelConfiguration.KEY_HEADER; -import static org.apache.flume.channel.kafka.KafkaChannelConfiguration.PARSE_AS_FLUME_EVENT; -import static org.apache.flume.channel.kafka.KafkaChannelConfiguration.READ_SMALLEST_OFFSET; -import static org.apache.flume.channel.kafka.KafkaChannelConfiguration.TOPIC_CONFIG; +import static org.apache.flume.channel.kafka.KafkaChannelConfiguration.*; public class TestKafkaChannel { @@ -77,16 +77,9 @@ public static void setupClass() throws Exception { @Before public void setup() throws Exception { - boolean topicFound = false; - while (!topicFound) { - topic = RandomStringUtils.randomAlphabetic(8); - if (!usedTopics.contains(topic)) { - usedTopics.add(topic); - topicFound = true; - } - } + topic = findUnusedTopic(); try { - createTopic(topic); + createTopic(topic, 5); } catch (Exception e) { } Thread.sleep(2500); @@ -235,6 +228,106 @@ public void testOffsetsNotCommittedOnStop() throws Exception { Assert.assertTrue(Arrays.equals(message.getBytes(), event.getBody())); } + @Test + public void testMigrateOffsetsNone() throws Exception { + doTestMigrateZookeeperOffsets(false, false, "testMigrateOffsets-none"); + } + + @Test + public void testMigrateOffsetsZookeeper() throws Exception { + doTestMigrateZookeeperOffsets(true, false, "testMigrateOffsets-zookeeper"); + } + + @Test + public void testMigrateOffsetsKafka() throws Exception { + doTestMigrateZookeeperOffsets(false, true, "testMigrateOffsets-kafka"); + } + + @Test + public void testMigrateOffsetsBoth() throws Exception { + doTestMigrateZookeeperOffsets(true, true, "testMigrateOffsets-both"); + } + + public void doTestMigrateZookeeperOffsets(boolean hasZookeeperOffsets, boolean hasKafkaOffsets, + String group) throws Exception { + // create a topic with 1 partition for simplicity + topic = findUnusedTopic(); + createTopic(topic, 1); + + Context context = prepareDefaultContext(false); + context.put(ZOOKEEPER_CONNECT, testUtil.getZkUrl()); + context.put(GROUP_ID_FLUME, group); + final KafkaChannel channel = createChannel(context); + + // Produce some data and save an offset + Long fifthOffset = 0L; + Long tenthOffset = 0L; + Properties props = channel.getProducerProps(); + KafkaProducer producer = new KafkaProducer<>(props); + for (int i = 1; i <= 50; i++) { + ProducerRecord data = + new ProducerRecord<>(topic, null, String.valueOf(i).getBytes()); + RecordMetadata recordMetadata = producer.send(data).get(); + if (i == 5) { + fifthOffset = recordMetadata.offset(); + } + if (i == 10) { + tenthOffset = recordMetadata.offset(); + } + } + + // Commit 10th offset to zookeeper + if (hasZookeeperOffsets) { + ZkUtils zkUtils = ZkUtils.apply(testUtil.getZkUrl(), 30000, 30000, + JaasUtils.isZkSecurityEnabled()); + ZKGroupTopicDirs topicDirs = new ZKGroupTopicDirs(group, topic); + // we commit the tenth offset to ensure some data is missed. + Long offset = tenthOffset + 1; + zkUtils.updatePersistentPath(topicDirs.consumerOffsetDir() + "/0", offset.toString(), + zkUtils.updatePersistentPath$default$3()); + zkUtils.close(); + } + + // Commit 5th offset to kafka + if (hasKafkaOffsets) { + Map offsets = new HashMap<>(); + offsets.put(new TopicPartition(topic, 0), new OffsetAndMetadata(fifthOffset + 1)); + KafkaConsumer consumer = new KafkaConsumer<>(channel.getConsumerProps()); + consumer.commitSync(offsets); + consumer.close(); + } + + // Start the channel and read some data + channel.start(); + ExecutorCompletionService submitterSvc = new + ExecutorCompletionService(Executors.newCachedThreadPool()); + List events = pullEvents(channel, submitterSvc, + 20, false, false); + wait(submitterSvc, 5); + List finals = new ArrayList(40); + for (Event event: events) { + finals.add(Integer.parseInt(new String(event.getBody()))); + } + channel.stop(); + + if (!hasKafkaOffsets && !hasZookeeperOffsets) { + // The default behavior is to read the entire log + Assert.assertTrue("Channel should read the the first message", finals.contains(1)); + } else if (hasKafkaOffsets && hasZookeeperOffsets) { + // Respect Kafka offsets if they exist + Assert.assertFalse("Channel should not read the 5th message", finals.contains(5)); + Assert.assertTrue("Channel should read the 6th message", finals.contains(6)); + } else if (hasKafkaOffsets) { + // Respect Kafka offsets if they exist (don't fail if zookeeper offsets are missing) + Assert.assertFalse("Channel should not read the 5th message", finals.contains(5)); + Assert.assertTrue("Channel should read the 6th message", finals.contains(6)); + } else { + // Otherwise migrate the ZooKeeper offsets if they exist + Assert.assertFalse("Channel should not read the 10th message", finals.contains(10)); + Assert.assertTrue("Channel should read the 11th message", finals.contains(11)); + } + } + private Event takeEventWithoutCommittingTxn(KafkaChannel channel) { for (int i = 0; i < 5; i++) { Transaction txn = channel.getTransaction(); @@ -396,9 +489,14 @@ private void doTestStopAndStart(boolean rollback, private KafkaChannel startChannel(boolean parseAsFlume) throws Exception { Context context = prepareDefaultContext(parseAsFlume); + KafkaChannel channel = createChannel(context); + channel.start(); + return channel; + } + + private KafkaChannel createChannel(Context context) throws Exception { final KafkaChannel channel = new KafkaChannel(); Configurables.configure(channel, context); - channel.start(); return channel; } @@ -585,8 +683,20 @@ private Context prepareDefaultContext(boolean parseAsFlume) { return context; } - public static void createTopic(String topicName) { - int numPartitions = 5; + public String findUnusedTopic() { + String newTopic = null; + boolean topicFound = false; + while (!topicFound) { + newTopic = RandomStringUtils.randomAlphabetic(8); + if (!usedTopics.contains(newTopic)) { + usedTopics.add(newTopic); + topicFound = true; + } + } + return newTopic; + } + + public static void createTopic(String topicName, int numPartitions) { int sessionTimeoutMs = 10000; int connectionTimeoutMs = 10000; ZkUtils zkUtils = diff --git a/flume-ng-doc/sphinx/FlumeUserGuide.rst b/flume-ng-doc/sphinx/FlumeUserGuide.rst index 5e677c6c0b..7e207aa825 100644 --- a/flume-ng-doc/sphinx/FlumeUserGuide.rst +++ b/flume-ng-doc/sphinx/FlumeUserGuide.rst @@ -2731,6 +2731,10 @@ parseAsFlumeEvent true Expecting A This should be true if Flume source is writing to the channel and false if other producers are writing into the topic that the channel is using. Flume source messages to Kafka can be parsed outside of Flume by using org.apache.flume.source.avro.AvroFlumeEvent provided by the flume-ng-sdk artifact +migrateZookeeperOffsets true When no Kafka stored offset is found, look up the offsets in Zookeeper and commit them to Kafka. + This should be true to support seamless Kafka client migration from older versions of Flume. Once migrated this can be set + to false, though that should generally not be required. If no Zookeeper offset is found the kafka.consumer.auto.offset.reset + configuration defines how offsets are handled. pollTimeout 500 The amount of time(in milliseconds) to wait in the "poll()" call of the conumer. https://kafka.apache.org/090/javadoc/org/apache/kafka/clients/consumer/KafkaConsumer.html#poll(long) kafka.consumer.auto.offset.reset latest What to do when there is no initial offset in Kafka or if the current offset does not exist any more on the server From 25e4bc6d80cf475862a1686fb2c3c97fcea27278 Mon Sep 17 00:00:00 2001 From: Attila Simon Date: Mon, 29 Aug 2016 12:15:56 -0700 Subject: [PATCH 323/341] FLUME-2954. Make raw data appearing in log messages explicit MIME-Version: 1.0 Content-Type: text/plain; charset=UTF-8 Content-Transfer-Encoding: 8bit Flume has built-in functionality to log data flowing through, mainly for debugging purposes. This functionality appears in several places in the code base. Such functionality can raise security concerns in production environments where sensitive information might be ingested so it is crucial that enabling such functionality be as explicit as possible. This patch adds two system properties, one to enable logging of Flume configuration properties and one to enable logging of raw data. If they are not set, these items are never logged at any log4j logging level. Reviewers: Balázs Donát Bessenyei, Denes Arvay, Mike Percy (Attila Simon via Mike Percy) --- conf/flume-env.ps1.template | 5 ++ conf/flume-env.sh.template | 5 ++ .../jdbc/impl/JdbcChannelProviderImpl.java | 9 +- .../flume/channel/kafka/KafkaChannel.java | 8 +- .../apache/flume/conf/FlumeConfiguration.java | 13 +-- .../org/apache/flume/conf/LogPrivacyUtil.java | 83 +++++++++++++++++++ .../org/apache/flume/source/AvroSource.java | 11 ++- .../source/MultiportSyslogTCPSource.java | 9 +- .../apache/flume/source/http/BLOBHandler.java | 5 +- .../SyslogAvroEventSerializer.java | 5 +- flume-ng-doc/sphinx/FlumeUserGuide.rst | 35 +++++++- .../flume/agent/embedded/EmbeddedAgent.java | 5 +- .../apache/flume/sink/kafka/KafkaSink.java | 20 +++-- .../sink/solr/morphline/BlobHandler.java | 20 +++-- .../sink/solr/morphline/MorphlineSink.java | 6 +- .../flume/source/kafka/KafkaSource.java | 25 ++++-- .../flume/source/twitter/TwitterSource.java | 5 -- 17 files changed, 214 insertions(+), 55 deletions(-) create mode 100644 flume-ng-configuration/src/main/java/org/apache/flume/conf/LogPrivacyUtil.java diff --git a/conf/flume-env.ps1.template b/conf/flume-env.ps1.template index 8bf535ad00..7cbc119714 100644 --- a/conf/flume-env.ps1.template +++ b/conf/flume-env.ps1.template @@ -18,6 +18,11 @@ # Give Flume more memory and pre-allocate, enable remote monitoring via JMX $JAVA_OPTS="-Xms100m -Xmx200m -Dcom.sun.management.jmxremote" +# Let Flume write raw event data and configuration information to its log files for debugging +# purposes. Enabling these flags is not recommended in production, +# as it may result in logging sensitive user information or encryption secrets. +# $JAVA_OPTS="$JAVA_OPTS -Dorg.apache.flume.log.rawdata=true -Dorg.apache.flume.log.printconfig=true " + # Foll. classpath will be included in Flume's classpath. # Note that the Flume conf directory is always included in the classpath. $FLUME_CLASSPATH="" # Example: "path1;path2;path3" diff --git a/conf/flume-env.sh.template b/conf/flume-env.sh.template index c8b660f171..07182ca406 100644 --- a/conf/flume-env.sh.template +++ b/conf/flume-env.sh.template @@ -24,6 +24,11 @@ # Give Flume more memory and pre-allocate, enable remote monitoring via JMX # export JAVA_OPTS="-Xms100m -Xmx2000m -Dcom.sun.management.jmxremote" +# Let Flume write raw event data and configuration information to its log files for debugging +# purposes. Enabling these flags is not recommended in production, +# as it may result in logging sensitive user information or encryption secrets. +# export JAVA_OPTS="$JAVA_OPTS -Dorg.apache.flume.log.rawdata=true -Dorg.apache.flume.log.printconfig=true " + # Note that the Flume conf directory is always included in the classpath. #FLUME_CLASSPATH="" diff --git a/flume-ng-channels/flume-jdbc-channel/src/main/java/org/apache/flume/channel/jdbc/impl/JdbcChannelProviderImpl.java b/flume-ng-channels/flume-jdbc-channel/src/main/java/org/apache/flume/channel/jdbc/impl/JdbcChannelProviderImpl.java index 845b794170..01caef3100 100644 --- a/flume-ng-channels/flume-jdbc-channel/src/main/java/org/apache/flume/channel/jdbc/impl/JdbcChannelProviderImpl.java +++ b/flume-ng-channels/flume-jdbc-channel/src/main/java/org/apache/flume/channel/jdbc/impl/JdbcChannelProviderImpl.java @@ -94,10 +94,7 @@ public class JdbcChannelProviderImpl implements JdbcChannelProvider { @Override public void initialize(Context context) { - if (LOGGER.isDebugEnabled()) { - LOGGER.debug("Initializing JDBC Channel provider with props: " - + context); - } + LOGGER.debug("Initializing JDBC Channel provider"); initializeSystemProperties(context); initializeDataSource(context); @@ -363,7 +360,7 @@ public JdbcTransactionImpl getTransaction() { /** * Initializes the datasource and the underlying connection pool. - * @param properties + * @param context */ private void initializeDataSource(Context context) { driverClassName = getConfigurationString(context, @@ -592,7 +589,7 @@ protected void updateCurrentChannelSize(long delta) { * @param context * @param key the expected configuration key * @param oldKey the deprecated configuration key - * @param default value, null if no default + * @param defaultValue default value, null if no default * @return the value associated with the key */ private String getConfigurationString(Context context, String key, diff --git a/flume-ng-channels/flume-kafka-channel/src/main/java/org/apache/flume/channel/kafka/KafkaChannel.java b/flume-ng-channels/flume-kafka-channel/src/main/java/org/apache/flume/channel/kafka/KafkaChannel.java index 684120f55d..e7f1f2e315 100644 --- a/flume-ng-channels/flume-kafka-channel/src/main/java/org/apache/flume/channel/kafka/KafkaChannel.java +++ b/flume-ng-channels/flume-kafka-channel/src/main/java/org/apache/flume/channel/kafka/KafkaChannel.java @@ -35,6 +35,7 @@ import org.apache.flume.channel.BasicChannelSemantics; import org.apache.flume.channel.BasicTransactionSemantics; import org.apache.flume.conf.ConfigurationException; +import org.apache.flume.conf.LogPrivacyUtil; import org.apache.flume.event.EventBuilder; import org.apache.flume.instrumentation.kafka.KafkaChannelCounter; import org.apache.flume.source.avro.AvroFlumeEvent; @@ -194,6 +195,10 @@ public void configure(Context ctx) { DEFAULT_MIGRATE_ZOOKEEPER_OFFSETS); zookeeperConnect = ctx.getString(ZOOKEEPER_CONNECT); + if (logger.isDebugEnabled() && LogPrivacyUtil.allowLogPrintConfig()) { + logger.debug("Kafka properties: {}", ctx); + } + if (counter == null) { counter = new KafkaChannelCounter(getName()); } @@ -257,7 +262,6 @@ private void setProducerProps(Context ctx, String bootStrapServers) { //Defaults overridden based on config producerProps.putAll(ctx.getSubProperties(KAFKA_PRODUCER_PREFIX)); producerProps.put(ProducerConfig.BOOTSTRAP_SERVERS_CONFIG, bootStrapServers); - logger.info("Producer properties: " + producerProps.toString()); } protected Properties getProducerProps() { @@ -274,8 +278,6 @@ private void setConsumerProps(Context ctx, String bootStrapServers) { consumerProps.put(ConsumerConfig.BOOTSTRAP_SERVERS_CONFIG, bootStrapServers); consumerProps.put(ConsumerConfig.GROUP_ID_CONFIG, groupId); consumerProps.put(ConsumerConfig.ENABLE_AUTO_COMMIT_CONFIG, false); - - logger.info(consumerProps.toString()); } protected Properties getConsumerProps() { diff --git a/flume-ng-configuration/src/main/java/org/apache/flume/conf/FlumeConfiguration.java b/flume-ng-configuration/src/main/java/org/apache/flume/conf/FlumeConfiguration.java index 9b3a434db9..8ae5bd9a33 100644 --- a/flume-ng-configuration/src/main/java/org/apache/flume/conf/FlumeConfiguration.java +++ b/flume-ng-configuration/src/main/java/org/apache/flume/conf/FlumeConfiguration.java @@ -310,9 +310,10 @@ public Set getSinkgroupSet() { * @return true if the configuration is valid, false otherwise */ private boolean isValid() { - logger.debug("Starting validation of configuration for agent: " - + agentName + ", initial-configuration: " + - this.getPrevalidationConfig()); + logger.debug("Starting validation of configuration for agent: {}", agentName); + if (logger.isDebugEnabled() && LogPrivacyUtil.allowLogPrintConfig()) { + logger.debug("Initial configuration: {}", this.getPrevalidationConfig()); + } // Make sure that at least one channel is specified if (channels == null || channels.trim().length() == 0) { @@ -368,8 +369,10 @@ private boolean isValid() { this.sinks = getSpaceDelimitedList(sinkSet); this.sinkgroups = getSpaceDelimitedList(sinkgroupSet); - logger.debug("Post validation configuration for " + agentName + NEWLINE - + this.getPostvalidationConfig()); + if (logger.isDebugEnabled() && LogPrivacyUtil.allowLogPrintConfig()) { + logger.debug("Post validation configuration for {}", agentName); + logger.debug(this.getPostvalidationConfig()); + } return true; } diff --git a/flume-ng-configuration/src/main/java/org/apache/flume/conf/LogPrivacyUtil.java b/flume-ng-configuration/src/main/java/org/apache/flume/conf/LogPrivacyUtil.java new file mode 100644 index 0000000000..357302527c --- /dev/null +++ b/flume-ng-configuration/src/main/java/org/apache/flume/conf/LogPrivacyUtil.java @@ -0,0 +1,83 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.flume.conf; + +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +/** + * Utility class to help any Flume component determine whether logging potentially sensitive + * information is allowed or not. + *

      + * InterfaceAudience.Public
      + * InterfaceStability.Evolving + */ +public class LogPrivacyUtil { + private static final Logger logger = LoggerFactory.getLogger(LogPrivacyUtil.class); + /** + * system property name to enable logging of potentially sensitive user data + */ + public static final String LOG_RAWDATA_PROP = "org.apache.flume.log.rawdata"; + /** + * system property name to enable logging of information related to the validation of agent + * configuration at startup. + */ + public static final String LOG_PRINTCONFIG_PROP = "org.apache.flume.log.printconfig"; + + static { + if (allowLogPrintConfig()) { + logger.warn("Logging of configuration details of the agent has been turned on by " + + "setting {} to true. Please use this setting with extra caution as it may result " + + "in logging of private data. This setting is not recommended in " + + "production environments.", + LOG_PRINTCONFIG_PROP); + } else { + logger.info("Logging of configuration details is disabled. To see configuration details " + + "in the log run the agent with -D{}=true JVM " + + "argument. Please note that this is not recommended in production " + + "systems as it may leak private information to the logfile.", + LOG_PRINTCONFIG_PROP); + } + + if (allowLogRawData()) { + logger.warn("Logging raw data has been turned on by setting {} to true. Please use it with " + + "extra caution as it may result in logging of potentially sensitive user data. " + + "This setting is not recommended in production environments.", + LOG_RAWDATA_PROP); + } + } + + /** + * Tells whether logging of configuration details - including secrets - is allowed or not. This + * is driven by a system property defined by LOG_PRINTCONFIG_PROP + * @return true only if logging is allowed + */ + public static boolean allowLogRawData() { + return Boolean.getBoolean(LOG_RAWDATA_PROP); + } + + /** + * Tells whether logging of potentially sensitive user data is allowed or not. This + * is driven by a system property defined by LOG_RAWDATA_PROP + * @return true only if logging is allowed + */ + public static boolean allowLogPrintConfig() { + return Boolean.getBoolean(LOG_PRINTCONFIG_PROP); + } +} diff --git a/flume-ng-core/src/main/java/org/apache/flume/source/AvroSource.java b/flume-ng-core/src/main/java/org/apache/flume/source/AvroSource.java index 8b9b956dee..762f690225 100644 --- a/flume-ng-core/src/main/java/org/apache/flume/source/AvroSource.java +++ b/flume-ng-core/src/main/java/org/apache/flume/source/AvroSource.java @@ -36,6 +36,7 @@ import org.apache.flume.Source; import org.apache.flume.conf.Configurable; import org.apache.flume.conf.Configurables; +import org.apache.flume.conf.LogPrivacyUtil; import org.apache.flume.event.EventBuilder; import org.apache.flume.instrumentation.SourceCounter; import org.apache.flume.source.avro.AvroFlumeEvent; @@ -344,8 +345,14 @@ private static Map toStringMap( @Override public Status append(AvroFlumeEvent avroEvent) { - logger.debug("Avro source {}: Received avro event: {}", getName(), - avroEvent); + if (logger.isDebugEnabled()) { + if (LogPrivacyUtil.allowLogRawData()) { + logger.debug("Avro source {}: Received avro event: {}", getName(), avroEvent); + } else { + logger.debug("Avro source {}: Received avro event", getName()); + } + } + sourceCounter.incrementAppendReceivedCount(); sourceCounter.incrementEventReceivedCount(); diff --git a/flume-ng-core/src/main/java/org/apache/flume/source/MultiportSyslogTCPSource.java b/flume-ng-core/src/main/java/org/apache/flume/source/MultiportSyslogTCPSource.java index b9f2438aa7..4436094162 100644 --- a/flume-ng-core/src/main/java/org/apache/flume/source/MultiportSyslogTCPSource.java +++ b/flume-ng-core/src/main/java/org/apache/flume/source/MultiportSyslogTCPSource.java @@ -36,6 +36,7 @@ import org.apache.flume.EventDrivenSource; import org.apache.flume.channel.ChannelProcessor; import org.apache.flume.conf.Configurable; +import org.apache.flume.conf.LogPrivacyUtil; import org.apache.flume.event.EventBuilder; import org.apache.flume.instrumentation.SourceCounter; import org.apache.mina.core.buffer.IoBuffer; @@ -357,7 +358,13 @@ Event parseEvent(ParsedBuffer parsedBuf, CharsetDecoder decoder) { return event; } - logger.trace("Seen raw event: {}", msg); + if (logger.isTraceEnabled()) { + if (LogPrivacyUtil.allowLogRawData()) { + logger.trace("Seen raw event: {}", msg); + } else { + logger.trace("Seen raw event."); + } + } Event event; try { diff --git a/flume-ng-core/src/main/java/org/apache/flume/source/http/BLOBHandler.java b/flume-ng-core/src/main/java/org/apache/flume/source/http/BLOBHandler.java index e24d4c6814..e1891cbbbb 100644 --- a/flume-ng-core/src/main/java/org/apache/flume/source/http/BLOBHandler.java +++ b/flume-ng-core/src/main/java/org/apache/flume/source/http/BLOBHandler.java @@ -29,6 +29,7 @@ import org.apache.commons.io.output.ByteArrayOutputStream; import org.apache.flume.Context; import org.apache.flume.Event; +import org.apache.flume.conf.LogPrivacyUtil; import org.apache.flume.event.EventBuilder; import org.slf4j.Logger; import org.slf4j.LoggerFactory; @@ -67,7 +68,9 @@ public List getEvents(HttpServletRequest request) throws Exception { Map parameters = request.getParameterMap(); for (String parameter : parameters.keySet()) { String value = parameters.get(parameter)[0]; - LOG.debug("Setting Header [Key, Value] as [{},{}] ",parameter, value); + if (LOG.isDebugEnabled() && LogPrivacyUtil.allowLogRawData()) { + LOG.debug("Setting Header [Key, Value] as [{},{}] ", parameter, value); + } headers.put(parameter, value); } diff --git a/flume-ng-core/src/test/java/org/apache/flume/serialization/SyslogAvroEventSerializer.java b/flume-ng-core/src/test/java/org/apache/flume/serialization/SyslogAvroEventSerializer.java index 05af3b1e7b..22e7e9e9b9 100644 --- a/flume-ng-core/src/test/java/org/apache/flume/serialization/SyslogAvroEventSerializer.java +++ b/flume-ng-core/src/test/java/org/apache/flume/serialization/SyslogAvroEventSerializer.java @@ -26,6 +26,7 @@ import org.apache.avro.Schema; import org.apache.flume.Context; import org.apache.flume.Event; +import org.apache.flume.conf.LogPrivacyUtil; import org.apache.flume.serialization.SyslogAvroEventSerializer.SyslogEvent; import org.apache.flume.source.SyslogUtils; import org.joda.time.DateTime; @@ -147,7 +148,9 @@ protected SyslogEvent convert(Event event) { String actualMessage = msg.substring(seek); sle.setMessage(actualMessage); - logger.debug("Serialized event as: {}", sle); + if (logger.isDebugEnabled() && LogPrivacyUtil.allowLogRawData()) { + logger.debug("Serialized event as: {}", sle); + } return sle; } diff --git a/flume-ng-doc/sphinx/FlumeUserGuide.rst b/flume-ng-doc/sphinx/FlumeUserGuide.rst index 7e207aa825..0fd1ec9464 100644 --- a/flume-ng-doc/sphinx/FlumeUserGuide.rst +++ b/flume-ng-doc/sphinx/FlumeUserGuide.rst @@ -234,6 +234,36 @@ The original Flume terminal will output the event in a log message. Congratulations - you've successfully configured and deployed a Flume agent! Subsequent sections cover agent configuration in much more detail. +Logging raw data +~~~~~~~~~~~~~~~~ + + +Logging the raw stream of data flowing through the ingest pipeline is not desired behaviour in +many production environments because this may result in leaking sensitive data or security related +configurations, such as secret keys, to Flume log files. +By default, Flume will not log such information. On the other hand, if the data pipeline is broken, +Flume will attempt to provide clues for debugging the problem. + +One way to debug problems with event pipelines is to set up an additional `Memory Channel`_ +connected to a `Logger Sink`_, which will output all event data to the Flume logs. +In some situations, however, this approach is insufficient. + +In order to enable logging of event- and configuration-related data, some Java system properties +must be set in addition to log4j properties. + +To enable configuration-related logging, set the Java system property +``-Dorg.apache.flume.log.printconfig=true``. This can either be passed on the command line or by +setting this in the ``JAVA_OPTS`` variable in *flume-env.sh*. + +To enable data logging, set the Java system property ``-Dorg.apache.flume.log.rawdata=true`` +in the same way described above. For most components, the log4j logging level must also be set to +DEBUG or TRACE to make event-specific logging appear in the Flume logs. + +Here is an example of enabling both configuration logging and raw data logging while also +setting the Log4j loglevel to DEBUG for console output:: + + $ bin/flume-ng agent --conf conf --conf-file example.conf --name a1 -Dflume.root.logger=DEBUG,console -Dorg.apache.flume.log.printconfig=true -Dorg.apache.flume.log.rawdata=true + Zookeeper based Configuration ~~~~~~~~~~~~~~~~~~~~~~~~~~~~~ @@ -2001,8 +2031,9 @@ accept tab separated input containing three fields and to skip the second field. Logger Sink ~~~~~~~~~~~ -Logs event at INFO level. Typically useful for testing/debugging purpose. -Required properties are in **bold**. +Logs event at INFO level. Typically useful for testing/debugging purpose. Required properties are +in **bold**. This sink is the only exception which doesn't require the extra configuration +explained in the `Logging raw data`_ section. ============== ======= =========================================== Property Name Default Description diff --git a/flume-ng-embedded-agent/src/main/java/org/apache/flume/agent/embedded/EmbeddedAgent.java b/flume-ng-embedded-agent/src/main/java/org/apache/flume/agent/embedded/EmbeddedAgent.java index ad3e138198..d33fa8b145 100644 --- a/flume-ng-embedded-agent/src/main/java/org/apache/flume/agent/embedded/EmbeddedAgent.java +++ b/flume-ng-embedded-agent/src/main/java/org/apache/flume/agent/embedded/EmbeddedAgent.java @@ -32,6 +32,7 @@ import org.apache.flume.SourceRunner; import org.apache.flume.annotations.InterfaceAudience; import org.apache.flume.annotations.InterfaceStability; +import org.apache.flume.conf.LogPrivacyUtil; import org.apache.flume.lifecycle.LifecycleAware; import org.apache.flume.lifecycle.LifecycleState; import org.apache.flume.lifecycle.LifecycleSupervisor; @@ -149,7 +150,7 @@ private void doConfigure(Map properties) { properties = EmbeddedAgentConfiguration.configure(name, properties); - if (LOGGER.isDebugEnabled()) { + if (LOGGER.isDebugEnabled() && LogPrivacyUtil.allowLogPrintConfig()) { LOGGER.debug("Agent configuration values"); for (String key : new TreeSet(properties.keySet())) { LOGGER.debug(key + " = " + properties.get(key)); @@ -255,4 +256,4 @@ private static enum State { STOPPED(), STARTED(); } -} \ No newline at end of file +} diff --git a/flume-ng-sinks/flume-ng-kafka-sink/src/main/java/org/apache/flume/sink/kafka/KafkaSink.java b/flume-ng-sinks/flume-ng-kafka-sink/src/main/java/org/apache/flume/sink/kafka/KafkaSink.java index 9453546ba0..89bdd84588 100644 --- a/flume-ng-sinks/flume-ng-kafka-sink/src/main/java/org/apache/flume/sink/kafka/KafkaSink.java +++ b/flume-ng-sinks/flume-ng-kafka-sink/src/main/java/org/apache/flume/sink/kafka/KafkaSink.java @@ -20,7 +20,6 @@ Licensed to the Apache Software Foundation (ASF) under one or more import com.google.common.base.Optional; import com.google.common.base.Throwables; - import org.apache.avro.io.BinaryEncoder; import org.apache.avro.io.EncoderFactory; import org.apache.avro.specific.SpecificDatumReader; @@ -32,6 +31,7 @@ Licensed to the Apache Software Foundation (ASF) under one or more import org.apache.flume.Transaction; import org.apache.flume.conf.Configurable; import org.apache.flume.conf.ConfigurationException; +import org.apache.flume.conf.LogPrivacyUtil; import org.apache.flume.instrumentation.kafka.KafkaSinkCounter; import org.apache.flume.sink.AbstractSink; import org.apache.flume.source.avro.AvroFlumeEvent; @@ -174,12 +174,15 @@ public Status process() throws EventDeliveryException { eventTopic = topic; } eventKey = headers.get(KEY_HEADER); - - if (logger.isDebugEnabled()) { - logger.debug("{Event} " + eventTopic + " : " + eventKey + " : " - + new String(eventBody, "UTF-8")); - logger.debug("event #{}", processedEvents); + if (logger.isTraceEnabled()) { + if (LogPrivacyUtil.allowLogRawData()) { + logger.trace("{Event} " + eventTopic + " : " + eventKey + " : " + + new String(eventBody, "UTF-8")); + } else { + logger.trace("{Event} " + eventTopic + " : " + eventKey); + } } + logger.debug("event #{}", processedEvents); // create a message and add to buffer long startTime = System.currentTimeMillis(); @@ -300,8 +303,8 @@ public void configure(Context context) { setProducerProps(context, bootStrapServers); - if (logger.isDebugEnabled()) { - logger.debug("Kafka producer properties: {}" , kafkaProps); + if (logger.isDebugEnabled() && LogPrivacyUtil.allowLogPrintConfig()) { + logger.debug("Kafka producer properties: {}", kafkaProps); } if (counter == null) { @@ -370,7 +373,6 @@ private void setProducerProps(Context context, String bootStrapServers) { kafkaProps.put(ProducerConfig.VALUE_SERIALIZER_CLASS_CONFIG, DEFAULT_VALUE_SERIAIZER); kafkaProps.putAll(context.getSubProperties(KAFKA_PRODUCER_PREFIX)); kafkaProps.put(ProducerConfig.BOOTSTRAP_SERVERS_CONFIG, bootStrapServers); - logger.info("Producer properties: {}" , kafkaProps.toString()); } protected Properties getKafkaProps() { diff --git a/flume-ng-sinks/flume-ng-morphline-solr-sink/src/main/java/org/apache/flume/sink/solr/morphline/BlobHandler.java b/flume-ng-sinks/flume-ng-morphline-solr-sink/src/main/java/org/apache/flume/sink/solr/morphline/BlobHandler.java index ca7614a0c1..fe98746997 100644 --- a/flume-ng-sinks/flume-ng-morphline-solr-sink/src/main/java/org/apache/flume/sink/solr/morphline/BlobHandler.java +++ b/flume-ng-sinks/flume-ng-morphline-solr-sink/src/main/java/org/apache/flume/sink/solr/morphline/BlobHandler.java @@ -29,6 +29,7 @@ import org.apache.flume.Context; import org.apache.flume.Event; import org.apache.flume.conf.ConfigurationException; +import org.apache.flume.conf.LogPrivacyUtil; import org.apache.flume.event.EventBuilder; import org.apache.flume.source.http.HTTPSourceHandler; import org.apache.tika.metadata.Metadata; @@ -69,11 +70,11 @@ public void configure(Context context) { + " must be greater than zero: " + maxBlobLength); } } - + @SuppressWarnings("resource") @Override public List getEvents(HttpServletRequest request) throws Exception { - Map headers = getHeaders(request); + Map headers = getHeaders(request); InputStream in = request.getInputStream(); try { ByteArrayOutputStream blob = null; @@ -88,14 +89,16 @@ public List getEvents(HttpServletRequest request) throws Exception { blobLength += n; if (blobLength >= maxBlobLength) { LOGGER.warn("Request length exceeds maxBlobLength ({}), truncating BLOB event!", - maxBlobLength); + maxBlobLength); break; } } byte[] array = blob != null ? blob.toByteArray() : new byte[0]; Event event = EventBuilder.withBody(array, headers); - LOGGER.debug("blobEvent: {}", event); + if (LOGGER.isDebugEnabled() && LogPrivacyUtil.allowLogRawData()) { + LOGGER.debug("blobEvent: {}", event); + } return Collections.singletonList(event); } finally { in.close(); @@ -103,15 +106,15 @@ public List getEvents(HttpServletRequest request) throws Exception { } private Map getHeaders(HttpServletRequest request) { - if (LOGGER.isDebugEnabled()) { + if (LOGGER.isDebugEnabled() && LogPrivacyUtil.allowLogRawData()) { Map requestHeaders = new HashMap(); Enumeration iter = request.getHeaderNames(); while (iter.hasMoreElements()) { String name = (String) iter.nextElement(); - requestHeaders.put(name, request.getHeader(name)); + requestHeaders.put(name, request.getHeader(name)); } LOGGER.debug("requestHeaders: {}", requestHeaders); - } + } Map headers = new HashMap(); if (request.getContentType() != null) { headers.put(Metadata.CONTENT_TYPE, request.getContentType()); @@ -119,9 +122,8 @@ private Map getHeaders(HttpServletRequest request) { Enumeration iter = request.getParameterNames(); while (iter.hasMoreElements()) { String name = (String) iter.nextElement(); - headers.put(name, request.getParameter(name)); + headers.put(name, request.getParameter(name)); } return headers; } - } diff --git a/flume-ng-sinks/flume-ng-morphline-solr-sink/src/main/java/org/apache/flume/sink/solr/morphline/MorphlineSink.java b/flume-ng-sinks/flume-ng-morphline-solr-sink/src/main/java/org/apache/flume/sink/solr/morphline/MorphlineSink.java index f7a73f3537..0917d393a4 100644 --- a/flume-ng-sinks/flume-ng-morphline-solr-sink/src/main/java/org/apache/flume/sink/solr/morphline/MorphlineSink.java +++ b/flume-ng-sinks/flume-ng-morphline-solr-sink/src/main/java/org/apache/flume/sink/solr/morphline/MorphlineSink.java @@ -24,6 +24,7 @@ import org.apache.flume.Transaction; import org.apache.flume.conf.Configurable; import org.apache.flume.conf.ConfigurationException; +import org.apache.flume.conf.LogPrivacyUtil; import org.apache.flume.instrumentation.SinkCounter; import org.apache.flume.sink.AbstractSink; import org.slf4j.Logger; @@ -136,7 +137,10 @@ public Status process() throws EventDeliveryException { } sinkCounter.incrementEventDrainAttemptCount(); numEventsTaken++; - LOGGER.debug("Flume event: {}", event); + if (LOGGER.isTraceEnabled() && LogPrivacyUtil.allowLogRawData()) { + LOGGER.trace("Flume event arrived {}", event); + } + //StreamEvent streamEvent = createStreamEvent(event); handler.process(event); if (System.currentTimeMillis() >= batchEndTime) { diff --git a/flume-ng-sources/flume-kafka-source/src/main/java/org/apache/flume/source/kafka/KafkaSource.java b/flume-ng-sources/flume-kafka-source/src/main/java/org/apache/flume/source/kafka/KafkaSource.java index 90e47159cf..86782c36e6 100644 --- a/flume-ng-sources/flume-kafka-source/src/main/java/org/apache/flume/source/kafka/KafkaSource.java +++ b/flume-ng-sources/flume-kafka-source/src/main/java/org/apache/flume/source/kafka/KafkaSource.java @@ -38,6 +38,7 @@ import org.apache.flume.FlumeException; import org.apache.flume.conf.Configurable; import org.apache.flume.conf.ConfigurationException; +import org.apache.flume.conf.LogPrivacyUtil; import org.apache.flume.event.EventBuilder; import org.apache.flume.instrumentation.kafka.KafkaSourceCounter; import org.apache.flume.source.AbstractPollableSource; @@ -240,12 +241,18 @@ protected Status doProcess() throws EventDeliveryException { headers.put(KafkaSourceConstants.KEY_HEADER, kafkaKey); } - if (log.isDebugEnabled()) { - log.debug("Topic: {} Partition: {} Message: {}", new String[] { - message.topic(), - String.valueOf(message.partition()), - new String(eventBody) - }); + if (log.isTraceEnabled()) { + if (LogPrivacyUtil.allowLogRawData()) { + log.trace("Topic: {} Partition: {} Message: {}", new String[]{ + message.topic(), + String.valueOf(message.partition()), + new String(eventBody) + }); + } else { + log.trace("Topic: {} Partition: {} Message arrived.", + message.topic(), + String.valueOf(message.partition())); + } } event = EventBuilder.withBody(eventBody, headers); @@ -342,6 +349,10 @@ protected void doConfigure(Context context) throws FlumeException { setConsumerProps(context, bootstrapServers); + if (log.isDebugEnabled() && LogPrivacyUtil.allowLogPrintConfig()) { + log.debug("Kafka consumer properties: {}", kafkaProps); + } + if (counter == null) { counter = new KafkaSourceCounter(getName()); } @@ -388,8 +399,6 @@ private void setConsumerProps(Context ctx, String bootStrapServers) { } kafkaProps.put(ConsumerConfig.ENABLE_AUTO_COMMIT_CONFIG, KafkaSourceConstants.DEFAULT_AUTO_COMMIT); - - log.info(kafkaProps.toString()); } Properties getConsumerProps() { diff --git a/flume-ng-sources/flume-twitter-source/src/main/java/org/apache/flume/source/twitter/TwitterSource.java b/flume-ng-sources/flume-twitter-source/src/main/java/org/apache/flume/source/twitter/TwitterSource.java index f5c832807d..d812023865 100644 --- a/flume-ng-sources/flume-twitter-source/src/main/java/org/apache/flume/source/twitter/TwitterSource.java +++ b/flume-ng-sources/flume-twitter-source/src/main/java/org/apache/flume/source/twitter/TwitterSource.java @@ -107,11 +107,6 @@ public void configure(Context context) { String accessToken = context.getString("accessToken"); String accessTokenSecret = context.getString("accessTokenSecret"); - LOGGER.info("Consumer Key: '" + consumerKey + "'"); - LOGGER.info("Consumer Secret: '" + consumerSecret + "'"); - LOGGER.info("Access Token: '" + accessToken + "'"); - LOGGER.info("Access Token Secret: '" + accessTokenSecret + "'"); - twitterStream = new TwitterStreamFactory().getInstance(); twitterStream.setOAuthConsumer(consumerKey, consumerSecret); twitterStream.setOAuthAccessToken(new AccessToken(accessToken, From 53fb37443eec3261b32f39cb363183988498e47f Mon Sep 17 00:00:00 2001 From: Mike Percy Date: Mon, 29 Aug 2016 17:18:08 -0700 Subject: [PATCH 324/341] Upgrade to ASF pom version 18 MIME-Version: 1.0 Content-Type: text/plain; charset=UTF-8 Content-Transfer-Encoding: 8bit This gets rid of some annoying warning messages from RAT, such as: Warning: org.apache.xerces.jaxp.SAXParserImpl$JAXPSAXParser: Property 'http://www.oracle.com/xml/jaxp/properties/entityExpansionLimit' is not recognized. Compiler warnings: WARNING: 'org.apache.xerces.jaxp.SAXParserImpl: Property 'http://javax.xml.XMLConstants/property/accessExternalDTD' is not recognized.' Warning: org.apache.xerces.parsers.SAXParser: Feature 'http://javax.xml.XMLConstants/feature/secure-processing' is not recognized. Warning: org.apache.xerces.parsers.SAXParser: Property 'http://javax.xml.XMLConstants/property/accessExternalDTD' is not recognized. Warning: org.apache.xerces.parsers.SAXParser: Property 'http://www.oracle.com/xml/jaxp/properties/entityExpansionLimit' is not recognized. Reviewers: Bessenyei Balázs Donát, Hari Shreedharan This closes #63. --- pom.xml | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/pom.xml b/pom.xml index b50693ed90..2332a29ec6 100644 --- a/pom.xml +++ b/pom.xml @@ -21,7 +21,7 @@ limitations under the License. org.apache apache - 9 + 18 4.0.0 From 330e572879c9903f5b1084d84b237685aa9970c6 Mon Sep 17 00:00:00 2001 From: Grant Henke Date: Wed, 31 Aug 2016 18:44:00 -0700 Subject: [PATCH 325/341] FLUME-2983. Handle offset migration in the new Kafka Source Similar to FLUME-2972, offsets tracking the position in Kafka consumers change from using zookeeper for offset storage to Kafka when moving from 0.8.x to 0.9.x. FLUME-2821 makes the client change in the Kafka Source but does not ensure existing offsets get migrated in order to continue consuming where it left off. Flume should have some automated logic on startup to check if Kafka offsets exist, if not and migration is enabled (by default) then copy the offsets from Zookeeper and commit them to Kafka. This change should also fix the backwards incompatibility caused by removing the zookeeperConnect property. The bootstrap can be looked up if zookeeperConnect is used. Reviewers: Denes Arvay, Mike Percy (Grant Henke via Mike Percy) --- .../flume/channel/kafka/KafkaChannel.java | 2 +- .../flume/channel/kafka/TestKafkaChannel.java | 2 +- flume-ng-doc/sphinx/FlumeUserGuide.rst | 7 +- flume-ng-sources/flume-kafka-source/pom.xml | 1 + .../flume/source/kafka/KafkaSource.java | 183 ++++++++++++++++-- .../source/kafka/KafkaSourceConstants.java | 5 +- .../kafka/KafkaSourceEmbeddedKafka.java | 4 +- .../flume/source/kafka/TestKafkaSource.java | 163 +++++++++++++++- 8 files changed, 343 insertions(+), 24 deletions(-) diff --git a/flume-ng-channels/flume-kafka-channel/src/main/java/org/apache/flume/channel/kafka/KafkaChannel.java b/flume-ng-channels/flume-kafka-channel/src/main/java/org/apache/flume/channel/kafka/KafkaChannel.java index e7f1f2e315..66b553ad62 100644 --- a/flume-ng-channels/flume-kafka-channel/src/main/java/org/apache/flume/channel/kafka/KafkaChannel.java +++ b/flume-ng-channels/flume-kafka-channel/src/main/java/org/apache/flume/channel/kafka/KafkaChannel.java @@ -193,7 +193,7 @@ public void configure(Context ctx) { migrateZookeeperOffsets = ctx.getBoolean(MIGRATE_ZOOKEEPER_OFFSETS, DEFAULT_MIGRATE_ZOOKEEPER_OFFSETS); - zookeeperConnect = ctx.getString(ZOOKEEPER_CONNECT); + zookeeperConnect = ctx.getString(ZOOKEEPER_CONNECT_FLUME_KEY); if (logger.isDebugEnabled() && LogPrivacyUtil.allowLogPrintConfig()) { logger.debug("Kafka properties: {}", ctx); diff --git a/flume-ng-channels/flume-kafka-channel/src/test/java/org/apache/flume/channel/kafka/TestKafkaChannel.java b/flume-ng-channels/flume-kafka-channel/src/test/java/org/apache/flume/channel/kafka/TestKafkaChannel.java index e7ae68f3a5..57c0b288d5 100644 --- a/flume-ng-channels/flume-kafka-channel/src/test/java/org/apache/flume/channel/kafka/TestKafkaChannel.java +++ b/flume-ng-channels/flume-kafka-channel/src/test/java/org/apache/flume/channel/kafka/TestKafkaChannel.java @@ -255,7 +255,7 @@ public void doTestMigrateZookeeperOffsets(boolean hasZookeeperOffsets, boolean h createTopic(topic, 1); Context context = prepareDefaultContext(false); - context.put(ZOOKEEPER_CONNECT, testUtil.getZkUrl()); + context.put(ZOOKEEPER_CONNECT_FLUME_KEY, testUtil.getZkUrl()); context.put(GROUP_ID_FLUME, group); final KafkaChannel channel = createChannel(context); diff --git a/flume-ng-doc/sphinx/FlumeUserGuide.rst b/flume-ng-doc/sphinx/FlumeUserGuide.rst index 0fd1ec9464..53844e3f00 100644 --- a/flume-ng-doc/sphinx/FlumeUserGuide.rst +++ b/flume-ng-doc/sphinx/FlumeUserGuide.rst @@ -1260,8 +1260,13 @@ useFlumeEventFormat false By default events are taken as byt true to read events as the Flume Avro binary format. Used in conjunction with the same property on the KafkaSink or with the parseAsFlumeEvent property on the Kafka Channel this will preserve any Flume headers sent on the producing side. +migrateZookeeperOffsets true When no Kafka stored offset is found, look up the offsets in Zookeeper and commit them to Kafka. + This should be true to support seamless Kafka client migration from older versions of Flume. + Once migrated this can be set to false, though that should generally not be required. + If no Zookeeper offset is found, the Kafka configuration kafka.consumer.auto.offset.reset + defines how offsets are handled. Other Kafka Consumer Properties -- These properties are used to configure the Kafka Consumer. Any producer property supported - by Kafka can be used. The only requirement is to prepend the property name with the prefix + by Kafka can be used. The only requirement is to prepend the property name with the prefix ``kafka.consumer``. For example: ``kafka.consumer.auto.offset.reset`` Check `Kafka documentation `_ for details diff --git a/flume-ng-sources/flume-kafka-source/pom.xml b/flume-ng-sources/flume-kafka-source/pom.xml index 5f5c2a8479..c89ea1a299 100644 --- a/flume-ng-sources/flume-kafka-source/pom.xml +++ b/flume-ng-sources/flume-kafka-source/pom.xml @@ -51,6 +51,7 @@ org.apache.kafka kafka_2.10 + ${kafka.version} org.apache.zookeeper diff --git a/flume-ng-sources/flume-kafka-source/src/main/java/org/apache/flume/source/kafka/KafkaSource.java b/flume-ng-sources/flume-kafka-source/src/main/java/org/apache/flume/source/kafka/KafkaSource.java index 86782c36e6..195eca3e95 100644 --- a/flume-ng-sources/flume-kafka-source/src/main/java/org/apache/flume/source/kafka/KafkaSource.java +++ b/flume-ng-sources/flume-kafka-source/src/main/java/org/apache/flume/source/kafka/KafkaSource.java @@ -29,9 +29,14 @@ import java.util.concurrent.atomic.AtomicBoolean; import java.util.regex.Pattern; +import com.google.common.annotations.VisibleForTesting; +import kafka.cluster.BrokerEndPoint; +import kafka.utils.ZKGroupTopicDirs; +import kafka.utils.ZkUtils; import org.apache.avro.io.BinaryDecoder; import org.apache.avro.io.DecoderFactory; import org.apache.avro.specific.SpecificDatumReader; +import org.apache.commons.lang.StringUtils; import org.apache.flume.Context; import org.apache.flume.Event; import org.apache.flume.EventDeliveryException; @@ -43,17 +48,25 @@ import org.apache.flume.instrumentation.kafka.KafkaSourceCounter; import org.apache.flume.source.AbstractPollableSource; import org.apache.flume.source.avro.AvroFlumeEvent; +import org.apache.kafka.clients.CommonClientConfigs; import org.apache.kafka.clients.consumer.ConsumerConfig; import org.apache.kafka.clients.consumer.ConsumerRebalanceListener; import org.apache.kafka.clients.consumer.ConsumerRecord; import org.apache.kafka.clients.consumer.ConsumerRecords; import org.apache.kafka.clients.consumer.KafkaConsumer; import org.apache.kafka.clients.consumer.OffsetAndMetadata; +import org.apache.kafka.common.PartitionInfo; import org.apache.kafka.common.TopicPartition; +import org.apache.kafka.common.protocol.SecurityProtocol; +import org.apache.kafka.common.security.JaasUtils; import org.slf4j.Logger; import org.slf4j.LoggerFactory; import com.google.common.base.Optional; +import scala.Option; + +import static org.apache.flume.source.kafka.KafkaSourceConstants.*; +import static scala.collection.JavaConverters.asJavaListConverter; /** * A Source for Kafka which reads messages from kafka topics. @@ -84,6 +97,10 @@ public class KafkaSource extends AbstractPollableSource implements Configurable { private static final Logger log = LoggerFactory.getLogger(KafkaSource.class); + // Constants used only for offset migration zookeeper connections + private static final int ZK_SESSION_TIMEOUT = 30000; + private static final int ZK_CONNECTION_TIMEOUT = 30000; + private Context context; private Properties kafkaProps; private KafkaSourceCounter counter; @@ -106,6 +123,10 @@ public class KafkaSource extends AbstractPollableSource private Subscriber subscriber; + private String zookeeperConnect; + private String bootstrapServers; + private String groupId = DEFAULT_GROUP_ID; + private boolean migrateZookeeperOffsets = DEFAULT_MIGRATE_ZOOKEEPER_OFFSETS; /** * This class is a helper to subscribe for topics by using @@ -342,12 +363,44 @@ protected void doConfigure(Context context) throws FlumeException { log.debug(KafkaSourceConstants.AVRO_EVENT + " set to: {}", useAvroEventFormat); } - String bootstrapServers = context.getString(KafkaSourceConstants.BOOTSTRAP_SERVERS); + zookeeperConnect = context.getString(ZOOKEEPER_CONNECT_FLUME_KEY); + migrateZookeeperOffsets = context.getBoolean(MIGRATE_ZOOKEEPER_OFFSETS, + DEFAULT_MIGRATE_ZOOKEEPER_OFFSETS); + + bootstrapServers = context.getString(KafkaSourceConstants.BOOTSTRAP_SERVERS); if (bootstrapServers == null || bootstrapServers.isEmpty()) { - throw new ConfigurationException("Bootstrap Servers must be specified"); + if (zookeeperConnect == null || zookeeperConnect.isEmpty()) { + throw new ConfigurationException("Bootstrap Servers must be specified"); + } else { + // For backwards compatibility look up the bootstrap from zookeeper + log.warn("{} is deprecated. Please use the parameter {}", + KafkaSourceConstants.ZOOKEEPER_CONNECT_FLUME_KEY, + KafkaSourceConstants.BOOTSTRAP_SERVERS); + + // Lookup configured security protocol, just in case its not default + String securityProtocolStr = + context.getSubProperties(KafkaSourceConstants.KAFKA_CONSUMER_PREFIX) + .get(CommonClientConfigs.SECURITY_PROTOCOL_CONFIG); + if (securityProtocolStr == null || securityProtocolStr.isEmpty()) { + securityProtocolStr = CommonClientConfigs.DEFAULT_SECURITY_PROTOCOL; + } + bootstrapServers = + lookupBootstrap(zookeeperConnect, SecurityProtocol.valueOf(securityProtocolStr)); + } } - setConsumerProps(context, bootstrapServers); + String groupIdProperty = + context.getString(KAFKA_CONSUMER_PREFIX + ConsumerConfig.GROUP_ID_CONFIG); + if (groupIdProperty != null && !groupIdProperty.isEmpty()) { + groupId = groupIdProperty; // Use the new group id property + } + + if (groupId == null || groupId.isEmpty()) { + groupId = DEFAULT_GROUP_ID; + log.info("Group ID was not specified. Using {} as the group id.", groupId); + } + + setConsumerProps(context); if (log.isDebugEnabled() && LogPrivacyUtil.allowLogPrintConfig()) { log.debug("Kafka consumer properties: {}", kafkaProps); @@ -369,23 +422,15 @@ private void translateOldProperties(Context ctx) { } // old groupId - String groupId = ctx.getString(KafkaSourceConstants.OLD_GROUP_ID); + groupId = ctx.getString(KafkaSourceConstants.OLD_GROUP_ID); if (groupId != null && !groupId.isEmpty()) { - kafkaProps.put(ConsumerConfig.GROUP_ID_CONFIG, groupId); log.warn("{} is deprecated. Please use the parameter {}", KafkaSourceConstants.OLD_GROUP_ID, KafkaSourceConstants.KAFKA_CONSUMER_PREFIX + ConsumerConfig.GROUP_ID_CONFIG); } } - private void setConsumerProps(Context ctx, String bootStrapServers) { - String groupId = ctx.getString( - KafkaSourceConstants.KAFKA_CONSUMER_PREFIX + ConsumerConfig.GROUP_ID_CONFIG); - if ((groupId == null || groupId.isEmpty()) && - kafkaProps.getProperty(ConsumerConfig.GROUP_ID_CONFIG) == null) { - groupId = KafkaSourceConstants.DEFAULT_GROUP_ID; - log.info("Group ID was not specified. Using " + groupId + " as the group id."); - } + private void setConsumerProps(Context ctx) { kafkaProps.put(ConsumerConfig.KEY_DESERIALIZER_CLASS_CONFIG, KafkaSourceConstants.DEFAULT_KEY_DESERIALIZER); kafkaProps.put(ConsumerConfig.VALUE_DESERIALIZER_CLASS_CONFIG, @@ -393,7 +438,7 @@ private void setConsumerProps(Context ctx, String bootStrapServers) { //Defaults overridden based on config kafkaProps.putAll(ctx.getSubProperties(KafkaSourceConstants.KAFKA_CONSUMER_PREFIX)); //These always take precedence over config - kafkaProps.put(ConsumerConfig.BOOTSTRAP_SERVERS_CONFIG, bootStrapServers); + kafkaProps.put(ConsumerConfig.BOOTSTRAP_SERVERS_CONFIG, bootstrapServers); if (groupId != null) { kafkaProps.put(ConsumerConfig.GROUP_ID_CONFIG, groupId); } @@ -401,6 +446,31 @@ private void setConsumerProps(Context ctx, String bootStrapServers) { KafkaSourceConstants.DEFAULT_AUTO_COMMIT); } + /** + * Generates the Kafka bootstrap connection string from the metadata stored in Zookeeper. + * Allows for backwards compatibility of the zookeeperConnect configuration. + */ + private String lookupBootstrap(String zookeeperConnect, SecurityProtocol securityProtocol) { + ZkUtils zkUtils = ZkUtils.apply(zookeeperConnect, ZK_SESSION_TIMEOUT, ZK_CONNECTION_TIMEOUT, + JaasUtils.isZkSecurityEnabled()); + try { + List endPoints = + asJavaListConverter(zkUtils.getAllBrokerEndPointsForChannel(securityProtocol)).asJava(); + List connections = new ArrayList<>(); + for (BrokerEndPoint endPoint : endPoints) { + connections.add(endPoint.connectionString()); + } + return StringUtils.join(connections, ','); + } finally { + zkUtils.close(); + } + } + + @VisibleForTesting + String getBootstrapServers() { + return bootstrapServers; + } + Properties getConsumerProps() { return kafkaProps; } @@ -424,6 +494,21 @@ Subscriber getSubscriber() { protected void doStart() throws FlumeException { log.info("Starting {}...", this); + // As a migration step check if there are any offsets from the group stored in kafka + // If not read them from Zookeeper and commit them to Kafka + if (migrateZookeeperOffsets && zookeeperConnect != null && !zookeeperConnect.isEmpty()) { + // For simplicity we only support migration of a single topic via the TopicListSubscriber. + // There was no way to define a list of topics or a pattern in the previous Flume version. + if (subscriber instanceof TopicListSubscriber && + ((TopicListSubscriber) subscriber).get().size() == 1) { + String topicStr = ((TopicListSubscriber) subscriber).get().get(0); + migrateOffsets(topicStr); + } else { + log.info("Will not attempt to migrate offsets " + + "because multiple topics or a pattern are defined"); + } + } + //initialize a consumer. consumer = new KafkaConsumer(kafkaProps); @@ -445,6 +530,76 @@ protected void doStop() throws FlumeException { counter.stop(); log.info("Kafka Source {} stopped. Metrics: {}", getName(), counter); } + + private void migrateOffsets(String topicStr) { + ZkUtils zkUtils = ZkUtils.apply(zookeeperConnect, ZK_SESSION_TIMEOUT, ZK_CONNECTION_TIMEOUT, + JaasUtils.isZkSecurityEnabled()); + KafkaConsumer consumer = new KafkaConsumer<>(kafkaProps); + try { + Map kafkaOffsets = + getKafkaOffsets(consumer, topicStr); + if (!kafkaOffsets.isEmpty()) { + log.info("Found Kafka offsets for topic " + topicStr + + ". Will not migrate from zookeeper"); + log.debug("Offsets found: {}", kafkaOffsets); + return; + } + + log.info("No Kafka offsets found. Migrating zookeeper offsets"); + Map zookeeperOffsets = + getZookeeperOffsets(zkUtils, topicStr); + if (zookeeperOffsets.isEmpty()) { + log.warn("No offsets to migrate found in Zookeeper"); + return; + } + + log.info("Committing Zookeeper offsets to Kafka"); + log.debug("Offsets to commit: {}", zookeeperOffsets); + consumer.commitSync(zookeeperOffsets); + // Read the offsets to verify they were committed + Map newKafkaOffsets = + getKafkaOffsets(consumer, topicStr); + log.debug("Offsets committed: {}", newKafkaOffsets); + if (!newKafkaOffsets.keySet().containsAll(zookeeperOffsets.keySet())) { + throw new FlumeException("Offsets could not be committed"); + } + } finally { + zkUtils.close(); + consumer.close(); + } + } + + private Map getKafkaOffsets( + KafkaConsumer client, String topicStr) { + Map offsets = new HashMap<>(); + List partitions = client.partitionsFor(topicStr); + for (PartitionInfo partition : partitions) { + TopicPartition key = new TopicPartition(topicStr, partition.partition()); + OffsetAndMetadata offsetAndMetadata = client.committed(key); + if (offsetAndMetadata != null) { + offsets.put(key, offsetAndMetadata); + } + } + return offsets; + } + + private Map getZookeeperOffsets(ZkUtils client, + String topicStr) { + Map offsets = new HashMap<>(); + ZKGroupTopicDirs topicDirs = new ZKGroupTopicDirs(groupId, topicStr); + List partitions = asJavaListConverter( + client.getChildrenParentMayNotExist(topicDirs.consumerOffsetDir())).asJava(); + for (String partition : partitions) { + TopicPartition key = new TopicPartition(topicStr, Integer.valueOf(partition)); + Option data = client.readDataMaybeNull( + topicDirs.consumerOffsetDir() + "/" + partition)._1(); + if (data.isDefined()) { + Long offset = Long.valueOf(data.get()); + offsets.put(key, new OffsetAndMetadata(offset)); + } + } + return offsets; + } } class SourceRebalanceListener implements ConsumerRebalanceListener { diff --git a/flume-ng-sources/flume-kafka-source/src/main/java/org/apache/flume/source/kafka/KafkaSourceConstants.java b/flume-ng-sources/flume-kafka-source/src/main/java/org/apache/flume/source/kafka/KafkaSourceConstants.java index 1f255f9471..bf1a19df17 100644 --- a/flume-ng-sources/flume-kafka-source/src/main/java/org/apache/flume/source/kafka/KafkaSourceConstants.java +++ b/flume-ng-sources/flume-kafka-source/src/main/java/org/apache/flume/source/kafka/KafkaSourceConstants.java @@ -37,11 +37,14 @@ public class KafkaSourceConstants { public static final int DEFAULT_BATCH_DURATION = 1000; public static final String DEFAULT_GROUP_ID = "flume"; + public static final String MIGRATE_ZOOKEEPER_OFFSETS = "migrateZookeeperOffsets"; + public static final boolean DEFAULT_MIGRATE_ZOOKEEPER_OFFSETS = true; + public static final String AVRO_EVENT = "useFlumeEventFormat"; public static final boolean DEFAULT_AVRO_EVENT = false; /* Old Properties */ - + public static final String ZOOKEEPER_CONNECT_FLUME_KEY = "zookeeperConnect"; public static final String TOPIC = "topic"; public static final String OLD_GROUP_ID = "groupId"; diff --git a/flume-ng-sources/flume-kafka-source/src/test/java/org/apache/flume/source/kafka/KafkaSourceEmbeddedKafka.java b/flume-ng-sources/flume-kafka-source/src/test/java/org/apache/flume/source/kafka/KafkaSourceEmbeddedKafka.java index a3a2f9267b..53bd65c429 100644 --- a/flume-ng-sources/flume-kafka-source/src/test/java/org/apache/flume/source/kafka/KafkaSourceEmbeddedKafka.java +++ b/flume-ng-sources/flume-kafka-source/src/test/java/org/apache/flume/source/kafka/KafkaSourceEmbeddedKafka.java @@ -36,7 +36,7 @@ public class KafkaSourceEmbeddedKafka { - public static String HOST = InetAddress.getLoopbackAddress().getHostAddress(); + public static String HOST = InetAddress.getLoopbackAddress().getCanonicalHostName(); KafkaServerStartable kafkaServer; KafkaSourceEmbeddedZookeeper zookeeper; @@ -80,7 +80,7 @@ public String getZkConnectString() { return zookeeper.getConnectString(); } - public String getBrockers() { + public String getBootstrapServers() { return HOST + ":" + serverPort; } diff --git a/flume-ng-sources/flume-kafka-source/src/test/java/org/apache/flume/source/kafka/TestKafkaSource.java b/flume-ng-sources/flume-kafka-source/src/test/java/org/apache/flume/source/kafka/TestKafkaSource.java index 1598741cd9..95542015ab 100644 --- a/flume-ng-sources/flume-kafka-source/src/test/java/org/apache/flume/source/kafka/TestKafkaSource.java +++ b/flume-ng-sources/flume-kafka-source/src/test/java/org/apache/flume/source/kafka/TestKafkaSource.java @@ -21,9 +21,12 @@ import com.google.common.collect.Lists; import junit.framework.Assert; import kafka.common.TopicExistsException; +import kafka.utils.ZKGroupTopicDirs; +import kafka.utils.ZkUtils; import org.apache.avro.io.BinaryEncoder; import org.apache.avro.io.EncoderFactory; import org.apache.avro.specific.SpecificDatumWriter; +import org.apache.commons.lang.RandomStringUtils; import org.apache.flume.ChannelException; import org.apache.flume.Context; import org.apache.flume.Event; @@ -33,6 +36,14 @@ import org.apache.flume.channel.ChannelProcessor; import org.apache.flume.source.avro.AvroFlumeEvent; import org.apache.kafka.clients.consumer.ConsumerConfig; +import org.apache.kafka.clients.consumer.KafkaConsumer; +import org.apache.kafka.clients.consumer.OffsetAndMetadata; +import org.apache.kafka.clients.producer.KafkaProducer; +import org.apache.kafka.clients.producer.ProducerConfig; +import org.apache.kafka.clients.producer.ProducerRecord; +import org.apache.kafka.clients.producer.RecordMetadata; +import org.apache.kafka.common.TopicPartition; +import org.apache.kafka.common.security.JaasUtils; import org.junit.After; import org.junit.Before; import org.junit.Test; @@ -44,10 +55,13 @@ import java.io.ByteArrayOutputStream; import java.io.IOException; import java.nio.ByteBuffer; +import java.util.ArrayList; import java.util.HashMap; +import java.util.HashSet; import java.util.List; import java.util.Map; import java.util.Properties; +import java.util.Set; import java.util.regex.Pattern; import static org.apache.flume.source.kafka.KafkaSourceConstants.AVRO_EVENT; @@ -63,6 +77,7 @@ import static org.apache.flume.source.kafka.KafkaSourceConstants.TOPICS; import static org.apache.flume.source.kafka.KafkaSourceConstants.TOPICS_REGEX; import static org.apache.flume.source.kafka.KafkaSourceConstants.TOPIC_HEADER; +import static org.apache.flume.source.kafka.KafkaSourceConstants.ZOOKEEPER_CONNECT_FLUME_KEY; import static org.junit.Assert.assertEquals; import static org.junit.Assert.assertTrue; import static org.mockito.Matchers.any; @@ -76,6 +91,8 @@ public class TestKafkaSource { private KafkaSourceEmbeddedKafka kafkaServer; private Context context; private List events; + + private final Set usedTopics = new HashSet(); private String topic0 = "test1"; private String topic1 = "topic1"; @@ -86,19 +103,21 @@ public void setup() throws Exception { kafkaServer = new KafkaSourceEmbeddedKafka(null); try { kafkaServer.createTopic(topic0, 1); + usedTopics.add(topic0); kafkaServer.createTopic(topic1, 3); + usedTopics.add(topic1); } catch (TopicExistsException e) { //do nothing e.printStackTrace(); } - context = prepareDefaultContext(); + context = prepareDefaultContext("flume-group"); kafkaSource.setChannelProcessor(createGoodChannel()); } - private Context prepareDefaultContext() { + private Context prepareDefaultContext(String groupId) { Context context = new Context(); - context.put(BOOTSTRAP_SERVERS, kafkaServer.getBrockers()); - context.put(KAFKA_CONSUMER_PREFIX + "group.id", "flume-group"); + context.put(BOOTSTRAP_SERVERS, kafkaServer.getBootstrapServers()); + context.put(KAFKA_CONSUMER_PREFIX + "group.id", groupId); return context; } @@ -575,6 +594,118 @@ public void testAvroEvent() throws InterruptedException, EventDeliveryException, } + @Test + public void testBootstrapLookup() { + Context context = new Context(); + + context.put(ZOOKEEPER_CONNECT_FLUME_KEY, kafkaServer.getZkConnectString()); + context.put(TOPIC, "old.topic"); + context.put(OLD_GROUP_ID, "old.groupId"); + KafkaSource source = new KafkaSource(); + source.doConfigure(context); + String bootstrapServers = source.getBootstrapServers(); + Assert.assertEquals(kafkaServer.getBootstrapServers(), bootstrapServers); + } + + @Test + public void testMigrateOffsetsNone() throws Exception { + doTestMigrateZookeeperOffsets(false, false, "testMigrateOffsets-none"); + } + + @Test + public void testMigrateOffsetsZookeeper() throws Exception { + doTestMigrateZookeeperOffsets(true, false, "testMigrateOffsets-zookeeper"); + } + + @Test + public void testMigrateOffsetsKafka() throws Exception { + doTestMigrateZookeeperOffsets(false, true, "testMigrateOffsets-kafka"); + } + + @Test + public void testMigrateOffsetsBoth() throws Exception { + doTestMigrateZookeeperOffsets(true, true, "testMigrateOffsets-both"); + } + + public void doTestMigrateZookeeperOffsets(boolean hasZookeeperOffsets, boolean hasKafkaOffsets, + String group) throws Exception { + // create a topic with 1 partition for simplicity + String topic = findUnusedTopic(); + kafkaServer.createTopic(topic, 1); + + Context context = prepareDefaultContext(group); + context.put(ZOOKEEPER_CONNECT_FLUME_KEY, kafkaServer.getZkConnectString()); + context.put(TOPIC, topic); + KafkaSource source = new KafkaSource(); + source.doConfigure(context); + + // Produce some data and save an offset + Long fifthOffset = 0L; + Long tenthOffset = 0L; + Properties props = createProducerProps(kafkaServer.getBootstrapServers()); + KafkaProducer producer = new KafkaProducer<>(props); + for (int i = 1; i <= 50; i++) { + ProducerRecord data = + new ProducerRecord<>(topic, null, String.valueOf(i).getBytes()); + RecordMetadata recordMetadata = producer.send(data).get(); + if (i == 5) { + fifthOffset = recordMetadata.offset(); + } + if (i == 10) { + tenthOffset = recordMetadata.offset(); + } + } + + // Commit 10th offset to zookeeper + if (hasZookeeperOffsets) { + ZkUtils zkUtils = ZkUtils.apply(kafkaServer.getZkConnectString(), 30000, 30000, + JaasUtils.isZkSecurityEnabled()); + ZKGroupTopicDirs topicDirs = new ZKGroupTopicDirs(group, topic); + // we commit the tenth offset to ensure some data is missed. + Long offset = tenthOffset + 1; + zkUtils.updatePersistentPath(topicDirs.consumerOffsetDir() + "/0", offset.toString(), + zkUtils.updatePersistentPath$default$3()); + zkUtils.close(); + } + + // Commit 5th offset to kafka + if (hasKafkaOffsets) { + Map offsets = new HashMap<>(); + offsets.put(new TopicPartition(topic, 0), new OffsetAndMetadata(fifthOffset + 1)); + KafkaConsumer consumer = new KafkaConsumer<>(source.getConsumerProps()); + consumer.commitSync(offsets); + consumer.close(); + } + + // Start the source and read some data + source.setChannelProcessor(createGoodChannel()); + source.start(); + Thread.sleep(500L); + source.process(); + List finals = new ArrayList(40); + for (Event event: events) { + finals.add(Integer.parseInt(new String(event.getBody()))); + } + source.stop(); + + if (!hasKafkaOffsets && !hasZookeeperOffsets) { + // The default behavior is to start at the latest message in the log + org.junit.Assert.assertTrue("Source should read no messages", finals.isEmpty()); + } else if (hasKafkaOffsets && hasZookeeperOffsets) { + // Respect Kafka offsets if they exist + org.junit.Assert.assertFalse("Source should not read the 5th message", finals.contains(5)); + org.junit.Assert.assertTrue("Source should read the 6th message", finals.contains(6)); + } else if (hasKafkaOffsets) { + // Respect Kafka offsets if they exist (don't fail if zookeeper offsets are missing) + org.junit.Assert.assertFalse("Source should not read the 5th message", finals.contains(5)); + org.junit.Assert.assertTrue("Source should read the 6th message", finals.contains(6)); + } else { + // Otherwise migrate the ZooKeeper offsets if they exist + org.junit.Assert.assertFalse("Source should not read the 10th message", finals.contains(10)); + org.junit.Assert.assertTrue("Source should read the 11th message", finals.contains(11)); + } + } + ChannelProcessor createGoodChannel() { ChannelProcessor channelProcessor = mock(ChannelProcessor.class); @@ -604,4 +735,28 @@ public Void answer(InvocationOnMock invocation) throws Throwable { return channelProcessor; } + + public String findUnusedTopic() { + String newTopic = null; + boolean topicFound = false; + while (!topicFound) { + newTopic = RandomStringUtils.randomAlphabetic(8); + if (!usedTopics.contains(newTopic)) { + usedTopics.add(newTopic); + topicFound = true; + } + } + return newTopic; + } + + private Properties createProducerProps(String bootStrapServers) { + Properties props = new Properties(); + props.put(ProducerConfig.ACKS_CONFIG, "-1"); + props.put(ProducerConfig.KEY_SERIALIZER_CLASS_CONFIG, + "org.apache.kafka.common.serialization.StringSerializer"); + props.put(ProducerConfig.VALUE_SERIALIZER_CLASS_CONFIG, + "org.apache.kafka.common.serialization.ByteArraySerializer"); + props.put(ProducerConfig.BOOTSTRAP_SERVERS_CONFIG, bootStrapServers); + return props; + } } From a0a50849d5ad33e6e6903316a3bd3dbba8547843 Mon Sep 17 00:00:00 2001 From: Jeff Holoman Date: Wed, 31 Aug 2016 20:12:23 -0700 Subject: [PATCH 326/341] FLUME-2982. Add localhost escape sequence to HDFS sink It would be useful to be able to just use %[localhost] escape sequence instead of having to pass in a header or use the host interceptor. Part of the problem of using an interceptor is the case where Flume only consists of a channel and a sink (think KafkaChannel). There is support in Flume for a sink-side interceptor at the time of this writing. Reviewers: Grant Henke, Mike Percy (Jeff Holoman via Mike Percy) --- .../flume/formatter/output/BucketPath.java | 38 +++++++++++- .../formatter/output/TestBucketPath.java | 47 ++++++++++++++ flume-ng-doc/sphinx/FlumeUserGuide.rst | 61 ++++++++++--------- 3 files changed, 117 insertions(+), 29 deletions(-) diff --git a/flume-ng-core/src/main/java/org/apache/flume/formatter/output/BucketPath.java b/flume-ng-core/src/main/java/org/apache/flume/formatter/output/BucketPath.java index f640ec9791..cf0fbb07d6 100644 --- a/flume-ng-core/src/main/java/org/apache/flume/formatter/output/BucketPath.java +++ b/flume-ng-core/src/main/java/org/apache/flume/formatter/output/BucketPath.java @@ -24,6 +24,8 @@ import org.apache.flume.SystemClock; import org.apache.flume.tools.TimestampRoundDownUtil; +import java.net.InetAddress; +import java.net.UnknownHostException; import java.text.SimpleDateFormat; import java.util.Calendar; import java.util.Date; @@ -39,7 +41,7 @@ public class BucketPath { * These are useful to other classes which might want to search for tags in * strings. */ - public static final String TAG_REGEX = "\\%(\\w|\\%)|\\%\\{([\\w\\.-]+)\\}"; + public static final String TAG_REGEX = "%(\\w|%)|%\\{([\\w\\.-]+)\\}|%\\[(\\w+)\\]"; public static final Pattern tagPattern = Pattern.compile(TAG_REGEX); private static Clock clock = new SystemClock(); @@ -210,6 +212,35 @@ protected static SimpleDateFormat getSimpleDateFormat(String string) { return simpleDateFormat; } + /** + * Not intended as a public API + */ + @VisibleForTesting + protected static String replaceStaticString(String key) { + String replacementString = ""; + try { + InetAddress addr = InetAddress.getLocalHost(); + switch (key.toLowerCase()) { + case "localhost": + replacementString = addr.getHostName(); + break; + case "ip": + replacementString = addr.getHostAddress(); + break; + case "fqdn": + replacementString = addr.getCanonicalHostName(); + break; + default: + throw new RuntimeException("The static escape string '" + key + "'" + + " was provided but does not match any of (localhost,IP,FQDN)"); + } + } catch (UnknownHostException e) { + throw new RuntimeException("Flume wasn't able to parse the static escape " + + " sequence '" + key + "' due to UnkownHostException.", e); + } + return replacementString; + } + /** * Not intended as a public API */ @@ -418,6 +449,11 @@ public static String escapeString(String in, Map headers, replacement = ""; // LOG.warn("Tag " + matcher.group(2) + " not found"); } + + // Group 3 is the %[...] pattern. + } else if (matcher.group(3) != null) { + replacement = replaceStaticString(matcher.group(3)); + } else { // The %x pattern. // Since we know the match is a single character, we can diff --git a/flume-ng-core/src/test/java/org/apache/flume/formatter/output/TestBucketPath.java b/flume-ng-core/src/test/java/org/apache/flume/formatter/output/TestBucketPath.java index ccc746066e..989c2a4a75 100644 --- a/flume-ng-core/src/test/java/org/apache/flume/formatter/output/TestBucketPath.java +++ b/flume-ng-core/src/test/java/org/apache/flume/formatter/output/TestBucketPath.java @@ -26,6 +26,8 @@ import org.junit.Test; import javax.annotation.Nullable; +import java.net.InetAddress; +import java.net.UnknownHostException; import java.text.SimpleDateFormat; import java.util.Calendar; import java.util.Date; @@ -278,4 +280,49 @@ private static Calendar createCalendar(int year, int month, int day, cal.set(Calendar.MILLISECOND, ms); return cal; } + + @Test + public void testStaticEscapeStrings() { + Map staticStrings; + staticStrings = new HashMap<>(); + + try { + InetAddress addr = InetAddress.getLocalHost(); + staticStrings.put("localhost", addr.getHostName()); + staticStrings.put("IP", addr.getHostAddress()); + staticStrings.put("FQDN", addr.getCanonicalHostName()); + } catch (UnknownHostException e) { + Assert.fail("Test failed due to UnkownHostException"); + } + + TimeZone utcTimeZone = TimeZone.getTimeZone("UTC"); + String filePath = "%[localhost]/%[IP]/%[FQDN]"; + String realPath = BucketPath.escapeString(filePath, headers, + utcTimeZone, false, Calendar.HOUR_OF_DAY, 12, false); + String[] args = realPath.split("\\/"); + + Assert.assertEquals(args[0],staticStrings.get("localhost")); + Assert.assertEquals(args[1],staticStrings.get("IP")); + Assert.assertEquals(args[2],staticStrings.get("FQDN")); + + StringBuilder s = new StringBuilder(); + s.append("Expected String: ").append(staticStrings.get("localhost")); + s.append("/").append(staticStrings.get("IP")).append("/"); + s.append(staticStrings.get("FQDN")); + + System.out.println(s); + System.out.println("Escaped String: " + realPath ); + } + + @Test (expected = RuntimeException.class) + public void testStaticEscapeStringsNoKey() { + Map staticStrings; + staticStrings = new HashMap<>(); + + TimeZone utcTimeZone = TimeZone.getTimeZone("UTC"); + String filePath = "%[abcdefg]/%[IP]/%[FQDN]"; + String realPath = BucketPath.escapeString(filePath, headers, + utcTimeZone, false, Calendar.HOUR_OF_DAY, 12, false); + } + } diff --git a/flume-ng-doc/sphinx/FlumeUserGuide.rst b/flume-ng-doc/sphinx/FlumeUserGuide.rst index 53844e3f00..0fecee6fc5 100644 --- a/flume-ng-doc/sphinx/FlumeUserGuide.rst +++ b/flume-ng-doc/sphinx/FlumeUserGuide.rst @@ -1785,34 +1785,39 @@ required. The following are the escape sequences supported: -========= ================================================= -Alias Description -========= ================================================= -%{host} Substitute value of event header named "host". Arbitrary header names are supported. -%t Unix time in milliseconds -%a locale's short weekday name (Mon, Tue, ...) -%A locale's full weekday name (Monday, Tuesday, ...) -%b locale's short month name (Jan, Feb, ...) -%B locale's long month name (January, February, ...) -%c locale's date and time (Thu Mar 3 23:05:25 2005) -%d day of month (01) -%e day of month without padding (1) -%D date; same as %m/%d/%y -%H hour (00..23) -%I hour (01..12) -%j day of year (001..366) -%k hour ( 0..23) -%m month (01..12) -%n month without padding (1..12) -%M minute (00..59) -%p locale's equivalent of am or pm -%s seconds since 1970-01-01 00:00:00 UTC -%S second (00..59) -%y last two digits of year (00..99) -%Y year (2010) -%z +hhmm numeric timezone (for example, -0400) -========= ================================================= - +=============== ================================================= +Alias Description +=============== ================================================= +%{host} Substitute value of event header named "host". Arbitrary header names are supported. +%t Unix time in milliseconds +%a locale's short weekday name (Mon, Tue, ...) +%A locale's full weekday name (Monday, Tuesday, ...) +%b locale's short month name (Jan, Feb, ...) +%B locale's long month name (January, February, ...) +%c locale's date and time (Thu Mar 3 23:05:25 2005) +%d day of month (01) +%e day of month without padding (1) +%D date; same as %m/%d/%y +%H hour (00..23) +%I hour (01..12) +%j day of year (001..366) +%k hour ( 0..23) +%m month (01..12) +%n month without padding (1..12) +%M minute (00..59) +%p locale's equivalent of am or pm +%s seconds since 1970-01-01 00:00:00 UTC +%S second (00..59) +%y last two digits of year (00..99) +%Y year (2010) +%z +hhmm numeric timezone (for example, -0400) +%[localhost] Substitute the hostname of the host where the agent is running +%[IP] Substitute the IP address of the host where the agent is running +%[FQDN] Substitute the canonical hostname of the host where the agent is running +=============== ================================================= + +Note: The escape strings %[localhost], %[IP] and %[FQDN] all rely on Java's ability to obtain the +hostname, which may fail in some networking environments. The file in use will have the name mangled to include ".tmp" at the end. Once the file is closed, this extension is removed. This allows excluding partially From d9c9a7dd9a6889ecf6b9dc88fb8e02ccc1cd5167 Mon Sep 17 00:00:00 2001 From: Denes Arvay Date: Thu, 8 Sep 2016 10:00:55 -0700 Subject: [PATCH 327/341] FLUME-2797. Use SourceCounter for SyslogTcpSource MIME-Version: 1.0 Content-Type: text/plain; charset=UTF-8 Content-Transfer-Encoding: 8bit This patch uses the newer SourceCounter class for the SyslogTcpSource. It also marks the SyslogTcpSource as deprecated and improves how the unit tests are written so they don't require DNS. Reviewers: Attila Simon, Lior Zeno, Balázs Donát Bessenyei, Mike Percy (Denes Arvay via Mike Percy) --- .../apache/flume/source/SyslogTcpSource.java | 36 +++++--- .../apache/flume/source/SyslogUDPSource.java | 32 ++++--- .../flume/source/TestSyslogTcpSource.java | 21 +++-- .../flume/source/TestSyslogUdpSource.java | 85 ++++++++++++------- 4 files changed, 113 insertions(+), 61 deletions(-) diff --git a/flume-ng-core/src/main/java/org/apache/flume/source/SyslogTcpSource.java b/flume-ng-core/src/main/java/org/apache/flume/source/SyslogTcpSource.java index 185c00ca32..c7e8248973 100644 --- a/flume-ng-core/src/main/java/org/apache/flume/source/SyslogTcpSource.java +++ b/flume-ng-core/src/main/java/org/apache/flume/source/SyslogTcpSource.java @@ -28,11 +28,11 @@ import com.google.common.annotations.VisibleForTesting; import org.apache.flume.ChannelException; import org.apache.flume.Context; -import org.apache.flume.CounterGroup; import org.apache.flume.Event; import org.apache.flume.EventDrivenSource; import org.apache.flume.conf.Configurable; import org.apache.flume.conf.Configurables; +import org.apache.flume.instrumentation.SourceCounter; import org.jboss.netty.bootstrap.ServerBootstrap; import org.jboss.netty.buffer.ChannelBuffer; import org.jboss.netty.channel.Channel; @@ -47,6 +47,10 @@ import org.slf4j.Logger; import org.slf4j.LoggerFactory; +/** + * @deprecated use {@link MultiportSyslogTCPSource} instead. + */ +@Deprecated public class SyslogTcpSource extends AbstractSource implements EventDrivenSource, Configurable { private static final Logger logger = LoggerFactory.getLogger(SyslogTcpSource.class); @@ -56,7 +60,7 @@ public class SyslogTcpSource extends AbstractSource private Channel nettyChannel; private Integer eventSize; private Map formaterProp; - private CounterGroup counterGroup = new CounterGroup(); + private SourceCounter sourceCounter; private Set keepFields; public class syslogTcpHandler extends SimpleChannelHandler { @@ -85,14 +89,14 @@ public void messageReceived(ChannelHandlerContext ctx, MessageEvent mEvent) { "rest of the event is received."); continue; } + sourceCounter.incrementEventReceivedCount(); + try { getChannelProcessor().processEvent(e); - counterGroup.incrementAndGet("events.success"); + sourceCounter.incrementEventAcceptedCount(); } catch (ChannelException ex) { - counterGroup.incrementAndGet("events.dropped"); logger.error("Error writting to channel, event dropped", ex); } catch (RuntimeException ex) { - counterGroup.incrementAndGet("events.dropped"); logger.error("Error parsing event from syslog stream, event dropped", ex); return; } @@ -126,13 +130,14 @@ public ChannelPipeline getPipeline() { nettyChannel = serverBootstrap.bind(new InetSocketAddress(host, port)); } + sourceCounter.start(); super.start(); } @Override public void stop() { logger.info("Syslog TCP Source stopping..."); - logger.info("Metrics:{}", counterGroup); + logger.info("Metrics: {}", sourceCounter); if (nettyChannel != null) { nettyChannel.close(); @@ -145,6 +150,7 @@ public void stop() { } } + sourceCounter.stop(); super.stop(); } @@ -161,16 +167,24 @@ public void configure(Context context) { context.getString( SyslogSourceConfigurationConstants.CONFIG_KEEP_FIELDS, SyslogSourceConfigurationConstants.DEFAULT_KEEP_FIELDS)); + + if (sourceCounter == null) { + sourceCounter = new SourceCounter(getName()); + } } @VisibleForTesting - public int getSourcePort() { + InetSocketAddress getBoundAddress() { SocketAddress localAddress = nettyChannel.getLocalAddress(); - if (localAddress instanceof InetSocketAddress) { - InetSocketAddress addr = (InetSocketAddress) localAddress; - return addr.getPort(); + if (!(localAddress instanceof InetSocketAddress)) { + throw new IllegalArgumentException("Not bound to an internet address"); } - return 0; + return (InetSocketAddress) localAddress; } + + @VisibleForTesting + SourceCounter getSourceCounter() { + return sourceCounter; + } } diff --git a/flume-ng-core/src/main/java/org/apache/flume/source/SyslogUDPSource.java b/flume-ng-core/src/main/java/org/apache/flume/source/SyslogUDPSource.java index 175bebbcc5..ae0b8ac32e 100644 --- a/flume-ng-core/src/main/java/org/apache/flume/source/SyslogUDPSource.java +++ b/flume-ng-core/src/main/java/org/apache/flume/source/SyslogUDPSource.java @@ -28,11 +28,11 @@ import com.google.common.annotations.VisibleForTesting; import org.apache.flume.ChannelException; import org.apache.flume.Context; -import org.apache.flume.CounterGroup; import org.apache.flume.Event; import org.apache.flume.EventDrivenSource; import org.apache.flume.conf.Configurable; import org.apache.flume.conf.Configurables; +import org.apache.flume.instrumentation.SourceCounter; import org.jboss.netty.bootstrap.ConnectionlessBootstrap; import org.jboss.netty.buffer.ChannelBuffer; import org.jboss.netty.channel.AdaptiveReceiveBufferSizePredictorFactory; @@ -60,7 +60,7 @@ public class SyslogUDPSource extends AbstractSource private static final Logger logger = LoggerFactory.getLogger(SyslogUDPSource.class); - private CounterGroup counterGroup = new CounterGroup(); + private SourceCounter sourceCounter; // Default Min size public static final int DEFAULT_MIN_SIZE = 2048; @@ -85,14 +85,14 @@ public void messageReceived(ChannelHandlerContext ctx, MessageEvent mEvent) { if (e == null) { return; } + sourceCounter.incrementEventReceivedCount(); + getChannelProcessor().processEvent(e); - counterGroup.incrementAndGet("events.success"); + sourceCounter.incrementEventAcceptedCount(); } catch (ChannelException ex) { - counterGroup.incrementAndGet("events.dropped"); logger.error("Error writting to channel", ex); return; } catch (RuntimeException ex) { - counterGroup.incrementAndGet("events.dropped"); logger.error("Error parsing event from syslog stream, event dropped", ex); return; } @@ -123,13 +123,14 @@ public ChannelPipeline getPipeline() { nettyChannel = serverBootstrap.bind(new InetSocketAddress(host, port)); } + sourceCounter.start(); super.start(); } @Override public void stop() { logger.info("Syslog UDP Source stopping..."); - logger.info("Metrics:{}", counterGroup); + logger.info("Metrics: {}", sourceCounter); if (nettyChannel != null) { nettyChannel.close(); try { @@ -141,6 +142,7 @@ public void stop() { } } + sourceCounter.stop(); super.stop(); } @@ -156,15 +158,23 @@ public void configure(Context context) { context.getString( SyslogSourceConfigurationConstants.CONFIG_KEEP_FIELDS, SyslogSourceConfigurationConstants.DEFAULT_KEEP_FIELDS)); + + if (sourceCounter == null) { + sourceCounter = new SourceCounter(getName()); + } } @VisibleForTesting - public int getSourcePort() { + InetSocketAddress getBoundAddress() { SocketAddress localAddress = nettyChannel.getLocalAddress(); - if (localAddress instanceof InetSocketAddress) { - InetSocketAddress addr = (InetSocketAddress) localAddress; - return addr.getPort(); + if (!(localAddress instanceof InetSocketAddress)) { + throw new IllegalArgumentException("Not bound to an internet address"); } - return 0; + return (InetSocketAddress) localAddress; + } + + @VisibleForTesting + SourceCounter getSourceCounter() { + return sourceCounter; } } diff --git a/flume-ng-core/src/test/java/org/apache/flume/source/TestSyslogTcpSource.java b/flume-ng-core/src/test/java/org/apache/flume/source/TestSyslogTcpSource.java index 10ef8d874c..f07acc61df 100644 --- a/flume-ng-core/src/test/java/org/apache/flume/source/TestSyslogTcpSource.java +++ b/flume-ng-core/src/test/java/org/apache/flume/source/TestSyslogTcpSource.java @@ -35,6 +35,7 @@ import java.io.IOException; import java.net.InetAddress; +import java.net.InetSocketAddress; import java.net.Socket; import java.util.ArrayList; import java.util.List; @@ -62,7 +63,7 @@ private void init(String keepFields) { Configurables.configure(channel, new Context()); - List channels = new ArrayList(); + List channels = new ArrayList<>(); channels.add(channel); ChannelSelector rcs = new ReplicatingChannelSelector(); @@ -82,15 +83,14 @@ private void runKeepFieldsTest(String keepFields) throws IOException { init(keepFields); source.start(); // Write some message to the syslog port - Socket syslogSocket; + InetSocketAddress addr = source.getBoundAddress(); for (int i = 0; i < 10 ; i++) { - syslogSocket = new Socket( - InetAddress.getLocalHost(), source.getSourcePort()); - syslogSocket.getOutputStream().write(bodyWithTandH.getBytes()); - syslogSocket.close(); + try (Socket syslogSocket = new Socket(addr.getAddress(), addr.getPort())) { + syslogSocket.getOutputStream().write(bodyWithTandH.getBytes()); + } } - List channelEvents = new ArrayList(); + List channelEvents = new ArrayList<>(); Transaction txn = channel.getTransaction(); txn.begin(); for (int i = 0; i < 10; i++) { @@ -152,5 +152,12 @@ public void testKeepHostname() throws IOException { public void testKeepTimestamp() throws IOException { runKeepFieldsTest("timestamp"); } + + @Test + public void testSourceCounter() throws IOException { + runKeepFieldsTest("all"); + Assert.assertEquals(10, source.getSourceCounter().getEventAcceptedCount()); + Assert.assertEquals(10, source.getSourceCounter().getEventReceivedCount()); + } } diff --git a/flume-ng-core/src/test/java/org/apache/flume/source/TestSyslogUdpSource.java b/flume-ng-core/src/test/java/org/apache/flume/source/TestSyslogUdpSource.java index e5b7a066a6..6ba75120de 100644 --- a/flume-ng-core/src/test/java/org/apache/flume/source/TestSyslogUdpSource.java +++ b/flume-ng-core/src/test/java/org/apache/flume/source/TestSyslogUdpSource.java @@ -37,6 +37,9 @@ import java.net.DatagramPacket; import java.net.DatagramSocket; import java.net.InetAddress; +import java.net.InetSocketAddress; +import java.net.SocketException; +import java.net.UnknownHostException; import java.util.ArrayList; import java.util.List; @@ -71,6 +74,7 @@ private void init(String keepFields) { source.setChannelProcessor(new ChannelProcessor(rcs)); Context context = new Context(); + context.put("host", InetAddress.getLoopbackAddress().getHostAddress()); context.put("port", String.valueOf(TEST_SYSLOG_PORT)); context.put("keepFields", keepFields); @@ -85,18 +89,12 @@ private void runKeepFieldsTest(String keepFields) throws IOException { init(keepFields); source.start(); // Write some message to the syslog port - DatagramSocket syslogSocket; - DatagramPacket datagramPacket; - datagramPacket = new DatagramPacket(bodyWithTandH.getBytes(), - bodyWithTandH.getBytes().length, - InetAddress.getLocalHost(), source.getSourcePort()); + DatagramPacket datagramPacket = createDatagramPacket(bodyWithTandH.getBytes()); for (int i = 0; i < 10 ; i++) { - syslogSocket = new DatagramSocket(); - syslogSocket.send(datagramPacket); - syslogSocket.close(); + sendDatagramPacket(datagramPacket); } - List channelEvents = new ArrayList(); + List channelEvents = new ArrayList<>(); Transaction txn = channel.getTransaction(); txn.begin(); for (int i = 0; i < 10; i++) { @@ -105,13 +103,7 @@ private void runKeepFieldsTest(String keepFields) throws IOException { channelEvents.add(e); } - try { - txn.commit(); - } catch (Throwable t) { - txn.rollback(); - } finally { - txn.close(); - } + commitAndCloseTransaction(txn); source.stop(); for (Event e : channelEvents) { @@ -139,18 +131,13 @@ public void testLargePayload() throws Exception { byte[] largePayload = getPayload(1000).getBytes(); - DatagramSocket syslogSocket; - DatagramPacket datagramPacket; - datagramPacket = new DatagramPacket(largePayload, - 1000, - InetAddress.getLocalHost(), source.getSourcePort()); + DatagramPacket datagramPacket = createDatagramPacket(largePayload); + for (int i = 0; i < 10 ; i++) { - syslogSocket = new DatagramSocket(); - syslogSocket.send(datagramPacket); - syslogSocket.close(); + sendDatagramPacket(datagramPacket); } - List channelEvents = new ArrayList(); + List channelEvents = new ArrayList<>(); Transaction txn = channel.getTransaction(); txn.begin(); for (int i = 0; i < 10; i++) { @@ -159,13 +146,7 @@ public void testLargePayload() throws Exception { channelEvents.add(e); } - try { - txn.commit(); - } catch (Throwable t) { - txn.rollback(); - } finally { - txn.close(); - } + commitAndCloseTransaction(txn); source.stop(); for (Event e : channelEvents) { @@ -200,6 +181,46 @@ public void testKeepTimestamp() throws IOException { runKeepFieldsTest("timestamp"); } + @Test + public void testSourceCounter() throws Exception { + init("true"); + + source.start(); + DatagramPacket datagramPacket = createDatagramPacket("test".getBytes()); + sendDatagramPacket(datagramPacket); + + Transaction txn = channel.getTransaction(); + txn.begin(); + + channel.take(); + commitAndCloseTransaction(txn); + + Assert.assertEquals(1, source.getSourceCounter().getEventAcceptedCount()); + Assert.assertEquals(1, source.getSourceCounter().getEventReceivedCount()); + } + + private DatagramPacket createDatagramPacket(byte[] payload) { + InetSocketAddress addr = source.getBoundAddress(); + return new DatagramPacket(payload, payload.length, addr.getAddress(), addr.getPort()); + } + + private void sendDatagramPacket(DatagramPacket datagramPacket) throws IOException { + try (DatagramSocket syslogSocket = new DatagramSocket()) { + syslogSocket.send(datagramPacket); + } + } + + private void commitAndCloseTransaction(Transaction txn) { + try { + txn.commit(); + } catch (Throwable t) { + logger.error("Transaction commit failed, rolling back", t); + txn.rollback(); + } finally { + txn.close(); + } + } + private String getPayload(int length) { StringBuilder payload = new StringBuilder(length); for (int n = 0; n < length; ++n) { From 591b138321280cea1e7d61efcaf625a67202cb3d Mon Sep 17 00:00:00 2001 From: Siddharth Ahuja Date: Mon, 26 Sep 2016 15:50:51 +0200 Subject: [PATCH 328/341] FLUME-2966. Fix NPE in JMS Source MIME-Version: 1.0 Content-Type: text/plain; charset=UTF-8 Content-Transfer-Encoding: 8bit JMS Source does not check for null text in a TextMessage. This can lead to NullPointerException. This commit fixes that problem by checking for nullity of textMessage.getText(). Reviewers: Denes Arvay, Attila Simon, Mike Percy, Bessenyei Balázs Donát (Siddharth Ahuja via Bessenyei Balázs Donát) --- .../jms/DefaultJMSMessageConverter.java | 5 ++++- .../jms/TestDefaultJMSMessageConverter.java | 21 +++++++++++++++++++ 2 files changed, 25 insertions(+), 1 deletion(-) diff --git a/flume-ng-sources/flume-jms-source/src/main/java/org/apache/flume/source/jms/DefaultJMSMessageConverter.java b/flume-ng-sources/flume-jms-source/src/main/java/org/apache/flume/source/jms/DefaultJMSMessageConverter.java index acb51186ba..003f591fd5 100644 --- a/flume-ng-sources/flume-jms-source/src/main/java/org/apache/flume/source/jms/DefaultJMSMessageConverter.java +++ b/flume-ng-sources/flume-jms-source/src/main/java/org/apache/flume/source/jms/DefaultJMSMessageConverter.java @@ -100,7 +100,10 @@ public List convert(Message message) throws JMSException { } } else if (message instanceof TextMessage) { TextMessage textMessage = (TextMessage)message; - event.setBody(textMessage.getText().getBytes(charset)); + String text = textMessage.getText(); + if (text != null) { + event.setBody(text.getBytes(charset)); + } } else if (message instanceof ObjectMessage) { ObjectMessage objectMessage = (ObjectMessage)message; Object object = objectMessage.getObject(); diff --git a/flume-ng-sources/flume-jms-source/src/test/java/org/apache/flume/source/jms/TestDefaultJMSMessageConverter.java b/flume-ng-sources/flume-jms-source/src/test/java/org/apache/flume/source/jms/TestDefaultJMSMessageConverter.java index 0b2193c938..f0e46ca3ac 100644 --- a/flume-ng-sources/flume-jms-source/src/test/java/org/apache/flume/source/jms/TestDefaultJMSMessageConverter.java +++ b/flume-ng-sources/flume-jms-source/src/test/java/org/apache/flume/source/jms/TestDefaultJMSMessageConverter.java @@ -66,6 +66,13 @@ void createTextMessage() throws Exception { when(message.getText()).thenReturn(TEXT); this.message = message; } + + void createNullTextMessage() throws Exception { + TextMessage message = mock(TextMessage.class); + when(message.getText()).thenReturn(null); + this.message = message; + } + void createBytesMessage() throws Exception { BytesMessage message = mock(BytesMessage.class); when(message.getBodyLength()).thenReturn((long)BYTES.length); @@ -117,6 +124,20 @@ public void testTextMessage() throws Exception { assertEquals(headers, event.getHeaders()); assertEquals(TEXT, new String(event.getBody(), Charsets.UTF_8)); } + + @Test + public void testNullTextMessage() throws Exception { + createNullTextMessage(); + headers.put("key1", "value1"); + headers.put("key2", "value2"); + createHeaders(); + Event event = converter.convert(message).iterator().next(); + assertEquals(headers, event.getHeaders()); + // In case of a null text message, the event's body will be empty due to + // SimpleEvent's body not updated with a valid text message. + assertEquals(event.getBody().length, 0); + } + @Test public void testBytesMessage() throws Exception { createBytesMessage(); From fca8c5b970cf0bbae61bae08f2084dd4f3bd1143 Mon Sep 17 00:00:00 2001 From: Attila Simon Date: Fri, 30 Sep 2016 12:01:12 +0200 Subject: [PATCH 329/341] FLUME-2998. Add a missing configuration parameter to SequenceSource docs MIME-Version: 1.0 Content-Type: text/plain; charset=UTF-8 Content-Transfer-Encoding: 8bit SequenceSource's totalEvents is configurable, but it is not in the documentation. This patch fixes that problem by adding the missing configuration parameter to the docs. Reviewers: Denes Arvay, Bessenyei Balázs Donát (Attila Simon via Bessenyei Balázs Donát) --- flume-ng-doc/sphinx/FlumeUserGuide.rst | 23 ++++++++++++----------- 1 file changed, 12 insertions(+), 11 deletions(-) diff --git a/flume-ng-doc/sphinx/FlumeUserGuide.rst b/flume-ng-doc/sphinx/FlumeUserGuide.rst index 0fecee6fc5..ab71d38ff0 100644 --- a/flume-ng-doc/sphinx/FlumeUserGuide.rst +++ b/flume-ng-doc/sphinx/FlumeUserGuide.rst @@ -1352,21 +1352,22 @@ Example for agent named a1: Sequence Generator Source ~~~~~~~~~~~~~~~~~~~~~~~~~ -A simple sequence generator that continuously generates events with a counter -that starts from 0 and increments by 1. Useful mainly for testing. -Required properties are in **bold**. +A simple sequence generator that continuously generates events with a counter that starts from 0, +increments by 1 and stops at totalEvents. Retries when it can't send events to the channel. Useful +mainly for testing. Required properties are in **bold**. -============== =========== ======================================== -Property Name Default Description -============== =========== ======================================== +============== =============== ======================================== +Property Name Default Description +============== =============== ======================================== **channels** -- -**type** -- The component type name, needs to be ``seq`` -selector.type replicating or multiplexing -selector.* replicating Depends on the selector.type value -interceptors -- Space-separated list of interceptors +**type** -- The component type name, needs to be ``seq`` +selector.type replicating or multiplexing +selector.* replicating Depends on the selector.type value +interceptors -- Space-separated list of interceptors interceptors.* batchSize 1 -============== =========== ======================================== +totalEvents Long.MAX_VALUE Number of unique events sent by the source. +============== =============== ======================================== Example for agent named a1: From 27c725802b16915acaa3ea05cdac0236972af1f0 Mon Sep 17 00:00:00 2001 From: Denes Arvay Date: Fri, 7 Oct 2016 11:49:27 +0200 Subject: [PATCH 330/341] FLUME-3003. Fix flaky testSourceCounter in TestSyslogUdpSource MIME-Version: 1.0 Content-Type: text/plain; charset=UTF-8 Content-Transfer-Encoding: 8bit The event processing in SyslogUDPSource is handled on a separate thread by Netty so message delivery, thus the sourceCounter's increment can be delayed resulting in a flaky test. This patch fixes that issue by adding more delay when necessary. Reviewers: Lior Zeno, Bessenyei Balázs Donát (Denes Arvay via Bessenyei Balázs Donát) --- .../java/org/apache/flume/source/TestSyslogUdpSource.java | 7 +++++++ 1 file changed, 7 insertions(+) diff --git a/flume-ng-core/src/test/java/org/apache/flume/source/TestSyslogUdpSource.java b/flume-ng-core/src/test/java/org/apache/flume/source/TestSyslogUdpSource.java index 6ba75120de..cb3860d8cc 100644 --- a/flume-ng-core/src/test/java/org/apache/flume/source/TestSyslogUdpSource.java +++ b/flume-ng-core/src/test/java/org/apache/flume/source/TestSyslogUdpSource.java @@ -195,6 +195,13 @@ public void testSourceCounter() throws Exception { channel.take(); commitAndCloseTransaction(txn); + // Retrying up to 10 times while the acceptedCount == 0 because the event processing in + // SyslogUDPSource is handled on a separate thread by Netty so message delivery, + // thus the sourceCounter's increment can be delayed resulting in a flaky test + for (int i = 0; i < 10 && source.getSourceCounter().getEventAcceptedCount() == 0; i++) { + Thread.sleep(100); + } + Assert.assertEquals(1, source.getSourceCounter().getEventAcceptedCount()); Assert.assertEquals(1, source.getSourceCounter().getEventReceivedCount()); } From 7d5ceacac49f5d15bf8f75e0209592c5524a3dda Mon Sep 17 00:00:00 2001 From: Andrea Rota Date: Thu, 4 Aug 2016 10:09:16 +0200 Subject: [PATCH 331/341] FLUME-2911. Add include pattern option in SpoolDir source MIME-Version: 1.0 Content-Type: text/plain; charset=UTF-8 Content-Transfer-Encoding: 8bit * Documented what happens when ignorePattern and includePattern both match for a given file. * Added two tests to simulate what happens when both ignorePattern and includePattern options are specified * Refactored of ReliableSpoolingFileEventReader test and fix of code style violations Closes #60 Reviewers: Bessenyei Balázs Donát, Denes Arvay, Attila Simon (Andrea Rota via Mike Percy) --- .../avro/ReliableSpoolingFileEventReader.java | 15 +- .../flume/source/SpoolDirectorySource.java | 3 + ...DirectorySourceConfigurationConstants.java | 4 + .../TestReliableSpoolingFileEventReader.java | 191 ++++++++++++++---- flume-ng-doc/sphinx/FlumeUserGuide.rst | 9 +- 5 files changed, 179 insertions(+), 43 deletions(-) diff --git a/flume-ng-core/src/main/java/org/apache/flume/client/avro/ReliableSpoolingFileEventReader.java b/flume-ng-core/src/main/java/org/apache/flume/client/avro/ReliableSpoolingFileEventReader.java index a0f929cd9e..1e1d9556df 100644 --- a/flume-ng-core/src/main/java/org/apache/flume/client/avro/ReliableSpoolingFileEventReader.java +++ b/flume-ng-core/src/main/java/org/apache/flume/client/avro/ReliableSpoolingFileEventReader.java @@ -92,6 +92,7 @@ public class ReliableSpoolingFileEventReader implements ReliableEventReader { private final String completedSuffix; private final String deserializerType; private final Context deserializerContext; + private final Pattern includePattern; private final Pattern ignorePattern; private final File metaFile; private final boolean annotateFileName; @@ -118,7 +119,7 @@ public class ReliableSpoolingFileEventReader implements ReliableEventReader { * Create a ReliableSpoolingFileEventReader to watch the given directory. */ private ReliableSpoolingFileEventReader(File spoolDirectory, - String completedSuffix, String ignorePattern, String trackerDirPath, + String completedSuffix, String includePattern, String ignorePattern, String trackerDirPath, boolean annotateFileName, String fileNameHeader, boolean annotateBaseName, String baseNameHeader, String deserializerType, Context deserializerContext, @@ -130,6 +131,7 @@ private ReliableSpoolingFileEventReader(File spoolDirectory, // Sanity checks Preconditions.checkNotNull(spoolDirectory); Preconditions.checkNotNull(completedSuffix); + Preconditions.checkNotNull(includePattern); Preconditions.checkNotNull(ignorePattern); Preconditions.checkNotNull(trackerDirPath); Preconditions.checkNotNull(deserializerType); @@ -183,6 +185,7 @@ private ReliableSpoolingFileEventReader(File spoolDirectory, this.fileNameHeader = fileNameHeader; this.annotateBaseName = annotateBaseName; this.baseNameHeader = baseNameHeader; + this.includePattern = Pattern.compile(includePattern); this.ignorePattern = Pattern.compile(ignorePattern); this.deletePolicy = deletePolicy; this.inputCharset = Charset.forName(inputCharset); @@ -250,6 +253,7 @@ public FileVisitResult visitFile(Path candidate, BasicFileAttributes attrs) String fileName = candidate.getFileName().toString(); if (!fileName.endsWith(completedSuffix) && !fileName.startsWith(".") && + includePattern.matcher(fileName).matches() && !ignorePattern.matcher(fileName).matches()) { candidateFiles.add(candidate.toFile()); } @@ -658,6 +662,8 @@ public static class Builder { private File spoolDirectory; private String completedSuffix = SpoolDirectorySourceConfigurationConstants.SPOOLED_FILE_SUFFIX; + private String includePattern = + SpoolDirectorySourceConfigurationConstants.DEFAULT_INCLUDE_PAT; private String ignorePattern = SpoolDirectorySourceConfigurationConstants.DEFAULT_IGNORE_PAT; private String trackerDirPath = @@ -695,6 +701,11 @@ public Builder completedSuffix(String completedSuffix) { return this; } + public Builder includePattern(String includePattern) { + this.includePattern = includePattern; + return this; + } + public Builder ignorePattern(String ignorePattern) { this.ignorePattern = ignorePattern; return this; @@ -762,7 +773,7 @@ public Builder consumeOrder(ConsumeOrder consumeOrder) { public ReliableSpoolingFileEventReader build() throws IOException { return new ReliableSpoolingFileEventReader(spoolDirectory, completedSuffix, - ignorePattern, trackerDirPath, annotateFileName, fileNameHeader, + includePattern, ignorePattern, trackerDirPath, annotateFileName, fileNameHeader, annotateBaseName, baseNameHeader, deserializerType, deserializerContext, deletePolicy, inputCharset, decodeErrorPolicy, consumeOrder, recursiveDirectorySearch); diff --git a/flume-ng-core/src/main/java/org/apache/flume/source/SpoolDirectorySource.java b/flume-ng-core/src/main/java/org/apache/flume/source/SpoolDirectorySource.java index c8c7cda674..107a381879 100644 --- a/flume-ng-core/src/main/java/org/apache/flume/source/SpoolDirectorySource.java +++ b/flume-ng-core/src/main/java/org/apache/flume/source/SpoolDirectorySource.java @@ -57,6 +57,7 @@ public class SpoolDirectorySource extends AbstractSource private boolean basenameHeader; private String basenameHeaderKey; private int batchSize; + private String includePattern; private String ignorePattern; private String trackerDirPath; private String deserializerType; @@ -89,6 +90,7 @@ public synchronized void start() { reader = new ReliableSpoolingFileEventReader.Builder() .spoolDirectory(directory) .completedSuffix(completedSuffix) + .includePattern(includePattern) .ignorePattern(ignorePattern) .trackerDirPath(trackerDirPath) .annotateFileName(fileHeader) @@ -162,6 +164,7 @@ public synchronized void configure(Context context) { context.getString(DECODE_ERROR_POLICY, DEFAULT_DECODE_ERROR_POLICY) .toUpperCase(Locale.ENGLISH)); + includePattern = context.getString(INCLUDE_PAT, DEFAULT_INCLUDE_PAT); ignorePattern = context.getString(IGNORE_PAT, DEFAULT_IGNORE_PAT); trackerDirPath = context.getString(TRACKER_DIR, DEFAULT_TRACKER_DIR); diff --git a/flume-ng-core/src/main/java/org/apache/flume/source/SpoolDirectorySourceConfigurationConstants.java b/flume-ng-core/src/main/java/org/apache/flume/source/SpoolDirectorySourceConfigurationConstants.java index 5859aa239d..a065dc0f15 100644 --- a/flume-ng-core/src/main/java/org/apache/flume/source/SpoolDirectorySourceConfigurationConstants.java +++ b/flume-ng-core/src/main/java/org/apache/flume/source/SpoolDirectorySourceConfigurationConstants.java @@ -59,6 +59,10 @@ public class SpoolDirectorySourceConfigurationConstants { @Deprecated public static final int DEFAULT_BUFFER_MAX_LINE_LENGTH = 5000; + /** Pattern of files to include */ + public static final String INCLUDE_PAT = "includePattern"; + public static final String DEFAULT_INCLUDE_PAT = "^.*$"; // any file + /** Pattern of files to ignore */ public static final String IGNORE_PAT = "ignorePattern"; public static final String DEFAULT_IGNORE_PAT = "^$"; // no effect diff --git a/flume-ng-core/src/test/java/org/apache/flume/client/avro/TestReliableSpoolingFileEventReader.java b/flume-ng-core/src/test/java/org/apache/flume/client/avro/TestReliableSpoolingFileEventReader.java index 59fb1c39a3..b25799995a 100644 --- a/flume-ng-core/src/test/java/org/apache/flume/client/avro/TestReliableSpoolingFileEventReader.java +++ b/flume-ng-core/src/test/java/org/apache/flume/client/avro/TestReliableSpoolingFileEventReader.java @@ -17,30 +17,15 @@ */ package org.apache.flume.client.avro; -import com.google.common.base.Charsets; -import com.google.common.collect.Lists; -import com.google.common.collect.Sets; -import com.google.common.io.Files; -import junit.framework.Assert; -import org.apache.commons.io.FileUtils; -import org.apache.commons.lang.SystemUtils; -import org.apache.flume.Event; -import org.apache.flume.client.avro.ReliableSpoolingFileEventReader.DeletePolicy; -import org.apache.flume.source.SpoolDirectorySourceConfigurationConstants; -import org.apache.flume.source.SpoolDirectorySourceConfigurationConstants.ConsumeOrder; -import org.junit.After; -import org.junit.Before; -import org.junit.Test; -import org.slf4j.Logger; -import org.slf4j.LoggerFactory; - import java.io.File; import java.io.FileFilter; import java.io.IOException; import java.util.ArrayList; +import java.util.Arrays; import java.util.Collection; import java.util.Collections; import java.util.Comparator; +import java.util.HashSet; import java.util.List; import java.util.Map; import java.util.Map.Entry; @@ -52,13 +37,34 @@ import java.util.concurrent.Future; import java.util.concurrent.Semaphore; +import org.apache.commons.io.FileUtils; +import org.apache.commons.lang.SystemUtils; +import org.apache.flume.Event; +import org.apache.flume.client.avro.ReliableSpoolingFileEventReader.DeletePolicy; +import org.apache.flume.source.SpoolDirectorySourceConfigurationConstants; +import org.apache.flume.source.SpoolDirectorySourceConfigurationConstants.ConsumeOrder; +import org.junit.After; +import org.junit.Before; +import org.junit.Test; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +import com.google.common.base.Charsets; +import com.google.common.collect.Lists; +import com.google.common.collect.Sets; +import com.google.common.io.Files; + +import junit.framework.Assert; + public class TestReliableSpoolingFileEventReader { - private static final Logger logger = - LoggerFactory.getLogger(TestReliableSpoolingFileEventReader.class); + private static final Logger logger = LoggerFactory.getLogger(TestReliableSpoolingFileEventReader.class); - private static final File WORK_DIR = new File("target/test/work/" + - TestReliableSpoolingFileEventReader.class.getSimpleName()); + private static final File WORK_DIR = new File( + "target/test/work/" + TestReliableSpoolingFileEventReader.class.getSimpleName()); + + private static final File TRACKER_DIR = new File(WORK_DIR, + SpoolDirectorySourceConfigurationConstants.DEFAULT_TRACKER_DIR); @Before public void setup() throws IOException, InterruptedException { @@ -111,6 +117,62 @@ private void deleteDir(File dir) { } } + private void processEventsWithReader(ReliableEventReader reader, int nEvents) throws IOException { + List events; + do { + events = reader.readEvents(nEvents); + reader.commit(); + } while (!events.isEmpty()); + } + + /** + * Verify if the give dir contains only the given files + * + * @param dir + * the directory to check + * @param files + * the files that should be contained in dir + * @return true only if the dir contains exactly the same files given, false + * otherwise + */ + private boolean checkLeftFilesInDir(File dir, String[] files) { + + List actualFiles = listFiles(dir); + Set expectedFiles = new HashSet(Arrays.asList(files)); + + // Verify if the number of files in the dir is the expected + if (actualFiles.size() != expectedFiles.size()) { + return false; + } + + // Then check files by name + for (File f : actualFiles) { + expectedFiles.remove(f.getName()); + } + + return expectedFiles.isEmpty(); + } + + @Test + public void testIncludePattern() throws IOException { + ReliableEventReader reader = new ReliableSpoolingFileEventReader.Builder() + .spoolDirectory(WORK_DIR) + .includePattern("^file2$") + .deletePolicy(DeletePolicy.IMMEDIATE.toString()) + .build(); + + String[] beforeFiles = { "file0", "file1", "file2", "file3", "emptylineFile" }; + Assert.assertTrue("Expected " + beforeFiles.length + " files in working dir", + checkLeftFilesInDir(WORK_DIR, beforeFiles)); + + processEventsWithReader(reader, 10); + + String[] afterFiles = { "file0", "file1", "file3", "emptylineFile" }; + Assert.assertTrue("Expected " + afterFiles.length + " files left in working dir", + checkLeftFilesInDir(WORK_DIR, afterFiles)); + Assert.assertTrue("Expected no files left in tracker dir", checkLeftFilesInDir(TRACKER_DIR, new String[0])); + } + @Test public void testIgnorePattern() throws IOException { ReliableEventReader reader = @@ -120,22 +182,71 @@ public void testIgnorePattern() throws IOException { .deletePolicy(DeletePolicy.IMMEDIATE.toString()) .build(); - List before = listFiles(WORK_DIR); - Assert.assertEquals("Expected 5, not: " + before, 5, before.size()); + String[] beforeFiles = { "file0", "file1", "file2", "file3", "emptylineFile" }; + Assert.assertTrue("Expected " + beforeFiles.length + " files in working dir", + checkLeftFilesInDir(WORK_DIR, beforeFiles)); - List events; - do { - events = reader.readEvents(10); - reader.commit(); - } while (!events.isEmpty()); + processEventsWithReader(reader, 10); - List after = listFiles(WORK_DIR); - Assert.assertEquals("Expected 1, not: " + after, 1, after.size()); - Assert.assertEquals("file2", after.get(0).getName()); - List trackerFiles = listFiles(new File(WORK_DIR, - SpoolDirectorySourceConfigurationConstants.DEFAULT_TRACKER_DIR)); - Assert.assertEquals("Expected 0, not: " + trackerFiles, 0, - trackerFiles.size()); + String[] files = { "file2" }; + Assert.assertTrue("Expected " + files.length + " files left in working dir", checkLeftFilesInDir(WORK_DIR, files)); + Assert.assertTrue("Expected no files left in tracker dir", checkLeftFilesInDir(TRACKER_DIR, new String[0])); + } + + @Test + public void testIncludeExcludePatternNoConflict() throws IOException { + + // Expected behavior mixing include/exclude conditions: + // - file0, file1, file3: not deleted as matching ignore pattern and not + // matching include pattern + // - file2: deleted as not matching ignore pattern and matching include + // pattern + // - emptylineFile: not deleted as not matching ignore pattern but not + // matching include pattern as well + + ReliableEventReader reader = new ReliableSpoolingFileEventReader.Builder() + .spoolDirectory(WORK_DIR) + .ignorePattern("^file[013]$") + .includePattern("^file2$") + .deletePolicy(DeletePolicy.IMMEDIATE.toString()) + .build(); + + String[] beforeFiles = { "file0", "file1", "file2", "file3", "emptylineFile" }; + Assert.assertTrue("Expected " + beforeFiles.length + " files in working dir", + checkLeftFilesInDir(WORK_DIR, beforeFiles)); + + processEventsWithReader(reader, 10); + + String[] files = { "file0", "file1", "file3", "emptylineFile" }; + Assert.assertTrue("Expected " + files.length + " files left in working dir", checkLeftFilesInDir(WORK_DIR, files)); + Assert.assertTrue("Expected no files left in tracker dir", checkLeftFilesInDir(TRACKER_DIR, new String[0])); + } + + @Test + public void testIncludeExcludePatternConflict() throws IOException { + + // This test will stress what happens when both ignore and include options + // are specified and the two patterns match at the same time. + // Expected behavior: + // - file2: not deleted as both include and ignore patterns match (safety + // measure: ignore always wins on conflict) + + ReliableEventReader reader = new ReliableSpoolingFileEventReader.Builder() + .spoolDirectory(WORK_DIR) + .ignorePattern("^file2$") + .includePattern("^file2$") + .deletePolicy(DeletePolicy.IMMEDIATE.toString()) + .build(); + + String[] beforeFiles = { "file0", "file1", "file2", "file3", "emptylineFile" }; + Assert.assertTrue("Expected " + beforeFiles.length + " files in working dir", + checkLeftFilesInDir(WORK_DIR, beforeFiles)); + + processEventsWithReader(reader, 10); + + String[] files = { "file0", "file1", "file2", "file3", "emptylineFile" }; + Assert.assertTrue("Expected " + files.length + " files left in working dir", checkLeftFilesInDir(WORK_DIR, files)); + Assert.assertTrue("Expected no files left in tracker dir", checkLeftFilesInDir(TRACKER_DIR, new String[0])); } @Test @@ -522,17 +633,17 @@ private void readEventsForFilesInDir(File dir, ReliableEventReader reader, throw new IOException(ex); } } - } + } /* Create expected results out of the files created in the setup method. */ private void createExpectedFromFilesInSetup(Collection expected) { expected.add(""); - for (int i = 0; i < 4; i++) { - for (int j = 0; j < i; j++) { + for (int i = 0; i < 4; i++) { + for (int j = 0; j < i; j++) { expected.add("file" + i + "line" + j); - } + } } } - + private static List listFiles(File dir) { List files = Lists.newArrayList(dir.listFiles(new FileFilter() { @Override diff --git a/flume-ng-doc/sphinx/FlumeUserGuide.rst b/flume-ng-doc/sphinx/FlumeUserGuide.rst index ab71d38ff0..1cf4100152 100644 --- a/flume-ng-doc/sphinx/FlumeUserGuide.rst +++ b/flume-ng-doc/sphinx/FlumeUserGuide.rst @@ -1018,7 +1018,14 @@ fileHeader false Whether to add a header storing the ab fileHeaderKey file Header key to use when appending absolute path filename to event header. basenameHeader false Whether to add a header storing the basename of the file. basenameHeaderKey basename Header Key to use when appending basename of file to event header. -ignorePattern ^$ Regular expression specifying which files to ignore (skip) +includePattern ^.*$ Regular expression specifying which files to include. + It can used together with ``ignorePattern``. + If a file matches both ``ignorePattern`` and ``includePattern`` regex, + the file is ignored. +ignorePattern ^$ Regular expression specifying which files to ignore (skip). + It can used together with ``includePattern``. + If a file matches both ``ignorePattern`` and ``includePattern`` regex, + the file is ignored. trackerDir .flumespool Directory to store metadata related to processing of files. If this path is not an absolute path, then it is interpreted as relative to the spoolDir. consumeOrder oldest In which order files in the spooling directory will be consumed ``oldest``, From 585c4c92ed65c529871765ab64a5b3637fbdb07d Mon Sep 17 00:00:00 2001 From: Tristan Stevens Date: Mon, 10 Oct 2016 19:32:24 +0200 Subject: [PATCH 332/341] FLUME-2999. Kafka channel and sink should enable statically assigned partition per event via header This feature is useful for anyone who needs greater control of which partitions are being written to - normally in a situation where multiple Flume agents are being deployed in order to horizontally scale, or alternatively if there is a scenario where there is a skew in data that might lead to one or more partitions hotspotting. We also have the ability to specify custom partitions on to the Kafka Producer itself using the kafka.* configuration properties. The Kafka Producer provides the ability to set the partition ID using a particular ProducerRecord constructor, this is just a matter of providing the option to use this constructor. Reviewers: Attila Simon, Mike Percy (Tristan Stevens via Mike Percy) --- flume-ng-channels/flume-kafka-channel/pom.xml | 5 + .../flume/channel/kafka/KafkaChannel.java | 31 ++- .../kafka/KafkaChannelConfiguration.java | 3 + .../flume/channel/kafka/TestKafkaChannel.java | 149 +++++++++++- flume-ng-doc/sphinx/FlumeUserGuide.rst | 16 ++ flume-ng-sinks/flume-ng-kafka-sink/pom.xml | 6 + .../apache/flume/sink/kafka/KafkaSink.java | 38 ++- .../flume/sink/kafka/KafkaSinkConstants.java | 3 + .../flume/sink/kafka/TestKafkaSink.java | 222 ++++++++++++++++++ flume-ng-sources/flume-kafka-source/pom.xml | 5 + flume-shared/flume-shared-kafka-test/pom.xml | 85 +++++++ .../kafka/test/KafkaPartitionTestUtil.java | 213 +++++++++++++++++ .../shared/kafka/test/PartitionOption.java | 25 ++ .../kafka/test/PartitionTestScenario.java | 26 ++ flume-shared/pom.xml | 61 +++++ pom.xml | 9 +- 16 files changed, 886 insertions(+), 11 deletions(-) create mode 100644 flume-shared/flume-shared-kafka-test/pom.xml create mode 100644 flume-shared/flume-shared-kafka-test/src/main/java/org/apache/flume/shared/kafka/test/KafkaPartitionTestUtil.java create mode 100644 flume-shared/flume-shared-kafka-test/src/main/java/org/apache/flume/shared/kafka/test/PartitionOption.java create mode 100644 flume-shared/flume-shared-kafka-test/src/main/java/org/apache/flume/shared/kafka/test/PartitionTestScenario.java create mode 100644 flume-shared/pom.xml diff --git a/flume-ng-channels/flume-kafka-channel/pom.xml b/flume-ng-channels/flume-kafka-channel/pom.xml index c1cc844fad..a3c6de24f0 100644 --- a/flume-ng-channels/flume-kafka-channel/pom.xml +++ b/flume-ng-channels/flume-kafka-channel/pom.xml @@ -37,6 +37,11 @@ limitations under the License. org.apache.flume flume-ng-sdk + + org.apache.flume.flume-shared + flume-shared-kafka-test + test + org.apache.kafka kafka_2.10 diff --git a/flume-ng-channels/flume-kafka-channel/src/main/java/org/apache/flume/channel/kafka/KafkaChannel.java b/flume-ng-channels/flume-kafka-channel/src/main/java/org/apache/flume/channel/kafka/KafkaChannel.java index 66b553ad62..47c06348d5 100644 --- a/flume-ng-channels/flume-kafka-channel/src/main/java/org/apache/flume/channel/kafka/KafkaChannel.java +++ b/flume-ng-channels/flume-kafka-channel/src/main/java/org/apache/flume/channel/kafka/KafkaChannel.java @@ -98,6 +98,8 @@ public class KafkaChannel extends BasicChannelSemantics { private String zookeeperConnect = null; private String topicStr = DEFAULT_TOPIC; private String groupId = DEFAULT_GROUP_ID; + private String partitionHeader = null; + private Integer staticPartitionId; private boolean migrateZookeeperOffsets = DEFAULT_MIGRATE_ZOOKEEPER_OFFSETS; //used to indicate if a rebalance has occurred during the current transaction @@ -191,6 +193,9 @@ public void configure(Context ctx) { parseAsFlumeEvent = ctx.getBoolean(PARSE_AS_FLUME_EVENT, DEFAULT_PARSE_AS_FLUME_EVENT); pollTimeout = ctx.getLong(POLL_TIMEOUT, DEFAULT_POLL_TIMEOUT); + staticPartitionId = ctx.getInteger(STATIC_PARTITION_CONF); + partitionHeader = ctx.getString(PARTITION_HEADER_NAME); + migrateZookeeperOffsets = ctx.getBoolean(MIGRATE_ZOOKEEPER_OFFSETS, DEFAULT_MIGRATE_ZOOKEEPER_OFFSETS); zookeeperConnect = ctx.getString(ZOOKEEPER_CONNECT_FLUME_KEY); @@ -421,10 +426,30 @@ protected void doPut(Event event) throws InterruptedException { producerRecords = Optional.of(new LinkedList>()); } String key = event.getHeaders().get(KEY_HEADER); + + Integer partitionId = null; try { - producerRecords.get().add( - new ProducerRecord(topic.get(), key, - serializeValue(event, parseAsFlumeEvent))); + if (staticPartitionId != null) { + partitionId = staticPartitionId; + } + //Allow a specified header to override a static ID + if (partitionHeader != null) { + String headerVal = event.getHeaders().get(partitionHeader); + if (headerVal != null) { + partitionId = Integer.parseInt(headerVal); + } + } + if (partitionId != null) { + producerRecords.get().add( + new ProducerRecord(topic.get(), partitionId, key, + serializeValue(event, parseAsFlumeEvent))); + } else { + producerRecords.get().add( + new ProducerRecord(topic.get(), key, + serializeValue(event, parseAsFlumeEvent))); + } + } catch (NumberFormatException e) { + throw new ChannelException("Non integer partition id specified", e); } catch (Exception e) { throw new ChannelException("Error while serializing event", e); } diff --git a/flume-ng-channels/flume-kafka-channel/src/main/java/org/apache/flume/channel/kafka/KafkaChannelConfiguration.java b/flume-ng-channels/flume-kafka-channel/src/main/java/org/apache/flume/channel/kafka/KafkaChannelConfiguration.java index 3ab807b53c..ad5a15bf8b 100644 --- a/flume-ng-channels/flume-kafka-channel/src/main/java/org/apache/flume/channel/kafka/KafkaChannelConfiguration.java +++ b/flume-ng-channels/flume-kafka-channel/src/main/java/org/apache/flume/channel/kafka/KafkaChannelConfiguration.java @@ -49,6 +49,9 @@ public class KafkaChannelConfiguration { public static final String PARSE_AS_FLUME_EVENT = "parseAsFlumeEvent"; public static final boolean DEFAULT_PARSE_AS_FLUME_EVENT = true; + public static final String PARTITION_HEADER_NAME = "partitionIdHeader"; + public static final String STATIC_PARTITION_CONF = "defaultPartitionId"; + public static final String MIGRATE_ZOOKEEPER_OFFSETS = "migrateZookeeperOffsets"; public static final boolean DEFAULT_MIGRATE_ZOOKEEPER_OFFSETS = true; diff --git a/flume-ng-channels/flume-kafka-channel/src/test/java/org/apache/flume/channel/kafka/TestKafkaChannel.java b/flume-ng-channels/flume-kafka-channel/src/test/java/org/apache/flume/channel/kafka/TestKafkaChannel.java index 57c0b288d5..276fee1fdb 100644 --- a/flume-ng-channels/flume-kafka-channel/src/test/java/org/apache/flume/channel/kafka/TestKafkaChannel.java +++ b/flume-ng-channels/flume-kafka-channel/src/test/java/org/apache/flume/channel/kafka/TestKafkaChannel.java @@ -28,6 +28,9 @@ import org.apache.flume.Transaction; import org.apache.flume.conf.Configurables; import org.apache.flume.event.EventBuilder; +import org.apache.flume.shared.kafka.test.KafkaPartitionTestUtil; +import org.apache.flume.shared.kafka.test.PartitionOption; +import org.apache.flume.shared.kafka.test.PartitionTestScenario; import org.apache.flume.sink.kafka.util.TestUtil; import org.apache.kafka.clients.CommonClientConfigs; import org.apache.kafka.clients.consumer.ConsumerConfig; @@ -69,6 +72,8 @@ public class TestKafkaChannel { private String topic = null; private final Set usedTopics = new HashSet(); + private static final int DEFAULT_TOPIC_PARTITIONS = 5; + @BeforeClass public static void setupClass() throws Exception { testUtil.prepare(); @@ -79,7 +84,7 @@ public static void setupClass() throws Exception { public void setup() throws Exception { topic = findUnusedTopic(); try { - createTopic(topic, 5); + createTopic(topic, DEFAULT_TOPIC_PARTITIONS); } catch (Exception e) { } Thread.sleep(2500); @@ -248,6 +253,41 @@ public void testMigrateOffsetsBoth() throws Exception { doTestMigrateZookeeperOffsets(true, true, "testMigrateOffsets-both"); } + @Test + public void testPartitionHeaderSet() throws Exception { + doPartitionHeader(PartitionTestScenario.PARTITION_ID_HEADER_ONLY); + } + + @Test + public void testPartitionHeaderNotSet() throws Exception { + doPartitionHeader(PartitionTestScenario.NO_PARTITION_HEADERS); + } + + @Test + public void testStaticPartitionAndHeaderSet() throws Exception { + doPartitionHeader(PartitionTestScenario.STATIC_HEADER_AND_PARTITION_ID); + } + + @Test + public void testStaticPartitionHeaderNotSet() throws Exception { + doPartitionHeader(PartitionTestScenario.STATIC_HEADER_ONLY); + } + + @Test + public void testPartitionHeaderMissing() throws Exception { + doPartitionErrors(PartitionOption.NOTSET); + } + + @Test(expected = org.apache.flume.ChannelException.class) + public void testPartitionHeaderOutOfRange() throws Exception { + doPartitionErrors(PartitionOption.VALIDBUTOUTOFRANGE); + } + + @Test(expected = org.apache.flume.ChannelException.class) + public void testPartitionHeaderInvalid() throws Exception { + doPartitionErrors(PartitionOption.NOTANUMBER); + } + public void doTestMigrateZookeeperOffsets(boolean hasZookeeperOffsets, boolean hasKafkaOffsets, String group) throws Exception { // create a topic with 1 partition for simplicity @@ -328,6 +368,112 @@ public void doTestMigrateZookeeperOffsets(boolean hasZookeeperOffsets, boolean h } } + /** + * This function tests three scenarios: + * 1. PartitionOption.VALIDBUTOUTOFRANGE: An integer partition is provided, + * however it exceeds the number of partitions available on the topic. + * Expected behaviour: ChannelException thrown. + * + * 2. PartitionOption.NOTSET: The partition header is not actually set. + * Expected behaviour: Exception is not thrown because the code avoids an NPE. + * + * 3. PartitionOption.NOTANUMBER: The partition header is set, but is not an Integer. + * Expected behaviour: ChannelExeption thrown. + * + * @param option + * @throws Exception + */ + private void doPartitionErrors(PartitionOption option) throws Exception { + Context context = prepareDefaultContext(false); + context.put(PARTITION_HEADER_NAME, KafkaPartitionTestUtil.PARTITION_HEADER); + String tempTopic = findUnusedTopic(); + createTopic(tempTopic, 5); + final KafkaChannel channel = createChannel(context); + channel.start(); + + Transaction tx = channel.getTransaction(); + tx.begin(); + + Map headers = new HashMap(); + switch (option) { + case VALIDBUTOUTOFRANGE: + headers.put(KafkaPartitionTestUtil.PARTITION_HEADER, + String.valueOf(DEFAULT_TOPIC_PARTITIONS + 2)); + break; + case NOTSET: + headers.put("wrong-header", "2"); + break; + case NOTANUMBER: + headers.put(KafkaPartitionTestUtil.PARTITION_HEADER, "not-a-number"); + break; + default: + break; + } + + Event event = EventBuilder.withBody(String.valueOf(9).getBytes(), headers); + + channel.put(event); + + tx.commit(); + + deleteTopic(tempTopic); + } + + /** + * This method tests both the default behavior (usePartitionHeader=false) + * and the behaviour when the partitionId setting is used. + * Under the default behaviour, one would expect an even distribution of + * messages to partitions, however when partitionId is used we manually create + * a large skew to some partitions and then verify that this actually happened + * by reading messages directly using a Kafka Consumer. + * + * @param usePartitionHeader + * @param staticPtn + * @throws Exception + */ + private void doPartitionHeader(PartitionTestScenario scenario) throws Exception { + final int numPtns = DEFAULT_TOPIC_PARTITIONS; + final int numMsgs = numPtns * 10; + final Integer staticPtn = DEFAULT_TOPIC_PARTITIONS - 2 ; + Context context = prepareDefaultContext(false); + if (scenario == PartitionTestScenario.PARTITION_ID_HEADER_ONLY || + scenario == PartitionTestScenario.STATIC_HEADER_AND_PARTITION_ID) { + context.put(PARTITION_HEADER_NAME, "partition-header"); + } + if (scenario == PartitionTestScenario.STATIC_HEADER_AND_PARTITION_ID || + scenario == PartitionTestScenario.STATIC_HEADER_ONLY) { + context.put(STATIC_PARTITION_CONF, staticPtn.toString()); + } + final KafkaChannel channel = createChannel(context); + channel.start(); + + // Create a map of PartitionId:List according to the desired distribution + // Initialise with empty ArrayLists + Map> partitionMap = new HashMap>(numPtns); + for (int i = 0; i < numPtns; i++) { + partitionMap.put(i, new ArrayList()); + } + Transaction tx = channel.getTransaction(); + tx.begin(); + + List orderedEvents = KafkaPartitionTestUtil.generateSkewedMessageList(scenario, numMsgs, + partitionMap, numPtns, staticPtn); + + for (Event event : orderedEvents) { + channel.put(event); + } + + tx.commit(); + + Map> resultsMap = KafkaPartitionTestUtil.retrieveRecordsFromPartitions( + topic, numPtns, channel.getConsumerProps()); + + KafkaPartitionTestUtil.checkResultsAgainstSkew(scenario, partitionMap, resultsMap, staticPtn, + numMsgs); + + channel.stop(); + } + private Event takeEventWithoutCommittingTxn(KafkaChannel channel) { for (int i = 0; i < 5; i++) { Transaction txn = channel.getTransaction(); @@ -713,4 +859,5 @@ public static void deleteTopic(String topicName) { ZkUtils.apply(testUtil.getZkUrl(), sessionTimeoutMs, connectionTimeoutMs, false); AdminUtils.deleteTopic(zkUtils, topicName); } + } diff --git a/flume-ng-doc/sphinx/FlumeUserGuide.rst b/flume-ng-doc/sphinx/FlumeUserGuide.rst index 1cf4100152..25db777c3d 100644 --- a/flume-ng-doc/sphinx/FlumeUserGuide.rst +++ b/flume-ng-doc/sphinx/FlumeUserGuide.rst @@ -2583,6 +2583,14 @@ useFlumeEventFormat false By default events are put true to store events as the Flume Avro binary format. Used in conjunction with the same property on the KafkaSource or with the parseAsFlumeEvent property on the Kafka Channel this will preserve any Flume headers for the producing side. +defaultPartitionId -- Specifies a Kafka partition ID (integer) for all events in this channel to be sent to, unless + overriden by ``partitionIdHeader``. By default, if this property is not set, events will be + distributed by the Kafka Producer's partitioner - including by ``key`` if specified (or by a + partitioner specified by ``kafka.partitioner.class``). +partitionIdHeader -- When set, the sink will take the value of the field named using the value of this property + from the event header and send the message to the specified partition of the topic. If the + value represents an invalid partition, an EventDeliveryException will be thrown. If the header value + is present then this setting overrides ``defaultPartitionId``. Other Kafka Producer Properties -- These properties are used to configure the Kafka Producer. Any producer property supported by Kafka can be used. The only requirement is to prepend the property name with the prefix ``kafka.producer``. @@ -2786,6 +2794,14 @@ migrateZookeeperOffsets true When no Kaf configuration defines how offsets are handled. pollTimeout 500 The amount of time(in milliseconds) to wait in the "poll()" call of the conumer. https://kafka.apache.org/090/javadoc/org/apache/kafka/clients/consumer/KafkaConsumer.html#poll(long) +defaultPartitionId -- Specifies a Kafka partition ID (integer) for all events in this channel to be sent to, unless + overriden by ``partitionIdHeader``. By default, if this property is not set, events will be + distributed by the Kafka Producer's partitioner - including by ``key`` if specified (or by a + partitioner specified by ``kafka.partitioner.class``). +partitionIdHeader -- When set, the producer will take the value of the field named using the value of this property + from the event header and send the message to the specified partition of the topic. If the + value represents an invalid partition the event will not be accepted into the channel. If the header value + is present then this setting overrides ``defaultPartitionId``. kafka.consumer.auto.offset.reset latest What to do when there is no initial offset in Kafka or if the current offset does not exist any more on the server (e.g. because that data has been deleted): earliest: automatically reset the offset to the earliest offset diff --git a/flume-ng-sinks/flume-ng-kafka-sink/pom.xml b/flume-ng-sinks/flume-ng-kafka-sink/pom.xml index 195c921434..2ab4b4addf 100644 --- a/flume-ng-sinks/flume-ng-kafka-sink/pom.xml +++ b/flume-ng-sinks/flume-ng-kafka-sink/pom.xml @@ -58,6 +58,12 @@ flume-ng-configuration + + org.apache.flume.flume-shared + flume-shared-kafka-test + test + + org.slf4j slf4j-api diff --git a/flume-ng-sinks/flume-ng-kafka-sink/src/main/java/org/apache/flume/sink/kafka/KafkaSink.java b/flume-ng-sinks/flume-ng-kafka-sink/src/main/java/org/apache/flume/sink/kafka/KafkaSink.java index 89bdd84588..dd40224dea 100644 --- a/flume-ng-sinks/flume-ng-kafka-sink/src/main/java/org/apache/flume/sink/kafka/KafkaSink.java +++ b/flume-ng-sinks/flume-ng-kafka-sink/src/main/java/org/apache/flume/sink/kafka/KafkaSink.java @@ -114,6 +114,8 @@ public class KafkaSink extends AbstractSink implements Configurable { private List> kafkaFutures; private KafkaSinkCounter counter; private boolean useAvroEventFormat; + private String partitionHeader = null; + private Integer staticPartitionId = null; private Optional> writer = Optional.absent(); private Optional> reader = @@ -187,13 +189,34 @@ public Status process() throws EventDeliveryException { // create a message and add to buffer long startTime = System.currentTimeMillis(); + Integer partitionId = null; try { - kafkaFutures.add(producer.send( - new ProducerRecord(eventTopic, eventKey, - serializeEvent(event, useAvroEventFormat)), - new SinkCallback(startTime))); - } catch (IOException ex) { - throw new EventDeliveryException("Could not serialize event", ex); + ProducerRecord record; + if (staticPartitionId != null) { + partitionId = staticPartitionId; + } + //Allow a specified header to override a static ID + if (partitionHeader != null) { + String headerVal = event.getHeaders().get(partitionHeader); + if (headerVal != null) { + partitionId = Integer.parseInt(headerVal); + } + } + if (partitionId != null) { + record = new ProducerRecord(eventTopic, partitionId, eventKey, + serializeEvent(event, useAvroEventFormat)); + } else { + record = new ProducerRecord(eventTopic, eventKey, + serializeEvent(event, useAvroEventFormat)); + } + kafkaFutures.add(producer.send(record, new SinkCallback(startTime))); + } catch (NumberFormatException ex) { + throw new EventDeliveryException("Non integer partition id specified", ex); + } catch (Exception ex) { + // N.B. The producer.send() method throws all sorts of RuntimeExceptions + // Catching Exception here to wrap them neatly in an EventDeliveryException + // which is what our consumers will expect + throw new EventDeliveryException("Could not send event", ex); } } @@ -290,6 +313,9 @@ public void configure(Context context) { useAvroEventFormat = context.getBoolean(KafkaSinkConstants.AVRO_EVENT, KafkaSinkConstants.DEFAULT_AVRO_EVENT); + partitionHeader = context.getString(KafkaSinkConstants.PARTITION_HEADER_NAME); + staticPartitionId = context.getInteger(KafkaSinkConstants.STATIC_PARTITION_CONF); + if (logger.isDebugEnabled()) { logger.debug(KafkaSinkConstants.AVRO_EVENT + " set to: {}", useAvroEventFormat); } diff --git a/flume-ng-sinks/flume-ng-kafka-sink/src/main/java/org/apache/flume/sink/kafka/KafkaSinkConstants.java b/flume-ng-sinks/flume-ng-kafka-sink/src/main/java/org/apache/flume/sink/kafka/KafkaSinkConstants.java index 1bf380c8f7..7c819f5df5 100644 --- a/flume-ng-sinks/flume-ng-kafka-sink/src/main/java/org/apache/flume/sink/kafka/KafkaSinkConstants.java +++ b/flume-ng-sinks/flume-ng-kafka-sink/src/main/java/org/apache/flume/sink/kafka/KafkaSinkConstants.java @@ -38,6 +38,9 @@ public class KafkaSinkConstants { public static final String AVRO_EVENT = "useFlumeEventFormat"; public static final boolean DEFAULT_AVRO_EVENT = false; + public static final String PARTITION_HEADER_NAME = "partitionIdHeader"; + public static final String STATIC_PARTITION_CONF = "defaultPartitionId"; + public static final String DEFAULT_KEY_SERIALIZER = "org.apache.kafka.common.serialization.StringSerializer"; public static final String DEFAULT_VALUE_SERIAIZER = diff --git a/flume-ng-sinks/flume-ng-kafka-sink/src/test/java/org/apache/flume/sink/kafka/TestKafkaSink.java b/flume-ng-sinks/flume-ng-kafka-sink/src/test/java/org/apache/flume/sink/kafka/TestKafkaSink.java index 76eca37160..7eccf76467 100644 --- a/flume-ng-sinks/flume-ng-kafka-sink/src/test/java/org/apache/flume/sink/kafka/TestKafkaSink.java +++ b/flume-ng-sinks/flume-ng-kafka-sink/src/test/java/org/apache/flume/sink/kafka/TestKafkaSink.java @@ -19,11 +19,16 @@ Licensed to the Apache Software Foundation (ASF) under one or more package org.apache.flume.sink.kafka; import com.google.common.base.Charsets; + +import kafka.admin.AdminUtils; import kafka.message.MessageAndMetadata; +import kafka.utils.ZkUtils; + import org.apache.avro.io.BinaryDecoder; import org.apache.avro.io.DecoderFactory; import org.apache.avro.specific.SpecificDatumReader; import org.apache.avro.util.Utf8; +import org.apache.commons.lang.RandomStringUtils; import org.apache.flume.Channel; import org.apache.flume.Context; import org.apache.flume.Event; @@ -33,6 +38,9 @@ Licensed to the Apache Software Foundation (ASF) under one or more import org.apache.flume.channel.MemoryChannel; import org.apache.flume.conf.Configurables; import org.apache.flume.event.EventBuilder; +import org.apache.flume.shared.kafka.test.KafkaPartitionTestUtil; +import org.apache.flume.shared.kafka.test.PartitionOption; +import org.apache.flume.shared.kafka.test.PartitionTestScenario; import org.apache.flume.sink.kafka.util.TestUtil; import org.apache.flume.source.avro.AvroFlumeEvent; import org.apache.kafka.clients.CommonClientConfigs; @@ -46,9 +54,11 @@ Licensed to the Apache Software Foundation (ASF) under one or more import java.io.UnsupportedEncodingException; import java.util.ArrayList; import java.util.HashMap; +import java.util.HashSet; import java.util.List; import java.util.Map; import java.util.Properties; +import java.util.Set; import static org.apache.flume.sink.kafka.KafkaSinkConstants.AVRO_EVENT; import static org.apache.flume.sink.kafka.KafkaSinkConstants.BATCH_SIZE; @@ -71,6 +81,7 @@ Licensed to the Apache Software Foundation (ASF) under one or more public class TestKafkaSink { private static TestUtil testUtil = TestUtil.getInstance(); + private final Set usedTopics = new HashSet(); @BeforeClass public static void setup() { @@ -298,6 +309,186 @@ public void testEmptyChannel() throws UnsupportedEncodingException, EventDeliver assertNull(testUtil.getNextMessageFromConsumer(DEFAULT_TOPIC)); } + @Test + public void testPartitionHeaderSet() throws Exception { + doPartitionHeader(PartitionTestScenario.PARTITION_ID_HEADER_ONLY); + } + + @Test + public void testPartitionHeaderNotSet() throws Exception { + doPartitionHeader(PartitionTestScenario.NO_PARTITION_HEADERS); + } + + @Test + public void testStaticPartitionAndHeaderSet() throws Exception { + doPartitionHeader(PartitionTestScenario.STATIC_HEADER_AND_PARTITION_ID); + } + + @Test + public void testStaticPartitionHeaderNotSet() throws Exception { + doPartitionHeader(PartitionTestScenario.STATIC_HEADER_ONLY); + } + + @Test + public void testPartitionHeaderMissing() throws Exception { + doPartitionErrors(PartitionOption.NOTSET); + } + + @Test(expected = org.apache.flume.EventDeliveryException.class) + public void testPartitionHeaderOutOfRange() throws Exception { + doPartitionErrors(PartitionOption.VALIDBUTOUTOFRANGE); + } + + @Test(expected = org.apache.flume.EventDeliveryException.class) + public void testPartitionHeaderInvalid() throws Exception { + doPartitionErrors(PartitionOption.NOTANUMBER); + } + + /** + * This function tests three scenarios: + * 1. PartitionOption.VALIDBUTOUTOFRANGE: An integer partition is provided, + * however it exceeds the number of partitions available on the topic. + * Expected behaviour: ChannelException thrown. + * + * 2. PartitionOption.NOTSET: The partition header is not actually set. + * Expected behaviour: Exception is not thrown because the code avoids an NPE. + * + * 3. PartitionOption.NOTANUMBER: The partition header is set, but is not an Integer. + * Expected behaviour: ChannelExeption thrown. + * + * @param option + * @throws Exception + */ + private void doPartitionErrors(PartitionOption option) throws Exception { + Sink kafkaSink = new KafkaSink(); + Context context = prepareDefaultContext(); + context.put(KafkaSinkConstants.PARTITION_HEADER_NAME, "partition-header"); + + Configurables.configure(kafkaSink, context); + Channel memoryChannel = new MemoryChannel(); + Configurables.configure(memoryChannel, context); + kafkaSink.setChannel(memoryChannel); + kafkaSink.start(); + + String topic = findUnusedTopic(); + createTopic(topic, 5); + + Transaction tx = memoryChannel.getTransaction(); + tx.begin(); + + Map headers = new HashMap(); + headers.put("topic", topic); + switch (option) { + case VALIDBUTOUTOFRANGE: + headers.put("partition-header", "9"); + break; + case NOTSET: + headers.put("wrong-header", "2"); + break; + case NOTANUMBER: + headers.put("partition-header", "not-a-number"); + break; + default: + break; + } + + Event event = EventBuilder.withBody(String.valueOf(9).getBytes(), headers); + + memoryChannel.put(event); + tx.commit(); + tx.close(); + + Sink.Status status = kafkaSink.process(); + assertEquals(Sink.Status.READY, status); + + deleteTopic(topic); + + } + + /** + * This method tests both the default behavior (usePartitionHeader=false) + * and the behaviour when the partitionId setting is used. + * Under the default behaviour, one would expect an even distribution of + * messages to partitions, however when partitionId is used we manually create + * a large skew to some partitions and then verify that this actually happened + * by reading messages directly using a Kafka Consumer. + * + * @param usePartitionHeader + * @param staticPtn + * @throws Exception + */ + private void doPartitionHeader(PartitionTestScenario scenario) throws Exception { + final int numPtns = 5; + final int numMsgs = numPtns * 10; + final Integer staticPtn = 3; + + String topic = findUnusedTopic(); + createTopic(topic, numPtns); + Context context = prepareDefaultContext(); + context.put(BATCH_SIZE, "100"); + + if (scenario == PartitionTestScenario.PARTITION_ID_HEADER_ONLY || + scenario == PartitionTestScenario.STATIC_HEADER_AND_PARTITION_ID) { + context.put(KafkaSinkConstants.PARTITION_HEADER_NAME, + KafkaPartitionTestUtil.PARTITION_HEADER); + } + if (scenario == PartitionTestScenario.STATIC_HEADER_AND_PARTITION_ID || + scenario == PartitionTestScenario.STATIC_HEADER_ONLY) { + context.put(KafkaSinkConstants.STATIC_PARTITION_CONF, staticPtn.toString()); + } + Sink kafkaSink = new KafkaSink(); + + Configurables.configure(kafkaSink, context); + Channel memoryChannel = new MemoryChannel(); + Configurables.configure(memoryChannel, context); + kafkaSink.setChannel(memoryChannel); + kafkaSink.start(); + + //Create a map of PartitionId:List according to the desired distribution + Map> partitionMap = new HashMap>(numPtns); + for (int i = 0; i < numPtns; i++) { + partitionMap.put(i, new ArrayList()); + } + Transaction tx = memoryChannel.getTransaction(); + tx.begin(); + + List orderedEvents = KafkaPartitionTestUtil.generateSkewedMessageList(scenario, numMsgs, + partitionMap, numPtns, staticPtn); + + for (Event event : orderedEvents) { + event.getHeaders().put("topic", topic); + memoryChannel.put(event); + } + + tx.commit(); + tx.close(); + + Sink.Status status = kafkaSink.process(); + assertEquals(Sink.Status.READY, status); + + Properties props = new Properties(); + props.put("bootstrap.servers", testUtil.getKafkaServerUrl()); + props.put("group.id", "group_1"); + props.put("enable.auto.commit", "true"); + props.put("auto.commit.interval.ms", "1000"); + props.put("session.timeout.ms", "30000"); + props.put("key.deserializer", + "org.apache.kafka.common.serialization.StringDeserializer"); + props.put("value.deserializer", + "org.apache.kafka.common.serialization.ByteArrayDeserializer"); + props.put("auto.offset.reset", "earliest"); + Map> resultsMap = + KafkaPartitionTestUtil.retrieveRecordsFromPartitions(topic, numPtns, props); + + KafkaPartitionTestUtil.checkResultsAgainstSkew(scenario, partitionMap, resultsMap, staticPtn, + numMsgs); + + memoryChannel.stop(); + kafkaSink.stop(); + deleteTopic(topic); + + } + private Context prepareDefaultContext() { // Prepares a default context with Kafka Server Properties Context context = new Context(); @@ -325,4 +516,35 @@ private Sink.Status prepareAndSend(Context context, String msg) return kafkaSink.process(); } + public static void createTopic(String topicName, int numPartitions) { + int sessionTimeoutMs = 10000; + int connectionTimeoutMs = 10000; + ZkUtils zkUtils = + ZkUtils.apply(testUtil.getZkUrl(), sessionTimeoutMs, connectionTimeoutMs, false); + int replicationFactor = 1; + Properties topicConfig = new Properties(); + AdminUtils.createTopic(zkUtils, topicName, numPartitions, replicationFactor, topicConfig); + } + + public static void deleteTopic(String topicName) { + int sessionTimeoutMs = 10000; + int connectionTimeoutMs = 10000; + ZkUtils zkUtils = + ZkUtils.apply(testUtil.getZkUrl(), sessionTimeoutMs, connectionTimeoutMs, false); + AdminUtils.deleteTopic(zkUtils, topicName); + } + + public String findUnusedTopic() { + String newTopic = null; + boolean topicFound = false; + while (!topicFound) { + newTopic = RandomStringUtils.randomAlphabetic(8); + if (!usedTopics.contains(newTopic)) { + usedTopics.add(newTopic); + topicFound = true; + } + } + return newTopic; + } + } diff --git a/flume-ng-sources/flume-kafka-source/pom.xml b/flume-ng-sources/flume-kafka-source/pom.xml index c89ea1a299..80bb7f45fa 100644 --- a/flume-ng-sources/flume-kafka-source/pom.xml +++ b/flume-ng-sources/flume-kafka-source/pom.xml @@ -48,6 +48,11 @@ org.apache.flume flume-ng-core + + org.apache.flume.flume-shared + flume-shared-kafka-test + test + org.apache.kafka kafka_2.10 diff --git a/flume-shared/flume-shared-kafka-test/pom.xml b/flume-shared/flume-shared-kafka-test/pom.xml new file mode 100644 index 0000000000..00260a9502 --- /dev/null +++ b/flume-shared/flume-shared-kafka-test/pom.xml @@ -0,0 +1,85 @@ + + + + 4.0.0 + + flume-shared + org.apache.flume + 1.7.0-SNAPSHOT + + org.apache.flume.flume-shared + flume-shared-kafka-test + Flume Shared Kafka Test Utils + + + + + org.apache.rat + apache-rat-plugin + + + org.apache.maven.plugins + maven-jar-plugin + + + + test-jar + + + + + + + + + + org.apache.flume + flume-ng-sdk + + + + org.apache.flume + flume-ng-core + + + + org.apache.flume + flume-ng-configuration + + + + org.slf4j + slf4j-api + + + + junit + junit + + + + org.apache.kafka + kafka_2.10 + provided + + + org.apache.kafka + kafka-clients + ${kafka.version} + provided + + + + + diff --git a/flume-shared/flume-shared-kafka-test/src/main/java/org/apache/flume/shared/kafka/test/KafkaPartitionTestUtil.java b/flume-shared/flume-shared-kafka-test/src/main/java/org/apache/flume/shared/kafka/test/KafkaPartitionTestUtil.java new file mode 100644 index 0000000000..cb1e258524 --- /dev/null +++ b/flume-shared/flume-shared-kafka-test/src/main/java/org/apache/flume/shared/kafka/test/KafkaPartitionTestUtil.java @@ -0,0 +1,213 @@ +/** + Licensed to the Apache Software Foundation (ASF) under one or more + contributor license agreements. See the NOTICE file distributed with + this work for additional information regarding copyright ownership. + The ASF licenses this file to You under the Apache License, Version 2.0 + (the "License"); you may not use this file except in compliance with + the License. You may obtain a copy of the License at + + http://www.apache.org/licenses/LICENSE-2.0 + + Unless required by applicable law or agreed to in writing, software + distributed under the License is distributed on an "AS IS" BASIS, + WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + See the License for the specific language governing permissions and + limitations under the License. + limitations under the License. + */ + +package org.apache.flume.shared.kafka.test; + +import java.util.ArrayList; +import java.util.Arrays; +import java.util.HashMap; +import java.util.List; +import java.util.Map; +import java.util.Properties; + +import org.apache.flume.Event; +import org.apache.flume.event.EventBuilder; +import org.apache.kafka.clients.consumer.ConsumerRecord; +import org.apache.kafka.clients.consumer.ConsumerRecords; +import org.apache.kafka.clients.consumer.KafkaConsumer; +import org.apache.kafka.common.TopicPartition; +import org.junit.Assert; + +public class KafkaPartitionTestUtil { + + public static final String PARTITION_HEADER = "partition-header"; + + /** + * This method checks the retrieved messages (passed as resultsMap) against the expected + * results (passed as partitionMap). The behaviour of this method is slightly different + * depending on the scenario: + * - STATIC_HEADER_ONLY: Don't check partitionMap, just check that all messages are in the + * passed staticPtn partition. + * - NO_PARTITION_HEADERS: Check that messages are evenly distributed between partitions + * (requires numMsgs to be a multiple of the number of partitons) + * - else: Check that the contents of each partition list in resultsMap is the same as that + * specified in partitionMap. + * + * As this is a testing method, it will issue JUnit AssertionExceptions if the results are not + * as expected. + * + * @param scenario + * @param partitionMap + * @param resultsMap + * @param staticPtn + * @param numMsgs + */ + public static void checkResultsAgainstSkew(PartitionTestScenario scenario, + Map> partitionMap, Map> resultsMap, + int staticPtn, int numMsgs) { + int numPtns = partitionMap.size(); + + if (scenario == PartitionTestScenario.NO_PARTITION_HEADERS && numMsgs % numPtns != 0) { + throw new IllegalArgumentException("This method is not designed to work with scenarios" + + " where there is expected to be a non-even distribution of messages"); + } + + for (int ptn = 0; ptn < numPtns; ptn++) { + List expectedResults = partitionMap.get(ptn); + List actualResults = resultsMap.get(ptn); + if (scenario == PartitionTestScenario.PARTITION_ID_HEADER_ONLY || + scenario == PartitionTestScenario.STATIC_HEADER_AND_PARTITION_ID) { + // In these two scenarios we're checking against partitionMap + Assert.assertEquals(expectedResults.size(), actualResults.size()); + //Go and check the message payload is what we wanted it to be + for (int idx = 0; idx < expectedResults.size(); idx++) { + Assert.assertArrayEquals(expectedResults.get(idx).getBody(), actualResults.get(idx)); + } + } else if (scenario == PartitionTestScenario.STATIC_HEADER_ONLY) { + // Check that if we are looking in the statically assigned partition + // all messages are in it, else all other partitions are zero + if (ptn == staticPtn) { + Assert.assertEquals(numMsgs, actualResults.size()); + } else { + Assert.assertEquals(0, actualResults.size()); + } + } else if (scenario == PartitionTestScenario.NO_PARTITION_HEADERS) { + // Checking for an even distribution + Assert.assertEquals(numMsgs / numPtns, actualResults.size()); + } + } + } + + /** + * This method is can be used to create a list of events for use in Kafka partition tests. + * Depending on the scenario it will deliberate generate an artificially skewed distribution + * of events-per-partition (populated into the passed partitionMap) and then ordered randomly + * into the resulting List of events. + * Four scenarios are catered for: + * - STATIC_HEADER_ONLY: All events are put into the partition specified by the staticPtn param + * - PARTITION_ID_HEADER_ONLY: Events are skewed into three partitions + * - STATIC_HEADER_AND_PARTITION_ID: Events are skewed into two partitions, with all others + * going into the partition specified by staticPtn + * - NO_PARTITION_HEADERS: No partition header is set and the partitionMap is not populated + * + * @param scenario The scenario being catered for. + * @param numMsgs The number of messages to generate + * @param partitionMap A map of Integer (partitionId) and List of Events - to be populated + * @param numPtns The number of partitions to be populated. + * @param staticPtn The static partition to be assigned to. + * @return + */ + public static List generateSkewedMessageList(PartitionTestScenario scenario, int numMsgs, + Map> partitionMap, int numPtns, int staticPtn) { + List msgs = new ArrayList(numMsgs); + + // Pre-conditions + if (numMsgs < 0) { + throw new IllegalArgumentException("Number of messages must be greater than zero"); + } + if (staticPtn >= numPtns) { + throw new IllegalArgumentException("The static partition must be less than the " + + "number of partitions"); + } + if (numPtns < 5) { + throw new IllegalArgumentException("This method is designed to work with at least 5 " + + "partitions"); + } + if (partitionMap.size() != numPtns) { + throw new IllegalArgumentException("partitionMap has not been correctly initialised"); + } + + for (int i = 0; i < numMsgs; i++) { + Map headers = new HashMap(); + Integer partition = null; + // Simple code to artificially create a skew. In this scenario, with 5 partitions + // and 50 messages we end up with a ratio of 0:0:27:13:10 however these will be + // interleaved + + if (scenario == PartitionTestScenario.NO_PARTITION_HEADERS) { + // Don't bother adding a header or computing a partition + } else if (scenario == PartitionTestScenario.STATIC_HEADER_ONLY) { + partition = staticPtn; + } else { + // We're going to artificially create a skew by putting every 5th event + // into partition 4, every 3rd event into partition 3 and everything else into + // partition 2 (unless a static partition is provided, in which case we'll + // put it into that partition instead, but without setting the header). + if (i % 5 == 0) { + partition = 4; + headers.put(PARTITION_HEADER, String.valueOf(partition)); + } else if (i % 3 == 0 ) { + partition = 3; + headers.put(PARTITION_HEADER, String.valueOf(partition)); + } else if (scenario == PartitionTestScenario.STATIC_HEADER_AND_PARTITION_ID) { + // In this case we're not going to set the header, but we are going + // to set partition which will then populate the partitionMap + partition = staticPtn; + } else if (scenario == PartitionTestScenario.PARTITION_ID_HEADER_ONLY) { + partition = 2; + headers.put(PARTITION_HEADER, String.valueOf(partition)); + } // Logically no other scenarios + } + + // Build the event + Event event = EventBuilder.withBody(String.valueOf(i).getBytes(), headers); + + if (scenario != PartitionTestScenario.NO_PARTITION_HEADERS) { + // Save into partitionMap + partitionMap.get(partition).add(event); + } + + // Add to ordered list + msgs.add(event); + + } + return msgs; + } + + /** + * Return a map containing one List of records per partition. + * This internally creates a Kafka Consumer using the provided consumer properties. + * + * @param numPtns + * @param consumerProperties + * @return A Map of Partitions(Integer) and the resulting List of messages (byte[]) retrieved + */ + public static Map> retrieveRecordsFromPartitions(String topic, int numPtns, + Properties consumerProperties) { + + Map> resultsMap = new HashMap>(); + for (int i = 0; i < numPtns; i++) { + List partitionResults = new ArrayList(); + resultsMap.put(i, partitionResults); + KafkaConsumer consumer = + new KafkaConsumer(consumerProperties); + + TopicPartition partition = new TopicPartition(topic, i); + + consumer.assign(Arrays.asList(partition)); + + ConsumerRecords records = consumer.poll(1000); + for (ConsumerRecord record : records) { + partitionResults.add(record.value()); + } + consumer.close(); + } + return resultsMap; + } + +} diff --git a/flume-shared/flume-shared-kafka-test/src/main/java/org/apache/flume/shared/kafka/test/PartitionOption.java b/flume-shared/flume-shared-kafka-test/src/main/java/org/apache/flume/shared/kafka/test/PartitionOption.java new file mode 100644 index 0000000000..97bfe33321 --- /dev/null +++ b/flume-shared/flume-shared-kafka-test/src/main/java/org/apache/flume/shared/kafka/test/PartitionOption.java @@ -0,0 +1,25 @@ +/** + Licensed to the Apache Software Foundation (ASF) under one or more + contributor license agreements. See the NOTICE file distributed with + this work for additional information regarding copyright ownership. + The ASF licenses this file to You under the Apache License, Version 2.0 + (the "License"); you may not use this file except in compliance with + the License. You may obtain a copy of the License at + + http://www.apache.org/licenses/LICENSE-2.0 + + Unless required by applicable law or agreed to in writing, software + distributed under the License is distributed on an "AS IS" BASIS, + WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + See the License for the specific language governing permissions and + limitations under the License. + limitations under the License. + */ + +package org.apache.flume.shared.kafka.test; + +public enum PartitionOption { + VALIDBUTOUTOFRANGE, + NOTSET, + NOTANUMBER +} diff --git a/flume-shared/flume-shared-kafka-test/src/main/java/org/apache/flume/shared/kafka/test/PartitionTestScenario.java b/flume-shared/flume-shared-kafka-test/src/main/java/org/apache/flume/shared/kafka/test/PartitionTestScenario.java new file mode 100644 index 0000000000..8cafd146cc --- /dev/null +++ b/flume-shared/flume-shared-kafka-test/src/main/java/org/apache/flume/shared/kafka/test/PartitionTestScenario.java @@ -0,0 +1,26 @@ +/** + Licensed to the Apache Software Foundation (ASF) under one or more + contributor license agreements. See the NOTICE file distributed with + this work for additional information regarding copyright ownership. + The ASF licenses this file to You under the Apache License, Version 2.0 + (the "License"); you may not use this file except in compliance with + the License. You may obtain a copy of the License at + + http://www.apache.org/licenses/LICENSE-2.0 + + Unless required by applicable law or agreed to in writing, software + distributed under the License is distributed on an "AS IS" BASIS, + WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + See the License for the specific language governing permissions and + limitations under the License. + limitations under the License. + */ + +package org.apache.flume.shared.kafka.test; + +public enum PartitionTestScenario { + STATIC_HEADER_ONLY, + PARTITION_ID_HEADER_ONLY, + STATIC_HEADER_AND_PARTITION_ID, + NO_PARTITION_HEADERS +} \ No newline at end of file diff --git a/flume-shared/pom.xml b/flume-shared/pom.xml new file mode 100644 index 0000000000..c6ec4fd7d8 --- /dev/null +++ b/flume-shared/pom.xml @@ -0,0 +1,61 @@ + + + + + 4.0.0 + + + flume-parent + org.apache.flume + 1.7.0-SNAPSHOT + + + org.apache.flume + flume-shared + Flume Shared Utils + pom + + + + + org.apache.rat + apache-rat-plugin + + + + + + flume-shared-kafka-test + + + + + + hadoop-1.0 + + + flume.hadoop.profile + 1 + + + + + + + diff --git a/pom.xml b/pom.xml index 2332a29ec6..ded7b7d396 100644 --- a/pom.xml +++ b/pom.xml @@ -75,6 +75,7 @@ limitations under the License. flume-ng-tests flume-tools flume-ng-auth + flume-shared @@ -825,7 +826,6 @@ limitations under the License. junit junit 4.10 - test @@ -1143,6 +1143,13 @@ limitations under the License. 1.7.0-SNAPSHOT + + org.apache.flume.flume-shared + flume-shared-kafka-test + 1.7.0-SNAPSHOT + test + + org.apache.flume.flume-ng-sinks flume-hdfs-sink From 2fd0d2572ceb2bc0138c880a3b763647349b64f4 Mon Sep 17 00:00:00 2001 From: Attila Simon Date: Mon, 10 Oct 2016 20:48:06 +0200 Subject: [PATCH 333/341] FLUME-2971. Add secure Kafka Sink/Source/Channel setup to the User Guide MIME-Version: 1.0 Content-Type: text/plain; charset=UTF-8 Content-Transfer-Encoding: 8bit The User Guide already has details about configuring Kafka channel to work with a kerberized Kafka cluster. This patch adds similar description for Kafka Sink and Kafka Source. Reviewers: Tristan Stevens, Mike Percy, Bessenyei Balázs Donát (Attila Simon via Bessenyei Balázs Donát) --- flume-ng-doc/sphinx/FlumeUserGuide.rst | 563 +++++++++++++++++++++---- 1 file changed, 471 insertions(+), 92 deletions(-) diff --git a/flume-ng-doc/sphinx/FlumeUserGuide.rst b/flume-ng-doc/sphinx/FlumeUserGuide.rst index 25db777c3d..71fd32e162 100644 --- a/flume-ng-doc/sphinx/FlumeUserGuide.rst +++ b/flume-ng-doc/sphinx/FlumeUserGuide.rst @@ -1240,44 +1240,45 @@ Kafka Source is an Apache Kafka consumer that reads messages from Kafka topics. If you have multiple Kafka sources running, you can configure them with the same Consumer Group so each will read a unique set of partitions for the topics. - - -=============================== =========== =================================================== -Property Name Default Description -=============================== =========== =================================================== -**channels** -- -**type** -- The component type name, needs to be ``org.apache.flume.source.kafka.KafkaSource`` -**kafka.bootstrap.servers** -- List of brokers in the Kafka cluster used by the source -kafka.consumer.group.id flume Unique identified of consumer group. Setting the same id in multiple sources or agents - indicates that they are part of the same consumer group -**kafka.topics** -- Comma-separated list of topics the kafka consumer will read messages from. -**kafka.topics.regex** -- Regex that defines set of topics the source is subscribed on. This property has higher priority - than ``kafka.topics`` and overrides ``kafka.topics`` if exists. -batchSize 1000 Maximum number of messages written to Channel in one batch -batchDurationMillis 1000 Maximum time (in ms) before a batch will be written to Channel - The batch will be written whenever the first of size and time will be reached. -backoffSleepIncrement 1000 Initial and incremental wait time that is triggered when a Kafka Topic appears to be empty. - Wait period will reduce aggressive pinging of an empty Kafka Topic. One second is ideal for - ingestion use cases but a lower value may be required for low latency operations with - interceptors. -maxBackoffSleep 5000 Maximum wait time that is triggered when a Kafka Topic appears to be empty. Five seconds is - ideal for ingestion use cases but a lower value may be required for low latency operations - with interceptors. -useFlumeEventFormat false By default events are taken as bytes from the Kafka topic directly into the event body. Set to - true to read events as the Flume Avro binary format. Used in conjunction with the same property - on the KafkaSink or with the parseAsFlumeEvent property on the Kafka Channel this will preserve - any Flume headers sent on the producing side. -migrateZookeeperOffsets true When no Kafka stored offset is found, look up the offsets in Zookeeper and commit them to Kafka. - This should be true to support seamless Kafka client migration from older versions of Flume. - Once migrated this can be set to false, though that should generally not be required. - If no Zookeeper offset is found, the Kafka configuration kafka.consumer.auto.offset.reset - defines how offsets are handled. -Other Kafka Consumer Properties -- These properties are used to configure the Kafka Consumer. Any producer property supported - by Kafka can be used. The only requirement is to prepend the property name with the prefix - ``kafka.consumer``. - For example: ``kafka.consumer.auto.offset.reset`` - Check `Kafka documentation `_ for details -=============================== =========== =================================================== +================================== =========== =================================================== +Property Name Default Description +================================== =========== =================================================== +**channels** -- +**type** -- The component type name, needs to be ``org.apache.flume.source.kafka.KafkaSource`` +**kafka.bootstrap.servers** -- List of brokers in the Kafka cluster used by the source +kafka.consumer.group.id flume Unique identified of consumer group. Setting the same id in multiple sources or agents + indicates that they are part of the same consumer group +**kafka.topics** -- Comma-separated list of topics the kafka consumer will read messages from. +**kafka.topics.regex** -- Regex that defines set of topics the source is subscribed on. This property has higher priority + than ``kafka.topics`` and overrides ``kafka.topics`` if exists. +batchSize 1000 Maximum number of messages written to Channel in one batch +batchDurationMillis 1000 Maximum time (in ms) before a batch will be written to Channel + The batch will be written whenever the first of size and time will be reached. +backoffSleepIncrement 1000 Initial and incremental wait time that is triggered when a Kafka Topic appears to be empty. + Wait period will reduce aggressive pinging of an empty Kafka Topic. One second is ideal for + ingestion use cases but a lower value may be required for low latency operations with + interceptors. +maxBackoffSleep 5000 Maximum wait time that is triggered when a Kafka Topic appears to be empty. Five seconds is + ideal for ingestion use cases but a lower value may be required for low latency operations + with interceptors. +useFlumeEventFormat false By default events are taken as bytes from the Kafka topic directly into the event body. Set to + true to read events as the Flume Avro binary format. Used in conjunction with the same property + on the KafkaSink or with the parseAsFlumeEvent property on the Kafka Channel this will preserve + any Flume headers sent on the producing side. +migrateZookeeperOffsets true When no Kafka stored offset is found, look up the offsets in Zookeeper and commit them to Kafka. + This should be true to support seamless Kafka client migration from older versions of Flume. + Once migrated this can be set to false, though that should generally not be required. + If no Zookeeper offset is found, the Kafka configuration kafka.consumer.auto.offset.reset + defines how offsets are handled. + Check `Kafka documentation `_ for details +kafka.consumer.security.protocol PLAINTEXT Set to SASL_PLAINTEXT, SASL_SSL or SSL if writing to Kafka using some level of security. See below for additional info on secure setup. +*more consumer security props* If using SASL_PLAINTEXT, SASL_SSL or SSL refer to `Kafka security `_ for additional + properties that need to be set on consumer. +Other Kafka Consumer Properties -- These properties are used to configure the Kafka Consumer. Any consumer property supported + by Kafka can be used. The only requirement is to prepend the property name with the prefix + ``kafka.consumer``. + For example: ``kafka.consumer.auto.offset.reset`` +================================== =========== =================================================== .. note:: The Kafka Source overrides two Kafka consumer parameters: auto.commit.enable is set to "false" by the source and every batch is committed. Kafka source guarantees at least once @@ -1319,6 +1320,142 @@ Example for topic subscription by regex # the default kafka.consumer.group.id=flume is used +**Security and Kafka Source:** + +Secure authentication as well as data encryption is supported on the communication channel between Flume and Kafka. +For secure authentication SASL/GSSAPI (Kerberos V5) or SSL (even though the parameter is named SSL, the actual protocol is a TLS implementation) can be used from Kafka version 0.9.0. + +As of now data encryption is solely provided by SSL/TLS. + +Setting ``kafka.consumer.security.protocol`` to any of the following value means: + +- **SASL_PLAINTEXT** - Kerberos or plaintext authentication with no data encryption +- **SASL_SSL** - Kerberos or plaintext authentication with data encryption +- **SSL** - TLS based encryption with optional authentication. + +.. warning:: + There is a performance degradation when SSL is enabled, + the magnitude of which depends on the CPU type and the JVM implementation. + Reference: `Kafka security overview `_ + and the jira for tracking this issue: + `KAFKA-2561 `_ + + +**TLS and Kafka Source:** + +Please read the steps described in `Configuring Kafka Clients SSL `_ +to learn about additional configuration settings for fine tuning for example any of the following: +security provider, cipher suites, enabled protocols, truststore or keystore types. + +Example configuration with server side authentication and data encryption. + +.. code-block:: properties + + a1.channels.channel1.type = org.apache.flume.channel.kafka.KafkaChannel + a1.channels.channel1.kafka.bootstrap.servers = kafka-1:9093,kafka-2:9093,kafka-3:9093 + a1.channels.channel1.kafka.topic = channel1 + a1.channels.channel1.kafka.consumer.group.id = flume-consumer + a1.channels.channel1.kafka.consumer.security.protocol = SSL + a1.channels.channel1.kafka.consumer.ssl.truststore.location=/path/to/truststore.jks + a1.channels.channel1.kafka.consumer.ssl.truststore.password= + + +Note: By default the property ``ssl.endpoint.identification.algorithm`` +is not defined, so hostname verification is not performed. +In order to enable hostname verification, set the following properties + +.. code-block:: properties + + a1.channels.channel1.kafka.consumer.ssl.endpoint.identification.algorithm=HTTPS + +Once enabled, clients will verify the server's fully qualified domain name (FQDN) +against one of the following two fields: + +#) Common Name (CN) https://tools.ietf.org/html/rfc6125#section-2.3 +#) Subject Alternative Name (SAN) https://tools.ietf.org/html/rfc5280#section-4.2.1.6 + +If client side authentication is also required then additionally the following should be added to Flume agent configuration. +Each Flume agent has to have its client certificate which has to be trusted by Kafka brokers either +individually or by their signature chain. Common example is to sign each client certificate by a single Root CA +which in turn is trusted by Kafka brokers. + +.. code-block:: properties + + a1.channels.channel1.kafka.consumer.ssl.keystore.location=/path/to/client.keystore.jks + a1.channels.channel1.kafka.consumer.ssl.keystore.password= + +If keystore and key use different password protection then ``ssl.key.password`` property will +provide the required additional secret for both consumer keystores: + +.. code-block:: properties + + a1.channels.channel1.kafka.consumer.ssl.key.password= + + +**Kerberos and Kafka Source:** + +To use Kafka source with a Kafka cluster secured with Kerberos, set the ``consumer.security.protocol`` properties noted above for consumer. +The Kerberos keytab and principal to be used with Kafka brokers is specified in a JAAS file's "KafkaClient" section. "Client" section describes the Zookeeper connection if needed. +See `Kafka doc `_ +for information on the JAAS file contents. The location of this JAAS file and optionally the system wide kerberos configuration can be specified via JAVA_OPTS in flume-env.sh: + +.. code-block:: properties + + JAVA_OPTS="$JAVA_OPTS -Djava.security.krb5.conf=/path/to/krb5.conf" + JAVA_OPTS="$JAVA_OPTS -Djava.security.auth.login.config=/path/to/flume_jaas.conf" + +Example secure configuration using SASL_PLAINTEXT: + +.. code-block:: properties + + a1.channels.channel1.type = org.apache.flume.channel.kafka.KafkaChannel + a1.channels.channel1.kafka.bootstrap.servers = kafka-1:9093,kafka-2:9093,kafka-3:9093 + a1.channels.channel1.kafka.topic = channel1 + a1.channels.channel1.kafka.consumer.group.id = flume-consumer + a1.channels.channel1.kafka.consumer.security.protocol = SASL_PLAINTEXT + a1.channels.channel1.kafka.consumer.sasl.mechanism = GSSAPI + a1.channels.channel1.kafka.consumer.sasl.kerberos.service.name = kafka + +Example secure configuration using SASL_SSL: + +.. code-block:: properties + + a1.channels.channel1.type = org.apache.flume.channel.kafka.KafkaChannel + a1.channels.channel1.kafka.bootstrap.servers = kafka-1:9093,kafka-2:9093,kafka-3:9093 + a1.channels.channel1.kafka.topic = channel1 + a1.channels.channel1.kafka.consumer.group.id = flume-consumer + a1.channels.channel1.kafka.consumer.security.protocol = SASL_SSL + a1.channels.channel1.kafka.consumer.sasl.mechanism = GSSAPI + a1.channels.channel1.kafka.consumer.sasl.kerberos.service.name = kafka + a1.channels.channel1.kafka.consumer.ssl.truststore.location=/path/to/truststore.jks + a1.channels.channel1.kafka.consumer.ssl.truststore.password= + + +Sample JAAS file. For reference of its content please see client config sections of the desired authentication mechanism (GSSAPI/PLAIN) +in Kafka documentation of `SASL configuration `_. +Since the Kafka Source may also connect to Zookeeper for offset migration, the "Client" section was also added to this example. +This won't be needed unless you require offset migration, or you require this section for other secure components. +Also please make sure that the operating system user of the Flume processes has read privileges on the jaas and keytab files. + +.. code-block:: javascript + + Client { + com.sun.security.auth.module.Krb5LoginModule required + useKeyTab=true + storeKey=true + keyTab="/path/to/keytabs/flume.keytab" + principal="flume/flumehost1.example.com@YOURKERBEROSREALM"; + }; + + KafkaClient { + com.sun.security.auth.module.Krb5LoginModule required + useKeyTab=true + storeKey=true + keyTab="/path/to/keytabs/flume.keytab" + principal="flume/flumehost1.example.com@YOURKERBEROSREALM"; + }; + + NetCat Source ~~~~~~~~~~~~~ @@ -2564,38 +2701,41 @@ This version of Flume no longer supports Older Versions (0.8.x) of Kafka. Required properties are marked in bold font. -=============================== =================== ============================================================================================= -Property Name Default Description -=============================== =================== ============================================================================================= -**type** -- Must be set to ``org.apache.flume.sink.kafka.KafkaSink`` -**kafka.bootstrap.servers** -- List of brokers Kafka-Sink will connect to, to get the list of topic partitions - This can be a partial list of brokers, but we recommend at least two for HA. - The format is comma separated list of hostname:port -kafka.topic default-flume-topic The topic in Kafka to which the messages will be published. If this parameter is configured, - messages will be published to this topic. - If the event header contains a "topic" field, the event will be published to that topic - overriding the topic configured here. -flumeBatchSize 100 How many messages to process in one batch. Larger batches improve throughput while adding latency. -kafka.producer.acks 1 How many replicas must acknowledge a message before its considered successfully written. - Accepted values are 0 (Never wait for acknowledgement), 1 (wait for leader only), -1 (wait for all replicas) - Set this to -1 to avoid data loss in some cases of leader failure. -useFlumeEventFormat false By default events are put as bytes onto the Kafka topic directly from the event body. Set to - true to store events as the Flume Avro binary format. Used in conjunction with the same property - on the KafkaSource or with the parseAsFlumeEvent property on the Kafka Channel this will preserve - any Flume headers for the producing side. -defaultPartitionId -- Specifies a Kafka partition ID (integer) for all events in this channel to be sent to, unless - overriden by ``partitionIdHeader``. By default, if this property is not set, events will be - distributed by the Kafka Producer's partitioner - including by ``key`` if specified (or by a - partitioner specified by ``kafka.partitioner.class``). -partitionIdHeader -- When set, the sink will take the value of the field named using the value of this property - from the event header and send the message to the specified partition of the topic. If the - value represents an invalid partition, an EventDeliveryException will be thrown. If the header value - is present then this setting overrides ``defaultPartitionId``. -Other Kafka Producer Properties -- These properties are used to configure the Kafka Producer. Any producer property supported - by Kafka can be used. The only requirement is to prepend the property name with the prefix - ``kafka.producer``. - For example: kafka.producer.linger.ms -=============================== =================== ============================================================================================= +================================== =================== ============================================================================================= +Property Name Default Description +================================== =================== ============================================================================================= +**type** -- Must be set to ``org.apache.flume.sink.kafka.KafkaSink`` +**kafka.bootstrap.servers** -- List of brokers Kafka-Sink will connect to, to get the list of topic partitions + This can be a partial list of brokers, but we recommend at least two for HA. + The format is comma separated list of hostname:port +kafka.topic default-flume-topic The topic in Kafka to which the messages will be published. If this parameter is configured, + messages will be published to this topic. + If the event header contains a "topic" field, the event will be published to that topic + overriding the topic configured here. +flumeBatchSize 100 How many messages to process in one batch. Larger batches improve throughput while adding latency. +kafka.producer.acks 1 How many replicas must acknowledge a message before its considered successfully written. + Accepted values are 0 (Never wait for acknowledgement), 1 (wait for leader only), -1 (wait for all replicas) + Set this to -1 to avoid data loss in some cases of leader failure. +useFlumeEventFormat false By default events are put as bytes onto the Kafka topic directly from the event body. Set to + true to store events as the Flume Avro binary format. Used in conjunction with the same property + on the KafkaSource or with the parseAsFlumeEvent property on the Kafka Channel this will preserve + any Flume headers for the producing side. +defaultPartitionId -- Specifies a Kafka partition ID (integer) for all events in this channel to be sent to, unless + overriden by ``partitionIdHeader``. By default, if this property is not set, events will be + distributed by the Kafka Producer's partitioner - including by ``key`` if specified (or by a + partitioner specified by ``kafka.partitioner.class``). +partitionIdHeader -- When set, the sink will take the value of the field named using the value of this property + from the event header and send the message to the specified partition of the topic. If the + value represents an invalid partition, an EventDeliveryException will be thrown. If the header value + is present then this setting overrides ``defaultPartitionId``. +kafka.producer.security.protocol PLAINTEXT Set to SASL_PLAINTEXT, SASL_SSL or SSL if writing to Kafka using some level of security. See below for additional info on secure setup. +*more producer security props* If using SASL_PLAINTEXT, SASL_SSL or SSL refer to `Kafka security `_ for additional + properties that need to be set on producer. +Other Kafka Producer Properties -- These properties are used to configure the Kafka Producer. Any producer property supported + by Kafka can be used. The only requirement is to prepend the property name with the prefix + ``kafka.producer``. + For example: kafka.producer.linger.ms +================================== =================== ============================================================================================= .. note:: Kafka Sink uses the ``topic`` and ``key`` properties from the FlumeEvent headers to send events to Kafka. If ``topic`` exists in the headers, the event will be sent to that specific topic, overriding the topic configured for the Sink. @@ -2635,6 +2775,132 @@ argument. a1.sinks.k1.kafka.producer.linger.ms = 1 a1.sinks.ki.kafka.producer.compression.type = snappy + +**Security and Kafka Sink:** + +Secure authentication as well as data encryption is supported on the communication channel between Flume and Kafka. +For secure authentication SASL/GSSAPI (Kerberos V5) or SSL (even though the parameter is named SSL, the actual protocol is a TLS implementation) can be used from Kafka version 0.9.0. + +As of now data encryption is solely provided by SSL/TLS. + +Setting ``kafka.producer.security.protocol`` to any of the following value means: + +- **SASL_PLAINTEXT** - Kerberos or plaintext authentication with no data encryption +- **SASL_SSL** - Kerberos or plaintext authentication with data encryption +- **SSL** - TLS based encryption with optional authentication. + +.. warning:: + There is a performance degradation when SSL is enabled, + the magnitude of which depends on the CPU type and the JVM implementation. + Reference: `Kafka security overview `_ + and the jira for tracking this issue: + `KAFKA-2561 `__ + + +**TLS and Kafka Sink:** + +Please read the steps described in `Configuring Kafka Clients SSL `_ +to learn about additional configuration settings for fine tuning for example any of the following: +security provider, cipher suites, enabled protocols, truststore or keystore types. + +Example configuration with server side authentication and data encryption. + +.. code-block:: properties + + a1.channels.channel1.type = org.apache.flume.channel.kafka.KafkaChannel + a1.channels.channel1.kafka.bootstrap.servers = kafka-1:9093,kafka-2:9093,kafka-3:9093 + a1.channels.channel1.kafka.topic = channel1 + a1.channels.channel1.kafka.producer.security.protocol = SSL + a1.channels.channel1.kafka.producer.ssl.truststore.location = /path/to/truststore.jks + a1.channels.channel1.kafka.producer.ssl.truststore.password = + + +Note: By default the property ``ssl.endpoint.identification.algorithm`` +is not defined, so hostname verification is not performed. +In order to enable hostname verification, set the following properties + +.. code-block:: properties + + a1.channels.channel1.kafka.producer.ssl.endpoint.identification.algorithm = HTTPS + +Once enabled, clients will verify the server's fully qualified domain name (FQDN) +against one of the following two fields: + +#) Common Name (CN) https://tools.ietf.org/html/rfc6125#section-2.3 +#) Subject Alternative Name (SAN) https://tools.ietf.org/html/rfc5280#section-4.2.1.6 + +If client side authentication is also required then additionally the following should be added to Flume agent configuration. +Each Flume agent has to have its client certificate which has to be trusted by Kafka brokers either +individually or by their signature chain. Common example is to sign each client certificate by a single Root CA +which in turn is trusted by Kafka brokers. + +.. code-block:: properties + + a1.channels.channel1.kafka.producer.ssl.keystore.location = /path/to/client.keystore.jks + a1.channels.channel1.kafka.producer.ssl.keystore.password = + +If keystore and key use different password protection then ``ssl.key.password`` property will +provide the required additional secret for producer keystore: + +.. code-block:: properties + + a1.channels.channel1.kafka.producer.ssl.key.password = + + +**Kerberos and Kafka Sink:** + +To use Kafka sink with a Kafka cluster secured with Kerberos, set the ``producer.security.protocol`` property noted above for producer. +The Kerberos keytab and principal to be used with Kafka brokers is specified in a JAAS file's "KafkaClient" section. "Client" section describes the Zookeeper connection if needed. +See `Kafka doc `_ +for information on the JAAS file contents. The location of this JAAS file and optionally the system wide kerberos configuration can be specified via JAVA_OPTS in flume-env.sh: + +.. code-block:: properties + + JAVA_OPTS="$JAVA_OPTS -Djava.security.krb5.conf=/path/to/krb5.conf" + JAVA_OPTS="$JAVA_OPTS -Djava.security.auth.login.config=/path/to/flume_jaas.conf" + +Example secure configuration using SASL_PLAINTEXT: + +.. code-block:: properties + + a1.channels.channel1.type = org.apache.flume.channel.kafka.KafkaChannel + a1.channels.channel1.kafka.bootstrap.servers = kafka-1:9093,kafka-2:9093,kafka-3:9093 + a1.channels.channel1.kafka.topic = channel1 + a1.channels.channel1.kafka.producer.security.protocol = SASL_PLAINTEXT + a1.channels.channel1.kafka.producer.sasl.mechanism = GSSAPI + a1.channels.channel1.kafka.producer.sasl.kerberos.service.name = kafka + + +Example secure configuration using SASL_SSL: + +.. code-block:: properties + + a1.channels.channel1.type = org.apache.flume.channel.kafka.KafkaChannel + a1.channels.channel1.kafka.bootstrap.servers = kafka-1:9093,kafka-2:9093,kafka-3:9093 + a1.channels.channel1.kafka.topic = channel1 + a1.channels.channel1.kafka.producer.security.protocol = SASL_SSL + a1.channels.channel1.kafka.producer.sasl.mechanism = GSSAPI + a1.channels.channel1.kafka.producer.sasl.kerberos.service.name = kafka + a1.channels.channel1.kafka.producer.ssl.truststore.location = /path/to/truststore.jks + a1.channels.channel1.kafka.producer.ssl.truststore.password = + + +Sample JAAS file. For reference of its content please see client config sections of the desired authentication mechanism (GSSAPI/PLAIN) +in Kafka documentation of `SASL configuration `_. +Unlike the Kafka Source or Kafka Channel a "Client" section is not required, unless it is needed by other connecting components. Also please make sure +that the operating system user of the Flume processes has read privileges on the jaas and keytab files. + +.. code-block:: javascript + + KafkaClient { + com.sun.security.auth.module.Krb5LoginModule required + useKeyTab=true + storeKey=true + keyTab="/path/to/keytabs/flume.keytab" + principal="flume/flumehost1.example.com@YOURKERBEROSREALM"; + }; + + Custom Sink ~~~~~~~~~~~ @@ -2792,7 +3058,7 @@ migrateZookeeperOffsets true When no Kaf This should be true to support seamless Kafka client migration from older versions of Flume. Once migrated this can be set to false, though that should generally not be required. If no Zookeeper offset is found the kafka.consumer.auto.offset.reset configuration defines how offsets are handled. -pollTimeout 500 The amount of time(in milliseconds) to wait in the "poll()" call of the conumer. +pollTimeout 500 The amount of time(in milliseconds) to wait in the "poll()" call of the consumer. https://kafka.apache.org/090/javadoc/org/apache/kafka/clients/consumer/KafkaConsumer.html#poll(long) defaultPartitionId -- Specifies a Kafka partition ID (integer) for all events in this channel to be sent to, unless overriden by ``partitionIdHeader``. By default, if this property is not set, events will be @@ -2808,9 +3074,9 @@ kafka.consumer.auto.offset.reset latest What to do latest: automatically reset the offset to the latest offset none: throw exception to the consumer if no previous offset is found for the consumer\'s group anything else: throw exception to the consumer. -kafka.producer.security.protocol PLAINTEXT Set to SASL_PLAINTEXT, SASL_SSL or SSL if writing to Kafka using Kerberos. See below for additional info on Kerberos setup. +kafka.producer.security.protocol PLAINTEXT Set to SASL_PLAINTEXT, SASL_SSL or SSL if writing to Kafka using some level of security. See below for additional info on secure setup. kafka.consumer.security.protocol PLAINTEXT Same as kafka.producer.security.protocol but for reading/consuming from Kafka. -*more producer/consumer security props* If using SASL_SSL or SSL, refer to `Kafka security `_ for additional +*more producer/consumer security props* If using SASL_PLAINTEXT, SASL_SSL or SSL refer to `Kafka security `_ for additional properties that need to be set on producer/consumer. ======================================= ========================== =============================================================================================================== @@ -2839,43 +3105,156 @@ Example for agent named a1: a1.channels.channel1.kafka.topic = channel1 a1.channels.channel1.kafka.consumer.group.id = flume-consumer + +**Security and Kafka Channel:** + +Secure authentication as well as data encryption is supported on the communication channel between Flume and Kafka. +For secure authentication SASL/GSSAPI (Kerberos V5) or SSL (even though the parameter is named SSL, the actual protocol is a TLS implementation) can be used from Kafka version 0.9.0. + +As of now data encryption is solely provided by SSL/TLS. + +Setting ``kafka.producer|consumer.security.protocol`` to any of the following value means: + +- **SASL_PLAINTEXT** - Kerberos or plaintext authentication with no data encryption +- **SASL_SSL** - Kerberos or plaintext authentication with data encryption +- **SSL** - TLS based encryption with optional authentication. + +.. warning:: + There is a performance degradation when SSL is enabled, + the magnitude of which depends on the CPU type and the JVM implementation. + Reference: `Kafka security overview `_ + and the jira for tracking this issue: + `KAFKA-2561 `_ + + +**TLS and Kafka Channel:** + +Please read the steps described in `Configuring Kafka Clients SSL `_ +to learn about additional configuration settings for fine tuning for example any of the following: +security provider, cipher suites, enabled protocols, truststore or keystore types. + +Example configuration with server side authentication and data encryption. + +.. code-block:: properties + + a1.channels.channel1.type = org.apache.flume.channel.kafka.KafkaChannel + a1.channels.channel1.kafka.bootstrap.servers = kafka-1:9093,kafka-2:9093,kafka-3:9093 + a1.channels.channel1.kafka.topic = channel1 + a1.channels.channel1.kafka.consumer.group.id = flume-consumer + a1.channels.channel1.kafka.producer.security.protocol = SSL + a1.channels.channel1.kafka.producer.ssl.truststore.location = /path/to/truststore.jks + a1.channels.channel1.kafka.producer.ssl.truststore.password = + a1.channels.channel1.kafka.consumer.security.protocol = SSL + a1.channels.channel1.kafka.consumer.ssl.truststore.location = /path/to/truststore.jks + a1.channels.channel1.kafka.consumer.ssl.truststore.password = + + +Note: By default the property ``ssl.endpoint.identification.algorithm`` +is not defined, so hostname verification is not performed. +In order to enable hostname verification, set the following properties + +.. code-block:: properties + + a1.channels.channel1.kafka.producer.ssl.endpoint.identification.algorithm = HTTPS + a1.channels.channel1.kafka.consumer.ssl.endpoint.identification.algorithm = HTTPS + +Once enabled, clients will verify the server's fully qualified domain name (FQDN) +against one of the following two fields: + +#) Common Name (CN) https://tools.ietf.org/html/rfc6125#section-2.3 +#) Subject Alternative Name (SAN) https://tools.ietf.org/html/rfc5280#section-4.2.1.6 + +If client side authentication is also required then additionally the following should be added to Flume agent configuration. +Each Flume agent has to have its client certificate which has to be trusted by Kafka brokers either +individually or by their signature chain. Common example is to sign each client certificate by a single Root CA +which in turn is trusted by Kafka brokers. + +.. code-block:: properties + + a1.channels.channel1.kafka.producer.ssl.keystore.location = /path/to/client.keystore.jks + a1.channels.channel1.kafka.producer.ssl.keystore.password = + a1.channels.channel1.kafka.consumer.ssl.keystore.location = /path/to/client.keystore.jks + a1.channels.channel1.kafka.consumer.ssl.keystore.password = + +If keystore and key use different password protection then ``ssl.key.password`` property will +provide the required additional secret for both consumer and producer keystores: + +.. code-block:: properties + + a1.channels.channel1.kafka.producer.ssl.key.password = + a1.channels.channel1.kafka.consumer.ssl.key.password = + + **Kerberos and Kafka Channel:** -To use Kafka channel with a Kafka cluster secured with Kerberos, set the producer/consumer.security.protocol properties noted above for producer and/or consumer. -The Kerberos keytab and principal to be used is specified in a JAAS file's "KafkaClient" section. See `Kafka doc `_ -for info on the JAAS file contents. The location of this JAAS file is specified via JAVA_OPTS using -Djava.security.auth.login.config=/path/to/kafka_jaas.conf (in flume-env.sh) +To use Kafka channel with a Kafka cluster secured with Kerberos, set the ``producer/consumer.security.protocol`` properties noted above for producer and/or consumer. +The Kerberos keytab and principal to be used with Kafka brokers is specified in a JAAS file's "KafkaClient" section. "Client" section describes the Zookeeper connection if needed. +See `Kafka doc `_ +for information on the JAAS file contents. The location of this JAAS file and optionally the system wide kerberos configuration can be specified via JAVA_OPTS in flume-env.sh: + +.. code-block:: properties + JAVA_OPTS="$JAVA_OPTS -Djava.security.krb5.conf=/path/to/krb5.conf" + JAVA_OPTS="$JAVA_OPTS -Djava.security.auth.login.config=/path/to/flume_jaas.conf" -Sample secure configuration using SASL_PLAINTEXT. +Example secure configuration using SASL_PLAINTEXT: .. code-block:: properties a1.channels.channel1.type = org.apache.flume.channel.kafka.KafkaChannel - a1.channels.channel1.kafka.bootstrap.servers = kafka-1:9092,kafka-2:9092,kafka-3:9092 + a1.channels.channel1.kafka.bootstrap.servers = kafka-1:9093,kafka-2:9093,kafka-3:9093 a1.channels.channel1.kafka.topic = channel1 a1.channels.channel1.kafka.consumer.group.id = flume-consumer a1.channels.channel1.kafka.producer.security.protocol = SASL_PLAINTEXT + a1.channels.channel1.kafka.producer.sasl.mechanism = GSSAPI + a1.channels.channel1.kafka.producer.sasl.kerberos.service.name = kafka a1.channels.channel1.kafka.consumer.security.protocol = SASL_PLAINTEXT + a1.channels.channel1.kafka.consumer.sasl.mechanism = GSSAPI + a1.channels.channel1.kafka.consumer.sasl.kerberos.service.name = kafka -Sample JAAS file +Example secure configuration using SASL_SSL: -.. code-block:: javascript +.. code-block:: properties - KafkaClient { - com.sun.security.auth.module.Krb5LoginModule required - useKeyTab=true - storeKey=true - serviceName="kafka" - keyTab="/path/to/keytabs/testuser1.keytab" - principal="testuser1/kafka1.example.com"; - }; + a1.channels.channel1.type = org.apache.flume.channel.kafka.KafkaChannel + a1.channels.channel1.kafka.bootstrap.servers = kafka-1:9093,kafka-2:9093,kafka-3:9093 + a1.channels.channel1.kafka.topic = channel1 + a1.channels.channel1.kafka.consumer.group.id = flume-consumer + a1.channels.channel1.kafka.producer.security.protocol = SASL_SSL + a1.channels.channel1.kafka.producer.sasl.mechanism = GSSAPI + a1.channels.channel1.kafka.producer.sasl.kerberos.service.name = kafka + a1.channels.channel1.kafka.producer.ssl.truststore.location = /path/to/truststore.jks + a1.channels.channel1.kafka.producer.ssl.truststore.password = + a1.channels.channel1.kafka.consumer.security.protocol = SASL_SSL + a1.channels.channel1.kafka.consumer.sasl.mechanism = GSSAPI + a1.channels.channel1.kafka.consumer.sasl.kerberos.service.name = kafka + a1.channels.channel1.kafka.consumer.ssl.truststore.location = /path/to/truststore.jks + a1.channels.channel1.kafka.consumer.ssl.truststore.password = + + +Sample JAAS file. For reference of its content please see client config sections of the desired authentication mechanism (GSSAPI/PLAIN) +in Kafka documentation of `SASL configuration `_. +Since the Kafka Source may also connect to Zookeeper for offset migration, the "Client" section was also added to this example. +This won't be needed unless you require offset migration, or you require this section for other secure components. +Also please make sure that the operating system user of the Flume processes has read privileges on the jaas and keytab files. -Sample flume-env.sh +.. code-block:: javascript -.. code-block:: properties + Client { + com.sun.security.auth.module.Krb5LoginModule required + useKeyTab=true + storeKey=true + keyTab="/path/to/keytabs/flume.keytab" + principal="flume/flumehost1.example.com@YOURKERBEROSREALM"; + }; - export JAVA_HOME=/path/java-home/ - export JAVA_OPTS="-Djava.security.auth.login.config=/path/to/kafka_jaas.conf" + KafkaClient { + com.sun.security.auth.module.Krb5LoginModule required + useKeyTab=true + storeKey=true + keyTab="/path/to/keytabs/flume.keytab" + principal="flume/flumehost1.example.com@YOURKERBEROSREALM"; + }; File Channel From d40abcbfbf31528dbbb1739118ddb1dc60bfbeeb Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?Bessenyei=20Bal=C3=A1zs=20Don=C3=A1t?= Date: Mon, 10 Oct 2016 21:17:53 +0200 Subject: [PATCH 334/341] Update RELEASE-NOTES for 1.7.0 --- RELEASE-NOTES | 6 +++--- 1 file changed, 3 insertions(+), 3 deletions(-) diff --git a/RELEASE-NOTES b/RELEASE-NOTES index 99e1328da9..7834d9d2fc 100644 --- a/RELEASE-NOTES +++ b/RELEASE-NOTES @@ -1,4 +1,4 @@ -Apache Flume 1.6.0 +Apache Flume 1.7.0 CONTENTS 1. What is Apache Flume @@ -17,8 +17,8 @@ simple, extensible data model that allows for online analytic application. 2. Status of this release -Apache Flume 1.6.0 is the ninth release of Flume as an Apache top-level project -(TLP). Apache Flume 1.6.0 is production-ready software. +Apache Flume 1.7.0 is the tenth release of Flume as an Apache top-level project +(TLP). Apache Flume 1.7.0 is production-ready software. 3. Major changes in this Release For a detailed list of changes, please see the CHANGELOG file included From 5e400ea8c471ef41c12dd8ad2079ac8a351ec509 Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?Bessenyei=20Bal=C3=A1zs=20Don=C3=A1t?= Date: Mon, 10 Oct 2016 21:41:19 +0200 Subject: [PATCH 335/341] Update CHANGELOG for 1.7 --- CHANGELOG | 151 +++++++++++++++++++++++++++++++++++++++++++++++------- 1 file changed, 133 insertions(+), 18 deletions(-) diff --git a/CHANGELOG b/CHANGELOG index 6e6a668151..3cc040d9de 100644 --- a/CHANGELOG +++ b/CHANGELOG @@ -1,3 +1,119 @@ +Release Notes - Flume - Version v1.7.0 + +** New Feature + * [FLUME-2498] - Implement Taildir Source + +** Improvement + * [FLUME-1899] - Make SpoolDir work with Sub-Directories + * [FLUME-2526] - Build flume by jdk 7 in default + * [FLUME-2628] - Add an optional parameter to specify the expected input text encoding for the netcat sourcef the netcat source + * [FLUME-2704] - Configurable poll delay for spooling directory source + * [FLUME-2718] - HTTP Source to support generic Stream Handler + * [FLUME-2729] - Allow pollableSource backoff times to be configurable + * [FLUME-2755] - Kafka Source reading multiple topics + * [FLUME-2781] - A Kafka Channel defined as parseAsFlumeEvent=false cannot be correctly used by a Flume source + * [FLUME-2799] - Kafka Source - Message Offset and Partition add to headers + * [FLUME-2801] - Performance improvement on TailDir source + * [FLUME-2810] - Add static Schema URL to AvroEventSerializer configuration + * [FLUME-2820] - Support New Kafka APIs + * [FLUME-2852] - Kafka Source/Sink should optionally read/write Flume records + * [FLUME-2868] - Kafka Channel partition topic by key + * [FLUME-2872] - Kafka Sink should be able to select which header as the key + * [FLUME-2875] - Allow RollingFileSink to specify a file prefix and a file extension. + * [FLUME-2909] - Bump Rat version + * [FLUME-2910] - AsyncHBaseSink - Failure callbacks should log the exception that caused them + * [FLUME-2911] - Add includePattern option in SpoolDirectorySource configuration + * [FLUME-2918] - TaildirSource is underperforming with huge parent directories + * [FLUME-2937] - Integrate checkstyle for non-test classes + * [FLUME-2941] - Integrate checkstyle for test classes + * [FLUME-2954] - make raw data appearing in log messages explicit + * [FLUME-2955] - Add file path to the header in TaildirSource + * [FLUME-2959] - Fix issues with flume-checkstyle module + * [FLUME-2982] - Add localhost escape sequence to HDFS sink + * [FLUME-2999] - Kafka channel and sink should enable statically assigned partition per event via header + * [FLUME-2821] - Flume-Kafka Source with new Consumer + * [FLUME-2822] - Flume-Kafka-Sink with new Producer + * [FLUME-2823] - Flume-Kafka-Channel with new APIs + +** Bug + * [FLUME-1668] - Hdfs Sink File Rollover + * [FLUME-2132] - Exception while syncing from Flume to HDFS + * [FLUME-2143] - Flume build occasionally fails with OutOfMemoryError on Windows. + * [FLUME-2215] - ResettableFileInputStream can't support ucs-4 character + * [FLUME-2318] - SpoolingDirectory is unable to handle empty files + * [FLUME-2448] - Building flume from trunk failing with dependency error + * [FLUME-2484] - NullPointerException in Kafka Sink test + * [FLUME-2485] - Thrift Source tests fail on Oracle JDK 8 + * [FLUME-2514] - Some TestFileChannelRestart tests are extremely slow + * [FLUME-2567] - Remove unneeded repository declarations in pom.xml + * [FLUME-2573] - flume-ng --conf parameter is not used when starting a flume agent + * [FLUME-2593] - ResettableFileInputStream returns negate values from read() method + * [FLUME-2619] - Spooldir source does not log channel exceptions + * [FLUME-2632] - High CPU on KafkaSink + * [FLUME-2652] - Documented transaction handling semantics incorrect + * [FLUME-2660] - Add documentation for EventValidator + * [FLUME-2672] - NPE in KafkaSourceCounter + * [FLUME-2712] - Optional channel errors slows down the Source to Main channel event rate + * [FLUME-2725] - HDFS Sink does not use configured timezone for rounding + * [FLUME-2732] - Make maximum tolerated failures before shutting down and recreating client in AsyncHbaseSink configurable + * [FLUME-2734] - Kafka Channel timeout property is overridden by default value + * [FLUME-2738] - Async HBase sink FD leak on client shutdown + * [FLUME-2746] - How to include this Flume Patch in Flume 1.5.2 ? + * [FLUME-2749] - Kerberos configuration error when using short names in multiple HDFS Sinks + * [FLUME-2751] - Upgrade Derby version to 10.11.1.1 + * [FLUME-2753] - Error when specifying empty replace string in Search and Replace Interceptor + * [FLUME-2754] - Hive Sink skipping first transaction in each Batch of Hive Transactions + * [FLUME-2761] - Move Hive sink out of preview mode + * [FLUME-2763] - flume_env script should handle jvm parameters like -javaagent -agentpath -agentlib + * [FLUME-2773] - TailDirSource throws FileNotFound Exception if ~/.flume directory is not created already + * [FLUME-2797] - SyslogTcpSource uses Deprecated Class + Deprecate SyslogTcpSource + * [FLUME-2798] - Malformed Syslog messages can lead to OutOfMemoryException + * [FLUME-2804] - Hive sink - abort remaining transactions on shutdown + * [FLUME-2806] - flume-ng.ps1 Error running script to start an agent on Windows + * [FLUME-2835] - Hive Sink tests need to create table with transactional property set + * [FLUME-2841] - Upgrade commons-collections to 3.2.2 + * [FLUME-2844] - ChannelCounter of SpillableMemoryChannel doesn't register actually. + * [FLUME-2881] - Windows Launch Script fails in plugins dir code + * [FLUME-2886] - Optional Channels can cause OOMs + * [FLUME-2889] - Fixes to DateTime computations + * [FLUME-2891] - Revert FLUME-2712 and FLUME-2886 + * [FLUME-2897] - AsyncHBase sink NPE when Channel.getTransaction() fails + * [FLUME-2901] - Document Kerberos setup for Kafka channel + * [FLUME-2908] - NetcatSource - SocketChannel not closed when session is broken + * [FLUME-2913] - Flume classpath too long + * [FLUME-2915] - The kafka channel using new APIs will be stuck when the sink is avro sink + * [FLUME-2920] - Kafka Channel Should Not Commit Offsets When Stopping + * [FLUME-2922] - HDFSSequenceFile Should Sync Writer + * [FLUME-2923] - Bump AsyncHBase version + * [FLUME-2936] - KafkaSource tests arbitrarily fail + * [FLUME-2939] - Upgrade recursive SpoolDir to use Java7 features + * [FLUME-2948] - Docs: Fixed parameters on Replicating Channel Selector documentation example + * [FLUME-2949] - Flume fails to build on Windows + * [FLUME-2950] - ReliableSpoolingFileEventReader.rollCurrentFile is broken + * [FLUME-2952] - SyslogAgent possible NPE on stop() + * [FLUME-2972] - Handle offset migration in the new Kafka Channel + * [FLUME-2974] - Some tests are broken in TestReliableSpoolingFileEventReader and TestSpoolingFileLineReader + * [FLUME-2983] - Handle offset migration in the new Kafka Source + +** Documentation + * [FLUME-2575] - FLUME-2548 brings SSLv2Hello back for Avro Sink, but UG says it is one of the protocols to exclude + * [FLUME-2713] - Document Fault Tolerant Config parameters in FlumeUserGuide + * [FLUME-2737] - Documentation for Pollable Source config parameters introduced in FLUME-2729 + * [FLUME-2783] - Update Website Team page with new Committer's + * [FLUME-2890] - Typo in Twitter source warning + * [FLUME-2934] - Document new cachePatternMatching option for TaildirSource + * [FLUME-2963] - FlumeUserGuide - error in Kafka Source properties table + * [FLUME-2971] - Document secure Kafka Sink/Source/Channel setup + * [FLUME-2975] - Minor mistake in NetCat Source example in documentation + * [FLUME-2998] - Add missing configuration parameter to SequenceSource docs + +** Task + * [FLUME-2935] - Bump java target version to 1.7 + +** Test + * [FLUME-3003] - testSourceCounter in TestSyslogUdpSource is flaky + + Release Notes - Flume - Version v1.6.0 ** Sub-task @@ -169,7 +285,7 @@ Release Notes - Flume - Version v1.5.0 * [FLUME-2139] - upgrade morphline library dependency to a 0.5.0 release * [FLUME-2154] - Reducing duplicate events caused by reset-connection-interval * [FLUME-2155] - Improve replay time - * [FLUME-2181] - Optionally disable File Channel fsyncs + * [FLUME-2181] - Optionally disable File Channel fsyncs * [FLUME-2202] - AsyncHBaseSink should coalesce increments to reduce RPC roundtrips * [FLUME-2206] - ElasticSearchSink ttl field modification to mimic Elasticsearch way of specifying TTL * [FLUME-2207] - HDFS file suffix style index suffix in ElasticSearchSink @@ -224,7 +340,7 @@ Release Notes - Flume - Version v1.5.0 * [FLUME-2199] - Flume builds with new version require mvn install before site can be generated * [FLUME-2200] - HTTP Source should be able to use "port" parameter if SSL is enabled * [FLUME-2208] - Jetty's default SocketSelector leaks File descriptors - * [FLUME-2209] - AsyncHBaseSink will never recover if the column family does not exists for the first start + * [FLUME-2209] - AsyncHBaseSink will never recover if the column family does not exists for the first start * [FLUME-2210] - UnresolvedAddressException when using multiple hostNames in Elasticsearch sink configuration * [FLUME-2220] - ElasticSearch sink - duplicate fields in indexed document * [FLUME-2229] - Backoff period gets reset too often in OrderSelector @@ -383,7 +499,7 @@ Release Notes - Flume - Version v1.4.0 * [FLUME-1788] - Flume Thrift source can fail intermittently because of a race condition in Thrift server implementation on some Linux systems * [FLUME-1789] - Unit tests TestJCEFileKeyProvider and TestFileChannelEncryption fail with IBM JDK and flume-1.3.0 * [FLUME-1795] - Flume thrift legacy source does not have proper logging configured - * [FLUME-1797] - TestFlumeConfiguration is in com.apache.flume.conf namespace. + * [FLUME-1797] - TestFlumeConfiguration is in com.apache.flume.conf namespace. * [FLUME-1799] - Generated source tarball is missing flume-ng-embedded-agent * [FLUME-1802] - Missing parameter --conf in example of the Flume User Guide * [FLUME-1803] - Generated dist tarball is missing flume-ng-embedded-agent @@ -441,7 +557,7 @@ Release Notes - Flume - Version v1.4.0 * [FLUME-1975] - Use TThreadedSelectServer in ThriftSource if it is available * [FLUME-1980] - Log4jAppender should optionally drop events if append fails * [FLUME-1981] - Rpc client expiration can be done in a more thread-safe way - * [FLUME-1986] - doTestInflightCorrupts should not commit transactions + * [FLUME-1986] - doTestInflightCorrupts should not commit transactions * [FLUME-1993] - On Windows, when using the spooling directory source, there is a file sharing violation when trying to delete tracker file * [FLUME-2002] - Flume RPC Client creates 2 threads per each log attempt if the remote flume agent goes down * [FLUME-2011] - "mvn test" fails @@ -457,7 +573,7 @@ Release Notes - Flume - Version v1.4.0 * [FLUME-2043] - JMS Source removed on failure to create configuration * [FLUME-2044] - HDFS Sink impersonation fails after the first file * [FLUME-2051] - Surefire 2.12 cannot run a single test on Windows. Upgrade to 2.12.3 - * [FLUME-2054] - Support Version Info on Windows and fix failure of TestVersionInfo + * [FLUME-2054] - Support Version Info on Windows and fix failure of TestVersionInfo * [FLUME-2057] - Failures in FileChannel's TestEventQueueBackingStoreFactory on Windows * [FLUME-2060] - Failure in TestLog.testReplaySucceedsWithUnusedEmptyLogMetaDataFastReplay test on Windows * [FLUME-2072] - JMX metrics support for HBase Sink @@ -510,7 +626,7 @@ Release Notes - Flume - Version v1.3.0 * [FLUME-1408] - ScheduledExecutorService does not log uncaught Throwables, we should log them * [FLUME-1418] - Improvement for Log4j configuration * [FLUME-1419] - Using system time if 'timestamp' property is absent in event header - * [FLUME-1434] - Distinguish background worker with channel name + * [FLUME-1434] - Distinguish background worker with channel name * [FLUME-1480] - Replace object descriptor with detailed component type plus name * [FLUME-1487] - FileChannel format needs to be extensible * [FLUME-1490] - Option to limit number of events sent in Stress source @@ -618,7 +734,7 @@ Release Notes - Flume - Version v1.3.0 * [FLUME-1576] - CHECKPOINT_INCOMPLETE should be synced to disk before starting the checkpoint. * [FLUME-1577] - CHECKPOINT_INCOMPLETE should be synced to disk before starting the checkpoint. * [FLUME-1578] - Proposal to modify file channel encryption config - * [FLUME-1582] - flume-ng script should set LD_LIBRARY_PATH + * [FLUME-1582] - flume-ng script should set LD_LIBRARY_PATH * [FLUME-1583] - FileChannel fast full replay will always be used if enabled * [FLUME-1595] - HDFS SequenceFile implementation is not durable due to not using syncFs() * [FLUME-1606] - Rollbacks of Put transactions does not clear the transaction from inflight puts. @@ -631,8 +747,8 @@ Release Notes - Flume - Version v1.3.0 * [FLUME-1638] - LoadBalancingRpcClient depends on com.google.common.collect.Maps * [FLUME-1639] - Client SDK should not have dependency on Guava * [FLUME-1650] - Fix flume-ng-hbase-sink compilation against Hadoop 2.X - * [FLUME-1651] - in the hadoop-0.23 profile HBase version needs to be at least 0.94.2 - * [FLUME-1653] - Update Hadoop-23 profile to point to hadoop-2 alpha artifacts + * [FLUME-1651] - in the hadoop-0.23 profile HBase version needs to be at least 0.94.2 + * [FLUME-1653] - Update Hadoop-23 profile to point to hadoop-2 alpha artifacts * [FLUME-1655] - Doc update needed for Regex Filtering Interceptor * [FLUME-1656] - flume-ng script disregards stderr from hadoop command when finding hadoop jars * [FLUME-1659] - JSON Handler should return simple events, not JSONEvents. @@ -728,11 +844,11 @@ Release Notes - Flume - Version v1.2.0 * [FLUME-1041] - maven warns of duplicate dependencies * [FLUME-1046] - invoking flume-ng script from bin directory fails * [FLUME-1047] - Client SDK has dependency on apache commons - * [FLUME-1070] - Fix javadoc for configuring hdfsSink + * [FLUME-1070] - Fix javadoc for configuring hdfsSink * [FLUME-1074] - AvroSink if any non-caught exception is thrown, an exception is thrown in finally clause * [FLUME-1075] - HDFSEventSink begin is called when transaction opened due to other error * [FLUME-1079] - Flume agent reconfiguration enters permanent bad state - * [FLUME-1080] - Issue with HDFSEventSink for append support + * [FLUME-1080] - Issue with HDFSEventSink for append support * [FLUME-1083] - Why does flume binary archive produces the following empty directories: bin/{ia64,amd64} ? * [FLUME-1087] - Restore Client API compat with v1.1.0 * [FLUME-1088] - TestWAL.testThreadedAppend fails on jenkins build server @@ -745,13 +861,13 @@ Release Notes - Flume - Version v1.2.0 * [FLUME-1108] - FILE_ROLL sink doesn't accept value 0 for unlimited wait time before rolling file * [FLUME-1109] - Syslog sources need to be refactored * [FLUME-1112] - HDFSCompressedDataStream append does not work - * [FLUME-1114] - Syslog Sources does not implement maxsize + * [FLUME-1114] - Syslog Sources does not implement maxsize * [FLUME-1116] - Extra event created for max payload size of 2500 bytes in Flume syslogtcp source * [FLUME-1119] - Remove default ports for syslog sources - * [FLUME-1121] - Recoverable Memory Channel cannot recover data + * [FLUME-1121] - Recoverable Memory Channel cannot recover data * [FLUME-1124] - Lifecycle supervisor can cause thread contention, sometimes causing components to not startup. * [FLUME-1125] - flume-ng script allows flume-env.sh to clobber some command-line arguments - * [FLUME-1128] - Conf poller should use schedule with fixed delay + * [FLUME-1128] - Conf poller should use schedule with fixed delay * [FLUME-1129] - change foo to agent in sample config * [FLUME-1130] - flume-ng script bad ordering on FLUME_HOME var * [FLUME-1135] - flume-docs exclude is not sufficient for rat @@ -779,7 +895,7 @@ Release Notes - Flume - Version v1.2.0 * [FLUME-1216] - Need useful error message when keytab does not exist * [FLUME-1217] - HDFS Event Sink generates warnings due to recent change * [FLUME-1219] - Race conditions in BucketWriter / HDFSEventSink - * [FLUME-1220] - Load balancing channel selector needs to be in the configuration type + * [FLUME-1220] - Load balancing channel selector needs to be in the configuration type * [FLUME-1221] - ThriftLegacySource doesn't handle fields -> headers conversions for bytebuffers * [FLUME-1226] - FailoverRpcClient should check for NULL batch-size property * [FLUME-1229] - System.nanoTime incorrectly used in filename for HDFS file rolling @@ -890,7 +1006,7 @@ Release Notes - Flume - Version v1.1.0 * [FLUME-931] - Flume NG TailSource * [FLUME-932] - Making flume-ng components pluggage and name aware * [FLUME-935] - Create abstract implementations of basic channel/transaction semantics - * [FLUME-939] - Load flume-env.sh from flume_conf_dir environment variable / system property as opposed to bin directory + * [FLUME-939] - Load flume-env.sh from flume_conf_dir environment variable / system property as opposed to bin directory * [FLUME-945] - Add the ability to specify a default channel for multiplexing channel selector. * [FLUME-956] - Configuration File Template * [FLUME-958] - Miscellaneous build improvements @@ -902,7 +1018,7 @@ Release Notes - Flume - Version v1.1.0 * [FLUME-1021] - Document API contracts and expected behavior in additional interfaces, including Source ** New Feature - * [FLUME-865] - Implement failover sink + * [FLUME-865] - Implement failover sink * [FLUME-892] - Support for SysLog as source * [FLUME-914] - Port the IRC sink to flume ng * [FLUME-930] - Support for multiplexing into different channels from single source. @@ -1042,4 +1158,3 @@ Release Notes - Flume - Version NG alpha 1 - 20111021 * [FLUME-787] - Write javadoc for builtin sinks * [FLUME-801] - Write NG getting started guide * [FLUME-802] - Complete PropertyFileConfigurationProvider implementation - From 2526ea06c27678a3f67a2bcf72ab4b89a08c7121 Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?Bessenyei=20Bal=C3=A1zs=20Don=C3=A1t?= Date: Mon, 10 Oct 2016 21:34:21 +0200 Subject: [PATCH 336/341] Update POM versions to 1.7.0 for Flume 1.7.0 release --- flume-checkstyle/pom.xml | 2 +- flume-ng-auth/pom.xml | 2 +- flume-ng-channels/flume-file-channel/pom.xml | 2 +- flume-ng-channels/flume-jdbc-channel/pom.xml | 2 +- flume-ng-channels/flume-kafka-channel/pom.xml | 2 +- .../flume-spillable-memory-channel/pom.xml | 2 +- flume-ng-channels/pom.xml | 2 +- .../flume-ng-log4jappender/pom.xml | 2 +- flume-ng-clients/pom.xml | 2 +- flume-ng-configuration/pom.xml | 2 +- flume-ng-core/pom.xml | 2 +- flume-ng-dist/pom.xml | 2 +- flume-ng-doc/sphinx/FlumeDeveloperGuide.rst | 2 +- flume-ng-doc/sphinx/FlumeUserGuide.rst | 6 +- flume-ng-doc/sphinx/index.rst | 2 +- flume-ng-embedded-agent/pom.xml | 2 +- .../flume-avro-source/pom.xml | 2 +- .../flume-thrift-source/pom.xml | 2 +- flume-ng-legacy-sources/pom.xml | 2 +- flume-ng-node/pom.xml | 2 +- flume-ng-sdk/pom.xml | 2 +- flume-ng-sinks/flume-dataset-sink/pom.xml | 2 +- flume-ng-sinks/flume-hdfs-sink/pom.xml | 2 +- flume-ng-sinks/flume-hive-sink/pom.xml | 2 +- flume-ng-sinks/flume-irc-sink/pom.xml | 2 +- .../flume-ng-elasticsearch-sink/pom.xml | 2 +- flume-ng-sinks/flume-ng-hbase-sink/pom.xml | 2 +- flume-ng-sinks/flume-ng-kafka-sink/pom.xml | 2 +- .../flume-ng-morphline-solr-sink/pom.xml | 4 +- flume-ng-sinks/pom.xml | 2 +- flume-ng-sources/flume-jms-source/pom.xml | 2 +- flume-ng-sources/flume-kafka-source/pom.xml | 2 +- flume-ng-sources/flume-scribe-source/pom.xml | 2 +- flume-ng-sources/flume-taildir-source/pom.xml | 2 +- flume-ng-sources/flume-twitter-source/pom.xml | 2 +- flume-ng-sources/pom.xml | 2 +- flume-ng-tests/pom.xml | 2 +- flume-shared/flume-shared-kafka-test/pom.xml | 2 +- flume-shared/pom.xml | 2 +- flume-tools/pom.xml | 2 +- pom.xml | 66 +++++++++---------- 41 files changed, 76 insertions(+), 76 deletions(-) diff --git a/flume-checkstyle/pom.xml b/flume-checkstyle/pom.xml index 10d74f3b9b..a5e3b37b9e 100644 --- a/flume-checkstyle/pom.xml +++ b/flume-checkstyle/pom.xml @@ -24,7 +24,7 @@ org.apache.flume flume-checkstyle Flume checkstyle project - 1.7.0-SNAPSHOT + 1.7.0 diff --git a/flume-ng-auth/pom.xml b/flume-ng-auth/pom.xml index fff392c167..5ba9b76d56 100644 --- a/flume-ng-auth/pom.xml +++ b/flume-ng-auth/pom.xml @@ -21,7 +21,7 @@ limitations under the License. flume-parent org.apache.flume - 1.7.0-SNAPSHOT + 1.7.0 flume-ng-auth diff --git a/flume-ng-channels/flume-file-channel/pom.xml b/flume-ng-channels/flume-file-channel/pom.xml index 5b11a3ef85..11ee415de5 100644 --- a/flume-ng-channels/flume-file-channel/pom.xml +++ b/flume-ng-channels/flume-file-channel/pom.xml @@ -24,7 +24,7 @@ flume-ng-channels org.apache.flume - 1.7.0-SNAPSHOT + 1.7.0 org.apache.flume.flume-ng-channels diff --git a/flume-ng-channels/flume-jdbc-channel/pom.xml b/flume-ng-channels/flume-jdbc-channel/pom.xml index 9e1b625257..201fc6137c 100644 --- a/flume-ng-channels/flume-jdbc-channel/pom.xml +++ b/flume-ng-channels/flume-jdbc-channel/pom.xml @@ -23,7 +23,7 @@ limitations under the License. flume-ng-channels org.apache.flume - 1.7.0-SNAPSHOT + 1.7.0 org.apache.flume.flume-ng-channels diff --git a/flume-ng-channels/flume-kafka-channel/pom.xml b/flume-ng-channels/flume-kafka-channel/pom.xml index a3c6de24f0..47ba4bfdfc 100644 --- a/flume-ng-channels/flume-kafka-channel/pom.xml +++ b/flume-ng-channels/flume-kafka-channel/pom.xml @@ -21,7 +21,7 @@ limitations under the License. flume-ng-channels org.apache.flume - 1.7.0-SNAPSHOT + 1.7.0 4.0.0 diff --git a/flume-ng-channels/flume-spillable-memory-channel/pom.xml b/flume-ng-channels/flume-spillable-memory-channel/pom.xml index 60e49d2d0a..edc481176f 100644 --- a/flume-ng-channels/flume-spillable-memory-channel/pom.xml +++ b/flume-ng-channels/flume-spillable-memory-channel/pom.xml @@ -24,7 +24,7 @@ flume-ng-channels org.apache.flume - 1.7.0-SNAPSHOT + 1.7.0 org.apache.flume.flume-ng-channels diff --git a/flume-ng-channels/pom.xml b/flume-ng-channels/pom.xml index 17a7bf3fcb..f93e6aeb04 100644 --- a/flume-ng-channels/pom.xml +++ b/flume-ng-channels/pom.xml @@ -23,7 +23,7 @@ limitations under the License. flume-parent org.apache.flume - 1.7.0-SNAPSHOT + 1.7.0 org.apache.flume diff --git a/flume-ng-clients/flume-ng-log4jappender/pom.xml b/flume-ng-clients/flume-ng-log4jappender/pom.xml index 7f40fc45bd..929086ec74 100644 --- a/flume-ng-clients/flume-ng-log4jappender/pom.xml +++ b/flume-ng-clients/flume-ng-log4jappender/pom.xml @@ -21,7 +21,7 @@ limitations under the License. flume-ng-clients org.apache.flume - 1.7.0-SNAPSHOT + 1.7.0 .. diff --git a/flume-ng-clients/pom.xml b/flume-ng-clients/pom.xml index fbed914bc9..5f946a5481 100644 --- a/flume-ng-clients/pom.xml +++ b/flume-ng-clients/pom.xml @@ -20,7 +20,7 @@ limitations under the License. flume-parent org.apache.flume - 1.7.0-SNAPSHOT + 1.7.0 .. flume-ng-clients diff --git a/flume-ng-configuration/pom.xml b/flume-ng-configuration/pom.xml index 529e5cf6a1..5a11abdbd7 100644 --- a/flume-ng-configuration/pom.xml +++ b/flume-ng-configuration/pom.xml @@ -20,7 +20,7 @@ limitations under the License. flume-parent org.apache.flume - 1.7.0-SNAPSHOT + 1.7.0 .. flume-ng-configuration diff --git a/flume-ng-core/pom.xml b/flume-ng-core/pom.xml index 41586ddc89..bb5697e53e 100644 --- a/flume-ng-core/pom.xml +++ b/flume-ng-core/pom.xml @@ -22,7 +22,7 @@ limitations under the License. flume-parent org.apache.flume - 1.7.0-SNAPSHOT + 1.7.0 org.apache.flume diff --git a/flume-ng-dist/pom.xml b/flume-ng-dist/pom.xml index 7fdf36afcf..a04e07e7d2 100644 --- a/flume-ng-dist/pom.xml +++ b/flume-ng-dist/pom.xml @@ -24,7 +24,7 @@ flume-parent org.apache.flume - 1.7.0-SNAPSHOT + 1.7.0 flume-ng-dist diff --git a/flume-ng-doc/sphinx/FlumeDeveloperGuide.rst b/flume-ng-doc/sphinx/FlumeDeveloperGuide.rst index b848c0d13c..70e0c33368 100644 --- a/flume-ng-doc/sphinx/FlumeDeveloperGuide.rst +++ b/flume-ng-doc/sphinx/FlumeDeveloperGuide.rst @@ -15,7 +15,7 @@ ====================================== -Flume 1.7.0-SNAPSHOT Developer Guide +Flume 1.7.0 Developer Guide ====================================== Introduction diff --git a/flume-ng-doc/sphinx/FlumeUserGuide.rst b/flume-ng-doc/sphinx/FlumeUserGuide.rst index 71fd32e162..bc65a49321 100644 --- a/flume-ng-doc/sphinx/FlumeUserGuide.rst +++ b/flume-ng-doc/sphinx/FlumeUserGuide.rst @@ -15,7 +15,7 @@ ====================================== -Flume 1.7.0-SNAPSHOT User Guide +Flume 1.7.0 User Guide ====================================== Introduction @@ -4101,7 +4101,7 @@ Log4J Appender Appends Log4j events to a flume agent's avro source. A client using this appender must have the flume-ng-sdk in the classpath (eg, -flume-ng-sdk-1.7.0-SNAPSHOT.jar). +flume-ng-sdk-1.7.0.jar). Required properties are in **bold**. ===================== ======= ================================================================================== @@ -4165,7 +4165,7 @@ Load Balancing Log4J Appender Appends Log4j events to a list of flume agent's avro source. A client using this appender must have the flume-ng-sdk in the classpath (eg, -flume-ng-sdk-1.7.0-SNAPSHOT.jar). This appender supports a round-robin and random +flume-ng-sdk-1.7.0.jar). This appender supports a round-robin and random scheme for performing the load balancing. It also supports a configurable backoff timeout so that down agents are removed temporarily from the set of hosts Required properties are in **bold**. diff --git a/flume-ng-doc/sphinx/index.rst b/flume-ng-doc/sphinx/index.rst index e3fd1b702b..10d8199d9f 100644 --- a/flume-ng-doc/sphinx/index.rst +++ b/flume-ng-doc/sphinx/index.rst @@ -15,7 +15,7 @@ ================================================ -Apache Flume 1.7.0-SNAPSHOT documentation +Apache Flume 1.7.0 documentation ================================================ Flume is a distributed, reliable, and available system for efficiently diff --git a/flume-ng-embedded-agent/pom.xml b/flume-ng-embedded-agent/pom.xml index 5bfb6e872e..12741f22a9 100644 --- a/flume-ng-embedded-agent/pom.xml +++ b/flume-ng-embedded-agent/pom.xml @@ -21,7 +21,7 @@ limitations under the License. flume-parent org.apache.flume - 1.7.0-SNAPSHOT + 1.7.0 flume-ng-embedded-agent diff --git a/flume-ng-legacy-sources/flume-avro-source/pom.xml b/flume-ng-legacy-sources/flume-avro-source/pom.xml index 3ba53bdc22..f8e717441a 100644 --- a/flume-ng-legacy-sources/flume-avro-source/pom.xml +++ b/flume-ng-legacy-sources/flume-avro-source/pom.xml @@ -23,7 +23,7 @@ limitations under the License. flume-ng-legacy-sources org.apache.flume - 1.7.0-SNAPSHOT + 1.7.0 org.apache.flume.flume-ng-legacy-sources diff --git a/flume-ng-legacy-sources/flume-thrift-source/pom.xml b/flume-ng-legacy-sources/flume-thrift-source/pom.xml index c0b37f35b8..31a6c368ec 100644 --- a/flume-ng-legacy-sources/flume-thrift-source/pom.xml +++ b/flume-ng-legacy-sources/flume-thrift-source/pom.xml @@ -23,7 +23,7 @@ limitations under the License. flume-ng-legacy-sources org.apache.flume - 1.7.0-SNAPSHOT + 1.7.0 org.apache.flume.flume-ng-legacy-sources diff --git a/flume-ng-legacy-sources/pom.xml b/flume-ng-legacy-sources/pom.xml index e83be9c8a4..02d1bd64b8 100644 --- a/flume-ng-legacy-sources/pom.xml +++ b/flume-ng-legacy-sources/pom.xml @@ -23,7 +23,7 @@ limitations under the License. flume-parent org.apache.flume - 1.7.0-SNAPSHOT + 1.7.0 org.apache.flume diff --git a/flume-ng-node/pom.xml b/flume-ng-node/pom.xml index e33b566c94..ab659073ff 100644 --- a/flume-ng-node/pom.xml +++ b/flume-ng-node/pom.xml @@ -25,7 +25,7 @@ flume-parent org.apache.flume - 1.7.0-SNAPSHOT + 1.7.0 flume-ng-node diff --git a/flume-ng-sdk/pom.xml b/flume-ng-sdk/pom.xml index c3b3dd7ba4..1d3c936967 100644 --- a/flume-ng-sdk/pom.xml +++ b/flume-ng-sdk/pom.xml @@ -21,7 +21,7 @@ limitations under the License. flume-parent org.apache.flume - 1.7.0-SNAPSHOT + 1.7.0 flume-ng-sdk diff --git a/flume-ng-sinks/flume-dataset-sink/pom.xml b/flume-ng-sinks/flume-dataset-sink/pom.xml index 186fdf8434..1e8a07b123 100644 --- a/flume-ng-sinks/flume-dataset-sink/pom.xml +++ b/flume-ng-sinks/flume-dataset-sink/pom.xml @@ -23,7 +23,7 @@ limitations under the License. flume-ng-sinks org.apache.flume - 1.7.0-SNAPSHOT + 1.7.0 org.apache.flume.flume-ng-sinks diff --git a/flume-ng-sinks/flume-hdfs-sink/pom.xml b/flume-ng-sinks/flume-hdfs-sink/pom.xml index 1aa281f051..bcf655652e 100644 --- a/flume-ng-sinks/flume-hdfs-sink/pom.xml +++ b/flume-ng-sinks/flume-hdfs-sink/pom.xml @@ -23,7 +23,7 @@ limitations under the License. flume-ng-sinks org.apache.flume - 1.7.0-SNAPSHOT + 1.7.0 org.apache.flume.flume-ng-sinks diff --git a/flume-ng-sinks/flume-hive-sink/pom.xml b/flume-ng-sinks/flume-hive-sink/pom.xml index e273f1ff4c..6d9ee4755f 100644 --- a/flume-ng-sinks/flume-hive-sink/pom.xml +++ b/flume-ng-sinks/flume-hive-sink/pom.xml @@ -23,7 +23,7 @@ limitations under the License. org.apache.flume flume-ng-sinks - 1.7.0-SNAPSHOT + 1.7.0 org.apache.flume.flume-ng-sinks diff --git a/flume-ng-sinks/flume-irc-sink/pom.xml b/flume-ng-sinks/flume-irc-sink/pom.xml index b574cd2693..6345f59c40 100644 --- a/flume-ng-sinks/flume-irc-sink/pom.xml +++ b/flume-ng-sinks/flume-irc-sink/pom.xml @@ -23,7 +23,7 @@ limitations under the License. flume-ng-sinks org.apache.flume - 1.7.0-SNAPSHOT + 1.7.0 org.apache.flume.flume-ng-sinks diff --git a/flume-ng-sinks/flume-ng-elasticsearch-sink/pom.xml b/flume-ng-sinks/flume-ng-elasticsearch-sink/pom.xml index c372c0b734..527bcca83c 100644 --- a/flume-ng-sinks/flume-ng-elasticsearch-sink/pom.xml +++ b/flume-ng-sinks/flume-ng-elasticsearch-sink/pom.xml @@ -17,7 +17,7 @@ flume-ng-sinks org.apache.flume - 1.7.0-SNAPSHOT + 1.7.0 org.apache.flume.flume-ng-sinks diff --git a/flume-ng-sinks/flume-ng-hbase-sink/pom.xml b/flume-ng-sinks/flume-ng-hbase-sink/pom.xml index 575a61dc34..66ffa4d64c 100644 --- a/flume-ng-sinks/flume-ng-hbase-sink/pom.xml +++ b/flume-ng-sinks/flume-ng-hbase-sink/pom.xml @@ -15,7 +15,7 @@ flume-ng-sinks org.apache.flume - 1.7.0-SNAPSHOT + 1.7.0 org.apache.flume.flume-ng-sinks flume-ng-hbase-sink diff --git a/flume-ng-sinks/flume-ng-kafka-sink/pom.xml b/flume-ng-sinks/flume-ng-kafka-sink/pom.xml index 2ab4b4addf..8ad229e156 100644 --- a/flume-ng-sinks/flume-ng-kafka-sink/pom.xml +++ b/flume-ng-sinks/flume-ng-kafka-sink/pom.xml @@ -16,7 +16,7 @@ flume-ng-sinks org.apache.flume - 1.7.0-SNAPSHOT + 1.7.0 org.apache.flume.flume-ng-sinks flume-ng-kafka-sink diff --git a/flume-ng-sinks/flume-ng-morphline-solr-sink/pom.xml b/flume-ng-sinks/flume-ng-morphline-solr-sink/pom.xml index 8e0bc5ec04..055c2c2b4a 100644 --- a/flume-ng-sinks/flume-ng-morphline-solr-sink/pom.xml +++ b/flume-ng-sinks/flume-ng-morphline-solr-sink/pom.xml @@ -21,12 +21,12 @@ limitations under the License. flume-ng-sinks org.apache.flume - 1.7.0-SNAPSHOT + 1.7.0 org.apache.flume.flume-ng-sinks flume-ng-morphline-solr-sink - 1.7.0-SNAPSHOT + 1.7.0 Flume NG Morphline Solr Sink diff --git a/flume-ng-sinks/pom.xml b/flume-ng-sinks/pom.xml index 00791e41ad..2b7bec58a6 100644 --- a/flume-ng-sinks/pom.xml +++ b/flume-ng-sinks/pom.xml @@ -23,7 +23,7 @@ limitations under the License. flume-parent org.apache.flume - 1.7.0-SNAPSHOT + 1.7.0 org.apache.flume diff --git a/flume-ng-sources/flume-jms-source/pom.xml b/flume-ng-sources/flume-jms-source/pom.xml index 2fdf656431..de8a54e0bb 100644 --- a/flume-ng-sources/flume-jms-source/pom.xml +++ b/flume-ng-sources/flume-jms-source/pom.xml @@ -23,7 +23,7 @@ limitations under the License. flume-ng-sources org.apache.flume - 1.7.0-SNAPSHOT + 1.7.0 org.apache.flume.flume-ng-sources diff --git a/flume-ng-sources/flume-kafka-source/pom.xml b/flume-ng-sources/flume-kafka-source/pom.xml index 80bb7f45fa..7bb66a9720 100644 --- a/flume-ng-sources/flume-kafka-source/pom.xml +++ b/flume-ng-sources/flume-kafka-source/pom.xml @@ -22,7 +22,7 @@ flume-ng-sources org.apache.flume - 1.7.0-SNAPSHOT + 1.7.0 4.0.0 diff --git a/flume-ng-sources/flume-scribe-source/pom.xml b/flume-ng-sources/flume-scribe-source/pom.xml index 8c9f7b6c55..21f11a2562 100644 --- a/flume-ng-sources/flume-scribe-source/pom.xml +++ b/flume-ng-sources/flume-scribe-source/pom.xml @@ -23,7 +23,7 @@ limitations under the License. flume-ng-sources org.apache.flume - 1.7.0-SNAPSHOT + 1.7.0 org.apache.flume.flume-ng-sources diff --git a/flume-ng-sources/flume-taildir-source/pom.xml b/flume-ng-sources/flume-taildir-source/pom.xml index 09063fb54a..1995f3a491 100644 --- a/flume-ng-sources/flume-taildir-source/pom.xml +++ b/flume-ng-sources/flume-taildir-source/pom.xml @@ -23,7 +23,7 @@ limitations under the License. flume-ng-sources org.apache.flume - 1.7.0-SNAPSHOT + 1.7.0 org.apache.flume.flume-ng-sources diff --git a/flume-ng-sources/flume-twitter-source/pom.xml b/flume-ng-sources/flume-twitter-source/pom.xml index 3073df67a3..82cce3cdc4 100644 --- a/flume-ng-sources/flume-twitter-source/pom.xml +++ b/flume-ng-sources/flume-twitter-source/pom.xml @@ -23,7 +23,7 @@ limitations under the License. flume-ng-sources org.apache.flume - 1.7.0-SNAPSHOT + 1.7.0 org.apache.flume.flume-ng-sources diff --git a/flume-ng-sources/pom.xml b/flume-ng-sources/pom.xml index f52695656b..5df918f7ef 100644 --- a/flume-ng-sources/pom.xml +++ b/flume-ng-sources/pom.xml @@ -23,7 +23,7 @@ limitations under the License. flume-parent org.apache.flume - 1.7.0-SNAPSHOT + 1.7.0 org.apache.flume diff --git a/flume-ng-tests/pom.xml b/flume-ng-tests/pom.xml index 692e639f3f..abdf79024f 100644 --- a/flume-ng-tests/pom.xml +++ b/flume-ng-tests/pom.xml @@ -24,7 +24,7 @@ flume-parent org.apache.flume - 1.7.0-SNAPSHOT + 1.7.0 flume-ng-tests diff --git a/flume-shared/flume-shared-kafka-test/pom.xml b/flume-shared/flume-shared-kafka-test/pom.xml index 00260a9502..810b5ce9d0 100644 --- a/flume-shared/flume-shared-kafka-test/pom.xml +++ b/flume-shared/flume-shared-kafka-test/pom.xml @@ -16,7 +16,7 @@ flume-shared org.apache.flume - 1.7.0-SNAPSHOT + 1.7.0 org.apache.flume.flume-shared flume-shared-kafka-test diff --git a/flume-shared/pom.xml b/flume-shared/pom.xml index c6ec4fd7d8..272baf8ed8 100644 --- a/flume-shared/pom.xml +++ b/flume-shared/pom.xml @@ -23,7 +23,7 @@ limitations under the License. flume-parent org.apache.flume - 1.7.0-SNAPSHOT + 1.7.0 org.apache.flume diff --git a/flume-tools/pom.xml b/flume-tools/pom.xml index 737e7b7a8e..ad68ea22c4 100644 --- a/flume-tools/pom.xml +++ b/flume-tools/pom.xml @@ -25,7 +25,7 @@ limitations under the License. flume-parent org.apache.flume - 1.7.0-SNAPSHOT + 1.7.0 org.apache.flume diff --git a/pom.xml b/pom.xml index ded7b7d396..021ed71912 100644 --- a/pom.xml +++ b/pom.xml @@ -27,7 +27,7 @@ limitations under the License. 4.0.0 org.apache.flume flume-parent - 1.7.0-SNAPSHOT + 1.7.0 pom Apache Flume @@ -206,7 +206,7 @@ limitations under the License. org.apache.flume.flume-ng-sinks flume-dataset-sink - 1.7.0-SNAPSHOT + 1.7.0 @@ -1088,162 +1088,162 @@ limitations under the License. org.apache.flume flume-ng-configuration - 1.7.0-SNAPSHOT + 1.7.0 org.apache.flume flume-ng-core - 1.7.0-SNAPSHOT + 1.7.0 org.apache.flume flume-checkstyle - 1.7.0-SNAPSHOT + 1.7.0 org.apache.flume flume-tools - 1.7.0-SNAPSHOT + 1.7.0 org.apache.flume flume-ng-node - 1.7.0-SNAPSHOT + 1.7.0 org.apache.flume.flume-ng-channels flume-file-channel - 1.7.0-SNAPSHOT + 1.7.0 org.apache.flume.flume-ng-channels flume-jdbc-channel - 1.7.0-SNAPSHOT + 1.7.0 org.apache.flume.flume-ng-channels flume-kafka-channel - 1.7.0-SNAPSHOT + 1.7.0 org.apache.flume.flume-ng-channels flume-spillable-memory-channel - 1.7.0-SNAPSHOT + 1.7.0 org.apache.flume.flume-ng-channels flume-recoverable-memory-channel - 1.7.0-SNAPSHOT + 1.7.0 org.apache.flume.flume-shared flume-shared-kafka-test - 1.7.0-SNAPSHOT + 1.7.0 test org.apache.flume.flume-ng-sinks flume-hdfs-sink - 1.7.0-SNAPSHOT + 1.7.0 org.apache.flume.flume-ng-sinks flume-hive-sink - 1.7.0-SNAPSHOT + 1.7.0 org.apache.flume.flume-ng-sinks flume-irc-sink - 1.7.0-SNAPSHOT + 1.7.0 org.apache.flume.flume-ng-sinks flume-ng-hbase-sink - 1.7.0-SNAPSHOT + 1.7.0 org.apache.flume.flume-ng-sinks flume-ng-elasticsearch-sink - 1.7.0-SNAPSHOT + 1.7.0 org.apache.flume.flume-ng-sinks flume-ng-morphline-solr-sink - 1.7.0-SNAPSHOT + 1.7.0 org.apache.flume.flume-ng-sinks flume-ng-kafka-sink - 1.7.0-SNAPSHOT + 1.7.0 org.apache.flume.flume-ng-sources flume-scribe-source - 1.7.0-SNAPSHOT + 1.7.0 org.apache.flume.flume-ng-sources flume-jms-source - 1.7.0-SNAPSHOT + 1.7.0 org.apache.flume.flume-ng-sources flume-twitter-source - 1.7.0-SNAPSHOT + 1.7.0 org.apache.flume.flume-ng-legacy-sources flume-thrift-source - 1.7.0-SNAPSHOT + 1.7.0 org.apache.flume.flume-ng-sources flume-kafka-source - 1.7.0-SNAPSHOT + 1.7.0 org.apache.flume.flume-ng-legacy-sources flume-avro-source - 1.7.0-SNAPSHOT + 1.7.0 org.apache.flume flume-ng-sdk - 1.7.0-SNAPSHOT + 1.7.0 org.apache.flume.flume-ng-sources flume-taildir-source - 1.7.0-SNAPSHOT + 1.7.0 org.apache.flume flume-ng-sdk - 1.7.0-SNAPSHOT + 1.7.0 tests test @@ -1251,26 +1251,26 @@ limitations under the License. org.apache.flume flume-ng-auth - 1.7.0-SNAPSHOT + 1.7.0 org.apache.flume.flume-ng-clients flume-ng-log4jappender - 1.7.0-SNAPSHOT + 1.7.0 org.apache.flume.flume-ng-clients flume-ng-log4jappender - 1.7.0-SNAPSHOT + 1.7.0 jar-with-dependencies org.apache.flume flume-ng-embedded-agent - 1.7.0-SNAPSHOT + 1.7.0 From 38b5b3a7ed98cedaaef2b9351518a9fe09703a05 Mon Sep 17 00:00:00 2001 From: Mike Percy Date: Tue, 11 Oct 2016 15:48:33 +0200 Subject: [PATCH 337/341] Remove checkstyle from flume-1.7 branch MIME-Version: 1.0 Content-Type: text/plain; charset=UTF-8 Content-Transfer-Encoding: 8bit Something about the checkstyle Maven config is causing the source artifact build to fail. Let's just remove checkstyle from the 1.7 branch for the moment until we figure out how to solve this problem. This should unblock the release. The source artifact appears to be being generated after applying this patch. This closes #66 Reviewers: Denes Arvay, Bessenyei Balázs Donát (Mike Percy via Bessenyei Balázs Donát) --- flume-checkstyle/pom.xml | 34 --- .../flume/checkstyle-suppressions.xml | 74 ------- .../src/main/resources/flume/checkstyle.xml | 196 ------------------ pom.xml | 64 ------ 4 files changed, 368 deletions(-) delete mode 100644 flume-checkstyle/pom.xml delete mode 100644 flume-checkstyle/src/main/resources/flume/checkstyle-suppressions.xml delete mode 100644 flume-checkstyle/src/main/resources/flume/checkstyle.xml diff --git a/flume-checkstyle/pom.xml b/flume-checkstyle/pom.xml deleted file mode 100644 index a5e3b37b9e..0000000000 --- a/flume-checkstyle/pom.xml +++ /dev/null @@ -1,34 +0,0 @@ - - - - 4.0.0 - - org.apache.flume - flume-checkstyle - Flume checkstyle project - 1.7.0 - - - - UTF-8 - - - diff --git a/flume-checkstyle/src/main/resources/flume/checkstyle-suppressions.xml b/flume-checkstyle/src/main/resources/flume/checkstyle-suppressions.xml deleted file mode 100644 index 598146870f..0000000000 --- a/flume-checkstyle/src/main/resources/flume/checkstyle-suppressions.xml +++ /dev/null @@ -1,74 +0,0 @@ - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - diff --git a/flume-checkstyle/src/main/resources/flume/checkstyle.xml b/flume-checkstyle/src/main/resources/flume/checkstyle.xml deleted file mode 100644 index 7035e091c0..0000000000 --- a/flume-checkstyle/src/main/resources/flume/checkstyle.xml +++ /dev/null @@ -1,196 +0,0 @@ - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - diff --git a/pom.xml b/pom.xml index 021ed71912..d33be9910e 100644 --- a/pom.xml +++ b/pom.xml @@ -47,8 +47,6 @@ limitations under the License. ${project.basedir}/target/docs 1.7.4 - 2.17 - 6.19 0.90.1 2.4.0 0.7.0 @@ -60,7 +58,6 @@ limitations under the License. - flume-checkstyle flume-ng-core flume-ng-configuration flume-ng-embedded-agent @@ -642,47 +639,6 @@ limitations under the License. apache-rat-plugin - - - org.apache.maven.plugins - maven-checkstyle-plugin - ${checkstyle.plugin.version} - - - - org.apache.flume - flume-checkstyle - ${project.version} - - - - com.puppycrawl.tools - checkstyle - ${checkstyle.tool.version} - - - - - verify - verify - - flume/checkstyle.xml - flume/checkstyle-suppressions.xml - checkstyle.suppressions.file - UTF-8 - true - false - - - check - - - - - @@ -821,7 +777,6 @@ limitations under the License. - junit junit @@ -1096,11 +1051,6 @@ limitations under the License. 1.7.0 - - org.apache.flume - flume-checkstyle - 1.7.0 - org.apache.flume flume-tools @@ -1470,20 +1420,6 @@ limitations under the License. - - org.apache.maven.plugins - maven-checkstyle-plugin - ${checkstyle.plugin.version} - - flume/checkstyle.xml - flume/checkstyle-suppressions.xml - checkstyle.suppressions.file - UTF-8 - true - false - - - From 1a62453e2c1c5b05fa8102c9814795c163a7239d Mon Sep 17 00:00:00 2001 From: Denes Arvay Date: Tue, 11 Oct 2016 16:23:51 +0200 Subject: [PATCH 338/341] Add flume-shared module to assembly xmls MIME-Version: 1.0 Content-Type: text/plain; charset=UTF-8 Content-Transfer-Encoding: 8bit Fixes the issue that the assembly plugin created the source package without the flume-shared project in it so maven compile failed on that. This closes #67 Reviewers: Mike Percy (Denes Arvay via Bessenyei Balázs Donát) --- flume-ng-dist/src/main/assembly/bin.xml | 1 + flume-ng-dist/src/main/assembly/src.xml | 1 + 2 files changed, 2 insertions(+) diff --git a/flume-ng-dist/src/main/assembly/bin.xml b/flume-ng-dist/src/main/assembly/bin.xml index a61180d9cd..68814aa50e 100644 --- a/flume-ng-dist/src/main/assembly/bin.xml +++ b/flume-ng-dist/src/main/assembly/bin.xml @@ -69,6 +69,7 @@ flume-ng-embedded-agent/** flume-tools/** flume-ng-auth/** + flume-shared/** **/target/** **/.classpath **/.project diff --git a/flume-ng-dist/src/main/assembly/src.xml b/flume-ng-dist/src/main/assembly/src.xml index e5f4156bf8..bf874cab4b 100644 --- a/flume-ng-dist/src/main/assembly/src.xml +++ b/flume-ng-dist/src/main/assembly/src.xml @@ -50,6 +50,7 @@ org.apache.flume:flume-ng-embedded-agent org.apache.flume:flume-tools org.apache.flume:flume-ng-auth + org.apache.flume:flume-shared From 174bed476e6d750f44a7026d2fa88a7e913b8681 Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?Bessenyei=20Bal=C3=A1zs=20Don=C3=A1t?= Date: Wed, 12 Oct 2016 18:32:03 +0200 Subject: [PATCH 339/341] Add README.md and doap_Flume.rdf to the generated release artifacts This closes #69 --- flume-ng-dist/src/main/assembly/bin.xml | 4 +++- flume-ng-dist/src/main/assembly/src.xml | 4 +++- 2 files changed, 6 insertions(+), 2 deletions(-) diff --git a/flume-ng-dist/src/main/assembly/bin.xml b/flume-ng-dist/src/main/assembly/bin.xml index 68814aa50e..185eac9217 100644 --- a/flume-ng-dist/src/main/assembly/bin.xml +++ b/flume-ng-dist/src/main/assembly/bin.xml @@ -75,17 +75,19 @@ **/.project **/.settings/** lib/** + **/*.iml DEVNOTES - README + README.md LICENSE NOTICE CHANGELOG RELEASE-NOTES bin/** conf/** + doap_Flume.rdf diff --git a/flume-ng-dist/src/main/assembly/src.xml b/flume-ng-dist/src/main/assembly/src.xml index bf874cab4b..9e39d9a279 100644 --- a/flume-ng-dist/src/main/assembly/src.xml +++ b/flume-ng-dist/src/main/assembly/src.xml @@ -99,12 +99,13 @@ **/.project **/.settings/** lib/** + **/*.iml .gitignore DEVNOTES - README + README.md LICENSE NOTICE CHANGELOG @@ -115,6 +116,7 @@ flume-ng-doc/** flume-ng-tests/** dev-support/** + doap_Flume.rdf From a1b55d2270cbb2ada813ead94fa8711b22242466 Mon Sep 17 00:00:00 2001 From: Attila Simon Date: Wed, 12 Oct 2016 19:21:28 +0200 Subject: [PATCH 340/341] Update LICENSE MIME-Version: 1.0 Content-Type: text/plain; charset=UTF-8 Content-Transfer-Encoding: 8bit The LICENSE file was outdated and missing some license texts This closes #68 Reviewers: Mike Percy, Bessenyei Balázs Donát (Attila Simon via Bessenyei Balázs Donát) --- LICENSE | 53 +++++++++++++++++++++++++++++++++++++++++++++++------ 1 file changed, 47 insertions(+), 6 deletions(-) diff --git a/LICENSE b/LICENSE index adcfc3451e..92cb484825 100644 --- a/LICENSE +++ b/LICENSE @@ -233,13 +233,14 @@ For irclib-.jar: jackson-annotations-.jar: jackson-core-.jar: - jackson-core-asl-1..jar: + jackson-core-asl-.jar: jackson-databind-.jar: - jackson-mapper-asl-1..jar: + jackson-mapper-asl-.jar: jetty-.jar: jetty-util-.jar: joda-time-.jar: kafka_2.10-.jar: + kafka-clients-.jar: kite-data-core-.jar: kite-data-hbase-.jar: kite-data-hive-.jar: @@ -264,15 +265,14 @@ For servlet-api-.jar: snappy-java-.jar: twitter4j-core-.jar: - twitter4j-media-support-.jar - twitter4j-stream-.jar + twitter4j-media-support-.jar: + twitter4j-stream-.jar: velocity-.jar: xalan-.jar: xercesImpl-.jar: xml-apis-.jar: zkclient-.jar: - zookeeper-.jar: - + The Apache License, Version 2.0 For @@ -527,3 +527,44 @@ For CAUSED AND ON ANY THEORY OF LIABILITY, WHETHER IN CONTRACT, STRICT LIABILITY, OR TORT (INCLUDING NEGLIGENCE OR OTHERWISE) ARISING IN ANY WAY OUT OF THE USE OF THIS SOFTWARE, EVEN IF ADVISED OF THE POSSIBILITY OF SUCH DAMAGE. + +For + xz-.jar: + + Licensing of XZ for Java + ======================== + + All the files in this package have been written by Lasse Collin + and/or Igor Pavlov. All these files have been put into the + public domain. You can do whatever you want with these files. + + This software is provided "as is", without any warranty. + + +For + lz4-.jar + + LZ4 Library + Copyright (c) 2011-2014, Yann Collet + All rights reserved. + + Redistribution and use in source and binary forms, with or without modification, + are permitted provided that the following conditions are met: + + * Redistributions of source code must retain the above copyright notice, this + list of conditions and the following disclaimer. + + * Redistributions in binary form must reproduce the above copyright notice, this + list of conditions and the following disclaimer in the documentation and/or + other materials provided with the distribution. + + THIS SOFTWARE IS PROVIDED BY THE COPYRIGHT HOLDERS AND CONTRIBUTORS "AS IS" AND + ANY EXPRESS OR IMPLIED WARRANTIES, INCLUDING, BUT NOT LIMITED TO, THE IMPLIED + WARRANTIES OF MERCHANTABILITY AND FITNESS FOR A PARTICULAR PURPOSE ARE + DISCLAIMED. IN NO EVENT SHALL THE COPYRIGHT HOLDER OR CONTRIBUTORS BE LIABLE FOR + ANY DIRECT, INDIRECT, INCIDENTAL, SPECIAL, EXEMPLARY, OR CONSEQUENTIAL DAMAGES + (INCLUDING, BUT NOT LIMITED TO, PROCUREMENT OF SUBSTITUTE GOODS OR SERVICES; + LOSS OF USE, DATA, OR PROFITS; OR BUSINESS INTERRUPTION) HOWEVER CAUSED AND ON + ANY THEORY OF LIABILITY, WHETHER IN CONTRACT, STRICT LIABILITY, OR TORT + (INCLUDING NEGLIGENCE OR OTHERWISE) ARISING IN ANY WAY OUT OF THE USE OF THIS + SOFTWARE, EVEN IF ADVISED OF THE POSSIBILITY OF SUCH DAMAGE. \ No newline at end of file From 511d868555dd4d16e6ce4fedc72c2d1454546707 Mon Sep 17 00:00:00 2001 From: Mike Percy Date: Wed, 12 Oct 2016 19:32:13 +0200 Subject: [PATCH 341/341] Remove test dependencies from binary release artifact MIME-Version: 1.0 Content-Type: text/plain; charset=UTF-8 Content-Transfer-Encoding: 8bit This patch removes some test-specific dependencies from the binary release artifact. These were introduced by the new flume-shared-kafka-test module that is intended for sharing test code. Please see the new comment in bin.xml for more information. Reviewers: Bessenyei Balázs Donát Closes #70 --- flume-ng-dist/src/main/assembly/bin.xml | 9 +++++++-- 1 file changed, 7 insertions(+), 2 deletions(-) diff --git a/flume-ng-dist/src/main/assembly/bin.xml b/flume-ng-dist/src/main/assembly/bin.xml index 185eac9217..68296a0c96 100644 --- a/flume-ng-dist/src/main/assembly/bin.xml +++ b/flume-ng-dist/src/main/assembly/bin.xml @@ -38,6 +38,12 @@ false org.apache.flume.flume-ng-clients:flume-ng-log4jappender:jar:jar-with-dependencies + + junit:junit + org.hamcrest:hamcrest-core @@ -50,8 +56,7 @@ - - + ../

    bq?@Q1)c3-n}MBP}+m(xz1P%~lL)U>0H?@L`P zoJ8-R<@_6w>`27KzrDAe+L7YPw_BsPrf%u|vswRZPtJzI_fP^q@Tm7heBkw;9}elw zx4&QFy^0wmE~31uL@6mNszY^Dw#A_OL`Wa3$^TLiB43Rp{4Z~TQ@bsx&vd*6=}(2} z4sr$prJw|2?q9A7Nx?AN-8)SZ3&QNQ)|gvUx6J(??EkffIm7I6N`R?XUo^Lj^Z#)D z6O8lq;c+gd@qeZRh~<3be1Ah_g;g56uaV2QXSMlJ*hp#oAI^U)jrE%c2&6CmZU}Mx zOL1yg&hvgmmH-jJ5aRw<64LJo2(MWS0EaO8wJl(nM@Mtb>TA0M3+hlijOv9VN zy#&GRe;Cb2WPrd!Dg5$7L2V_FjXvqcLfXSGya*vS`}J>tqRmphGvL?jM2lqa4A>Tg zsKxWbN}cbY!G9a&xSR^xW_L?^Li}&8j{!A+!IQ-N{9>!|+b%vN)KIi*2#(8A5XzEwNnh(4fvE^UChV7Xic=zp?*KDv>uV7%Hg$64rwsFFfZwp(1|&?Ojlz z-}TD`@S|K-0R2w=89;le09qdoAmR4=K%1MA`H9LOUjj8t=HK@enIrE6FjVGSRC4-K zmVwu7DikUr^Y80KX8qy-0_lstORE3-*9}ew=M`_JQ`_Cee_{DFQ{|-M z=Hg~los}(SJS?B$i3>u~w3bi*W&bbW#u9ya94{F3o+LXg2!~3EPCTnb|F2p9YbjZx zkBtGQJ<)hEtJLCyxcx8J{{atb^r|<@Z+v*O($@%Q+EAnLig1pwk~P$Se+;psO1s5T z=6X+yO2b7Z;#n&FF*t-rmG+1Mrh~20zFMU}%>KW?Nw0bP%3IS$I^MqNtAyh_s8X;b z#6}wRb`4SIkHi_)nXE1!Sb0~lK z@?5?D3D(xj^E9m;=twI<$IRg&>MWa{pdGSoYwO(KF>@|pIn9hy)6B+XWm8Ao4c4<> zi_GJbcR@R5VgH{`rQL6&vdcv;nU$l+oW=Z4nI|TZSsw_|F^lyN z=ZJ|8XPON-y{uwq*+03Pv$G1QvchI89qXH03|BVTFrddwoH|Xlq&NqW2U7{$ibBce z!}R0Dabl(SF#U@8u2j+w@vQWd&G|o&kCVgr7+|vPaF{MUQ2$!6)TMHO?5P~)3`k!W zJlsTGq`AiS9XS|HSpNhPQIqO84vF&a7fptUCZ@e8ft}MMghx###0X<+Qm-}1q5lIO z)TC-hr&dk!yE{9kuL&9>)MQ-s)M>S2YS7P()l_T$usKa`qE(LB|EuGGTE*fxR?gcf zS`~{{rm1bT`e#T|YBezi7{AyhoSwWp=58H=rB*|CaB9`W=s7#2uNC|4&}!;5C{&|6 zptFRHsFQnCh!xd|$MJ$x??F+AZ?Y#HGUOkW*TBBIVTee;0!A`?-j&BPERDl;mMBfGo@M43WShIp1TZA|^AeM)5} z#{lEn%19n)6ZSuq8NQQK+mWEVIXk7VjN}3J<^UR)HLsNi+V+m#k_Wc4qynoa?eH`fqQc-0eYE zbYarITt%z8qGTC#MLV{8J)}&#Zs7N-Vsdq}riuF-@zV_hCqs#W2aKc;$Q_3R30dn z*pK6lJ9QHllI*1#&OHto(P)HIM^>=%(IJN2)4vP9$($Br=Tf3V_ zDXgNtV>tgaxUTo`K6}%8u5XRCHFe9{ow|f{WNba9B(SY5qr1~?fDK0Xn(*kB0=X0F zU)9Q*n4&=Lg#AB$fvk)AwugMieWzs89UqM1PT2pT3S|9KLj0p#{N0Gd^)JPlq>B1= z$y!+*J8ibP6_CoH-v`sgq#`nR-b!SSyf08#b*`|fPBQp)b0a{6E!c!oO_52CYxLgXKLhXoVcChCxwPX&LS)*SDCjQBO@^K{cUf zhOTwzf*{#QJ=NtBh{HsxS+SlP9p4R!^cfdGSc~i?H3;#nda8?=|1=(Jkf|}is3HlV zDz}I_T{A;G*b2G4i&NW$96YBBx^qr@%BQ*@SZJg_bThqEK!a=~J#{tr|Fl?4ls|g! z2I=WtHFVRm9J?Q+ZVg>5O>JW_QQl9OOlrc*N|aqq|3^#0Id^;S94WkzD9=%nR24N> z;-*nUi$3H8^*N|h5=oTZ%>1XRGg02-y(Vi_B+A#)kC61Ot);LMWj9m*X(=W{iX!hO z*){|nZ)%OT107{r>Td3TXzEOqPty*e<*tnK?#=KFGt*Qx%3nyDD7&8oAP(TEDF$%& z>#YH-6XogmcF2d>$wrb+_ZLBh16aS+6FV)d8Qh~B?>!Y#Y30MR!4+jH(+n>Fz*dXJ;NOD zjr4WGQBv3!IVz@B%JV^8pZ=ETD?tomvJ))(U&;TS7w7FJt3lq0H-|%ve!| zc(yWMgz%`$^cY}z5sepzHCIsZOHb4P36$B5X<8k%hXvca=_}LC=>lbvG<6mWJMmG^ z6GGxqtv%wSq1G#+)@V_Sc$Qi{H-_-2*4{C|sCNtiRH-!^-&E_X5G=!O1bfozwi$+y z?n$>|2#uQ<-?Nr8qqN`Vfe(3!PTR2l35G0n+A}@^Iz1~ojS!uPXX&(!x&NWfPn~AO z0Hb>60JU}6Cagbo8p)YP?HGnmdZuZuPGcwdI^kxxty8bUh)(0;BcaoCqSHvxiFlSy zz1aT_Y(bs&i2+8XFQXG1s_;{RD^JxAxEaXsqo2@%lb#jsbF2G~ z;-=oF{t42<`1mNO^pdDFN>n1ArBZM1{|&1&GX@xCdVnf7?T>G$)O$_{mMT^AwvP%^ z+_8NpeU)&>)~GaXpGmq4w+LLw$!PRyh#LcMLc9hVy(}8lh(^S-H0skKghwas8v~3o zJV0%Y`c#ErsZq>d^FpJyTB}j`hV24yvoz{+Rfrokni!8mqZdV^xM)N?Tch_wc+_ZC z3@~cQ15|0m&Nh71w|xkf8pXXgWp#^9uWzQW5ju~8kUj3%; zQgE`S*ZyYzYmf}4#K%CH*F>2yq73mYW%`Hbe=1WO1B_~o7-gi{+JBj?xXY&RJ#1ir zvT9E121a3daZ_n=W6QfBaaL*>kVv>UK8UUUtbhHi1ZUzJ@3HiAMS6 z{k*gkg_dRlx&Nbw5BDjK@y>8+hcW!1(HX6AfTnIaVB2v1l8b3C6yQI9&n{Cg`&DjF;5@IY6~ zk0Y?F@efEr%F@-?J{8-JLprcrOKsf(1#Ga)c{EeN2lQBMEcxa4&5FX=Q0Ep-qPzNA(2&Sj#Y7D^r z-?4zz=LzvI=aVt}{0m=X!wvwcaQftTCd zl*~@W&zyj%l#%%>D`}tB2Vhl zJ1sq1lP4*Kl_z(4D-I@14nJ4#6GJcIX z{}yrf6>*Z~$z6Kd+zP3{8Sap5L8XBXc8qs;rJka?H=Eh-(3AeHy``Qa6zA~NrC zw8*?)5}Ee{EKInIS^w-OIj4j3hPS`ma;kQ`{Z-B>F0E`v&T09U6Yl23Cx@Q^=A-^( z#2f-=oY=s=O(x>?x`c&r)TO+5e^S zs7iedFlyTXK$R-e+Zkl)zrZ(dd9&r9Sad>Wr>_ck<4;c5y1o+JtWL-v)Bg=LnjW73 zjouNBCWuDFvo$jP-$0{=7+_RVG8*w#9h@`R^nU}5-uAF%M3tPl!HF#+NK9!(8@H}c zuAQ>igxczTPpHpH{EDKzt65Ngdaz~m?KO<~yo%meRvwQ@%o+SImIo1p1-NP;$~{}Qs=%s3wW z_1+e3#)~$@v$QGNHiSpt%!vU;RV@HjzG2mdfm0OrKec(EogMXzJxas+^tC~0h)DB0 zyoZ4SnNTn?x-%pcU9)d|5|nvQl$j*T5YJZTn-CtAnHvL4f$bV*8}KOZ9fGAYA8-?w zn&Oey8q!xr=e4~jPT8|&ipkW6g9|x{M#Y@}!;{Xe_+)7GzGyUAG$Nj*QSn0|JZdyA z2AFMajim8b{4X2UW#jE$YBZRz*cI5IN}`6c=0=>|((*<-G~UXAWHsJO_6QMYO3UOao4k#=sE`|Z?1$H95DBPcR)9fJS_*5JsC0)H^tpR2N;c7>{ZjMJw@xlhrGm> z1E4%ioOy_P^*>i=`=wZ+RiXdkcZD*Z2bS~F&UPyAIvM2|>clic8t9D?=v!U4 zOhK0g(EWw>*AeL39Jdrd=46*T8l0fLH2BcIKKMd3C=2ng+ac`mr7l8kg%5-tYVv=r zvsgi9IDgkb*0bRqhaLoa^hRx>H^e?e>3)YkoS7au^F0Yu})lsgw6Xfq-^ z#Di1z)t%&7-SJK*^)R$*0w>PftmV_6OfqqsM)Q=(WQ4i@sb%EU{U7h6U=!{~YQjv* zifjmzIw{L!GQ!;d)YNBV>VAkYp}U15$;jN01&RPW&^u3dykS~9Pf3D@CQ1v7o3{Mi z#MdIp!T5%KM>d71bLzgIJ4@6O5>vO{S(28X>07&-M=2TT>@xv9`zgk6BDxn^X4-lpu7LfFvM~>mQEzCN<){q*J$dK7g2jnob?Toq(E?3}z9hI!nZ97IBhI-PJn@p*cyWrX9c#ey$!57}{Wt znwmJ~4jkaBk=X%XZS}EhJb4YTIFS$0j)|bn-`-_63kOTYR;5IP{t4U;+RkO$h-gQo zMEfp+wqRSAOJ<@So(e5S{lmH~EO6O7AlhLm(Pl@`7UjF_1`+MhlxXKf&=&W1*{>wp z%9LoEB4|tcx$M>u?T}Pxag+Z7zb)iHUY_O0=aBwB@~B z-k>Ae@(7wlY23d5;a&=8XXWpA9E~pmNCd`t|068Y!M$AW^Cj(N5!!+_Dkp-rVjGwH zmx#7BB^vL4gt;Bk(~T|BXiHL}EsLP7+}h=?U2S%ytlQz;UG5zu+JXq0lub30fX1xVoFZ%0EVi;%yLUm(Ilg{;9lk6# z)|?$-endByTm8uVpa^quV-4${u*|Evy4&f=B7kr{tuhvXu!|WXme7b?QZV> z1R3F&JU7;?(VA1DO^9$iw!O^iWF;^*n*UC7Ty-u-$83Cs@Q5Sp%dsE(rexAP4WGiGEVI{_dvx$2nT%P#tiv}i zYkj##380wb<|L*#sk!!G|C5Wpc}YqoSzGU6?*GnHr4ozthjU$3!nehju9U!+;0DF$ zAD}#v6K5P(=$D7cb_yYJtPAHd+x?R};fvy%WKD3Educ&Vlu>6jgkRQQ%10(2jjrDmdtmzS~H!+Al(^#LS!de@_dJE&|Q;qd$DlF6g4=n#S zhSDb*>yuPiKSy}IgVFS{#`-uBOA4%sgCkh)VnBTatPa`uveAel8F->u{~xG2eEImL zJa}ls&cG8-^%*b&Gtl3|H!mv#PrMya98(9T7*i8p0|3X=A~mKo0scQTG%GC(---VL zjH6(_!>+7JXxrikd^ub#;Ofl(BSId2Y4bT=e4gMJP#dlVgCSGZrU01JBfZI&dlO@F|V=^Mgy32lQk zp6Nd0ouOLzj!~DOqV5#a|5~CVbT)q30c4V>JEcg}U6QEo2|xnpDN}5973XL12!Pgi zeir`B&PRsczO1OYtf8#s^AYmlA$2-SJ}A4>6ka;#q$}_#;7OULiCo72iyYSDi$U?U zeB7bN-%gdA|GymsQ>S@MHTC}@hif(RSVtbT<7l40p9%H(|0n$~C(n=>0}{rb`e4Y| z?4bUNB&JVfDRsP$Gg@J5EuN+=p*F2Q=rh!oCJD6yEyJ|9$gos}8mq?BV{2T`*@)l$ zU?Z>u+cZ=EE>*#H8h$B4l1UP5(=G;Sj;*DM!Nx8Z^q&^ipS;hAt-)^`Ty_K$yME{a-cEaZ{j^v)eAx zMjzqqlKE#P$x|AU$xgzE`Jc&2GvYmg{o^?0$NSg?K%NudE2DraXhb53x%T=xgh+Fp z89Nf160gpQ9oZUd*=NCNgCW?qhQYAoBtE8^!_U85TFaP=>m!(J`e7kqXs!{v`diO; zZH>7SMaiR5YKyyAY`VGs7g($|J{=bOgqp)*(~ZT5NF;tk$h}SdqY-Jbvtmcdww@*@ zc2sLD7CsD6+YD-MImjr?{vyyV5vz?HBvy+%gIZ&?s5p64O>q*t&EWVCN80Q-9w_oY z6}#_$W)ouI{>F(ZUXyPX|78YU&)d?$8vYwVWvXh?0otaQvrnV9q8&l!0B#Zrht z=M5I~?coe=jrmeal}AMtL$TsMrv3|}uP#0VR{Tt?IKx$pR0JIuqqFkH8Ba;I10A=mJTU1LTpMPX|K;>;Wr5{JgC9wIl% zJZDI2jF%ux9<@?z#AGvB{{@;e#ns36fyq7>lkH{sqo%E#AGv# z$%sgk$s|9E{a;S<%TiDBvvw5X%hV*#g~|KgH?9}1C<{PTCiz+W_z3JH_wS1FrEcOjH5HG2$Dzx{>Ts=27NkfIiq)(@&EE^Hpu z+*E>RpLx}nHx-o>7b?l$PatmgKL(Qjf(LLt-ot}gR%qgt6=kJO6?GQz#;xv?ISep!p3r}PVIdj$>n|} zEO*`bMB1)1*u`!KMSHo>u5WH=D%-LzA55Ta?tl5dblsg?Z>6TaQqgW~sB5SyF1ELp zCYPIiTFWN&*%7%jc67Z%HSI$+?fP=qq{Mz`JUQ)W6KI?JKY`qt16-UftZcGM(=IKq zuPkh^GfMJ7&s=HCeSbvmtbVS?<_~;%u%=yDTG7x{Za;^cT<(X$v>VL*-$3r{zAlc( zP_*^Y7!6HzjmW2VnoTbE+X=L}|Aoa(KsyJGPHt0x+=nQ+>+2g!8;grewm9aW657&^ zYGD1t*>rj0Y$`2-2DAS|&!(6_++6S2pk;7uM%uF%YocyyCMIo$B9diexc|e=P7m;4 zAa(G}emoec74vDD+noYID8ytD)aG3iprLN+4r=pEADSIh(xw@b z6oAdg0FohlaAL@=1rqdazCuVTPl6N(l*`ogw=D zV@7}cKp83PG_?^E{~`Tf{IU`Moh9q-#VYD}cp+H7?JTbGZRNOrs<;bO*QAXoN_92OiQcNF=go=`zswu$daTkw{3Rh?*jQ9tY(Hd_e13W+ zf%x4K;+M0UepM6y6E8iqBtKxQ2=ObtnCsoHiQldzFRd%CEN(W{MryOL6KE4R``-ea zUx|a^Jr)iy%d?tzNnJ@pW%HKHB#K1hEDJEw4tOCV`Bfub98#`mzoKXtH&>QdVi{yj zt;whDcZIIdKG5X9K=7-v40vACeqLx7HIz1$*;~GoyK`d#?Gqw$U$dKw(|VQM`lR0C zrutGm-fs?qN^Nk?4V2l){YixOwL@L+VNLsCEq7I2d1G;fDK}EnJ|Rqdz89f=U8U>Y zt!dw_X~U1Db(Ln>m6|rQr6u?L3nR3zFLgZ@uZWYUHSOYxrpn4D`?a3Q?M+MCw11A! zzM;hRp3=0R(zJ_;@g~|PySz(I`^GTsgSh{NZel1los3v=>EaKiJLn?$EUF(6k%M>l(_MwrrEV5@(S$o`%X=}zOft! z9Beth_YTuuwqu0$L!DiZjXTi3O=ve17vi0>Ta51s3ADNX3(uDicXBIMvolnjU(_TJ2Li^DUu6K{7eUGMH zSyf+HS7plk)MI{L0&R2uD;V>SVHe_~n)aibHX2(+#Z_Cj_o)fAzmLfMc#ezn50t(C z#MuXyy$_xgq5VX*>#;(DFK^UxhtE|=Z4<7hZQ9pIXg`Tvg`8DE`*BUXxT>hIu+-Q) zwJ$Fb+FkICb;F7_pv@k{io_noBmpqbKbf{!hWnKN3?G_FZ|Y`JI?Ghut-|tO?c!nk zh60=@%4z!5I5gs2Z=M_;u&3kA%S^e7Z&j0wY{_!6W{b3{$6nARyP0QIV6_JT=ry;s zO)@edaan~Q!u~H;;fEw?LluY+$BGb#sGEyC-{V^@Hv1=)#U8Z+Os}1oq9KD_ zZj)Nk3IceIzA+799cm65lZ>tUThpV7K9 zNwiAS^e{gE!0BowqZK#6Go0Hi+j*bak9{_>h>1e?`ohjm>1wd1pd6 zI&Zn-otdq|ufD3WuCT1-u(_SU?&koV&Ca&{h z$026v>3S!2T-#Q7np@4{e0km2QF^X)Q|uWeaerSgCD{)j82ZTVL(*)(e$gYnxV+=K z+VSF;>)^-qv`vsEk7_AKI|;LOA!gjg)fnFwGxrx14?9)&%_yK68j(mx3F6ZsMB429 z*zt1QMY9t-zBM-E*3h_rNUSy+e455~K}s`QOLUZ3|LgHeyB+1>#X9QLuu+aTsx@|t zN|Z;q_RtOFug z^Ktejj^I>S_UW58*7^w6K{$sP#}O+m`y^r;>$M2h0v!2^lYbSKedw=+bu7lUnk55k zEW}ysIC5QK*{7}BSbYA6b8CRL2rpxckJVWA>)33pLn2s<@s64J7>#AWWyZ$3F@m)O zFXW5wuCaEPqkz!VQ;|zd#Qd?JM6j0Py=w8%8q0pOn#Jq5UJz6jO|yh14+(^&RPlx!@s|0R&{ z5WER6Uahh0cL7>h$D96FfVC1YIEs53%M;6UjpMW(g!s}@7Q8!+$vC0LjE&=P% z2-Y#)BW}D*W0eWk>;$ZvBUs0J54-VFja4dGwFy|J{txsz&U?s>muRdK!D4Y~dHHXB zBD{|GHoEa*ja4jItdeXjv;QmLb%OVx8!ysWMS?Xe0qcYauM@oo+<2kJDikbscWqt| zM6mwm-S5VSXsjWEH8TP0`v}%a-hFO-u*MoJSo?&rPTDzwwbr}WjStdTg9K|v0@myZ z*2&&IZhTjbwX0z5oq%QP-@t~acz3(;T{PA%f;Bw>i}g=9Z=CAg<;Hi`SUU?AmsOSx z*JebpPV??`<18%^^E(L^3nUxM^nU^g*Lioi@f|hRj)KJk$;Mg`;dQ!qyBps@W9=YV zERbxh%OhB4cz<`}1c8JD1&alejm7n!h9H@>~b z+Fr0&AlX=>BUop9x47}`G}d;4#RAF3Iy{1Pj(4*g-&SL7D_AU$Y^+-&Sm$~-x$y#x zRUlX_kZi1fM6k~DZgk`M8Y^G0SRh$gr}T?po$uY?#`|lm{({8<$;O%%!Meb^-i`Ou zSp5Wx1(J>Rw+PmS-gR!At7mw*uVAr2vavQsurBhhb>n?BRv*D)fn;O-7{R*OyT*<8 z)>yp-Yjhav)LkQ3mv~pZad!G3VK2d|Nx-U)V6FGAa^u@*tZf8~1(GGGVgLXzLmz>O0ZZU*;qp(SeJX3 zx$zzvtA}8D30R9FSXX$Lx^XViV8iZ$#RAFZb!7zWN^gT3@20W32^I?^8|$42)>Ynm zH{MlabrmcYNEX(*t`V%Oy-VD97md|Luvj43SYsks*LWAZaV|okS7*Uufn;N?ieO#q zUF60)X{=6y#RAF3`g;WHI`2X^-ce(96f71riv<#~=tiuGPX8HL>|7p^*tz7q2Y@ro{(s%PV58uc_$>5cE_MNcKFq9) z0^Nr>9{)|G)%Xz6)IS=LeV7YkD6`c&An*X<32pNsNb9;q)v<9kx(Sn+stH4>))ep( zQCOByoDao3K@;X^5AOn2=W?QXRRfBDR-8QQWjC+RoDgD0qum!jv5u@hTkG`wvB%`5+X!rE|IEf_B24`VorQ>R5b%dCF` z!yV({jpWL3HI6sBHHJ$fQXchF%xDML;4I$%VZjna+--5Z;|VDX05IPE#&|>|(i?*K zXb6$J%Rllq%QNctIo{?r%EjsQnOhMHZNMB???tV^u^*2tEh%j(#ba$P?!whNvPOe} z&b~gRXAy2;%c! zj^(VSF5dU~3m;rL7xn*G5Vjs%Sr)cNhprLN++h6B(^Pv}N>%a&}Z-1S~^Zz)BtV*0j)@oJGGyNasNEjM~a_|$) z{Z~G?qVxyN4E3=eTv;UCGB)Cm(1R=I{m1e}cb1>d&l~1kgfIBQJh*bcIsZXNB2&x9 z9%3dF8n}3H<-7mW)N*7#Vj}Uu6>RUx|NqZ}EA(Z{53W>|Hy7ji5wpuS`4i3B;4=I# zF#SJm^deY$2b_joyDh^8dG35Y{L>+<(+eD?MmM{77+@Ee{)eW|tw!&82gutmv3KJD zr+^8Lm!pPmAkr z`12w&|F7mfEti{p=6hq+iz*7;k!o99b5l`yXY?E)f%XG52yd+hje6d;o2b2%O z<565A6uOm~a%EFdWl^Eopq`R4ml-zYL&RAD^GCVTU^7cd6nXN{WHKdP=kY0?ZXvS#LV>D^LAE3&Y zI{D5Tj@QBn)%Umv} z3*Fre;lk3gW|R9;1_xb8qK2@ke*&486}wcd&@Ix08%oM?n1gwQD`jkSO(6VrMB?Rz z>h&!6e@(cwq`0!KA|RaXSwEP9y%9<)4&1PNgz&+5yrmituyVB~U0hUA+0?kjk;3Mh z6&oAO{g1#HEARvgBOavpP^25`n~F-C%3D5Y0aBL=X|eN$-y>2Vvh)Ar?K|M4Dw6-- zY+hK{Wm%QHvnncjz(&{=(P5WG36fD!aalG1ie&WEI|TzKR6IO$01RZ*-)z#JE#aO63R;>xw4joe7JjGrlm+!jAMOkfbj2jj$tmTT5yymjmP%Z zmRH-`OM_=D(1HVmtRekSfb{Qi6o=6qjANuGUC~yBqfdr(q&jvM(xQ&5%>38W^YLXd zCSS^9HHz>o&}}WRFm5B#IG7(LQA60A|MUo-5RWlMSsvR#6K<(%YbkHpnuCsYN+SGR zKpD$RV@$b~$A)Rb&DG5L$sph4gCXzSM-S`Xq3?MI8C~8Xe}_CgB77+ z1-eEyb9q+p-mpAC`b3;#Y1E_}6={rfl`VMgj5SjZwT)eqNSpf)JUyRO97`lL>4YL( z*NTC#x!LRA5$=x3Y76z%LjtHz#>oe&3oNmmXTij%Ng^+{DiTCC%0v;WCMeM&)$ zS}Tw3sR>uqRaaNGZ8er*64h$Nt4;pXBfJtfG}mj)^@?+YspFcT0)$V`iX|u&Fpt%w>)Kme+J>4A9vO1bDX^kozSgY&c$m+~jIo`| zV^x}PZF_n7&|1&z!XvOOW(7#J8{QaJuEqXeHQzoVdA==EjBCyMhnfPYNchYI9*e6U zLy9TJL_NhwZ3@6$?`%TS<}?NXt{Wsu)AYIP{YPAm)wDZjW|%oG-6k_9&6X@htm`)T z*p>mx#sqe~s|QO$EZOy*YCgyli-Q})x<1>bKM#N5%;|ea^^Bd=0f_$UrSqsU^%uE0 zJ`1J!A|1wXq%{e&IAK{HoxQn!nebcA>`eyijVg(ilhQK&gBRBdIc|{Rdbc$4F({3P zmgDd0i3!`R{SIN3^Pcse$n>10w~5*Fe&n)5H3c$p9%N{e=h zr;6VtF2-;p^ItTE6(PnD2M0)Z$A6D-CnVa9&iUQ&Sa6g9l+q zGu&kMzi24UfLB}XH1*fbP@P^I=THc%jc@wIHlWxdxUH%Rr)dvOyh0OC4LGjJHpHdV z-CQqPJ}z0y^xn|&%`H}ma_ttL)<3|<{OSzfy(^5_fv6Q(&i1~k}txr;?HdZtD zrhVj6eBxEae@Ji}pW3*kza>GZL&&A&(tjU>4@-hu#)t$bhe$wo236cLA7wb2pRCf> zkf5Wq^(~hsm4>*{Y3o~_aZ598P1+G@>sy%rW4|~#xy3kl2h3Zy6H+H}-~%a1TeDw; zp4uR@z^RQ(gxD;b7`k<4?9W)tz_nob{RAiRN9}ZxoT@-t&W<$#IS4M{_27w(ag9?O zM{-O5HaNdQm~%AL+q$?=+wd1Y2lKklc(yUVJNw5vtLI=wu~(s(N==pXW+1F<&8n(x zA6VVeihEDW%bUxat1A%2^YY3}^_&;-T!uIFecLH6-y`uC^1YsX>C+D8u$u{m>3Z83 z1OZIcApX?J1;Nl`1rmw>#$N)dtI|=PF}Za2Vi1ei+?_}I z@X+Qa;*nSS39PWWKcu6%WIP3w%(l5Z&HUd}+g%OKP}_P@TeDFcD^hJI;|;alRpNt5 zwXLl`iw`Hm?{l2`v(iypq0@yn_p5ee5YH^Zbgd5)RW}LG($K0KbpwRH_NDx6HArwrt0^=Mo=%ZpZjGf#(&+NgvxW7NyhVX)Rn5h_xN#-kp6wxNtcxae>pM zc633|3$&oAwUaHVM9}x14El`6PD+l)7+#`7-n+qyKXn9M{nUwwq3rncUdDfnpjU>( zpL2x}=f8}gSE>kl?f${gUnRU*NuAreRgeV;l1 zaH@)+*P=G;LcNaCANDh!?t2du8E>1?FrDZKdZXF@bE-P`ycYB#BIu3Gf3n3+O>Qx+ zGor;d9w>}gst9^52@xiHX&-&=c{|^^=e78O`!?_*iL>!0nagc5&P>G4FJQ?WANYMS zmdvwN7b|aTuS4H87wm=Rp5qQqP^yQxGi>HjSR@*zs_LnEBFRe-8D~I1cfVQxTj`i^ z`G#&z6Rp}w#JXiBaZD>hHf-hyt7}|Em|NQw2#U|nPosem5sYW! zvXymg7mGOQ4C3BNX9mTG;*A!4u&WQwI?SlccEUu2Xht|YX@{Gdu*qNwD3<+5o(GTh z@nTeWPeTjF*A3)`khmqIj8f=|*7*A1?Y@c}Uq4CwU7qlP@%8U%j<2c7!=8&*@3m<( zh&{dzsU6ae`K^6n*jC2Zhs^k=ZLb2lI4yyv#^h2zu)Tw{?WICkkFU^?J-$BlFP|Pq zVEle4@wU^X4%uUTeLJ1zPHTK65NmvWnByP4v=(#{fK(=Kb()Gr&|A~%WZh{DN>CYJ zAO0Bh>8)2Ldut9pF#Ao}*68FMk3Tx6O?dNmd|0E?rrkk>CqV{tVA;D6xjb<8LbH=QaFPmh9#_}3jk=Wyn2`MdxX@v#n2p0~PP{U&nx;j1*e_`V6W^!U!Fpf?y6H4K~=Q#g0 zTe4)g0NclFz?N2emaYC-5I${{1?B#vT}bn{NA%p+l?7 zw=(G>{!>wlgbne#l8I+ve`5^ZTM>U-EwZ+>45_QFY%%+dLK|Nxhtq!&9)~xid!l=Q zcq|@EY{stuzFE;8QdeGqtEfB7)+*Yog|4BE`;S!raqXT_+96gZsxkA}fjhp8nZak@{O-j}BYJdeGnMU#UMlD%I5=nt1Dvx3u*4%xWJd z>hIhH9%5G|`aAen>d(VeKoM>IJ)7xcN&Vf|&<6coBl>GI`eQ}vj}ao;`dKso)%H#O zT~&_`*p2mYOVzpQ=nwZo!TzjUsvu~n=14s3ma1o&|DhYvXsEeABye$5H7nqt-y!K} zE(uKm#X`XHSN37xHrU0)?2kL_@Bk++rFaPy_ ziWQrPY}j0ArkBn5=UKyLzloLO$Gu|Z!l-s=ZFzIY-N$Yu94^{J7zmSB1_$K1d=GVM zX|Gsi+Dudnp$p{-6DkjqymDTE$rZb+{j-T!Rq9NVn+J^Yz<^A|;$AWP-#nGA9iYzH z>=Ua+*6pVE8pE>3YECWfL#wNOQ{mtycQnrUhDmNu9kD5i z5A$Ds-*e@zG1Z^JXPWRyVODG#0W;7u{d`{I{NK-K{VuTt4HA3~Oq-ACsr9pjkMz{n zE(`Fv3J-CiX3sP$3`bsjue;cx}KT75XUj)B{rIm1GTM)hukp>|S3Q zV0HCQvBX4_g8C+ERzvU{;9B2vfJ4oJ>mAdkG)`6?;R#S659vp*zZ2kd&5ru$8y*R^ zf9ia&mO(yKrjD4ZJ?f*vM0EF;#sH&h%k*(4=x%)KjHo-!XfigqDVZJvE3%wpj(JWFIgZwHJ%ajX03;W)|J9T0hJLZcB$R?pOiG;#yDqq>_=nOw2@13c zi5cGbHXzN7ef3!^#0;Z^PfKO(kn)b76_pe-ydo^b#NRaQzn(ZZ;lvQ*B#1Lvv#F`B zEU&Aule3{F?(=w``Netjp@29y<54wJPzvHq(R?a~w%4`|F~c77X+b`Iaq2Tg9O+4K z?Gj*f%l5GZ155OzslujwR$XP4oo)=Wu~!9YZ1sIWf=n1i!VKOz2aK3gIy*V1L?6Ne z#{Ruo?yU)Ir0W~Q@7R3Sl+Fw!B(^W$M8P35nYz7QCZc<@kOv#^Tn=1KhC;}$%*`m% z*%ek~r4@Js>D#{izgpV8w*HS;?O?q&r~Z%W%tEkZ2|+y*^x6yQ8E61#B%EK|mcTYTr8;b(nwgI3!T=ReJjG3P_s$F-N6z3LYQRMU zxC{;jp}WJ4?pTqPmg5cGy>q^=B4=>d)t~P;+p``v?wy~G?sO|F-A$c7u~DloZ|kZf zmS&CTU1R^A)%_`ft#ti}2Ugddj_N}26i_(Z>fXJ;rvPg1;f5oixtm0DM;OhqA~km^ z-q75;FZwD{bJy2j;5ceq>k&@<1?gz6yVH$U=Nlrl`amT#IRy}2a*B}JHLUkI|Me%Q zZckt{U2lpr#!1XdM}bkW3Mi=JDE9ZB+5hSF-AxVipv9X-i}Q>YS&@}a#v59Ee|sNX zYVn5p3mr#oZ9``C!gRD4n96Duc8X9Tp}F}z;%N~~*ZVNB``(ejehKwlZDf{P(otb3 zrUHs+XGS;o^|7QCKGJX`^mmKs??|IRR;2zUGrIWzUq$Ng#`=pKrzh(nGkQ@v`qK#w z26ouiU3pzUR?h(yBZ&0PQY8{eILY{||>yr->i1 zyw7^r2=qf<8r$1cdM%aJ@;ng7tM{#ncxnqd_C}!A=2_LaM8F(h4qx8%ZM6O9Dqmw% zIN7nh#7a3FhIO=+85&9oglP`lsUv$M(8pOm`rK%{C9zoOVZMqq3yjjJrRd*u5Aeu!Ou=Eorv5;Za@Hz9r3IEJ{sW zk6I)w#6L0n-#p^u6C-2TKA?!7qlu@sXy09}vf9Lt2q2%pMSCu*0{PkL5q~EM`BMSn z6LFu8T5AUJv(h8}wx9T?{R6}&;YnhN6&m>pO&p8JxY5sES`XLx{Yk`63=p5p$AK@` z#FwW>e4`MTb^A}v`nT7hQ!ry;s1B{1r)gKW;K7ou<}r%l9g@cKC2Enb(Eg0;-yBvv z+EX#P<6;D8uSt*g7fG~D{O8f0h6&vtH0?j6NBi?6+9v<&(LMk(d4?LmeX^!qRa=cm z>~?(R1ak(5}Lz6D=K|G$3udc}w4)-w>dE5FTTGhNgW+ zdbI!Ir~O54fcEsdSYoNBy;RX|!wVTo+i`~*_>D=lTLQEX#^p{+H0>qn(Y_&xwweEX zZF$JhSmGp2`y@?!Ru!I9TVv80;q!m$`$0eLFM9`YABvl!PSmtdOpo>hNwkj+(4J8f z!|4uXCyO-g>gHxV`r2H@6pj1pB-*b8Xdj04n%`;Kzf-hZYbz^iYuh`{iBKnZ`DuS; z^8cQ`aSKD@I8FPw^l0CiMBA+Yc(i9?UFKL#``Gko-;qT7vw%*Tu@1u^8U6BjO}nDK zytb}<>-xUePy4S80ot>0LDlay?cb+I`<^7)R|IIc>=8>W)3lc<+AXQ99I!2KN}~O* z0PWV@V`>K#baJlHPGPlxv_J9F{`$ZG?Y03ioG(@SUY#E8kCSNM9-!Th)s&+(?V~mA zRMO|feT$#=-wFe?XJZBBC{6pQ^l0ClL^~M&%)zP&cRE1dCurKK4$z*5M@aux)BbIGv~Ts({&r}9_K~8lH9gvY7TS^*-*OFTGcSHla$cOPW|%r{G4a2i3v*hu zC~=WwC^1L8NalzsZ3w3^nK|NIP5-;y0KfqNeQaQp<8YEq=8TarCoaN=dSRRUw!AX> zyLHyFff&&^pFNj53Lo3yU4l2H|LzH(JLOl zWdq&c5&w+FS9OlF=vn^!P4T{E_&|Xe!%i0R_W#euUznHqEb$NhB+A%qC-ING|7Big zQyf+ixHP(|JdPdsac5)fq*xt5UUQdBn?HBavbF=~E}b)S*}{2qTbh@) zjbAoz-jdj0)MaDj&z;}e_S>n)ENsL7WY595e2GA)jbG%Xo_+!WyWAtK~9&9N<%&J1uIQd5bG zjMCETD}IjuH20r)96!f?rY6m?DTHGM_PeIVkujMi<3COQ&*S(7b~PQQIUW|uG0on= zLD1s(FSGyE?0z!*XVXavYMLj87+Vd^BLIU+ox69I81U8su1Af#X|gY4r&| z$DhsqH&4dDVz$L?N|5o8P>yM~<0DBN8UHfZ>T&!!9>c*9#qr=!j%jL@*)OXd|J^gd z@o(7nG+lF?9?G#MJ*_ekZgVvGA5X?_`sv+%&?-}cq} z7{L+y7(+Owsnt9E9Di95;J5|*q7Kkn#V*4Tj%jL@vtUcBznJsCo>sr>t@ngNM(ha< z;h3f;zs1k-KT`rSe&0)PmI6m?mI`pJtw__8GsA4j_@BoE9RH4!k5jZ(u?sAOW161) zS|7(kTvC$UAAjhfx9dShY}X5NEKkz|T$RM}x&X(2V5b}7TX4jt(GZSldh#oUqn@f2 z#+d)+RPEg4sTu>wOdK=ZOCaf>ngQ*P32ZM>yBu&E@Vv~(sYr^Ui5F{=rHs8 zleE%O6N)!OLm`T`*h1!hKKi^F`u~uo)_1JJ7M5)jThMGRTgWu)Kgt$jyczm_%&3}b zMXWjL*@7q5LWlaJ>`2oyr zn(7Fk3j0vfL4Z4>fnAZ7E}r!Z$g_)|ncT&7zN0W3^1JyCeeFiw?;ECp3Jd7TZ(;UG zP~qfvO-S}>F3LfX*>g~a`Li|Z+Krc(L(m4a6rcaQ{8?c(#y?jAWMA;Qy5hUr@L@@i zlOqzG7b1bqpB3ipjxr3v&Qtj_NYFw4tT5-Gq|y*KGJjT>v(hcS0Qoc0X8I(9Jn-`J zXN5W3|H~FTFS*5-dxoNOUJ_DkaMX{ato+%c|2ls**v+3Q`N5JHb^grGp0RN=9>zcR zBi9OZG5>MfB6Apj9kZnTHGH5gV$Jx30J(p#1#p->`#kX@Yy>o8*lYS@UK;)ooh~)^ zD>=g6FU66`#l zk_B-7%k5vR_=e> zUXoPU`5a4v94^&4e?05eG5DZ}B0=XnM1l)KBoJ@i`3;nzx4uAm>&>VQyH>BG^oMm% zVqpROzg-&QMtkdmYPU4K^=8nE@YV%~p(I=Eg5(w(ib`m)f)j+)`N~^wCLzM4I|{ug zaWj6f?nyic|4TAY*kqiUNN>NCxKOTyTjC@x%vQdkrnS5V*CE+YXA9)b<2X=JpzcXj zCkB4~MTvyQ91P%fX>;Mx1Mr%nTZ-`DLwv{Kx+3Jp{Vdby|Cwq}0r`mOb}{|`e24v? zyly2g_W$wh|Cvw>|6T2D1IA=3y}#=iS4j5$6}assFY$%UhTulO&~7xfHqbCNZEhri z61$rH@5)rvd77K+FM<8BYc@Oem*l6>#@=x!JzI!HY{my^Z!LNou>VAeT> zPD6!kkKV_r$o$n%3Mi6wbwpty)<0F()SS4X{PBi`*d6^R%8fHL3p2_ngRVFLe|rOO zLVU|tk#WQqiT{&FYGZfw|K%x=sa^hwfi<;;GjYV`G&ErEj&2?@v>m&nJG{zNWE8tA zdc-Fum$n8ZE|sU5!tO5Bc9#laJv>53%BGLedhBM-Kk5eHHHi=7;wOFf2fL%+NvE-) z$J7+g;g3KFLImx0gHO;&X!J+9O*@N*06`b0*TFP>(w5=1xHIU}S)ZTmtYvsD{*5&_ zsiPjwzxFs~AShUaQ!(@399}O98JuXIFwNp+BFROr*U!j>U#B1)U6WW?e6v8jC<@{W zHpI(7&#BnA`3HR15dSU^FA0H2j|Pa{x&F@x;1U%9e9KWsw*=>Fcop#<4#?X$0_c9O zCBb$-lgk6^Paro-g6?;T1ebwB5|7f zKm3^_x&4}CP5h(CIy2!FUdDbHeKzE1?1zzvpnXVXc};tVd%xpOu~WpyevD|CI(owR zrb*Kpr!~gGy#PBHm%6uTo9UxH{$@PWV^3i zekL-&Ge4TL7#U2~k-#|izXeDryI>X!OT5k#`vHaYc|N_3DCeOzyYTP7I#u`yz)#0G?M5Z=)94krJv zMP+O-vQlP2!<|YQF46{4Kyefg@uE9y^oh*K;E9GshzxG0%!mvYWt8d2fE8KkM7#-* z{?BU*>gv{d_)2vUxZSD0EFE3p$^z)>*E~{$Xk?1q!{mQFl{Hq0${L)ibW|3^Q$X1i zCsAAvzx-6(lMNV16L*N>78}K}A}i_JxC(m?3@Gll`pdDyjP-CE*X8Lb4i^4>C~j7wh%xClnx$8YXR(f&+@UO>Yi@EGb0msiRzXb)v+Q~_dC2< z@e*G}cGKJI*E&v|^>9_*+H_Q>8(7_7qOC1^>j&V8LrkrYzx}DR{f0PAI}u948Iq39 zl8_WoB-_;bnEa=Pv=zKFeBsMU zcuU9Zx5_6AjciP<)3giAk92C&(OL+g0!n9h%zpp!v7_pqYd9LJyO#_xejaU9$BI6)cx{_EM=*gCgs z)}^DhBrF9K%I=zR|M*9ZJ>T$KXzV`G*l&%-SdkiAjyE(GZ}+L1#&&o8RgR-h;Nw1z ztJ2XJ?vO#(oIGuO(}*crWvHEkr4h@ag6AJTJhZfNdy39RID4j}vt%>{luPlU{E)pY z=>PxRa14~TQIvL!Q5q|<(i>9I@Bf*=`7-g7JTMOloS$GRDd!WIz)30O%mZ`0dVi}E zI4RY!6F6g8M6 zb)8~^D4g#Je}m!70ES1fkMA!U!(SAJl3_BMFFg%K#eQ#%p3aUXUeImvg2GVVT0R61 ze(f-ql&Zs4{^6}83^xSy@JuY0cv)k3S!2M>v#`mISHIKI!~23ka>Tn01_q|{E=x}5 z9fN{!q`R5>Z*(-mZ27Z^_hc0a+4A?&8Cz2X>Un*FkWpo~H$__-2(#sn;mm!LdT=U6 zl_h#qNo`QHN0qr+&;fOzk96ZMDnZON6Nf7@r#r_e>W(u=XnM!B{v$7x+3M^4_TO^t znlCbC{ude^fajl&##eO&D>fV7oc(XRhJD6L5D)kZ&~ePWJUT%<2EX-Y%f=mhcZ2fC$^AXc=gh76*`s~6s4QM)t$ON+WZ*`j1a&+bb?1E*En;o>M1$>N_J3$g zWE}TZ;-C5vzOm6x;-CM^IId4Dj#Y}lak!fDc-otKsFeT7rXCWG+SCL1@B-y6Lz=Ov z$G(mvIMix|OZWecKmT8${16^)W;XS_t-F6JgtdD?gQ?!l4u?h^SZiA)w%O!kzbCFn zQ`dE2zvq$ga}jz^%d3$sS&CQ#Z}G9ES9=_{lr*U)8pEqCP1ma#u^!iA4KnecwpH%^ z{ec&%XR^0p?{6a}^}s*0vZlGZrro{@CD0zKU~29)LQ&gl@9!XU|CGu{+65gS2x=Tc!>uS zc*cmzwqx&adpZcP_ZQghy}#xEgm@fyE=eAEWT35p{O&-jcaxpjIzpVgTKa{ODiWZ= zfp%@kK+92^Wm-{&p~70OCHE=PV;6fgLj}C>m;cFHas|%6x)S89C6|+b*|9f1tWcpc zLnK%iB0)ctg#?wmpbRa1oeC8cUXHa)#+eE4_X-YJ;Q5XC z{RAAa$X33sxvjOlwx->_S1H5^_Huft-DJrN$SO_x>(-D7bB;iL$c5U5zi_wL?S)QL zA-+5N#|kqMX+^T9BTr>B3fx(yx_OK|m*EW+Rh#`!Yu#ODcaSez?O+a=nNXOnw~av% z0J{!BnDsB!+-k>Jr^Hv@;fJp>(r70QS<{w0Z%rRpxpceCTHaH?!EyT24v^>Fke>#9 zb2veUGzApP zA)^7B*$|We(9D?WeWBr4BrWf!?8uEDn^8tFbVXK@q~#DZ|I-y2?EW?JFH~VYBrX4y zr%j4vtr!qBQE*Nz2yS>ed#-D%KS{fvybtR?>2)iT|{mOj>rrH8Ofd3^B!N z+V)Z*tm8}QNZGZ_qlaGTvpwF)`5mXPWz-|)(M!^4?G!t@2gI63*OvK!a8vhEZm(09 ztU^$;y)GqZ$a%C8ROWxRX8yM}*{RDgT#NaiJBaCBXCLMFTnjZ=gIMj8K*?cveaIjt zHeGA>zpQshhtzhR)$I)8j1<;KK&)%DAQn#tH0RIHtfEm`5bF*Xh#Ny74n|q@@w&4` z*&7|WbDa2yIrs3UV)aC_?zpKqLycu*5J&k84$rK*8{VvL#=nh}P*uM-!6(@@(P}#L zxKx>2k9GFc_4dqm+0MgIoxtF`E=JlrX7zLO`M>Eh=3OXjxCU)nUUdCAg|t@EbNJ+h5*6@wZna&IBz+H-@C z>TNdFVy7%gDXh$jy`}PpzRC<^_hb>OgfA8oKEt~BimpSMSy+HnJB*8Q+Ra5j#DHYQ z{&)lAFth(zDLdytgbw0~|m6?7D@Tmez7}#)=|5ZxLS!DC+;a3vl3)WF8VRcc-@W0vI^2%%yTZOv! z9(LuNnPE_c&)#PLgN8aOH9os4%!PZK^&eGPd9nLEFV=}(jK_DCY^-9cJ!8XDX53*l zwji{+Pt2~SeBb)yhL#4{2m7B#a2BM1 z!#ocYHxgPlxew<*9{-7GfF_!)ZIcEn^S@qELY=*r(=>!ddpPLv5-~X%s?ujEezR;P z(aT4IP+bzm&R$7SS=oY`F!^7tDngx!B4OPJ7s}dD%fQDSGNPTk7+S*2e>9&Jsa4(| zVA1r6pM9LqmCD_zq)HA}{hU563+6AKJ0C;T@DWSqHVBsv!z5C zvq*SnO?GNYicYlJ6x8>ym22+?sClVC?xEs7HInP@%pKW-PvrRSZ(BIM5(+oT$ANZruiSCmi>!A489L7^u$T3UvX?PMOg zWHzyy`eLaKb8Q+5t4pKUsZPSm$`;m!tL@4P>r@m8>rM^k{Fmwx*)IX>puSF%dj3nX z(>I9^vn>{%k>>t;UHM_?q*^CrP!2Bx3pyFe^)F9-qISU^om5hg}p)wfklTUxV zS@|fpGR!ZRm8YAw?}UaciCtEg!P zi`ae2@HxgVs-mVe_}t~I%)T4oBW`BY2fm_=SaY9eII&FsthpKIzWYH~?z zRVDj2+m&->hCvmT>}&Qvs`kv8WJ8VSvn#-&OYQr#%}06k2Rx6CsHt5KIh2i6q}qOt zuUa?o?64X=0J!Zp)UKwYrmK@1TBd~i&9aN;vH@I&w1?Bzvr_J`jh+(HgIH6-{ruzK zCYx%|%^CM&(Es3r#5NhqBDR^z)Od8iMl0=m%x-4_QFQjyjj z&H2A)MHM!sVrQ_|v6Sy)WvgS2zQ(ScvmA&6gH?=ydQ_7vD-6>pOr`$%2 zy~PsH5HFQO8iIF>FaFCTUYAa>Q=WvEl`XtueDR;c8#Q2!@x^~RfxVd zjR-%92p=SiAguQ{_P^O#pa|k2dV28_L^SK|@d?)Zj}(xbu>hl4@1@G%fra=KhWoSs zvmG}lGh{Q7%$Zg@c6yo=&K-I1!T7M+aev-_L&bg+qBKszNa$6;^N~^oCtuAm>JPcw zQqcteFRqJ^H~Am0_$)^ya&oi#*_so@jIVOZQ!!kOiBQb=8E#1`hCo_SnlDQ+<4-~< zD(0hP#fYJdzX}B?^CuxP?*w9ydAvFQ_lc63Q>XoO^LIk?tNfV?09o9~@h?I5iE3cl zUxXYJO#J_eYG4v)Mw%~6jtOH$j!z=wm@rS|_$)+@G7y6t6IKhT&nQQ(16I>wF?x-> zSZ^H<;yVlPC>1sZDjeX%C}MtgFUPIH(jxZ+0rfRy&Y`(>B=h9?f#My?Jo$3BJY)v>NSP;_{{J;)RzPO5I9`^_ zlbQdd%wI>yJcaw8DDyYbGEXu7?;9oa%&o{g#l*i}@eXC4V$Q#PL)*rLAyVckX8!k$ zl9@nQnlDS{DIbc=-$cla8%edyTS8>s1;o%jr<(oGTa?TRu)0p$b{C8h?o5^4lr>B( zcqA%G{~M zmnFosYek6fql94mL(BakLgl=pfoZJ?+oQqDr)T5b{6n zvVJD#ole!Q#qZ<8igpj8|3~BQ3U`BF{8NOT@E3A=M1w*ZMh*lud@GUPK{EeG6?9Hi zVT&sr;G1rJwi<{}FLjBdT@~@YRT*EqPH)u0f?m6WT`~GaO-KYgyY-J2H5HYW6*aXLb%Vt^2erPV);Z1CRuD%Fki}<410E$9 zj5nna``=Yd!Ol}&DyLIQoYD+Sh&ak;P(oIf?sLc>R|)KU>^_a`J}BXIk41JL`g1Tp zgz-NHHZVU@YL7CVub;33&*JcYNFvgphujH5=v*L_WN*^B4td?`T+Zo)cj3alcgKg- zxenp{hn=e=q}8iY7LX4$_Fv*4Titsz1kQ}*;;;n8eq+%Zd{}4?y-v{X5Q6q(lqK5dQHBcMfi~LvRv>YA_{?>33p~8B z(68uTp8qQ=W0S;=R{t(mfwNG;cvm zGxVe$Bflp;foX>1e+3g0mB7>#qmoLU+{{oQF~yX6CKa*W8M6K%lwPtal{mSU?LrP9 zXLVMcfj5DC=_-d!N#*FKY(!HQ=yeBGuerY`EX)$Es^sq1zduwc6G|#ZE%0fT#MIoH zjHJyEWB!L0xpANE_T&5IsFP3O(;TVTz0n;&+Ww0UVro5xWWVjU*)-(5@THgg;} zz-BzBf$G(}RrwjmTioVhTSN_|YE0*^J;L2)6X!pap)^?r@#;-`3B2AR@VE;M@S0`{ zyWTWO1#ea?m3jLoIwz*DzePX z6Ipf)ktL3DkY(m?M3xfE(kQ}xV4-SP`m`AAjI`sSTOnnO}1%^L;!&MIJd7|k)XTPzL?YL_-Yt&~ag7j9`ZM)t$_HNx&^ zbw^1y#?Hx&!2tw~F{@fgDV>27=0e@@6E#(c#n+gpv@-mOwL~kkWd5(Ul1&%p#;zrZ zU9F02EVXRRs$!(GZUSR!WtR}G#8HmmWc?duXr%N92leT|AGE|om|f@(=rg1A(z1u> zWtU{VjL@uG#@TvNtf%+EPt?mUS$&*rOT02}yuq%9I8In)&Yi#1Y#DMK5*vayWND%Q z<9McxlYRP7+*C9;hM#+@n~IEadgP!r@+1XOdJE^jE)})&z@<{+)C82y>a02vZ-8js z(XOtfIv80hHGmCp`X&jb_dih3oFz*aV;W&L7-@R41k-wyk0u41f$N0IDOewMoscD% zAqmNt9U}>mtX1a!(#aT(xnxS5Apyx)omIu0T7L!#n$xb)=F}D!q`Ru&B>{7~v7_Oo zP3FI#xd8q?0d5i9tc~+ucC%fR%qg}TDnapWm)J@IsSM{P89HKJ;Pdi5bAn5^6jtF3 zPGt}oJ%8b{rK8mD$vF7M#+!fVI!rz z(f*20Z4;=Okta%!-Y|%7(xc9|0P7Bmr+MW@WL< z+2?@(?Q%efU5fLceUC61pzLxcj00g$97MvL|LjlQ(zME%C>3FqbBa)sRyiQaD%H|3 z)HA0-Na;9(jNOsAZ(t-AAD2a3cyXF9zHN@||ImF`?Phgbsdw_gtLoD{)i7)MwmA>k zXw=pWmr4r5?3`avotoY~MALDAg}`%-|Jz+@Iu3zH6?ZV8%$+Qdc2^#NR3dae_jf2s zUGJW(>r;fa&;RF;wb~Cz!#^C`!aMuAeb&@BFT2CmFHBzygMAjnIK|SQ+<(LJ@+M4=RP=EyDn?Aej&9C4Og9^Dk>BTeUsyHS#+Gceh7rl1n6_J}uxlnx!pdKeWz zmDIv?tiZM`j;v<*#+x{fdUdL(uB@u59$H&fg}Nj!G5NeKG)_fBqU$-*tb{Fi2}ZfbGX;+yl{4R?Hi4=b;=U~F;x+YS+u96 zn3(EVt1mf+Xf4GoY7)iND8={}Fb1G39KxcL1XK+dFjQhLC#H}W0i4^2Y)n*OC26}w z_Y1_DWQgJk7riUohlMmtjpn}C%zuZu8D4E$ef=5UO#ZK{+zeDE0YS2u?N2KjmW+hD zfQ*Z0+sb+qe{`nbe9tZTu#~lUmB8IQ1n${F$c+DcyD8pW=w7uT_UlT z`Y6%NT}mZbAn-{f)#=t|;0=8*`HNfKBzyA|i$+k1O7>^(mDD6Qzi%} z>!J_ZZJRU^oPg4&G|6wx{7+l@B)h3QvcL-6B*$?6=QfE-6FtNyuPO^hnut$cwOT?< zn3WBIV|@hJ10o4{eSnq486LCICXh4J?gM?2`#@46h9o2rb8Lo>%pzM4y^~Eo3-vIB ziZoUt=GgsgDk>3^ih-My*RiLfI(_Mw5Z5FruVc;n=NQh})LHp4-D7bgTE>6yV#!{3 zqstuorT`kl85>GR&e)F2L`jYYW0IXKDw5Q3JKM4;#$&tVCpPcYq^%DyRh@N}} zsb?d0#|2%o7DGL5y^nnv8mMMRdDd8ovtz){2#r-u^$&g3SEO2fB1k~^-y#0@>^%Qm zsjd<(HO`J05KbUHbUrg~bHbGCUyA8jr<+Bdkjwf1_KP?Ls4pXqV%$|3$USgyeRS zK>c`=|Itb#<2JITs!Fq9r(VY_i(}b7i>JT!FoY5+P#5=k!UzGSDK$e&bA~%jLm9~Q zc82>2MY_)u=>NQ4u*s%U;tUV;@i?fn1Bo~%JmITOaoWoyYAeM_cAqqOx{*4*uEilv zQW0^M`{F-!$HCma`}5;h{J#l@e#~fHzggfeE8KO~;*ZYx|DKe znBug-=&0>%GwbYwS*II^c*zmuUi6fcBqXzPVrL(jmuxa>Pnz)~YyzbuE3z|(e2nJBmz7yLafv8qN`zugME=*!n<+-ljQ;=G3skJuJ@H;q%rvE#BE^OO(EUhe zPW9IgIctct5tmL!(`lmc(wZ3+QMdv)R z5pb=path-gI>JREtuKynCHH?(zl&T)*j0&D<%wlE!hU_}2qAE?Bkac`XJBuom{+!l z5Q`#&Sb4H7gmQ%M&By5@pAaj}{?8>LLWm<=Y1TiMxQ;N_7)c)(iR%cxZ^{wULM%sk zs)_$J8Cr--rWh9Dl5vcESdQ>iuK!Sxx-uPMq3K3oV>`lA&HC?>2*sR=_21E`qZSbI zh|n3Z-|P3_!)o1A8UL|$mnp@BI>J-g{|RcD>j;Zts(FjDtwszw!qZBn?Up4&6i0Yk zLb$I8X_jgrg^N0^Rk*KEwzU>23Ef28IcyX0T%lq5{|e=uCW3~$wRgOei?wkxI>l74 zUEuY&vwUCiWGMeW@v!~7WXe7d@TZ&kk7nGg8SDMK{$014F&6+>RdAm^)0KTT8lSi` zfm9tf81Lj+xZ!xgJ9@u+){FtlhxeT<14jGGkW5~q_j ztZ2pnW#c~cG+%Y%&gqPOnp)qTE!^4L7Wb-{k1=tBag$tCL}^Se5{!HfN}tA#sn+u}ZZKma!w zHz~#?PFD-JW(-g^?z89ks#D*c3NeY-xC<@Z`P&%x2R_EsHyAf5#wAXng!@V zabI=f?u;R}lg3?S;qJ6;ai4RMk1=tBv3`CbZV|YrE4XaX0tI&$ z;Ny*}UbJvAPrUHz2sd~?OU1UNB)HGrM{u7Wf%{yH|1R!~n^Du5`_6ms;={sy?kR$M zRS528lm+f{8UNDXtx^HdLg01-pixdP1{IFre8nOrVq4ABqAr=yiooy>y?hK;Xog!4 zd~TQO(|y%%MfGkN^J45c7doA+>7*a}e=S8E-~-dmejCa{_Cbb)euOVswKaWwB5Z;P zd1gp2#WT`9mHR~ie~#{?BD!4D?xVyRg-gjF9K;lS6CB8#bB)>mvr5@4by=)fG-h>* zoO9md8S*VK0;6~jaeje68+#Z;99BMi@Z&~*v zJ}kI@%oVs{_lKX2vVi+Xp8p`+^=^WsJE9>K$}e#`TNaH-$ueW})rshrWuX>}v~5}m zl0QD}Lq}uHH*s94)7dgsKROsILpsY?&+qP|L}Mi-iDd1F5E4ms+BuyY)O7x2Uv(O5 z34$**b6DdPgsaIK&Z5+`RreLH#*7ryNaWJzFEIIU-N;m9ClgkdItAfs1bXO!5vHZa z3z+})%!O(sB?+wU09C8e1gO&IFEHc(dbiJSm%)~5%<3G~=SvZK!P=N-D$PTm?;P6a zOL3hJI}sVp#v5&RVUCM+9%iX*v+WhL2^}!&>@mabn=Gi4LDWUFJ7$*Vd5{!avJ@FF z{C$A!4w=+LBew5E>{2T2>apz#84E0~-jUSOjG&C>{Hv$X9*Wt74ww~t%!(l+P=(6a z)s?ZUE2H?Qi<%)L-D%Xr11>tz@=wmcAiZ z%^O^c>8TpA%4Yy?-C&0OX;r4(_MhHt1OFju}50; zbMXq%&kYg7&Ba$*`q9_ht$V$l;jX2hi=Pzz+!Uf8$qrupl?ZW@(*`4F*Y0dQWFL4) z%CBDDHY}ALWT&&TgO^nL$gp3NjB<}Sl97}hWaWW)GnaCQWGx8C^uA$d9q7}_*7*J?xl?X>BDqo!tDe4 z%9n$Uoy@$H`48IpO%d&QDfa)mG$-NG)-?SmU8JnT#pF17>)o7d8wpODk-+-0`8Km31-ch*GDEHQxEr!B!xI zJ>FeD#z%>UMoKCZm@3TuuMGuwkDR)M$kbSM2LU zPc?Qm*^yGGYq%P9*Z?7IHC|!zpW0lgMpBaekez4IYGie)QHInjO#bt3cMvGeqN*G5 zZ`6=F(i`PQVo2>`4RY3yIy^9>t{v`5ZNi4QVm6@zX8U@~_Qe1MRG~xaNOw3I=?4>}qkfWZWkp8G$NP#xbsp zV_X@$hDZA zR3lb-jcAYUQy4b5t!9nw`+8%$Aw45NdS|7(Do=L-(yp>B(&IhSqMs}85&b+Ap`RAH-N)SUCk46CBfcc*r?_@Caz>dH5U*p39zTpG`{zM4yW|Rf+4QBu6 z6DojM4dDNy%paQ&46iop-%luuUjc@h=A@SIh?TPQTvW2!{_07Ph#{k{yiiy6Zv+xk zz1rk|bWxfrQiM8^?t}cAYhzjNH^q_JG0Rl1e$>a2rb;quWuh1>Z^6nI+-pqyqcPA_ zcLn`e3_SQ^2NbnjWAZ<`Xfuiyq^y=}%=(9}OzZ0k&H4C4BTRp7Ei|zy|KWu`O=bIq0ACrI}xxS|$ zc`+FYjSO6_-_x=}XAS=79ClE=6(5!rUf(2eUkQOLD>B!gByeAG9dls@=i<9yr=@R2 zrgRA+w>+O;4y?#r&+(5Md?gC^cNT8FBJ=j38)<(Q?i+Rx+^>h=7QcMMFv0!0^2_W! z*j1>z^e87UOT~a7i|KvD6mF|2P1Geb#`n{))_v=KP-)fg?t-S3L;(crmMh=CjEMZN#FFiWzsse^2nd2uLE4mq;F#UL(dWR1c(}xg^AqU zRm6JV^&FkTJV(CIbI?{uK;YgyMc@V=4nZ{JvebgBQ!6j7ej*D$T5xZ^NZ@`L(gV*H zLJtVs58c$tU<`D!B2#2pbe?C%G4HDr(J#xwE%r+MJ&i4^x0wCE8Xb+byIH|6b&4!w z^`nEaaas{3`I(vZsBUwot6e^sqt2xe`Tlj)JRIQ z6{MSAVyKbTtv-L7iT^%SorC_fB7-f}m{k?TZMfQq3B;wT7Xt;k%@mQ`-(dKF`lrTDc!ne(rnj8&P`Lt|FA zNEvVQ*xm*iyIEW^?U?_JMp?br+nN8MeR!mYDBV?gx(kqYm1U8>-6Jjfxud)2=gSEF z+)-`mN2gZi{Ih6`Uq5$D7ybM-L_abCyyN#G#9y5@Ott7Cw=*8H4?Lvnfz%oz=%Y<& z%j|U41n`c>d}QbzNk%4sMcxFE&95pC1OZ8{+!^y#X0IL!`e7?FcmBp#lyJ8TA^MoxOXn`ks(}?QEsT%>6QeSm2J3Y|DRSF;qvayuoam*-v%Gr`ClX2@y?&E zcGO|R?{4^F6c}1|ei!qfwDa#n+OY;@(WCBa6p6lf!-mdbVM8Zh*pQFPBp_yY*BJu$ z`v|yqVgBcKOC2_RvGW&iB*A6=n{fXT0$0L@yZ$b4|KWxWb-02`Wz&{8#nyP3ubd4y z!0^1UQR87*oc5J`(xvnp{5_4Z;qE{9(9z0vD0Z3(^mtcnjdy-@V9k59tnu#dcYKs+ zXr!d}E|xgOf$@&jt+3&q-oENIwLOt!P+7b+PWNy%`H4boxv!ntbMIx_>WSenHEu~k zjaJxj&znBUC*NX7aqJU(8^aGLR99R82iR}87e)*xpMsow;`hI?Odv6`@9FU4#^2h4u* znEiwS2&h7b)E_-`?vN^B!+oOyY=>u44~;nE93|t=9^0QGV|R8{GtU4XQ! zEQ|DiJkp|{`^SiWeu>b}{YO~((P6{RujF0q*U$Z{wSIPULi8hH!~J(?AyCGI4LyyA z?Bf)B<8VG(Q@vE!z)ok4!}tHoM~3~HWR!a>hI@1?bBt5I)f$H%sP$E*bA)l|F@zi$ zT?pI^~OV2d?*#AzYGCZm1Y;$Pbs5ZMdKLDidx!iiEAm zJXnmPuybg)9Rh(4UuCaB3~Z4Ck_mqBAW(sw@1~#Jf*<9@-(^MS zL5crShR$jtKozOP->%7tzj$|8I0g_J9hy|lU6!a0YOTmT_^bdAJcY#o$NbL?AoPmN z?>iZ-KY7n2a(s#+fY zr%x@kzHU&9FYp)Fwy8ggvP)z~bcszE|2)g0l{hDKeq#eZEZ4sY{!euY{gfR}V)Tn$ zB91p2dK2^iw5)u0MJ89DSkH~+g{;VIdPtB3o&+QPaMNaC+F6;ipXmV2^pPxK+Sy%^ z8JVrZm?De~GEmBJsaIo6#o>X9jakZ57U7|r9aTIkP;peI%C{6{Ta#4s1VnN(9B&C! z+&4o_>5A}}$c{LE9;moqOl53~vXQaX()nbRwttlKAGUqxhE z_|bOZUg%Dc^r4fas{lItCP-Z8Bmslu20>Doj70p~qt97Z=&V6ar|^o{ZTPUP@X;*- zw>Si@tjIjJoxm-29dmJJKIrU%qnN%GnfL%iZuz!f?iW~*d5q&9HCP;h`>~TP+Ob1V|UdI>6at!5&%O45!4~8-Y(T8{~^`s>PEw=6y6K5Yu>}#$@?1fzW!<{6qw|R zYP_l}{S}!f5lJwesCnTlxa7)fJy){6o*tFT|wT@nK<4t`)KRxt>GsT1oO81wPNw z8I?)E!2Lwv_KSe~RE`B#r&gxF{)Zp^aG%;k;Fg8JmDI{p69jIVn_3wQiDgA*JIkUw zduAN-zG55<%^8%1TkOI3dm5>gr@r=~qp|kg&S@&t#=4zltbTMb2R55!tWV4O-)>q? z8Y?NOBu$C4on@?$NUGCVnV>+cK5fQ7Pfg>t$1eo3#@RkxO@5-GYTBy%3RmNdvaPGp zO07JD{IBY%8a>t6Z+oYyNUL%Ca5Vxw^Z+4kH9m8^j}p~LN-9t+!4VXnVMukW&p%`0 zzcST1=s)+*U`sV-Z5P?+TNP2&=UdU|w+rv{2YY>9Rx6(6{Ga~GV>Vtfo6rHX*&egm zn8E;+#2669Tam%837OTzTUEqkt;jr!{clNZCuCBMjal1A%6PcPmN)rsZ*jH!+KhmV z&zbn&lX0SAHlYJ%^E_trAR|zP%6PadBksxqe#$8R>A6!NBi(6PgnxPt^PkZuZ>`8& zIAFJ=)WLI{|IwX>JvL=5YNO*lC;1PQF{8vJ)reJIBii|o1x&5Dtp?nton}KukR(|l z{XFv@v=5K;WTm?*Pj>;*uCgrB^E}d`pXWD-egY3=p|L}(=by0jqf;x_{V<^sKeB)X zs?UEV`q?AIe@m|7&xIny9!?u(hV0s<#zXdTw(}-{o!vQ{ngEiGH39sy?0?!#lVQIm z8RZ^};U3+}M6Z17KoF1>nLn@bVPLPG0{Xs*nSuMDuPEWhjVJ2kY##IVJ$Xb@D=+*-Kn-*w`0gr#-!%}yp9l*_R8ONb#yvpUwv}3WaTfs|c-#na z!2n;x`3JXy>eR}-<7=z^z+apvz{^7nuvrK(|3__>yQvkR)7sp&Av~K9$X}B6KQN@l zuR!yKU-pzqQ)K{ewf##b{@0ZkqH;cp#5fZ2Wu;bLlKtPpho)L)0_8pkl-(}T$^E8! z7CWzHsxRH@>W6S?swAVv-eN?_Ze<0Rm2J2t{?Tx0s?$)!2Upbcat|M$W)$seuHo_B zffc5fmredh^O=asC8iOLTEw-zyc~SkCCVeZ#LF1}Mx&B){jqOYy=lN({II&j%kY1y zOH?b%@@F~XD2s-E`Ad62f$Vl;QqI zn)47hwQ@kFih_!=E#J=p6BSs=RJ<@y@xTn_DT}hL@hGGs_dohM9u!j(p`vU|gjm76 zgqyU)tDR7hEioiQ-d9chzh_AItrZr?hd8Q} z?InYWj@QNs?%@%*UxWX5ZCNKeoO{OA`EkF-{3mfILU4d z;!OyoE51<*X6%ZI2$OyT`5)J$b)uvHsGKE$Y<0yq+CYUS9rmOQc6?aa8>fp{>PZ>B zH$y9hVVopH0yuk z+(gGAJ={b`56cF+cxD{)zB&>8vMk(UAH(0%NOZjUx+qhlqp|MP!)Yqk#@fR&RzEtJ z1B9@R^{u#%5{;FVRDQ3->0ucwB$DbhR!MZcWyU{GO^4!+DM@tn3|Et%=(bFBJduJL ztwhJ$eSN}HjXU>rn!0N>_6%1e&_fTrNNZ2L#ecp%+ee9NBqf!3Dsg)HC6?+|pMU#m zx4P-`GcwpxjafY+`~0(BpMMs8zDIbUf6VLilIVD+dw|(tirIt?m_6??dma-Cpz=J7 zmFRfZont(!A|5Nz@y;0mwoRE-V`Em&NEu)B*uDrEds#vcRBW-4YAI$-v) z$LwXu2vngmzUa#MqAR2Lr+4RrE#2w32zUDKIhKFYdq_r~RkRFRu-xgp-2Xs#IyJ;U zNjBo$H$=dxZla@EHDZ<5h;}yObtXF8Rs(J`7knNvg1wdKc&{ozdX~~%m8ZJ^X;)bm z>6bmyqM!FL|5f^#8lj)}Zn5;E6CHCedn;)P|Gig5KL>>9hjS?mH}CzQ2yuX#_OWaC zHy*N&)5DvNb#Z4@YC6VFM-l@ynfLeckzv0k8RZ^};U3+}WU72CD@vl{eY5{TV_;O& z0{UTzj`y$e@gdw@j3?^j^bCWm5*_S6Hr&7Y$Pg~cC^u9LH{^%Q$`;(sjQ=^a@Zh$h zNLZp{GxI;R^8+H#`>L~Nw4>9=*3;hMYVs4^hG9clmlV`!g$*A~^`WO4 zckAsmZKuPA-r;K0VFNvb6*hcyxsMXnNJ?tbQR4LWON>@vbt`Q6=v}wE2^(f-QPqw3 zH)2h+th+nPm0{PF)5{vqrQh8^a9_&Bs+xzkTf{y*%Q6%salS||eQ zXI8M|9ibYr%4B9r0=PBJ)dAbXbc9mt3?(LBl{e1d3 zNYj5uJtX7PPc8lEu;K8j^{4pt^I2z5;c#k$PU}X#L=d%CZf;%5Y!nU4%eirz!9S@JN z^UrU#+woyAa$dXf+ZFh*?ELdrK!xqNAfz2x4q?ORjQ<@~<~x-}#}kwEiv`B(LSOU| zu@|th@!Y-~=io%W>=3SW67?C*F3!%?<#j__YTAYjEU&1ZHE?KUdHcZjwyL&)<$TQ= z(pFJbS&nVA-4j0w<->&fu|Db}oLuK9+_0iXi3VQOg4=2L8`m`Tpz)(eHjN%Xvhg5t zt5syn%R#=ia^S40+V+9fEv?Pu)?D6PT~XOm?r|e0Ba7Mpre&Fg8`_s8E^(U5I4D7u zOO!0F6;&EOBf{{+jZtBUmWnu$*+(FBhXM8+k!QBY(NZYGn05 zGUw`@KD!JQtVaIw9#CN;FN|*FFW*5KHu6G8H*(DEW+oxAgs-svW!AqIhhUcB>?^bWwOHBQI;X%nB{5RA zd5>@sBQt3Qv+7!#%WGOYZl^9z=cx@%BS%afQ9oizBTTTM{!{=WquuaEFZk-;5R0Bf zGrJADm|tu-)M-jwPOIqbG#rY#h^faq(%gTo@zq<6dr3>ZtY=B{xAneyl;-|=lu=h_p{Qyb2{QiX zYyVg?+N-YGY3ff7DEj6uG9Xh#8C#O5=SZR-McDG@~7D2@73a4ogO}avn zZm%w%Ra?{2nks1ym9VI9PVteRh=Hbo+)0`*7TXJN7B%a?iy37*-#XdOKQI#E>rPqt z$Tb@sm8VmGkzkX5AU)C%WqtFr%{HSOejkTN7Y@=A4Npf|nRIcMvc5I*Kh;Mv%7nCf zDq=aFXGkZOqOx4lqO5Pt`iCY>t?Wncwz6c<`_{~VRM8kCv9l5r<>8_j^d_b*VwRJQ zUZICmrYX=ATe$w~dQa?L&vO2mSn4$GP9*y|iKUs?c~d-P(zuG6^2+kM%8Hty6;%}& zo>L#jHBa&riMHJ0<5>nA4<=?iP37bXiDoN_QW_p?iNvpeXXgLfv(dPIq}0%yezBa4 zo8R&LI|tXrYH(eFypJbWVKiH0?=tko)6w*ovN+0!^5cpLh_YzVE*CX5O7=SOu<~Br zF3l4>X*Zs2T%cu<~VItLX(j z5R7Z?{+F{58D+QhxUfKA9N)=oJK8<7y|!v$WR(4+2PnVJI97RVR{N z@R5r+sv@E~fWPpfgv?p$w#-?yc+LbxYEq^nRszLQD8iJb=EPKtD3E}w(Jz_9J~cS( zBb8q%O1m(A^x}n)_T?o3!{|N_J-j?H(i&2{^|4|Ya>vr(|3&?k?^Wm% zv`|1MBhB?sE#=;Xn8x=X1ugtdO`nQYi58i$G@gGl0TeIMU%BoOKLaRk0w}Dk01~0h zVXrj?feK7MekRt$=gb-LIbH^Nrq`oocO$jbU2T~cE#hs)mpXZQ%o`pc#R)@xn62JGAHz0nSr_6B0Na=Y{jb8b>&9Iv`_?Wkb{p zCtGjH63!((tb#s)GrV&+`M&TzOnKmB?^&h@=d(WuI8y?L|KX4U$8jDw+1J)M!g)&> z^8mjK=Q@5t1Z2NivJup}m+reCF7n+EK2=YmbRp0k=;WT;|I(&$*rQD;a_oOuW)h~%;ApBTwQ!XsMk+|| x4un|s5@jk#4*NevRqx6BFV97bRFK>|Xd!JHEKxyd4Yw!vRoodAcdV+X{s12SxTpXC literal 0 HcmV?d00001 diff --git a/flume-ng-sinks/flume-hive-sink/metastore_db/log/log4.dat b/flume-ng-sinks/flume-hive-sink/metastore_db/log/log4.dat new file mode 100644 index 0000000000000000000000000000000000000000..9efa133a9c5c1e9a4c9a54faaa9b21d3ad3d8a1d GIT binary patch literal 1048576 zcmd442b>f|`aWL0d0~NtB}tqG0TF?X>@LaP8CKxPNyRiQ8(@_jjHiM*dw$h3qj>6_ zdY*cQGYgpW>3Qayb2#G}|IbtPR!{eYWk$W<=ihvKXW4mkz4g{xZ&g<-r8X#~3UTww z!JVy&RY!dGkpK5q8>iw|=ROBNOL42uzxnxcd=Q{3ZX!)k8|%BNGj}>nwVFTRZ~9A1 zWj`O4j-;f1_w=_z@Iip?xJl(`P;o%zaCIgA7rXfVp129~XTWUGFfI7sX4OT_C{xic zBsoEqWviiT$LjKm>ZaGgDRN6=gjDo zS@Y(DZc+GH5Fn48aD$K2CBWx;m0j>$WG@wUFC?dqDzaCWeJ zT&=Uf?Jlru1F+O(<_HxnCg?adB7I#7R33GiI!Q~k0qEplHF-Y3g_=AcSpZGW5lt3| zCdBh;lGQbUM@_bc3FFsfG5!y0WwHH(tukwrigqRFWHl;%O}eXY)MR@7y!p{74fS*9 zha2im`y35Y8J`to%s#g<2DKtQ6^_vfDx5> zDY841Iaid~U6di7PnjI{|Ckq1nft}S{uuya%o==zDSphuia(@^wIeuMc00#||dzc^A zsYegH+ih0SI>MGKw>gVZt+lFbacy-|bGuP3TV>S%iG=$K@#g}>%T$)Sl^|9Qg3UCb zdD#I_2SdSW?kp7@O4thJ&Prd79F^;ag>th{ZwyfH3eCTF*Q)4nQlFySwb>X4y3Lz2 zy}Y`tqO7i>yt<~mvaGI*y4UK^T~>w=-dr(qab<1m$f~C1Mn?3;vc{_Nil(wM&@(*A z&uaAV0MAk=f04UHMQcT)C0e7j#tTcGlqJ^ZSNf@|&s47ZR%>l-OE?(-^P_-OdffUC z+NKd*6kqZpfQXxaOn_iHy76P~^C}t=ai7=XRJS%zrU*7yejQ=x2?r*aV1=4Ie-+3uM@CVgqZ!iE5kjmpXD zpq{NmFB8 zEn-U3c3tUHp?n}d(X1Ut2B_D=Q6F_*knp>ga$iV4{-)o%OI%7p3BZp zZWl?U4p!yz2I9k)C3n0MRG1|nkt9pD;yfne9HC6KanNY`B7g<@kDps9zutdB=ZOL! zG8b|Dqs&L7kh!SVPesf8UhAQ62o+}TU|qDY$b4jKnTs}x%tvaOXRBl~7hU1!re&UZ z%o;Wbm9jEFA~GMDO6HGzRI<+prdqcA=!=Ak$lQtfFYUZBiOiBxb*csoGnS321=yQ0 zbs98SMRl@2Q4Q9;psb~}uBw`YRXej0q*ODRQ_XDH)1S0YVt z^~l%e3`j8tQdGLv$wUS^UYCirV>*hpi|Yc^879A=9Rf362|2|r zfMuB6sQdUONyB9ENdRKV*_fjH7GLe_zDkFjWhXy!DikIg>AuDP1QmAQqmp!AE6$f9 z&QT&xvXIkt2S2w`a8PI8dl7?%FXVJ({HM%ErI5KR@;@)MclnW2_WpXMQ$niK|!qVWF-3#-bT0X9_SwFvWt zD@Bzhq6+bRs+9B%;8B%_NLv>I0IF4y5uwD^e`3vZtJ@-J<^<)oq_0Y$%7-dmfr=jI z8xcyb3kXGJUXQGVGS`bTD@7UN`IUJufJbE>4g*H}k5#4#zo1>ZcMrg_U2bz1t7w0+ zn5^8z=_^yBxZDR z0jO@HMfjvz9|mBl*6l9l^aIEOU1(wYYMCxHeO6>zlp~{5;7$h}@)DhT>>c1oo!*RK ze(zo(IyH+<#PjLYsQ0sI!> z7M*&I4#-BG-ipA_-9L*?Eus_gd^+`962PNQkA(rFi=F`0Iz{n`cIp|_pE}*?E|TeR zM7fL7*Qtx@ME7Gmp$aEutFys|tVE?=HvchQmnyv-*&izXRaDwvR3e^FrCwtbRC+uN zm>xvq#Yo3U50!eg24JbuU2da_mJ&9s+{W}(>a2>aO7j=aHibB*S>$9jdOE<38od)a z02*B*8XX`S5znVl@1g)68|jHKVDvN=pni>dBmeh}57g*xH>#q8M5Ab?8qG$Lbn5)6 zvu1))QTQxy^J&x@`M-}FHF`IK8isqdXw)Ve5znvDD*-%e^iR^(^+K!BUibxVv_ppg zEH&EV?yI7M$zq~%_f20T6c^D(^XtRYP5ZE1KrYfgJFE+EqBieESRJ`av{@?J5YMO0 z4u1{cQJW{jfYGH~fa>9}9m$z0`G&NPXRJNPSq3&A?BwyXPFY{k7Pjp!Ky$v7^Y4h^cefxeN0A=0xba#%5 z4kIk)2XoSwgH_T=z~aoK88^C9zl8znto#1pu2<0!q>i%V`ebECRDLsTv+SrV>d?qk9n(sZ{Tj)Pr~e2)bzS%UFSUk8n2T_n96!X*NJiBWZvE!T851=as=tTPn8e zP+vg`Qa%tj6b%LazaEwIuiQ049aW7r=?CFVsv4An5{QnW>jUOJ2S{UOEnv#QOMk!9FfU$Z z{zsXQO(F9j)IZ~7-sy#BJ`pM+^Ps&%=HpV!Jm?6K`8X}}bxDiAgRuS)cB0g);%tS0{N3r%$yZ&Li0HfsBv_l~+$cu0imWDP$5=O}rbQgauX_Op#9Do>c zPDs&vhyK~udrg#dx1Pw^o-axceH2s}B~MJ!d#yO1i8v>UILV^qupWMHrQo8Ls>hxe zZW1MjasE%4PfQ{6Fw}p%xYg+ZQn~Q%W_DGd%)>T_%qJ(2d2hg?n-04SFpM}Sdl6@T zr6g6Zsx%JOI;Ntmx)oa`+F5SGN={@>bW%NLruFm8)Re1VWFt{$Cp-QbR*X6yMc_B? zjci&(oi!45i06wsJK6b~wYAAApD6(1;l%-4Q;}!FfYJK}0O}8)!^a0;ITGw% zEhUh8Rh_;f2(wTmSOK{hy!H{OFO^=Yeoh_%bjc6KR~C*9U~6mwg&J z1j^hl${Zrf5YMN~&esI+sLXSut&17})ynLNPb%|#0G5rjXPwkPCaAjfl|eBF%IHme zT9-e8K6!{PBkBWusLN-OgQ3f9qRYXe3-Nrqj5s8KM_rx|17&!^8QJO7Wl*~?+T3?dpYb{Q$MjS9qnhefunE~=8c8Bt_w z)d^5dMRj9Ub!+=mVHDZwgm7Pc!dRni{-XuaMYh`k0;$bKFiAE?bN=Uz-6+FlsZZPu zvi%Xs=7#j;U`eV$C^zGig4`9L&LZ27E<9rwN`e~Cfk|sT)tMF9$dmj;qcS`HG0L;Z zw#7v)-lmpqCbdi_AMtC%wu44xfADLhi)`Of>uhUV!bvVyb_ZaYTAs{Yjw5syzT`y! zEAHz7g5^3N$6kyMB!~uhQHxvFG^P>NRy-`wPWr@b|2v^94N$&ofI5q8Ke$*X*UPGC zfaSlw0Sr%H1C-wo;K|(n0T-!&u5Cf1b(u6kOe1N4ieUU>ZvPFnHcxLrd%`WkjmD}7 z#y`#bf!dEqQdtv#Wu>i?d%TK{7Oy&9OVU!?+=8UM{p}d^ zs{cX?QkFSn<=Fx1oI^h0z9MsF%pqS%KOAQ=hols~Ib>DefE4tz2f1@wdjhPzHWO>d zbQEh>1@j;J*|)TV?w1fmw~BQ1D&{|4NAHVrmA|9Qe5{J~FV4qKPB9;=!ur2AAItg| zd3@35_|Ets8>w1VGymrt@{}a=F%hS_O2j!u;!a==X$dV9LZ@m%#bl>1em?*=`cL)Y zfZ^b9s@flGG+eB?Bg?77m$^<|R&K69TIj12pzm~?S`A$rgB~cfZT=IZeV1}-@MT_Z zjWXZ_?PI{l*zx~VwNi_s)R5SKrlEQYq>cZls?{R&-S~zVA*40N>*HNiug=|4~rOL>6$ejD_9yzVg3@UQ~51j zbD;+bOKbIi#wRbIrE9hUik3b--qPZ(HQ4{;`MoM7`dROPx&|KvkoKx&`;*EkDX7#A z_fsJ~b=$M&Qv+Y`spa|)8}*FTy{DG?vooKeD=?oWt@+lT6*&~S@f{8TP=Gm9a%1B8 za^t!O0}^uD@=6#mLx{$UVRtER^7t|U%VGPrF7hzt4AmtYKG<8AedEXOOt=4@FFa+? z9J?&Qm&12A_rus@IUni)B`qtm1Wf9rECb3|#{XCwarl0bFoD%UE6LXZSPtKv-4mqL zfuZdLElG22RdZ8y>zH=ilf(D-A_)gi*mvCg0Cg^uea(Thlx>0G8;?9;SWZjNOonet zK?y|1ad!RJcoG-N4r7^Hr!&w|%G~MrtY1g5_W0U>jvT)KOFQT!6hioi@A1umWsg5o zkCzwG7uKVxXG?da$l z8PG0Ei3a_(f5*^XAMCJXL~egdiDu_NF|;=ZIh-~V?b1|e6RAIYQw;6RQis(8qHRuz zHY)+`t&qc75YaA4iN^YWklWh>9o8v{c5zCyI}^~}8Q`#5L$pmP(QN#S>Gp1ahvjOb zU6cxKSG)clLwm2E!{RW}E=-BGI6=tw`#M~bBiaQi(auXi`=F1*ExSZJKPB2f6VN`~ z!QoaUqMert&9(VYOt+7EJKVTUv~yFUO-$hSaW97(aEW$KN;F&ljdA;=r^Bs=MEhe( zG+Y0Pp?%uJ;hsyPot+ZR?*ERVeb(L4TMU7ARw}gmJ_)*gUgB`S9l4#EfF{wU9_xQz zww1jXpw*J+PdWo11dtF=e_#Ub|8#To#&yvCzXaMA&CY*gntajK;ifr?bVf?Fw-UI0 zS?qB8Ez$mv3N38se=%-fb#b^8nP{h{M4Owy?d#4CpFSemX(`bd|AQgm8^H7N4t}IW z+me9xZIKf`#h{&%678D=wEuQ=!Y3QFlT)EZMkSzq*TD&&WYA7ZiN@z&g1UWQ=!8!+ zXeXvbv-y9l0e&cO!Y3HC6H=o6D}mdO`A+zFgLZr>v`HNj(0W8 zPu$W{yA1*qTk7oimXewLZV>=5=Q<}zsx%mcpv-Pdgw;7ZRbq1fQof^8_(FVm^LT<= z6svy><>fp_mjr=wUQ(3D61cpQ>*&2AKsh%V%4C&*@@kHwH{k*0oTMm~2`I0jDnTCu z${&-Wv?iduj+(^T2IcIeDAxaD>c4@C#90RAtN_Z6_#=!w`OyR}Z=xQ-5vxNkK0F-4 zi?>XP=}&K2gnK764nFinDEb1ZPuba{PH$lozNKLT-ZEu2py(~qoDoR`E~ zSSBGWTmL*ydkZHQ-xWD}n#GR-qlnI{iD9uG5dmXZ-=lc*9Uy@9T?(wJP6E~s9UNVe z1=fF4VcGg`jMtB7{BI4fZ&P6{PvG@af#ZH-u)ax!^```^pV4=|HdtS$!g@X)3r8I| zdcPy|`YHw1@9h2$Pp@nTJ?Klr>&sMFcKtVom4#09g~9qF71m)15@w?xVJV00^`BH& z*C$}*peucDus%7>}M4pcMk2fNB z-pX%x>9CNkOaL;3bkmO)4K7Zifz-dJ+xqv#3aw2IP|M~IQ}*B|AuY7^e+vzD^Nf<~ z35tu5(quF2{O4jenWpW)47N@|sIIsn>5Ot|jEmzN1J>05HviqEmS|ESL@^6+56yx9 zgs}PMU-*w)npqx>6Pby7KK3vz;1?t&wwb~H&(^&piEUWIr&JFBhK_UzZKH$p+c>MJ z48JR@E~Q{ckmE{(RW!6 ze$5TB+*LQGtg5N8y4@Al5L!qd#u9=s&U}rR&KW=&d@guWruib5ZU0TG1%Fu$ihty} zFZf;6IiX*7?n|TLF>8WYeUoCQNgn5r2kp392-eAl`uut3uQ_Oj%-Bz1>{);F7{a#{ zE2t&zM=DzB;D1%h{U|E~TO08VZE>~PcK`F{B(7F!WSCtcGHlkahN@RVUib>fRa@}A z=WGNPXPa&F-_2e^T8SW&#Mx$_D$ce!-q{WZIp{w-s6Tn|BjG__Z(t19<1> zv8fObLuv=?c8lN_pW|3bHqDAj}yTXpGG`Tkvpqn(Ud`Rk_o$ zf2k}*S|_p@F08Hyu;Qw~*O9}pDsU$ozDPw5%POT329Zc&usN>=5NWW2@SmY8@fub5 z&zacEe*l~@8QiTGOon|Ykty};fZpZMUe;jy;mF&F0$rx8X*ceG~aTY<8E}?Dy7YL?qHiBj&u41e+Cx|03IaT2%Ninb<6N z2B5JT)ZD(KQJ6&{klp_kv)hsG46z#?T%VDN-4X>VqH>y-SZ;pzfB|T^ZzG4pa(9d6 z4!4#gB9Trog3g~2K%}>I2w$npd}5gjUzv&JlAZ}^449LS0VxxMBph%4+X1=gcpKcA zV!&z2otcRNQ_59DRW(bo5&uy!OOk&ZO-F0k_-)8MpY$M989 zg?LRWd{riP#DXJF?%@-!#(KFSMCQ(L!TjCp=f?Nj4YuD>lzojjyYCp#gXY`l;($QC z%d8#;1ZHBsIB|-omu4hJ+x?6HGaBu?$dNGGRx#R<)@VfZ8EyB61Bf(QQTS?k1O!IA z`WG0Dr*IgfzyjVz?o+aHDVg2E69S5IMY@Z7qMW&c%?yjQL9Lm(-yIYA!cwrZ-ve ztJ!AM+>7hKu~}}pLW>yY>+p$YEyjnwIKBHtv)or(59V41FEZ_c8}{9Mif@+daNJRU z-0`lv1weY+-mL$yxBV$eZEQkdCeXl{s@xy{r-7wBgxP)Eskc>_>W08Sq%LxfcIqHzJ`^r&ThZKd@ccto zwctOw;rVkL8t_Zb;-*z)&S=2UD1`^K95TIS#gcVPy+7r(t*u|%6a(qgwsygaw$Bz%d}@)emzHLI4yjp*f2i*e(! zBypn@JmAmKheY+u^eBx}i5=zc=D1qMWYhBKcOC+Qf!>EW1V^bR2?mRp|1lU`79R}m zH@p`G^(Sw~-3{NdOJ9|PUl9*r;FiUeRb`E1+I>hdgMlbTOQLi%Tl~|UAKl*Z(Ykch zKxZ-+HIi0swCmrxRS_4m)gRos@(3APb*^dE^pxVG4zvhqi?cS``nRFahc|w5@feKu z8aV4gzlpONwmxTV{ExJ|Vaw+?wz@d)YPdM-{+T&zOf7NN#jJnQSuaoGtTM7lNE0^q02MajpOZA9WLnL2fMKTfXT6KgJMG&42f4(BU|8sV+bFMSS=*cvdv{OA-yl37fwM3~T0p(V3P@N_qT| zbUmSEcpNm;O`U1A?BPMfW&63H$$>hF%eHI;BwOq+@hvt1NNCcQONG>*88|=+geWFm zc92<5=o?s1a5bJ;Nmzj2aX^+*XXrnw7VH0-m|BDZ_+4xYHTA&t*c6(pBWHDWU3G2a zn5y=6zJ}1kompAnF5y`-8)nX*r-SMI%wjLAh7@i;BVg+vJZ){+{@hYbgYd12IHi0` zO&n8`iEKa8!0%ZBYhea0&7w_& z2S87qM61AGQdHnb+^dyS5~-ChT@sLnkC*N{P9A7-)i{iFh^Fc38YfE;mC|g6;}fzi zeJa3;wU8eo8&M0nhr*#&zA>wmN*F{U9c2*PItCDFvCiRZWCMAt3SX0n#aL#IctUFLSS?YWBI@QZnC~CdpN9J}auf`AuNdwqYd9j( za8j<^e^Nk98m>$DZ*Tt|~=iW4;=EwlUoVrE-3(K}#zVkTxwB2W>v)2yVM zE(^v#n(pVwF)-czV!C6j>4-?A=R{77|9aRqa&qF{(fvu5XYNq$PX!qaKxs#v0x-wb z&&?oG`744=Wfd(LQQLnu$ml2o#B#g;&FIOBV6l6Wit01`F+iM@rA0}HFg>D#Hqryc zA^3y~F1PbPL!S?s{?mO`URtuRa$n7)#dR*c{V_Nu2!)tq1X*!(ph4$g5_*Hn!(AuS z6QF6E(KNl`!D>q4ZNlI!Fo>n9_=bdOW~QoW*FC_byb!J%~X}NX@)HFz~!sf zuj2k!rm9!Qr>asETxHk)v`c{$2r*Tqxqr)|pw=HWDPtC>$BKe3{z7h9Tks=L6kLV< zPgC(H{X)aO|K_)sJT-X)KF}7nD7d<_*di8Nirk;g0;Yqa;Lrcl*fJ++ajLv5^#7lt zAg$YeQLwCROjT9mnD&p<2owd^1p7ZvXn2Arrs?A}(EVRB-9H_|CcdD-bnlir%bGU> zjmo0nPcDwk*GD>`&ax7LE=ehgcF!oRku6!uBzo;$V9S1emG0MGjM3lE032fQTD$-6 zD!tK_q3=e#BMWENV$(K0%oFzMyw{Rp?=xVLuqFPY=p7?|6W)=8YJ(=eBs1cl28pk; z_5T?0n{nc)+hmA08RBKFwN(|1+g}EwG*_M*4g#%81aAX>fjXwTJXi0(S8G1 zq-$w^o?w+*COR%NZdiOjO}nb4xv{*W(qE^nY;LJBZxU^8s%x%nu7#W>?xR84>uvmt z$$jet2gjmo+EGIrrS0a5D*w68$!R|sNBhbIxo@j-+(m}=B15~Xs-n80%5U%Fw4V;r zJ}@_d_U)q_cMn5*4@0}Wy0*5g(x1R2r+r==?ZyP!cZ_sUN7Xi&YiO6XG}bm(S=!}` zYev@8weYonxE7CT#;aS6y_q%m+Va3V6KLN#)WI3nnzlK=y0WEdOj&cKJ%b}P?e#(0 z2knqR`>r95JKNBnt>sReHuDUh+y|{spndm_4tAtyxo2iZdu<%;ClhFI8R)pX8``@Y z+Rcm0$}4KEf2X#|$wAr&k4&Jwb%5hG8rqG9c4Kp0Q(4RQ-AH!#X9}6>0l^N}=akSeKXg{3ixUGh^IgYHNsGNIPX6Ed5!h zO*`*1^bcqL$K?exm*ApkmOB{(%oxD&H+z$XpW`_R z9G#sVobskQu1l&T-i4gzd<7QE)n^;|IBqCR;FvYs!LcQpBaSRd@T_0-mS#UkyZ-I9 zqm#XpgHx(C$2CbfmepnWevo?5@^kz+LC2h7j=S1$T%D9-hE{cjVQp>o)JSXU-~pW%GQ5l3=0N3*hHu`T%LqsZCwVh6NgF12N@o1_rJy%77lXo zDizIeSyF~o8EH5!j^SAeZP=mIak+SiHas9H!-|X;a@T;*q{rC#UrfV}AqRt@X1IR> z!!gwrn3!earFC%(?f#D#!y-&lSg(bKZAloG*Jai%YU3Ebn_$vTn7Odd0)|VIGR&-7 z)C3tGyJrH!&izg81q_!YWjH1yla7gF_)r4FF8xe?35KnS46Dk=WY#UJgA9)wo4~NR zuSq(=uq7$O%-XOjj-h@2Gv+j1`PP5n8De) zh%^<#P;%DePX$BftXIe9tV;lh;pF(gi9|Z5Wa{ITxHvdnA2*9q<{{a?MJdx{Wn?K? z-w8JVF>Fhb^=(4miH>ssgH(B`$@CMpdTD+1R{@UkB^WgG2Nc zA}CG^Bo4sz_9!CjpR2d+MDxle;G$CAreo4+GBzXp8%jT&5+ zL<5F1YVZkQ*psi*>&T1nSNNR1IDO_tNP#}hj(-*!>ZV>tKJ9lNZLT9P0-Y4=$fvCX zB>Tm6@h!$wA7Y+n*MF|nTOJpY5XEG_NV?^5k;aSL@_2#%qdM&kkxLUpx6XFn!!rk1 zWyJSK;+cb7-EV3e>#DIg-Cyf$ZK=d@1!fHbOAr-h+)~q<+a0Gj%*UJ4f1AyZ6q9)< z^gf;AKbz-T)q=n9VAk6^s%S2LS2LWB`e4>n?9dxZdz4TBlH-&hC=zz9kqPV$6c>ah_UB73->sAxIc2}#B$CQPK)PDW%y`$(bx zL4ZDYsdWusCkKvWm)doO8TbL;aPtdxL{gOfj5uf5{ZGa%`F!ZnSh*aL- z*r;}WCRXFSL4M(`NQ%=aBH8+1%yP%@xeRT&20WLMiRF@rR7CwWGui<*IMe1oG2=NA zyjIhFP>gq+H69U(v_TN}O)y?b`0ubc@tRfm@0l3Sx?H9^BPomc+?lri6|>y2SgDHY zx3I(K=4E2JczKGbncwHm+B={bEF^ zu;t$%*<4-GihV=vzi%gqY|qsyU;OI?DCs54MS%0|_#YGChytfv3t--L zTUS|8Tiduewt44Q3c%h)=HdkBZ%$}}Bl8^=KNmXY&3APb<#=+s$u>bnS?kDz%sr!y z@NeNRCb+=Xe_|%skgcmldpc#h4Qgv^>#Az1ZS4BBZP2Z2&;%D;mSBR7Sx&V!fqBbh zZFNOmRf{J8O^~<%pGiNsTTDPn&<`#g3JEx4yFPx#c8~x}77@*u!#H>p-nOXcSo`2@ zix~_GX^I(wTqa~w1errz_;^5!IVjUz&8?n2IrL#jSZRiY^hPvGpR~me7mX4-TpMqP z9>xwA&GOCEO|7BmU*~Z3jR16QSo0#ze>t$+kYuLbCNSp$hP8$pbgg04|Fc@drmj8> z^0?tn^{oH-WqkNF*fd5oxG{+aQa#vY_dndIYYot#y;{Sj6F`wuUki;EOI>T&bh8J| zS_5gPs5NYQ4UqKQ8{=Dy3pQwsiya|#gRV8K^4A*rna3-81Rk$6%;r-ak~mGsO!1RZ ztTMdsZbD_5p0`H1n{ss*sIF>l#bY4Nz9BOqR)rKE*3$1^VS#~9+@UZ#-}E3otdwzQ zWOK#H#g(VzMjEOn+>z@=O5Lt=S!Zmg|A)z((S0r2R(5_tM@$R zr>5(t*8oP!K>1+va8ECmCpanT5xx;;akGI@KDg0LO46f`xzkKGE?b#4f%~R-pb}rj z9;jsbKr+G2w*-bnaHJf{%cIjnMh6qXA`-PPsRe+?pMBApTVkf;L>GZCjC1Fy!oke)tvCbBAn9_OElRJPVNVn>$E zsFLRsftkpk8Ul?`j>xqS_SZ!7sV9b=i1EF22-A)ww2}5}hF^HcpHTnR!O_@}k7_;X zz7~5_>$OapJEQTHf=~$QWS4RLW0W?6aL?Qm*q7iPbaFsuoh+uPjIWoS2KtQFH^fJ4 z4nDBNW!L&fC;h0iTKL%m)%cPCGQM8+G^lWVy*bI~BtCuFx1!0-Uf9pdM>nLxo$=17 zkS;dkKNt_p>{>*;yj(=QIfaOq&-IIlLNne=e#Epn`0$B%IiLTeh_@sWu??`$_3{e< z!xZ2aodRqyqOc{_;l@?WAJq9bUT(OeyHA6iipZtQxld%{!>7R&b)vzoNi>iY;0oLS zZ`CQl22x7B+;GLoph)Mk(6D^!6yS&WQ^}s>772yEjCq&x%Z&_e=8v3dSwX)chw;IGY#T-5GC^#kXo#RulpZ zGfE525iT4o!RaqcJx}R)WFV#H; z-e>$*8c)Ac@ zlt8@qM8|#0(0)tPUR>AQSXS-T5K_}VOXynKcO=l>VZ4Jk>S?(@)U?xj%N*rCKS=wk z-U+n(jC0(#4ehrLZOpuzs+-&0x~$vu>^Rz-|FEOS4Bi)eqdqXSf5A(9X%DfQlFOl+@Rc7bNx3ccmJM_`=+7&rlFnQ!+P2JtsFkN zueS4_nA`)f&Hf!j`yEZYtfjTCs+#G3yDzCWDLHA&kahK2kej3GZSet+qY`?+U+wyz z4uCp$9_W6K1kYuVLIU%-8Q0SbQl>AP$GeXa2CZT_!+^vDqosW<(^??{Pz|>umi$X0si_ zwGs3UfUn<~r4yi-<(xq#WkDerf=bU9?a5R%PD%g8IO%nV3r&HMZ27 zo62I8YC|b65k(T8>z{)-Onh!nlKAW)BL5&F->wrMTJ?8$H1}Wl7EdkzOP97=vBFYS z){JML+Zl5TRX$pmCyc`qMPa5tWU;4V^Y^9%Hq-FD&Lj91nb0HYv+=pQ=F=cNET2ab z_)H(+xQ`h=k7dk<*4KO{3m+-H+)$aoX9k|lc^pt-e5B6@dk z`bFGK?xDFwgj*v%q1#RK6Lg!4XKK10)ydCa_KGgDYBz$f&&+2+47E6yM6=tB@NHFz|A zHZ}rlCcA++8Awyzd`5yU3-GkfQ-FYDJe58Z;~4dkISZ!F#b!OR9MfWQl9+Mx&k5Xi z$D=lP0RpkQiReO7p~kUAy$c|E7H<~Bnt`!zm z9lQTEX0(0r*vzwlf;!Kp&nABOJpdHQL-^cg_y5ND?1!gio;Q4+&zO%l{4N(BG9=z+ zpZ|;T!6J|Q0-!MbzK}kjWW(Cwf#LVL^w|W4-(|u;n(8h)|A{eK zQtY^|01A%rO8QKa55L^hLryX*-gQKR<(77F+frHCAKCj>zw3^j8VI$47h3kK;u*O&|@8Gyw0EK4SqFLcNp{ClFYP;Vo z`Cg~EW~%=L6nxFJOT`c(o9?TOW?Dzz;W*Z6q@Ot^!X%d zruxr8#n((*A5P%4D&KLp0t(HvHGN)*%~by>h|n9l;zs7Tts1OYP`*9Bpe$qAw%tXR zScPjf)()b&c7zJo+nLmBB=4Y&L)vMW?~&1Pk>;KNGgbu)Bd4Kc^G^o=sH&c3OEyF# z($Ruw>mRWSSKlyFa{XWT0a zG4b%0#M?75UVO@8te3z2w@XqOP7o-N_5YaZ*1GSC>99lp-AqiEOrRpFr@4vk?i(L4 z0d3bY@(0-NDY4xjtnG+Mqz$+s>V4}2h_qe*@EzD_M?5?iaz`e%Gp+2Umd11i+czC$ zX*BZOA6)-oA#k1ho{E-G9(dh*nV2q#JVjK_?{)Vd7%%`WR}?t|miw1j?hI==BGPil zk9WJ3xi+%*1lZ@;hTbP)S2@r223SZ5f#-O#s40#>%Ye1X~j;F|AQ5u7AyXrwIUITbc~2=pMQ@9zJcMp zl+y1Wz;fJOnOHF)(K9COkivu%&07?qJWWQX2Tx2e;el8ZHcxe7Iq8E;Oqf)rBC4p_ ziTxf7#(&zcbL33e@84p-Gp+rINTef-tPgSg3&y@s81;c2h==8*yECz0Z1!#p*D=+Y z0ot)rKPBPd59fO*IR4T34#FmYsFuc6?uVIJE+|bAwbFdVWDk!HXik%LiJS$KJtHPN z%bJXcK9fBh+yAiy-!SBrhPNoCPp*K;wq$NH`!-CB@tK1BASwN6E zXvyE>!D=H2c3E6x6iu(4{ED`L2kAcw`bsB{|3M3$fjtqzoRtJLY%xCc#hr4$;W_t@ zH~2OTI+R9zwD%R98u!30E0T}z59zofc}MCEgOA(!-yM3xAT(%i!{Fm~{Ik$dH}!_W z$3GD3-@$V`XdM_K={fgLVE?0@-rNyCy*Uj?F!U3F_zzMbl&9(8Nbg_$^2>7X<(FE2 z(9Oo31t#2YxEFrM)o%lS(7&=@!w>Tq0*y`SQ2&~WU-e1%xvcZ5&}(O0f)4;VN8^(~ zcK@r1fDA6h_29!T%gz#2pZz}#E@y<`6$!7uR!>n{tTarcAzC(|k)l!53{Nz9BR2)tl(S)BsVUkq=}v)@<(_d^U%~!6)th zSJNcrD%%-_XCOW2&6HV0fZ`I8??2Qc(`4*t(Li7?uq^l^C9 zD#XDQt&&mHBU`c*wVn!&e~fyEQ!V?xK~XFHsAtu>j(wt!kZJSxYXBIkSHoYg6q zMneAbsv10wS<~)jMGcsqASn!mM{j0pDtIDMU)z0Br-5x{E-EBO` zcQJJkUpUJ6LV8N^#4Q-a?+gF3^?yU32Z?oe4~@Mj_0ZqM7Ytka!W`UC^XV1-wsLrk z);2yr;5LX}2N=E5YXhqniLVj{PXrk7{f^tlp`Vld0(*>6dR9xbs|oK}hX2231^Cw^ zg52Zcq%{2`2wv~FG?RA28Do&$|9Z!N{|)3>c)KNjWRlqbZ|LvEe)Ifc!NH-cLB%%` z|NBc&;YfT}l98CBH(Kf$?*C;jahHDhU^8Q-@u2^4F7XW3|2+*b65%>w=_%*q!>7SB zH;V>$C(&R4V5z|ifMNQ4x6UP?L3_ExvjuU`P&aih@$486nz;mNr^qFq<^FHB*xm6h zc8U@6*^NT#F8%PqW)iZw#Bce|9IZcCGG{Qkgg=vjwkm52egxi`^IU&lTV&nCZ|g^v z?}`s*5{>wTou3Q)6;RA1y4K_GeEXZzpz3q}rpoI zfAbP;`bDx7#(|756=KZds_L4u#=sf-N#;x1*fKUee?!3F%o9fA;fd%37@T>+#U{q2 zLpX#6EMa5mv7r&4`27XD|3P~sHl*gOqsK}iW;Zn!BSR94XOtn3A(@FnFB}t4h#A5W zR7jr>7gH$RK_RA4S5e6Bf4?if4@%U05%o_m=9sAY)|sDP4s2h{d2we*$*8$CNz9R` z`Ql8#FluhqQS&-Be=KU^*J9jyLi8Iyvi0HvJ`M5?C71rc%jM#+UxSZCgKbGPkf`~R zUH{#tqb4+HFKWItQCiMIL*3L-^Cdg~-KJ+?q@5yazVt`XWz^gj-(odDg8RH=*FU%F zsCgX;F>11Uk@Tr;!)!jj)=zDpj!%;LN#xgL)FO1H58X@T0oAq2y(Cxn(dH_ganV@g zKj!x+5ZfH9fQ#*H?3!A1l$Aq)=+Q5iuXdf>0-w4oW<#AAWt5SVQzfpy^7G}`KD38vZp6O%T6hvC|)s+EZ zSgSq6#an-Mg6iU}zlg4A#PpO63R6U-{DsF?UkvbKt+s3Ak4Uzkqvlu_{bN=s8`dBW z$KRyz_!{dU%$!WG;E3L>K385&io#=Vffh0yPs>0Xc3oWkuFC+&UwEu(X)Z@6ZTDye z(DxM{U$gapqh~p6aS~o^;*C2CjpwC9n073ojkH&@<_FJyeV>3GS$O=`U9Y0UXnzzQ z*Jsk$X?E?HAil!m>u(1H;Q)IidUJFT1wj|PB(p9SQ&a}nH&Fl6;d*O)xRwF-jaJ|A zWFFi8{E=SAgS&5ddV}>34zTwo8J_5#FzFlDiYE7ZaX-Dsw%>S9M7%eJh;JhPO$8HO zH8VNDhJh!X6%_H!Q6l2~NknWjBEE^|zw4O;+^=(hE&tDQfHxoUX;8cw(x@l%U;aBj zd`5orGtuCIBpOH#@RsfW59k~K8nl-Kyk+x$3k|1AoddkJ3j7&c9?&@eX{X2m-Z~$U z%mE&VZ?VZhf-k&vyO6qH`@$A~4zP=PYMkQ7wG@t&m?&!ZjgPu zo1d&=PVkWnYwBf}gOs~DR|lvuEp=s0jkW&w_$A2+x~XCw*jgW%KXum3=$r)&3mQTw zdi;{(Tqxcm5%DHZ#1{O8WA!~9R20Hvxt<4;C1IOUZg3t5EyB%A094-@;_wc#l*%Db~P2IJO2(F*p^6o!>n*Aj)P{=nQpn(28Ek z#ba(KngJtoQHrRL|1r0BZT(O89mU+BTjU(%2G3J#1ZS62bTn-q|t_k?^9|Z@vuklzD$gU85)e{J#c3n zE+>`Yz#-`#y$>q`BGGWexn8Xehuu=AWMa4^A{9|TT^|wSeRx@bAw8}~Pf z5)XsH1DRM1^E38QJZ@kFBGP!=yah=={e1wD#@jjk5MFviJnV^jC==s(U1IKh<81{gk`EI1HsDbb z>7bu=2uMS_jdZb_)ppQ7WMa2CX^NR&`ez3Pn9*iCM6f9Az9u%?WNk)7B5e@F z>jH?h*@!Tn8s3q3*gE@gCN{%UBj})W7tD;-&of2?cJh-6NPa^pFaCmiKip8{b2lIl ztv1TVPF8I-9I1XrCRU4=r-+(qPGYvt{}$jzv-OFfQtrMkX1my$jfg~&lFsM$`4MF*UcK!E3UB!R~?Nu?pvg^MV8tSI5Vti%Se;?FU4AM?f#rSFw+KE++2ji<4 zvi|#(UH{cRv)_{>#I~lnf6Mh>tv_gHiL9}mWLxxKlCA%K?O*@Ry#;XUnx7B+BRMk~_ONs|o+WM* zOSJ~<=w4cdR}Ymn*0z6wxiPpDMZU4?U&iR=@YQ|YNiysp&P_7iKOMp*&Y{6{?_OdW z_1j*7@KTRqO}VQj`NUqa)rfORwm{MD8AUa+B}-B3+tUJU8Py)fi=DlS{DGN9HLKQr zqShw?`rIp)>rTbwnY|5r#ikT6#)xPZbQ}_l55U`;DewI@!ZLOneQ#aM*9+3x=)F7rqP3e;n== zy9Y1bH4lWq#g}E~3o*4;8DID=Q2%haRqUSeh=R1*0yRDpqrqj;>1iNL-m?tpvD69+Ns% z__5KWy%MSLYoNzrg~KqfKDfK$rW9QNVv9Wz-(s@q;m5xUsfYD~$7>`+F_~s3-SqGp zzTmELTqJP;Zu0rUPeyTp<74+yv-O)V#JeAQX}e5n>_#H3A>!M$Xmx_J50})JOPCq5j^|)c8pL=?I?SjYs zlph``%)k#Y;h%}QbcP4oQ)>*{~2vJBWeuZ(R+w($LwD& zOA(dQY=#Rf?tf)v&9LGux^D!NME4D9j>V5lvP#*=29Zc+7VS7A1BjeO_ozqj)9-V` zEV`gTOPTh@i1NyGnhp$j^?NM?6ZmJ*%`Me+=*aCKck7=;PnZKi zs@&^UR6kCP9OL@Tx>!t66VjdR{h%mA`or-dog-9^aWLJK-k!Rifc@)KH^uWxnJ# z(JMPLVJhX?I-1PjjKwGM-th2n)=i#^q4!#cFoSYJs&GiDdAaxy6)Mb)@i*@m&jl2z z?1meZa(D44L4iVMY-0vV9y~$~%8sF{UaH5sHH|CQwzaHS7hTr4cHN}rW%JvXw@|(k zp*T{g9u=T^y`O3~H7L#~L?&WiLB2bH%p`VKg3u(q2xj@%=ijvX3pWF%72A4sp|YDA z8{<=Jg(Tt-+)zDln8ydk*T_rodT9Mm>0UOI%o(b%5tYH4&55U~V(C3;6Q!R9>S64+*MxP2OuBI=zPI?R(8Q;6vej z*1tGLXxX*bT<4JX;2e65T&4Vv_=-;jfRRiCZjx&9NVOo#VgonB@Y?Nn#(wIU>Z+&e zzP4peTjR2}LmSt%tz0pxb>*7njq9velc2O_zODb6&h)yUXku%I1W~*u|EqvRm!M&q z0pcX_8jNVx4iG+%_Va|_E}w#Z1IUOFBBy#UusT z|5R_945Aj4`blh8)IwYT)Ar0;luT6NQ)u&l13D)?KEog=^enXdpG>wP1yf;IRJ$InBGbnh^8SQmDZz zu>m?v_j4=U>L<}nO^8JpFTB&DB5DVl{~7+X(Ev+A_}cH2R@55G;YLwC65Z6qI8ll06SZSifImfD3Xo*or(-k#S_(eikdZpLi=%bC zFo3)(e>7LL}D^M9{56rLf--D%GX4pswHzowNd*0rs`5H+EGZCm5$rj_egtXp%) zgsCf<+Zw&7q#LUQSR}loAzozb-^P5KL42tl4hk_GXp<9%@Y29V`v(N#XfT5y6o?nW zB<2^L?FZFE{XOmyIS~Mv!jic587Qa{^NXJL&>8a`u(;RV-4AO1^pY5h*N{-tePz}ELjbW_!FCLl5q#n_#m4e)2=-J2kOV^`t-5Vdnw0F;{*58$oty1?|_CY)wvRph_I_5#l}DSUI@ZX!8T3|06E=MN0V;o zRF(fVF6a4B4gW8sqz82gw7*V!QuQFMLR=3jj_J=HM4w2j2NlosRJT3o244^IvL^N* zxnf#C1NNW>+^}Ntc^;`2$(ma5Mp1nz$j*49s1L85tciM&ptNhTt^XT8eI0svM7ycF zxOOG7&wsj(2uO4Zny7^!N&Khl?tW06HD$o(%m6a`LjoTOGhOcrfUfrI(>rJ+Z_uX5 zzA%F$`F!Jq&YIE}wcA8La^9k3qKfIe+5KPI^m%jqqNXA#a1Q3@o%f2YGRnP;m1mE_dl7~#WopF5Wkzs)VSnFzw~)$0A}}c6&*p)fvVgu zJ+`KJ*@7LcNJS*>&iTKWZ4{LIDRoojv5uv)iMYY4`!RmxybX{zVX&$NFE~PX`}~LT zB1*hRg^KP%iTkSxzr@(`CJBulut>wo8_55(ydIrysv=I_xB;t2Apg_yrW&w%1oFQ; z+Qsva)GaAlR*yjbrw0-*11Q8mqO&YEE#b`?fz#ukep}`8a*GG{obN}@3!y!y#(SU?$a)?LP!`CZOHqC5d5N$3q+Wr9 z>bT+sZ{ovOAnSP_sIWlxW0LAqAHc$MdcH40{pg@C>rQ=>6Vhuh0PwBDIK4_lrXS;F z;u;o6^UFmeK1t{!k?AMC@1?K|?cyq@1YhRomZ)r#L=aX?*1Mkv zzf!|9A%xe-MW~**LAkfh|9-+6jiFpGP=pXodoL4Be~Q;s4BMN}KftiVog_*y0+ZGC zMkv5g2~NJ+9H$gt<~gOJ13^OqF7+QD`2Nb_m-)HK!xD7fq1Yo&ow*c~LgyXEdXUta zf*b&7UOt_7SOh4vF8H@nesj?vqVo&%Y{|8)|peMnWo`7py71= zOecT@m$uvCIS++;&>vFh(MR%s2tYkZd={X2`Sj?sljxD1LXSREMUUJhda!w*N1x>) zR4(<%S75cT<1o4wIQ>mNh~Ei3NDww9And0c4j+ZN{gj76Xwkcm#J}OW+?fRysrAPC zH-{`hLT@chl;F60diQ1hi|v=2Lhruy0ATy&C(+xiFFSo(gjGKE&fNmN$0~#6Q5G+4AVF?djZqn&=P%Z&&C3hk5YO8RS#y+~4;9 zBI>LKNuqOq)<3CpQ3{>=GyYTO&Z%`CVB=qBt@EPo=sZC3Uy!7U2;Q#F16GR8o$1?v zPpR_&ng4+@btdH`IuE#8bncu&=K+lW)VVl`&O^Z-VQC=!pT1qJm2ko8AI7&kVT|x* zs=0tBhQ+e+KZbRSaq&)Y@dPaC+yhs6yl4Oy^inwbK&k&gDjI-N-3n-4K1Uz8MGR1! z!TViQwlErUA*uw7_bPY?$;FM@T&4Mv0M|BSSK;#HNDeK6`j zQvs{@)@5_vsYdi!ekMMADf?i?e>QF?N#in=Bd3c+s1O@BmqLZNllcvn`9H|QNr8f` zBgwP&!9RG!(XN_!1kk*E?YiS2Q8pB>tYok|MnsRDlC*0G#GuEHON7-<>T_W=32y*0 zwp~w-xI(zaId?qAgSSfbhwfPf!$l}*iygO$csr#M?+p=eSQ7DW1|f(yBv054*_=mNjcfAF^Wf zlA6_(2dybvF{ZYzd|A^02OYSgtbA=$>^@W2eE@;Cn5OJLb@&?0hcf@?AUcev z259W5w(|`TcHo(!ACibP=+OTIA$G1|@eL}SYv>KW&V_AbxTa_Ir|0@Q*HF%X*tsf_ zw0ad_DYEtd3hqzn$3vvDoqT-PFDG9)F>7V$$v^B+pXX5XGTEF6e6)O)~p-+07z4nl%rOO*vj9Rv`sS$S4FA*4vPsp^B z)PF^$GmK2;G^2qkKRc-)j%G?blL{S>lw|!wC~fvr>aOyAz6(7dInjx_ub@BR8HY_t zB-0-rI7qvzISByB#N^&fg@b&570zQEV! z#diZ!{o{>%gc0);sSY;?*S)Z=t|U5O#HRNiE*~rTB_{WnvEMirT>Vk5YA|AvHn` zU%6(`s|(F91E+Ko3gQv|Y>`nS+mkeeL7}-@wY282yb0N-z=wMy2 z`0B)bdJz6ZEV0T+JO4MBPr@o*?%Ex>s}E5-DX9`woOI+#U`(q_N@A4|;3%A|e*=b2 zsvz8=0qw+tk*|m{lNb*eGgH}RRDsxKQoLR24eL?E{dUo;8`%Q1%g~$wD%Yo8X$Zn% zS3^i)$3WYPb^NcnNNfyl=rW4&&+F6q?9+>CRJ7c|*L_uu&p8Kq`#>(*czVH6`U>B$nz z=y?H}RIIU9MJou_Pt^t$vm~Jz_-YcO*=RfeH=T^>m`A3&s!h<0=tLEF8vQ(^pgT3B zcBj!lilhy6r#u(#bX$AFOPQVjHt6BS1-BI4tc>$tcC&^!cXD%#j5q|cGJ5jb!41rf3~S5|Lfd+v{_&?&e0*H zDpXlIBF|dC&L^`~8JV%RnRrF{bpgB0fme7Ty}gl;BLmD{{;3~bFQnhk1K9OO-X6;3 z`RkT5j4eYck%d4-S%3hYJ0*eGSWIN`%Zk+?K);-v#4p)9A!Wtc!ep}c%SG@@jKv{* znJoBM+~Yy7gcmOYR0^-G_!yA%%E@tFscL})#+99glQdi#e?q>*yg%-XVqe`pu}HSw9&sgx6?mgHX1Jyu1 za(Jp(1x+_4CEs+an(H6lu$cdOx_XG~jN$$-y3VxJt}|w=u$V^I$!A~LwLRAvv%d$A zZVLF6-E_=ZfTZh8i+7!QK!WRxxk*UrZmL zwUyPtC3(roYkCJ*um^jNFiM&N(70g#)Ep2%DHw^0i_|dwc`Z}0)lWECmPmA>O3TzR z{bMa5 zPDc?eq^g2C`pHWANHU^f9rM4KpOyPbbyrmh8rt-+tC3_Z{huve7vIt>VL;TenE%qP z&Ew;j;5XId%=d2M2X67!vHO7vy>(U+Zxxprdz1(|OIP7GK+qgr-l>e0ZiWoekyF=W z)}fliZ%nbqJUOJlSb)EPHOX2fIRB**vr;MXlV6E1fvLW{_*U*O@F_8FuqZJni4v0Y zkCXjx!{?|LWch5k?0h-ayB`W+UQ*>RB%;p5DVcBE)i~S!hRkL^nQo|pd1REWfpPgm z7BI$f{g*9nkZH(V(0AxOynG87LG@x6S8ar96#wNE{n`3y6DyclMlHcW8O z(Qv@RO~x-0q2_S`qaTIJbF|vU5YB5vHYN~=ByBhTJP~nTyolmk<8Kr0%ab%qwQ#rd z-{oF~*Z5ZdV1+lE|MiTU1!NKsNhYxU=|#)qC7~^#QstrIcMTB0=?0q_>_tPl$79=nBKAKSM-L(|V*!rtdwEnJPU+Z^x9Ac`{ zdEo=`;cNX}x&FmKxIRhi?+IA6{;nsBTI;=P>H&kK^(WwootRa|mS$W`B@Y=HcKss2 zrz@<#WIUevV4Uc#4?fZT9se7vXC;t9qfWsm#B?kC==qmmi1)-C85FbKqM}m{dZcOT zvi?Q)((3DCr6<-%HQuj&yL7WCFKLRpF4zA&y1H2D(QQXvXA($2-Fmjaho0v`-8;a6 zebO1O1{L~RCHCu}Utgy0I;!*Cl00)3QwGRLvn@e|_hHV)d+aSlu3I3aatRs}1P5u8d_a0zU71{gny)z8VzyL!K1E4bm z1u;x!hUA@YMnG~DP#A^@g5)HMiDbYC<^Tfbj2Ktfyyl!;S6y>nbK3o_;s3r>=XO{1 z?H+C``#oR#v^{j!iFNAKsZ-}xRlszhzF4m}wS;dXNibxfMWV{=-vd+!g`$8VO~#BQ z!H^3q#M@28P1&9F%#c?B&Yn3wNzcfT9`YX{alGo8F;LIkZf-vKNH(O0?h|XxexVVi z=$D~m(USdgd}6;y7&&yYpwj(9!t>~#sj9tj*ofUPD0Jbo$r#)CnJ;1y_c`=l3sWt$ zH1{BBn|m9M201-G;qvar6+>632t- z(F=$~1O#q}HyAmjBM`6)mnC)~BU&~nGeTZx3%+#5&Z;;cA4bS~$W&mV5uTXD2xaYm z2>m~Ge`0LypQ9rAl6WLaqP6S8Z z03#jY`~M4KYyT{jct}|LH?!9MnE)mNQPxQIKZn8viJVXupfVEmUv%6m4d$w~|K*iy zf5kV3vPL!tyB8;6SJwU`*P#uE!o{(*e{PQS;1VatSo<4pz>UvgPQFpz#jj)v0O(yLN#qI`6#-HF0q#=a4_8;ZvzcnVBc??rWC&%zj zkVwGkZe{I1%HRLIIOZ{0D-zYsa#F4RdmO3M#3W&lBeC|+NxJs$(GXbskM1hHe{qW5 zA3ezEeeDSUwcBz!H>3AQ*GcbhPSX402uGhHy}vmoRvRT&Nn9+`5w4~qgusc8u*Z=B zM>v}6Uz+gd6he&2G=)%(aF;_$TG~R4;qyQA=3A2pA&zj&2oYjy%n`Qmjid{VB<8{5 zeu3o#twm~$;Ow2&B@_OOr`8s!KHY&gO(FId2Igl+v{gqEfw9K-s5DrRd6#f2=iRky?I$iCk7qi*vo{)D-u~0 zM>zHYp}s9ix0HfYxTvuo2=#5swiZGqNy}VYB`b5Ct#OR=$Nx6vo$`SrR(v|jX@T8a ze|B=!s9g{rabH<~;zA9(k{Ndi2P#ZSlw%&ukJ}Ps{s>I2MVhd#FtzJKHDM;y3F`Cx zjs;G>Gr_5PBw&=2ZSW1p18=j__#QvmB22zPSU-zkr#LlXs?Xqi{Dl@c`NnWZzoFSE zC&%C${&r{bJt5B`Ouj)_J9^taYrg@{};EDEO-(eGDyB#8hkg~$$UQ%;2VT% z6yc6eOJiEmgz4T4eNSv*fm7ejTcMIr^WDneJ8S3iJ#nQ)nED1`ea34?+zuNbSeh`H zviRosr)P=lyG1UFU^L&k2H*H|XWGdm#(#0bY#?63q ze!A(%0r)WZK8*D*y6b;%V)43ixk zxQcNq{Vf4{lCB2nF}WvVr2Pq+m!+nz09$SsquIeTUX)|}HefEYc;Jt<6j z+O393JLf@4=b)44^3yQsX|DqdP5OZ(CVddv!osF;{X?xh5F3iE)ll3gaVYj|uS#yL zvG`v#7Kuv6;`Bp=@dr{cK7EeCcm}5B$< z9RQ;bVxEpSv9A>cXzW#ZP-!g=$ZkvIQJ(#biDKx8nhBFf4I5lDY}DZD$)pzMmIYwf z)JVyfh52u~Jxy&a7E3EBOCL*?#>Uc;irT(SUp?q!CuSB6LYBHQ7It0{(TVb^Kpjm~ zJF%X+djm=tb=hI(Pi~0h7#zUDrJfEVS$Al6Xw%QUP1aw6HteStl+A5L?fy*cPOGQcJrqaK2Fl7R%PMNio9-pt={+GDyJZ}xb0ar_ z1j+Q$q#Cj;QoInPOLVnU<2^!~Xyrs#BVpvj5k@+@VW0)a8&viuZnc_3Gu7^Jd_tTC z*8kb!)Zx%-xosPVR)-Y_88nC!yTn@YwGA{#$Crouzj0cWr)zXAi5d+Pc1VwdLhaUG0QOra4C|(F*EW`&pKo3Sh*tRp@ez+N(Qh$a4Vd`a;rCG(lP;ssp)Gn9WZNm%T&L;pv1UR0qxauJ-@_+kSe(QAU} zC56adb{3!=`gsBBpVa7iiOWuxQ|>Svu#F$HvpPEdVwa8g&;Sej!~hm30L-}Y5Ja`= zBrPY#S}qdQhu{~!*g|x+@L05>&R$B?nMkp4BN|X0uOyM88xTQ?g%1j;S15&=3vo0q zRgsgmOVSf!7y9eJS6Jx_2%a_@b1$YCi@L;cLkxv`G@IjPh_NUlV!VSdcwW4#Nlo$m%s#6h6KcZq;sL_*YbkhMJk7w;oY9L@{MhwVZJrme7M_z`9VecC z@zrQUPp^)BceXk$om0GVk|KyW-ZgfqqY%7UXfB@wjAO(M>+J*>7A=QMAuQ-cdLlsIG47bmtZyu}u0*+~{Diqk!M z62y62#5qaCA)F!3vimGJic=av8?{S`rpn-E;S;^H>@%w^uhM( zJ%NWK{PD196X=nqOyoz^T67sU_3*xu`-L!(f2Ah!P(zn@*b=)DHXZeVg`Hv3*K7lQ z=@*Bb&&d~7TZT=$c&9rxJ=qqVoUS;jFDq_DC|Y7zdR`(4=@EeUWhavlb6y5fIiR3EUl|8YiQD3 znPS6$9y)sb1YpjM+yH!}LRSJxE4cqnZ>lNL{q%^gg!_5eMFY%tE5-c~&Tu~~IRD4t z=zhwGJBwg^(QqWiwgU05(U#Jj;SHDL8wNPu@bJ>0w3*UKitTEl;mh)ZMTfHVj2;VF z?h{##6J!c{Qzb!@5Cma?4bjgqwqR^6k*%YyU>Wf@U5enQRA z>Z)-QBGpy8kFEnP(i6Fkz1E^hx%Q5p0J$C%xlRzd2xrRmt_4TA`bN+u7VxrF59S0A&6>TS@KfmhEIpT{#_`FbsH8&AUZ*7$EGf^!L9t z!Xo&lV#IJd3lDZT>8Pc!zPzHgtggw4j_IslMG?{gE8jDf;V?y-;mwzgH*~=KKi>hG zrqKbb{QQTOXe>J5CerkiSM!xbAv+*0k#xYS+boF~hwVYBEtfGtch)LD|F1b&1Wqo^ zm;7+0laD2;SC+id^^@WvnrL|>^JG7)B7BcLkiJN`W}BsJTGq|pJPD&9;kqR@>cQu^j)vE5^6rrj|tCjr{Wp;pBR?R zO91U$cXmy0!6H1L%=nLbekTdfQ_vQ2oy_$QC)RhI27JS<^2UQ?Iva#%t~m(GN*f!o z^HAU907F1EiV@YQ>d89by&dRCL;|XlFSZD6M+mZS6l2+Y+C?KQO4dmTLO3G?Ir&8k zjv+{Y92nxL%@H(Je&sNHQkY+@w#g6`9=+zxELlDaXF{a|C8r;Q}BG+R0B)f--u=ASuaBYCeNqA|0|x~Pr~yI zw1v7)y8><4neU6sX_c|U`&?4xgB|a4Wm2IE_H-d(RdnpcVdIeHl%@9lphH^Xc}~ak z?*=`3o<7lY;dx%A^}zF-E1rjNhUYnbxCKYgGa!OCyAg~pn$MwlL#ET0TWz=ENv~tQ z55@KLcf1e7%Y-u}P#ifp0<^i2zX%Cmlz&=eD9V1(Ga<_JBFdQ}3gHY<&hXcNTICdF zAaSeQH<~I@WJWlH`(JkV9LMlawH=7?FT5ytARVH_(o_$EkTD~iaf3yO!t5VCAHuvY z!kjO{5Y80l0}GDA42qzQsw6~HB}^SY(JyEE^&fFtJKp=usp&uxLmcn3@WOO(+Ci8k znVtK9isYn^&gA}ATz?enfap08>lG2}91)9fhFE9bW5H3ZgCc07W)w74V$HxO#bW+5 zPJ`xlg7=;*u`q<*3on))LL){;hu3gslq}pGphH?B(^-=&dX(wFC@K`ZXGEscL?*%+ zGM#mn1xJ~@2->JpU^F#lI?I-yGM$LC-t_TEkm>zDB-5}_F`4cJy%~6mOlSM`AKEl1 zQ(+X_2i~(H)9E4;;S8D1o*9s-DuOm@rw>h)OtLF@wkmkxhBGP&hiExHU>&pTnMIvaUPNwojnvOqUdh7lD zpZJJc<$Wpv0k&>G4KEV5Zhes^Oc|roaQA`=DM=@-f8OGZom3p%0Eu1}i8hEtgfk@C z(87YF;YA~8qjpZvRCSU_v;pyd!eCnMeJlkskm%!dC9-#J?*TPKq7DA~H{M4j(X%1Z ziz3n4A`#(CiQcfdq(p-wXrpQyeTk;x5A@MFe*J5_kJfk}#iDxeqwo?Tv4}p3Z`|Gu z3Zx?abIt;bEA~%m^ejm8f=F|gNJBV7nsctP;3!RX1Z|XGMpM;4zBDgeZ8?b6dLKwt zBGRNEq%BQs-}Ww0GNd_ophbz<s^gG|!1NXNWX}Go(3pnFU8_4vwIW+9~&?;mn7= zIoIxgN^_F;h2*a=y?zm18a=(na=s6M0EtM~oY&G~io%pf&x0_pi7@AhFoZLNInSQ| zDa?=v+NeYC(NuMfoPo2n=qIp0|-E-xzWTVGe(cSq&P8U2CW$oZbf z_%}AmshsblXyTH6zB=PA>x1)tvYN6i_InQn|Ef|0YoOKP`Cx{0s9=_qRTrB>v0)PW#dcgBuA!6yPP5{RZFm9)n|Bu-OE%D z)YcZ(me%&&VO?xhCP)*r^uPuF`cGqL(l^svD5arD`Y!zQJ!hg zY*Mv~4$@2gZU#rBB35_NAd4$zHTLnoh1EsH>b_Oo7*gm=JcgxFi_jNsvd}ZF@g3V( z+exDSi=MRFvbtke55BlmI5|Uc(ooq@h{w8iSQJ~uxdw$ zo7j=oSQ_37hacn}6c^+`WW4w$OGcX5YVMRME-+~HPDw~@YwU)JUDC?JP7~Xc9H<00 zm{6uDWBii7Xd55nEA25ZDS}H5MP-cOMtxa4>vMWE9frOQtzG|5O-@{-Geg=vis zlB5WHc9{PnJ0GaD1|FPk1_XSuf#2v0lUf0P-F>?Iu{02#FU=F4KS;szr9~zdrb(Pr z`agcMU=f}#<@$#_f0&x*OS%6^ooX9-3&9)LD0RhtGtGZ;CUHSn&Be<2w8QVoOp(vo5fvZlA{ z5R8l(5zqcU1O!GfvhhNT5QCAv(F@>M{!U8B!CWB02;q!iWTPMdf9#M!1|vfwXrmsp z!xzei5636Q^hW>rcion5@+lngq1I(pPF|*i^@hs+szp?q>EA7&K|10nHyvTgLr2-m z`wsGWlm|oiJLR52X^#0{qBo}JO@97Yi$X{FJ=<6rNpT{Z{QR$OTLe{{?oEl$Q&SWt zeH-eF(1%Uu@R|S30w;!}yv)!4YV35B|K}}pYWAU=aFol!TSz*Nl3W;$@-lz^*Idw1 zu7N&kx{0>8q@(S)j4~~4_Vd3QJ00a-)Pwbyq&S_;e*X7kHOf((sVdIFE*pr`+3e?k zKTa`#HwW{-bDw;AfM5})vzhTP2k<9J2C&c0*Jwkh^NFyN%u#L$=6}aOdvlS^^AINjpse?L=WdMosNARl*83k;Mc zou*8xyrHtNuC}72$xSEh=S0Waj{^S1Vstze(Tgxp|3TA%KuG>VAW6bK4D=-7 zi}I#LhN4tPFN7#>h$t6|D1X_C5q%iEm(W#VZcm!?sA{bvB z*oWbb&bgwU)s_+!d&nRu=}|MbA-qJaVjo*GZsMq#s_{Ub8+l5|NC#cP{a@SG`bTjR zf%l$>vsuI;oFUE?w_9)&X9RJpiX}8v;>cRyig&HH6sN>Pici@kk|FEEi-Xh-#8D-E zN}4A@$w-D=>92p{oiiYc^^f!KHwUCC^{}6z z3OpdqnSVeUNsT^cOLNt379|dzfzeAL&D$c)r6LXC3~8?N=fAi#qatXNpG2Cg?EYup zl(8UPnI)t-JzZ(yX}3obr1A6r@xB=py%^HGDbidl(h$y&=IT8yEzr_NN6<#;8%>p^ zNo;tv-T#!P+(T+lNsid?tnktxHiV_ca=ed&kP#bReVfG;yXK(iMhNpy5oV(ZLpW0y z#Q(-{qcCG4Xj5Qzjrg@|cD344n7$seuBt8k+S%cS(SGgF(c^|!kMlkC6X4A7Yu9l8 zw;h2O#k$UWMP%m^Y$ws(bD(cO!7u<5hw z$Nw53>pyC|!J;CtJ6N$>ibv@RYb%PH45t|Xw-7~02V8IazbXuzX5J$4(CC0gf4&1W zO``*@=lVB3e8xg;PkTqjS2RbZ(*bdbqyui~W?^TvpHHc+r}}2-&bncg)s{=jr(4P? zwADPotG$!Qhpeii6NZf*RWo*C^~CBlvc8$#R?s0W>B}4Lx9D+M+1vX*Uhw#RI+APb z#4M8E*u}!mRpk%lz)A=SCpX&hZxPy-dRTG!={LRNs;w-+s1aDHqiFl4(=65Xl(fseiMMPhG=ezvjyspv10 z=%*(Tktsj^{gV1o+c2#%T{Q`|!bb~z(oGhAB7hI%*T2Mpax+}DX$7F6$$@V1n!bhmztr@XiJBI3z2yhP?wtaH|8!k`5kA;J`fEG;pIAOmfn|F? z6ANPk=l+|XxLvSFr*5wS7Ix~_sYlOt%Fn_4wHiGSCQSry$Nmp#7-hg2zyC;nO`bXb zljpBf@%)v6CG%=PJHwW5c|ouU&$n{@PoBR`!t-RbMNi+#{67ckw@w4T+3MsuJ$gt= zW}wqUMaM-YGjSYXMMaY#1QwN}5bMXPfS(37(qNQJI>>(;bQtSbMKP^-@4IM#bj%is zj0tB%#<#w1!7(x(7eSkz1mlZNcNX5Daa*R9kc(Myot<;>^uxKKyi& z>0K@~Wy-vb=U>Ei$jN(O?*}>i9f8;nYB~v}MW%#_-I%;@^Xor!4{`GTk!`FEL*eB1 zEQ=HFo$bbDwO=lKrsAZ&p{x@9+4S)WoV>Gz6Pdhkud}do@_wC#S}Kgdmy2eCJ&_9RE`R?w&g7v5F0%&&dpsrB z^Z>QV`7ZN81bZwc*!cmlDSNrhSQ6~f0GRaTUC$UMk)Z;tj(co66;Ny->b~pS0Pd-K zx=iX2_agz^0``|)0kFfmy3BAB>~AT-IRDu;ad;P(*<^w}oD%GU0NAw7E;sTB_E1VN z&VM$w>3g`52Q}D(sle{`pMQ^cE;bif!$fKiqy+09pf+PSmlfXxyFVq^;s99ft}ZKg z33guqOyZHduLBqz!gq-dL3-%!rvi9qc5+$7NWAw3@QTsg;~GZOYB*Xudu*)z2_J^h z+_QH8cik>7OFoJFuL0b`(mm4xVD%kcRv{AXo|Itc1;83QxGeW0*xf0?xc_DMeq(!A zmBa(=FR8%(O8;kr9nsEZIUcFql@hEn05&VnWnnMD?o0``FaS2Yt;-r`g58l4?3w`B zk!^SpvI4t3B^dp`E!!NB;UPc#NCkH9E&;H)xo(7q=>Y83lwczQVDnnJJOPPd+f#zA z34qOS=|*nRV7H_M^XtFkqjo`#%j0TEZCgsPp#NK#?eb(;g58`7?7pD?Thzjh+@!&7 zN(mP9e~X*DJiC?DZcGUl^nXjTTy<^-z-~wh7W98ho4M-Yet=z{3he%%|67*n@{|ZN zd0k4dp#M86!{u2F1iLmRSkV71cU_)&K(K29V7>4IqKo_K|7=q}8drgH4bvtI9~K;8 zWOaG}doeh!F4yB-8V}3N)#U@NVl5Y`mbux-;FGUTc#9u?pa`w#hrdts!xD2nFb)kE zD*c!wRO${y5axlSgw&5JRAO-cQwvvx@HzO4x zL9qT+-5&?}1^I}L8f2pd(gR;Y$qzFBMT?1p{EBSErTDdF7CvGyY%kXGP+Wdm%M85t zMd#o{z1WJj0PGL>{{Kgni(nW2rW={o@({;At>woQ*7DFwG@!NooWxr87E)J=AU`W> z;o_ouuB(<={K(dVsJNOCKGZ(~P(50?%v{6#t6HRhdN?lt)w89`JS3r-r-JhLf8reO zo#V15hEQ3lppFaB+9%tsYNnx@rGmOO0M)C7Ta~GyGE+f)8i4BE+*JjZ;4mWv)Zg|D zK<%65qQ8_}ZYrn=0jNIBTrBw%6pq)F7yfPZ*WZ}`qq)X)xF5#R?+VKMJr&f00jT{k zlxT{%_(%oyQvm7!jHX{T)USz963#v1=fC3|9*6<;3!qwO;ll(Yh>WR6{Pq9OJaE&= z$Gczr!41;`KXPV_EqY)YdLcgfGCc4j-2dg6`Z>jzdgK)};F$U)$*OlYkYV^f@^7?p zR2vqle}l{#ei(!IQoQlXyD%r_##n>w;#p+=P>+JCGvKi zz+b!l^X4*XAals0oc}4ouSq1>6Kx^EqkjDJtMjdv0GVu}(|Z>DWJ|xbZ~uO4h}&EX zxr|NmVoT|=$2x&NZShxj8MOl?aI03o*h;3u$2_!TXZ)Jj8PG1-e5_Vb{h}`G`G$zd zW;Wh?H*4;=`6$Ctdk^>mFGZ|5&KDVl_XKAm{=R0Rzk64xB);Q|YvlSL_w!%BDGQwekUkD}x$()DA@1V`inzZeihC*=Lfpsw{ZAz> zDQ7s|75I(v&TWQ2v+^z_>9Vo4)wlsdV%I+(9t4n#L1;{QrwHPiSl+4eIgWd~lHo@%jHt|IEoV zB*t+Hp`P&TAGETWLG9!H=+u?#cSd}nu*K=0}_j|>}v$71nS2fWoQyMQnJNk$IC8_Ya) zh$upL22Us3;7xRDirE<`oST^WM|sIrGLcMgA$5mEin{`l=vM3sd_Y>cPIPNV0fo?j z1QLZkh5ZlZ*EEk**m03(AuHi(oyfE4=*v73POA*&wkuSIs*~u@Dh|Nd%cZ`2g-yrT zP}tLc{y(m;bG%7TO$nJnjnSlZ6qd+LF2zzcMxwK){rtD4Mx8~YmqTYCiq02@KGQ9rv*RPrNpVku6L~Hjo!RFCYOO)c9UB^%kx}^!=YLE9`{E26e5)pEeCpq2 zf9e_8|3MpWl+qZ_Epoll_6lt&t@M}u;nJxD9ZpmScKi!%mc^7sf( z$Lv34xz`908C}oH`o|!2{(&w|XT;oykMM--9#EvN1rm~(i8o|=j{AR@{0&kU9N>D~ zD;kQA@T8hP+>+EL2}P-YGbmoTzl-BRaajvK!c&a7FeTU3LP2cexi128U9_L;aV~*P zj0sOgY$A3}#SOqf8bas!@c}v)_tBg5pfgOd@bIHDX zw;i^3aCkZa+baVh5)s>bp+|twrM-160BmnactS~RuUyE8?Y(ezfYiobx>fUUhhiw(XSyp4Kfk@!}ByDwpl;dWWC|Y~qmcR1$5k zQfP=xy!d>8&gS+mjxAUIXsn_W>J)0&ggSve_UQX9KtUox=%tDPp)KuP9QUpW{n?}5 zNn{${@J26f4G`Lz=XwlBqWB0eQOaZTN$7A*?BzBAVwbmdaT+KlWPF4t#-iV3w+xVu zXz=9~0a{nIaXk+$Fnm2ttE{-Pw4uyAjJ%}2q3`~MeAU-g)|b>*pbEn7swymK&8~X+ zg8-o`Tf1IV6N;u!s46l-9llZZgAkIgdPU-Y!;N2sB*b8}LRSqAPbgVeRs9GwMpwNe z`@aUUt5KO0K`V4sBs{SsT~+lxu&}FSk^5>-Xb_8B&}yVn>KA{Ufj5lwRX_g~59O}$ zCQBv%V8@%BnMUkud&zt?l_B0zv?Q2&^-kTT$Z#BiUl_~{j$VOa?js52uJD670uty{ zyvgX-79a!Q6C%$$SRdo}xlZKybb`60K9>A~RG65KJdu6y_1Y;GUyRqT_3%)ldK6-~ z<4s9NfhqaQrKpOe=<&7REV9&Nb@WQ;@ng~BmA)PcNT3rnzpwlCPr3uB#}gwjKor8& zIguCA(W5cXYwcy_W3kujD& zbG8=qE#$weVu^P?hDiO9_)@<$FxRFJ_$oP~!ca#-q;?93;96-sr!#J~tBYF6SL* z#9cp3l?F1}-fj!-INCCjj5bN7zb*a`Z7|yO1HQ~?<8_q&Fca@@*J^#nT0`8_iTAht z^-l(qg9R#kDZ;=nIC*{XAobpeR z$w348)6I7dbYIi|(DLJN@<<{R)I*{8hin_*mk04L$01+R7ymrU=!@n*^VfIA*DxcG z25tjBA;3Q`)tZw5N6%7H)e^r$0(Y2G)$)JoSh~9T@W2UomzLq~(tOa*cQ(0e+)A+7 z0s`mEo?qW^^n_y;HsBvwkqP6ftMNy(nRN>b-2>5vO39hiaLkB?`A00B75l3B>?Ku8 z>f%lIn7w4;{Mn0^H5@v7>8!!a7S5erSG%-f)UvsAmnaw_%J$pDu(VmQxV~X=Lw$6? z+-39T>z3e;Ji-fi zI!ujj3H09e>z{ltm*Zx6i{pXbV)(ffdQZ!%kt|92Vtr!S6200j9v(|8ge}o|FG|;| z`C^IRcy9>s({E%Z@f!u80egOLj^+VvI7{=tUjfp5lPhvM~ zjKUHwC_wXS8k2reBI(ult^E)9g-qw1*o|}O)$zJl!wJP_FAoviVyqu<|A*V4%2-`K zO}dANOI?)D!>Z)Conow-tYNGl`1OyPEFUO+oQM4P5o4W~p0UQo5@Y?~2Z%*uZI;AX zWs>_)p8r7`7%KwF4Cm>AZa(xAvs7ch_(NI$qBW*16?XOlL=sE=kmH}0+APsh7XdX! z$%h*(os;l01Mw3LbyBl|PC8v0+5=)jj2+eUQG1JlD!APzydx#fE8*=aX^a$XB9r;i zc(=nF@_oek2MT~qHS!Dn;*T;nesqp8H#(d6d%@`yk4OXQ!jJ9(7ItCtBwZ+x)<>vM*Yh$pj9Ofd;Ei>p6*`9yemZ%2|hkXBxsRD z0x`mmH=zw3U<(y#K?253>XjtY`uORD)(|&!r1kNCVyziz5jQ95BqsYwceJF*wn*$T zb}hQ}lfi#hWMcp5HM)cuC?IgX zE5RH)Z<*s=nWfCHsIsA?uD-CzGcxlLOSj=23-JrERcXM4^n5R~pJZ-2-eB<4;+V7y z)E2=zT~+{}h`*8ME7)1LZqE^FTVmAa3AN3n#(o-;4}LPC9R7df&bjcLWMy{wtaFSs zdtw!;3-tf*|8RWrbAXmUV_cCr)N5mmvYkRX(nxO zVJg%0)C%k+MFx(@W83%SWE$S!=JOnjo2{_kk75^9AsezBa~=F?IB65O$)#9kHso`} z|7u`sBj1i}$k6CD$cB7I_OO9}O-2F5(15-1-D);L``9~L9)(2e_vm!=7UU?G(kWWvH^0~p4bbyv zE5E6HB3fv_x;H+dr7vC;EoH0SZ>o0~o|ueo!OyMmVLUTQ2!Z-ns{qs?IEn}-5-BM2 zFd`Gmmp@KxIF9_m$v+CpJoLweS{|S^0;iqe&=Un^o_AtG-4uWtiBn&q8#R=9?u!ZK z?|;NO9ED?iaGH;zWgg^XKz+G;fYxZ7Glk=(6qI?=lnFID05t}O?BJ{&1!W$yV?z1+ z-*FDd;=nPSJEow_BgagrzXoWH!`WUq;!8o9r+b-D-v*$@<5b1y1scjcSJ8m_YM%hq z1l+q2JzqnaH!qk_(*jTvah!DYJPlj153ILjPInky*t zG;Q#jf->)f zGNC>S(5l7F=ePh}L78`-8&Kc+^M9PyOk6M?Jw-#ASB#rbV*|A6aB+O}WDR9r9dAON z5`e15?ccZrTybdL|7}9a`q$85gZG#lU8|wiil5(xf0|I=2WT~VkGjz{8fuL|-E2dB z*E;}pg!hOWiht$*|>~2x>7@}6e!l+ z7;=5*$N%yEIuaL_;@(olp?PJg0rkBbpfv}V>P8tFK`!%JT@y<7{|ve2;^tahV5^|a zyK7CTc>!AUa6xSJI1Oc95ox^-$v7%*OXlz2g5rA6Y-RnkIXs8vU!>jO5 z1L}vi0jP!EU)|_28tNENszk8|4}hwB`vE10)mb*#Ok>-py`wu7;W`Pz;a^sGluEygjx`QTIt>3Mwzk!t=R&_0Lg^9 zDgd?0yWWk?(onMmiUE=d^=1HSwRfEx{Pz;bv zs38HUwca&uv_V5P2owV(6YBT?)JfjeZnR!Q)e95@Bopek0MyCeRc^FSL)8fs10)mb zvjEg7-j!~2riPj+Pz;a^sQ>I0fI1Zql0<7YRINZUKr*2w2B6k?m%Gs!8fu0>F+eh* zP7gqx=52MOH5#f$pco*TP!9#5PWQIB(dim$xi_l+K%MDb=0;g61*pRXiUE=dRTqFd%iH8e57SVG3Dn66P!|WF&h|FC(Wx40sz5P7 zGHE>@fLiZe>PDw%s3`))0Lg&*Z&ms{nVCuyij0>uEygt{gGb)I*j8=a`3CJNL_fog|OQvW2$3nmKAk6w#B%w;YbAP;kGMgj98ItxJ{22%k1#gBh<0*85+HIY}4St1-t zd0t5;4MVS|WWBPhMRRy$(v-EK0l1R~iASG#U=L{8j3ogi9)IZRAxsH~t7G@@hItL)jvd zSFx+ZwBnjIp}36SeZ`TTT4Bjw|7uy7z50|_6{(}hs~vAeI*LoeQZD6F#Khu$ll@=A z;x3F{59MtUnyFa8lbeQl{|)WuT#}3TXk``9mUYLkPYH^bg}w~C;lHadL##64RHn&Ghi#ewtF+g}BaL&R*b6mN8F z9zN8I`#w9b&?v)2{67qc)%KSzG#d2Yky-d=7rZ;--3+oG4epr6%}KfamErpP|2b;= z3lcQ3{e{a*q+|Wo5I5!aSBA^ypSg_6QH4gtO*LeY23~vMttXL;3_kzGNhBw65@FN> zQ8N7ekLt*-1w_;wKT+K~v(QM%51iyN1K4AQMzVRAq|hiM;~k?faykLhIpp0#55x!h zLJEyCGUfae@kw+oWmUPlD4K;vx&KqwQjomhr6~XZKZQo@%ccvB3QNk1D(VZHKG@eR zG|Fh^=l}HZr=NSt`=^|Kh~fWFJ^aHU)_x8hrbaiHH?YcPe*Qz_=RwrXy(MyFFlslJ zz|SQsPl4Xk@@gbYl5{K}n>9wpRotuf^=^}=U?J9R>3TI^tVad^Ef#(j^0eoJa%y|{ zI>#H{Lh;|Xu)HtITbn$4GYvnVLQyEpIXDe<*>yY=NR-z6BvUAelHvU5t(Ak7p|rJH zY2k!o9S;;@+CuY>Ec~qF>FgaZmEN$0<9CWJXtIVav^d-%TL7`PcsO!j-HZ*fW~XNh zaj^zyTWE1_4F6nwLtE(Js2Zm3s-ml>dU15XKGZjIzi+E%{%FYHZ&PRB1K>g^AX%|rZc9KD)vcI6O@rXHP#v#B06oSJ0R2__}iFave%*|XPB0W zGtAXO_$ooA29+W_mcE?P+3C$XIa%r1d^n#1FA%P93}!t})m4lGJLGsI-cVQ0VHVfz z;6z6sBzHL%I|rp>)0s{t}a;1Eey9(C1M=;Jphx$99o`>mNbV}iRQqJ zH&NYg%C1O{U}!~in9*z(K754E3+F8|oEKELU&>$Pq4XqBcSg8ejg9gYoA;r#s z^!m6tg)sWKl^_2pvv(H(dEXMLEGTwLH1cqa<9rVvxrBAC{v%p$o~UIugqCynFhVC? zhxXJd%ih6{Y#^4BOZjOjt&>=a1md~O|1c15t?JMU8BJ*SycC~!!j*FyqXe1V>Lu1w-$lB~~BEzuG z%*gPrf+J_=o#=R1X{TFWTwISjw3$sUaG8|YjWuW^swY$rW|7iPs7NE5vxL=0Vyrgc zn?>O9*;a@o@wYS5ZKV$IPDMU!um_c7ssrjsa~a-HQQHAAy7TZ2={`rg^l4M`6~t8n zx09J517m~=@BJ*-xgf~+S-XjF{X`+r#|pR3^kw7~#DrvM!=%+ok=Nu} z3S|0T%TfL_9DegrWhw}^9;Qr0W0W!18+`aIl=k67rjQvotl83q@kr{Y{Q2UkvLPwUm^ngR$J;M{i zQh@_}MAi5SHAAbb#!ZM+S4~j#IPs9y1iVRaMS$L%61OXU?C*9BPtQJfp^e}dOK30W z-@;nBFSpfc>&z{7yWr1(ZkO;h#1di$esll=QV=Q~{QBQGm3c*OXGNv6rczj3R9{?d zddcMWCMBFoU%o~ARIU%`Ykr~2=xU(5N7_{UXZRzXM2@NWM6Yzr3s6~bpu0PMta5h` zPbDE0tmN4hNJvEtyJN8b7arj5h6bLyTX;%#$)qC0*_}jukv<5Bv}k{~0u8F%3Pq@_ zvAnpjtg*?%*zi0y@29lyal2foOvERYx68-?rN#TXeevS}w{Lh#3D@6k1w09e9qe*z zfXI?QZYdfJbW6h%flc#{y_g+bCKQB9C;$16xJ@tJ*DY66%G0HSJ0sX{6(IB^9cq*K{tm{0l1O}22JKJ@5fXwoJ z+%hz%a?8S#(RLQw*ija}S*VQ1C+uw3I|7uB-rFt4kNw=@@RY2}V>bet=w)>77VQ6H zdb%ZO;JGE?NnmtBXM?L?XI10Xp&^k?LPW;QZhrh5H?$Q6ZV?&{aEmmd`oglpvWBMb z9Z9%OoYA2%W_J5Hps-`Rs+*z)y8ShgvdXfG+VZ9on6gnBI}_(dP~muyF|+%K0F~pq zxD3t*x&zau5*wK;#3KbUv)zN^|M<@CAT+3Q2Zg7S%tLPzDq;hBbP33F!XD~2F3%kh zo>CGIeG>?ghz!s@{QO_s22R}FJqQg3xd&-NrHzf~o~ExO#{eD0*dunC(CtD(`e%=S z1>{-TQ9VgJ&^=I-DJm|jE2}f->EL!^8HE@u~47}lNy2$x|Mhvy4oqGU&9O51jo>1zzaE;KC zxUkDP0m0VhxqZ;6zuPA~Jv%O332@>UOX%XCe-SshliI3G=0LYFJdMP-@FbxiRJtA> zkmTexZf`Ui==RoB3L7g*i)S`AwR*;}B-ym;iQ}ussmWlvP!W6U`fz~CDXrbT(P)sn zcX%oRdpismNJ8vw&)xwtr{=o*pn>P^6P`>Gdplf+%)lq?ZO>r;uWRM@!jD7TUf~I) zwzp|OMOtESdu0R!JFTVL6Ak*iJ;T$p?TxUMsCIfWlDFu3l)iW3jF%dxbD_Bb9k8C37tcNNM_dmrlYw;esU?9X>&bZx1^`dT@t+sYP&+zc9X9* z0#a>j@rK%Z`uUHz&7B%~gICkw_gPNljdawO=zFy4nl+)ic#$6YnJGM7X$eE4JC8SZ zE7cv08@tm{T{50>DWRf9O+X8Kcd{r_eV0aWhWf4)_1)~NkAMWC8VNYs0%R#oc@<_L zbuJ#N^jc>t&l0X^jEUM%+Y8wsj#t;hG~AY1rB`3yw-D7Ub%xIkhW%!hUdBFt{i`+; z-k7<&cUJs}jKB32bF0}N>D!n(n52a!OReB$$)8)`(IEDL+ zkO|pd=#_6VG8W7|%g4g~8iwUs&~W|a<0o#Jt9q>je$G{`6X&X3v?h8X|2+(Us%kpD z4cTVX6TjmdHs~mgSpUmifp$qY@n%aKoc~qDZOC?N8`1+x{6JQg;*A&c;s1+m$ljR$ zViGhzmQ?=p%H=HMHzeqNsz}g2i3EG1EsUb~jcCJd$o6U*(!;mdSX8g0^oMcqSVr%E zB(#ROskb3fBJShnHlzo9DYhZ^Ek;ZBSo_2t>kA;Txv#(f*-mXkdPGDvIZC_e<1luy zj>GWq1IzjFA__jZK{S5+r_eaufXwu+cAQLh+hWJNI!g_)+QyPvRM`+(u zav{YN5>T@-L4k)G7%cG+&LleRQz80l=QQ9K?)^RA&Z(hQV?op|Gl3$*)i9(Wc&55K zi)3cu4Mp_1B1UQ+z9Fd>NGb#W7@4mit`@jIPnR@h=@5CU5B+c*K)KL)3ipZlgk1X_YvE^vvM%xuT<1po*jfBXP8uDIh#;2xp-x~?DRvg| zQ~Ur7vLXS>{{H@#wzsV~hhwsLz7z~1KsjH#+;HM1vXe`}%m8Koqb=b%!)}aj!wh>B zrAJV_Eu(;PXu#h1PG;Esw_AXmVf$8L+bfH3m|@#wr(xHy%5d^#=`E_@?24MqzMGwn%*#r3o0}0f$sx9{dK!Ue=RdC z5?=B7$vJ`jytaGxxA-vTr@~&q!Wp(>lKF||i2+^c=f69~9Dha&3`F)=8ywM+uciR} z1KwUojlyCf^S@-VV=5M}Hd*8jg7e8KEfRu&!l#ABPDxmt4G^;UHQF!;=%j)GcsjE3 z|3MH?G|P}+S59%xS1*2l5k3qh7o8~*?3zRZ2?C1z@xQAI0?2w3K|m4e-&M%qw}!Z> zgMi}JvDOR%h&x3PP|Wz3J+^CNkBtNn6kR-7P<2waFouZyAfQ+W0e!3>fW&eU1o)SJ zO~eIY))inxe5N4x_rDZ!hoM0;?;6l$XD)KQYjkK(*HBbhS6{S)GqQ4>mP3b4s;(I^ zIy!v(AB_yim?V=*{QQ5XSY+^G8>fc(F&ymQCKFN`1k(x(HVYL_$Ix-fy)hln!#4~J zULqw11|#y(A`?pC|2I2-nk-35uwBahA6>z2%1ZO_i+=IryBXE8hKGk9d-?yKH!@CnY#+66c-DtEnqY25!(xI^6>MBJAGigV z`^fRSoi^nkl1dO&(I7*gMp34uQVj|2Z`S>7+2P|DT@M!#ODm2M4ES zDrSa={~ilJ!{^f?Z#s_33L#VVrVgLO;m1LNLq{kVSW`WDf?6Vlj-kmJoP zKMRohJ3aCi(g1{m{@(h7`omRuG5w*7cYL*(p})$rES{*pN4z7Pn$F}2`a9we>Mzby zE=4r;*Uz;`Qh%34w?lu|i2kKagQNlkTaiQ<_!97ckMP}gm~ zx(G-h$q)?KWC2oLXGPv|^yR_G5WJI)x};O%TNqiXG)Asg7VsskuPy&X!sUX*sbuQUQuX={Ybmn<%+w6i1c zB6=encFo>RM`_rif{n$B(X^>GZ$feL4GgkVt63Vks}#bMWMPCAN9!cs2f zQ^dsL2EAqxr1CD0-T~#^Aj-SLR~`Wgvw(U!JY>TW~ieO%p5IP5sSpN{TyFLO;+Ghxb@YOT8L9jlJK z)Z$Hk-n96k>Ym~;>U)wOSlzsIRF{OGTncAeU6r5zh!5E-qjy1bH<2PVcbBg@0#b9Q z;0;cznEzm$8rR%8kq;baSHjJ4A|IrqxeiV{TAfvYuhj=EN#|4$iO;EIsS$6m74hr; zG+R{Q)82fEMhpm5erMmPaWP628TD9JW++u zcnjhI@q%^Ke6l4chvN;+4Yub$YVKLyL)3%Zc({is9nIO?>gib1T<7uJ;(y2%Fd5`>^Lfhg33NlUuFIcVsIGgs;v4M ziyNx!IS-edbP<&;N?&C@H@TF`RN2A){#V@BT^s!?RJL7I_E%qJ1fv9!6#g8aj3HAjIk|r^(;FhQ z50?K!G88MNArNx}b)b&S{fi-YTll$u@e4QR_n{uKe|YFxSd4K0 z;v^3ji|3OO2zt2|G#n$QptAuFM=uAb>_O4~ z?&=&L;-(rhNCU4u@Ro$tFxEdXVYPc=!iu{QQ1l^||L*tyD&`2 z51g5JX8=1h{#=vdrM4W%d*-|Rrz!=YvdP_bMsH-XQqbv?HE0b!(7@#<8$Bo5&|Zno zrLqe4!M!w`#8j2{KXop(`ACrmt_&+F%vUG*{8g?D%Yn$r4%4A3!rTFt{JJrHn^=*0 znt=w8c)J*{Bh$Z&h^KU01oQnSCo7E9#4yeoRJphonJA)LLD_N~t}( zvmKshsocZaKo0j425~;RxC6UWjs7F>i7p>u=l^)lPYZ9U9KnFIU6x|fOVKN7c|DRP zNfB$LJ^#_`o$TQT-U1Qp3N2Q6ujh;Ps1|GFMV8ulwo6A3X>+AEobxlHg;HB_X>A$y z)tk&-tA^niNuel|zvziGObLtHLaSz9=@$hOrH%6UKh*HVzHo-~lXp_QnCv92v~WW4 zoFDEZ5VzREsAo;3sr$ORcqha!mpfsn*n%c&*uv<+7Fpic^#s>R{lymMr)LXsu@*u! znD}VF{!^=sS7JRo2o8@v9P{7ou`Y={c7|qi+$3SMGlyo^ zMMUI>5{v$HC=tJVRLKvL-mExypInC61Dh* zp2zQM3g9r1=vUPa;U;jp)lOBr|7jqxC?7#BH{{fYNX@WOgR3X!t0_30dq?55|2OxJ z(k_xkFfMq6s^DWP%lr04w6}g~+dDG8{2G!;%YlX7de0=jPQ3M`OVNhjdQau8SJLd_-Wq?Z&STNZPChs4 z(2j-#x?ua?k4$X^LWZ|Kv{EG4D~SZ+tq+}sHuTneDQ~?J7~p@^D@nojq4fV|YlxfL zTOaDL|MybfdL^)?@YaWZ0KDw6y%Kv&3brRV6T*8cZ@rR;$R?8}Nei|$u?hHD6>Oi4 zPgus};U*eqhLf3*CgkPTI%II@c>nBUS=Z6G{(QKP$C^M-6{U~-_zmvC#_dVi{pP3$A2ai!~bt_ zb^>7%mC--t`IwL#{i{%PQsMpAsVNmV`d>Zz!}Wn?Q{z4!*wDvQ547+z;dw#i6UR{n z3bjt;lbkgAIQ|qPRRa4>kzB`5@#H!gksh7u*S{&dby%@I#k&9na1_>aybCh_Ff+Ln z%FK5j*2@x{DdOv*_afhUD~Vxm=iZD0N}&OJ<2%WB9_FuqHIs}a`eQU=nTj^Zcec+~ zOoo|h(vXHj7q6}>(||Gaoed?GrO0*=W|*#Se$;G zu-H8bixdYePUrqNgMjWT2v`s9@B?j*OE$dn_`m-_5KyzbAwk|&QWt5vpXLK-eq9_1o? zGc=2}c>Y0kQ1@71@M=q^hBJ~=&FBABRvKOYPU2K z3gX}~a?se;dEa1ElO5Csev^*g64=S5T#Aq=t-*i(Maxd5-57lUO1ndp z_JFT60upEq-q3c#R~DC4+9i=M9A_`Wp*HY~bd-iA8kCk$8`vzB;y@v3ZD8XnizO=V zH1Atc9ICXwO-FG_Sjwe*PA%S|yhh~z#NE>aQh7H;AB6Jm6y-hWE02H#Iu36r?}$DY zAeDD%INhoG^$L}L&68Y3VzCX>M|KmQRQqZ=b% zI!*!MFd2NAj>fQB?CXnw#_*hdrUgg~+Z4gmLEQ+4;rUfM`qE<|c3!FWYb}}> zp7Hr$4an8v4WmCY-(rb+I};Dk)ToQzFg$-qM{h}3%B6hf@H}#ZMUcw7CHglg?`~1v z-+bi}kjmo%h@3uU0aAIFMZU(R6NE!8*4ODMFE%8`A+}`jk)0y=Ao5t@A32c$I>&$h zF>Y^Xc|XPn=#S~>ErFX{ilqoqe26~hQA>2{Y8Z9`qmM#ue-*Vo>Z^@_1d^Q2JpcI*%_eg?1FAZpI+Sq8>2%0W zV?lykM7R(%8_4nMx?Uj<2V>@RDvB!_>*|Y}d>j_|jhxQ>u9kv|oJ_YbiyTxnJQfR5 zF694At4J1Op`ZWksj?W5ps6gzLO=iQw}!Z>vlt84iuQY| ztqkI(8Zt-&FQbNS;mv5tEJn}7EQV}6FZA=@YGiiZN<_?JP~AIo<5|fMoaB+b#w*y? z3$=^~@rSkXyl9Tm7kP|JoQGbj~AU)U|9) zS}Rprn4k9MGnJB1MaHiZ%hJLWyamhCQf)rFYLu1~&MbwWYx*f?t#FP&D1C`v|E-l? zgzcJ_y?3P69K-(|J^aHU)_#ryhq$@GApQn1aE+feYT4d0*^@<$+A>UQNlLG%_q4nk z$&#cm)>{@?dbM>pgr}w#M+3yVI$f{ki}k1$Yv}+BKWo%>@kTnBCGb0H)J7uO1pj^O z>nls^i<_*SRt>}X`4o!USv6|Q+L=Nj6`tYz?49h?l!?+#)=CQ}6l>I=5YrZxEwb?Q zF1oJXiBc{NTR3s2*n%cYTVS8T1ebCBLm$7l^6`22R=@Z=Swx$oa)q_Md1!;nI<_ah zp)N|j)S(S?)bsL}bsBj|NquQy-42z~U-0_4 z{}EvPIi8YnsK$M$!rg!uHu{QyQaq$Dbr)k_XOX?~b z%FLX8Qrsm8xX%gTe!;79J?2D!dz{8yQQKHoXvMNgaqpdg+s}W+`F;`mb{;dvz&%yr zuCFLAs;Fo*&zet)yHIcs$0tV5F^2^Be(50Bqb~;PF&cGoLt$lcUtc9fGyCq}x3U3d zqpyRm$9>QhwC`>JB3d8yRRPp54|2W3H0r}N>Y~QN3LNjTqn<1hv|=SIum_OPldlYP z@#MU+?pcC5)KMx~IFnRcwy?e8@Br>t2e{r?&GuM@yRM9i3Z~g|l-lWgq)wfbBPcbc$wsipE`9 z+E`K1utUo}P}mk_ALr-);%xr|TCUZoYZdC+iiVkpx%__QMdwL1QjviA!~oZC?&o^r zHR|ySb!A0ET}5H>4lR2hK`qKYF6jT>+SkSN`bybHYSiV0rFFGsP1Yh|Y-i!%IZN5c z2mRmMy{`t~W!Yo}p2P zxp53!vjmVFi?Ua?4RHNYGuNAq7H}rB73xs8WpU6{3*L74#00q#>tD5a-#c;fF6)(* zX9#h%Oj7aZ$9&{>KMB1TW~8x9AOnoFh2(&*vzfgY(ZI`iZd%dBd(&!mI{` z_GnFi)m%%yDy)Z};#m$o$v_Fta?tXHTUy{b3Qk-^E&5&cwCJ~cqJEj{K?kgY{2Tzr zQOez)JzzI{!v?ZYSk3hh7YeRTJw>K#$gdo3-5aUSS0q-z3a2oz-nlm*%xbpS|fx^QiOxMTvcUmVkOR>I9Hi$ zGU5cQYyJFJFNeV#D`#JCi=*!GH|}z6tL}15MLuk@1avaJ%Yl$Im*EY%CuPS}G!Ng9 z?%zq5y*4#pL0m6zJDCYGz>t_s`1^mVyH#;cKkrv;NMJy!jLomwLxk%nN{LQZ$WIUg z{`yA~;C|1R$hVH8JP`KCzRgL)5U?vX3L8G`4Jrs$o??#-KPBBGJNdYn&KUx8Z*e`Z zu2m<9VH5l3%sWqkv*)z~1b(-ZIvIf`)-5$=F|~c|{5E;SeW|T7e@BS5t(0JqCw`LURwZ?|m+DcWc(o$>~|pQ@@;U zIazV~S^6bdxa0$-)BXIP5+H{`TSa#{f2)tvJu?oD`kh^tF0P%wWOl>+r8RSFmnz5@6#A?pCAcmv?-QEtAQ(28-Ji%(tcr^x2 zt08#PR?}beerD?!oNqPvE34_QtVWEcfd3F%F+>0gqNx^Ce*K41d<%B)EGS-?{k{%v z&lCp_gYD#WT+*<3cJ18RD{7a{UNHZF#s!P#)h_j|nh2#g&%7zdAJy`ni8R=oREl9W zXMS!1x7dP?nTsZF6013D7qe+|>MzFp4(XgV&I0BhX@Cw>zA&(}{QZBWq~?oEI=k6Q zVtj#}CG8*xE-E?eL6cf`fr(`) zM5n!2OL{TamyIAk*hUMC-OwbMxQDa3{^8n2vr6ChKw?Kr2b}HaKUCA4T2PpOkW*7h zo7m04)vLahc7qsv1v_AUl}Rn5{uU2pcOjZ2 z8$uhb@t;!wO0j7pwSyO9+i<=`XH_`hkzf?Yw&5iUc!L0|nw?@!SbI8M>K4pjI(t5* zsex5XX4f8Aw_w@)rHhXlIPCv1_a0zU71{gny%QN2Bnpaz9)ekzfguUV?QTb5MUsPA zhhb(2iez+MSH;!c)!**wnq4yz6tkE!m;*?VAc~4vG3Tt``&OO0UDaXeR{THT^||)Y zT_?P!PF0;cRk!Z6`YE+;g|T-h{a^_);gN8sGDsom?zjQ{f|VgT`+XnUt>8f%VYDz! z#w^m2h8;HOE-Vb`OuxU(W0`IRWmdGneNs_}y2W>fTJH*Q%XFnFPZ6bIsHu+Dj66K8V z7s~0EkTa0RXd`lZ;1dSDctj`%?pI9WT2uE;QE_D|QY8@yn7t#>86gpUF9PL8A~41v zP|(I}rb7-DuJWJS?dHPOYe1x&a{oRlWU}K;HJtXH>#TRzZEJKIc2ZvW&Iub`m}5<|AZR!_Ncqhzc>SoMF2_#e`W0-$;*G8v>Z0Ods@PF?3x7_BM+Rc&Ja?}4Y{~yME#jKMki`Z@| z)8qef{vQ@CIv-_@>?0YFE?ys7d+eePqXjcqk&D#s<$=wqNrla+wbv)nu+;26X|TH7 zjsNFC1iu;lzJM)HlFg%kg&fh3FoN-&V9D2r9g^0HKLg<1dC zozxp5P=$RRwL=!wPHmVqXLi_uswyEHvJm@R+0wR8@#op;E$!#ZgKgrX2<$T|*#jj> zcIBT@lu5FR3>l~^k^dSFz4T($Yxv9CdBpC2!H1QBy7CclVFv2hG#RK4C<|j=`KBm! ztdHtcjrwuFt6DT8Qp$bUv9F{Y^-|^e zEKpq7*2~vH1j~xat~T@k@!XJ&@*Lyfb#k#(JKW%WwTb`6W64BwW_QMmCB(2-bNPD1dpEW^&;N)wj?{kJ$rXs8pUKZwSxc*Pf6Ea}F<}WBfXR1$wnXrKQ8ngaauP}4! zRIMwnDJo9o&{P1(;!=0ltP$<%Ro&4aGT|`sf4!-z>}uBEh981QR`Hj7YH$-H_eeIVaZ`7YsW)h3zvT)De8KsDOVg}sv_eBBL-{ds7sVQL0KUPFdCho#z zz&+IS*%uTRw)Xr&{K1j}VZMp~CKCcmXM`~So{%pfD82zo^Rgjq76_9wL16xi5ROX& zL7LrltbbPBajNRB1%zq7Ps11aT~nLgb)!>?PxpM*Y;j>(nr3(1ys$j0?OGJf;I7x1 z_J17R6(x~;L1|tV=Ic2BVYMBX0rPdAqX02a&5U`0>Hnq@a}nLOG0Y407YvEwjbdJa z^&e%@sjO$P&xm<}Y5!A+St*hR^MWS?^VAHO7qI^)=4qKRFEst%G=+KmHZU(V@o%Dd zqnH;?7R=KW=JA;^FEsPNX$mtHNrQReeS&#f2FwfD{}c0!G?;gVcr?$2CjUD_VGbbb zN$uOsF-C+lRW?&n2-PCf|0k%P(k|W{E}lXqjeF4)F~AIEfB^h5IQk-!|Cyl-K&_re zX0U}@q11$PQl$xa)H*FpBe1>erg>BNb*2O);@@%c+ zQ7{9D#U}oprCKXF$5EP>1!D1O0%BGM5Q{ngBZxU^K*->-_;FEcj;h6Z5Y_nLviPeM zf^7^g*Y6rqu*J>;iVV1~H~rrnRe4k<&ERtViQvaxYEA~+*K_!fRGpc`Zc0VCQN%scu-WV}o{^?%c|EKcM-5>B!i`OV2v9G3yyQr)N4e_nDzg2xY16D z8jkJE+O~61hF1i)IOmQ371hoOL39E3JQl;nQc@Q;J|OCylS#c-M7^`qsCN$-5yGz` z?rb(wrQV-dQ_L}5QLv*|6bWa`-O3`~1aJJT+LQL}HDlJ46Km%*^g6M2R@CE4H#H4weI+nPMlU=F?(RS+8Q)zTCeI}$IYEKWzK+#-u-)* z_wL_kVBdcDEbmj%rtX67vr~%1;w79uvQ*l96?v$MVBSLXnV^*(e}a& z+TL=crLDh;EDqhjh@p(7?JcK^wwI*Q_AHczwzn)m85;N!wabUE(hhg|O!10K6Z-mT z|3vdj^nYb#tP=ku_5WF_ynoQFmb;yf?@?;*wBx5uKM8(@e|7x9TwN5oYjnu^}&QGg|uY}UhDl~t(9QoZh==xTc;z@B6M*`o(cC=*H=MlJAZllW?GO+ixU zw=w@i54|Kqo!>Uns`HkQg3@a@@shLgVb%Fp5yx z-60LL>d^st6e8YnJL5n4z82I9)q9*-Z*GhA=7gUOVlC%U36dHtuL+jC5gK7G73oO3+316d^kxHy`r}nAzCnZ;sij&@XCIr(e zi_%ynf^t;udQonXa#9b{|&jQod+S8w%))Lqm!JZ`{Sk(-`V6aP@Z>O83uVOv4>Zd1WNCJ zAkd=Ob7o;0p@Fe5`N+PF@jN~L0ce;Bmw4ggJ&g@Qieti*!^=Hp{<~NWF9D=wXlD0t z{>x^zILV#--9ZGy-!s!Tk`n!FH~a*5@|&a2D@j-hT{@kN7d)ZD>y^zQGJM*MxpNLs zJ1AApIMV!oiZ|(xj@^6d1vc#*O(R}!uRI%-N6npM>1-emf*dG*v)}u^O|BNwUkIklX5JmWV0wyMu~JAZ4DQ>*VL<23$RIHmlUV%nzT?1vetAP0 zzm(4ZzH>#$4azUa!!I!wN5CXq=YQWrVd?4c;_)bz!7J~37bWSHHzawbC(FzC7Yi%Z z=D;e?3rFHkDI+oL$v&=tdf_4u>D%r%`#&_FnRYGhl05J#desBa*5n{{egDO_Hfn1| zR&_0wzJDFaY3U_tEFFOu=zqWI|CT6AM*w&%o&{!)maugMe|Nx*@(=7TDlJhx0Ds9~ z>j%c7ByGJU+17s-R!;w4MA2$LARYfWYz=SbH2Z?;O;<}2XK*lSWDG5Lc{Y?BvNpyC=G!)a=g8u9~eau>5pYKBMbA^dFR@>)f2|I)`XGJ#?$E(y<6d zS5b4CWB^>p9bz**j}gJ+UQH`1d-v&E*?&MEwmY(ynEc^ZjtI8k&=Gn{RiHF3*gvxb zsTndA7kQZdUsy9umfMWeMTy2mq-Gvw|IeDaHFM28yi!Emsv7k?n5WSw@nN(6ajR<7 z^Pp^oM*YZc;LMu2HMwTwarZ}t+31QWdP)TRSTp4XDN&-mt4Os|JFEhUdM14cKG8TH zG5HVOr=AbP9o^Tf>BYv;)$8llxyx9>MlzCGc;qu%PqqIiWY*T}Yc*6s{_!8qW*mqjaViB*Oi_vBa70ZN;AbgbZ zU&7BSY*uZ(eklk^PSRGmp~*7(KWqB(>Je7~R0ofe>qLVfH=^^#QIrs+R$<~12 z{FfkZ&jjLI8^l+j^roEq>tXn?KrHVlAeN>9Au<1Qv;S?W*ML+a6Jhzqo=${yL?TQz z0J66eiO4f?N@i`lT5jrJvth3jvT2PhSjfhMQz~Mvo5%vja<2cf#x*f>V1?lC(|LGV z3mD5k6~ipeV3@~RT88o2dcBf+PM?Gi%P^1i6~o-64AYXO)WyDP58QB*$Bq!C?&1Ol zUhIp*|9qu+ODGpsA}dpfr0yQOKvcXdSw-myAG=eeuSioZeMP#N|E>r#yxO-qxf$Mk z{xdXg9*D_6GGrHa?ANnWyjJt|RntHTTy~0A^)-4XQpP?kHQBor-)}3~OU)!Yd7?WI(7&{H z^zj0d;qT%#u{tMLJBX$Ok&E?t%00Vb%VVT>oN6 zxGqiYAFg3Nd6vLh7iLpucbD29;&r#|JGd<8WGZEd^`HF6k<$w1JAOZ}W&pjct$L5F zYVY{hU^VtZ2SqvzpRoQ@d)eg0*J6kd)!fHbv(coYUjcsPY4|e#MfXzh)uWQ3TKjnW z+3*|Xn@4@gQ{XN>+w^8-eTZ4;Fy?H0wVkFA4znIY?le^!M<(mn3`@)@lM8 zUfq6wp5h~%@4r)~Plqz|~w14gCW3u%?dJL-M8Pon1 zxk_TJ;sHU*gOSAG0YO$xAT~mU*-4CNKC$amJr|IXL81*%x(xTvn)pux7;m?Mj>&BR z8rx|CvXC^vvnSems0Uv|*W4x9Ow%<< zLgu+HqUc7>DAd#3hwkV{sL@>h50{mVf9~M0^mJ7^<9y<|lTebK>&E2H6@;wn=gj^G zg-r=QjNK}$w>oKI9&5B?`$AW*N) zN!K$PbmBNGDuIrU^Ku%;iD(Eb4i^i&tQ@Bt4duYb_I<24J1jpPu2T*!8K$BumZBtG z=jCMAktD&27loBZN72KK>kNSFSOW^IlX(i?NRnV>KSxB=uHT+cH_)gAsTnItu<|^I z@f|kf#{5odX5|YYXU%L*Q!~<}SAHQPHmjN$4fD+D?z7s`HR%^3VQJP2ohU=SyznQK zWW8)ot``X-UpQV^>3X5yx$K|Wq`MS!PpTJebm6nfXxsRi7r98b_JzA0PPMSoj3DaO z3}PhH(F>w(6-&1u9VmIoc=CeDe`t-CJA5#nL{r*1^(C=DTD6~}2qB%=Ln5~>UJq9d zyvyoRHZHNU@=l>sHOTzG(9DfGVr~@1BZ(50fyuL_uem{fG>yiJarYh;#^apa8 z``Ql|2p4h8^<4j_&Rf&yd<@FM=hn|a8G6N5<#Y4Vus`5yjMYg7QNG5Qrx;cYZ0J*0 zU*32<;q8jm@tXakSkH}DHlp_&IwHC*hBE3c+{8pgN8&~mZEWL|C87`cRwDI|bG$e7 zROMvDYL9;~Akj_H0(ZoO6iHK%F3}u|=Xwbm_ zlN|1&;3PTL4`@!x8O`A%hwrb@Fy%!lg65@z97#lTZye|91y>20>je$zEHp2f{7(YS z!Whb^R|KP|svC(JUUKT+uZ12%ajaGuC6x0#Grn)6ykGzF`tkvd9b<}7w1^!9*$V4T zX8)H`!=DM(1Mp0|YMAAGk{zsT>l(@{>&xxeC8VCJXt9hR->@jR-140&?Y!Vm__JF9K8nqbR@Kv~Z*({WTFsAq45G~;uLcZ&tsrkS zPIpq%$Z{Udl)bU?+X-7?{c?YYb(yg4s_Q|s&eB*RYcbZ#9REXOVc5?1#!|X%EE)7( ze%LOGcqh;ME;!4p*^^~4=$(~4L>?DSY~);z8&&bjE)I9Tsikvps!WrIdBLgqh;~~a zGG;_YU+m%^*t?={|BAi?Dk^B!Y)%7(3@wxL1ko#I{->R=42b3jQ@onif=Hjhoz?JQ zxRWz`HzNAjMx>mpJyFAQ`eHdcCdzK!8D%-RZdHS8S=tC{vzhFSaqOA(pe5Ipq6uosxFx#t%P+e5O?B8DjTBcP2uOj|U zcpn~j%0oU_HbWMF;uaz~SyXb0Egp$_$G z=(tyw&n^1ZhmWk`In1L*R*kGaBnQXKM{MZCkFPaxq@CgUd4u@oMKz=V_cyuV*4!<} zdu^h_eLN1JZbA(l$CD=Tn$!LfmFM~EgIB$pwzLs6c~xuDw|8Ik#P%CF()(01@u?H> z3AcXjGe?ssxW62{<<&GL_anXFt?Y-x?3;O;D7Bylg7I~e|J8m#FB=#vm1DwyacM5r z*4!;?zs~(%VGruj%c9f(!>_)qX1>Kk*+etPRL-6#NqD@a2MRHy-CBnN>>r9B4WjnyFq zV5qu8X3YnMx(L?5_#rF_7>akbl(JyF$^0K-e3A@9YV1wK|KZHYqd~n%-47g#4;Bz( zyvg=YE+1y%g8WBL5CQmOST^nj`)7{` z8#jr_ofUGc1w{#@x6Sj92}t+HZv~`<0_j$PL^>Oi$^U9PLRuU{ndUa6qd)^l@0k71 z2}qX)amnWl^@4czkoaO^XIl6YL{N;tdFLX>(yXBe;_C6or$q-Q!|M}5B8d=*G4c<8c|iv9l*amY+3|)lqm!G zdqW+$44Ve?6$Wb7h=KfDHIQf3bosgla<>NZ_pWldGi)kh8Q4o-Y;s1EXPvSw}edNEaPJ?07E5X|`7ed(dcJ}U)OV~s$ zsDWU7->m;@PZ|TwUSKZ6%#KdV#oBG62BQS$A2f3?GHi;mOo44}89_ea{FlSb7iyR( zm!TG_TBF_vN23r&y)QD1ARi$AlQim`vGm?Eg^P@OA6y46j1#_0GlCd(o)UGw6m`;# zdLMjfODlsE@22_x{Ls~TE9XDd`O6GCZ>_MnG%F#OX_YUXAzVb~t?d7)^Vex~mc-On z#y=c~zLrt1sdrOwl(-ZUQ%5P68rZL2Ig&Yzzgd%MHZ@^fJkR`Kp%E==-2@*A@4$x@uK4#jvEc+!(mR@`8Xi|X1wgH zrVGms_JVQQqrp=PL?cPI6(WOrQO@m-9AbGmJ|9>v6)f`w3+Zeu?>ltFa$}5TY%H7y zL&isKowCGo3l{jodG`_7V?lz1SVlw-sjeAT6&+h0tBUG6S_N5@C%8U3+mT6JkHoQl z5nL{~778xXS-3u0=Fkz>O)-=SLtfUAp8#4QfJkQn`uO}5KsU!w2Bq~yKoXODZ1P_TK)2y_ zel=ZL7K8Yq*#kmMB7my(Qo>3RBXB<1-BF8hmc_AF7F;H9t`j(OBx%!NmJB?ofLXu40<=u40>0q-e^>!XlID9$gPHQU3J}fA zr2-O&qyoOU(BaNFEKbx;m%&s88rc`-`G0NOnb2fDj*aTbx*F(3uoQ#6frpRGEj79T z#N;pa_hmOnAIA7Yg0B$(oxvM9&%q5_er4ic&7B(-{)L0r)uQhaJX0KzdsdAqmk6Ia zP`)7i%B+8A9q3tia)+;a*b4|}y!e;b`m()W#s|5y#SLfqW`OAY zZDyUn;rxd>e@mTV!gM-+1OHD*Q##-F$!$-FG|~B+#iH}KnRLedKje~sC8)heufOhh5wEUIVD-BL#e9aIuUn2&pbs&7gK-pIWRwOl)+C86nk3 z##DSkB1~2qqX{WSLaJ}g{x>a(LC9nA+Yp2-@=*xWlG`K*A)OV3e0!&(B7>0I$Xlfg zQB?IT2F~kFS>{DvZY5)X;I+#45Xh?6X;M+lII?fL`N3j{q8Mqq-*NwE!gA~;_Y1z0 z69&+9ztf#lHkM;5gxtwln(lY>{{(k-lg(HrpSI$~az>MNPFW7P^>~AK7#|#`C4 z_11}@AUkU(Fp9N*XyT}`~&8H zVFT73W%)I$uY>8WPUi=Xe{8@%q-nrLoi(D)528-GPUi>2{|RYjkm7Y&dChE*CY{cY zT>q!eKV;DP$H5kt7C({8s6$@mq}tN?$6C?(r!+c`M_F{HKmHwM=-fX^r&Hjq3UD}w z+C#NJ4(CuVmDT1wED8@BQ4@`i88>_^GMzHnej9QqOM0FkvHoGnW6$$={C@O2x3PZF z^V}~z59zF)=cj!fI`%xb$53V`+bs?Q4KV#==D!I{s{`z0P`juGdBK$IF=690Fzp{x zo#<;|PZ0v;r)7>xgz`ij>-E7BfpVunA)O7y=f=kAV- z`S@-P!(A0+4neq^Jrq232`FKr>Rkw+5CQY^e;h#s^JM%%fVo4!JSbpDXMy?ItbZiH zERCUz@*fmcVCwKmFrPaz2xcwcsG{~29EcaHWDll|*9u_L*&rH%CxuLZ9HfI>z8MovV`?wFj2CJXhIXShozg)h*9x_ zYA|w}JlrdgLs^39SH?eyCQD3D$8QIw#e(T}!9+R>)2}leI%2vjhB9hp3`JE{jmIZ2 z{p#W;rggz2nc4x?V9LiCST-Fo z-5o<2wbdC#6(+qhnD4pxiD`X+_rXRe0xN@4v&V#$LAoF733)h4TfG4`iIu^8&%{5v zWw4r_jo$-EHw&bD1QO{iNPhbiNcY50M!il2MHM72sv=bJ<4##Z+7PBAuzY($_K>iA zYamr0FoNl1lRLaHJ8@bMbqD=oUeAuOK3wjpzf1j+Jz83}w`AKoqqR;dwE; zLWyW2U&GgxB7o@lToJjew_Cu)LX>x=BaMijkKYYMHwmJ<1rg~iM0pQ7bVPJt3}w_t zArw`Jj>ISGDDOR|ED^oLWL!6jKx*;0>=7Zgh&oE_>3tm>C`IZge~crEXja7U0-75I z&0T_qbQYTYa~wLNxj%+7ZAiweBW^*Hf45VXXf_3t<-rA@nVd74aP9V0ks@dcx;T=E zW@Q{3m4X`t%~C-_Itxv~VGbS9JP<<}wY-d?3XL9K^9$ViCz_Wz=kGxg7+z;&k46u# zVb=E@Fh~rq`2{BbpXdx;h(7=@w+omD1PtjcFoo{;PcRS0P)05CqNoC+v%dL-qix_m zv%V|XbkyVwSzo-t5>bEe%G$n|du(fqvU>g|k@d|lWc(WrIzF?$tM{`KW4k~XJqOA! ze90-wt+C$+e@`rx{ylp=m?Iq_oa^z4=2X(y>o&SzpSeJV8`+ zqa%}9-}f1~?8VB%jAxoclvR*yBVw}xA{@72nO0?ecP46hguYl#Pcia~D^Zp+o}bhd zqt>1Ah`PmY{LA(D&cS(dC_U6YPpLbh|A2`D>MORrHMS+hle1JnvC02yS7z3?SujVY zxXAj>`TZ5pCQ6xB1vJ^qF%hSn|3RiQoK`Q$r2;f}B9aPd;`V=>avnepoM%p?|7+s* zf6QvUvV&Kn_PGr8c3>dWblC7it5k51TkiLIXhbPeizROV$E3#4;0Lc}KXJMra>1M* zkh_Js1nd8a_KZ+0Kd+Q!oaH2=$r7h5lNt@dffC@tpAJ--R1WMrv9i8m+uLGWh$cFz zk>9k~;m)Ln&z%*!6DN`y`YDmD8dok!4Qi1{YUDR{=RfqYbNN7?3J!tsx?HT?CTcK> zhc(6i2g};@umf2p+~jFB@n($w5>0%x+r)K(Aio*r|0+T7Q~Cr!ezRk&1c6R!EPwjc zsQ{fw5ac)e8@Mp3@pGC4f%G`dnE#|YKTD63CaIC%?0H*S8KiipeE5G?h%}?~XQK1Z z8FVgfWpiPY#9Q#(uu;NAXHxS^+5c1LUoz`l%Jom`{EN~VPo)-+87_hnV5O(q(v;34 z_gXnw3W(073q|K&GU<%@Zydk|T(5)vEYI2t}6qD2NQq zLlTOR&I(1E%kxj7AVZOdVko1sDk!RY)1&c;A+7l?nuP5YEGr(tukA zFbG;X%6@_8-{+<6C|fq55HhJKG?4)G#nx@O{KIO%x}%)_c*`DAKpJq%o4^H~4g$U< z!a{%Wcf)9vsN;(|={lX3U)j>i$mE4D2h0>M(&@C~`X6=nGw9rEUyDogQ_1D!e?HPd zxQNcJYDDM!G&&!HvZ&2gGXL2tUpk#;dFKb8N~DNwNuR1tr@nt>zXA2V`!>4dgw>q* z826Fj4LTbS{c2D3aT0Bk`YAHx~w>om! zl?iDh@sEXQZG0IJ-6M#W2_n*2h}ua0SLlf7(HP38&4&ijQTPK6(B^XoJuBxTKIo(OlRYBtV;@s9cHGFC=pkTT8v6_)D2A#*RNJE+QS`MJ<4*$4g97JCfkQeAPTPwd zI>K2--rY&Yi#{n+fwq$Ww<_pjZq86a9a14*XAcL79l%jr`V^W^MT(%=QJ#OWDra5% z37~mE&^#e%NN1thaY723Azb7}CIJW~iXpXG`sk#yVYqG`wd z&u*`_A^s@P+%0Gx6*Q!?(fs1j5zXT_JsyO;7y^G_(4(ojCrx)$>yPaUi;15Iru4NM|A1=_-eg zh@On0OjjEb^JD02c6!z>>of7Tk_!txSQd%5sVXGuUtd44vc7lY^J5$oH55c-%#0eC z@4TzSoQbzJC<3kZ#bOeF?kxFVU3)$gZ$AY8@M`+9EJCw?WUmL)R8)eTCGWGdiT~B` z?KAOK7Mw0KWaNELPn-9t%q{UonO5GXLpKLJ6K@|dKfAXe(tEP9icf4r5?FM={vXRX zc;G#K4}iq%J$k^RFV0)5>V@lUFlTT<@n`kmm>FPR<7TPY!ZUD|cWS zCm|%7baLb03i#)~On#~bLDYdxeh%n8U?RM#@pF6VRc%BQnO1hf{Ku+cPAl_y0SPF?)HGj6n>F4ItkExC}-bE^fMwRt`Vn?iqRUSXk> zG^MXhDgC1F@25)dmnhvuct4j?dY$JF!H-3SLp)uaML9#ID&L32rz=%>a$7i{SbMiDq`Z_V%kTo!t9do zt3j|CK8p2OH#8Fo4AJ9)Btg_M&sR%UAR3W@$i%-1qE1M89|X1_Iw*-~S9}fi%JxZ- z)Y?dK6Tw~2wxubZANc8-w?vxg+?C@$buP)I^9vT2e0KNV1805mOOno<|5N8? zX>^upWjE&k*-)E#4WMRxd^w-%RqNN7x_qnRnn#1SgN(xo!!*5%MT=l$P5a>mQaCIef>0pMeeLSHbYlYDCG( zjDQi4Jz2|$()}TaJ;(1BEaNO0%J|*=eWxtb!ja&7*~5qN`+TKI-GuTAjNgr)vB2@0 z4T+p({O)mp!=2-I0T01bAu`5qeF$DwICB}lsf9Iu_gLX*!SVZiEGO3NBi6n?7i%Xl zinVv$)8WqX`ykput-3-HRV&m8HMQ%ZD9a|_Of~u2(~jS}o`OPb&CN2j=3O!VvENY- zA{4OS^bT3Q@t^ph80mp_T@Eg6&82Btvr&isPjyN~opj^(ZcT+z^EBhrZV(7uyP5d6 zxf-9?=$`K3v)SRyh}h>PGCrw+oFse}e4UZa#BIl=f*x zvKIk_eOH3)xvoCv%_RFI6WQ*>?~*-V<}-mnvX3*8MN`ODM0}=%NcK@ivOlMgt?c46 zX-TpVGm@G4Zvxv3oqeWkNVYX2*}EyyR(0}YA84`fUn$bocJR56mt^l|Bzr7{?8Tk^*gKl+os48A{!1viZYQ6+xhd`K zOk{hS`EP=3eS4p~a!K}9Ml$aIcb#iPJ3q!fiXeM4Bbk~1C8TZK(T}~M$==9FwmwDL zOKttw7EQJ#6B)XPWFBC3T7l^C42EcR6hUxE)gwHC`@C~ippokG&8$PYM;1Id!KB#)+$v@Z13{t=Me zkd}ncztRsAinS^5x$TZpu1`l&(IbVVEz%N;HOb<%B!{Pv?1-eqB2BU=Er}Wb6Ny-y*2t#5O38w%O$Mqyd!$f}D)9lFGoWbeOjI=~RHK{t zQ8qMCwaP?waSGL#VqfjY1y##TRA&7zfop7$uXaI#sznB>NEe&ux_X{-9Ie2MLS-oGhxt!96z%Xhf1PyeR}V=68AJM=EC|}CK_Ch6 zerEl*JzrHs1W0D{e>&~KPnI;LjsKg{fLo+N&Sgr&F$&~@?ssbkdZKvMexeRQ4s@Vkht_4M&AHhx>wfulDa4BI$IAUZZ?rLuw z{=-FPVw?VK|E%4e(%44Q{r&Gp8TMj3(Kf1c-Wg9!m6Mf$Yo+`9oBY>K%0tJa)IJ^- zx>+0-jUsRdv=+EKCBr=%gV&dO#Qd>8t~0baJ-IZ!QgLipda~tXTj8J83TKT z*iR4{3n+tjJRm|X0H6O~@OuuQp)j^nFdBGmC~y+KVG1=K{307pE4|>Cyc}$;)ziGi z)%FEXpakVn7!@k``hn3rI_JtpK%kX>XY%DBoYu|Apfr(oCqmxPFZ2O*$ zlI-YqPIk6a!4CTF>*A;EPh;rLl(W?q;LnmA3=kfz$Ereq81Z(-F6j1NQEd2ucrF%S zwZW~fCS|+7-WAEGr`M*_{QiX|P_7G8*Iik2M z@N)cVq|P6vtWq!jbY2-jXhIT+!3LTA&ss+s?6Vls1U~6%z1Wwz*vmc@PMZww)*dFq z#*_H4C>vmS`LvfY*s=H;1{-{=gMtPd7+feT%GhRfVJ-$sR;HA&)HWlr*i^EsG3sSdWJUy_UYlGG_BE+t5e7M$q_ zqtRZAKM$iV7o$CIj7CDsXu(4cA)W85*than2#ogaZ!j8<=Fmnf%GGGmgB*2ev?0Nz zGCII$m;MH$8FfmD%Qjl{K1Udh_Ii8;jP{rqZG|x!32C&pxY07A-2ctkvP0$pKenSV z@1O>JPEwAHw(|owx-0`j8SrCf{?j3x{|v!Unuij+tmrbv`40#D4#@*P^S;n8cD@6Q zvX6>#YSaYx!v*-WiE6spMTuF5q2!LM)3O};1>Y-+c)q77$}4MoS2j#&{LN6FN01WP z#ZCTSH+k+;+!$<;o$H9~w&Z2am=4=U8Hv14#tlQpUv>-`1^>;#JF*Y*7%zAy7kA8Z zRXU*vl@Or&Gyjtq%*H^uReWY@u#YkT`ezL^yXk@rbczPL|Ks4FnCGtXP$ydW9DJf; zTjE1qoZ$VYdG6~I|9r0SpwJz0!@^aAt$D7`k;m)s=sEON7LfK?w*E^3|&1dYvetL{uH4Y---xm9n3r8&$OVO0sG0lH4@^4IUd!|=Z90AwAM-mp!_$SJ%3kBpaKjQV$?~M^1`8{={uGMP44E`arbo{am z|MEQQC3SJYGge&`{m5VE^mvcjp@7f%AOU=A1Mt|l935?s-+{m%?nT@Gp5CT2gSjGQ}l>TE?r zMomnakjoC!XVo{%YN(G-pE~!1X}TozqmJkW`r_4Bl?}CxKRBrkF3o?KdH%)pb4A|7 zV3>qrX#T@=^UsE{?p{z~s%w!@Z zm~LV|+^qj}RO9q#V6%r;n65MU(@qf2&Y)v*XYiDk{U;YcWtRoq62&HcV*&m|e*gp5 zmRB~^HB4x9qrEDaUH_kKXa+otUE!E;>E8HHv3<+ zUq#XLzQ8lW)Yd#yzzMl%fTQ7*`sCdTHy|GTwAi3svJGT>JQ(@k;V7#Q^}Pyzc{{KD z-#hrQVwI8H|G{ePlBOCZ+8SAbGK{vmsJ(Psz!FTT3ZpGtE%`tC#dcm*7@FP>D{`0CvH zU*-;}oMZ||pB*aLfNy7_hgw*MQz`Mc7x&Ay*V`_S2Rqny^f#e2lg3{}X{@Kw5te{) z`2U65=S7b#l$_vsz6uptL_T2s|u~DW`3FoSnCL2R^;p0 zcW_$rpWwy5E6Jg%nY+OYta&^f%>swXkccDmFlA0nj=>G>#*TNi;YRu^cq+8oNROn* zclp0vn^MBElOkhRI>ML~*%Ds~+dfY9kThSJS4J3`a1VU?EJ)#lFwQX3V!H-pj7dhC0n614nJfi$*5@uQj6Gt_;2xyCDPoeJ*xO)u@!{lq^~S z7N~BLV!AitFTiw9i0NK1rXwMVBy(9EbqMKh-^PA`Gms8RxF2#cow!>xHlj+~4Jgyw zjojyz@e)O$T>dt_GWw)jP|4$ZIv+^ z32C%5al_i>2++gNaunfnD;Ho3Ul<(Upn+yrPd3mg8tCDS{}UnW=N>|-M97MYRUs?B!uQ)X zL3qS5R>-Oogu}l*a}WesA?p!;gLVwM%QA|9d#TQWKyOwJGG%zw~_;-=Ho`g7ByvYycj+mSdwk>*Y)) z2zUNNyOz(>CaU_ENB;klAf$C0PY_o2uN^R;VM62QR5=O4qs{uC?$0aWtCND^@}>av zbHjD>&xWz?=TKp)YmuDG(Qi4G$^_vKJlnE69R)d;lhDtlNh{IrIrVB3Yvo*yInoi! zUhTqQiM*@{u$JW7s~K2NLJnHvF%LNWnSkpWj6www))FrmRjl-{=+}25#+`wUJ)^1$ zX4k!DFccDM2jpNbH~&UJG87zpo(&36^1L5|f64)jFxo%0(Xs=@{2L5n`@*rGI{bNl zO1Iz)IlcqFaK?7=1ufR{g_?q+Az#0sC_(IK7A^)lPhQ83v zQ@O`(Dihm7U2Glj5A}`g4CfcT$VYN{J9$^*>tJ{Q?>MNAsLzy^dboQ@;_FiFb zKA&Bmf?goIgfHBBtjdKjxj~KTeK0_6gQImTbfRN|K$d#@ctMbhSLb{AL`>F({SyxR zGQ++ry(_^!TMQw4afk`sz@84#EsaA&AOZ*{94IPuQ_0=Y=)*^2!+RXRIUp)^#W8d4E;PWBdT>p`Upy^Q%=zUf?`5Sy#2pU=mf?d-fpt3MU zLtm6(PIy;(JHKo!9B-&gDh;@q!|?{Q|7BOMdnn!+!tsU&!Iw3*YjTYZ5j8i9;2w-i zz)He#c9!Wf$VylNqG|3(A4gDRbOml;o@kzbQOwY=^Yc0ex1c*`<(=*Yx9G07&%l1Q z2>%*=!8}tFJMpNA5mk73GP8>{{MV?FRU@kpQT=DNYK2*^<*~b^nv+FUGymz~HQ*Z_ zcky~NoN$T1y@UN`>bQ$c;!AUs7Y%=B^ZF5f&6k6T3d~J!BC|a^>jCpT|E&b>R9oXeBa~>hd)zS-^YHEW1Nuh z`>90LU3UC=g6MC^FH@CcaYMby%N%v)>b8adOFgW zvwAze205#z*$R+gUXxeGiq?dC;JY(%L(M5>{ZF?ZTJ49}&vM8b5`I7DVl|$)_!~3K zRHs%{`oY0M%e@&)63ZdsHz^m(rBSJr`0Rw=amPCXX}ovhYhk=+#CU6s@kmJH&BG1j z9k?&w6*y&d4KfXZ|S@U4J3G}m-Gl@g(nMoYj7 zQ{QkT(tPj5*THY(a zK2ZVFCOG^t0Qs*5@5+FLH1oT;R5|RJT~uHcp%Mn@hmIm+u<_|do)9Y|BTk6T!RCfP zc}oO6o%_ETkM&57$9B|~m_F5tKXn9M_Q;)F8)gBSZB1wV#|V1&H1X#YVRSdjFoND) zMbIt(pGD9!F#ZjP4l0S-T()A_(?KgP^lUFvHCM_D~Tt5HuD+&sYSG z?8!`NxMt=Edd7=kX-3fGogspr@iR)Y#`Z|AvHn`knVp5z?ka+AX-Ci<^&X5J?L8PG zteSAhg1c1E1ZJA~zhdJtcJ371>UsHWVza#9)Z%JKudDOoo*rtm zk6`HeFB^2~cLQchs4W)br7N5=8|}2vFonl;5UKDS_Dm%AB|( zjU(*)@%0#CpQn2xC|;jeMjV>Z8e!-B)gj~vJ3hLLJimhxc9$GS*zE9_5w>LxZO$HH zC-kXjD{b_#X5i)sI|etrW3I`6>gHd8c0UL^i{uzY46zHf`(?wKhHOJ4DdsX}%zYAu zTy+XiV}KEVN!)kVEs$=p?j(mHn`eb)XT>^<1jbHhdR%4;>5~618Vy>{x-TS zhhU{A+chPK4kw>gJ-{DFr(+d6!oqC?X?{eYPS)CG<)M?Ke9_UR3OXf&CV zOI<_@-MaYmsVO=%+r!7gh|&S$5_PEPQ&--%q4DOHgHPh+AdXL-?5F5Zx;x)}r*!xO z-+O1(#mRq9(V_WnK3;CAbeN{>P*>M);DFwB+pTQ|Z@YH*3-iB>u@e2m4&8m6^RIN! z?_ixUu)emuZ~gYGi_epE_)m%sExP%5m_zAsqOwEn#6Go^_2rE}fRckhsFQB2x;Uj( ziViKi`T_THqg~9_IyBUm59m9w@ip!obih6<$0tv@I7Nq6WqvSI>o8O6FtB$;UElh~ zkDAX#hc8_n{;i)<6-|u5R6>UMQ_cF9R%8s8H}8he)aW;|VThlt zhxqI?v4?mu#i=)lDRxgbg-k9_eb$gbDOQloxklr>k@(O23qO+ae{hre zDdK2~xyM;?=ioN{M8BHp1-I#0aqoc>>+uNLHl9gv-uU}lC&g6kNUhu%eEyBCpr_ZM z)XDST;wy&nw>Q+^VSBt{m@`qB;f%k9l1>mwZ&MOu=MfQ>fKmAW>wo{7jVY4Nf?^r} zu`4J~GiNwhm-$;KQLS9f8Mu~oU~r%xe2$-c<3pV%p4Fs*`d3x=2|4HaQvWbtE7AoG~ zDBgH1xes_Z9u86Og`&3%T7Sn9sR}X5le<6;Cm{HHnE=zh6x+b4XuemYR#C8?ny&}; z?0P9GfF4ezrOp!mXE^+sI{GD!9#CbaC*#>CojS^nKc37QHbTAiyr$~#YP`LH{Rg`L zZ1P9crw%uC{Kt-tPF$sVCpgZlY0IAZKrc8>hgH8rr&2;x;w0+$hk5=>o0&;;JUa`+TxkwN<>S=7d};Lw0P}5u= z#W%rRtHfNJjJZfiBFT>bpUHnFTr4kM=;;GeksUA0#avRUiRJskOctY7^U8RMDy(|c z-v52+;Go4mK(3{xJqh4r({iy`k~*ctW&7B_#yi4jw2yHzPff5|jP|lI8VPB%GjT)9 zfBnxPq>tst5qqn8o#e%faxogVXTZmj7VnE1FhHm>HL@F~uEA$_;|3^0gn(jm7*0kxc?_owRrc0+# zDd8z;V!Qura!f$meHz~k+pQJbZ8o+eA&KVUhV9NZ^Z!I^EsQt8@x-L7_u@@*v7N5v z5u@S2+)>8MMs=wtT3Buw`aQeibaQ6G5S%v?;DMxRoGv%T)THYt|-EdDAi z_o7(tRbx34(sKXA4a=Q3!Xc#B6~#+DPjxKEd-0N7EZ4?sMX!q3}A+1;(Z|Zrfz7fzh&Bcm3Adbezj2k`{TTP7#n`ST}RSUxs z>Qkq6I)7J3Cz|jRruR%hJUbT?rq!vG5bc2Oe7FCn{l19fHMYTevES>)ek7#*B%nKg zt)naL*CgJ|^L8d30=j0o*bi^2h8K=Iqzcb9u-9vrY0SWHiNThd9E}@paDhAi(Q=<+ z(XmFoVLR}!@Tityvztn|+LRKNk|Qp6!3qaCP4;CRC+Y&V(yZw$u7Lm(S|1b4C`__28PMz=58{hO({{?CcE%gM;1-?Rs0Q@ zY@?X$4P!DA(qvrMM!jA%z#-(?eQmTO(vqaZ+I`0+s$R3(tcA>bb`H=4%yD&&+ z+WmmOwRjk=E{BJ(FOE1kE4+MvF!p`xfg5@KMR^)$7L3_~+R#S!?zoHRIsCah?zP}O zIhhG781Lm$**UFXP!X*VY&Erxc?hE`@PLrpD7KvnRE7+{!`qbLgnnqn`X$P3|!n&*$Ox8cp34WAxEZP5#Kvr6Q8< zao_>v5%n1MgK|z|2@IO<5P+cdZ8{bPtQAAIySr1XbLI)v3CkNxI4} zck_RINV^3eol~!~#e=zjMn9P5*<;p9PQ4z*Qlx?9j(_a+)-xAi&ik9DVP)Uz8CXwh zSXb0IX5)jo-2=$ZV{61i+M_Xzg#LXi`r&Ypx@|wCot+{X428#2N9SO!j1OsFY1Thf z^Tfh%p7&F5cH*5BXKSNn2Z|4Avod&#FI;(%Z8V<`X?G9)mN?P#Z`;Kev{=g*u4?9p zEd#9UnDs7WUBShV%gq-Ou(+QIz1mgBJN$VJ;Y&djy&m}=>IKnU%toirqOE~5rHi*; z^|7Nt6kW(Fdb^rh0ik$%wiTC0?G^7<_yoeM`TQ%#rk=@Tll1;qWB+g1`|BO>(J$vd z0csYImC36a|8gwulV&WQB9iV!8AcO*)D8ez5kC;-9~@0wQ*I&9N4mUn+t4xywh&yy z{67&?ra>TU`q!L{GR$pNs%QcT8jB{bxjv~h;AV~{u2~-Vj?T03KtBx+v;^=aczWUbOQc8cReYh-hOpf zg>yOY89nhIbKW!kB3%&ULr0hjoy^3_{y5I1uJJ9P+Sp^X!LJ+Z7@W@&b>q8Bf&dn0 zKYEew=d$4(K@GNHZK-`$$BKYELjeu=Ch8S?|K8WDX^FqFT)MA1D$=!hwjqL&#=eY_ zj>irESx^>AI|1KN+J7jGj*8J#>&O4M^0td;ik0qq!IhzC4pR(HHZb3=w(-M|({me; zv?0k`y6Xj8|6=3slic{FyIzR>A4%QyrhEUo0UWH(Zea{u*xB_<)7eRPy>KGRu)FT3 zx@+h_U!W@Nu5q>G|JeU;=M7s6U$+nx{e@DhPJM;j=Pd+_3Isv_GzjQ?aDzoA{_U^2 zYpRY8SY2s0Y%Mwz9ND>>(tw+}yIyowSeo56d1vUZ7u|}Itg-&dH6{-%Em|e4`l;@E zG#OD%CQi~mu%v}m69`$HhZ|@Yx3k%J+?v%TxLuyinC%6(7pl&ouKz&1^rEiOy|@u< z8Y~^j-3V$GfLjvKomJ-+hmIB1ju!Rq4%K51z$cd8X(>HG)Nh%eEHa1opRe8>OF81Y z*PHg=&tv4n<2rDpArdn zDk3#{gUNqrW#)s)D_o;c8Q36Pqxdb#loFJk^St3p$FR(KejR@k1Jp}Ij;$td=9LkJ zCfozxNzQYLS^v~ZGL)!~c9By%k@M`*BnPj?*qR+0lk>!2lQoT<^X%8B60c$0#sf9Z z6TvsObqrJiTl_mX(yPg%eK77EsXZ?n#0aOOqn~nj0cGvxqyK9a< zu*6NLS;Ldww0+lk8K+sqWaLbY4Nar%8;Z&JpfLl!dYp_ZHlXr#~ z;FhJ}OQ#x~Tw@1<2)=O3iz0ZC@`dqaWMY85^``9}&Zcb&E5Rmhee4f6Y>!j_!q;SZ zA#Sj}73V)GJc_Nym>?3|0ZeQxCwajgIwoisIH01gq4DQZB3^5+B~Rdq#;b>q8d)=D zT-CU$-x?QCz44;r%1}iU7rfWhtKop;RryWx(}~R~Hdq2i<8d<*fJtr}63R2N!TXfQ z*g!3q=EFcJ7cB9%lW>X|Bf=jDg`4w%ok{BF(6MqX<-9*#~4&KTui$En!xCg%b8*Zq0$8d*`)+&v+z=67? zo9M+`CalhZ;7RMYBL@;H8SVF=mtxpU5nlI+GzWG z!OyuEEmb3hPRXJbAaChGjsA+y=uF(y@_vVqM%y8dxT6c{us^U>E=I!?4Mt1aA6Ssd z;UFRH{=mD;{-=b)UB_2jD8pf=)~~r3E{#g1#D}SgwgYT%*TaqtXuR*^I9evyEXI4+ z7>|TB-d}N(@EeDa#%mF8?RgQ>VFPdLT#Toy*xDLcxb6LI)I;3uZu9(4!fFem=wQ{} zq9`0L7po=dQc6&^yWRb%1DqE7A&%_^!K-4i_l(6zNQ=o}aF5AwRM{5?amk-oqV2dRVJC5?ozLhvwNf7klZpaM-A^2UU#?$55;srXLR|VO|+8 z{Bb7kTHJ8@dp~qkV!yU1nip>0jppTIw=^o15}(~X?>pKNNN4*g{sD~lI#tA2`hhVX z32D63a6{Ak%=&-A*;>aDtal?Fwy^D(i}AuHF}6Aa=Ey%}2tLR>-26vPq=DYw&4EF? zE#?zIrh(??Vz(r1N(oB|B78u<-#q`Nq0?qR$G5^}Tf}Btjm=0%qIp7y=YOs6oDSC} z-p=#7lMdT<+vQ?29!fW+X6(3;HPJ(~(LkMrw^5g4wLTA+{V$1DdOg!u%4#t$T9Av? zlJzMiGW!gg2h998VYXl5AHr;Jh}k|gW+NfZCOMr4IsfG>O%pPw(-4iwahAyGMBpu1 zrx;K$yAZSq6u{Mad=3%1lhf(jr+;NbeWTCBf}^a%;C?R&W^$9iMI`(o=bISZE&NAmD71>jiU&2Iu{2wC5~FZ zNdwKUo(z8)aSCoU(1$C)Kas|m>3Osu3(Ucdie;RpF3#V#oJSDrunzT;Mi zVo$&xl-e=>&qU0S%xR2=P5yI;N@D;)V`+>>O#a)H2HebPj7LU^^@pf526@vAd6a>x z1MXJ1q3I)B|7IFvNOBrOR-PX*`ES)SJI^E|rZH&l?YZ)-@Pj9Hq^^lwY~$58=~kW} z?PsNw3eksPd-}tV*Wv?pVO}Qk=uu*eMAtGgTBgr@_3<*1vOlzInVELK)WGsc&a2Nf zO43d;ewDm6Et^emsaw=Ctv>szJF4$p(SJZgedCATXoIr_!mXCAc8t!|=kCG9(hs4Z zyI42>Y#2wtuMI0qwQlBEz<)XRe^}!M_o=nS!A&*0(k-!1?PT{Al|&g>_Bm!8Vq9l z!sCM-{@kb5BRDhhVwp3yi!W%gmM=W+%zu2oOynazjAC9UB1g?+R|Ph#S%3#$c*4nl z_}r(qDTtvPq%UBfS}Yf{!D+H-Yv4@jQgKf>@xLEMZ?ihU#s~EX8oXULK4e>QdDLF< zZiP=G{2CR!-GAaLPWJ2cH$E78J}H7LVq zVW?X4n8zVTtHNjjS4;kngZFmc`0(UQ7J{Prl=9;-)v$fCznX8aFS zv@nmHI1J-Tv+?06Gya>>fSWm5cx2SW3ut# zsb_@MP-g(Zq94swV9n*x$ z@`+gFX!IltPMCI58y}KaI@A>Yw{3!;nv+FUlm8#;HQ*a=O!%M$PBX^ec*{Wx^_GK7 z;(dthFN}iejnIZD%E@fbQ9evN;Gsq~w~j;zf`d1b6- zO}GcXld1G`CjYOsQ{oNI*K6m?hBFP>hDK5gZ!(vD-t2$ZhU8qjT`*5J zqGNx=yj-d~r>K*PSaazWogAc$IxofhKx)*xeK8K-l3Od&K-q$29A3fxpS}9fzqkododx9hu4iDMwk60NI6=NAxTp6`g zY+Va+-thfyVs~PxxRU!Hsp66}r zyieX;3%YwHJ<9Z}<9c}AO9~V6_1ohLh06iL$+Uk3poCG|zTM@ikiER_`3Z8by|?3> zS+&z^b|_PpzJELId@q*~oDp;W|Im4`wO3!{*xB>z-s35|eNjA%4vv zW+*$HMQ9RUEGBYRl{iJ`qs+AZMElm>ehE2)jglnnfg8qO#r-cFj+F6BNIZS0S2K`u zntDUU@{~-1zCv_LNCeB5w1jepE2rsHu%lN*yO;IgUo>`TyTOIRK)hzvvXDsws=XB? zXYv}Qc#_ww`ZXk{*O0t%uNkCuU)?{XGhVYoc}+LvHR3d7{D=681pzF`rb<|u^)KZ% z#jN0|FuZd6J-z)BZZp_x=e3>PFl$Qf)G2?doike}6G;-c~CR24aE&HjI_>X_{0>;^^A zkk+1O%Tc}9+C(qbl)ad5u#v=&J>`(G8X6^&?qMzGKf2k%ON!X{txT?HX{;~qZWk@7 zg@jo_)TVk{DSj?#F_`gzjHfF&!gGJp~vbxHuJ%oX%yRM@nkLo6vK;2l9x+Ev5sJqVOKb0et zP~AVHNV;aU?jMdM`erzEWbWj};ns2d4{NdHjO>UHhN5t|^=AD?D|x;Ut4u?04qBbP zj&;+g&6zR{L)DPz>?yUq>ZZ?~HfPq!LxxYQpHk~q*p6UI#;g@GKgQ8)KA6ui!$Jv$ z1!Z*7u)J7pJQpy8-+1dG^}EcLVgA-kw*EH5%mq9#G8pXo{U(+zA$DiOo_A478X z{XRsj3$W)pj2LFgxJ5eB48jdm8%+G0h!|dgzalIt=w?L>{70hG@0JJ|$VR%g4cPyh z09(vw2nm47MZ7L$qe*jD_uh#t-XDd zbS1H+>qe~qShk|BQ&A*cbK1DxE?T$-x>iPHGkt3>k|c*a7c4n1nfae;5rxYjN9~Fr zP+l~h;CX3?EvJ}r>IZ~!1|;MRqA{4|B4H2QFz8Ev35nr;inzhleN$9inTl3PM1rA0 zbX!QoxFZ7Ps#W@Iw}Q4_GaYuQkd^<`E*DnJApToeNeg<(secvzWNJYQ0=?X7L7Niz z*@Ea3S+$@|a~%``P^_|AP#8V21<92_0TtMSs&K=Ko1FMhMNcOTkuG3QZ%D!|2Q?Nw z5f>Rsy}rC-s9%FbPa)IR-jJkvC9&l{FFWIZ@wEUqmqpU}&&#LUa#Zw`gB+9p(s72D zPL6alFTdl+nF(D&IXk&V3Wsj0Y(-D6IOD&Ho|V)lP%RAX#a zGt1cx%EB35G5entvFg#+Cwg>5PqXxpLu@1wvdw({EgW)`8#hWOo#$rezeBPjdhzPy zik7M2|3}?hz)5v1Z=mO_vatA~L1xhqY+a&fIgmgECqiUdb{9zq4hc6#-2KMoLfjML z9^yCdx$zKpAs%Ay`?~AQ>7FyQ%q0BZ@4d-yGQ&<+)mPQk-PP4{-s1c>UNoN-5buzN zI+dTdmQM+@xpV&F(FQoC>C{CjAJR;4DVPEKXmu+$8n#74t<`&yGYZzT{%2CEv7 zz!Bbl)hbHF^;J%N36<;ZRGEme0ZsxM;w?r2@({f5^fCD;ym3CAoT>!8BsTEAW5s_8 zZ>j<79V`CJqg~?uk-8=&%6i9&|I|Pdj{piVkf_U;uZ%0$_O6j`Wu zA2+g4?xjHJ?4PxD2|gG*k}JI8-7mp~S*SyjWTATMGV3}Bs6*VoG9`M!9iS3+C}1wh ztlLsB9g>Ji#I0lfQ{16SoLcE34+SSEbC{GlED59-$^!PfO9j%Q&c4W#l)z3qIxh9f z&+X!5>p`7m0~KD9@`uG$coCy(CzRlerNpf3{y<5Z^{_-0gw=b2{~yW)Du`zDfkVlQ zrMRCAAm1As2f0W%EqZ?t^nXWFZ-_{#_lL!C-q2ft0fl-QdVjcs=zVkwy+52IdLNraZ&_;o@FWp+ zEcMP^3%y4>E~h8Cxt$$17hhOXRQNE6|Bj^!D4kk`5955Hg5o;{rTH>c_^7q0aBK<{ zJ{l@29G^r5X?7p6{u$^Tuc~epR9Nn+c9iE1h&H>A4viE)z;WrH`MDTxB3;`@JpZrR zv)WdnUU?4foj>9J7wUX+YMnm``oEKv z&NDWm^Cv<48y0U-=TFWSolmB3=gmm1^C!Xl?_{Mjfh5uSlaEE`lT+yYDf@rwd}J^mc%W(8hKM@0*n!*5|wGsoIp2Pq#FoOX;8z!JmSB;yt-obG> zK;`EaC26gn?G-Q2);a?PQz-FS5dWU8S}Qq^MQOeaB|f`OlsG+=5}g0B7SBwggbXg9 ztrt*ds#;tLQ7sQHpZALpY-Dixd}^G6mA4XFq|p2Gp#M8lRUW~BLcI*VKfet8*kaF2 zq4(z;|ETv_N%WR%=jWdZsI!#bF^Jk#S6+Kmp5%(uPzmv*5l{Rg82`>vO)-XFQYiC9 zu>N_LQic+CLutMYWxiM>%AA!#nJ>-|WzJX1Xa@X09ZlvPU)(OrobNPaM4NP%Z@WBHk)L5W}s(scPn19tj&Ya-`nKK~g z%c$3{#)-145|tG_zS>vxxH3t-%Cz*WQ$*C2T#PUH7FMmzVq&>(PAeyCU@Sfv9m0Qy z+4TV^LKb59ubBU(;jc`@B8LC!C&Tbr3qk4pHhJ1d_%ICr^=1P2+9ZHuC=1(s9jt#| z%iV#>Hsiaxo;Kq$l;JCex3s3O7l?w_#;slqduFj1UMwX#e0`qiaBZRvV$ZJ`|Ish5 zNdoV7Fe1EnMBFvp6rtcvV4cyV7vywx@aO>6c|Fp>pQbkfhSyzxg0e-V7tKy^m*mM+rQFC1rcB z#m^A_ZP5Q+!&?s&^@O0e=p*8gIO>wT;~*2M&~NGgbjoWID^#5F+Y=3^bdD#yUE432 zf)B$fzr6`uSnIbXsr3q!CFGYy;oI1XaGw@3KyE&M>y?x5xS_SOoP3 zAznA>S2VA%!!9mnmAD6}|1VbM-JKS9*rF%?j#A6!?>B$J0q`?iN;q-JipDwnH_RU~ zZ$V=N?4+K*z`hrlzUwL$)R>r$1EzYHM(XS2XD0>XDxtJ9xrpz6H^bsoXL2fX@(tgG z9zafVlFHf3@BR}fXI0WTx+=$@Dhu^OiK1)Joug8=f2eo=fdgSenE}%%MqDzhy2T zO3*i_O+r79f2Zu{zrRla-hgd{FesYwDY!aBmfTbUk|nbIGF!F7zey{t9A-- zy9B`acS^uNa{fz~crXRvA3eju*~dVDv*n?kifzC@a{Y%c@o*BrF_cA<`|)_O@WUKQ z^BjzcD+abOlKyyCy!--Hp2?Ay)Ia?ox;#cx=daqBsec|2FAr0rd`eUQJRT)!>cjua7($D8Ul`E3EAZ%$jQsSe@4!={=L^#1ZB*wFTmC)!@(i(kGlnrhoe zA;7u)uKS1K!|1(#<@yH!Kam8mI21d>@{tquBPxxMAaRI6eKwy<9d9oU9{#mmDk z9QRXoU>kz@|5J%}sT0vb{#!dgMK9`ypJw)WdM7EdZRcWbN- zvsf=@Yg@5^B{YLv#X>auy~<>xG>fw-a<-0WMskvhJN?f6FLbA;Q@hjeod3~dpH}0c z2kl>NX*~S>iFo-17@#~zOEDh)&in^Eil-CY$=w`8F#I1y!b;T}SY^@dZyPbw;K$>W!PfaM{xB)z)Rzm5&epKi8T4e?9*ksaLhNbF_{*Wll*t zkS8o#YG4jjMs%laCSLL9&X(OKnPX5(LpO&qnEg-8|5T0Z=Hi0=T6BW3?4`)~j7%sxXF1)y>L;AMA zxc(6z3e;Be_=vt)Vx*wx6yCi{>H#Ikc}gHX7thotw{?#5BTCZN&n4RWXf4{c`u{wNR-16?e*5A~MjOi$ zbEdw2(Y^LxP|it`-Vt*KUaWc#*=G`S@L11X>c-dYfaesd>+6b(!GwTx^U86rHBFvZ z3iobt3};kB(t!MPlU!|+;pS77GcLmB(@+83HUe^;63EFKVv)Zm($%vq`#QyHft;)*7N_;;>LIR^bu-A>NMB6tI$5uZh!^QP`D`oWT5_H2ym+q7k7jaQWYqT7wn%}0yC0czLa5AKd0KRbDOb*jbtRXkb9=dOl`W0Gb?hc zjfSe5S3e;gD_CAM$(5S%uyJ@KY(!E}w*N(RBzW@NO7qRz!s1IE8?f2CjKyM3jjf}} zR>DWJ5e@Sg|HU0xX&7f!@TDp1quD1ZSda2nx;RaI`XjoLATb|H>|=pXa3ix zUWpGDkbWd@1GvyzUrpkz;xhSt1n8@rnyU0;j!IFMhv~T<}^bI zjvLuQGPf6!x$&UNOok&@$|)IH496QrZ57nNX2VQwoJ}WWa^q|qgl3AEgj^0J@oX`q zCXRtP7yMm14__=MhG}(%80NJUhG}(+VHj5pHTj3WX3yhuS~35}+I>SAh8w%|i?u%l zZ-V+=K)u0LLsh$Zu0q}c;Nq3Y$`m4jSWqbt-$;ZgzE!ZRNdGQLwNz^91;PCHJLOwk zrO>|B+g_!}=0D@c%>pqQ1V~u_Yzp5cBB7H&$AWdHv0ld?opJt;hvCEUqSmx6h`xPRnq_iyv$_^)z|2mNHIf%=z_&D`1{5dEQm~^ zlh%(z1NJBy9eo@tr22^keVtZD=VYUUXgZJ)owVs_aiW{YITc}%%hE{5*{$u3H|*VJ zVw^naFw z$p8PIr1qr`DGc)ezsGZQ+ZMD3pT0Tm6LNI;cd8s+PGOM$|2+lZLNov0dL0Bf&iAoy z_`eC5{Xg6Gh9rP@M_JTEVUYjd5YJa1G)UZa8)uN=5Ze}KoUW#fQZ1*gC-tr6bcDMs z@I0q}OZIG-w0efaY^$PSpismdS^>1};U&l1z{6|#n?Rg8W4m0 zf0kZ3#yRT$rSuBUh^9U=8bh_?T7Z9;d)XGwOnXYDv=8$C+7BmL)!m{W3WE*>W~HdJ zL;v&ep!UF#ml}?YxHi_v-4#fFemYDW_wQ|NoO24wOzg8S3~y@MT=^XJSVSLRzWgFQ#@%@ZMeV6Pw%A zgq<;PSY$Zv(|!wEZRALF*k;Y<3%(RR5^sp_#QYD(YgI`FH11XdoqBb7eUURT;%kIP z(t!MSi~R6($tQ|*=6^!zrwh5tWmGx?(KDv4qHYR=N8aK(ou^s+SA+i|4hl-Wq^?5- zBo^1{d^Q+hgd652ah({-!GxXf)?X5=`E)?19%n2cNUJgJ#3%>S@vvXj(|H0mx>wFs19%}j)O4j$yjfFoI>?jrkt zP@45ZH%d`2U9JOP)=PF`y+{hvXHY`DV6PePGmvJDBIF{SXxDBQ zrv_MQspreX=g^7zb`_`WM7sw0Z{5xxwD_PU_>q<#j4Gm1*C79+8BnD|2TL5+*BNZX zz(Z!#r3_jnMqs8&;{Vd+P^D$wXs3Q#mV-*8ZC+H#RpSE0DYLMBqTO{XvrI#kLOxJw zbVMaGATe7I88H9JzIj7pHOic=+rN!Do3jd3PWjNUO7LL>!QHt3gN~S+#1SQZ(d}&w z6=j0SNP(J+l!ucMGktL{(-)YgMA8?hM$#AEYqSzzozO>!65aPOmH2^5?6c;G+ifMf zA1X@ZCs9Jeg6`b^i_T_PzSE4RQ3`wK22;CUnA*i7#4rPa%!HoTk%go$x_@S|sWaJh z;^xJmKXC$LBYn|>>)#=U7>HY=yi3>N%Sd1JVE&hlG(Ux5dMq{!qth3I58A2?xEQtD z;~a2d?G`A*bWl?GKXyZszUXm}fGUWmFWR_@VFwr}UWw84MUNi@VnHHA>EwEL5$U~> zR7)igz>9mPJK50tFKdFKr+4p9YjM3Gkvky^A;z%d7M*`Q(*K>&`8J` zR7v`xSNk}5&||b!BF5wtq)J~5?V`-YJm=6ZNM95rO25^nq@AVM4u&E48Qv1>o_WD@V8W`^g+bOlmL$!>KaOg;OgituKBOKZ# z(h>Hq6D0=vsw1R_7#(5n@5C0G4Lu~zrV~6Q&PH{FBsMz2J|it+c7*LwBw0t;XSx_>a0566Gj?hs{|q#a=&wtx1{L*gA_2Uk^Zhg_o)qa9(NKLp~C zM2ON6_GSK`(kqixO9hakBkVg$q*p56>Hw1@-EH0`*>3Z>!Xs$^mF%5*uLY0z9`Nx_ zeqPuQxvJGJ2wU7G)>k~1$39SyY@;`Y%0#N~Uvcmh{DKy&H+tLKv$f!SAr;)QZ7p)* z&PL~~j?s*FS{b;}@u0WaWVmB@Sb~Wgf_1*v&eCeZ95JYJfcqzloVd~4QEw7*_<8E!>&e^oM`$_&64PmzU2?WwzbCH!NA>m)8Za*J7Uap*>KKvS{u_z8~Q)iLN&t!)c+TUb$MrTlFf$6LO;Tn%-WC* zmI|d%p;a)X_r>$7LzCMoV3_0RPA+20Ek;|MIHQPjvV#Lx0Vg?0WzM;Uoc};o1ar<# zO2n9))+y(l4~5D<1V?V2e9n1iWX`$8x4_D#7J(~zvYsQ)>N#>jOa{^dhyF+4rnvYkXXsi3ZWduW zvpZJ}!G{U=BY`_PNk?}I$^tGjO&acGwm{|4oe>SGP=1Ni#_(uFN`@N;U!4@~vN);7 z4$H&eGf0pOyV62OXDtZgxV}ys!&z-~a8`zNhO-Wn{QoemC!Lj?B$72FLTDu9hO-W5 z{-3ovIqs}QSrjn_CpS~k*rid@(}b4iLFLz_Zm>&Za6#A$09eoq z42VJg|KE3MjBj4pFfRrp{_|ZL+aV?x9ac&y_dl`s(EdIteLs|1ExplXpmu4z<}H(J z=uxZ73`a&?7>*n;;P@9upzeB2$-r(2l7%cQH5ug1Fk3>e#FmhHmqyuv7Mn$uC9oQo z;P|Wy)gLQ2H+E^1Vf-`nDYp2uA=wGm>UZW75&ak}M&zC92wk>)-K z@t~kpm<vkQB&ZnjCvS1KioGrVtHEDAG(oACv~mcL#Jm9o04pxG z7;vM(_@HvCoo+_w2=k=g1(3MUfZRELK>}he(*z6$;uaiXZ8~aEf>aHeO4U?i{nN1R z8k2ELHOqwo;*$Ljx~EWk2oBp^M#etQ=CR)NYXx#YE%@L|N~)klL1L!~{Egi1T1EU;EzElTeZ4?Mf7Sg<^d z1&!(bure-T#e=_amOrc=r~LZ#iYQT&BTDR%LW!DzrV>AZ+W9ip`5k;1O4M-wAC=fM zi4xT)3nglT@o!J3S=RDf2Vs0Kgz@n>BHTf&marILlf`ghbDP*gv#CS9k&TMsJ0YtZ zX9M@r695u&k)9ZW^)F2v193O-k6-=8Vq%!u-eQziDxAgZ|GDOxzv1;E|`=Pr4YmwO}@W z!@sv_vdHP*ovC`L1MNUrxw-yXg3OC0e47wiI ze;B;YOfkIIVf-J5vU*wO1blP44}Iiee6WSHcLJ9>?*CxiGduP0UKgx?&sM`b&-%4W zmJ80ehIb~`D)AP$bzJ`;+}Ww%V*C%`o(W2)vTVjee3)>V|0LXflfc~;WueljE-1r} zc3;9(fkZc)i{oWF9qm=5VOJoZ+0k4@8g_o-D$*#%Kg7Ln3f!YG{>O3aRip!tj;*qB zk7E2w-1{fN-5|JS|1-+W2JUq*b9_6>_D-vua4wHHBpjJ-Py1RW&c%VO3dFHhP}7KkqeDfxsjN^K1|F@QyjzfAG-AZDO`HYZiY)c{{{?a z>jPV}Ng7RY%)#J7mtLI2rI9(LhmBb!YAud8#qO#p4vsWM?B3?zj1}EsSE#l~RsryD z1o+|u3*?ArjU9)gY!b^-c-C0t zf8xEoZW8}`<+j(s#_+7Mr-BQc#EK-IB~4;1&i^arSHzn{Pc@hhPTb=#>=`r(?0Ybp z#L$}(o5Z**0lp#y@NxYO;JQida&*sMY~bU@3h;xI0GB2)Zk_-?INl^u3{URg7@qXw zNUbI@E{OjRjyDNhY@jzeSXVYE2hE2@M&0iitp8|YHi^iU4bmib4Ceo;sA>{zgRKO; zu$3V0V>$6}N@8nBG@8VYLHw)Put|`O+N{?Jn+&7GCfuO^*TifRn}NUULIz)K=+Y#1 z4CcRDGd78CX`5xIVEscAucaJV&XOZdVyC^tvkp$-Svz6;k2i_zT?zBEhweK8A4Zed z=@Rj*mCCa^u#|c+{*2*`?rW#V#Isg%2eb-&^0HJPm+!X9;rg$X4a)x%DW9)zzQ7g* zv%F}s^B~b=Wuhh$1n<0qfIK1z$RQf!&ijebBjVe%du6LAphGT(Xrc^WARtjdcv*z7 zca~b$?|@5_TOthnKa|+VRa;9sVE1Y=LeUqe8FgoNL<)U(;r<`?Q^zK0?yNji$}T+r z>!>o{VFbk2Ed0P13rxV9g;t7g$FhkcU!Lm@^!|-pHn=m>?EO0nPf{5$3)eu^G&eR5 zFDb2oCwN*82$icP;jHxM%YeT>7Ol0KS8# z)?$;;*q!UA?mm9p==yQvM~~T^($tOiU<~$6Wy5Ed*UTDT(b&{LX$>U}6{Tg3C80D* z8n2~|59YsGnMt@}^M4*5NURcS&;<`92KTbsSxptqjn!%CLe9N}dvN|ibB{Tlv*%D> zwL26=R5_`@@!0>TMx$dBD^MzDJobN$1C5^A@tbquzQqZn(S+XM!dgBqbuCZe`486e zagMHKS1qA{A0>&j37G$d(uyf<&iI2*lLCTy!ij?UxD=QtTxW36WANk^rH=_0X^9hF z5X>he!7Q=zgdb6c;ll~a=~g3UewFvDJh->c@qW!_JvGlNDX(ar*>XL#b=pkz>qpn& z#=5#GW8i{?qkaV-3fdfRc=*JbmasKWc5BCf79RcxJ-iLRBlRB8kw&U};`tWcX-;85 z7xGqniBU|I6E~k2^nam3S9`z7<#wYS@3-_7Lhc(1O{|+db==f(6UHlKzd;%$iAIwu zEM(Nk^`D1EkBLUli$N*N#)NWlART@plC#*IpSpN$(y2jfe4}I9y z@iwHdQER6aG*VXzDs47E0;PyHlNtYn&OmLl{O6#}qoU1oq7CT`Z6gJQpY<`b#tXvm9t8kXV$jZ&1E}z zOuCY>tn*Dd-2~z;g1{{+ocbPwU*c3K)(s`-KbvQkr^A|T8T%0Tmlo@Z*l;zO(#e`H z7TX$c6rI{FUUVM*@-2p?L5|1HxPJ~l^0ByxR}4j2o`2CpxB3J5k&l>bD)T>a2g~XR zd6PZdhM-Ifb9tGVE4!O84w$dCm=_1k`w>K(d71``GZ%wRWB(r_#{|pSjq*){$Xnvb9q;gLu2)T& zv~y`?Nm)s4S!rc8+^@PMgTY+Oq&(4R*C75^J&;=r<-g*?q(@SF4CafKMrjQarbg1c z?;7m?(fu18Y%sNk<@Af?Wc1vX@t>oHD-5xi*;|2B05^#LHN#Rk_Z)AW3;}?PX&%P3 zis}Z$Q8O!BY&ndhdO*BU0lNkJ|1|eH^w}M~1DyH+go_F|;Gb83mT6SLZWox9?o2{o zbgp-`+_(shuFj+aLXD&XcKg-h&S+{bwazvxK!(@dgY_@1{UW%+6?oj3>b-qjILasz zA!NNjVV8*$#*Z04bxH<~m+f5#8&Z;l#JitqAv@f0v!-}|p$i{HXi)sGOcdAb$yyZO zJy`$O%FJ-Gv-%N&<$2;%dj$E9z^Tv;dS}Qe3?yeLB#jL-OVQKUY}Bc`0|hxR#~ZDB z59a^keTGwq3hw3|<5ihhyK`Nj zqgZ?T3`oIN)aI43f)XF^epc2YnX?db`llT^5E&(4cc7WKSm z6UwljPvXSa<+wY=!4=bpm2$h{@>sm^orQY}L9qUL60d{-f#VDgBN0`K)MWiYob@7M zJr%$3#S&t(JwHMz+U%r6n~4^C<%t%jB+;TD7(vutTZpJrs0F67+0O2c%i*vfw_CD@ z4Cm~{{0H?ug=(nDFYcfTD&ACMuVdmRp@!nk3ggRAV=v5qlp3cbsv$MD7uJ6grdfl2 ztbYJpSU`-i7u!FL5qbWraM`=F$wf^K_q@mbO1Ox{_O1aJI?!oJrda_a>Q8?@O*zmy z+kt{Yhnv#q1kSZZV;OWAnD^%RPt2#K(0uR54K8`T$z}U1*E}U$0?ds6i1~~pn5C=V z2l;mWtuKzMrdQwz+SyUpOp;GAg!c1hZC zs7}8B0@Qh2)OkVFA)TR4LlFOk4%cfG%BTaYfjV>W1+~*K#gefIW8D|QCdXa4$+4%? zosrVcW2W+||GytMT@QJbDFb=KQX4$cv=*I6(?-K4j*8^G9B(2C z)_-(eKri=Q?_)`7B5e9NeRs(uY$6IG5FHz9EghNbe9fCDn|+|;yiBa!xh~LAtlfBy z#hqc(ek`N5xB`T#6&XPqA4XXYGiRs_Bz{ywEk2DAq>=X@aRfOd#R$?=V2mJYcf51h z7F}NuE;52NjQ|&pAo>B*_)%W`UEnlL7C2`L94l-J3LRidD<+p;r>qFp$7Q5$x==7j zp8FhdxzFIzrY*T#^*|p+peztzX8%vjXC)c+rlTBOs+sW*yM?o4)GKhe_0Ew<9Qmnp zluK3D%&J11U)rL>GTVkGJ$AyBso-2V>Ssux6m}+*w9t~o4ycv?G&-P%*fucYJuMv& z>5LAj`6`Q!{b_IVR*PdOsvKf>d{RcR|1H#JJI-1LQ^eMowWO~N?zy5il50a4jF>O3 z%dD*}Ow^^oe;m5pC%QZ?x{%J$W!6%Qj=J<2#WE!0i;X}sAG20jWmjY3?RjT8^_#OS zCf;YIuM09H)Me+oDO2mmjzNNV)R;P5M?Z=@siWCNmQ3o^+J6#ytrfkV6un4i=rwz1 zi;jBr9fdOSke7PxjW_COc98!F>!^o0iwhq=7{)nJuYmJ~<&JP%07Sq7_pM_b%^O6v_(#Whg4y~WC;X}YS+@}z-0 z(P+O>mQ3bJzxEc%u6Ag&NNJST*ugw$ppn>iKfC{9o^*3n#if{b(Du(v zw9WR&ot#D6`Ry(4jPeJlGN!u;lKER%WlJ$EyymTwbsZpCr;wCXG?ZZ#xoo4J)K!oy z$0wY5{&^O6M)~W!kHX~WN9lLKnMCf2hZt(W@JY}}pe0v*NL z3l6ck)3^3lWlU>J<6dwz%7%@5t=+gKVqU=hk5T?vDWd!ZnE!?uGL_q!eATT-L1Bt1 z!W*Vv$oPj5^VvxvW`VPi`7gpbTjjQ}hbjlee6c`KXm3+mF@NoILEbtkAea|&{3GVG zQ(#_rsmVoQer0w;fp7^h|5q@d(*kCW|HOQb!n`&q=0zP%X$tce59hbHF)ykT%;%)Q zyl6LrOZLT}c3zpX>s%Z2qJstVc}Xx!ShXnV|Ic%pLA@Gj;C(nVQF#=EReEO9P*YY@ z(%gs*J1uMvLRdAPan<-SyQ{F*Cn_U1vbYW4uxHVt7>KmCj)6#jmQlyLD1+WE>MUMhm1S<^OYBBfCwn}nV>VJhZHP~$MKJ3~zv*UsH;5Q% zx{KeiM6sjX6<%FmL2Zx@aaH$}mgSfZA$M|?rn`jx9~z4t8<)uHim;Gnb?-He5Z3%xx}Bvj zqi)#F&r=x)tjU;hkaRmsx&Fy+=e!h6cp28e+1VI7#QH7y^i%wWVx-$ywjH>zt2{qR z6Banj8U@bz0w-Blx$HPoS}`Ozzm48C=;Q?Rvg-x&`Kd6Y|BqvC&y=jQ=Y7BJgAWt) z_k#JtB$#KQEL?VZH`|O4{Wqe`D zV4C?SPJ+6y>{gDqfBHJ%dIId496btb3w;iJ6d|e{!2DNeI;zsae;ukkC#t+Is*ui5 z<$&P)OQ^~~@>Zt@QBJznDkY_F({}Kq^tZwFrW}o z=D;BS(SoQS_@bRmwEt6UrQTq*a#n@(VYb zsK>GFfQ=^UE7RHO2xX!f9-j?~;sUjPvP4p?PX0Sk>t#{v9Z`#PhFU9_{|xJvY7HiD z1rB{KA}>U=(Hj@{}NVJ zXa7~`^t9;os_4Xj8an+e%fd;WHYabjY=EMsPXDs?r%oHZ{bY9Fj{&lg1jrCOI{|`DnBRc(0bRwOp)2kL8bsFNMjN12%qDm)OA3Vs`pE~{KEtJi$ zSRY)NzD`&lr2Da+kd4#HE1$b+1@sl=Whu6NKYR#~d_J5v(sSdO z`RQwf1S9GwzJ%-NLXweSJmdpQ61C~>zX5HY7j51UZAfQmb13(}*d8~Wyw$Pr1Z@u8 z!758_{^Dww%1R(zIxk~wf)!jp+t%jLApa41TMz#=X!ERS^O|TwI#Zj^EQG9^5#+6= z9)UJ8ydGxPKeTZjZ>c=L3&ZQu^tI8$Ymoc(+dz^rydDr*Y$ZWP3xr zQ2|HnWN~Niw=;8P0|^(&_XGcV1$4%Z3|0jkVaNYmg@2vStu_M1ojc76(A=q!RKStJ z`j>8=oOCWA0_zBh^yfzg^Iu(-nT@a7#q%`nvK?GPH+I~vV^nyMSqAttC+@ zsj>kFAsRQnJ(gA{63J^Eb%Dj58Q}N4Ph?>Qd5urf?}anTYY+tyh_#On^8eb}^sqNL zYg9Xbprc+oOWVT&9mT_rKFrdQ9=5O!zgfplq=_GWEy{*XJZ3j>Nf8{)^Dj(mT$mz7 zaP)d3MWFY2Jhsb0CqQ8%MQ}`4aA8{Gq9iE-fpg530_P%?)+kMyO+BXGlvWHW&cl;# zT_Vy1^D*52Ma&naz_f{i$$~RLn8#|B3nH7BGJ;m@ihCV@WX|+u4++ zFyA;}H}t5=LW22N=KqNK;uM&V-NWFLeE~6_yL`tpZOq4V{3qsBNia(=a_l82!%<+B z(+uj>80+5R^jJ-$5sc{RabtBYmRTBGUa&(jGJfYU|H}=2vkeqgz3E=~#2|fKkpILl$~;G8&9;H(xn$Qt5NrWL@X|A?q~Qo%Dxtspi_6(#GbdsuowBu%P2l zvJMFQ?E#2_*oUA`IO#}B82g=G{)gyy-jIIhL+N)&XY@NK-DA;50 zCkN*rw7scHnU69my%DG)$(@sLwq#J18+bfQP0NuA`73=@xI=%Ey`9}>`!G@=r^GC2 z)Tob-&9>fKqS42q5$Oz#PTAd}qekWAtsa3LXe5h*rv&@ILvOp09koiEY=@3IeT}%C zeNz48Y2)kbroguf{jQKjG2&~dVEx;0Y^u}O{{rf~C+d75>X6P*=hVR#9o4BI?*Sy^ zi~W?$1Wuh|m1PCp#Q0og>b6CEp1wNB>_8p0vrlQmF2%^~oO+)niQ2^c&!Npa(dKi} zhIFPjKU#FurqV~5{z4^Ul zJguLP{VU#^qRmI54e1PRPCwnEW8GByD5ElID5^Y7Lc`PT`lmLxpljDl4BKFdJAG{s z8p6}!Dc;TyWQ2xi46;D6YWn-1LYa3&nNLL-(izH}v5!SZWomqsDK@J{`n5BH^{>#; zZmr9bJk=asiTH|EIF8i_9Amn#YIdSdUl1S(-?4y-C7ZGH?G5_J3;b%)AZpjtmb+ zADJ}qQ=3`ljWUhQ+rRI$wCAqNZ@56M&Z)t+dbdGZ-4ZizfktB6v+VeXnYS%iCfl^F z#CK}vb~+_O{#G?kTB*EBa(v=~rv22gBQ zz*&!4+?jbR;toi4kQkY_75}^fv`nJ{&JN~3y16pmzW~Rm^%*p1be+;D%?b!Lk_tHc z97`j{`-`cyb--^1tbq9OY`g#GobsF2avS^@-agjJ>A>?=bv`e2sh>1$%(O8XZ7f;? zIg~X)%R8ss!p3>!9(B3sI?(m1okh8ssGjYSJz0zD=bT}&=frYzma$So;#TL}ZFk6(H)hrr$SOySzcYy+|=?#K6I-XP>?gHbujF?y)5pWSoZV|lY}lNmWQR^BWE(P zBnl!B9nTH&AEBdNh%Kq{dnm5W#M+^bV(s%BOGi4|k~$!;4)#e)KW`YyhAn-D-O@V( zGK_ZKBoyMr^3oKOwDT4lleDbYLFN4S_OgJNWob{_qoleEi{Ows~4oc|EcWzz9j z6U%_nI$?CVV#HuWzxaC$Z$Lgj2W2=GUheFRDJVx0EZM5@CJ!H3iMPj{UlA$)j_Z2B z0uRBnwBqH9g@03|{JRcT9`I{^EoW4B38N}4+UuME0b7hAQ-_)7i=M*uB4UJl{iDOeliK)+u%bk zz{$lwNe~br|5-=VhD5_)+KHNIVhW;vM~H^QzU(6)X+i?gApDu|o`?{QfRWjV3VEu) zpdy7>`hvgWRIoIZa-1!~6qae}3x~u@!qQ6pk@)1xu=Ir!P>PnmGSSlFv=`$1M?#|G z-&64X6Dv7$A?u%9E=$D)>z^SPesfO#X~k^eB9(d}^M9<=t5dh33#mWb(ABC9%}AOM zzG#3cjehDZnpfhBG->x2G5atZg1dpAF~7VxSpU+J z7$gt$zr;xQzKep$2!DwoYO8 z=5CmK)ngWSj^ayc2kW*0>48=;|H&qQt!nZslaAu6yP*(U^R+2j^XeL-HRs#~MV$+N zJFgNSMr&TZH@L6|x-Lm;7C5Wv|Acd$z)3cWuf9SUU7uup8VsSR=ha;QVz+R;8lTwc zwi@WFrr8!B1;O2_xhPH&A-lwlkZnD{Wo`?*Ui_{p$)JCNv~BvkOd^r2OG>f@5wdOj zxtu_gtSBWJ^|y7~uGnQJfn=RilD!=v^I|S@LnP~zitJLJf3c<2^>tNd6l5J!k_G!8 z!U`DG$7Sw@(mJFhJ0c>@@9nB2Imp_lB)cm@HoBL~q%oznOG)-^glx=aE|=>_);1N{ zWkLQk)NT8ouG%dOvci;PLH;Ks+o6Z6c0qxxO-iyL{tL;*c6WX3>n6z7DaoFW=r*pK z%l)||D@a9ld3JIm7S_O8z%64Kn1WSsxlwKTb%%h-)% zxVkEM;SWZycqRLPn`}y3m-q32Y}8*V$%6Htu$HD4y1e&>WPhe4J31n5S{qkgdIYjR zQj*;pA=|aJ%WG*U?e~;q-$%%H!(sj3G}&*d$gT?JKcQ~Bw{m&oE2V8nN!AdNwnx6p zi+oA;Yf7?nBV^O_T;9k^vR_h?Jr^O{GuKs@aDeRRRAg6k{%2SBUOBEhsSmQBQj%>G zA=?|l+m9OLk15HPM#%Qba(S@=b^9SD8TbF#AaRvHFEAk4_YpD%l8kx{s(-Yzt0$rCF0?FL*ny~$I?syvBHDSc1=8n>xT!Z}&@$MvV7799N zzIOk;_+SBs8o+tY332r}!DRf>fKR?yB0zPMpt?R0mCX9Cc>x6&bKRIERO$~#6!oKs zx>1EnjLx^ncU25ufDhdzpeTzA$+eqDNQNTWu?8hTvL-3Xo)MB^xvttr0unrjg0BQA zr$k7G=eTN%9!PFXO7dWYWCYR@H)votBqjMFLNXFbiR(4V^~p%C3&#J@VkJmPa6|yG z>nxH1_#=#b-SmhQ>=9Fk$y?{*BTmEe;!@Xz`qQOm;5{Ntx|`{U>w^6MjZFNYWOh~= zKKU}d<+^u(fj)d=3U9fN^M88Fnk3#LnU(ANiy~{3w{UW?tk6}{EPk}oim0k2ML-p- z|Akb`+qf!o531r+RGj~_jfYeRw07%a;0vl)Dyks=A5tAy;HrH>pz527>bHoj6|G$L z1OZU>NkMf(X@u%u`L5cR3##6!sDky65aB_2E_XUouU@IBE{e!HIM-F@UqQ85Dyo+v zREOla>V_>)^-Mu^V?l)KP;6)Ip%L~-MYUao>aZ+VU4jZ(-BVF5i%_j}U3HfRsJbOm zN#Jy2(Eo=vMAkyx7tl5rA0`<|q)puz^#5xdwy7B2+lFU2%rvE)Y=!rroIJ;kVN1?S_=midusmEiG9K@#&NTdo z7nzA|Zf5&u?cSNhHq2%M)y>mThP~LGw2kVV%e`B$uz**fE{lq~Hy;%*9`yPn9XwH; z#ql!Kz4-=F_s&Fhk3&I94&pzhu995g-Gbk!@4Os*&5bc@SU$6SR#RDJi`mgwa9v9b zVBD#SE}bW%sN$0$kC>+74P3Vb^?$q5jK3@b$E#vkf5&&H){XtMn_(J6##3U(+nxPI z#?_QTJ8mBXifpLQ|6lu0uA-qZwo@o|>(+6B@GS?OQP!Sy`Lau}!l}#8%D~ndp5`sC zcI$=UPglDuiK`V0hP8s>F6C;B8fW_5UC&vI-{a24!s2YVu8$MY+1R#faW*4~v)#5C zO0uK7E792w0TJ}S&DNi?8~yJ1jn3APgRcb{7$APt9;*rsKIZhmZs=}K-l%aq;Djvx z#B`T>wJPQdUkOAu(cSih=%qb>HB!xc@{$RaamGax5R0a!QHySWY~Y= zkF8?^jK5skD=-*$0a1$$78n|AFAs-T)bmTQkMiv0VkZg5yK9Msi;lMs*L#!!cfxW{CI(E2tB|TH zkz&WYKDWfujzj!!VaJcfj^74$Bq50o(|GR==0Cc^X~)?|3R(e`licdeV8*lt$_?UqouWgOTaJgoXubc2(RfLRHd<@zhSfhP9gO&jdt%;OBjtd)c+1f`&5kfU0^g4(r7Z^-+M9$IpE)& zdcePTjWEAk4fvd-99h@j_3Y@f7))irzc-lw+#R3)Y=@yV3nlnsDG8?U4){gC z`1?}4VeI?Q7h~U}(k?vy`TigmR{&y*0zTBLby^qiWwL6`c`oG(h6nmYFnln;g(Vb* z_Y{Vz=Gv0RhMJZi#+3;MPR~pX!T1;2;lr5geW)>fs4fcK z2O1dG4vN^}qrR^9j>hng!caM@s;mT$JZo{oNd_43KtdbCH4zLS_jbLvG={e{hNhCr zlFH_mZ!yRO1J@f2J3Kfdg5i_RT<=|t;a!cPvbM6Op{nIgyqRG5GXX;|{)PVVX;0T% zr!lP47%FDX0*02a*3SR~I%Ufa4^>6%@EImuZ)gl}Xbh$3aw;1)nNND(Bw)BZg5mQH zE*=D++UCa!Ls}0z$ixmDEewBnc-shuFWR{{q^>ahA0ALQF+3E(@MUY)`#@v(K-mGe z`Ie)9X!)dGCU)R`BnE~@wvS-=3acaUYYgve4COPcOYroyP1Qftaac4lJQ>08HP%kv z)EM4W7)qLF)mBvUNwzKAlbL}XydMOEL=caT2L?tE_asITD^L)F+M~h#7d^0YQ2WNi zu@m)_m%SbDvrOW!G=YXAFhR%&|L8AbEe(X%8obF{uhiKTjPMI(gik9^W{&W4wW5zP z{tXkns~rR*S~#l=RTm%gnc4hX3Ep>S9e4_O@kOCs@P>sSd%;NXx}0)3(|@}PNoI3n z)?+^b9RtmKlLVSf+k(;KJ;g%zsst}IXeq(__}GNfP&ajg_i@HQoI~HM61?P1GbD*g zA76!%^r(9iJ?bQZ8La=QitMwRj2J=C+>4yN{1~_%9Sbd-@+-=#7YfKXH|C7g4^Tt! zsr1JW#7L}D)kabLL(OCOkQ_y1Kd~2Kh1PBKJtQ7BKUnaKYU9P0a^RU9qqk z|HvIRb@G@o_>wcTabbx&5@l$Vg8iCT?A$zm_R=}=KjqC`Qn#crEYfrCl4(52AAgl#Lt$|8Q{Nx?phxrM)QAPH~(}P>+S^=rn(k3I8^GByIK5sBB_h_ zFL?wBavc9cKbOMp8TD!uOOc}1lh0aW*{i+f;o>`Wyc23I%CuJt)GE?yJ;nHs{l$1j1Tn;FRlJRMNMMWWTN7d&a^|Vnv9_{=@&JU zT7CL5vsTeBWIJbj)8y&5sMTq@R?}08r~POV@)l=(CdcB>^NU+}N69%SIO|cH#96gi z!&#p>$P!zO8o$iLWt!@VDNt*_%$zk;OPuu?=0EAI_a|{yF2A9jKEwSF9Ek3xvm%F* z<-F(lZVbQ1U6n;8m+~zAUl+&XiAuQ)14$BBeU{@tUG@G%S6u>jko@clVWql#CHZ~N z$8U7i-E#0XlUhx?6LctE{OZ}?EeLh!d0%D>v=v4k2NfVOP`ZAVrQnmdM1#-mB{sM{ z(FQU;K6i*QJ~|hH(%I_#3#Z_N1*96ECa>T!#J>2_m&iIO(|Pla}ke&7#cj^BpN)JL<5Poo<9y{=vfb{XbT#& z6m30!dqQcbn>yNh{=;}_cx@l@=3t$~YhP%GlJwdK6KiZ65W#C-7%8mQvhx5dFv`PE zblQG;Iejy0Ii13CWFdVTzT|v_(ZUs12-ZIC#HCOeXGpiaK zOUtTSyca%(8cuiwv+7W%X4e0*Ve zPK-+n)~~OnG}hC^7_68La)9lD!w^m-V%*39Mwd(tS~S zHKBoO-iHeRS#i}k0B|*6t`E;wXL@HkexHI2YMN^ncw#3nNTpTaFDVwF2tK}4OeA~b z4JUpn*#D=sV&J<6^)t|9dP3d`5U! zEdD;Lm}+RkA^6)7cmw6jLH<`0(q{Ym7>m?`_AJNmn~BZ1&gjQ)A<||;ou;w!L^Lxr zqE~qSmwi-dw+8Qgu^U!C&(FkeQACB*PRWv<|CI|Nk3IiGs^9E@U-gT>55gN(dS$I( zdPrr-Y2M*Do`+L4>5-?;A0xg7`N?ScdCkaEpwhGB1!Z zsQwqke<4*l?l;0UM~ce4;mD-gGJ>!Im%HGO7e!@W?_yHn{A+^cD{;{k?z>V{=2cfF z)fEw{D%>gMf1s(%Tcu2@w<1*4xXjOgUsIXa`59ENb&62c;JSbRJxyg^_-|4L`H#?s zwYbIDU#F?oVY>p`IP-=k)u9ntTjG%qc=Cg?hWXG3lj_z8)mFH<9~by5D)a7slPXyM z4iRpRE6QLjFd~!c`G~AB zc=WdavZgX0zHLyw8I1p-tnKlHYyTxpWj^EDq}n~J+<#G1y{LU&9pf>n7DuSY z;_=gX3boS9d;qmcbyb9FocEaPKd-5t7g@})nN-~WZd-mw?@`x(PE$Q6R2yunw~8WE zJ9&?|{p!Ha9ulgr5~yB`Q0?yB;rb71st1LN0g_?EcUwiM_V8|Z{k58Etxz#QGO2>~ zzfiB~-fgb`fTnsts2CubR7)eW_VjLb{rff5{X)e6$)pO_ze8DjVNKkaAemHON2q3c*Sr25n(7XrVt`~& zy*D62)#zR4`nPMU+l7h&l1Vi^Le=D5>-x88s@sHWT>{l<5vpeI8rMf!fbsmTLd5{d zl=WDIYL<7k>))cOZV@U5NG8?q5vtkVRjz-trn*_E7$6x`@0UfW=6F}S{u)iSMyMDd znN)KkRCB#6T>mCbb(2srKr*Q=j!^CEUGDlfYN{KBiUE>I^>T!2KkqWv=b1Rz@CKn` zfMig8&^kgj&%4z1uh&%93l#$-lgf)w?eAUU`qycy>x7B{l1a5ZLN(u8?fTbhs%wRc z0g_2|U4&|Zx61Xe(Nxz66$2!b>ir1SLhnDWf3>E%TBsNx8B`y3k5Dc0E_VH^G}To? z#Q@2q8Xuur>|NygS8A#&g^B@^Np)0&YKeED>tCU%t`I5)NG8?Y5vrx$1+IU&rn+3H z7$BKc-$tmGdFQ+SWt!?Tp<;k!P<=EgLbcpG&-E|WRF?`B10<7buL#uv-np)SiKeFf05YkcYY0 zMFHete#k0jUPNahNOYJc{5Xhz!aU4=ehir<(qSiStRMy;nPrRWM(sRCCtY zWiem)LMW2F`Z(DC875(7dKbuI4U$(EfaBkVQ%Jqchn|1o#jHkMaaRF+k>{Ge;CBlCCA^Rt7YC-Zv` zC&};aB4t*IEgn|+Jzg9-1Q$(u|HjW{_)ym-&8n!b!4rF0-r&nkI?T}#55wz$Mi-s)Vi!TXDj5#GZqT<3XMV z-lu5ZP1PkeO(h%mleRYRF9$|!QZn52j?ui2(YzaGR#a3rx9t9`%O4YXA0FXdx`pc< zr+FWzdDm1lHDZ5pi`(|do9Pe3CSSfC;axVw^-k2hPt?3C%Vt$HRJFV%koE?V$@{C| z{BLOQ@&PXH^i}QhZ`|i=;QlH&{~PkID0XpSv*N8UYi=s3ZLDah+^7%YDF-(0^%0v? zVjK18n)m6-CXF*oN*kNXTHd-#n{=^xf4xhDcU3RfJ45q6L-TH!i3iQiY=frP@L z{>R@r=ED`%2l4+SYJ&wdXlaARciqIEL20O)dV|GxW8$S3Kp=Ui*kJJ;_dnBPA4&As zlQhilg8YZ7NU#Em=v)&}qvex2@wVn0*VMblFXR5+EFg9&5 zK2R6IcGmAFi7m7zGI%|<4imL(SdiOU2mPN0uMZ~OR#{UL`~S{%R{BQE+gWRxY8xxd zHZrtngG&SWf%z{sfJakD(mw?ApGPH<&USrojO+ae^@kwrKe{KbsKgWxQVIiO-B z%{N-tk2^@Mhs;NNqg@=pSIjTd%*&f<@F2vEhP:V(WWxWX(y?Gs_X!?rG_>k9Y?j#VyZ;>so$YTi}Vvk**gG${3cN#Ol#g!fLEyl}XHi=V4` zmsd77mNhnSG?@O(=KaeS5#BqOxZY~bd$s0W+E7x`RI+goFG%2hOoaC?gI(;cQMk`n zylbmwl~q)3gQf#m$W$!CA z?@S&8Z1CPNIl_B#7uUN~^S)Gg*UZ8MW1|zgy3V#uHrx>5J*CL?F4DX&(!5KWs%jcG za@v`G@`ug)w;=x?*5%aBF80MKo17=S%PKGrX!(pQd2=O`HjzO6w^_i=u;IJnpcoHd=a0@wfvFV*0`#-&_27++ir@(fVmY@P(r{i7#lehA;fNuO+q^YJKZ1 zl_Np$g{7JKLa5f(P!BHt=aX^%^Y9n?Lb;=kdh}NZUk0idTL=6@{Y7q|{Ig?U7_GrhZ8LQc#0KfE~g>WMed{S~Z#KE^{nD9!7JliqRm z7uSE-S3H*3S6m^iZiy=j^D4o6GFife;E6fio!rcPKD|B#y+C^BAmG-M)zs);5zW3hU_a7kpRNvZ z_SzwGUC#gM5RWHuh!|i%Lw6^E@VGilGZ7Q!Y294UnSr0Z%8RUeB`>9gEL*?@kEz62n8vc^Pjl(V3p_W+6_O^C1&Jwb8<3pi7e-NRXROW zp)_O=hMo!KW?A!Jw*gMRWu+WyDsw6`tK~c@5l&Wevx4zo)v~(?x^D&M{aMLz=IH^) zm3skmv2u1BvGUqPD>KysD`%VYe|=nM-QWk9Uk!6Hx!Kg8p7KNzPnj!>mZ1!@Nl&Qb zIF&L-vNRHFkI=JJe)`W}Tf?<=uuJn3Z>!^x@bn9)4TQO|!(7(ZtG)G4a>oT}PjdihXxxqv{Ja|G(Eaj4Ds%OYT0-^QuW z#dl{Hw~czJPAc)~27E9{wz@N#GMC{E6XlJM%iSM;q1^8%mwnpwn4-8%=r%D30+=Wu zUW$WY_;EO{rQEx&z^QM?dct9z0^J9tS5H(Db*y;n4H3EdCy$K7hKPOz zs_xR`&j%L&jnh1YvJc*{dHzEdoYlAt-u0HtsgzNUw>dwf5xKzYFZcDNPf;LW<_g45*;ao!T7HUX|?%&e{RUa@7NL1KNG9*0k41aJP+YC z2&7>Dn}$ToWqAjP<;FPP0hw4X38X^mr-X^|3WD|D(0GUVzrc8xi1B{OD%Qp$A&FMv zjXEy4!m48$Z-GAm_9k7E;}6Khc!A4hx(SC^#OGR9S$JrHj5&r%kl86)`7&dJ)hN@L32%psgN2fY2tNlKCmRR$sXo!fcY*H^KA&s zM?w<4hBwSt*vlg1QpRz0gYgsTun%W&s|*GyOd!(h5nF*dUVU^FnjVe052qYQWw8&Z z<)@u$9c931n`7z8%P`(VCcAzx^~B8dgDhPoX)un#Zk=f(HDKsf(%iO97Jpv)^R0)+ zw5zTUtE8W2Qso(?V+bT%CCzR7l?7xHW@T@Ci{;7Mhpxa~o z8wU((J+|XYd~@<2J+c-bh6e3<|2Z{yCW!_TLAMX?KX^t((Cf&Fp($R=^@nkPd~W+I z!IANHP#Uh8I)ZNhO1v~?mVMTNUy2C2J@0>DjXjfCW7Qx+jdh3#tEW{2z0QoFd+7u3 zU91D|TG&)jJ7mHAM`SbGWaBV)uJ&$s)T*en*zs=9RZXm+rfe44Zj0+4F$YIy)e|8Z zHuDTL;@_zm^h6Ra6B%bfK-V!hj;q7yeVp7 zTvQMt|3Fb}1G~gfA{%Dm|8H>qxm?PWKCa`Zak)WIyiXmTl3lI)xVqN5>8Do#8l@E! zcN%N)XN0oIALKYH=tfXHNDsB?@kc~3o|Vg5rcL0!DUP_~r|^io5fpd&p9P0U+?%jW zpzm2jfO3)Ua{nHjLMmnkD4oYx(m2Ad^nb$$dpWI(p!m0}V(Os@jS;r-)BGz3H0ubj`ZFw7H_8A=o~V5_RVU*1`Gj&>;73?RtA?)_Y{cniWZ7 zaKRI{?mi`g`T=arn1f&8-g6XdY|U=0DJe5|B9}KcBWBmOu5H3r8Rg#R2;YGBjS=2! zv8$Xz8hFpnj5oKqkgxFW!Tc{fsn8@3f+-^cerVoR)<%KU&2oMM?;!pOc|Qd2Xwtl! zGULs_!NmP=1oy+(n6XInUL?G$xG`hn?WVQOyJv9!eTe%J>?&WVc`wY2cR>R0VErTH z{pbkSqvc@l`NF%TqOqZJqn$~@Vt#JVVEhkxKZYIav>bTvpBZmv_-Sv({qXP2g8W~| z`|)9JDSj+*OEu@pa_)l(w!x%olX)Bz$W(fg%}$C`sZN1(nfidi@$P_S4vMw4dg^UuL{tv3d9A z{Lk)5p27AS%*BX%f#O}$P+M8jP_^-$h9FY7X%~23?==zL&tiMc?)Vkv-d*#qsj01~ zt_-G8DQlARO`|6J6h>G-_y4r_Cg5>Z_5c5!G)vQ_d!PjhofKMVp>2{TT`8S3DQ!VY z3q_DGU(&Qm7bq=sVMoiZ?29M@LMtkYD2s}Kh}7+?2+AtRVp#dgA*(s0{01?^cd|{`|hMi&o(RmYCkmK zJI;Uy^9|t8L<^DsAP|$J2|kIRg7SNJUMLUJRD^| z%Bmojj`YW!$g<{2BL0E2h)tb36YCClCO%xm#JZDx6M29wIR5CPd8olNv5w;(O?))P zM8+UZ+=4QJDf15YbZ=Qx*b?jU#%mz|L!YNwmT0V zOYy*E!bYBrG8`=*Gnp`W&|M~MWH|nXr9r-UCT!%^NH#NJlwBkfwtF>7GO8cTZ!x#C z+I_4O^{9!3wNYVgH^@HvWy;hBnwrz%qL#dxC?cR$}39xy*P zCHFo2iE(v8ko%s^kJZ;t>S}15><=}|x+Y>Vz}FJ|ZDM167E1ffo%iwC?H@b1^}s{_ z-Sij@<392dh(kNu^8&|ZU|gPEEBy^;z`%8uO%5!Av3JT+YsX*v$Sp{Bo5 z6SHF<%c4XtK;eIH4gO=bROyC-QOy6bpFLiD*FI{DGwpG+Yv)zv_T2uN+(!_DDeNON zq;@Ta|4Zla7S0H>bzMIfeVT8Bf`Z>>EO`X!Pw5zk*g9Qf8 z?Rmn(>p!vO9n3epZ-nRGH^P^8HB7~>ecQrsN(vvHE3kYC@(nHJ*79;IeDc)Brmk%s z-R@1nGgMEq4NTT1X5T%-^3(M7cT8+o$B@h_IWY+wd6fUz5= z$9SP_>ZA^Q_u1cF7Yq|7QoiYAyIE_merQaNfsf~pfeTO&4YJpG*V<+bG)F6zx6}l9 z3_bpiqZKt}CZ*W3VT#USh%AwsyQ|VelPu-{9>DG=%UYWEY^Q!MPb8~{LX6q7x>-e`28#7$BN{~ zH6q%2@BWF3qpxKHnxQuyJ{xQIXRKD}@>l^lhq+^4r<@9PrQklc^(+^_at ztk4qVUM_`Po;5XeD`0AYpKyI2u79`+i`XQKo%`y8hWX1}7pYAG#`+r-J%r>-+ z4FdBWvx?p01b>Tf4nsTKN(EaI-w6`?F`gJ_`|5rHo*Gxe z_~F}9LywFWhilxFgeHdAIjtW7*Vi4ck3w9m$V#8W8=S|5=RaaxC${b#*e9~%LfPJ> z;Bt)`pX8}1MvPW%tmDg>!ls8gTWkPtY$VzA8jV7KD zuK!|qC%2BnHjVZ01mSU|z{827a8f;VQOly)S!lo!h;2W4nZ_)!2Gy@mctUhP%f05% z;Rw`grJzeQsD|ztZH`_2?-Lpsc;~m3f!!^RU0KME6(qpmTV z06=n|ZtDed-*n`9h2&U~#c-YjJ==pES6b~91xww>e*_pm{~P+)M#|^ zH*r})D=B`atsE4;R3QrEM_2bPHfKP8JlWC`US~n@H7Dmk z2Hyj@H{3zMT#$RC6mm&xYUq{G<3g_aw+YXQ?83H6AiK>Ws|=B`qK9nK;6z1++!?Kn zL9jFH0a;_|$igQ@WA9QbvPqi~))3h*b8oseFp#}jIJ%o(>V?0wX zPBeB48vE^NWWh0>l17%2j`7eCrmkI{V=BM@8E=Lb!jNfr_)M7R$bH8;+Ou}@9bE|> z7zn1G2>DUco~M);VN}kfJsNf(KFo{R>g{(sp^B-0_LBA)S`++i^wq8S@RIh^YGEDI zp-&V~+D{AjKTj}eZ_ZhDb9{ALIRAyE;coGy{j|?I`cLo+U`94Elo=oxi10ZGZ*ZK( z?|(C4^+bNc>JnEVG5$jo{>Stoaj*N2pN`oOJ{{Bg51GC2wzm_CZz%MZ`5pfDJbthz zVhJ1OPd~$Jjf&s$udRdo*Wd#MGH1I&Uvf0!!R5jhY$~f5HOqsRvAzF`gUg6a7D@8_ z=VM|Z%bFVseLrKLk68_%Y4`LYik>|1GX2?__R*OQ(L*rvAEn7>Z`!|7Px48eg^$q8 zdp0L}a0bp{l<1s1aS{$4hbO3b{kAY$YMa9i4>Ot*O~Tu%O~*}E?DtSH4qgr6Jrr!m zLFNc3dpX{Siy6uMAMe&MG(+j&64>|+{ifU^_XSGatzi+ik+2{49hubGIC!%)&48;)i-p*&wKf*;Z82|##cdPHf|q!f)nwH_ z9^~3e!PdjR2F=v&q8-=2d<$AIgglYMvtmuI9MOJ4DHfCmW#%Lhw}E6f;j^4(DEtY$8Ws3xP|Lq&c#or%*Ads(ee|N;cDVZei>#2^mCk` zDgWiw#H<;f2bhcSYP;#V3-RH3FpK$rdhk?=2kr>Ktn*NY>#e8EYJvp2SxwC1`47J| z+%3MEnDt^*nyU%QF0z_v<@m=Idn&)hErO{A5q;wL$5LX@|f*pAQIHZ~FgS3Gt# z!TdH}?f?URd%yWN?Q+r7`U|fudQIn_``>-!fwn-V1D|x@Prd^IH!2skAlJ%!xo2hv zEqni0*Aj!WSl9BJoVg~_K5u^8>?5*f7B1yE+Zc)eJI~o>T%;{xGO!V-Fvu)hHn|fs zQ}-9i0|VO)GPBzgz`0V`mycSU2H^Z^;TP;6(-XmVrGUmVJ^N_eptFA%$?w8%T7s(J z(rFmO@N>|bnP#5QnFLNhda@BkxY+xJQ`50{BYgIch0+)=!(!I?J%Lrf{^z%QRjse%Q_jw&uLb$gySi$jAo=Btb5BEY}@-9?(_a*FeK z{*QQ{4mbkv$kg8kx)GoyGq6{_$`bm&J&!{@E_4qF`~TAc`#*L4ox0fn=d=I!0vP`H zwqQFbrc&4Y4+!(WrYX(2lNBxfg7#*19h^JqXM2Co+Q6D=WuJliXyXGbBZwSbnDhK( z>%<_~h4MRs)`>Rf*)#b_%+CK-sdot6M;EhXpj>wi!$4_}qZdlfglxKJ_zQ|VJ_~;_k=GN!g!7KHiVur%B)z6Mwj=6|0z`CrN|vIbZX=6|2hZ?QS3gjiS*=6|=E zSg5C`!MrA9@sw2{0qOP$SbYbh2i~gjM#u{f@!(C;S-zj zq#$>=s@ zOboC8ZH?9i|G@YC%kkG>cBW6SG_@tI4Q_{^#dvci0MdudbG>eBv^IF3`nWbYgd9h%a7i2PqlSl$q9E1=)n#>TSX+^U?^$@m#6^n|6-jGj zYjY6n%6iL!*5*=Z;btG8)z-48Wme0f=ok!OwC2tj%6AH?KHhRj>)aNGe)QT{#~1!d zYZl{8i+-Q5h|recV6}zT7UaHB3baCtYUrKO<))y+KAzA=beFcFH{dC+CKvVt~Z*co~`2D9i*zn}j zpxvCu#_7OYO5v7gP7VDsiX60$UzqTn&@OAk-2?GJT@(bgz9BSLWTmt5rj^Wpatmaw zB($?yu?aL+<#0N1aw*WTLj$zD(}Cs10uGj?o(?>m-~WnDV!(CcW{DOY&S`zK6mTgf z)zCleOe_a5IQ-`cg@ku`8&2lq8OA6Ccr_tBRwTUB@kYfXGKq?WcXsQPAjq;Fj_*z> z1)gnU?_glXwokXw4i|1mT%53l)VlZ-ACrYI(NQ&?!IMjtKMrHfl)hiZ<*i z@vLYR1hIZ0F;*loHyL~)JpUAj+c~XMgP@M}Fd0lOg%~!A?4!|fz{+g{qZw|i_zA9m zl3})t_g6P(ud!9UwG?1UQ)=jvzg0Xk{Qg_a>&iCFv3Mdk3W8VvkQXa@-Sfz$2_qR{ z=eA0C;yhcmqOstVnpDw3hx4um8j?OnB$DP7i|J zSr4bNrk4UQ>Jo=wZOMtWcag;hg-1vKXo>5fM~zQdLvAaupo==_?WJ(bv!;e_8AWs- zjdawt31#e{SG6JCgvZUJ00M4c$cznSV2?|9v!}jo3cd zhFgr~zVEON3bC;wvALAa(c%41Y>YCcb4klMcVAnibjBfCdYoZE!JaKad&2;{M${YO zJ*(rVbnpyB?yq7OyP^{fK|PXZ4z2-~(-u>B1)*SQ6%c;-Tfz7JOl#FJxx$P`vT~)uE*&^|Vw2 zMx0D08)i*az32)Gs+8(B0&kQL73VFQ+CE!Bw*=#HOqC*Z^S9t#?6AKxh zQY*uAknLTk5epaESm>EkXnO+jV`14GA6kZ|)W+di-_i4>&fhMvVAXoDuq?6v$?%lg zS8wq0pRBG)lP4{mTJI0nsb=$tEX9VR!~9>2`Z#M-XOAQb_WW94rW9K#-k_kA^>SMz1zr=*XQyD8|D zsmnT-b|ns~rNZXU#cRh^{R)V@=`A8C;G<)qF>&_-v14vms!z`%^gk6%etMd*^W$Ic|eW} z$N!l1E4$k}^+I$fdMcZuU5-3pes>Z1V5pk_x0gPjs07 zh^>BgcqX?PrC{}9V|8<5X!lva6@U-ZN?T8ti}a%zhnDeOrT4=<4=fk4^Pmo%nzEuxa)z)bh+; z6W;$QHv5g*OzvQmg4qWfvnNhjIl@pu4cchr?TEVPjs07 zi$%ZPCu1JV39ApaR^xV^xczx}^VOnOkIoYvp8t(Szk_>7wxbk8+xuEKOzJ{}bavlq zz`8LhdhN~ytlz~gBv)9`_BPH9O_OnB?8H2i+Kf!ldcn7LLxJe`D(zi0Vf9K2WMXF{ z9BR7Vg=yiJ+SpC;BdrwVRZSSdB zKdG^+sj+jL^Gvf7JqVvzQLZ}=qL~4DF+T(3_HXNMa=PvQjZ6H$F_q_@X_j-Tyq@mg z%D9O!`!{F6`YLC@)_enYuokZ0%S-dvC%3G=^k!yNS-@>p)*l2Z+^oEul5$(=1o8ei zOd7syjy}OZ?xpDFdc5%($p28_c0T^R9_!zE5NpntRc^vh!P<-7*o+U)gAD_m2d|`f zup7#vu{MO)|6VakLz<2>pn08WKm6m*8#*A7(^^;>?iNoPZV1=^ub8AEWi!E4=Hz-w z!wq*sE<@&({1)Rbi)J$aNyRUlqfZMch-Nb3o_hS*D&w4Jjz3?EPl!3O&KDE#sMGY^ zJ@)ITeCpvn6(&E?fX8`HZS20g^18_|bNo5~ypy?-hvdbg(sCV#z(ktKF^Q%=7@69I z-|%?yKXHw10Di_(4+og19u`yI)07nS*jouw=LWo?_oTK+FIV`~`vLW`)sD=X8uz$* z+t~;jz-FTnzle@_F3 zaYb!H?-Js@$L_F$<27PI2?+cvYD=e1=G?oDZ#nr_)J_fe|7NJc<6SB;`{k|;+ReRw zkX^ji#$HbZlj9IFGO@>tS!jS$!|NYbehWI-ySxdAiEUg_b2kC$S!;u1&rYR>hIm)h zP7AMpTSFGYkQum#rG1Ds9Q|Htoh)|LO~a>Mf`H+m{&N1H&N&JsPP^5c#LO9i;E>B| zIjymPn}$#0{*Tk}t0|M1i|NzC_5Z8U>`+#P$%YZ#?IG#Y8=Q%+7BTVkLp&3Az6YGSv{Qp-lT`7`qO+aYVLUY*cT5*Wv_MhSch3Y)H@^0|Gycm!c?8Z;TWNd0sg00%bYIl!Hz=fD}U|DglV z^6{j(6PyDlIR{wJci^!|ZRo%=Wh^tYd{5tjJ&QZAKS~3|CwGk`p#`d2&;m0?xu+`a z9gOM~`yUJqc38i*WA(a~U8~o(uj*L0zO8fBqLs&W5%7+XMv2TeQR=#c)Z2ZjgM(3d zP*|B2cX5?(OH}6f#t&u@BH@e09R5!}lPJ0wW!B|#&V1m{#hUgD&16-gf|;KR_dh0n zs#$$KHt6m9I{LF5eQIZwW>;-tq!sk52g1PifMs?q-!Z6U@YIdsPmGv|-+qi73UH=> z>T{70fbY#O8W#(FV|rfbf9i>-`f^vj`_P|mZQmH?|KiYp)`b40 zf@0=4QAb23z=J9_@(F}*I`7_q9|zBVz*D=fYwgO8RVz>JSif@3>ha6gtUaz{eb_lE z(t+8C@vkNBQZFRLrnG4Y#zoJ@Kl#NV@+K`$hOTri~GNbYvtlJb6pdh z)tD%{u&0{XN6@X#;{GqvWQOJE;$gWDhviJ@AFG7bzLu!Q=6I&NYOcZ03cvqhu$aO8 zx%>uq&2V-fUqf{VG@P7k4cce0zoQvi{b5Fr7DBAaP=CShN1B#NM$*cXe1n&R~@956F*NtzP= zS8@g}(B#I_&s6wERi2RW2}6GZRW$U|GjA;Y%mKcT-gOBf+XnNQ&-m3VIRh7%>%e^G zJBgwTd#Z^BqkDYjUB9}SfqxgrCI+pO?4ZTs;TzR17@MfZknXmcYw!!g{HGZ$%nZC0 zQWFjCn&EJ{5`T_L*)Y0PsCQD%Oh}=< zo)j3of1bgt&$^+(TN3qY@Hr@APOG7mDdtA~3*Sf-WrRj^N%2wNRlX?9|JhOh%%YXe zG#O4A2f+fMx+pyVX)9muDw`vFjhw}b&%yJ%j_+8NoP#Nxf;pp7czFHa3f~Oj@xl$^ z+-k#_wOGS(yXYSY7@UKbph(&*c5#^hGYU+H*~*JAQ`ydFp>=T!8Rtjg*9<8)7hD{E z|0PaDJfAz+OGKRPrZLuIo#}YPuZ!dHFWkpJAF#~s<)gfPJpU1I!}f5~f)m5b_Y@Ug z|27^}vF{!;-Q7N8STH?~Q?6{wR(7&qjYL&eckO>kqWycb{UZ^BgXwt)SeX?a>`M~u zU&XO_9gFy*zMH%+x!i9TW9qT{x(wS+!M?t!6+y4#)-9*4yF(W1hqlK1-!ANbX=@~7 zaIkNlt*q?XdTE^hL1km>dK5_nhSPItV*Ok32=q*<3);1o)x(7lG8IS)-!`Jc5r$aDroylCWuqW#Tks#p&)F3=LM z2VEBX4;M`J9`&Hho=bR;+6jq{PqVulPmHN~-62 zt{9o9%)VQo#|<-A@cM7GHZ<#~NYAh&q-gr4$=)z?#bv$-vxSoHt74lR$RfV=F7y7`#QLxDa+FyzIcRTS(GkI9-+FutF50t)lNF_e7XQZQ(7VBx zG&q!9Og>b^6 z3}^3E=lODsy^);3!IV6E^CqmT66+shZ?OsMszm;?l3b$rDBhBLSZ6;Mo`126h(}O4 zS^-pJA~CC5HZ99fc6Hi)$Esd%3WpNVa2t@79jm(5u3r}oyrxMgh9VrF&wVu^=Ix%@ z?IQ4VFZq=#vuMx%$&WyH{q6JpQIyx;-Yjx)>+=)5i(8rZVPSCX@yo8n2PJa)MpS%$ z5v1Vy+qhj)E^h7Y%A9PC8n;VkguBZ05>oB7gD;i}ZTgb63DTBPynXSHp*prd)eY}P z-K$RB@JQXu$Sb6%yLymQH@t|ttC|0$y4{MayV{+99=BVh?iEbDgPrl7cUNz8>UJxp zZnIOjt5G-5iLl-S9f4o{QwJd5AJ?`w`NDE{fdTwbcG7@dOs z8n^#}cw*-{mwfDw=rv*cXAM!>{TY(GzR4SDGu>Dn3;AkmR2L^Wy6bXBydcBDx1ypp$W-75mUJ$BabZ*%E}(Q#7ruB} zR1z*2xd)&$U!DtJRzVlHo=9BWhp(?6~BIm zg5qapOi!)Mj4tfh*S8fEKP$+Lf*(~C$g$b+H;aEdMAxs6%CnU>;V(s^`}*snl5Az# zeKtz-*eEu|!lEYy;5^QwoZAhvzKXki-TCVeL zemkl~Y&lp_r1O1gm#8GM(U(7cIwOxbJ|h6KIKaA(n~^A{o5-VzEJZQxW5z?KU;vRksoO%9Gwkq!%*{!EtB2{VqOO zz-=7A@`2+qGo^8NM>%NvYStMwlZ{(Jqrik_6=pTg2ih)PMS91__Zrf>{#D5;lz%$6PB%6)!DUn-GozCPgp+r zgo!7utzSK9%G8EcOOHM2#MSi;>!#M5W6#T1PgpeJm<_8}uAe-$p?>0|sZ%CRoY*{d zD!yqtbT17jAfJ7wvL<2qK4U$tf_E(P0X<4(gT zZTg{alX<=$Y_d7l#(;0xY+$hxc)cZnP>Y%l4^r0FLvPvEN69AbzeyN z#@xsdm}ubGrkstYtg%;`Oudd#v9puCA4iP>LboR-7;{P2BFUJ`X;kFe{2SkeGPe1m zB5nSS-*|04@Rz6=Ty*95E%@--{2Tv-6pl+rrL=h#Wx@9r&i`!lqd07u-NiY$BLJJ+ ze*KA`S-jn_c}uwdJ<81K_OGeK<}If{>FHuo#vEAZMsSC_?sxiM?jE+5}ImfcJdjyT&Z+0MO?=a53O&FHUYrnodji~efb*NETgiqfrMf2hR=EE&O=2#dmewTS z3gf>crAb`w=hiU(JIu1-e!x9&hfEe$m@ohF)64zb8qR+mY>)yp6)ISj!D$w?3Mv`3Q(dE7)4BV7-baP$`vF zU7+5^{69y9r72XhCmPX%bB#)1S*6S<8m9%Ka$JRd@e0c z^=-a)Mm4^kXSs|n-y^8-+^fsNSe$0b;uc}OR<2&pf7%y?O~M<#;MvZ8xc@b=Y`i)Z&1$CB zF~O*cq*M0X3x8L+Zv9$pCahz$nDQn&nNRyBRnf8I_6yplu-NcmzocWKNO;C@q6p{N z9d*7K<5?u8HrTJgGgkM)=??aPhEr$paJu6hr=pYLRGC9K{js}A=8o?~<=2?f<-8-`{+j#f*;g z!~d=fmjAa`@DuEXRT~_AvO!}|-xHG$-mu=Ynd1##IYcADxWV^3lM~z)BDi`u_=W;b z_63+D6ffpZao?`OgnCLrV1?R40e2?^FuaQjK+_e@$lrM%0vMUgQX}@xLk}ocw&+>fU=UUa@ORRq?DW-i#{E6Xz z48E?{H_`da7yb5&_a%RWa!`(2fhV`g4k*R&g+Roh9rnusLBE4q=C{n5-DV%sP*;Z{ zG>GFTczo}=MB_D}@w%PKX)vpX)t~XJS3QF2JOyb|S+zI?89M;8&_BZJ+!R6kT{|bL z^XLY(56j@sBZ84;{)k)&9mL4!BvFbp#W4(> zD)yK8-G$Cwe~ZemG3A4xNa)vJx1)-CiXp4PM zHjDcR^l4~noY>qndCElgJc@U#^Si_Rr(GVf7e|3G-ZceE;|2c3HMo|cP}eedGyfae z-g}ELIBl&MhKOsKduE_ITV_@9mbs_HsaRz?^+w?7cCC2NkHS4{+Tk*0d z9+6ye=gy^Mdh^bz?j&CLr7{LIUD1e@^7VEFN`T!;@AaM612>yW}|U7ZrGE@XZ% zz!|!l899^2DL1K}iE$cd6j}kD+vH}BLS~15v56bxX~I8DwpYC=);aNmW1JJKi#YKE z?E~=M!%x8*KPuzd56_7oaQ{Omj!$vIt^9xRxO3w8pbKMl1`XFMOo|OhQq0V1 z6z?Y$k#oYn!#{L)U)cUu%phOPU}VH1F#*PWQ<;}S+{KLhj__?TAS)n0!{Fh|yO?p` z*$&L{MZnzmWe-dSgfTGw$mu2uzc)y`A5D(bN}khy*S0(OHdAi?hohxi=%XJ z6{DLih|`*0y4UNznRrowF(OX)nz6>Nd+NAEC`|m`hTm^1@k`6x>g4{r986=KiShC< z+dm@c?F~*z>!6wey{N7fO_}#+mzij=Bi?sfhUwf_J=Vh1~yf|RmWBBk|f6IH2!eG53rS*?M zS+xF-hB>#ch%&3^p+lI~pAn4l__oxQoLHrf2B&9ZLJ#S0h`!>BePFA{zr`#r*)3Ul zt`o4u7f`hclYC_8xTK-orWZ4z%mb|4AMccqdwaq#BraQh1l>w5qs5f!T>f!*{?ni} ziOZh3oN05Wh4~NLHaWQbhHIP5m|#qG9I-9&kC?Rn)11Y5!8!krau)aoe(^8lXAz&c z{QMs#G+GeM2;@_}Kt@|&jT8oQXhC@X z$%X}cLQw%|S5JVO_J8_I--61;eoq*l-xKmu1{A?f2YUX`-1pxwIxNQm{Def{BV~{m z_-qRd_0Am1aQ^)4NZ%ZjGUy>uH%a~MS_t4UbydnR&4N-Hi{rI3-h=Q4g#Q}~xg2P(${%P^(G-~CduJ3cABI11a#>V9 z%)eki8Ubq)298znFT0u>PQ-n-CH`>~GBaCW;J-t|icC-ym zW7V)a=YQ_|OdD69S-iRp)v!9xU_<-Q!~DOkez~h|QUlGwo){T>>|!jRvR&x>{Hla` zn<3xamZXkr5A>xr&%c()KUQ|3^K)GPG@)~KO6X)!j!rQDM+R4$(5XilEW$mZsUi-Z zEPiHKp!_e!M&;2E_>w_@uul!1Uo1jN2G7;`!Q)Z}zu4eJS$Q;jbop~tdLg#^ogka8#J&HWnG7lV&>g?3lq_m9d z)DL{tskp|p%mU!KU~DD}8`3)U13!vNvt1ZQMcUY8GxJm zD5TK|+LREqB`FAfVUE`qY+t7Y)()M*`46wJ)2WO0b}P9N!QM$b@N8l?2GJtg>N)k` zJ_$*5>Wti6I5o>TH8&~Dh%ns~`h2GzOvXQ*@&v)Dxdl#90-bW>^Mf4!*pye~H>KMK zJ&5>c)Y{GTCj1n9@8EGS;lo=9Klm?5VK82o5{w6;EL!p*=KnZMTo+{os?GX-V!Xch zGXk$MBY>5EK}O)tf{egJr#mOEE8@gMU-X@Lk4}8<`A>74^PG5y@lPkNPjSL64IX0t zn@(IGbh(T`MYvlUj@^>C+YE;Vew)wuNfF%GgT=4LCB$%v5{Vhib5&vGAgt_V1b)T* zM`VN9eD^{53}c5cFC*|Pj{j`k>x+Q-)tw#~V`Wg688-@2JTSj{22u!SlL0f(XyJc2 z6mjuIkHYJJo1%SPygkOIUZN znCmhE4~PB#&L|^LQ|(%CT~O_11iWz|6R%-{Y$+Lm`pI>se78rzyo|uZ55NP)m$A-d z1S;{ga&se;5ka+QlJ_I5Lzu$}KO^u+Z9)>mJd#u!RC{9+JaW}JJGzX(BjNeiJEIY! zpHVTpyt-IMpnkf+#GGRNbYujoQ!@hflM6BekKFHC|IQ+<|Huno>j#@)W-xw0`;GYU zTK~~X*ZTLSw7we%9}U+(_eO5@H*QUg-SP&)`h^?_;c)&ySUhbeXhRyPziLK|sGxZ>B-?W3UE1it6|37F(r^_HCI-WW|sH%*|p^WLZ zYvLaFK;p-Oi@0fLP^r|dh@WVUj9-qd@@A{poy+KujRZ#*?WsP^o}c)LCA z{>`@&ifK0#+krkgJX^&uDq;6KiR!ewq7R~w>yY_DAJ6Xc?Q8dwVgBC+DD8&gcB8m9 z=;KW*Rxyn7?SAs+0=p~w;*_DayRT<=#rC!PsofKbX*U$N8^yIjU(ar<7)JSaKgIrU z`}T5l5Av?6L1M|;UE|q}AGc?mJjL~2teAE~al27m8`OAqTg5QSxBIu@^^bTw>D>=! z1+3luJiDv5uid}BvcPUAZa0c+gMOafRxyl9*d3n#jP0)OAMgg~jPL%Q-MzQ3-A^A_ zU^f)E8^yIjf6s2K7)JSaKYc->I^(3z0Bq>3-2*(ktGBP+&-6+tW}HB=eYikvz~^Rg z%q#rCmw5A^Krvz_h!enK(rhT?XkxHcH**=-eb-shC%_6^T4 z|HX;>!TgEa?Hium#amkh7ocV^aJRRX;lo>7Jez|QF1j8`ncbUF7PI@a;rX9OqHRuP z+BT;mvCU!jfC8Mk&wkyR`$#c!G5^m)dAsAk5~ZiC&?0%-rB3X@70$C)AO7OQaE3!iUWdw+LYqWW8?J_vJOh8^eXppUnmOq%{5dttiaN#?(6 zlhNjEuu(C)yid_R=gYD4FC&oqr0+SOE7)^B-vnDZay(fiq@O>` z3+dpq@G`i5%g{&g;f3_`r$P$5;;&j)D`=guDEAGSNx+nh?D4x zkD0zmQLZmO&+9)l{MjOgztG<^yo{qyFni`LWANb_{=!6O_|_D|U0-}5%>SC_we9xs zf3GicV+m3({M?b+8gyZm#lF}pJoA${tb^Yd9M*ZUB2k$m2P!91b8Z9uV!dC~M0|A^ z@xu}E(HK)6{bV#UUcC+Uizg+-Fx(?CgRvGatV}=M*`gP}<5xBt=po_2fl{WJ!}8+R zL{ZL>H9f}i((pv(%TT$0J{A{QFY){jBdgf+^IlrvMONkvn)$WkH*(48Mb=9fLkhdZ zOD3|C8}%&8qM={9-5K?g*{HLv%Iz7Zs!Ub6+n=*+Xc_W9#?IZ7! zN*Vf4`toR}^pz+(P@6I1%Rp>lq7+YIcHrf9r@i1Qb)oi+PW#&_orxWg+HwBNi~?^* z>j(IXzvZP#D--L73KXCK$KmBaI7{Bn=fDPlmRI`tEo9Q9C;WQMSp3mzp;sn2bKgrb z_Yx=QP-pIYW*&YEEwlsnL}p%F7xeY!HSCwX$t?UaC^9?OrOsOoWAOVD4oGZHoi}CuetJR@`!FTB{j9$)gh#GAXFE4#{hs4L$M*N4DQicg zVs?4oVpCSbxM<317>6mVZ|aoQFs5M2`u(d=#{T(Uk=B0|`LAdd7F+}~gNL8F`4)V5 zt^X>of3SbPpVInnfP8hKbL)M&Rn69)Sy=-)Bg$)VE;erQIsCU-I8s?K3vY^W;ri;; z&fxcp8I1407LeU;(vZL4MTx{H0|{p6HFKOJPuu}zRvtV zd$@@gb2RN`J}lEbDhRE5nwKg-TX=!&1iUdSA%H!hsDNZoaD(C-%X|Tqi~XK3JijLt zKbrQ&Hxpt`NRYsxioC#QTbQZGg!vTwBlPDlz8rHjt!H5#tcDQulQ+ZbKZd9|{j~yD zqs$b4Xs}ad9NT*$Bzk!G%SBMwEBp=ut<>!Hif<pPPmQUHPasG5g zLOz%5<^dkYJw)ql&p#V%G?0}KaRxq?5Wv$_9}gR6apgXV*iC>OAUm@jh;5+%4EA>3PBNB?@tZ z-r6xCr~`Iwd(kV|6oI0SuDN11NK~BPnkif;#--|4k-u}F|SPC|wR`B9qCO?4>Z*}+fy^zA? z=!Yqbr2|oxwm$D{{V-Z(4>haoiSa7i%V;*3tM*ubb1TOeCut@&8^-Y;_B(LSiFdgE zrxPC*apE1Uf8r(Td#D}U^h+GwNjUM&nbwK%L5dS@`gn)m{{nE7>2k-H`i6U!;n=f8 zD~Wh@YLX7zuuuf)gT*^M{}+qlG)6II!!j5bvLZ1EFT2}eMVEAVcZ^@zBprr@yE>He zuiC%62t@$~<}RaR`yw?q9t?SRg9oNJC#Il&%&euZfZG|rdkv(Zb;p+(Fsb|UckgpX zl|}pVoie6*2Ud8EXc{u{g{F7^=1eThH_>T-Z4Zn(irvnD$6W2u+z-{bX< zSn*~k?l&lCH;QY6L0%WOirIb~rk&k?nVP6hy9W+-kM^D)4EF4<*}iuF<&K16+T9-y zBd;N_+F-C}w^adAtV<3EGX~+F*!hw^apyXGk@sN4YJY2Yt!H=t?Q8enMkW-K_rThq-6*aNYCXHHVs;=ePX5-Bs7||w zU>F!+?cTw&d%$+K`{{&Y+C6B8pxr304R-MCwu(76d3OKZegDaeliD3|1<=~Pqi6TP z?Q8en-TKcdrrm>g4BCz2+F(b|ZmSqZCG37A;Wl~iFccH7wR@;%_n_@-_dgCuD5l** zh6e3MacwZvv)d|$QNG>(_)MZY?cNdJ2^ehc9_HCSc>CJ@zRUl3ylaOA?M88JFwC>t zDuz+M-S4*)*gbS7-mekG<|np7L$&$S7~w0p;$gLb31HrUy-+bV`p#%|6(SUddlc2wu$qB?)(W(Q`0 zN4>>`y==eNy_a5yKeB+E-v7z@pNotB#i#cVT>cAXR$>Aez4n(=n# zez4e?+rNmpA7K8E7l~J*W>9uy^+EXX%>4l8KaIJAQq1MWcKH3l11Q4@Vvt#lut9g~ zODjf~4@zFPZ@3w$<3awV!iF32FWdi%>mOnsRK)IoVg8TowwLV>f2-~DNxT1bw6nW5 z#qJJk_rK0^cGsE(&~30AN6&38VgI0ad7MkYIwbWQ)!W>0LlKp6bO)$^x&8lmt8fbx z7Wu-nUHEV>%c|eD)jw=Z*!vc$?_A^Ty*jADq9fe=dVMGp|MuVZ!gRxv%zxL$E5AGH zdbsqX`P>bHz=tmXV+ojRAT{C1un|Fxw+&3XfaNDV^{;_^G}DJ+?iw0NN}e&ZELoi_ z-8GPpJ{eU<7c-lRU5tv^xHwSc%i|4q$Ijo4K(0x@269KiHIR?E|7A$mriXMTZb1`= zw7qPygBc( zePzJwe>8ljVut_JGu&Rb-}v|qv+@l8qcePXis7~|R%TrOd;D7n~@Sw|Gwom^?O<(SP*s)mAZb>UM-2ZdLvz0k=RH8_RQy5>Y5g-CYfCvzQ|3%>H))kk6}78g-h2J;B%RKGJ`g$#(24D;@@FPM4KU4b1j9!`Vl%38xfZNvn$Kt? zHbg90pa==E&2l6*k&sxxf~?q*kr2}CA~a%wK*|n*eeKu%LQ=V6hoUN}Z~I-JYhPcx zT)wA@91=pP{|nayRfiVJO z1fH!3{Pvx_cgfp+OpE;f=OpvLLYn^QwWxnoOD0pFx@Y7TnIAC$Pb?*~<3yj5>-2uq z(nZ^$=RgL0jeIxZ{|*{Lvfuf_f+NUC{q>yekt=t3vLxaCNpR z8pB9VvP(y%d1`n3H%4HLz!-rs0%HWm2#gUJBQQo_jKCOyF#=-*p6v+yn!Mr%cmr~N zU-;Eoa^;cy{{wQ}tKWGn2=Tjs$JAsx^XdOjmec=)$&Zrmi=1?33wrlDyvQ*{^Z+aE8>gRv4Ug%=X0_h@lv;17A4mHIdU@M8=0S(ZDm;?^Oc_`pNM#6`;yBB!P6a2`hr1WEH9mZ zb`oD;d;-cuzc@Ax`I9a90;8(Z@(vyErNkE)-#p_C%m3+m;tNbuR9YC+|J}qFG(yT! za)F$t^WDT3G(}O^LZSJ|X5tH)(kjo7p#Gdod_g0v&p&%KKF=k-pk>^$LSXU9R}){* zGONp?Se%Q8<||)Md_k*(wSr^*WHTE1g4P=6M}Xn!YT^qls~RQ?)c>=IFR+ER+EZ-E ze9^7M7g*2MXvr{sC*MhYf$bWtIeLH5ljIWIk6j^*l?Kbk^2>=|=w#WJl40pyeug|3 z+wY{$S9@&zsuY|N* zWB$;yiI3EsM80dKYPyGe`UcWDLq z))jt&IksO-E>Ic$X>7#{hpmSbQ>p;(S|hs><;PE~P!0HAJFYo9=IhnIGR`m8aJ8~? zyZ~ZngBrlws`q?ocf21C;5X^ZfH&NfrEtgbAHZ+XR{-zX#KwxH_a6iJZF(d5<78PK z5mUS!z@Mg{27Ko~@hqz&UTP4(Lq8MoWg&EE*Y5h4c(4DTdbEv@!wElcDP|~{om~$O)=|x(90Sx>L ztHLZFS_cfS(E2qn@E_8O@TqOu6$gp3tF%6dJ&3N=?C3SZputPD{vsH-);~1#(S0*$ zaE;bqiU!gT+8o_Dg9e<|Uj_psq$*v7@{a|rzY@pLI$k=78kDsDDi~3W>Q^-zzuIu$NGoRmr@&U+Qz#j&={%*)uAJ5femcITT z)b0i3D)h3PvCnS?{RH%@_n_Pdx%Qo4HIbpbu^q}YsGUCm%rwZA`$4V(KIZo8uLtDX zhe5{Neq}dc@f7Y`hxubLc&C0pu{$`uG1S|Nq6CfBVCK0NrQu AjsO4v literal 0 HcmV?d00001 diff --git a/flume-ng-sinks/flume-hive-sink/metastore_db/seg0/c111.dat b/flume-ng-sinks/flume-hive-sink/metastore_db/seg0/c111.dat new file mode 100644 index 0000000000000000000000000000000000000000..6a3ccab035eb83e7d241fe07161dbb2eef5f4c51 GIT binary patch literal 20480 zcmeHONsJps6n)+0ZS2|iHDSrZff2T6Nroh(z0Hg>o+V??5D=1;WkOg&IE0`GBoGR4 zAlxH_IKYW;At5Ki5s643mJ25i5#o>oD2P*n1mXYk)}HAJ`sRQ@)sny4Rb8)sy{fM6 zzZho#SoGn;uVp|>x2V*+%jo=0`B+!H>CYvqIZdAs&|bO7qR#2}U5_ulYG3*G!dj`^ zg{2F2-k>jE)YXYcd);T~RgW4|WEe0E7zPXjh5^HXVZbn87%&VN1`Gp+f&Ui+Z*|}K z3NO%@D)#+JDEjrthEZ1P*H!oeWV+jU1EH!X!1Ju!tY+w2h*{s4O0AA^E%$7bCM#IL z5JX!fcg>G~vS5kACLUv)Mm|7%bUTA-}Af(WRj&&$9mq0I? z6e$}>rKQe=E=R^rE{4L}zj@nQB6>UZxvx zWDF0ZugWm*7R7uaQIImM0_fLU1}o!2k*P9>0mTl;JZ%}WrD{ho=xQoiqjocf^c}{% ziuAR5FsxU1KHZ?Bvl}A{t!SIE$$Uxl$x zPkugga3!;l$`one%+a{kdQw7i(h2#2Ct;5Z5$Xq(9`CKI{h(sFiHiqQrq zp_foLeMs8215Q%+pIu024pkO|Z2X;YQknqWziusgfw@h;%-HQtfEr?V)7q)MOv#8srlZ~tdUpt&7 zy{T5)3l*ru&0MeP=F~7?7}$(~z#8@efuGqD4s3@*>Fz%vLGmIVCV2{bNRG1%#>hrS z(k3Q~WEEFr&-19bj2(j;g2N_y#}UhK~q z&3=t#@E5KP-C)e%CGIQq5$hIuPRkg};7PtF{4(~Fyv^E%_j7spD#zy|Q(O``tC43} zf7|B1tgEb#b(VXv_VAUK#kKfM+y7_y8tRWr=$L)?!nr#8KY}0?x9Cc@_&fW*d@Wi1 zwM2k&+2;0t!y$bA%vjAV=gC-9HL>AOevf(#p(0`T{F-yafMLKeU>GnA7zPXjh5^HXVZbn882GO-a7kzH-(>HA z^$>dptmoN}U|lKO|K#1M+5f*?JM?j#{r@9&QvEX<2JPQaQYxs#|I5{OHR{iis?xE! z{r}Ii?{zGrE`whb%dms&4WJNbelGK9Cavx9g~U9Cb!4`*-OpYDio|cgk*O14-@q0O zyI?5HwUR037E{If#FA944)zn=9ZwQ~u{$|aU<5;(>L(hmHOMIg6sa!5;rp-BD%WOZ|Fo&9;GP;w&~5_TVl*C{Gohasb63rE=5C74LM(i()@SO{AyV{w7V z6lupjIvz;AN%6V}<6d`L(ZX)ReIBzp7T+yd6RH*Wd!S70+9g71$2KpNDbe~=ec0}S zG8vWWGKd{sC}VNah(@r}6GfzpFN6oYP$tJDy7<~G=g2yk*u>;`Y;r;ZL&NA zF0v+SP0JoE>6`+})wf^byaW3|ERvo71D9Lpwajqu_=9s3@G0w$GmKNS|KEvyllu#( Cp=vAu literal 0 HcmV?d00001 diff --git a/flume-ng-sinks/flume-hive-sink/metastore_db/seg0/c121.dat b/flume-ng-sinks/flume-hive-sink/metastore_db/seg0/c121.dat new file mode 100644 index 0000000000000000000000000000000000000000..77dd26671e21a09d05b4f32e91b1339278ef40aa GIT binary patch literal 16384 zcmeHNON``38UAhe^lRSl-JRL6Nj9@dHj}L?m+dm-U@prgf`rEeLU{O5<{rU-gjPaG z3|zRuDtDw6r$`8CB_u$cR^m=ZLIQD#BD5Dc?STta{VsRUu;VFP;uKs`f9?ML{q^|m zw*7tGa0~#Bdg(Sm93q~DdjHv&+Wxm`{}pch_=ei^ZS@)htkon(4X!?4+8$rD_wRpfQ{y{Z=j<=J6Gv+ytv{P2+IR^+Ll z#soZYuFXyYSsrc&Xw?WkG0a?f)vg+=cJ1DH9qKIH&3oYQ13$Rv~$#Q-T`d%}xpH zPV)IG!H+qKLSi}R2AfXu1x#Qvpn#{OZ9{Pno@VXUwT=)f^8ZSCYiV2KmK~B^!KGh|)G2taA$*KgNN+H9* zmcSv&t0@JcAF$a?uyKUA;i0$>_Z8?X@V}_Q_%=|0(Gb*sxre8S4-k(L?;`GKT=)p_ zp~lgBnDs5h5b=*17fz6Kgm|EF@@pNRys2^UL`MdX5btXo|3k;eKgGQz#zVxR#>s~| zKKhqN?-V09^!D`68prP-{vIR0)aX6X9Pb$MF5-?x_anrIh$kAS-_?Bg2qOm?ohKOI z*73!uM&}X6?<1CoA>vTu`acYEN@#9VeS0+W)@M|1J6MmeK!V!C4+MIlIAM{GSMK*XaMY{3}NPx8+|o`aj}AJy}ZYdaoP( z-;(cc8~xvwzhm@&lyjw*O6z={G5WtH-`zF(KPuB)1V#O<(e$4bZ_Vicw)}OY|J(96 zjs9=T-!l5YEq~kSe?g;&_*VVttr-2^mcMHBzu-J3jM$EsJ2d*gE&qzq{~{L|Vbt3H z$msvJ{HsR)$FvlJ<@M8T)A``;8U5dq?~RQ9Z_6JW{hx}03ns1exo-4-OTITXdOjeh8qTS3B`1aHkW03IzG7h{8N9 zeM@}?i$IZ|bEP=dH_XOg0@kVjW$-)ouY`GJUldG|m__poT!X%Q&N22yE@Qu7Q7zYK z){yXgZC^-{mN{XyW!9*^4r==%7BUxv)#r3$!mZlAP?l39cv?&U#)Ow^`yymS<~*y9 zLSw@1+P;W*DRQ1$6JDw93#l6CMJ>!5M>wqQ3w07vE@lQvW5TPoeUWiBCowZf8WZl+ z_C+BW6=G(PG$xSRz9?u(L_9M{8WU)3Uz9=4xa6fdxf>H$ZC}XD&xH)?ldmzsukDM7 z<;u~H>dR4MLIB3T2)PVn#;hZRVC;)f-8~9rN!Zp9fw3=AA+v({)&wzo#_l)z3j7~c zVEV3Bs*`&fr@zoR{t0fc;r0P;e;T*n(l~ivTb!754%~?c8 zV7Q|rm0vJ=gjuD=@ozL|;UVrlK|I2}A>vTu@D$?*8lUzH2H4L4`Ra*9aVlTna^*jM zjH`HEqx(>2xhIH68mE84IF4Ak0GBKO#RcqbPvatbA7@8-^Kp{ D)e`)E literal 0 HcmV?d00001 diff --git a/flume-ng-sinks/flume-hive-sink/metastore_db/seg0/c130.dat b/flume-ng-sinks/flume-hive-sink/metastore_db/seg0/c130.dat new file mode 100644 index 0000000000000000000000000000000000000000..462c9af1607bb560599872e7f2cc60c73485b45e GIT binary patch literal 12288 zcmeI0JB%GQ6ozLun`AfHJV<~*AcR05;c=7Ww=D%KL=gnyRX~Mij_nT71R5GdheSaU z2`Zp~id1wINOW{Gh>ng1A&O9eghV)V?%vIky=xaV2zgff?~Z-$XZxQse%$V@uIqaG zovzcY`&hTjqJDcB{eNTgx}|v6+vlm~%DndIB4~peUt8Sl^7q+g<)2>v>Q;29)&4K1XbD%lU9B2+S2bu%Tf#yJSpgGVS_211WB zCq_F%qwEZ(E8?Jr08^Zf%6b?~DXw2*Btg7}F_%Ioq=GymjAJRTUuWcqAt7KYg-%E_ zMx_CVV%z~n3Jnqt#x#&_a|4 zeu$B%5WSF9q*KaiA38F>dIK~hqRp^D6uXkH9usA?cw+GARb`EOfRQ7h*GiRJpHe-`;FXHS$M_&4vq7zumBmm(CR%tv+46dSc0O=B zf>y5DORZ_%P_Ep|hZt$S&1hnA`@}mp;}w`#+2JwLMjMk*o{v&aXM}BY!IHRMGSWl^9`enD+>+1< z8Boz_X4~w)EbivX^56e=_;UAB*B$zu^)uENCf)dp^$XTlSocnP@JrU$ST`m;@D*!j zJwECBH>|I-o}6^;Th=SAXC~eM9qZey=O^9w1M9o27bo5OBkOytcGA_KSU+HWV$wao zuztw;)TArFv3|_@%%uGvte>;KIB9o-^&0D|te3h|b2s0A@@oH&^Z!TP9;y*Lyk2}i za7)GN1v=}~qof21MoPDtMH2E%#YDH$6){8=Ig7?I>L{fu#?4o6zyM3Eidc<$0fuT>JIv-=1WussI20 literal 0 HcmV?d00001 diff --git a/flume-ng-sinks/flume-hive-sink/metastore_db/seg0/c141.dat b/flume-ng-sinks/flume-hive-sink/metastore_db/seg0/c141.dat new file mode 100644 index 0000000000000000000000000000000000000000..c56f879992a03fd2bead84dffe3a51c94ec2899c GIT binary patch literal 8192 zcmeI0J8vCD6vxk8$Jf_RoH&W&yyL`=>*pmivpX+|VrkM5%qu_wNV~H$Levn64vM6L z779e02GLNYL!yfK1o;9)h>8v*DwsLzwJhRgItsKa-E+^L`TfuAoSmJ0z3`7X~(d4S$3l z>f{JL_Bs0Wh?YLoSD>#zUxB^?eFgdo^cCnU&{v?ZKwp8r0(}MkUlsV`>rV)L9LChX z-+u>r*n9uEQO-XXX#&HU@X6o-j1E2o9O)g1+xL6|H_6i`N?zJ;jh`Okw}9JlcfTdP zz~g}pehkjs0LHM!a>^<7bV6BPO%$u-#EMK#XxwUBW~HGp1!NA6BX5+Wl+h%A98Msg z`OIm?Y%2c=m`C1b)@sI+{5+gQKG&wyEKBky;Z&2qF12QPl0O9t$P4bJW+KTiz-i>2 z$x6$lOO5w5oIzfCMlDlG{tTQ&-U}`*(@Fj;EF!NeWi2yFei6=SVCTF zr!A|J{1TjR@_X+rvq}CuTtMC^ky}u>{kS8;ALo*eHDSV6v`R_9sfQ{!EMRph;}d7eq0;#XmB#Fh*W^Z>(@heheGPbEzvWvpki512&QOTu3TKa=qPzE#xy}SuRbP%D)BM z$WtwZm0{IStpD4vgS<7|Q0aAw-+^7^rQnJxn&fw3f_%8Xlvcb><)4t^6%=IJW|Z=_ z15W)UtDS0ZV5ZJ$9d2NQ6;kXqJ(yZsnzb{uQ?N>kS5aVjU1^%Phtw%pBgJbdV6||0 zUbb_tQ?O2o*BgNnrOvI`A9SZ+Oo}&9kTaolUbnNTQ?NmbH&NiKFo(1=sU9{-aR&v; zmNeHsIfgA#yw&s|m9*wyVeCAlZBo3Af^ZWksT1wW(ka*>#XBglm1DV+;nd=zN~d6# z6n9aejjyd!?cDAZOrW@j0$O_`I8XNQ0u=9}KyYdd&yrK)CKT_XKsv1rKY0A;Jf!%f z)^B|U`U> z%*KV{LNSV<;;_+$xX>&Df-A)V!3jllC?Z4KL7kv^pyXA%uA17am&?#}JpoaQo7A;`yJ={ z4*WM9`01e!BC=1smtlT?9+?+QgB9gSw=TB?hnD7eqm)EkkyD)WM~C{n&jF;;0CvI(gZ7MFrrOSSeHaspD<39C4Z*H-mr zsXDJ~L9L}asDq08zS`>(C3TjklqFhch_=@!AvKRmpR;6bRc}FRnw5+*uI07%$+uOi zuB$*$YpK>Yc{MQXoJV$L!xueCoiry+F~7KxzN zQmyUv=}65o#yQKiRBMlXXCQT1#^SuLwNxi%l2KOr%AxE`qz;p+ECQM)lnhdIEBXyNlk)YN(ueA+12dQ)Fi>Q@psn(|6hSW(~i$FJ2Q7?&zI|@PP zE#-6)px;W)wUr}+l9t#qvB?*D_q|8Xv(&@FRy`2wo^MO7j&-+NQfuTD7g$njDenWf)aTj3k8^lKueL(LM#7jUe zc?U`a-7(*Np4+&qf4GIc`EXMJZ5zgQW{g$kYkpg$kIU( z7mUYL>w2es<&dR=G7j@FrdnB*_ElPz0WucHVeE_4UY%9NJg$qVR9_oqO3$N;7jYQq zP-|Z~WJ;efzX;<M zW0}(DQ5gzqEw77|r8KV9afnRmGhbx9NubV@gx9*NJ(yJ#Fw3>PE_fn7x4AO<^khoU8-^C}Q$s;{ai9`R&KPk9zcL8WyVc~(Sz zP^l}MOGa4t)H&cBa1J;JoCD4Q=YVtI|K-4F0vUV%Q^W@lw*iiPgt&mX39$DL;!TKs zK<`DwD*(x_hzAjOlIx|5KOg>luq^+NurdHW0Lvd6moE;NhWWom$x+k_T8e5=#FF{H z1+@sU+o?U7|8qv6P^!uzIxm!ZvSwf|r<+2l*VGw|-GLf>i}7}a(jCN;#R&_0x+vT( z5U;6d82h6x+cXujD;gH6MMXoQT2?g7sUd+^P|J#j zIW-d{3I(;SXjo9Ue}dpJHWdxy!^o7QB9Rph3)S6pFv-RSb2;6NibOU;SjcHVrnaLZ zksTBkdb$}EiEO(t*P|X4Nz-;=e$FdLMIzfS%vGyLMbflgn5$Nfilk|~Fs4?IibS?u zm>Z%V6^U%Su%K@L1VOAQ+b%4qm7}63+b)7EsYR@)D#0&_6}uURJlU#YE~lG0eowY{ zSjZ6-^OlRKT@&ttCtE_yE8XG>^-s2)6F%*lMO58dV=fR2FK-Pjj!7}m^O1UUT);%R_W??b*lbsORw;N)wF*8ooZ zj`$qnC4l2^0^9gJ;MhT67@Gtf#hL$6ocZtL%zqze{`)xdKY}y=BRKOvf;0cUXMv%& z9?*Lb7)1ULcufBP^{W>?wk-d@lt8AxeT4(V`vZ>MAcGL|1K7>r*u)^_a=IBDo4Cb7 zPA7w76YH4E>1J?j;v;i89Sx36P-UT~o58V(#LV?{G&suW%|ec-ueDrS7o#_GYM9Yi zP}@dtOj$o5wvkxS9&3Ohwow%{94QR3jgX*KRY7eV?F6EAM?q~HxWEk&3ToRJL?Z)~ z4Y20twMI%Tn^`TW)hV&;OSPa@=dH3Y)tnk{6>Pt~>)6P|EjtIC1I_{GfOFsj208M0_6cGQiQF5T8Wkfc?)9A4Hr79Kou=$OPv9`!WCj4l;l5I?Vq+ zLfnZcT$%mj^~1~Z|8WAH{?jWATmAlj?Eg2vqQ~lRQqKRIQ)^VB<@~<|b$i_jLU%cr zZ%z&E+U=2QGcO;dNyTZk*zTSTlC%Ol#Uqc)$`APPyA;Gth($=t#qQ+Q_FZwn-3sS` zbHF*^9B>Xe2b=@W0q1~oz&YR?`1d%V?*ET2KJ}Vq`F|+B^p24AE8gQO^Z)&I^8SCC z21&};@S9_p_!6|Bte{U;Jjx<1gYsxjD$>z3VvwZct?{exyF!Fp;T&)dI0u{q&H?9u dbHF*^9B>Xe2b=@W0q1~$14r-wAN}6Dm+t literal 0 HcmV?d00001 diff --git a/flume-ng-sinks/flume-hive-sink/metastore_db/seg0/c161.dat b/flume-ng-sinks/flume-hive-sink/metastore_db/seg0/c161.dat new file mode 100644 index 0000000000000000000000000000000000000000..99c062f672cb0e492bc1140e6ad1c5ace2553857 GIT binary patch literal 20480 zcmeI3J8T@s8OOibJMP`_Es>&L6!joQy-a&|_HC0yY0-@f zH)i9;6&v^J48uUeDn)_-a$_JJQ>95ExQLq+Mw-B$`DX5TlmT;yPDb{H;FWLKueLYxAY=UanB9-YY_U_e0=5FS0=5FS0=5FS0=5FS0=5FS0=5FS z0{>eT_+Up8R<@sTMEAjjBRv88HmdBT> zEP^8Q*wbk(46QB5@1Qj(l7?ARg|&TdXl=72u#2sLt$?k7t$?k7t$?k7t$?k7t$?k7 zt$?k-e|-hMgcsN`{uA{kW`4H1_xWEwJe2?U;VPT{;HhV``qYe=a62rre>@uhCD9v&mlkmp)QuvDJIl}&d{NPqrjddtXP}3?-z0HL3njk>)5z1HUweV? zsrJ7PGlEaDCQST9!8`LXEBJz@DeWtd=K{%JM1y9hqQwlijSLq_{*sa563JgSGOUyQ z6(hq2$?q5$E|dIKBf}=i@0u9eF3DdrGIU7(x{;wv@;8hOJ(AxuGE9^FS4M^zlD}zW zm?imJMus_(|JukfPxAXlh6R%U#>lWp^1{e)jO2YI!*P^=Pe4PF?hX*m3RpkA{TR5kl};Y(F!RZyW}4)Seb$buxJw33f7 z(J;J4Zo&t-owgOQ6|fbs6|fcf|Exe~9ee=yU#NdZ{S@^+>a$#@{)758>Sw6$qGqUF zuI;~}K16+h`g7DJuC0Hdeu?@K>WiprT%B)G|A_ht>OItJT*<#tze3&R+PQ<{2H3ss zBt~_)!anLluAM#3z&4IQ#_>&fUi^)3-1*`chw^_6TWm`pwvOn`$ar)($p3PVrT&8( z0p#3~`Ct1QJ`wp}mtQ0z|LgL{h{*rC{Ba`kzb?Nd^S>^?40CvTr^}y!c_V%W7L53l zuxP}ef@4PfX*h1gpMfPK{wypT@vCq`@Ik)_eRdXp^gSkI9##Y&Renm7!Rh#vl?Db1ApIao!$@T($Cf>L$-i*QcxG>*gArz*Y&Yl08Ug!TKB$8#ReBj5M@(hI1v z-**`<2tKMp*6&lj9-IZZhtt~zt8EwCFH9(?1!;esGe^Z)&*b1?Df_tpYKK3 zkbIVel&bbm7T~hr>!9(d@X#{vRP`ata25F`p>;}Q zCBFjKBp-TS?5X~~jtkcXpJhcFm#Y0cb8rKB-T%+$;29%+4YrK<^W<6NGd~MMS|7Y$ zCNvye@*MJIRL5a7(tn@8cZi34Mg1hLt8n=GoWM^LA9;2Q9xXjuOMYT|VwO;G{4`F& zIvVuTC-MuUCn0uhD_|>ND_|>ND{xc=I+%@J%*HNeV;57gdy8jdcZKWJW7Iv=9j@I? z9N*>I`3}c8)5dWZ=i86?e0vwQL5;a~cer-MR+o=k`=}35w^47Qu5j(_qBf{8SLZS6 z9_kM2UDQpkBS`PTS+NG{3zA4M#!(m;9sy(sfPJ2lQy8YuI_F`574%@XE%U!FKO^(MEtzb-!~^M6A9GK?c-o}7{SKcR8O@{+Rt zCuII-x2N#$$f)uU*_8QTm%k$Ozb=1O=6_xOn#}*Y{B@cCb@>}I|LgM4$o#L#w_KV3 zb@@U5Pl`TEOo{USdrId2q~A0w4=Y~}CyB`ax_PxrME=+1dqm`aU4EK~{IAQ;5Rw0N z`B@_Jzb-#VME=+1=Vkua{eQkL^S>^?A@jd3e;Gvn*X1`seQeA2hN5GCw>6v;$iKTLD{vpP&LiF3(S4exAboEHeD0 z^v=HB&hoQJ?7e*)KSbR|y@k5MRVStQ&^3+wB#*eOhkGNtIL5v8aBtnme5SjHx`TQb eb(5>;L5QBh)BOU{aE8RFF4v>-fBoS8?|%S}z^?`X literal 0 HcmV?d00001 diff --git a/flume-ng-sinks/flume-hive-sink/metastore_db/seg0/c171.dat b/flume-ng-sinks/flume-hive-sink/metastore_db/seg0/c171.dat new file mode 100644 index 0000000000000000000000000000000000000000..e98af2e396f3c894bb3121b70f25a39415188c23 GIT binary patch literal 20480 zcmeI3O>Epm6vyA#Uhi(c+a%j0P0}<;x1XW*I^Ok`f_!N!RR~(z1}NoY*WTrTgoFfv zBLWc#Aub#WoVZ1B-~dRGxFAlEkT^$zb5(IbLV(l*mooEab~A3WJGQtKG>-Dt9{-+q z{9kW8|9K9q0|24FY{_A?G0LH{Uv8u8_r>x+bI;8ybk7^~w*WBKfe3wby4`oVv)|rN z{`l-NZDG-sXW7~Rzz)l%*4=;&D0F86_FWTcyGEbDS+c`h$?IaT zL0=8plPBS(*u$m^*gG3P(9oG9Fd)2ULK&6=hb-alZmJ~?pmKr8T2EASgU zO8fUU#`Ea;>E?T{27JBa?>~lfv^m~$CKG#Fc!Qm%mi>L%{tM9TI_F)}zw6`v82oy5$V^GY(8zV4~i64VO$q; z@k20~i64d%^3|Z)j{LZ(xE_om974Vp*BVYGb`?B13R9W*Wth&yAA^}JyfFZWGx39D z7I{8r=PKvy24F4|Un27|&;Ee)of8~`BboSFSde)($=L7sqyR@V@yE$x7TzqvQYL;9 zmStWw@ks{YSSJ27DQDr$5jdWSpMeuH-%{y+f}G64n}cvF6Mu+=Pss_Y3&LC=;T1VS z`3+bm;Z-?7b;{;35_(0f4NYt+h<@S2=ZQ6&tJ@VcC!x{9n32ye&womJcl=gekzmtL`UOnzXhIOXu_~cR!WS{YX~wOH{-_dY zmtNxM5V=jiLQ6570p0%RU=Nb;W!|MA@@N%@UK%(8tT^ozFK*F%w!~3A_F{I@_7Djh znBc~KYjw5JQK@IbFbSJHA!;?8mfyL@_DmRsFv0}-5!C%krE?BF6S|L`_81)NX03o$ z;NK`v_#Rw<{HGXgpigw$I~X@vEc}k`KeAZ7%3|R(i}pG)7crJt%-u!)7V=dVb6eQH zfpHCEnMLa!#%&gh3&{V?V&NlPe}zTsHZpCDRg5-^<_^Y9jO#28Tx5LlT^7xYtflxZ zu9w)laTntj#tn>X7|Se@dl;OVs~2s3`}yN`I5fXmYc=)$#wL!RWevtGm|M|EI3u6Yl@h@&|?g z7rg4GO^U+*r{&9p`~Q@@SrGm|Ek7arzu;FDLo;%caQ~l@H;07(Ps>jU|1Wrzzam3~ z`+vc++j&2~WF8RyU+}7dA{i9^KP`WTaQ~l@H%Ep4Ps<+`{$KEPMArX&^Mw2VwEVd6 z|AJSIDao+#|7rPCg!})Lyg3Y~ai}DC<>xyGXJlSAt~cjlMdp=y9wCp&yz2XI9wna4 ztFC9OKzx~3_1_vKfy}GEzt$)@C-bW7-#S1ZlX;c?)&zN6=2d!HlMse-f=WO~12rtPAN=cU>yeuauH9ZdD6LNx5)RT}f^~?bD{TAo)kTCTXm8`kWgJ|9Rr(V$t zXa%$aS^=$qR^XvgfIinwJ+tiR^XHj!=JN>L|C{KmxvRH-!Mz)pRbp|kx6cnEv&HsW zRb$mBAX-0W^W;QwjWfA=2&1eC`AtNe^Po%H`I-s;r;Gx_`Go)XeAV%j{$I^66JGxp{ZWqY4~qJK zTK*K_^?$*u2GV9Z;r~4qZ_OqAzenry{q-*<{J)w%N_hQW^j}$3Dkc2Cuj-Gvl<@yb z-kKr2{?Gg4(}GX`^BorT|Frxu!t4K>uPV*kILP&XC7-j2sQ))2-}R&l*Z$83856+k z|B)9(fm=~N|1vJ>|7rOdQU8zXmAX@HcdAvro>wwPMg2dD8$sZ%D*JN~7SXDv8PzpO!x?>i=o^S>W~mwEPiK|4+*= ziTSUTyirKZe|f6E54j@Y^IwtQj)O|9EZEv&{wrSfDnX;Rd%g82zZS?@yxv;Ps_z8d z?^n-K=ScUCuUE7JS^=$qRzNH8pI0D%k9plZ`scjO*6kgP=$3muN08gXtZf!^8yMFx zma+Xdi}G^>7H?@|dljS2qCCG~uCtarGJHxH0mfH*nB@RGie` z8Uc-fMnEH=5zq)|1T+E~0gZr0KqH_L_y8jCHv8OX*z>VAPvfZ{b%W1cj6VI8cT(9f8?33)7 z+q>6ST6bEVmwGGR-fsJ~l}>BFwR8LWO8vFH{oc*$?(I&mdv~|D$EKN46<=uF>RZMi zpecjJ%2p8^xJO`MuoO1O+O$K|CdT)Mz!uI6in%XxZ;Z+K|#CvfEqI~F<(H;$^5`0|sw z5>e%a=$0IhYdTBc;{zP4xCV-@Mr0W0a}3WNV-K^lV2zY*rp;U$Jt5<|j>%?uZ*$Fb z8oe#0>}{vm5?)ZPRSMo(v*y(k=x&$(f_Jx2xx`+ACHbBX>0qIdZ{$mrjWXl`e$Lkm zm%Yur|7(`&?H|uns*Pf$T#uw3z3Yu5yMnr%dJ}IX5)6_qf*e_s&2e&=ZrZWTR9Z<6 z@q$Kvwd7T6#jEJv8_Ce*Z#YeenZkvmk@rlPIsTIOJ7rRAo>Qh}rdu?`C@FKAo#Y** zxS6kAyW(9-q^JB9r%Jfs4=zR0Y=Kk6cGKG~lmsP37TBC**CdFMXPg*;G9?{o%!C{m zNn{Q54Dq9JtxBRTaRC-HzH`y$H;K3DK*U9fw+vh4B(uzn1(MlHlAUFzgQHEMQc8q! zCphJXiynu7!7*p9ny=+IlNWPT@95zSw7N(7S!{{Z&v7$32tg?03f+7W12E~orRF9^ z+R1C?`6*7hV4>sCY@pN4i|RkvRI}MhPPLwy*@jY)RZ>m#Iy5^`l_Ede?{$MkYYIY3 z&pHId5o}9eHNMW+;013*XPe zQs&!Jf9sqbu>Tj?0xp~pR|DT-U<;fOhp_)S54K&)hMnOmd2o)~u+6nbB4R@be6pH( ziBw-|gnFP}61GJ(-)MN;)ILQcGs%gAQ?qy!JJC2kuW$Eu$7jQ+b>#0EnvsK3*YJPGD(7IS|~G??C9db_#<_OUWS*u`?i4z3LT; z`6M-i7dewelucMg@W~7YAz#y0`VI7F1sC<;-k|~7P*zmZlESA?R3FwNKeyT4vf(1)AxE9jxdnijPGSgxpof*Qb!4#aL18RWZMu|+Lg&DXsx zxM!X9ZDp&^vvZ== z>ow1N(Mz($CU0>}@gui3!%p+oW;)3Q>XE5)V48oO`Y;}T z)_6;l_MyLS7b5Kcv*K#tTX0+8ggBJ;ziYW2fUk}-_E~K;U%1k&Cdq1-wt~DCE?+jw zZ=~26Y);&_R;)$JF5JfDKHVOr_T@6RZ)UrlmSvfaI_#HWDZ!ed+HK{_Yg=$Z!0^~Y z5Oj-z@x`m&)+S{jgxs%_7*xf8Fh<=Zs-dAafU3bRta&A`;mJm=@fK1>Lv##RK&~7V z)T6NDo3$;g-GQQ}4)FwP0O;Y&Zm7#mE}Z(pldG>~^}) zavgPaZa^cfHK7I|H6_-rnFRB8IC=q%aJUvh==#Gd@@YhuQ)?jcIJv&XXIVX1e2zLK zHh|LsKwH48Sl&pYlMmrET#L!zl=H2oUe|<6!BK~v`Z&S3?~fjnw=mWzOKhDw)umU0 z{1jb_AWla7z#)37!MNoSZ*VBITg&vVv!0!XkIqpCfXGM3turLg6s}Czo(LfJWsX=x z*%Buquw(PaBu=qLMm%=$R}L}Otdwu8Wx5&o#~pQ8V}L7%<;$do=m5s$Fjo%OLZVIk z*tEXa-xO#^7!?-M-ng1=+BPCRjyhzp#1@KWh>%ig>(fXajDi4|9;NC3s~m!XQe~4v z*6MV#M%kLvR56J8HLqH#T#M_cz$2cEVtE!NQ=tT=_(TDk@vStg>mYpVs6Ffew1$f2 z3tm!y?h8PJrAk0U3SAJ;D6jy562`+iqvacTY1>71f}{32BRiqyZC0QkC&YNKa-4#N z#=#^QZnjxgupB*0UXS>EQKf!s*|KoG|9~T~_-n=w!RCL}cm_8A)5deK`5!hOg3bRm z<3-r~KWRLRSoc-qMjjpU$0#EJ_WzSUJTL$8KMdIabI9RF&~OMml4B8@ zwEuJ67BasawS$5NEpkJd#+31X21qVe!2YV2}+$419)BS+IwJF~-NheNoQ zL^vRSE`sc$aB4Mep=6L#d%c}rXTQGR!c39gt$m;|0dh;yO`fQuW^m@Ltn`a?yGVm} z)Q<8#L5rKo_TyfVpaX>qa*D(d91|DQ_vHF9!kD|gJ3FmTFS2`5@zEVO{)*&_n7$ix zdYFr32@3ztz2<}DEZ6t@IQBMXFgWU%Lf_W(X%rRaoPf6{|A(AEd|NYEHbFug&27Ur z0(+dkaci%)yIw%bWWkb<_QhM>-m92D_MU~q1bJwfDvrXT1=Q?T3=*+5PKToj7h3;1@3o|zH z$aelBmk#P`2>J-c0LOho4P+AzM!2iZ+J?7Pt2|Hnpr_Wn^;&*o6CrPTYOw)t4AK)l z1aa2JBw}orQuH>(rM}`5>WC2?&rbPHMzd$NEHkT)VfCpTj$xH6&*zJY^ug^Qm1C+Q zsVvCbr*a@N6|Ztj;T45d+IJha9LMRHS#@;jN0=*mM7WZQc;8lXIQ~Azar`Puj6=?% z0LF)er{ctulCPEC5j!I*^?}?kT`}H7F2IxI7nsi(e}`EBzhV3mOuLUwy#bkieCk8+ z0$9ek(D-MKe-{uGzV^`fj>7&&S|euXXVu|AF{^(a(ooFO4Y`FUFsuL4pK~szS^Wpa zQp>65(_oCf5CVtIGN#9qpDu*D@l;*Es=TgL-^g?W+w!e)%W0V)hC1dba<(NZd=gtC znWi(GB9yZoDjUt>?F-rD%52lw09hXViFi5N_RVmo*Y08fQAa~R!ah9gPD294DzXJD zTiy!|zA{9plTE_x5-AF!REB6wTWJ2qB+c~2o!jlbb?LK<8rP{p_uE#F8|?Z}1-icI zCh=mx@lExf(``dzQ-_7aOvL?T_k)S*TrLbUQFLX9UbKaLwny*0!*Vc(G7reW{jhNa zjCIA{vsoQiLLZ0-7Ku_)v%V$W?rrB_wy9&=kRHC-7I5_m_E@h~Hj|VxBK2al>IeqW z?m&sdA#Ry_b_d$7Z~MdS&w`26*#IA9PYo~;xdZPH4+(W}q}oX6w99bf(I=%vH@3^r z&iS5;&UVJeIH*pr=zA_|Ua68_!yJ}EvsP?e+p1Pd#X=I{sw^J>efLyMEhJOY9vLR2 z1F(n*XF06@BxakWvvvlGq&jD!pO;M&cbagPZsiLFOgc%zW%@M@sg4K=79ab#1?S{c?6+Nuv+LOY-oS}e`-6M zhRLap|M&Tf5+{@3vp|a6%RxR1moJ+obm~|eL12KePoMu{Z^awM%OW%|CoQAr*<1>s zsiWup>_Ts|3UNevxgK{f7xKvDOrVQ8963LQI=T`PKiVtBQpUG$)7xzmVo4olhKHg6 zYqn7H&}3TOuu&OGC5-hM{V-b|S_$&T%{x0QpXs(cD`X_^-`?$Y`^!O!6yJWH?;3A( zMESv{QXXYYqLyHlDB@c(kecn97?sq4RJ?#7z-ZuME@cuJ9AwR060_^0Dj-3iRzfz* zela5}7z+#TkM9|Yk!X_jHQJn+ zf$VVAu{bOv;M1h!l{Xrf!$ZZS$gso75Gq?DPrQB)Jz+dS;>s|bYh#pg)nPa+jOD{r zZ>}aQKfcP58ZKUfCB;Uh=jRhY7a=#er~r3@;!24TBNtR~A1*O+{q@MQP-|RuEXe0h zX#5Fa9vIZValoAiWlO**%n~)8`TgAomM^Q!lNd-o$72>J@0E{4(i*b7AFlETP#Q=h z)C;UME9QP%ZVn2$tBwI7AV<6}6BH&YAp$fIDf5FxOA?T&7ej(EnVP>ZUhT-H{OJYT zW*ermtB$!qCtU?%_fsbwE?;6t`*C#AY?__nb9;Jj7oHMV9WemcQ5Ej1P8v}G34$15 zIK8Qlyf>XRn~|@5AMH<08;{Z-^%)S!3*R)-u=b0_&p?rU$9M%c|5uEs5fV6Wd=Vn! zXUWc=zD@T}?a=+kF8fb3|Fg_(_<>z@FyT@514Oux&c1rB zaSzK|zel@HWa!0LLy}nzz0m_@#K^lKG(^#u5%j|+*m-|h{fi%e<|5X(ySQ3xT;z$e zejQK<JyM1_e+HyX{?D3r`mB!>(Vw@8yWUA14=hadX%BB}Jcqwuw1j_D zO@7gz-zqA8q%fakBhnsqrs@(49DJg5dl_`?@!%IA8>8h6!R?tWYd-FWK-Sk}dIVyhedIEOtDe+-r{uoh?s zjM9UpVdA`evL{FMVyvv|FQkw*F>NBJc60Zkp^#OuY9W2>$T%_(GP@jvCS1ORMlfTn z;1qls1%q6q&TVBdp(m&IFZa77vML^FL2_V@yk#ye#Lm5dja4VFup^60oeav76AVA z*Z=+#1NQ#{dEeOl5WGHfhtD6I73=?p7Klxo+sm154~`_-x}97%({-_`z*4zcD0$8L zR=!>+7OC&zFa7w2JG<%Sb_PR*)#<`2GR>CRvQlX*E65yXQ82fQfAYI^x3kl`ySK7O z;oFtnZu{omN~_ztgZcI=YxIAAlvsZER%idl?OUOkt`POdTYH@wH~Q_McG1lE7)dUg zZ7`#Ndm>XdBRg=)b}}g|CI9v-v^rumI%n7^iOy!ef!IFJzey0EU*LUs=4NXjiy!Ru z5h8vV0ik0JWSry18Tm&EQ8Ev3K6wd94)=d?CDvCf08N|yTImCgfJQ(gpb^jrXaqC@ z8Uc-fMnEH=5l|w4_5ZL^AnbpC{l9mnzVaz<|C5zR8?F8)zxkp6B@E%<8GWn~&!A;x$gLvp)8Uc-fMnEH= z5zq)|1T+E~0gZr0KqH_Lcz_VN{@driukHT_h+w*^8Uc-fMnEH=5zq)|1T+E~0gZr0 zKqH_L7y|)q|BnHhencan5zq)|1T+E~0gZr0KqH_L&~zo1{$#WH*<|y#Nel=Jvp?BPVq&JJr*^ipGt-^!o+O*Bw)>MH zdJ;@Pyr@yhMbMMtEr^0Q5!9nzA_#)wA&5xMA`;9=5JBIop6U7P-qhw~pc>|FO;z{T zuU@^XuJ?KtZU8_GKXfFyOmW#MTJ4>6SpK(p^i)TFwXo%7{HOpUt&?JSbNs&Pu@u&N zXT^EPukT>-ietawpG4kojQfB3B){jwZrpd3tdJIv7LXQ@7LXQ@7LXQ@7LXQ@7LXQ@ z7LXQr{4DUs`#;`>>!^1tzJCrVTt3m)lU93ALo$V-dmP>dHOK^nmYSEf1N|7x#_syH zvG3fJQ>KLAg_k@fuaY$2cDn9Ap`txe6$&z+pg zpSZe?U+I{hox3)hhwZSbT1o>6z0?b3}b6v<4$O!zmHQd&S-Kw3arKw3ar zKw3arKw3arKw3arKw9AOx4;+RVkb_+D}GnpyT|_SivOR7C$VL*ucHz1Kco~4|KI8i z|C^U4g1EXJ^ta=8jPjLaZvbDII=*&%c50@uic!2Dz@NOjQ78b$=r!hQWzE%EaeF)q zsyJW>a?bgxuGN_1Xcg9i#1u%2Lq=hQAZFFFjSEJVRhScih9Dyj%0b2lHEQ)XVi2<8 zh!Ge<#DZxvqf%dD%K;{?Ku#P(nGwV^>bh%~bxX6gT0k05U{IXoTewAPLsBp#4)J?c zVXnJuF{8dJLyH5}K_1AXEd7!#+A!6^31)k;+sG9l8i zLmZWbbPE+I6U)L*aY7mtBs9uyaD6-lyZWx@DZ)ZXiV!Cs<^^4?TJ_FpcFXM&L$F(% zfz#S%HS+@NLX5z;I0DtMnN`*FZpJX|31@_5J9oj3X0}t)ml~Ewt7b`af<-Tpg1zFj zS(px`d0M^fu{w>uS2Zi1X6uUvcD-hn+jml6Li|prps?jTd3E!mX0(&JgRoEhhWlY6 z{DwWy9L)DYBOg^bOZBQ*(ru=>tZM3-YjSs5K%Yvfa6+7f^NUC}t1=wD4iu%WQ)wuO zqxQl$p>ScfOvj+5e!(_tj%yqB3i&!$E(0gUi4(9VgxH}ZM`q!aI1*Q-CbG)v6?ZXE zEC)}CQ*hC4Qk+Jqr-y@ZS{&3=$6;=p!Yw}pXT&M^SR{2^8y~Ug5Y#6Q8HT6DA?Q=3 zNS_*kv*IN5DI!Jq)HWy<#aVk`H?ex0sG= zaBXUCePfmO)T{Z`x#^kpd>+s+^@d($4ad_Q-7s3yO+~|0#4+1oxP_@{uD;kFd^Oz{ zM92zhf&Z5UQs3jdW=LIumjDJHzzZlp!gtRw;6WMX1nBtU`DZF8KY$|23s6Ft1^hvJ z;tnjJd=<{2JWl2KA7G&T7%C{A<#HStzFhx-&WYWkniwM{{6lBLSJVnOKm$nUsjU7^ z9OXsgtGDSK-=PVun#zjx+Ym&^GWn-qI4Wp+-JS;&F zsn$W*@59Q5v-u`n4M`Y8Vkzk3y=9jy({{b{4Q4M>cS*x!SjHd{bp$5+Q5Z{0yq9QE1$KO14soCL=p?YSbG8kd-AR;#EeiA|tU5W7V7m?H06pB%Ysg*s4=?&X( zmp#j@8hWsJ4CJ&GR#F{Z%usJ9W+J_nq(&FBJ#{>sNzK@sGc#Ms zTLd$R`rb2xh&C_?`^BF++;gvm#s(TdhX<5BPs92TbqeWlTD z(j2s0EqL{z48xH};B{!%=5hUk|DNpJ@X4~YfV6I=KYDY{{g!w-+ll9 literal 0 HcmV?d00001 diff --git a/flume-ng-sinks/flume-hive-sink/metastore_db/seg0/c1a1.dat b/flume-ng-sinks/flume-hive-sink/metastore_db/seg0/c1a1.dat new file mode 100644 index 0000000000000000000000000000000000000000..1cd053a6c06b93fae3900e34cfa9f0b31af89bb8 GIT binary patch literal 8192 zcmeI1JCoc*5XW09?MPbf?yY$oVDrYDd3f>YF{2R}EMA9=tH4my1}+UXLn4MRCRs5J-?Plt$&ZSTDS%P4*uC~LROGH zY7WPb#?b%wV7n`P-;Y;tPg9pdY58N&uH+o|)2#5HQeNo`dpZL#_M-FmK zJ{^-g@G*H~Xsp0kfw2N(1;z@D6&NcpR$#2aSb?zuV+F^TWZYJ%oK1@DW@+82OeW z_)_9mbo>cy!Rg+_os_X$%Q&mod0gvtqf)(=c59m|tqKV)I6ef#8<`}?#~dF(NW3(O z&Sac$d8s2Jx{aeM#|5uZqvU63Mk;pybJ!gApCv8w?EV+<81YGyWsOuFH=iXu zPQ1=klS$Qc{02NheBR`ll$z_W$qG&qZ!*&$&-Irp;MBl3RhCK3^_T0zlf+k9-852j z{pAMm6!C4-;`P#8f4Lz%O?;Mhs0YpUmpg-Ji0@j7dedBgxe+`&@G3z)Yp%cCIXp*v z+NRhqn(Hrj0nZaJmBN11Tz|Psc!Bt=)!6Tv>o0c$P7|LfgX@Xr`paFxi^Lagh3l2( z^zRASBEC^Iu7{e_zvsgl;;XjB^;UEG_X2o{c&&O|&o!riFNBwg@4Fc97dfYYZw9Xr zZ*+q9qny*f7s0E<%Py-jnREL0=5UtyN^9N7oYTLzfHm>Bx}`?uoc^Z*UL!u!l|-J? z|I~+b#OtnZaXy^>rvXUfJKffJzMTH2A;iSjU0+~5IsH#(kPwfh&ynZ!KaC(IzUfni z^WpSAokK=^lB*QYm(%}r0gCvz&l9XCr+*`@B6x%NLZ&z$!|7kl;XLt`QFy+F)4y22 zo5Xi02qO)rf3bwOh|f%c*UNDF7aMSa_+Hki2gB)Kte`B137HlJXSfCsC!l`BA*fFD5l9wj7&!lpM|6Zj#N?^8l$uYojN z7x*(Mw<)11DqOY<*9LwBAA!d6^R8MB%-u;CjJdIpxPg!rc$A`8qE;oT1%n-F9(m@XJq# zSNh73aOcx#BKvjPcBnycfQc63IgKG~&bBOdv1lXyWq*`?+EJ$lshH6*dTt##^F4CZm%1_KUe=g^&c(_0J{JH literal 0 HcmV?d00001 diff --git a/flume-ng-sinks/flume-hive-sink/metastore_db/seg0/c1b1.dat b/flume-ng-sinks/flume-hive-sink/metastore_db/seg0/c1b1.dat new file mode 100644 index 0000000000000000000000000000000000000000..9d0fe202a0d2eaf1402cce28873e1655454f43ba GIT binary patch literal 16384 zcmeI3&5ImG7{=fJtnT?vj9)PuqlwYWKzGl__9Uye8k4xYin4eS5AN<{1utGSih_us z;K3kX1aDqMyoumR#7ho}2N8)L_2xmue?YXJcV_l$YIJ&E5tM3}XJ)E;eqHla^;AFA zyL6d|9R0xNvxcm@Wm3PpOuznPyzMKV`Swv=^PGM}M6JnAjy|}4|Cif}srKrVYj2Ip zcd@wO&Y$&@my9ai(bJUjG5_wsQ=hU0)&c8)b-+4c9k32q2do3u0qcNuz&cXtl0kJA(5x>vbOhxs4%i%D2_>8wwu>C|SDcJd?{wv%LQqn-7;gLblbX;Z(&mUg;Z zorI3i;kCzB(_WG#gDgo0oqW*gkVkHFn}-A2%{pKmunt%UtOM2o>wtB@I$#~J4p;}Q z1OFojJ|WzKU!oUA-|5>cD{t<}|4&m}*E~G+jf1I@Q9-D5WPg2k{2%Y=ZKvy<(7%)5 zb;?)X{s^D9dmFt@JImK~jvqz%!KGoI8zOw(PjrNL6h$J+B2yxYWt0?ciK!x^WS~^2 zDx+X&!>LkDM(Ki5qq>aJ1Eo$48Kn1$SCM(w@TMbD1#o_5`=VvjDofVif$B9`e;iS(M>W6+7iZevy6f>Ags{Q zDGD!o=JPFjb}?SO2Hhf~z!$Gc$7B@v;hU$DN! z+Jy$cvVI7yyhj(buCdmj@m1DWSQnwuZ>%3-|7gg19IAd`eTQ`w8f>s0gZkH4-)3EA z^`YM9tS_==Q1=(s4_KdKJqmTcW4*$9k=2FLXRMc@wR31&^>@}!S)XOS3tIUJs`LLD zt$KTT;mhy$%)p*9Ou$}b$p57%n18+4kpD|jF#UQJL;f#A>0*ZQs)qbuih{}4 zs~PftDGIJ2y}BX)m!hDj^iF^Bh1@~Ov*f-Vz>wtB@I$#}`(}8G?{6Ei%!@s{qu#^86zWRFk EPj(?>DgXcg literal 0 HcmV?d00001 diff --git a/flume-ng-sinks/flume-hive-sink/metastore_db/seg0/c1c0.dat b/flume-ng-sinks/flume-hive-sink/metastore_db/seg0/c1c0.dat new file mode 100644 index 0000000000000000000000000000000000000000..c5b91e2c3fef26113800b2a821fa2bb19fdf1105 GIT binary patch literal 8192 zcmeI#u?@m75J1tpC<-o8(9=Lg%K*#}rA3TD3rJK@(l7zbumejVQ7{4=PD~;b3K|OZ zBiT9MS-$vnk+q1Zjpvj2IrDRS3iWo*;(Pr>>5t3OD$A-0<9P3_pmW_xQUDh(^bLUeR^QorsGgwRhxSeZerK=R_$3sS&>;9{Cwm(lE17DT= z%L&7~e#|&&;?Fg_>sjs^8XE{e00Izz00bZa0SG_<0uX?}KN2_`w?@`_v-rTyTvWW> zHRjuiywH)sUdnn9&>7WL@ttpEWNuzHRmj@WgPjG;WDzn~r|3|^1{ee&009U<00Izz X00bZa0SG_<0>2TM%F-89r%Uez@u)7| literal 0 HcmV?d00001 diff --git a/flume-ng-sinks/flume-hive-sink/metastore_db/seg0/c1e0.dat b/flume-ng-sinks/flume-hive-sink/metastore_db/seg0/c1e0.dat new file mode 100644 index 0000000000000000000000000000000000000000..6255f852394ad4f811f21dc9c26e43f74ba1e1b6 GIT binary patch literal 8192 zcmeI1&q~}-6vj_XYDdREEu|EpR12kv#etjL%*_P4Q74r^n;DZC#JU(VH`*6y3*w@E zfi7M77Tvh^0bKb4T^9Nf_T-MSR%gP1uH;JrB|iM zUyb4X{9;@Sk6gAf^LdbcR9rNJ#Vf^P#QE{C__y=w?jU~Yb3v0h{u0sTWG<15X!)tiO_q;l@gvn83xJ75)WWXl}I_~l6(Eq5)1Im+K5-9POsJOgsn)qvK9NkrgfNBj*jDdZawo# z9bdwt)ePPRG4|YxgSBq>el!jps>)fidddW6rOspaIx_$)S)(Cbd3#3#W|d>c9bXC` zUwz3=7_-MY<(S&3dq)H2m80;|jCoG^ZXK&f12-!TB0vO)01+SpM1Tko0U|&I vh``+t(5E2ApZ3Q}ca=WLRXb7o2|nWdX+DRaAW?T<3-v854(e}ahX;QF2UN04 literal 0 HcmV?d00001 diff --git a/flume-ng-sinks/flume-hive-sink/metastore_db/seg0/c1e90.dat b/flume-ng-sinks/flume-hive-sink/metastore_db/seg0/c1e90.dat new file mode 100644 index 0000000000000000000000000000000000000000..de30561b173c8af6549ade4f3499bfa20ab95f1c GIT binary patch literal 8192 zcmeI#u@QqH6oBEEB3_CVDbhGWT1IggwYA6^Ss^o|&kQM2<=jFa!og8+YATAKJMhUv z9)2Cti^xHHY*h?>l;%Tv@-0lP1dZlfx8`CWSM6slesXjB#?N5h?u%tF8~S73M*Cvy zkZ()PzY6E~B!d6~2q1s}0tg_000IagfIx`?kKgetot`Yc-*3MEm-tVd2>}EUKmY** T5I_I{1Q0*~0R&Qk`gwoNK%x>A literal 0 HcmV?d00001 diff --git a/flume-ng-sinks/flume-hive-sink/metastore_db/seg0/c1ea1.dat b/flume-ng-sinks/flume-hive-sink/metastore_db/seg0/c1ea1.dat new file mode 100644 index 0000000000000000000000000000000000000000..1a4f3aee6ac72569c8300458da83e2a533f54f11 GIT binary patch literal 8192 zcmeI$p$@_@5P;$9#5Tzj5FBzS6c0fN3WY`k2@ZoK!J`O{r{F1g26zg%w(HW3L4qZJ zlik|8Zp}WoleLH#z3fO_OkIpasK+^-Kg-cbe_W2WveYXV>4Z&257+hA(R%!4_~`kk zqIbDO|E@PPhz{}lYSEpUZ)-Ug2q1s}0tg_000IagfB*sr{G-7BxG}QQFAG1|iJN6_ z-x})8LYisIU?+Li2xy&EPlf+{BZay7Z1+@FZR^4I0;V!6D+ia+ro`1JhzKBn00Iag YfB*srAbl;%Tv@-0lP1dZlfx8`CWSM6slesXjB#?N5h?u%tF8~S73M*Cvy zkZ()PzY6E~B!d6~2q1s}0tg_000IagfIx`?kKgetot`Yc-*3MEm-tVd2>}EUKmY** T5I_I{1Q0*~0R&Qk`gwoNK%x>A literal 0 HcmV?d00001 diff --git a/flume-ng-sinks/flume-hive-sink/metastore_db/seg0/c1ec1.dat b/flume-ng-sinks/flume-hive-sink/metastore_db/seg0/c1ec1.dat new file mode 100644 index 0000000000000000000000000000000000000000..f0b7823eedab1831378c895a40ca1014aea49cfa GIT binary patch literal 8192 zcmeI$Ar8VY6oBE^iEWY-5F8Q&L2(EKif{r921sxiBqzY4P#gg+!5QEb@Y=3RHwFoo z{7v@OzSr$#pIc=uB1SJe5*Je!;}Gg`PUp{J)Y2c9W34Rp%0*gXlhNk7{yG}lpN5Z) ze=2&HOLXsgGySNE-&cvQXTGiESRjA^0tg_000IagfB*srAn=a@`@_b_O1~`pU?*;t zy?tw_*9&Q;M+Q5|%Y%T%S@~4>&o|ODH=k`^i>f_(ul;%Tv@-0lP1dZlfx8`CWSM6slesXjB#?N5h?u%tF8~S73M*Cvy zkZ()PzY6E~B!d6~2q1s}0tg_000IagfIx`?kKgetot`Yc-*3MEm-tVd2>}EUKmY** T5I_I{1Q0*~0R&Qk`gwoNK%x>A literal 0 HcmV?d00001 diff --git a/flume-ng-sinks/flume-hive-sink/metastore_db/seg0/c1ee0.dat b/flume-ng-sinks/flume-hive-sink/metastore_db/seg0/c1ee0.dat new file mode 100644 index 0000000000000000000000000000000000000000..1a75cd6bf01fa524e7291c29880aa4b16ca0207e GIT binary patch literal 8192 zcmZQ5V_;xl1Y!hWLolH%RC&1ce<*+p5MTrVn=YWXeB(Yi_ee-uG zLn+a literal 0 HcmV?d00001 diff --git a/flume-ng-sinks/flume-hive-sink/metastore_db/seg0/c1ef0.dat b/flume-ng-sinks/flume-hive-sink/metastore_db/seg0/c1ef0.dat new file mode 100644 index 0000000000000000000000000000000000000000..de30561b173c8af6549ade4f3499bfa20ab95f1c GIT binary patch literal 8192 zcmeI#u@QqH6oBEEB3_CVDbhGWT1IggwYA6^Ss^o|&kQM2<=jFa!og8+YATAKJMhUv z9)2Cti^xHHY*h?>l;%Tv@-0lP1dZlfx8`CWSM6slesXjB#?N5h?u%tF8~S73M*Cvy zkZ()PzY6E~B!d6~2q1s}0tg_000IagfIx`?kKgetot`Yc-*3MEm-tVd2>}EUKmY** T5I_I{1Q0*~0R&Qk`gwoNK%x>A literal 0 HcmV?d00001 diff --git a/flume-ng-sinks/flume-hive-sink/metastore_db/seg0/c1f01.dat b/flume-ng-sinks/flume-hive-sink/metastore_db/seg0/c1f01.dat new file mode 100644 index 0000000000000000000000000000000000000000..ade0cfa8c0057ffe866461f0bec65f6da269fcf2 GIT binary patch literal 8192 zcmeI0F%H5o3`Lz*YCEzc0b5sCm^cJOC(ghKNNg}LasqC^fjTq7#tpztNR^g?n9*|; zH~w~F#rhy$iFTRuycl&2e4V1 z1a6M&wXxOIJwAH5%k|oKk+AcMA41kPX}F*E^sX8@EHMElzyz286JP>NfC(@GCcp%k z02BCM1a9sx1)SqMmFN2*3~*JQr>nOu`;|}>-5!n%uEDDY0gY94S9#7Cuq#%%D&C52 zHhRcr0sC-RbY4iop~NE=y$;M^TP;m21D*IXh*IAKsrFs0v@f$*IIC-mgfLVg?AFw$w*=AXcn{My)sG+`_MDgI&{On?b60Vco%m;e)C0!)Aj rFaaj;X9UC(;J@5rNx4h;o$_s#VU^|1jOqu{HPw4?l0U-d^T+8oI+Tfu literal 0 HcmV?d00001 diff --git a/flume-ng-sinks/flume-hive-sink/metastore_db/seg0/c1f11.dat b/flume-ng-sinks/flume-hive-sink/metastore_db/seg0/c1f11.dat new file mode 100644 index 0000000000000000000000000000000000000000..15209bc19bc079ca210546093a645c30600a014d GIT binary patch literal 8192 zcmeI$Ar8VY6oBE^iER>shdCrD6o){dI070-a2O;f;0Bxm#Su6IIBo#1?YeYhkYLH* zWN+4G^s*yyF?BHxp&sXS{wzl${c$n=YWXeB(Yi_ee-uG zLn+a literal 0 HcmV?d00001 diff --git a/flume-ng-sinks/flume-hive-sink/metastore_db/seg0/c1f30.dat b/flume-ng-sinks/flume-hive-sink/metastore_db/seg0/c1f30.dat new file mode 100644 index 0000000000000000000000000000000000000000..de30561b173c8af6549ade4f3499bfa20ab95f1c GIT binary patch literal 8192 zcmeI#u@QqH6oBEEB3_CVDbhGWT1IggwYA6^Ss^o|&kQM2<=jFa!og8+YATAKJMhUv z9)2Cti^xHHY*h?>l;%Tv@-0lP1dZlfx8`CWSM6slesXjB#?N5h?u%tF8~S73M*Cvy zkZ()PzY6E~B!d6~2q1s}0tg_000IagfIx`?kKgetot`Yc-*3MEm-tVd2>}EUKmY** T5I_I{1Q0*~0R&Qk`gwoNK%x>A literal 0 HcmV?d00001 diff --git a/flume-ng-sinks/flume-hive-sink/metastore_db/seg0/c1f41.dat b/flume-ng-sinks/flume-hive-sink/metastore_db/seg0/c1f41.dat new file mode 100644 index 0000000000000000000000000000000000000000..cec402499509f7c5e789f07030cd7bac5e820973 GIT binary patch literal 8192 zcmeI$u?@m75J1s$iX;_c1L!EBqN0EeD5w~L8j$E{D4Bo_7=b2C!3;13ob5PrLO`NL z-%>92of9Xovt=tHMlahE7gHDG5bAMG`_Hm9(jSL&t*rIRMY>^<(ZzNAwY4sP9zXj2 zspwrUF}Ul^jG{&SyjpZ?=KETX0RjjhfB*srAb;BF`UDXH1Q0*~ a0R#|0009ILKmY**eh`?*%Fmv5F2xIoAT5~y literal 0 HcmV?d00001 diff --git a/flume-ng-sinks/flume-hive-sink/metastore_db/seg0/c1f50.dat b/flume-ng-sinks/flume-hive-sink/metastore_db/seg0/c1f50.dat new file mode 100644 index 0000000000000000000000000000000000000000..1a75cd6bf01fa524e7291c29880aa4b16ca0207e GIT binary patch literal 8192 zcmZQ5V_;xl1Y!hWLolH%RC&1ce<*+p5MTrVn=YWXeB(Yi_ee-uG zLn+a literal 0 HcmV?d00001 diff --git a/flume-ng-sinks/flume-hive-sink/metastore_db/seg0/c20.dat b/flume-ng-sinks/flume-hive-sink/metastore_db/seg0/c20.dat new file mode 100644 index 0000000000000000000000000000000000000000..6a51e91ebb364f7f4452837a914b8a0130176381 GIT binary patch literal 102400 zcmeHQYmgjQb?)9>?asdR@B<7aU>7M!Dw%S>A0X^qX~*)$tCh4`*>XZy^%GQagk(ts z0@yO-0e^y(N-9NRgB1d)A|xOR@*)A7q^N|DV1-bGJStMTl7dPg78ODg1IanJySJxj zdT#e=?|5cJkISu|p1pVOeD~aQ&v(wb_X=+i1VM!V@ujfqB)hg>lF!?p!TtZmua3t( z-yVcd&P1Q?O#WyNzPypV$^TcpDew8!7uokl&o}FeoB8su!t?7TK^TL7x8;?EXK|gm zRre@TKdup|5vUQU5vUQU5vUQU5vUQU5vUQU5vUOuNd)E&ANsIx7Mh;$`yIk|cz0vh zk4Zl7_+VlZO1@P5V%y`J;l3fP!hh?+5n;bLBkYHJVX7g#O!y&7VlTfbNxsw!eba2J zfp5WE$$|e|*_NdA?_`PlcZt(L7G5Ac|M0zs_a0c?w{l>AYi)78b$DZWb*Xh!7DQ24 zcvO75XhXB6ps~}(Px!$b*WYw9fH&Lrtgp3Na5Le0CnfP#)^zxrpAO!%5}bJLnb)yL zBK*PR@zaZ^z35Ny{R@wuK62vtsk6a7$IrZO>Fmk-j(g6TVD;>M_nl^C5Ya~RH=9qk zzy5wdxIggszW%f-d!wi@6h9fGI1LTX`ypc`Vae}-2w=G< z;Mhayr)hspbODWkcJv@Ntqo(1i}7^-Fh0~9W0fqa_@g?0=W{Bl1{cMdDk!2N=&cu$ z@qS9UPS^?KeSLAyN_LxErxE?POVe>F1;4U2H1IHtmhz!8O0;ZMu+{X_R+ z9=detM5AROn$xgU+mmFbIqjxmDR$EjJ+y|<{Ly4t1M_El1^ko(vjGend|it1JG}SK z*8atebW%ol-)5e2qg&z6-#hw;-<3T4=<j5ag9U6t0An_wOmb_qM$CYb!@*F6u9fx;Bn^aY~2sipCrML7#c;pzps&g@{;uk3=rNYzTalAgV3pPi3?#!X` zO_wT82bDipCdz0rwhHcQ-8-^c$mRBL6{kW;@PO0^bI`vGy!?*cdd9AnGN?o+w&|Bz z2V1L4!`HXyNiOzyXj{+8OX7Db4?j95LY?0iPE-;n$xGsQFt(Tz`O6(18h?7D zQC{#ss1S%@K^U{?`!3vB9*&kaFGQmE&d1%mxU#%>IO}cwN*oZF?IEGqrDftIynukx z5fDg20G$yTG#{eBM)wZBi~ct2?kZ(k;2$9q?E)%jy0+cnuA$N8t}1!hC(gtS_Vl4O zFypuD#|BML3(yvB6Eqr=@2ZlAU*ayz;1`)?!{Us^U|L1Pw+o7(scKHgFs6ATx~k-% zdoEU73A!DME2l0~_tACN*7V*LO8 z>^dbp0{s7T;;Z2La^d}u3ICM11XJNdq6@X}7F*GGeBkWT*E0N{9{;bvtTa@ig>gRd zQY>J1FSqU)KD!LHY6-iHPt&U~|B~oEF6lmC-9)`|08mo|IolnpbtKw=L zWrR4y-n&M%Z*{&CxC&n`3E$^(SE&m_y7@O?W~K8rV3k;WNenTM4ZTY380HQYgRcjx1hgeqrQDSD&}=%c ziEq*FffP3#V!X*vSrncp+yG64T$c6L{?_PvxZ8fO*9?3oG4kWzTZKQjRmp?!P!9-) z%fXF=4Vw$`z+PWl-nXx{HhN}w{;y4XH~zJ~$n;1(pMO+bg!sQ8JPOSJGvaL! z`47cc1M~kg;Wr@i??Dtpn7Ucq3-SNABHQ0)BikSU7Q5bq`2u4%VMc)XW~>4DKiuQ% z{fAe-lEeS|7XQ!hu^#eyF;_mXnRz`Q3yg!|@Qx&Q>4+L76K6gaNa)(&rMWortLy>q!$~4_qFp<*e|}|{8P(Q4nhzngGE?%HE z4MSF%x?`h9#tu|Rb#?f3(`dTVW;ZV%USHmu(f`BZA1|_;$_W6V5r!`K$4##T4VoK- z1n`q>c5@7HhX_cPVrUZdk|g!+ugJ;JCM{t4>#nXs`mn=94+KWGVG)(p5wc=`2gK72 z9A+-2f0x+JPseVgr6c?I-z#r)w660jp{g(l81GrF?me*5rKG+cXGqNVKpDHWOrP;GB-D-!30lTP zR60>wQQ0IJv$6`qpM{f=X_=lRA*1EsNb_Bgy2i=vnh{c9p6x)Ujs-?*q`(}d=>#&S zX|h*nn4u-m+!rJOPvrtQLh8$s+Gl_s>Wk)LApy9M#5oh?(4o8x$<$zy2frp~gaA+9 zAqbBF|Nk_40p=dS>&#Q)d*Hgr`2Y8i`A>b4@&Dc8gTVjai=2PrJNW7Nd(b;D_6*h_ zJ|b>Fb-Tr5aD~_MFYi2;;s1Tk|5qd(RDujH8qV)cH0ycTqVIX2`_~Q}Szlg-^vv%& z6HqdAOOn~}?`ba6JOrc;xX`l$Mo5QhQfNw&#W)8TfgYr!dzR1Aq)>^5n(6@OAZ4+T zK)l!!#1Yb?DO2T`fi!0Y3B+@;C7n?P9lBJu^Z-T+IM6t{g~s;-New|g13{Hx)>SP9 zD!2R)oC#qVD5mZzX3AHSbfvWe9Z{%ZrK$-zjqJR{28{MNp;6UTPxXe0mMlR+{z@F9 zYD0dwyu?XHaxZK-UaU47x;M5oFANFWtKHbr^;V9XV{7Ou*21H&hGK^h|AMSj8CKo< zR%kc%&^EIJnC3(wp_e4#MUvT}uz*Y_9-iJ^OUsEro~P?Z*p8kSu%$+A=+WE@ zB=in-qemweXtxZ+HQKIxAkZd**eRfK6WKFeU9%#Vy;axkFUH9K++}vXUAO@J|DTb` zPd_T|fh)@nm}1!hlPo)6;@`!e124dx!f%2XU>tM)#ZR%!|J~x-qBfndZ{3^W|1XA+ zkO0PZ=L>fQ%z07Q|C7YS=@yU3^cYryI_o&i8q{b4W?k<&dKZhJ-ff3TZvtBd^5Hlv zhZIJpY}E7G|TTm`3CBKOS35Hp0Dq8 zR=_=szZasHwkA2qvoyD9%B}&n3tQ_AKiBsh)v-5u*2KxuB@p2VP)&DLh^wO$pywBe z8UDJpZGMP3QV*>_@~-_~{hL&z^o1^@?Y^N&_;HGX# ziofT%p5a)hD}gElfatC5*+XK6dlmvHn}(uEE~Gs4u5z=}KZ>1Rszlo9PG|+539lV z2|66p(|QkBF~h|Q+w6F%1#L!p*He9Ows|M&fKnyWHv5|4!WVFrjF^e8Zf%=y$|4~pNz9Iu`|B;L>-n<@m*)*+rH`Iz(6v&zj*m$1`Il}I~X_KiT3eD2^9ZX~r? zb$r+sHQ*|{%{DJ!n@g2Q+w7_`EQ1L6BO!<04)oFX!@|cQ=Knst&b{<9y#`+~Pu+Wy8*U_5{7gZO0fo9ucUUMIebe?PH|_u~z`A6v(J@%Q8W|2IcJ?Z-a$n$KnU zf3*JphOQqD7IogMgZ~#wXQ_URD%&K<#S(ibtljk9ngTC0$STT<@f&O(suDLc+ zmc|zY2+6!2*aoGB_c1n9%9KclifROkB{82MM}N?UcbOI>ogSqeWjh_$`XOn$2JAY` zG$iOhzDUp<5u`E2mh&B1U51Fy#?m5WRl=rDafy;Z^FpW; z;XIG=o&>gygNtj6kd(fv*<=L!rjjs{+(RWX1P#Y<0YR?CA<%pxLFV%^Dd<=DZsS>G9_X92UBmsP|GO(oQ!#9Tv`M{7$F2VM?_i zR_~L$@;ZH#)Vt|RFf28Gr{~e%=j_@1zaWR7myyHIKFxdqyYYVd8N8p~hxd(1yq{Xb z`^jB+Kk)_33m89-`2u4KdJ4n`@KebDhv)dR#(y@*@c*5-)Bvp9&X?HvAbL;b|FW%t zPSwkeg`3$}C{rRG3!!ZLl8igEai-Nw%6uW^Cbe148w;Z}>q&|Rc8Sg(3$*CI9C%F^F@Njf=jg^(GnN)M8X;_Es?APidLtP zw1nX}fhF5~k)Uy89T*t+dCQ5!<4h#Vlt@Lwkqodw9ll7=I5MftmvTe`k`u3$i31!K zo+Y!C{N58kjX~>y3j`N8lq{hoE;ErRQz8`!#Z-WzyL^$LF=$eoujYxwwUf+*X`Yts z^0fpl&B4&Y2`_u4L}E%nktkIn6$vL$!6(2^&B*7(Tl;kTFCt@~e>t-Exzl(*djsCj z9K-u*5$_xO@qY3fct80Hyr1|7yoa6t;ePC2(HAhL;Jt7a@8SHvPeiSJ!>iY&4F8YT z|6kwr8@7o5m+FLg?ws0{H!7|jb1DpF%aRMyvuLgg0~}efUqHE0aWfkgWlE%@Le*h4 zp^O>)JWWi>4j{Ez&J&3tGpDla?GU@tq{}i4!;*czqNF)8OwEGWA1vJC_ETX_IG%+q;TUpYDQr8O-u*{&b5K$N zrk=pH`;cI`0UIN3;Q1!iGUcuV`>8Le^Z1AX2kM)PA3@$1TzQ}ha} z2L}Nl%b&4zIO9!G_;X4Z(X1czMqvJx$lhn45f^~}vz`2=zlyBB@lIs>Q(waS$v5Er zw$I@GgopRzAHn;vgLp4Kg7@kEe{X*H#CnGR&kJ+#`E^}w*dqQGR>=6qA3l(IOm)n_Q&jGPXo2QaAi|b2k5w4Yg8(qf1NB^0QwowcX$}q& zfR}RYRK3C~$3ZAW2B2%2j-{wu>B69-=QLp82~&rcs!bzkfe&Qd$~LU6g}oBE>W=0_ zA)&Tw6SWH34o_+s2x=N^Q{pMOhZSd%p!r6KsO4z8*g;K6fF1;ZY_M9o<7-Mmwld98 zLjq9D({?c{rvNyJamxVIV6S0I(fHbq=8B=@r8U^1_&rDV-qnIlv-ctUu137kQ8SPs z>?png2iNfxAI&dB2Qa%tO@UPv%#5Z;;2;2G5-8~k5D*=A#oQ{-4$B+#gXgF z>&pjLH`w{lqqRuVgcSiwK@xcvMSiXx%zq2L0COK_zJSwsol)@GV6lBT1;79=z%Sr+ z+Y&yX7{}|_=S36v{~7Tpv~#x@?@YMkZI`qA|M!{y)8(jhAjTQCBQX#?+Xok-&Yucs z5q?VYVPlbP<9A?feFSF##e|zgcl1x5DO#5qnHqGtQ^?8v48PkCHL|mpk!`PSN9bO z&Yj?nYnoqx1m!6{logx>gP>{*NF-jFfA73#7G&_a1+5 zJlhK-#pWw`nh$`4-^CpKD%iIJqY9?#I-X+l?OU1)fQVnt03N1|VT$Zq*U&;uf%C$6 z1cAo;NdR8Rv2Q`AkS&Zsj9_Np!V)J)LM>PyOym8Ol>m(=@cdnt1vQdJhOWT@778Bx z!Xc22K?b1#nf$qi=Ua-ycY@LwK4sZK;}Sc!G@60Y@3gwOf7E=RvTdp$ zgM34m#_@>|=E&eE`94ig_CN;tzE2v*r>t{mTw!OBx7L)f7^h5(tsVEPp4oZTJFy4s~DH z_3&4^eFAg%=l&G={Or$*w*&v*i}#Q+wio#Sr}2LB96sN6JKm3f53ggN!c_yn|KXbQ z|J`SvelWxT`@H_Ylo-T3@%NXbb+CE&`2nmuf-MW|qFzk@R5=6K5>!bWc{mZ0C16jdaU! z(2k`PYzvS8$j)>rRU++l*gzJ{WS_5YX*{=73z9Z3=I~tDwt$WFVOgDv9D2g_PFC3^ z3$~2c6rV2=G#2bwJ_OVYwgpJwJ%}@KsS>G3__7tk7w|=bMm5>~WSY=6bq%DD zuL)@$5H(a_@6v+pg%WSVljs2{RU-9(SY~KyYQgzmG!ICr79?%H(mnmD(lB*8LdA1` z`!)pHRD~ZUqWL%=4Z_#dg5$Xo0H4NAFI6J#bi=n?PgVINofhpb)q=RqGh$w+S8yxz zZKM}p zjsO3;`nDg=@c%xq|0ngagY?_OSM?Q3lg?Zq3#w~IFXmK{B;5M=!? zJbwDfiQ}iv2KO94^SY(8C+|D%IcI{^v-jP1TD+Bk3xwkM2|sw_`kPJ$P+;4h^|e+D zZYGbPUOeq-?5|MjqxWjZd!^&Oy!jq|pHXRSM^rkE9O9=@UCG%yQBAlX9)1?pgBy0G zK2-FG75Y%|%WyQCnh$P3i|+=cc^=Hr@hsiv?}wTU&y%bXWlE$f#j#S33(pDU(^?tiWT$hZB2(n3BBO-2O8DaB-`}#prB78H8t}vc6zB2X{W0W zYyzzZTbW^Bss)LlUdZWmYsgLycovxlx-aRHUU2#YjdzpA)Wvi!(oN01Ff8HwMB7Gs z8C7SYe#Nd}s!h=yO%D}r{Ey}vAmRE1?nS!1bQGggGFd|nBDlg}a{}0=UJvDlC7)Vb+)VkmLGhu))-&LS`G^mqT+! zkYIZvZ*HkT>jyDhnb+3$JqffN-)l?ri;w`kk_TV~59lBykO8Q}se+avZDr~`3BXHv z09IJPH3-cYMNmD@mW+aPX(IFU`{p|TATs&6+weN`UA#_z7WejRybG^WNAU_O1NQlw zxWM-R|2p~t#y-sW|9|5>@c%!K;JWUu55D`k+jJv=lH zPK2*3XsAPb$kt-t_$+7sW`iHr4l9x&%UmOy<`6JoZ)49;_@4fl8_qH#qEv~L&%z#O za7?zspE7A2yi^O4Ht)*O__j@_2z~2xteEG-@KsaM43*EUXrXozfVUukn6HrIa#UlN zvGgLU5Hc{B)STt(y7Za&moB}$KDP}oa(WrDh^2;&>faOLV|=jv<=rY z3gW6XM+B)`da0VVIK?}g+Yb~0$fe2pLOkzkQmDK6!mImuBxHAdO z@1Ri$1V*(i z%`*z}ipYM8?XP0||5am9a^^dX|9_HQr&#>|vv@!GPFxo-@iFnsu>SuPJ{kKH{8W4k z-UI(%h`#pG7f%05hX40@{l6Rfp=H>7hgdQLokJlkRU(}=9oy323-}QR8tE?8f~3vo zyZM))R`B`aFX|Y^sU}Dh3Cd`%tbV~-^0X8@XELi8rY;#ae_bVwbdvzQAnXw4;n2rt zVW$tmTgfKO(APD|aQF@j8u=$-cTu=exSs8HmNQL$KakfXq8rU~*@-uq`lG>vGLjt_ z9P$dWUf(hbb`hibBuM>wAy27{k{jQXLRgbnu!k6p{abEK49A}bJB~U#^ z9t+tF4fq27V5X7oQY}c@oD;h-)KTJJp%%K9QLyh6jdYU$ydu0vcmdOfLqQ!9vJg>+ zZcUXpDQ7iUmO3C6>DFLFLLCfGwxRF&u@9Q_g4Fj{+2QkGMc}53)u}gP+!rehC!4$u zPn3^h!@l;0$KPLo=DQ%`*X{AsDr{{(2mqtNaTtIAdVHcm^IbR=Y~y2u1xryQUImA! z!;Dj^L^|Uba7wdfg#0$s_;#rl#52xZPV5N0KXks0zo;*x1}p9{YFMY6aCRy55I7`1fogM~?Y?%cqB9TxD+rNPve*Slm!_WN>-p~F4-p{-d z`F!Io;&&ka|4Mu^`Ahiewma~C{Dk;-!2cDt^FP}`K-evOBC7Idmp=Qy9sD0e7vk=n zFLXBKye1}v2D7cC=+%@YUun9k6*e``2hWitIg+Evu#K@)8k_`9ImhhPWB<>vAXykKax#{5j150Qp#kttd%+UIC7#`ndcTU%qGis&7G! z>~+&+6oyhIQelu~BZS12!~)O%f`z{zc`ZscA!+p0tW6>7hA!K~ly+ZaQ@F6j5+Ink zym}z`IU0hJB14_Y5Y4YMM(_o(CNEJU?RMXDJdmqouc`bmSoq8SHM+xj8g=_HIZruL zC)50@Dfu#Fe8rl-2ZEm`A*cnuBg0Nc+&YtPzZpk+sS<77?ivPsK@z0x(dfS-HQJLU z*H$*oulzcbMn_)FG9}VRJFqbtd;wnvF8npA(T)|kzOp%Q^BcW@NS7+n)@a>?4La0< z0rrLdHTqDVUXA17!{V~cMraYdLZd(N2AiQ5f+QCVzbE^LuY>O3J6AT=kFG8+ZKS(3 zJQ}ChvMkD;>9r9(8id)(W(z#x3XWajE1HE%B=j`juzW@1&x45`h`b+VN~A-|vf-2~ zMN4>I{+M0(8&acT-cfaBb6G-uok^q5p&lqzB5iaihdQ9bpUD@#*uO@f@74nf*p9=q ztP!!|MJYp1g^{i$POtnfTKHQMf|{l&(Cx{x!~8mvZhwqTv}H=9-5y9jOtd~f#p~3Ea7Vz&A4gWd?bqj z!+YVIcn|Ua-;3JQ_~5-SVf;UOn_MT8o6UcF@c+%v>zf*Z8i5*t8i5*t8i5*t8i5*t z8i5*t8i5*t8iAoj;I$vQ@#{7JAKLDz8&)GwBTyqyBTyqyBTyqyBTyqyBTyqyBTyr- zB?#2~e@kGjE2$Bv5vUQU5vUQU5vUQU5vUQU5vUQU5vUOuY6NQjKh)h*H?BsYMxaKZ zMxaKZMxaKZMxaKZMxaKZMxaJuOAx5}|CYd5S5hNTBTyqyBTyqyBTyqyBTyqyBTyqy NBTyqS)CeH||9?%+8>|2T literal 0 HcmV?d00001 diff --git a/flume-ng-sinks/flume-hive-sink/metastore_db/seg0/c200.dat b/flume-ng-sinks/flume-hive-sink/metastore_db/seg0/c200.dat new file mode 100644 index 0000000000000000000000000000000000000000..7050478db1831b0e708f6220b9b33740c08b2ef6 GIT binary patch literal 16384 zcmeI2JBXb{6vw~2uWXV{OrnVzT@xQD2n+LmK(O-xg3+`QWu9VX;iD65)XKub%G%o6 zN>H>?5CYmpun-F?3p+jE%-;Pro838lur@Lq^3T2BJ$LfIzd3WhnR9n|Zy1IJ{2hNT z7u_hjUzV@$zXp&0+5I_i`N*d)!z=I9SI;ejw%|YCTs}GC-)9fI{@m@K9U|vajKY*5;!n!ns0)^ zb?{s;uCch8Bu>|yAjBHRBR06#+Cg0V6N1@fxeHXar)?s!Ri`8IOqf7P80n&50wocm zGlq(`*~!G2O2Sfam_SJgVhqH!2L}>?hhQjc$4w^A785D@V3?HHbP&lj#0e~J=5*Xs zOlYPe433frHY$i~f5Q2z<6JS3HA{vml!QrELfmnaIA2NloMCX31XNK0aVQC%c`}r? z@2~;0DbJ z$2gcZXim&rq8BTPoHz`Ql295d1v4m#nbUEpnn+m^X3?DRN@T(8ap%PS#e{VnmYzjP zWNKh3Ta?7ioOqy^pjkr^a%fI?6`-g&G$&@>${#EyoCr{k9ZEt63gz9QBxdHs;k`PqGjn1PaX5K{~Dkfr7u=FxYf(vlVGD<>6?FfssS^TDo`EW5Ibc%r!bOWn0 zVBiGZ2F}ch?K&OYC&FoaV+xhfxa`0Lx*?vK6OR-V$tDRV&^kdW&4N?3PRyK+E5(E- zaGc-_C1JG`U;-sE)5O(cB11VQil8Q93`EF`nwXgrj}{X_1!fdMm#B<_LL9n8XPS7d zn21Vvq42CXAy^=W9)4yf=5$W|FCLxE!-vCg=GUUHj=J?r(Ho)Rn)BzCG&V&!TURI{a4j z^`h67C;S&*yY<-}=l}D=7R28DwfXwtyx{y_rqwZ}grcljDWTrL&6p8I+Z7e!lufy7J`wZ&kZjYJ{M z5g0s>%pne~&@-2MZ82nPodMR^beNnh#2t6NsVxSkZ5A58w>ap^1WlnI)oxGC>~qu> zgNKp|CeWPFfdnvt=0vazieqk1+q61ri@{N7OlZ7`;tcUY^Y+pkm-4LM*P^x;OHw-++Bw3eKbTBp3~MG~X_2lZo15FtGM5 z>`l;wQ$kXLZ=!tSu;!5jo=nsh12IlXp3ubf!rRCbniIwaNqK7Dwf9UMeoN_BJAqE1 z6X*mwfli_+J>e?Sg)2OSri{2b{<>#WWjJo)t=<}lv-xR$zd{hUG z=KJsNIRD%7n*ev8|ChfXaKOvj`Fj2*mTSw{vh&KIqsU`|9d`Y8A2yHCjNk5aO(LDX z(>^ke#B3#F%PsohaeX6sU^e>GPM{O$1Ui9EpcCi>I)P506X*mwfllBK0(}26oyaF*pg;^8?Sf07w?^AmfNtczBiDvAw{$SnWSz7#IOY!0c=L*ya@(#Y@<07&@6F~XQ(To23n^;*dP zx)}c`_pW{!X1)oZ0${d;NZ5H;|JU*PRkJ^S_uEldU(>?E(?7z3(6yahu|4jy_~D9~ zp)@!!I50RcI50RcI50RcI50RcI50RcI50Ty|8n5xU%w~tZ8)ao@B8Nw)zMNqzIur70=|H+E@t8)Ha7Y_Eb|e};feCj z3tF`cT8et1%UUfO$<>0!;hKt9mclsDP#|x3U5Sz=`2r@$J7=|EtV-!mV2ZqyL7j1v z;-|3H^-rlajJp)S1>0Rep0|{Vr2aPSbo#ce4HHTI9oR+QySgz<%2YqQu!p>6$^|~D zzXvnqTg|*-Dyct%{Z7Ab+rTIF_u*Q{SGA=~C;hnw2gn=gO2c$g{{XHdPbfeN0$d|2FhE>V_Z^3QkE79n{C-rZ` z9pqWXjA3@0<$$w@LjYxQo2d-Zh-osq^zL-0S+`wWqwKDgGYZN8Vd61E2K& zK0H9are#ApOX@#>hsd{DP^xH>e+Z9|Z&bsKqDem=!DHl|617k?dA)iJ?;u|>rGzS3 z!kcL}DSt*&cn^7| zZ7X!CQ~F!*KJvA6BJj!nx8VcitI}0Q^CqRgLq0@aY7z8R(*IrZ5%Sy^E~+}&{~lQ& zZloTI6JX5?e!si~CYK6$+RAzlR>%-|CN3EPMxe52*aYiWewzNVp ze|fw;d`?Q_8|jshYyI@_1z}x3;r1vhlhmIOj(oV3h{mzY^SMWVN(AzbmD+eGR(|&I zTSOwSwJYuV{iTQBCJOoRxWXFB{DnRXPb%E`@O8)G8tb@2KX-^mo(g4*SY2;>`CVd= zm+`OHto-cd_lQN_GNpx=%k!<5pTQ?xKbC9D*z$Vb%P$}D;v?NK1_uTQ1_%DL1KA0L z|M70Vh;f9Gyovb+Msgf;l3}Hdar1SIlS9m3#aLj>k=dN%?8_K8y2^RXa~0$GB*rP$ zix*gVjuH6`V}2Sd*txLKN0f uVdFUFImY5V<{LPhond|&;~4YtVa%sEo4<+q=Lt!GkN^oU!<}SFC!i%ILzRL~x|0fO$UsK{bhI?|H1yO| ze1v=ihEG5XcdIyfNF4cE=Fa2QYWJtr>hH79mf;Wph<=(u(Rygd%_u(})Bf+{@lSZ- z?o*w4p`SiLe*_VIa=m}D%W!0YN;2JgT> zxe;)tcXIIPcRqpp}M9{YCe)u@UW4egL$id=3xQ(Y4urvzE%A`EFvFOQ7JfY?0*rKtm-ep zvQ_n zv~hi*^W<*bk4O`i7ftUMx zs+0QZAk9j>yqZxYs^pQ->u<{=tnNRqln9k-;=6<+A1fBKc&wiZ^?ec|uZB@5^7QX| zF@X5(DwtWc%YkEZ@-NQ`_cc@c+c+QiS2r0z$^h_P@q-$GUmB30_c z%}_E^kwQJ^NQyjHaXB33BmF1x^F$zD7D>#+iJlYr1(MbEWtp=w9?iFjd>5#(29jet=mYl1LNFSkNn44cRA44Y>Ez+QpRe{i*W967MGlo%}T zOAHP#?mIZRcwpbs{-Fc=67;jYR2V9ZCRjJ?JX>6!pU79J$OunMEF;fyV&9qNg9rBw z6_-b-NCE#A2S*bLfbQg|Cu55(Z^)&m(&PE~cq*5V_sz^tO~q5W_=!|@{MJ;qZ!kHs zXAix|pPoxEjHmMblrbGYd24zm-j^G{H9ei0NlmBYnOvO8?uq9qrCMd_DU{brre`MH z2Kej#cxozxCRM9ce}DY8)QQyiR4SL_%^{6HEQrJ;dPn&uQu)*Zfy%+G)}Jf@&mPbD z^W~;fQ&X9lJn!=uddh2yPiAMQ^==XQkuNlwU%==S1FnwX}9>Av{fz_DyLN}0#ATK5)Hg?S1^C;b_DxVF9whAKz;s7PB%~qVK2OvCP5KvUiBvc zic1oL{n0@&YYBi-r4|hJt zi`oi~mqpDaW8&!p9oqPUU;;iI-Y#>1R$zb4}qicP=ZNN z3O3#yn%gGE z#usw)W4F(zv!?|kLb5zDY=ZJ|!9FY(CrMsDNuk1W+T@*hAO3;QmpTKp=*;ZQ!u(7I za0$J^V4Q0Q5^+)r?BOzlA(4Cep%5@Sm6@4HpIW$m9`fyE27)QcJLaqK5(;dQSrA>4 zO5%NH{Upv(r3N_k(qAp);>=8HYGH0Rm&s>lXQ+_>oxjZZ?9}}94A*5)cJ0xhGnE@( zn9VLsq;unxQ+xF198U>AGWpZIPVdp5b1roZvK(sTU;L?)nW=m_yD&aGGvW3p@QzE9 zeQ`P61pb~oF`C3iIh_F3v8mbV+yG%Do0)@DJ+M-lUt4+8MzK#Sqb5_NapDszpT$hW z+~RDzKAEb59GFOMtMW@a3lS`ACb>$LiNbneu}~>a7uO5){m>yiuNL|0o?SmzEEE34 zbP}MrO=ukhIXGUtI9^)aIKMVms;sQ9l-Bw%S#kqn#>~u5kCBQ7RTP!L5MkhJ7z2m9b_(cT^1$_54f(&FwIQV^0DBGje}ir z7;2Wjr~NKNh^!(&EsEB2#YzQoP(bL(jn8s308Km!m2|vxG9Dl1xp3fg?m=cZ=655vb=syR3ZhlT;`5|RfCdr z#^xt+3b!zoo;jAkRY*hNxILy{7ZP=7jb75he%!^kv;vq&k7r;K(H}^FteMVc#)Apc zP#~C~MABeNJ~ML~#exZAnPZ{v(Ffqd4{JU{b|PVKbNSTt9Q5tNHYI$-=gP&U6`D!F zIu{y9ZuAUd6!b`x>I3f$%@2X6g$Y2McdbfXSc2leoV_!p^|?xMW4RQVzjL|t{KPEu zE~FYV3{b>rA!K}RgAkNWQ)Avch=aL&HU%u^;!CBqiYu5NPoK`k?<`*Q-ZUY}q9s>y zJwo++k0kU#SoU6!up1)F%1(Gf>=98+yq5wmui5JgKQ$EsA$0@V57kL6C9F&^cPN-r zrNm`@34tm}$|kPu;Vns&aR~GjF7bNH&&}rPa-KVxn?!Ej{w_3Ng~E*A)8JmQg)6g*BZV_di6rdNs+gx;&8`ALohz4?ipv}2Vv`#hOTVeT zWNZye-4h_PbJ^MP^u&Bt8S5CPI$k25VX!qD5GGMr#9o@v1D_bUUJ_q=zMht-_S_%$ z(uuC?=|Wk6biy__A>*m;6EEv-20YDu;%ReP5R@F$$2`q_;%S1+^8}6JE1NvCQ6lOt z6?);)baXm5o|?iK)FR!S&QeysO8CdqQGy6!OEts(V6J`C%M<%6hr5vbNsvndjK{mL9c9fGIFI zgZ_AC;*@?)w6<}6u~?oxGh4<`7nkM6B$#fnw?Jvmm^yNzJhyxoV3c#m$2|qOD+!J_ zJL=_}6fYwgL&@Qo$T9d6ns%y?&gLB97?s?O;)*CU)s=w&rr0vhknj$yLq4LlFfZ0Z z)nfr^ULJBf0O1O)^m!PfX?z4G&hC_#-uCjAPr~wkOUG_V^?%XPkH3f6`-FtQ z^@|UEGs*ujQWifjp~sg_`Q&}5bngxS2K)aOHv7I8-5{r@apG_pkL1mEHL?-lZiM_} z=64yGfVkqO_|K1BzJ`jp5B^*d5E61VK!nU|f`FV0)2UMnCo>cITNg4j3uCAA;`lmL z3ih~g-+{ghFt=#;@07GBx&ot35Ts*e6F5^0REL>$NQ##LW^kLYR}kj?N}5B-`6she zSW64UfiCdfJG4_~7NvpkS0i?Ovpq}-Z}&F~$jzVfyL^C|L(ZCs^rSXv@T35EKCn6} zSWpvCZ?ehrZDUI0Zl79kb2I?^kpP3or>$X1qica|Fa_(O86?#uNztF?>k-Nq0n|`! z{+ugVa3os=QE38??3oN6-qyP?5s2(R{c3Vui!(x zyy7`LFITB}S@~li=j?oTJnbzAp56D(Uj`jY&F5z`GvlBX=^1fj$@ea#%(3(gXImx& zA)%J&l8o=|@Et0Jej{}=~!%A%~Ppd^9hJ>K@IAo-Jx_!0ah4G)K-|yo)BA<-vB4R+l zuM`(yu+$t7SE=8PA3t)im0zif{ScJrmI_?IS)JyZY+HYv@FIUMb@Gde$t&|?dZL>0 zMoqSm?YFKm53mN8V->m+%imS&q#}kTxUbg9zz)Jrn0RnOg3#rksD2tR|W zh8LgS2nP^rsL1)@k=wE0iOv$Y%Tzjb0a@+rB`Q!++>Y9$6%%N&p>^3|4|Y1@!$A?4 z-_-n6o{R6^b!>joQjrW}qr{FGU#aLs}zekhRcqf-Zyw~ zct3J4%g6o6gTuINN-wAZVz5NPGaGA5WJ4RjyR_E3j=$wQ2jVz6Z}+_V9p3RXLXiLN zCk9?&$p6Crzi#=T;WU2oZ70ix3k1Y@pdns}i}TI63kSyoDAsCxy%ayQau0{sMN@yg zcvo=^?@AkI&+#OMSkaxrDzLSDFP4^K}`b6eFWAwni zXY+e7^cdY<8oYAxJdXRtaMvh{$weh;pOaHZtWO41Bv*}@#Ns}!Qqji z1H%WE4CcNND=?I!3s%NauVKDUdBdZP3T9AG7?#tZ1O*K-q*_l^A(z_>Qc-auTo4o7 zgrgY}ZZBa7DxxJt=GB~WJye*IN?nn zJilvPA9jqjs&);%J$E#;jX}DR;7U|q`oMZAw_Ya0{yhVnu2)Wi^!CMjU7-lhIZmcv zJCMSKZ!(?c+oCq0{KjzA5p}B)D;!XVFmC#6&a91HTrXB!bC@vwpEqfB&jwOQPCg6d znF=38-K~KD0^~-j-Ny}cd+5|%3kT9UD`hl4{@3jAxjwq8w+>ZOJNQv*Y|%n6LE*(n zu^OxMo-Obm_seTtRIop=W__CJ=rO`TgnP!L5Me;Wz~PSX6i=T&_uhLA z`~SxM|8et}LSl3nCX+)|&0}`4n6m%pn)N2G|3i(@+1v$y<&n|lkwkL1Dug$%S1Jet zw*XSBtJ0RHVf&rQiye1|8c1;Ajs#Ht#eb9|RRx?+_B02A`xPRXd4xuysc`H^HR>+^ z+!TKk?s*6S;Xe3tNkBH)j6rGcY@G;GQjlU*sUd74D;F_+l^S;JN-gQvkDFyoOLd^f zd=Ykh=ITv?YSidj2l-m$>X1-_7DKoXzsf+vW1PqZ4g(rj)L)k4gG)IKz|vEG5M0d@ zw>$qDzSaTNM>y1tYCl4C9@VagD0PS+Skga%(tr?5y3YY=SjZI&P<(K`5gwFAgX==1 zrp2>H6efB$m54|+3mIw-k%70u+oeRZ@FPXbE%C8*{$v_{X2n&9r1UYsZ#=-Q<_r{E zVTjcgk%3$PjhPBcH>Jag!U61+`>XDixXq_ufT#>78r~Pj-q!L~W!K7&^eqQNo?$Qd z=p*o}wnZ;dDv7KYAIuFyk)v!HjxJB6CkB#{m-3-LRWy-nIvba=dXo-iC2M#8&$6?3O55!>bP-T0$pzfi}l(NOj#_Bp}Ds&dW zLlbTDP#n%-Pn^t*D>SNS;jTwtdD6Dr)iuiLEtgewTi3cQ#l`e**qJb2G-9wDHVrp= z+u$0}_~__sgn9WAg;XAfD`WF{>Df=L6vdHxIvp}T zk+CV{^?&fshLs2zi{Tub4_M8%_-S`7FO@+UL+6Rv7?uxu#qU}0tE`3L z(S&pb0%@m1+Ogo|t<&t#95=GSkUmTo5EdcKB5`wOqraiLrlPNqpRP^K;wm)Pgu2d0 zeXsl_rZY2{XtN_}Evsn6^cyp%>;@Ys!AFec(-qtmi0kSIU$q(`|GP>-es4CI?ccY*e{i&4K&k+s*}bRkck#h~?x(pwDL=&%Ic)Av;NM_kPk&$U z6dcp4DIA}Ks&_1t=f1fB!f^jk@?av--+QXJx3?d^y`hb}4JG<{<)R(&*X)2~ul;>+ zU;8ZO)q9F-)GL=+95#zSFoysp^p-l9y{pBwv+L)2`(4rR(c3>Lkni?YpAx)- zLz~vY5E!t^>CRU#7T?fd=&CVShFIFY7nIAi{t8{QGUl(ge8txURV`MrJbmo#$6B}y z&B=w?+dqu<>+W4xkG1uwlMNq{QpCKse?R_()l=`g`r6}{6MOp)^zTpB-I_OcA)}`E z%>-4^IA1&N>m(6r(gF1LAM8&K`Lb=wjv#<*0xWUeuD5@r5j>gghz?jrrD?$AUI8M2 zrs^Jk+)ChwQB$ zgF%uI>P%vm^z!eUQzaL!Q14KGa#U0--ZWqPsHCMLLMJFUBB1(4OySD+y6IK;oDJmI z)a)4V!EnEK^{H;C^{rB@K|dnowB92*OLIoKi57Ow2Zsfz>NU;Fan+sudUvp-C0D8g z%yH^pYA#4&B*jhKBzVEBo|*=|#suh zgBQda2g5YnpWok~KhU2)*qzc@ff!*R!a#(92m^Iv;BfbsMBdE4@B6G_|KDW%zoC((A^86v zscJF!QuY!Uz6m3GFS7ZDxFIF^c2f64r5&JnPxft4A_|SNc;N9^ zXW=P}f1xTTv`??~NXwOH71d3ctAPl-5 zMNuGg^Dyf&Cxb`8bclP#Hmh3jE4{r4mF4?v)VX)vt0T%7VIaalgnP!L z5Mdy~K!kx84h9alJR~~vmGAk3dkp!1E9*iEMG7!~EdKvHI@s0@)=uJkBC)g#6~Jg! z6~H#O6)J!xZ~r??#PS;+Sw4tZ9EGYaUBg~XEm0t-OVu@`o^ih1fUe=|QN18_2wY2` zs}SnZ&D4*tOHdkUK{TwY3>WYQF9ddR7Z}MzaS$#r7OT$67~76nncKKfEc2<^cr3}I zzr}t-yEE%LKXff1Ngu^ejwlY1P>Ydrs#Zp++ELZuR4+~zn_f*qcNUhOo1aT(1y=cO zQgwMWGd4YdTb&d0)6=I1<#kc0d*UZE`CH@j*#@yJ;_xgb&K$(mNWIJX0f}hoox@Nf>PD+Lqw~UF!d)%DlGGpfIXdvWn+WwMp;V2^+NN)?x^mII zwH&H?L@NFpb!iWB(Vj?WDI}Zd9n^l8qqmo`k{g^iJ;FiFXd;Q#S*$v5cC%f|{fise zukjT+HrFzvtKI0DzCW_#a&QzW8aLEKpDPpxYO(?(Md|sRH zwKf_)*`J#oNo`mLq2^6-V|F#?HPRhSlx4NJ1^y-Dr{hLnR2!y+IimUTo0eG|9>E?? zY^$=R`CRH4olu4C(KcTYE{;`mWW3@Fui~}{SHy0cn&)A6U2N_)y`4|KGoyorL}DmW zb@KJFEyN5VD&hG;<7D>ehuK5!{(NhCI>mVp&{c!K3q`AU6-Dt#DT)iD_!eSX8gOzU zvhvpbeke3h8eGFyaWpQCjxG-;hN??C*RpGX#(=-F^!W?x7aKlTO?|ubtu9tosBvOx z1pB8g9qk&Av1Xd!dKA+`gB9;1zc-p)lG?dK8JvWhij@#{E{hC_ehP+wiur9B#s9c$ zmHAfkL}R!(mYqc~HbmWlmy~oCx1QX{>(3cF46dlb3B%a7ceV^N)-&xKW32lRTfUjG zj=yO6C&t>|-!j5j>rv+nMEaj@`9b_W%+`c>{~MEUzRR%xZ*MIAALRd+b+jV{;J56! zSK9;h`&tNSSl3k|Z9ZA9h0!fyei2O#x_&t;x)esV3?>klcqz;cHWh~Cx&|`36GrsQ zZceHhk?U5Y(wsE96Xvta1!3obZ)~qE*0$~~r07nVc{NI{YmKet_;nEmZZW#HC2(e6 z#NJJ#25*VU=^S0#;!&*xf<0tL*S4zd9DOIvYUk6v8Y8!`878wHj>B~UE5Nq zu7SDe+E#RJOEP@=Ehw=Dqib7+V}@$H9A`nIYg_Qyl}&T8x$bLQtf%v-mTtEDs-dnw zo@R-CXO<5h+&5HQhCzNJQNX{&!O=v*akz93`F{s#0r2w^|M8Jm8R!2^+W(J^6qj%; zw^UW%CywRD%cV*sTRc-N7uS}G4J)W-KD+%%L(>qV7viTY$A$8ZtDEQKgFNI^k3y$4 zcRZjkww~Ec>b2&Q(l4~qJgQXLOg^cFz=BAql0~y<6cvc3y$V7=8Haa}so7~{o=E4$ zvzfX4Y?dMlLPZ} zbp1FxKb1bxi%4j-f`hJ{zfdY8k|jWa+hIglL;OqbeqRsG>8m!|P_;9qln7?Yt92co z1s&{bpm=3usMI~CMv<@}I=VnT4adYpdMf=KU_!6T;yv*r@m`$phOz~NIMme8 zRIaH;_?#2FyVe~sG5E*~$U(#O95;e<9ydt_C3^^0fh(x6UV}f_1;1ej{eg7htMH{Q zd+twG#ZcuNe@Lb(e20%V@co)Qc-=$Z2Si8<-W7M)_yl$vkze=-AVtn*?)yNJNSz?^ zgWG!8IZ2?rDG$eGY53aZvvM-o)L0Vv(5eZYE^1H8#h03I1UQjfn>S}PN+n4hj>|%5 zU0sw)CxXddL>_n7m!x6wT=SJ1WDvD`mFr{eE#8_fVQJV%u2G{qAV!nk0*$QG+c=P>0L zeb3Pa7wx@UfH#G9#R%hx0|F%YFfi_W!?S6uYgqs7bf~rfMAX@jqfkO5Kn<`iP!|x@gQQdHk+%4L+HBUWIf3D9&HF1O|h6HSXmk|&Kc0o+Nw zj4pF21R|6Hk2C>>uDGU3(G#p9O#oD{ktV>WZi_SlrPYn|YpxU&X#%P#1`h9dt4RJ} z$NXO#@_#F9K}vU^u;u=L8}9$}%m1T8NnBJ()+_&C;@#Zwovk~h{BMS>;+Hh|nVq<; zVWw(KFXho5)fupv-@Wa^Yt*CZx+(t9Q0^@TD&ju)bIF$Lh+JsgrlC5G7yf$9t7(+o zeB;k=UO`57^94UwbL0J7h6^y_7_rZFxTwy3qg&7c=Z82TF(zw+>O;!H`m)d~p2c1KJOi)AeZ@1l>)y-Y zzTz3&n@nB?_Z81Dm7Y14zg4CRK*cLgs8k$Ifa^AV{go7b^HM_Gq0>A{-}SX)5@L9H z))xlSp7)TYzc#DILJNl@K753KyD5fJSQ{b+y;hIabf)>UrTMJfe4w4=X22` zXlyErb2f!To)1H#W7fFvEj>fTQ^6DPew;d=jF-x12MQMoxUD__yROBH151VV!fNU4 zz%e*bPu-Y>5e-;epH$8 zKxNeCr)AgX7~%4IsWjA_Ao+Jt_gUF=%SuoK0 zf4aU24IOU%MbX?Z{qnp2+>rk}SQ}Ed27sv{|L-3iUK~jb4ObN;cCc*_BvR|J6JOj| zFE;M-V|=-Fu`U^cbKCVenjz0!Qif2Qli~S{(gj5{HhY!V6*>t(x zs*7de6=}kP(X6VCx3<|HYC>C5d_XO*bxzIWiNbp~k)HHbr}`BV)Yas#fJvE0i|HB) zcsL8@fu@@D?`1y70zEJ|7Ng}rR<180DVChe)T&ox7Mpy1dAT`~FRRj*aBeF%JD(j- zFSt|BWp_5$;g9smIkc9Rcp!Jsi`W@D{8HbvNNJ(>!!`?6rHP)6ZoB6P==$KnPUsk z1ZaFi6>v~!;aGZx%n~5A8%vAhDCe#$v3T&@r~GR5<`JWj=j!EGY;$`A^B@X zd#f#MOam%b?W&BbQ+8Cpgelgotb@17XQ2Mt128ia@4KZxF8`y8?sDdN7R0O*-{nLQ z4^*dj<2JETJX-HHRtIso2RoSKux@^CRvbJygmbflRadoeWj(&SQe0clteq)A+W`N; z>nlxg&_{op{ie3YUG6HdOuX6b=20xubn4Us^k6>LBTxg_ycBs0<&&^>W^uDspW2(t zrUWFK=tYEquL}%p-Eb~2*7=^+&%zhr&8=@hrw_N$DL;OmKK(skGUWdz&;J)kM+Ra4 zUp@X`oV^^}Nx)HMYIU{QL&b^P+HTdzS-h``u6hv{)U&0#bHz#pcjXNxm_NAg9sM_E zjT1sVsB{vnXp5jbadZ^RqZ%ytaX!Wsx&-0Fb@=ddR7PWoEr1=EFlR?csn zKe@8Jeoizk!S&Pw9#f$xxX*?_@GRQ392>MoFKPGm&IaVsi!?U~9;Uz_ZaSM84<-O6 zatRVlP?AV6C7+o&jbg!svCOg1_ZSU`F7Diruu>rO@V7jQxjOZ+jNDwgxU@nu30T*J zD8(Zx==qrtpAz01jI`e@LkxinVeq6y&fb}lbk-c0zaf?zQgV@O5{^RY1i|AIIWNQA z<>H+ZQN`@x$KtuT6wWvX6w{jK%p1z)+zU~{_Ke<$CIovYTJv62=j%kO1Tl=G6B)dA zi9tDXi{SKbiI1i8C)4Q}dCinO$PyEI%i)_7WH>Q{Ypr3!xKdH)RuGXJ6U$^MlEmQ1 z8PL7zK~G-Ht_F$lpAiiw>SY>PD|2$>x#iaBuHy!|Q?o5VIQ+ilb&R@4X5tHGqnDcx zfO}%)%Ndeiu!%U46Z3|PGWsDdKbOk?8kC$HB{mYvAartgB#(_z}&V&)c>_}n+E|(H)>K2 zVpaU(aXF1dLV8ivQUuzu8pa%;Kj9}AC<$IdLxUPnpM6-6!wA@NCt0(cdc>1MT~}5D@_K|G$b}e8(dvK5EGSZD2MT+Y;zwjr0HHU||ub6V-#C zS>%6&=R030UxW|2h8g12oTEZ5EtS?tFjxLkAcJ&=#q+2t1WAF(s|C(mN^2EYv(A0S zz|TiT3PzqZ!bKhq0+*`r$j_(zGSCsyw`*b&WeNg*gnab*I^*VChsJ=;wp(r;1 zAll$sf?4FL25Evg{2d%!Ea33By7$L@>@~{!LqG*IzqazGjbeseXf>+}n*3~s@(XF$ z#Y6JJO*>7)W2S0!mhxziN^@i8cV&)%0Pd#vFIl+42NiK2{JCVybzBKS&IQuJBRWa` zR=B!Z7&}cauuyqWucH_NTPMOeBjQQb-ftdfc`6L9@$gj&+fAG*FHArQlh3K+W51%q zEMqDyXn)V8=O#0Kx)KdDzo5^4CtW&-h)8fOR$*VJQ18osYx4Iije_z3C> z;mq3DMcCQ7df_m#aS5BGcI!Sfamw8##p0DJMB(7hw^4G?b%kPKZCR-&J%wS|X6dU4 z&AM7hs)L^O`}@#3kHIg~5$P{MTL6^PbEg~hIc27YyS>v!Kr|#Z!P4uUa(XA!kfm~b zt+XCGYPuQ{*Saq4VwVU;yEM2oiXFLnnAEG;RnSX{4cCj2+%OY1@tL+iF|?8v3k}X0 zh1}|*T~T-Er+K{OJ0?#Pat(Ptq@96tGAJjWGy{sh_}qYuCro7ndQmslFl{Do<}~LI z!ZW9FMxfEnxdGb2i2lE#aCsGVKl^cSx?5i^CtJ0+vxKL!FW#HaW{%-B zDJ&OQZBn}Ezo^QM`CR8H`-jMnUq5pEJzMwnjIPI2?Ge996U>30;3ce4`HZF|VyHu~ zFyM-K8vP5BW%mtVby$4v3n?7Mb5g#+%%qyRaHIL_Mcd;8?vR9}Y7!2JxJS|Xp-gql z9{r#~S?x$sMF2xfU!D8(8T3G9r&i`9p~wxaRN##ox0@8@5Ie%Y_@HL)V~`PdMl>i2 z1w`hrtry)-w8Q6L%T1l%54a9HUlq{Z^vWl?4f(&x^Z(@N@(ASr>fx1z{Xe%(HUV239>*uZSJ*X-gWnpH5%fevN^@=WwU?xKrz9O;!=YZL`LT)y-;QWIcHl z57Lb$F~&2x@=`l+l7e=Rq_&2&O~*XFb7IrH2l zFvZ;nd!5--*Y4V0dBnX7DwK)c-Udv5gCV`^J235eW(K*i_~e!9NWL)C3)+~~cPcFs zC2uR0i~OB8H~3CZLrF50oG-F!O|n8735GdsO$WuRkW6(ie9rF$Jc~=PT$z*u>oz05 zDCgp?p|7^{H%Pd@ws!Xjng|0C1|keZ7>F5#psx zr1M_V2-GnDC(amWZ(*fK31Tt+=N98syQkY$>f*4P=v0+z%h2_MuruAuVP$Wza1?H< z{N{I!UlibWmF~H8$RGqQ@-X_-}Uzn`(~kUZaLX@ydFkydLMi zCid-%=g%Q-rQXC|SecP~KVMug>|2H-v-lRM-Qp`};_E0& zFBS`xV*KtxCB9THBHH0{{O*N&lj2FUnY6g(r{ipK9Qt@WA7L279< zqi!{uuobBTYB>*#t;@Bq5AIwcX7$!g;nwtlMd`os7{+pN1N-hct#Wq>@?)Pvhh#?- zeXT`UeS7xacyb^Szd4>996mTaGIU`0pj?_PKi9SttOZ+vp9fI^w3Y0e3`jU*`L=q zsPX$HQknH>U{EhfAjy1&8keKp2zHF>!rF63LpUuk6xLiju4a-*A6PHpNM&X1Y~P*% zPI@aRK^N=-!a^OD)*yOW&LSCOdY2YVu`YFB3Mj0uta-R1@xn~m(lriXlQu*T8{nFlJ zZ~sKS$I9f3;{f-?p~<;lA72k|e_V9*@7Dg`V}|_S(VYGNNHRh4fA#o(!u=Pw@+LPX z^^TLg{_3!aueAN+Wzqm>w8%X}^2`NcR3a?aA=U7OJ((gNm~&?N>^#0nhqrK^voJrC zxqV(3k=eOP9))`0_IX@mn#|B`c(+ykJgrcu8pXY(76!*kB~@7}t}PRZ^>EQR zpo;LOlsn=H?v0pGk;;uP%w`v;PJt0~-$#z|lyG#5K<}0ubE#u#T%M;b*DW2`L?<&- z+~9wE}pDsGwx_w+@RyAjGL8ZG~#*q_} z$O#F!h038rbsADx6b3<~Ms7~34&gPOpFDE%-Vl>wX#EtYvT2)8_(?`JH^X&%*oxE> z0WHcEubzl**gfyo1zkVd{%<7zcNGLE&wk2%lKnA zwTv43u*EjhbB8J7e{qVW4$r+rsj?9foj@(U6mhg}99?Pylym3xJ3P1h956xptqLi$&pJrJKO*+0}E z^$~n$F?nDaN2k@}^K4^V!FP^}aKMev>Z7+Y6~saqIbqTYeD^_}I5iL7elS8NBQlH; z(kPsuazlYfHw4q*kwl|T?4Ngf6af%nAi_X|fd~T;1|keZ7>FkXpkGX>hcf3$pDl8Y(JxTX2hOg53hAM@lRk}%iiDSkG zE`0Pvf92=_RJBNzGdgdLswBQT>zzL96U8&da+z+;1SSsd*QT@C)L3Q;w?Tw<#^MXn z@FCeJiPdCs;7;7DG2G_m;^;zVq8}gmF>%c;MeQoe;1P;3-W@$n^?WLqls6Mq-~E(C z-d#v)?)btz?QtUF*2dlBa*dNmI4IGNYIhXSLvK0?P&KlGPKKmw1-MaMBN)vmT>}7t zvoVfK;6tbdJ$$&x?T@QMMk0UU?fX4`Cc|7&a+jav032FKzIB9owiGZxr5akr8%rx| z)m(D&x@8rOd73nOFRogOdoB`N)K_`Y#l;lA;{(iy2Jys)ohD_xk55hC9F-H6s=}52 zuoJ!JJMB#{&k?ETFLZJ?o6a1YA){Q(r?A1P3$t1^9(MJfxYoX8%`cAKnr}_kP?uNI z=30@?PeUD((f|&~QN*Mj*H#Ub)ctjUDM}i5h%I){I?d6K0?b=g zdaed26K4$kUO+{E+~E^FQX@zm)05;hrxO;;b47{uNCfD(7JR@*SzWs5kwBf1qE(`8 z(3X|b^c?mEDUAFGx)koRxUMCl6Sicp<_g#)Y)wOM&^4oOzOIiqT`?7}dp*}`1#4vn z_fXOkQ?s-1p|2BteHw)~$X6oYPir-E#NJ5ga`t@oVlBHHASdzU!a(iLeBw_Yr>Tv_ zkEtyJBFi^&<`OxOYoj)Ezm{_76`n#fAu)n-@slGW}l}8f2+5z zZba{iEYF_dYN9I{ zClWlzO`MWP;<65p!}4EtBELxF-h}#)(Y&z>-p#zHM0q%QRufW_2`SC%D_=cum?HLD z+(^bnWWEN>Y{p$?5}kRg)sOMb9&rxZaO?p9{FR1ITn@2D;DNJLcq9Ahd27|~oNL>Xp z5dDoX5Mdy~K!kw^0}%#ZkQiwBiS}_Gvxnda&bTG{}-zYW-nzg zQTPAdO90J^V$)8~SEVRs-fj2n*2#>TPW!X7r<278$6V8H*}`MB~79~$<%)wGv%ag!uz4is0#m}r3lN^_C!c_i20 z`bUedpYZsvK)?F#&m*0cQ~SE#a){yT@{00XB793DpPU4)UuiRZya~2D->e?Xn#SCy z&{)~p6At%$(?7)Fb$!qu33%-1Gg0Wz1_Q}9xVWRaGLA>pH)KMg6529B|pq9d%+Vn3c|@vQWd5i%w`%UqgX`QCMB(@_S@dq*znB z`l?fF&|C|_i{joDEZEGD&57!ABaW?AWBvt?{@gk{K^;P0+y{RyqvyzIJadbPzPO?9 z^ivDh_qVqzV%LQdr^b!D0dY%5z;)k zR@6ccrXZxmX%q`4h|mSW_eAD`Z|l6dj!x&sQ&SiWzN1y6MHujqE9r_wxMHzaEr?7I zt(G`RmpBAGs7if9U~(CJ?khUN=eK&`DUx5{2`=w|k#dO&PjR2bO42AQPYx0m^KxfA z5E}!z#uatgPSM|E7BxRbd6_8_)Fq!1@&8M9dyChHw9DbUpAy-FagF0Sup>?#d zGy+E|OH~E6Zq^CLV16Q>qXXHc#NZH|yD#rcB+iWN+rPMfXkT)qfTLdiGdNmY9vN{D zWM9Ru2Q8Q)voP-8PE^x(OMl$^FK(&UQ?280bMzOaDE_>3YTMtkUn2Q`+fR$;-;w&g zM-BPEtvUbyOUdO$Oe@ThK_~$Z4lW+px3qr6giQdZeUWaTS4%w8S+E_CyK%7Q(G7XV1w} zX2MkI?$GzC^&G`ovZSTg5P^@wuu&;irb>uXG`Rvm!d}k7ccuncWxZ4`R^~6jr6#Pi z;b#uKUe2?txiDl^Wb8Zr>>39!AZEs%LYXvt`3sJctP{Eg-hj zvg!+7&TO17;^x4l3;?C2h|nrw2^TBt#q%Kv`IYlUaM;C=q+GF3UOFeLOvC8?f|?`% zA$`w!xj+#Z$IlhYg{5^3*Bck;mP#eHikEzpD8Nfx&T4YDb%RU>3c3FV1vRGHXW~bpvV&%eWQTUX=eE>h` zDnq_pSgQcz)CXT3=zI|q=2*FK{(PbAOT>sFDnxPFnjtll%Zw&7YZ$ik#pM;u31szE zBJxav4g}Mv-fvH(tg%x}&al6_+Qlhpj8RrB!4PyY-x3M zv9NS!NIvYs*b6{0YgOpa=&KV81U~%7R@MsTi&I#c<-+PzaqTRkd5L{oq|*jbB~^r- z1gQ=yDk3jt)=Z)+>s5(-;+RdUB$LAX${A$Xm&RwUugpRyUT5W2>b_HEl&F+<^{*)txN{ zJ~)OfFz__Y{}2q%SBmAZQXowQ?10+lmZ_9NiK&)_ccOQ&KiNOrKhhiL%%fW4M3Oyv zms(r)(`bJ^+HPGf>+IU9+RZ>Zv?j(2#J6g-8tc@|n>o8iL?pAOWv9->yZe?3mGyX^ zjC)DJkz75m%Nf8iB4qOg+{HPGFiQ)wGgGJIh&IZdrw=A5ASRSW>QsVWP-xA*-hMbj zqA&HgkQ>j=PEFxrKrRj5AfGuQLV8mqPtguq)!W&F9`wSPd8i+M4$_~Io5;4=eB2NI z4L;uGf69&r^FyU-V=`7)UpglUGB9?5oI+KDmu&2*!(w9o<+3dJIhYl~IIHMna7e}vP9d?uIAj5n?e*vDSu(*<~y znHm9T2O8~2*H)4t4-6%eFbF#WKCul8ioIIi~rjnW}jee z+y7#}&E@}}745(G?LYTHL;l~!dXTb9q-uZ3D?rFkTLPC+%1| zuRNGFtO6Ke2Ydq|ZYjYz8efaF9>uTL%8|j|Mn^_0k&&Tf0X?c7(NHWAS6{aolgq12 zn_rD~K`f5M;!trovA=rUudS>bi$h?t;UU2a3?C9t+1JF4 zq;GJ{3eDa$H$K~WpB5JvmqCiFJC^||#_Hj8j2O$t2j>tQ@%5+5_TshE#2VVcdzBbk z9zw6GUzrnYh&qNo(aBcBYv_6gs|VviMyrjFQKJ(r#=&1&ERJHLRlkb0o9$AYi$_o! z?gBzn`ntf;@71VHwMIL$lXoULbmj~wKy~v)F&1cb8m23B*|_13;AWbyBU){j(1{Sn zEP54TAi_X|fd~UHI1F_C2YVbY`fg-TFxGj2{VHP}Kg>RZ*nYpk{)Dl%e`CLgzy0j@ z8EbtV`$NWBzK`BGf6P9PzdP8cfRMxN(*iBskN)=ONdAYRh4^XV1{a<3$@@_0-W&d{ zt)sJzwn5m3@RNQV_%A2+omoD3aNiK@Lg@UzfPagFqlpALD7dap2L--nqQ?jW5e6a* zL>P!L5Mdy~K!kw^0}%!y3`7`+Fc4v&aSR-0`$eI*-}QyRG35W&M&*Cj!e-yczU1r) zf#R=GS%mAPdgPoPpeH^@SgZ0*M`3M)jB6ao+(Coj-- zIJ*3{keeTC9;BU{1QJ={5J;&7Is|GSp4eu=Go3n>nZ`|Ze(TSG=S8eb-AcrmwBr}a z&_MCez?kjWG=MTQn*z}GO#*1g)GW?<_;7K@4+f5|VV%<4jL<_rgrc|b&j5mox~pn{(in0&Hwl()SX|gR7Wf^_oB>eyn%~Xm zbYO`=MZPIp*>;4gfY|&ExZFK-dn=IY9^zE(fn+*_d+Fuet6r^Y~Hv4Y(%0DXf zW^zM}2Qf^W0!zI+Jgq7V6GVqMn*s|r%_fHn242Ev@msvOAyF=E+8>@-HP$XcOZ>)3`9s< z@k?w!DbM2kLn+IjHlz9r2MDw6*mI{x2X<-G8 z8ju7R5UodxT)$cBI|<2%0ZBtMWwZ2l5|V=kBn{zYv$S>+k^=@LJ6RiDL{}uTik^3~ zlywpelgic)#%8JLBp4=z4FSW#+HaO(PJ&U3x>i{-o28bMkeI}^p^3U#N;wIMNn3qG zLkPJBzadK8ru1=A6&LXZ^&TIqZKu}EC5mDp_RkGp0QvtFA@cus#dmT4f0ro#t0Vh{ z_L7L}2wSHS*8!~f4xqk%xei*gA;di5I#xH%uPNJ~i0cS0nTYGK?XQtGB8;LuH(W== z7YH*)#TG`|2+TcY`zZyQNE;EvNlk4;#20`u;y+419>OI>d_lw))Neul!sZJg|Hnh* z|G(}0*}vZm`~N15=OV5nj2DZ}KthOlbOsV6=9l{nB;qD0xTPh#*cPZ3MroEcI29ZG=yO7uiPmZ?&r((6r_Wa!ggM z05)420r~$MLgfGd{PKHtZG!x7u8D>^8d+q9sQ@C|h%l@~wh_QeWE%k?oG53cjfk`n zN~8;JIfNUFg!87)pU5IJ(ndtK5wsA3GK^9IM0`QS7u0eHB<0=6Z-k~o zlQ_A8{6<840cQ##z97`2J>kZ}z;5m-1?2x%hRFYa`%S4AZG!y2YD#pGR|=3Uwbe&w zAYog5bOsWJmB=D9xYtJ(nL(UHXCRS9W|(*>*<+KqM%oB(GyWQ~$c*@cupQ8xx(yu- zL^hMyC<4dckX%w-8*7DBA_t#o8mxzRA=0;vD!^mf(jR@IO zB5eewijU4k+K5OS5%C2+zLXFWMe-^#il7`+8CBVgBY#q1n{i|t5r&n>HX^tiN460@ z_T+`(Hv;nit3%}fkKggur#C_VcU{~>XCTp4>%Thw#f}PThtK z5+bf6;yTi)8Ionxk44uWaUCy6t|Q_LZlsAHVgwiQ1;9%bSNw{PE3O={Mtni!#WM0@ zxmp~n)2qkjzQGCkzc)nw|H(Jr^w*mp|NDhIxw~B{mKW``M+a45QT)&#PBzyoMdUXk zXvU=s{vvIJu18Z?iL?=6Sc&{bfKvI){vy8-L7ZG}t|Q_LDy6bhLkCdH@{BYH7R$59 zX40oNi)<#b=&jQwvW)ZG!wyTa;bN z9dR8Ed8>-F5i|MpvAUeRN7@J*XBb8D3ft78n;KzQiEe6yaD~xLjUfLM-PCwKZ)#A) zlaPrTol*oRYQz^rd;#msn0#av#Yn^#gi)4=F9;#GS14Zq`F~G{{Qs9Dt?^Be|7$C` zBd-*`v-XC9T8tpzayF}%x3ay zgD}de|+XuKe`F>|M*6^ z4nw!dA`|?sPYn>+Ml@s_5pf-nHbPvqGxcW;Ictct5n*!o<<>@2*b)O_2ZV_j@o3WVWpmXgMQ`YCO-o8Ck!Db5=R(P}7Vr64>Jy%@6sf%rO zSR$Cy58zm#Qq;lQ?yzm^=gOtKGiSzEOO@iX!>;a|dSl_P!i}qiwX-+o*2^nvXKy07 zV{GRZwu9|**!IeLp}d~veMTpH*lz054t6yvJF~L7>VoC4-Ie0{T%lYzUtBMiPZU-+ zih$N(?R}X$sLq$LYpCE$9d?z3VtS=gL5nDJzS6^9CMv$pVXq4IO<_Ur&nq1Eva`kY zDZsu_I9mi(*NPa8(i*<47w#(&DVqzIcQL!L6ZVfltkQ1J} zhfT5Rt?W3PaoCM@!9P)~ER|O-UrUWjNs(BH|jSU@0Y<0ZP3;;{XFnX0{U z`{6aiX=$F#Z)G`l!eKqYnVZXDudh+z%*Oe}qKDub=<(<7VW-$>LeCvy8qRF2uF`-x z>~%Iu5RT_YN9A?w_0;S)I&7iuj;a_NE0tD@g*6u?HF_AxPF;KhwN(I34Ij*6xm+q^ zXDh5N7s|`&dzOk9#9qCGb>vYVEdiVK7Wc8z#r8YwV0hh?V)?F>B`mGgQUP4XjmL^> z#d2Z2xIA`o9lOzTv6HP}3SrQviy;2kZjXno5F2`}wTsHIeD7qdTiF}g`How@Lh2~e zgNCpzemdnR%AM>@U`qfb=DeHcuph8Z5?UVGc3~OdTF~E-nV7KlTw}6kN^=|D{)0|e z*!oshW*Z$xd6%ivR|EK&Qu%yg72B)Ug*&TtK@nSD0~x?|M>Bvy{_hWw|38^}=-N$? z|JgS);vg5a=vhm5;Yw8|LrIUu=hv`9 zYGd?36JunXa`tJ{>}2~+_FaVa@9sF-C1x9;KiR|HA<+N*4tue}eBT})^52O~d$qWB zcKuuz`@t^u4k|j-!`>~5{xEnSK4O7#KhnkC-ElPLwKv?ueq0oKAGQ#ExVzX-bg>`r zIJ(=*yuXKiP-OmTk~uIZy4Z)h*atg~zC>i|IQlX#_kkYvF_HV{A&AV(Pp7k)@htesr#Xi+>^m;GfXb<~ck?;2?a5E~K;*8Gkcd_5?IQlA;v9&mu=wY80 zIsXJw#pKj%3jEM#Kw({2yV#$0u}^m#eYI8!tK|!#)L#H2*)#$B#V+=Rj-!2AE)445 zh+Kb*ahsT(ADc>dvA^qLf75Yvua*xZ_+^prAJHp0_+9KPUF^#pNBgym7`1;98UHUb zj?KRUa6Iuxj-B)=DGEZ*93Ql)9E$AqH>+N#dy4Y7CRu%^j z^f;X&Ul;P_GtV2$$U@~P>$F6Twq^V~<& z@scj*D&E#ek8`ak@iL%gcI>wFc)rVdd6#o-$I&6J=FuMKl_KA(0Mppi>{yrc>elz9rlg3H7}@0V4JT_bOPCXoITE7P#xzrUCt{zjt+ZZCKEkQLV!64FvmHV zLtRdyZ&dgS4%9$0sbmsi|jrGFfDp|60h?*0Xl8y zIVcy_dz^3N`>nG8N(#w^3y9R8s?~?h3g`fwg0o2D21dE1RPJ$#SX4mQnJ%=>uEkb9 zS1ysbj?El;BmOD$Q;xIC)}XFF^D$l)5Nq*~O%}^ETP}0Q9nCSW&WiI!@Dk3QL|2nT zJ1qT@>te|wy7^+=Fb@D2gLJ_@aN$k=RM;2 z$M`b_?#IPq26Kn{0s2pf=bz-yfbRn$|4;E}!1vSQ`62!c_&zM2e}*ors&BS*sK=*3fS2GSxdaacP@^ zBxYQbE9;k?U!j%t81}-lQj$CD2FrZo>K&C0wDebfB@4W!Y-jjh5m8%M)3|k`y=O%sp3y- zYF0f1szXbm)v03vJv+634b6^qhy!ytFyH`?AX&bvu-fDNsp!U^As4C*&s9}{tt4{l zJD}=x{u~%7lisxdwN=K_R}B6B3+E{s(Jw;T+`h;aN9@)-Yw zh{Zo4J@HS-Mf?-84*!HK!#^Rb@J~o0{1dVT|AhR%FGT+TfA1dpz$VE5kWF|ikUjV( zL<{~2fq_0rAmE>1@A)T~cK!*boPUDr<)7ed`6n0%{)tV9e_}J?pV&nBC&(xN1a0J> zpo#nww2*&-2J%nPKK==s$3H>q_$O!_{{(I0pP*^{6SRzff`;)=&@TQ7n#DgstMIvn z)&&2=6y~3pT=={mf)P&C@V6UJJMl?Jv+a9-fpz^HzK||x3zA#dS$rqAJ;+gk8J$5q2cD<54!=!uBpa%trVAJR5D>mB#DLg7YYwZDpTew?E2G;z9>f z+GGE&QeJDObi}T*q{KSgY$;u>DrM2EE!N$xQhe>iwphz-wWjpA5cu2K=9cldv)!7q zBeo?BI1`cg0bTdC?fP#jr-{h>+GGErQp|?$>)3hUuUL{gcfD1m_^RvL^)^e275n$a zY}mgq)@|wgeX%VTOx_o>VgJ5Zk6W9LeG^^x16}vG?fQZ6Ufkav`%jf(qU(O3>%EpF zE4uEtqU-)x$F6s&>P#HpZ^h*OR!rV+oyYfEF?qigllRw!$p?VR2ikW1Xm~FkXpjAu zN-;6{K*vsIOR{3}0V^gSuw(K8D<&VXV)6kiCLge3@&PL*AE*nH4+4`9w(a^!g)ehH zJlNi%5==xsXkWn(cG}kIgH}X7XvNlpu`bJMd@$DSrucxfV(URGwjQhtTMq$S54G+3 zutJK7t%uqz06k=%b`M#x^^g@?580>PLsmpSWJTmdvF;AlArp}gSrPe=6_F3sg~*42 z$cNi@{eo)OMC8NmmYwn8j@TtKDS#kWB6~~WRar~GS z$B$WY{FoKTk6Cg2m>tKD#p=TG-!HUKw ztZ00~ipD3bXnewo#wV<3e8P&xC#-0EqCPY}2{b<0w(GMBTP7NxY>!#i{F5EAVO7S& z@RL~cf2NXr)md@;q!q_cT5q!%%8K!)tQddFit(qc7=Ox&@u%v- z_|w4n(`~!{J{;pux5q5&`DyC}ecFnXr>!`7+KQ8>tvGqwij$|UIC;7*oIC@ZJkz%8 ze~07bnf6$ZN-<}{Gj_4?87o)$jFqc=#?DnfW5w|^R<80HD_8l9m8*QlisNT2I5yYw zv%uuDZM*(e;mf2~&$h>6D#gU)vmGwu=1Z`4{8{Vdde%C*p0#%TS!>6it*7ITv-js6 zr|l}oRYDj&cbxXvYeRePI5tMaaXPPZToUT5&Nhb}$A&MYSTr$6vFO2&V$sAP#iEHp zO1LHl__1xfNU?A)NU4Vlbh?55ZlHgwLb8dk?)KR0REjabobC<_!E?HS{%xv|uR1IG zyRGQ&wxYk=ivDgZ`n#>@@3x}9+lu~fEBd?ZL;ux4|J6W0_T&(Su5)#J?DZsx=$wdFvR`g$OMgP@S^j~d7|J7FXUu{MI)mHRhZAJgp5Q4f@SAF=Y5C7Kz z|JMQku4rx|?7H@yoqrhKz;#xPU1!DEbykdBXT{icR*YR|!I-hJI5(h&H=u{TsvBkx zZ$JikBN~8g0}TPA$zoCegy<=?CDp&h8A3{@=il!yF=xM`F9w@t9Mhj^bEsK_uTL9-VI&k8d+m zzsF2{hxzy}Gc(uMqhVPUKhyBwcuT1YCb-^H+JkX=L>f{?0o5@ zIw#!9di>{v>@cnm(t%=P@2+i+Fgo0O5q@8e5&z_GGBAtIukT$zC7=8N>!f$5KjHkw zqt0(T>}Tod5w*d7z+Y$khxwr;Yya`Rk207>KFapuCRW>{EP>O-M;NWH=k|B+u5*vF zH5`M-UbLH)wm-`5>gEkE#9rLBo87&=^-*@Oc(Zr!uD8pqFLATRuBJ?nvhV4VrLKu} z>}KC<7W*Mt?4@oo>efQ++E}NTH@2I;yEUP;_sC)|^A!WJ-_3qZHW9muzU#%_FN?k0 zSL_wB?%nJIs@N|2t{3|mSuE}=_6@NuyV*xnv7PkYEw&K5KDKo?`#H0_za(pVrBM^) z{~JT(|LY$A*_UjB{J#);RjkK@Bo?FZE}TH-CnQie7(gw=ULD)EoBf*V-VXY%7yB() ztk+kpFSdO*`)yT>f7grsp)9t?S8Q(#E&P!xhL-=xXzg>d*lT>n`l&ykSH;lJ&l|=5 zQWo3iD>e|raQ&4khJpGkquBqH#cuQ!OT;ia{y`PP=>LOJ?4M+@L0_>XO~rp!#W2_Y z*(mmJve=NX*f34}e^bkKKqFc)HnVlUb4?DT=~Qd!kazN!w#uJJN=(O0)7f*>x7 z9q|>rIrh@s&Ns+5I_axXY@aN4i?7(x*tNTz0a*-i517UF%VH^Cv9Z|8b~^`Tv3B}u zw01-m8}}8Ph+Vhaxmgx#qpzaaRgNH`PAu(obhftg6lWFwY|-OKcxACkr>(tJl%bTp zt+8F$FMdg6{rDESMc@u0J0Q0RGE-S*XSgNHNp7sd*l}nqM!A+zY`D4|3Ri^%Uke>> z@V(fR9GE+WHTWjkU=a=8XQ|@87?(Iv^>?)B7BTn5wqv3aOW=RMBmBJr+ z+$%iaWK4fX~A=Tu^LYurrj`0x2_~G#R{c!mFepprf%QY*0EWF~! z!Yh7ERs5;YiuD!wczDH+hgbZ#Wpp1`4gPM;2A>FT@QLsSpRhFeglh29p$(Sgm@HvT zo(ymB$?yiBv`m^OEtBR+1;`gd0r@rw$TtCyr@{ewDjblfEIoh90^m~?0H0C-{!L8) zpAHA`>2Ls_wgC9F1;D2*06wh%{BkIO`ig$WQuQ;Is-Ll}=x0=u{}S5dkI6wJBerM5 zaq?_9PM)>kg&sqR~)&lsm3h=Ln0=`EAPFgA_92CcqRluq?Z1dEs=(beRt*W@H zW))Xks<>KJacyW7kI0VvG&*t}Kgjj($j)`41LIs5Ixx<47C5d89~k5hQExI zL*F4g^v&o{(gI@A0%FpF^P~mDqy@yJ0&yf1M12k&3ZFxV!iVBe_)r|O%%MY;{v5LO z=a8j8hg5%#hW6*(!qA(2xBt-Z^xn3%N1TaIIFn$h7anzPYh|a~9(Cqg*t^-*N1eRG zKEUp9kIRJK?)2W)N1RiiaPGkPy&h$jq4B~uTLqB+6Cv{dr*8hG|1#u%iebiBCuxj~ zA9@@1)8Vl3M?S!IyU@Xy>`~`zi}<7PaXt(mEsvue)Z6kX057%ejin_L@OYQoqq|P; zWN&G0ebl+PjX|;bEyTo2J9y-)JnH=SHg+0@@=gVhZ*z;iO)KWfDNd;q&2$P}Bmeif z`M>8hG?unjzSC>vJ5lN#ZmD;8rSwL+&`6hPBna|5y~^H+%HHKx_O9^CFlS@wScePq zkGchZ^mJ^u73LrIn)zWg^Io^qd%aRR%rGB8GXga?2w+qn+&p&NdgEkGds3YGm6%*|xjc9(J=m zY-Hm{?tzV}6x$glIyxmWy?5g}$nRq!)1{9XK5416ebjm4LwvpLcK*QmLzzlfLR#>) z{o0Q?PY6}-Zd`+UQYLGK|3p>&8RxV9!ms(5^CznCpN6&kIo0y#LtFlwD9oXY{zMpv zFc4uN!a#(92m=uYA`C?>?LyT)~3 zrmCho--2I~vAUPIYpc0Z#kmo$>I#BTN0@?Z0Pf@hph{=cGvk?=V+-T6QwuYx>2&k3 zZC2od?)>Sw^d-X5g>LLvslX9)X=QDxTs&WF&NuxQ4ohzQ*7S7qh#~dQW&qR0ff-Nb zn+JuwHf#nco!RtsDtkOv0}{+w+f1hAu^EtT$*15XaDh-$0~lIXJ@^e-R^+9kSzu^% zySe|f8TeeMhump6^aD^FJ8R;l1hbA6Rt0B7wHgK!Wwun{(Eofv!q*D;B#jEdz6EJ~ zwx@L9w+bBBvka<0Pde94g`TF6qOjcp#M+Ca3V<{*8U~PN)yuF2&`(7AFOl3vq@Siy z2m=w3{t%=@q@QLBh}KR^=piIUq(7DASdxF6fe757)q3^aI*7Bh*v-XWLYkBP*C%-p7y89Pj|CcyGx#t2^f?xGRmA2xS z_8dB3X{}a}XWWKd?H3&5gTvOU!{U6J;lKCu1}ep7*gSseAG?FncrC5(bsPsk_UDXM`{$NcSm0`FS^ z+dTfs&S<^(5*5Rb0Kn1u@Cm%-AmK~ZIvassG7Bvs6ir4&HN#%+6duF@iu+?>W1Hfz z>|hL{@1C3*0tTTbgqv<^F>MZ3aQ>eRIsgC1{@&HikpDwE8{s@`3S7}xvByC>GkRul zI8i)UD-PQJH@k7r{>B~$t*+W2qJ;(&@rN0U;-E!w(BR59ii1{^u?AQCzJgtMz%p1D z!q|MB#X+O^LYcMY;yvVb8qR$kHe`3QuX~9nXTZE53IT0L-GFEmm(KNHeknoFCRiH2 zui0dc@C5$0Iw@?L3T+{Mi`NXN9CwJlJKvh;0C|u7EZBKnmCH7&%o6Bgmtn#VHzEqY=AKL8jcaz+4EO z@Z1IF`bM^}mA#(5fx_h=&`6?A@73BoUt~*g-OZM%21FXE9v>n&PtLHjR2-iCDdgNG zhqgeFu{&vvtkwu^;RB?TA-F-~k@+r&p%K9#dk}ujPr25l7W?462t=IaO6A7e5tC@;%u2D0!Tve+Q6fkm!#+|jQiS9c9<0yJ zg4I3L*J!mAsP&tmT5LS|+neQJ>6lil+0C%^Y~egE?7h=%sz z*Ztdv{_O8U1)N&!TPOQ95ei{!WA)BF52mtIfc!rcBL6?W_m=;)3G%-dGS9i&Mj(8V z8EZ(VYj-``=sOJRNy0fgcOt3a(i?Dqw`_Xl0SuB9YlI)nMiQvA{anUj-tK zAsXP{ZiQy>X~AGzQlv5V_w0XCUH`y`36%KDVwnDsMiBu8zamEQfAeIF>OYI*|ARAO zx!=x;9T!2;|F7ucvzH^d?%nKPX+!)sz=iO+&wboJ_8;s&2~YoJ3C)WM$bMXeC{nC) zzC=OpAz`>8w!^;CXQzDj5TQ-eyZ~zhBGW7-Y?3fJtxg*Vlhe*;&=58?+XgMTQkTop zct`}5bpC&PR{|JSb+zw(FUbt#0WXUzf*KbTBC@X{nh*j;AS?-LaKR8ph(dx%P_b62 zinUd%b!oM=Rn&@gtD+*Y(qi4JR_j`;-CSE+t+jTqZT-J<&Rb{ROh|~W?f*u@ox7iV z?so2dZ@w={bU}S+RD!0eP!$jxk_B%w?D#9P?K;_u?7k3LnRlg<^oa|upZQeBpvc{pULBi%XIH_)jDbn0t+sf-W(Pv`rpQ49B zjl`n2Cl7JGS5kqWD`wGKIi_)KBVG~8c5@bH*+PHsqxYqe4k#_`cpy*PU-S-k=&^-*w1yG+99IKcwux-g zYS4oRUR_FR*vCcwt4v~hj!)`o1&96ksqKILV>{+3@m0HKW6-c$mu=|Ah8^g}R)WR7 zt+{38^h{IFG3?(wI7odlm$Hl`^Jgf=!r%z_C}&TQUd+u{L7V9nL#8QJR8# z5tMs#B^vi~@Pl%%R{WqTHCAFoE|TQ9MME?g0_!Du2}pxE2qb|=OPvi#^@oCr-4CAn z${~9X)@pY$2XQNqS~0J+b?p&&qcpJ3y`5{gxU6Vt6DaG(lzn5&nx(7L0GZjDeU)vo z4Nl55p+Re^aiR7-jSDp>5UcHS(N+_lih_<9t3I2%V0ZL(HcPMMpS1;dTDuwDZWMJ|DZoku}vvCG`a$yT z?NMb%^uqbS;IL;Fy|Z!8ZFMXPcAtwylrKP(%A^K2X`-&7c0p~!{G`4tsmluvdr^iA z(VEktNPju0Cvs*ia9@$kU&UJ#N&Phr#Ts|5tV-9BpLkBM7yk{Y0^G0Pfc5n=Ck1-&;lS+t`og_73rXD+!uyY8#f@Euw+DlfyMCbHunn`e6yDl^bMAj5!RrqFAT{%Qu&mo$=GL7j zczswuLUf>ydIYbyuo>I6Wfzi3wY4Qc@CtkaI#@l6 z6nG?;o}zo4t}o~psVv<1l#Xz&)q^Chz0Bzfd;_WtM6A8V>G})Iz_uQ6y1u4gr%rss z<8&R4>tvVH6;I3XK9~VCMI!z*6t3>s-QnnO!V!>|$YK7pocEz6^vxE(Qtv37$f!;do7Ir&v?z{Ym;gN&_Kn z7$@=X$dqw>xt(xnQ-MtCv8Dnh8aT#|HwFBTIl%Rp&#=&nNG$`ggykHFCA8)liB9#D zBz`;``%2jRr^I%33c=411L+J1Ah{Tie@4D50Hdn=HR@567_m;`s!*AFe%O0tIPuc3 zcLi`#Rsc!mM9KtU9NkRBt*3OVIwFYGEAmV9$x%@u0oEhsdKB`hsFa98A)k(l3Kg;u zg=|D2pN&e1C=~L!sHjjOm!ObKPzYgjM>OXW6moe~6biBXdJ4UUl6d{CH-){ohrM^g z8_ax9xjk^ZeicyKZ>HgKhejaYc90)S9-5BFt?t>TwxJ!9#d%E7xgTeA?#Be3`!S$% zKiR3DPDkes)nGiMMqs=vFyFTPX0 zn83%or}%zd{D5iNmSGpbu>m2gwwJ2!cB)7BROQ9ba?*n0`|ScdVJAVqP@)YEbJOX9 zVuiqg9EdMK=S6i%eA}D9C6d6Nl_K@<{)&*i7*)JT?bv9Svad+Jw$UDl93wdgz=%rp z1pb<*jXct2iJq!U^ueZSu$Ww+p5Cb|hEd(KBpa~(r`on%WEb1um!Uc-iHLU;CqY`# zhMfeHiakIiqg0SAU+3pZu46Hg8vCOSQ2^*DW3|Xepx4{`cj~n+YI#2bAU*{kz1jgX zC)nY$oW#JXb`k@&8>s+xB)H|22;k;WS{@-(hf1^woA^SfM1XE@ zIPsNkzCQF_@|EbTIg5sh>_PTmLJX!jZhs#8O10B|f)KuoQ`@CFzkncwuS6s#HqAW5 z-pjE?eMSK(+KKD52w)iNp@#au!^s~?hZqixb8HW^t}AsQhT}A1H*)5#vxuo_fWRsQ znWLZTN^P_3N`Zy4-QrvgrlY9I`;Gq6N1 z+o~?urid9}&N8HL?o^ineT|s6o8i53S*N<1kk!K5Lsgr&HlXCdf=0rT{@VhBE}kzF{ZC{^mACJOeB?Q0HB-Z-TG@&T_GBB$7wM~FS&>AXtR`#6y>rFGupG~X%g8Nd4;C)mF9Ft#9Ep45i=sRTFF4pS-I7l)LJ`MpRwOvx3I#hrriUACW5Yk30VG zptXyR#QA?x#QFa(zyH^k(D|R#D`0!(A-o(60NeX+h40roGwgDSmF^mdOmcNimnz zku>EA@#Z<+qf2 zrAb%0nFY8weaWeDmYf;od7`kaTZlfwUvl&uZ{F)UsCd3R@Adrjd2hReI{TXQP}g0|=-0gvbsy=~ePkDPM=d%t%RK=-DY`SCf=d)VlgOAJE$=ev zC2rD*kx377TD!NmUSeo9l!H|&u(|20Om{BhVN!(B^r2!wX`