Return-Path: X-Original-To: archive-asf-public@cust-asf.ponee.io Delivered-To: archive-asf-public@cust-asf.ponee.io Received: from mail.apache.org (hermes.apache.org [140.211.11.3]) by cust-asf.ponee.io (Postfix) with SMTP id 4E68E160C26 for ; Wed, 3 Jan 2018 05:05:08 +0100 (CET) Received: (qmail 35603 invoked by uid 500); 3 Jan 2018 04:05:06 -0000 Mailing-List: contact dev-help@kafka.apache.org; run by ezmlm Precedence: bulk List-Help: List-Unsubscribe: List-Post: List-Id: Reply-To: dev@kafka.apache.org Delivered-To: mailing list dev@kafka.apache.org Received: (qmail 35590 invoked by uid 99); 3 Jan 2018 04:05:05 -0000 Received: from pnap-us-west-generic-nat.apache.org (HELO spamd1-us-west.apache.org) (209.188.14.142) by apache.org (qpsmtpd/0.29) with ESMTP; Wed, 03 Jan 2018 04:05:05 +0000 Received: from localhost (localhost [127.0.0.1]) by spamd1-us-west.apache.org (ASF Mail Server at spamd1-us-west.apache.org) with ESMTP id 0D01EC3C12 for ; Wed, 3 Jan 2018 04:05:04 +0000 (UTC) X-Virus-Scanned: Debian amavisd-new at spamd1-us-west.apache.org X-Spam-Flag: NO X-Spam-Score: -0.021 X-Spam-Level: X-Spam-Status: No, score=-0.021 tagged_above=-999 required=6.31 tests=[DKIM_SIGNED=0.1, DKIM_VALID=-0.1, RCVD_IN_DNSWL_NONE=-0.0001, RCVD_IN_MSPIKE_H3=-0.01, RCVD_IN_MSPIKE_WL=-0.01, SPF_PASS=-0.001] autolearn=disabled Authentication-Results: spamd1-us-west.apache.org (amavisd-new); dkim=pass (2048-bit key) header.d=confluent-io.20150623.gappssmtp.com Received: from mx1-lw-eu.apache.org ([10.40.0.8]) by localhost (spamd1-us-west.apache.org [10.40.0.7]) (amavisd-new, port 10024) with ESMTP id auzT3HwAGzdS for ; Wed, 3 Jan 2018 04:05:01 +0000 (UTC) Received: from mail-pl0-f52.google.com (mail-pl0-f52.google.com [209.85.160.52]) by mx1-lw-eu.apache.org (ASF Mail Server at mx1-lw-eu.apache.org) with ESMTPS id 80AD55F666 for ; Wed, 3 Jan 2018 04:05:00 +0000 (UTC) Received: by mail-pl0-f52.google.com with SMTP id d21so483208pll.1 for ; Tue, 02 Jan 2018 20:05:00 -0800 (PST) DKIM-Signature: v=1; a=rsa-sha256; c=relaxed/relaxed; d=confluent-io.20150623.gappssmtp.com; s=20150623; h=subject:to:references:from:organization:message-id:date:user-agent :mime-version:in-reply-to; bh=8Av7cYzc80ujp90PkZ6DSa5ETj3ws+leyE6q7s3Rqog=; b=hXz9WaH+mbw3pBqWRJZxpovD1Xn0Z7Oto6iVw0rKjqqqeepWOre8/+uo4bCO+4khi6 ImpG4kduXeqDPwdW+Obi/9lbxF2pOvTfDDeWuYnL7x9R9az7vw+7vMFxyA121CUCyN/f gIGgdGe7RkNbNdDL4XKtUE2/W6dbgc6jsn4tLdyOWyBrWSeN/e8Dm380VVdFJ5giMRFP R9HsIT3I6aIL2+nlnEX3ZzBFSJfCdZi3GGRqQvIrybVqS2AFMPvDJXgTIRBY8RQtNffj jynrL85PC2yWxDXbRxWpIyqQ7I6LbYTLSzo6I/AQIuJTnfgxo2oRz/XZ5dpiHQygBa5o lhRg== X-Google-DKIM-Signature: v=1; a=rsa-sha256; c=relaxed/relaxed; d=1e100.net; s=20161025; h=x-gm-message-state:subject:to:references:from:organization :message-id:date:user-agent:mime-version:in-reply-to; bh=8Av7cYzc80ujp90PkZ6DSa5ETj3ws+leyE6q7s3Rqog=; b=FWtElzPwNU5JFXxqrevnpTrwC+gIgYSWPPWQ21yOhM6WW2xKXM1Qy2qf1TFtsAbe6q 6HoqkxbZZDitw4AKYjE53a5bZZ2RFsrCJKi49PGOBDnqLp5AcEmkpHxb1uJGgR2epbQE nRg/8ofDttL5dk69sZj4AkuJtp5W6K7Jhqnk4br55zQCeIz/qxMJBPT66nlCVLizb7Yn gRUWMcB/SlBLkMaTdrvwoiH/zLDhjiphBtHy0dV3mxRtg3/aLTtTXF2Pw5XD7oIQrAHh F68Y0+Hal/VSZt2Jav+fFFa1fndY9oB+9dLGPVfTVp0GMx1k1upL5XvZ15KAbeMTHoSc cnpA== X-Gm-Message-State: AKGB3mKP2bl39UCgonLKqR/LFluE1Mn+ZMC3TEnJvf58AfYIzGQAZs5c KZZbnt3dFM1JVziFsF0M27oHZHf8MKQ= X-Google-Smtp-Source: ACJfBovTMKpPUqJWbqqwUUfJ8+8UEMihrHgaC5idVgC9TjF9nm2LaEft61RoMCSI0mu2k4v0S2yL0A== X-Received: by 10.84.194.228 with SMTP id h91mr214459pld.177.1514952298448; Tue, 02 Jan 2018 20:04:58 -0800 (PST) Received: from Matthias-Sax-Macbook-Pro.local ([2601:647:4a03:2600:9181:9a0c:a78b:f2a]) by smtp.gmail.com with ESMTPSA id 76sm260787pfn.179.2018.01.02.20.04.57 for (version=TLS1_2 cipher=ECDHE-RSA-AES128-GCM-SHA256 bits=128/128); Tue, 02 Jan 2018 20:04:57 -0800 (PST) Subject: Re: [DISCUSS] KIP-228 Negative record timestamp support To: dev@kafka.apache.org References: <178138FA-CA30-44A8-B92A-B966B8CA173F@gmail.com> <24183eae-53cc-75ae-21aa-ae08937fba08@confluent.io> <636803ae-d8e8-9715-ad85-f61406eb5d12@confluent.io> From: "Matthias J. Sax" Organization: Confluent Inc Message-ID: <9273628d-02b5-c448-b3b7-73907fcaa752@confluent.io> Date: Tue, 2 Jan 2018 20:04:56 -0800 User-Agent: Mozilla/5.0 (Macintosh; Intel Mac OS X 10.13; rv:52.0) Gecko/20100101 Thunderbird/52.5.2 MIME-Version: 1.0 In-Reply-To: Content-Type: multipart/signed; micalg=pgp-sha512; protocol="application/pgp-signature"; boundary="X3JPatsQZapEiYKW9f3aSWI8FC6FRuTtB" archived-at: Wed, 03 Jan 2018 04:05:09 -0000 --X3JPatsQZapEiYKW9f3aSWI8FC6FRuTtB Content-Type: multipart/mixed; boundary="IQM7xF8zIbJAiEdap2tayQQxWxtjuEK6x"; protected-headers="v1" From: "Matthias J. Sax" To: dev@kafka.apache.org Message-ID: <9273628d-02b5-c448-b3b7-73907fcaa752@confluent.io> Subject: Re: [DISCUSS] KIP-228 Negative record timestamp support References: <178138FA-CA30-44A8-B92A-B966B8CA173F@gmail.com> <24183eae-53cc-75ae-21aa-ae08937fba08@confluent.io> <636803ae-d8e8-9715-ad85-f61406eb5d12@confluent.io> In-Reply-To: --IQM7xF8zIbJAiEdap2tayQQxWxtjuEK6x Content-Type: text/plain; charset=utf-8 Content-Language: en-US Content-Transfer-Encoding: quoted-printable I was thinking about a broker/topic config. However, I am not sure if we only need to worry about data written in the future (this would only be true, if there would be no records with -1 timestamp already). Assume that we you an existing topic that contains data with -1 =3D UNKNOWN records -- for this case, we would give= those timestamps a new semantics if we suddenly allow negative timestamps. (Assuming that we don't allow -1 as a gap in the timeline what I would rather not do.) Also note, that it's not really client specific IMHO, as one could implement their own clients. There are many third party clients and we don't know if they check for negative timestamps (applications could even assign their own special meaning to negative timestamps as those are unused atm) -- furthermore, all older client not embedding a timestamp default to -1 on the broker side... > The implementation could easily be made to map > those values into a range that is less likely to be utilized (e.g. use = the > values near Long.MIN_VALUE and have the consumer convert back as needed= ). > The sentinel for NO_TIMESTAMP could be changed between versions as long= as > it is handled consistently between client versions. This opens Pandora's box IMHO. > Introducing the new > config seems like it has significant compatibility concerns that need t= o be > sorted out.=20 I cannot follow here -- from my point of view, it relaxes compatibility concerns. If we only allow new topic to enable negative timestamps, old behavior and new behavior are not mixed. IMHO, mixing both would be a real issue. Thus, for new topics we can change "unknown" from -1 to Long.MIN_VALUE and don't mix two different approaches within a single top= ic. I see your point that we do have too many configs -- we could also make it a new value for existing `message.timestamp.type`. -Matthias On 1/2/18 7:48 PM, Ewen Cheslack-Postava wrote: > For `allow.negative.timestamps`, do you mean this as a broker config? I= 'm > not entirely clear on what the proposal would entail. >=20 > I think taking into account whether we're talking about compatibility w= ith > existing data in Kafka vs enabling use of negative timestamps is import= ant > here. If they're effectively not supported today (though admittedly thi= s is > really client-specific), then we need only concern ourselves with data = that > hasn't been produced into Kafka yet. In that case, we can always handle= > sentinel values in special ways if we really want to. For example, the = Java > producer does not accept any values < 0 and the API supports passing nu= ll > rather than the sentinels. The implementation could easily be made to m= ap > those values into a range that is less likely to be utilized (e.g. use = the > values near Long.MIN_VALUE and have the consumer convert back as needed= ). > The sentinel for NO_TIMESTAMP could be changed between versions as long= as > it is handled consistently between client versions. >=20 > IMO we already have way too many configs, so we should think about wher= e > the impact is and if a not ideal, but also not significant compromise c= an > be made and avoid most of the additional complexity. Introducing the ne= w > config seems like it has significant compatibility concerns that need t= o be > sorted out. In contrast, I suspect the use cases we need to support tha= t > have come up so far can handle 1 or 2 special cases and the necessary > munging could be handled safely by interceptors such that it is trivial= to > make sure all your apps do the right thing. I appreciate the pain of a = ton > of mailing list questions about an issue like this, but given the > likelihood of encountering that particular value, I just find it unlike= ly > it would be that common and I think it's a reasonable tradeoff to tell = a > user they might need to handle that one special case. >=20 > -Ewen >=20 > On Thu, Dec 28, 2017 at 12:58 PM, Matthias J. Sax > wrote: >=20 >> I agree that changing message format or using a flag bit might not be >> worth it. >> >> However, just keeping -1 as "unknown" leaving a time gap give me a lot= >> of headache, too. Your arguments about "not an issue in practice" kind= a >> make sense to me, but I see the number of question on the mailing list= >> already if we really follow this path... It will confuse users that >> don't pay attention and "loose" data if Kafka Streams drops records wi= th >> timestamp -1 but processes other records with negative timestamps. >> >> Thus, I was wondering if a new topic config (maybe >> `allow.negative.timestamps` with default `false`) that allows for enab= le >> negative timestamps would be the better solution? With this new config= , >> we would not have any sentinel value for "unknown" and all timestamps >> would be valid. Old producers, can't write to those topics if they are= >> configured with CREATE_TIME though; APPEND_TIME would still work for >> older producers but with APPEND_TIME no negative timestamps are possib= le >> in the first place, so this config would not have any impact anyway. >> >> Kafka Streams could check the topic config and only drop negative >> timestamps is they are not enabled. Or course, existing topic should n= ot >> enable negative timestamps if there are records with -1 in them alread= y >> -- otherwise, semantics break down -- but this would be a config error= >> we cannot prevent. However, I would expect that mostly newly created >> topics would enable this config anyway. >> >> >> -Matthias >> >> On 12/18/17 10:47 PM, Ewen Cheslack-Postava wrote: >>> I think the trivial change of just recognizing using -1 was a mistake= >> for a >>> sentinel value and special casing it while allowing other negative va= lues >>> through is the most practical, reasonable change. >>> >>> Realistically, the scope of impact for that -1 is pretty tiny, as has= >> been >>> pointed out. A single millisecond gap in available timestamps in 1969= =2E >> For >>> producers that really want to be careful (as the NYT data might want = to >>> be), having the producer layer adjust accordingly is unlikely to be a= n >>> issue (you can't assume these timestamps are unique anyway, so they >> cannot >>> reasonably used for ordering; adjusting by 1ms is a practical tradeof= f). >>> >>> Other approaches where we modify the semantics of the timestamp from = the >>> two existing modes require eating up valuable flags in the message >> format, >>> or ramping the message format version, all of which make things >>> significantly messier. Hell, timezones, leap seconds, and ms granular= ity >>> probably make that 1ms window pretty much moot for any practical >>> applications, and for the extremely rare case that an application mig= ht >>> care, they are probably willing to pay the cost of a secondary index = if >>> they needed to store timestamp values in the payload rather than in t= he >>> metadata. >>> >>> Given that we have the current system in place, I suspect that any >>> translation to using Long.MIN_VALUE as the sentinel is probably just = more >>> confusing to users, adds more implementation overhead to client >> libraries, >>> and is more likely to introduce bugs. >>> >>> Warts like these always feel wrong when approached from pure design >>> principles, but the fact is that the constraints are already there. T= o >> me, >>> none of the proposals to move to an encoding we'd prefer seem to add >> enough >>> value to outweigh the migration, compatibility, and implementation co= sts. >>> >>> @Dong -- your point about special timestamp values is a very good one= =2E >> The >>> issue may extend to other cases in the protocol where we use timestam= ps. >> Is >>> this the scope we need to worry about (2 values instead of just 1) or= are >>> there others? This also might be something we want to look out for in= the >>> future -- using special values relative to .MIN_VALUE >>> instead of relative to 0. >>> >>> -Ewen >>> >>> On Tue, Dec 12, 2017 at 11:12 AM, Dong Lin wrot= e: >>> >>>> Hey Konstantin, >>>> >>>> Thanks for updating the KIP. >>>> >>>> If we were to support negative timestamp in the message, we probably= >> also >>>> want to support negative timestamp in ListOffsetRequest. Currently i= n >>>> ListOffsetRequest, timestamp value -2 is used to indicate earliest >>>> timestamp and timestamp value -1 is used to indicate latest timestam= p. >> It >>>> seems that we should make changes accordingly so that -1 and -2 can = be >>>> supported as valid timestamp in ListOffsetRequest. What do you think= ? >>>> >>>> Thanks, >>>> Dong >>>> >>>> >>>> >>>> On Mon, Dec 11, 2017 at 12:55 PM, Konstantin Chukhlomin < >>>> chuhlomin@gmail.com >>>>> wrote: >>>> >>>>> Hi all, >>>>> >>>>> I've updated KIP with few more details: >>>>> Added (proposed) Changes in binary message format < >>>>> https://cwiki.apache.org/confluence/display/KAFKA/KIP-228+ >>>>> Negative+record+timestamp+support#KIP-228Negativerecordtimes >>>>> tampsupport-Changesinbinarymessageformat> >>>>> Added Changes from producer perspective >>>> luence/display/KAFKA/KIP-228+Negative+record+timestamp+supp >>>>> ort#KIP-228Negativerecordtimestampsupport- >> Changesfromproducerperspective >>>>> >>>>> Added Changes from consumer perspective >>>> luence/display/KAFKA/KIP-228+Negative+record+timestamp+supp >>>>> ort#KIP-228Negativerecordtimestampsupport- >> Changesfromconsumerperspective >>>>> >>>>> >>>>> Let me know if it makes sense to you. >>>>> >>>>> -Konstantin >>>>> >>>>>> On Dec 7, 2017, at 2:46 PM, Konstantin Chukhlomin < >> chuhlomin@gmail.com >>>>> >>>>> wrote: >>>>>> >>>>>> Hi Matthias, >>>>>> >>>>>> Indeed for consumers it will be not obvious what =E2=88=921 means:= actual >>>>> timestamp >>>>>> or no timestamp. Nevertheless, it's just =E2=88=921 millisecond, s= o I thought >>>> it >>>>> will be >>>>>> not a big deal to leave it (not clean, but acceptable). >>>>>> >>>>>> I agree that it will much cleaner to have a different type of topi= cs >>>>> that support >>>>>> negative timestamp and/or threat Long.MIN_VALUE as a no-timestamp.= >>>>>> I'll update KIP to make it a proposed solution. >>>>>> >>>>>> Thanks, >>>>>> Konstantin >>>>>> >>>>>>> On Dec 5, 2017, at 7:06 PM, Matthias J. Sax >>>>> wrote: >>>>>>> >>>>>>> Thanks for the KIP Konstantin. >>>>>>> >>>>>>> From my understanding, you propose to just remove the negative >>>> timestamp >>>>>>> check in KafkaProducer and KafkaStreams. If topics are configured= >> with >>>>>>> `CreateTime` brokers also write negative timestamps if they are >>>> embedded >>>>>>> in the message. >>>>>>> >>>>>>> However, I am not sure about the overlapping semantics for -1 >>>> timestamp. >>>>>>> My concerns is, that this ambiguity might result in issues. Assum= e >>>> that >>>>>>> there is a topic (configured with `CreateTime`) for which an old = and >> a >>>>>>> new producer are writing. The old producer uses old message forma= t >> and >>>>>>> does not include any timestamp in the message. The broker will >>>> "upgrade" >>>>>>> this message to the new format and set -1. At the same time, the = new >>>>>>> producer could write a message with valid timestamp -1. A consume= r >>>> could >>>>>>> not distinguish between both cases... >>>>>>> >>>>>>> Also, there might be other Producer implementations that write >>>> negative >>>>>>> timestamps. Thus, those might already exist. For Streams, we don'= t >>>>>>> process those and we should make sure to keep it this way (to avo= id >>>>>>> ambiguity). >>>>>>> >>>>>>> Thus, it might actually make sense to introduce a new timestamp t= ype >>>> to >>>>>>> express those new semantics. The question is still, how to deal w= ith >>>>>>> older producer clients that want to write to those topics. >>>>>>> >>>>>>> - We could either use `Long.MIN_VALUE` as "unknown" (this would b= e >> way >>>>>>> better than -1 as it's not in the middle of the range but at the = very >>>>>>> end and it will also have well-defined semantics). >>>>>>> - Or we use a "mixed-mode" where we use broker wall-clock time fo= r >>>>>>> older message formats (ie, append time semantics for older produc= ers) >>>>>>> - Third, we would even give an error message back to older produc= ers; >>>>>>> this might change the backward compatibility guarantees Kafka >> provides >>>>>>> so far when upgrading brokers. However, this would not affect exi= ting >>>>>>> topics, but only newly created ones (and we could disallow changi= ng >>>> the >>>>>>> semantics to the new timestamp type to guard against miss >>>>>>> configuration). Thus, it might be ok. >>>>>>> >>>>>>> For Streams, we could check the topic config and process negative= >>>>>>> timestamps only if the topic is configures with the new timestamp= >>>> type. >>>>>>> >>>>>>> >>>>>>> Maybe I am a little bit to paranoid about overloading -1 semantic= s. >>>>>>> Curious to get feedback from others. >>>>>>> >>>>>>> >>>>>>> >>>>>>> -Matthias >>>>>>> >>>>>>> >>>>>>> On 12/5/17 1:24 PM, Konstantin Chukhlomin wrote: >>>>>>>> Hi Dong, >>>>>>>> >>>>>>>> Currently we are storing historical timestamp in the message. >>>>>>>> >>>>>>>> What we are trying to achieve is to make it possible to do Kafka= >>>> lookup >>>>>>>> by timestamp. Ideally I would do `offsetsForTimes` to find artic= les >>>>> published >>>>>>>> in 1910s (if we are storing articles on the log). >>>>>>>> >>>>>>>> So first two suggestions aren't really covering our use-case. >>>>>>>> >>>>>>>> We could create a new timestamp type like "HistoricalTimestamp" = or >>>>> "MaybeNegativeTimestamp". >>>>>>>> And the only difference between this one and CreateTime is that = it >>>>> could be negative. >>>>>>>> I tend to use CreateTime for this purpose because it's easier to= >>>>> understand from >>>>>>>> user perspective as a timestamp which publisher can set. >>>>>>>> >>>>>>>> Thanks, >>>>>>>> Konstantin >>>>>>>> >>>>>>>>> On Dec 5, 2017, at 3:47 PM, Dong Lin wrot= e: >>>>>>>>> >>>>>>>>> Hey Konstantin, >>>>>>>>> >>>>>>>>> Thanks for the KIP. I have a few questions below. >>>>>>>>> >>>>>>>>> Strictly speaking Kafka actually allows you to store historical= >>>> data. >>>>> And >>>>>>>>> user are free to encode arbitrary timestamp field in their Kafk= a >>>>> message. >>>>>>>>> For example, your Kafka message can currently have Json or Avro= >>>>> format and >>>>>>>>> you can put a timestamp field there. Do you think that could >> address >>>>> your >>>>>>>>> use-case? >>>>>>>>> >>>>>>>>> Alternatively, KIP-82 introduced Record Header in Kafka and you= can >>>>> also >>>>>>>>> define your customized key/value pair in the header. Do you thi= nk >>>>> this can >>>>>>>>> address your use-case? >>>>>>>>> >>>>>>>>> Also, currently there are two types of timestamp according to >>>> KIP-32. >>>>> If >>>>>>>>> the type is LogAppendTime then the timestamp value is the time = when >>>>> broker >>>>>>>>> receives the message. If the type is CreateTime then the timest= amp >>>>> value is >>>>>>>>> determined when producer produces message. With these two >>>>> definitions, the >>>>>>>>> timestamp should always be positive. We probably need a new typ= e >>>> here >>>>> if we >>>>>>>>> can not put timestamp in the Record Header or the message paylo= ad. >>>>> Does >>>>>>>>> this sound reasonable? >>>>>>>>> >>>>>>>>> Thanks, >>>>>>>>> Dong >>>>>>>>> >>>>>>>>> >>>>>>>>> >>>>>>>>> On Tue, Dec 5, 2017 at 8:40 AM, Konstantin Chukhlomin < >>>>> chuhlomin@gmail.com> >>>>>>>>> wrote: >>>>>>>>> >>>>>>>>>> Hi all, >>>>>>>>>> >>>>>>>>>> I have created a KIP to support negative timestamp: >>>>>>>>>> https://cwiki.apache.org/confluence/display/KAFKA/KIP- >>>>>>>>>> 228+Negative+record+timestamp+support >>>>>>>>> confluence/display/KAFKA/KIP-228+Negative+record+timestamp+ >>>> support> >>>>>>>>>> >>>>>>>>>> Here are proposed changes: https://github.com/apache/ >>>>>>>>>> kafka/compare/trunk...chuhlomin:trunk >>>>>>>>> kafka/compare/trunk...chuhlomin:trunk> >>>>>>>>>> >>>>>>>>>> I'm pretty sure that not cases are covered, so comments and >>>>> suggestions >>>>>>>>>> are welcome. >>>>>>>>>> >>>>>>>>>> Thank you, >>>>>>>>>> Konstantin >>>>>>>> >>>>>>> >>>>>> >>>>> >>>>> >>>> >>> >> >> >=20 --IQM7xF8zIbJAiEdap2tayQQxWxtjuEK6x-- --X3JPatsQZapEiYKW9f3aSWI8FC6FRuTtB Content-Type: application/pgp-signature; name="signature.asc" Content-Description: OpenPGP digital signature Content-Disposition: attachment; filename="signature.asc" -----BEGIN PGP SIGNATURE----- Comment: GPGTools - https://gpgtools.org iQJKBAEBCgA0FiEEFpAnjJ4fbvgzKNUmjQjbao0qTDYFAlpMVmgWHG1hdHRoaWFz QGNvbmZsdWVudC5pbwAKCRCNCNtqjSpMNm7xEACJtzmXbSd9z3rmC4Kn9r25xxrc P0u+P68U4l/Vio0yGjOC3PG62On+5aIYZGS7SSxAuC6jbqVRK++QfQ+u1qXH7dZx BkZZ7o1iLRuPhVE6/CMc5d6dXN58KGn1p3nTah4B9ajV879wBC8dYmSfxBm5wZTw M/tyVi00ppNXVVtrhX/S8yuoHkfyQWk6r/0ITu+bhu1sEmRpJmm0+P96sjiwe0Xf xnWwg+yIlIwL8thuqLeduLIRElYU79H6j3JLklKWWZ5BqaKRQeXKhSGtQh+o+6I/ 61p7vnRCExfzHdkZsIcchVaORzPLVDBgJWqcgmYKImw5T18mUZHKto25RykBvd6N 3hNzRDJ3urDBUr1bAE+iBeiFXF1L4LmZDB+k2Rq+KBiiISt2c/tXXpX4y2+K2+SA VNrl6nr3XY7VUGsrE+lscle5bVp2Yyj49iT7w4OUYD71ph7H/D2dSpE3VQvLb/Fb c9EHuYopUjpIoZSNMgcfjnetDcymlLmkaHyqwfjLCvRAsNg8rYSIJX52Yuloqo6T c1zihXNrMHYKtTbg8Rby2/tD3SXELIjChIgauO3Eizfnw0FdLGHzRMsrfAKCMZzo I0b591MiAfHqMwnSghPC4lazLQ1YYPS6yANVmfCg5FBOO2z3PIOHszFei7QrHdBH UoNCcg3dVFPuttUd/A== =nY1g -----END PGP SIGNATURE----- --X3JPatsQZapEiYKW9f3aSWI8FC6FRuTtB--