Well, the answer was Secondary indexes. I am guessing they were corrupted somehow. I dropped all of them, cleanup, and now nodes are bootstrapping fine.
On Thu, Oct 30, 2014 at 3:50 PM, Maxime <maxim...@gmail.com> wrote: > I've been trying to go through the logs but I can't say I understand very > well the details: > > INFO [SlabPoolCleaner] 2014-10-30 19:20:18,446 ColumnFamilyStore.java:856 > - Enqueuing flush of loc: 7977119 (1%) on-heap, 0 (0%) off-heap > DEBUG [SharedPool-Worker-22] 2014-10-30 19:20:18,446 > AbstractSimplePerColumnSecondaryIndex.java:124 - applying index row > 2c95cbbb61fb8ec3bd06d70058bfa236ccad5195e48fd00c056f7e1e3fdd4368 in > ColumnFamily(loc.loc_id_idx [66652e312e31332e3830:0:false:0@1414696815026000 > !63072000,]) > DEBUG [SharedPool-Worker-6] 2014-10-30 19:20:18,446 > AbstractSimplePerColumnSecondaryIndex.java:124 - applying index row > 41fc260427a88d2f084971702fdcb32756e0731c6042f93e9761e03db5197990 in > ColumnFamily(loc.loc_id_idx [66652e312e31332e3830:0:false:0@1414696815333000 > !63072000,]) > DEBUG [SharedPool-Worker-25] 2014-10-30 19:20:18,446 > AbstractSimplePerColumnSecondaryIndex.java:124 - applying index row > 2e8c4dab33faade0a4fc265e4126e43dc2e58fb72830f73d7e9b8e836101d413 in > ColumnFamily(loc.loc_id_idx [66652e312e31332e3830:0:false:0@1414696815335000 > !63072000,]) > DEBUG [SharedPool-Worker-26] 2014-10-30 19:20:18,446 > AbstractSimplePerColumnSecondaryIndex.java:124 - applying index row > 245bec68c5820364a72db093d5c9899b631e692006881c98f0abf4da5fbff4cd in > ColumnFamily(loc.loc_id_idx [66652e312e31332e3830:0:false:0@1414696815344000 > !63072000,]) > DEBUG [SharedPool-Worker-20] 2014-10-30 19:20:18,446 > AbstractSimplePerColumnSecondaryIndex.java:124 - applying index row > ea8dfb47177bd40f46aac4fe41d3cfea3316cf35451ace0825f46b6e0fa9e3ef in > ColumnFamily(loc.loc_id_idx [66652e312e31332e3830:0:false:0@1414696815262000 > !63072000,]) > > This is a sample of Enqueuing flush events in the storm. > > On Thu, Oct 30, 2014 at 12:20 PM, Maxime <maxim...@gmail.com> wrote: > >> I will give a shot adding the logging. >> >> I've tried some experiments and I have no clue what could be happening >> anymore: >> >> I tried setting all nodes to a streamthroughput of 1 except 1, to see if >> somehow it was getting overloaded by too many streams coming in at once, >> nope. >> I went through the source at ColumnFamilyStore.java:856 where the huge >> burst of "Enqueuing flush..." occurs, and it's clearly at the moment >> memtables get converted to SSTables on disk. So I started the bootstrap >> process and using a bash script trigerred a 'nodetool flush' every minute >> during the processes. At first it seemed to work, but again after what >> seems to be a locally-trigered cue, the burst (many many thousands of >> Enqueuing flush...). But through my previous experiment, I am fairly >> certain it's not a question of volume of data coming in (throughput), or >> number of SSTables being streamed (dealing at max 150 files pr node). >> >> Does anyone know if such Enqueuing bursts are normal during bootstrap? >> I'd like to be able to say "it's because my nodes are underpowered", but at >> the moment, I'm leaning towards a bug of some kind. >> >> On Wed, Oct 29, 2014 at 3:05 PM, DuyHai Doan <doanduy...@gmail.com> >> wrote: >> >>> Some ideas: >>> >>> 1) Put on DEBUG log on the joining node to see what is going on in >>> details with the stream with 1500 files >>> >>> 2) Check the stream ID to see whether it's a new stream or an old one >>> pending >>> >>> >>> >>> On Wed, Oct 29, 2014 at 2:21 AM, Maxime <maxim...@gmail.com> wrote: >>> >>>> Doan, thanks for the tip, I just read about it this morning, just >>>> waiting for the new version to pop up on the debian datastax repo. >>>> >>>> Michael, I do believe you are correct in the general running of the >>>> cluster and I've reset everything. >>>> >>>> So it took me a while to reply, I finally got the SSTables down, as >>>> seen in the OpsCenter graphs. I'm stumped however because when I bootstrap >>>> the new node, I still see very large number of files being streamed (~1500 >>>> for some nodes) and the bootstrap process is failing exactly as it did >>>> before, in a flury of "Enqueuing flush of ..." >>>> >>>> Any ideas? I'm reaching the end of what I know I can do, OpsCenter says >>>> around 32 SStables per CF, but still streaming tons of "files". :-/ >>>> >>>> >>>> On Mon, Oct 27, 2014 at 1:12 PM, DuyHai Doan <doanduy...@gmail.com> >>>> wrote: >>>> >>>>> "Tombstones will be a very important issue for me since the dataset >>>>> is very much a rolling dataset using TTLs heavily." >>>>> >>>>> --> You can try the new DateTiered compaction strategy ( >>>>> https://issues.apache.org/jira/browse/CASSANDRA-6602) released on >>>>> 2.1.1 if you have a time series data model to eliminate tombstones >>>>> >>>>> On Mon, Oct 27, 2014 at 5:47 PM, Laing, Michael < >>>>> michael.la...@nytimes.com> wrote: >>>>> >>>>>> Again, from our experience w 2.0.x: >>>>>> >>>>>> Revert to the defaults - you are manually setting heap way too high >>>>>> IMHO. >>>>>> >>>>>> On our small nodes we tried LCS - way too much compaction - switch >>>>>> all CFs to STCS. >>>>>> >>>>>> We do a major rolling compaction on our small nodes weekly during >>>>>> less busy hours - works great. Be sure you have enough disk. >>>>>> >>>>>> We never explicitly delete and only use ttls or truncation. You can >>>>>> set GC to 0 in that case, so tombstones are more readily expunged. There >>>>>> are a couple threads in the list that discuss this... also normal rolling >>>>>> repair becomes optional, reducing load (still repair if something unusual >>>>>> happens tho...). >>>>>> >>>>>> In your current situation, you need to kickstart compaction - are >>>>>> there any CFs you can truncate at least temporarily? Then try compacting >>>>>> a >>>>>> small CF, then another, etc. >>>>>> >>>>>> Hopefully you can get enough headroom to add a node. >>>>>> >>>>>> ml >>>>>> >>>>>> >>>>>> >>>>>> >>>>>> On Sun, Oct 26, 2014 at 6:24 PM, Maxime <maxim...@gmail.com> wrote: >>>>>> >>>>>>> Hmm, thanks for the reading. >>>>>>> >>>>>>> I initially followed some (perhaps too old) maintenance scripts, >>>>>>> which included weekly 'nodetool compact'. Is there a way for me to undo >>>>>>> the >>>>>>> damage? Tombstones will be a very important issue for me since the >>>>>>> dataset >>>>>>> is very much a rolling dataset using TTLs heavily. >>>>>>> >>>>>>> On Sun, Oct 26, 2014 at 6:04 PM, DuyHai Doan <doanduy...@gmail.com> >>>>>>> wrote: >>>>>>> >>>>>>>> "Should doing a major compaction on those nodes lead to a >>>>>>>> restructuration of the SSTables?" --> Beware of the major >>>>>>>> compaction on SizeTiered, it will create 2 giant SSTables and the >>>>>>>> expired/outdated/tombstone columns in this big file will be never >>>>>>>> cleaned >>>>>>>> since the SSTable will never get a chance to be compacted again >>>>>>>> >>>>>>>> Essentially to reduce the fragmentation of small SSTables you can >>>>>>>> stay with SizeTiered compaction and play around with compaction >>>>>>>> properties >>>>>>>> (the thresholds) to make C* group a bunch of files each time it >>>>>>>> compacts so >>>>>>>> that the file number shrinks to a reasonable count >>>>>>>> >>>>>>>> Since you're using C* 2.1 and anti-compaction has been introduced, >>>>>>>> I hesitate advising you to use Leveled compaction as a work-around to >>>>>>>> reduce SSTable count. >>>>>>>> >>>>>>>> Things are a little bit more complicated because of the >>>>>>>> incremental repair process (I don't know whether you're using >>>>>>>> incremental >>>>>>>> repair or not in production). The Dev blog says that Leveled >>>>>>>> compaction is >>>>>>>> performed only on repaired SSTables, the un-repaired ones still use >>>>>>>> SizeTiered, more details here: >>>>>>>> http://www.datastax.com/dev/blog/anticompaction-in-cassandra-2-1 >>>>>>>> >>>>>>>> Regards >>>>>>>> >>>>>>>> >>>>>>>> >>>>>>>> >>>>>>>> >>>>>>>> On Sun, Oct 26, 2014 at 9:44 PM, Jonathan Haddad <j...@jonhaddad.com >>>>>>>> > wrote: >>>>>>>> >>>>>>>>> If the issue is related to I/O, you're going to want to determine >>>>>>>>> if >>>>>>>>> you're saturated. Take a look at `iostat -dmx 1`, you'll see >>>>>>>>> avgqu-sz >>>>>>>>> (queue size) and svctm, (service time). The higher those numbers >>>>>>>>> are, the most overwhelmed your disk is. >>>>>>>>> >>>>>>>>> On Sun, Oct 26, 2014 at 12:01 PM, DuyHai Doan < >>>>>>>>> doanduy...@gmail.com> wrote: >>>>>>>>> > Hello Maxime >>>>>>>>> > >>>>>>>>> > Increasing the flush writers won't help if your disk I/O is not >>>>>>>>> keeping up. >>>>>>>>> > >>>>>>>>> > I've had a look into the log file, below are some remarks: >>>>>>>>> > >>>>>>>>> > 1) There are a lot of SSTables on disk for some tables (events >>>>>>>>> for example, >>>>>>>>> > but not only). I've seen that some compactions are taking up to >>>>>>>>> 32 SSTables >>>>>>>>> > (which corresponds to the default max value for SizeTiered >>>>>>>>> compaction). >>>>>>>>> > >>>>>>>>> > 2) There is a secondary index that I found suspicious : >>>>>>>>> loc.loc_id_idx. As >>>>>>>>> > its name implies I have the impression that it's an index on the >>>>>>>>> id of the >>>>>>>>> > loc which would lead to almost an 1-1 relationship between the >>>>>>>>> indexed value >>>>>>>>> > and the original loc. Such index should be avoided because they >>>>>>>>> do not >>>>>>>>> > perform well. If it's not an index on the loc_id, please >>>>>>>>> disregard my remark >>>>>>>>> > >>>>>>>>> > 3) There is a clear imbalance of SSTable count on some nodes. In >>>>>>>>> the log, I >>>>>>>>> > saw: >>>>>>>>> > >>>>>>>>> > INFO [STREAM-IN-/xxxx.xxxx.xxxx.20] 2014-10-25 02:21:43,360 >>>>>>>>> > StreamResultFuture.java:166 - [Stream >>>>>>>>> #a6e54ea0-5bed-11e4-8df5-f357715e1a79 >>>>>>>>> > ID#0] Prepare completed. Receiving 163 files(4 111 187 195 >>>>>>>>> bytes), sending 0 >>>>>>>>> > files(0 bytes) >>>>>>>>> > >>>>>>>>> > INFO [STREAM-IN-/xxxx.xxxx.xxxx.81] 2014-10-25 02:21:46,121 >>>>>>>>> > StreamResultFuture.java:166 - [Stream >>>>>>>>> #a6e54ea0-5bed-11e4-8df5-f357715e1a79 >>>>>>>>> > ID#0] Prepare completed. Receiving 154 files(3 332 779 920 >>>>>>>>> bytes), sending 0 >>>>>>>>> > files(0 bytes) >>>>>>>>> > >>>>>>>>> > INFO [STREAM-IN-/xxxx.xxxx.xxxx.71] 2014-10-25 02:21:50,494 >>>>>>>>> > StreamResultFuture.java:166 - [Stream >>>>>>>>> #a6e54ea0-5bed-11e4-8df5-f357715e1a79 >>>>>>>>> > ID#0] Prepare completed. Receiving 1315 files(4 606 316 933 >>>>>>>>> bytes), sending >>>>>>>>> > 0 files(0 bytes) >>>>>>>>> > >>>>>>>>> > INFO [STREAM-IN-/xxxx.xxxx.xxxx.217] 2014-10-25 02:21:51,036 >>>>>>>>> > StreamResultFuture.java:166 - [Stream >>>>>>>>> #a6e54ea0-5bed-11e4-8df5-f357715e1a79 >>>>>>>>> > ID#0] Prepare completed. Receiving 1640 files(3 208 023 573 >>>>>>>>> bytes), sending >>>>>>>>> > 0 files(0 bytes) >>>>>>>>> > >>>>>>>>> > As you can see, the existing 4 nodes are streaming data to the >>>>>>>>> new node and >>>>>>>>> > on average the data set size is about 3.3 - 4.5 Gb. However the >>>>>>>>> number of >>>>>>>>> > SSTables is around 150 files for nodes xxxx.xxxx.xxxx.20 and >>>>>>>>> > xxxx.xxxx.xxxx.81 but goes through the roof to reach 1315 files >>>>>>>>> for >>>>>>>>> > xxxx.xxxx.xxxx.71 and 1640 files for xxxx.xxxx.xxxx.217 >>>>>>>>> > >>>>>>>>> > The total data set size is roughly the same but the file number >>>>>>>>> is x10, >>>>>>>>> > which mean that you'll have a bunch of tiny files. >>>>>>>>> > >>>>>>>>> > I guess that upon reception of those files, there will be a >>>>>>>>> massive flush >>>>>>>>> > to disk, explaining the behaviour you're facing (flush storm) >>>>>>>>> > >>>>>>>>> > I would suggest looking on nodes xxxx.xxxx.xxxx.71 and >>>>>>>>> xxxx.xxxx.xxxx.217 to >>>>>>>>> > check for the total SSTable count for each table to confirm this >>>>>>>>> intuition >>>>>>>>> > >>>>>>>>> > Regards >>>>>>>>> > >>>>>>>>> > >>>>>>>>> > On Sun, Oct 26, 2014 at 4:58 PM, Maxime <maxim...@gmail.com> >>>>>>>>> wrote: >>>>>>>>> >> >>>>>>>>> >> I've emailed you a raw log file of an instance of this >>>>>>>>> happening. >>>>>>>>> >> >>>>>>>>> >> I've been monitoring more closely the timing of events in >>>>>>>>> tpstats and the >>>>>>>>> >> logs and I believe this is what is happening: >>>>>>>>> >> >>>>>>>>> >> - For some reason, C* decides to provoke a flush storm (I say >>>>>>>>> some reason, >>>>>>>>> >> I'm sure there is one but I have had difficulty determining the >>>>>>>>> behaviour >>>>>>>>> >> changes between 1.* and more recent releases). >>>>>>>>> >> - So we see ~ 3000 flush being enqueued. >>>>>>>>> >> - This happens so suddenly that even boosting the number of >>>>>>>>> flush writers >>>>>>>>> >> to 20 does not suffice. I don't even see "all time blocked" >>>>>>>>> numbers for it >>>>>>>>> >> before C* stops responding. I suspect this is due to the sudden >>>>>>>>> OOM and GC >>>>>>>>> >> occurring. >>>>>>>>> >> - The last tpstat that comes back before the node goes down >>>>>>>>> indicates 20 >>>>>>>>> >> active and 3000 pending and the rest 0. It's by far the >>>>>>>>> anomalous activity. >>>>>>>>> >> >>>>>>>>> >> Is there a way to throttle down this generation of Flush? C* >>>>>>>>> complains if >>>>>>>>> >> I set the queue_size to any value (deprecated now?) and >>>>>>>>> boosting the threads >>>>>>>>> >> does not seem to help since even at 20 we're an order of >>>>>>>>> magnitude off. >>>>>>>>> >> >>>>>>>>> >> Suggestions? Comments? >>>>>>>>> >> >>>>>>>>> >> >>>>>>>>> >> On Sun, Oct 26, 2014 at 2:26 AM, DuyHai Doan < >>>>>>>>> doanduy...@gmail.com> wrote: >>>>>>>>> >>> >>>>>>>>> >>> Hello Maxime >>>>>>>>> >>> >>>>>>>>> >>> Can you put the complete logs and config somewhere ? It would >>>>>>>>> be >>>>>>>>> >>> interesting to know what is the cause of the OOM. >>>>>>>>> >>> >>>>>>>>> >>> On Sun, Oct 26, 2014 at 3:15 AM, Maxime <maxim...@gmail.com> >>>>>>>>> wrote: >>>>>>>>> >>>> >>>>>>>>> >>>> Thanks a lot that is comforting. We are also small at the >>>>>>>>> moment so I >>>>>>>>> >>>> definitely can relate with the idea of keeping small and >>>>>>>>> simple at a level >>>>>>>>> >>>> where it just works. >>>>>>>>> >>>> >>>>>>>>> >>>> I see the new Apache version has a lot of fixes so I will try >>>>>>>>> to upgrade >>>>>>>>> >>>> before I look into downgrading. >>>>>>>>> >>>> >>>>>>>>> >>>> >>>>>>>>> >>>> On Saturday, October 25, 2014, Laing, Michael >>>>>>>>> >>>> <michael.la...@nytimes.com> wrote: >>>>>>>>> >>>>> >>>>>>>>> >>>>> Since no one else has stepped in... >>>>>>>>> >>>>> >>>>>>>>> >>>>> We have run clusters with ridiculously small nodes - I have a >>>>>>>>> >>>>> production cluster in AWS with 4GB nodes each with 1 CPU and >>>>>>>>> disk-based >>>>>>>>> >>>>> instance storage. It works fine but you can see those little >>>>>>>>> puppies >>>>>>>>> >>>>> struggle... >>>>>>>>> >>>>> >>>>>>>>> >>>>> And I ran into problems such as you observe... >>>>>>>>> >>>>> >>>>>>>>> >>>>> Upgrading Java to the latest 1.7 and - most importantly - >>>>>>>>> reverting to >>>>>>>>> >>>>> the default configuration, esp. for heap, seemed to settle >>>>>>>>> things down >>>>>>>>> >>>>> completely. Also make sure that you are using the >>>>>>>>> 'recommended production >>>>>>>>> >>>>> settings' from the docs on your boxen. >>>>>>>>> >>>>> >>>>>>>>> >>>>> However we are running 2.0.x not 2.1.0 so YMMV. >>>>>>>>> >>>>> >>>>>>>>> >>>>> And we are switching to 15GB nodes w 2 heftier CPUs each and >>>>>>>>> SSD >>>>>>>>> >>>>> storage - still a 'small' machine, but much more reasonable >>>>>>>>> for C*. >>>>>>>>> >>>>> >>>>>>>>> >>>>> However I can't say I am an expert, since I deliberately >>>>>>>>> keep things so >>>>>>>>> >>>>> simple that we do not encounter problems - it just works so >>>>>>>>> I dig into other >>>>>>>>> >>>>> stuff. >>>>>>>>> >>>>> >>>>>>>>> >>>>> ml >>>>>>>>> >>>>> >>>>>>>>> >>>>> >>>>>>>>> >>>>> On Sat, Oct 25, 2014 at 5:22 PM, Maxime <maxim...@gmail.com> >>>>>>>>> wrote: >>>>>>>>> >>>>>> >>>>>>>>> >>>>>> Hello, I've been trying to add a new node to my cluster ( 4 >>>>>>>>> nodes ) >>>>>>>>> >>>>>> for a few days now. >>>>>>>>> >>>>>> >>>>>>>>> >>>>>> I started by adding a node similar to my current >>>>>>>>> configuration, 4 GB >>>>>>>>> >>>>>> or RAM + 2 Cores on DigitalOcean. However every time, I >>>>>>>>> would end up getting >>>>>>>>> >>>>>> OOM errors after many log entries of the type: >>>>>>>>> >>>>>> >>>>>>>>> >>>>>> INFO [SlabPoolCleaner] 2014-10-25 13:44:57,240 >>>>>>>>> >>>>>> ColumnFamilyStore.java:856 - Enqueuing flush of mycf: 5383 >>>>>>>>> (0%) on-heap, 0 >>>>>>>>> >>>>>> (0%) off-heap >>>>>>>>> >>>>>> >>>>>>>>> >>>>>> leading to: >>>>>>>>> >>>>>> >>>>>>>>> >>>>>> ka-120-Data.db (39291 bytes) for commitlog position >>>>>>>>> >>>>>> ReplayPosition(segmentId=1414243978538, position=23699418) >>>>>>>>> >>>>>> WARN [SharedPool-Worker-13] 2014-10-25 13:48:18,032 >>>>>>>>> >>>>>> AbstractTracingAwareExecutorService.java:167 - Uncaught >>>>>>>>> exception on thread >>>>>>>>> >>>>>> Thread[SharedPool-Worker-13,5,main]: {} >>>>>>>>> >>>>>> java.lang.OutOfMemoryError: Java heap space >>>>>>>>> >>>>>> >>>>>>>>> >>>>>> Thinking it had to do with either compaction somehow or >>>>>>>>> streaming, 2 >>>>>>>>> >>>>>> activities I've had tremendous issues with in the past; I >>>>>>>>> tried to slow down >>>>>>>>> >>>>>> the setstreamthroughput to extremely low values all the way >>>>>>>>> to 5. I also >>>>>>>>> >>>>>> tried setting setcompactionthoughput to 0, and then reading >>>>>>>>> that in some >>>>>>>>> >>>>>> cases it might be too fast, down to 8. Nothing worked, it >>>>>>>>> merely vaguely >>>>>>>>> >>>>>> changed the mean time to OOM but not in a way indicating >>>>>>>>> either was anywhere >>>>>>>>> >>>>>> a solution. >>>>>>>>> >>>>>> >>>>>>>>> >>>>>> The nodes were configured with 2 GB of Heap initially, I >>>>>>>>> tried to >>>>>>>>> >>>>>> crank it up to 3 GB, stressing the host memory to its limit. >>>>>>>>> >>>>>> >>>>>>>>> >>>>>> After doing some exploration (I am considering writing a >>>>>>>>> Cassandra Ops >>>>>>>>> >>>>>> documentation with lessons learned since there seems to be >>>>>>>>> little of it in >>>>>>>>> >>>>>> organized fashions), I read that some people had strange >>>>>>>>> issues on lower-end >>>>>>>>> >>>>>> boxes like that, so I bit the bullet and upgraded my new >>>>>>>>> node to a 8GB + 4 >>>>>>>>> >>>>>> Core instance, which was anecdotally better. >>>>>>>>> >>>>>> >>>>>>>>> >>>>>> To my complete shock, exact same issues are present, even >>>>>>>>> raising the >>>>>>>>> >>>>>> Heap memory to 6 GB. I figure it can't be a "normal" >>>>>>>>> situation anymore, but >>>>>>>>> >>>>>> must be a bug somehow. >>>>>>>>> >>>>>> >>>>>>>>> >>>>>> My cluster is 4 nodes, RF of 2, about 160 GB of data across >>>>>>>>> all nodes. >>>>>>>>> >>>>>> About 10 CF of varying sizes. Runtime writes are between >>>>>>>>> 300 to 900 / >>>>>>>>> >>>>>> second. Cassandra 2.1.0, nothing too wild. >>>>>>>>> >>>>>> >>>>>>>>> >>>>>> Has anyone encountered these kinds of issues before? I >>>>>>>>> would really >>>>>>>>> >>>>>> enjoy hearing about the experiences of people trying to run >>>>>>>>> small-sized >>>>>>>>> >>>>>> clusters like mine. From everything I read, Cassandra >>>>>>>>> operations go very >>>>>>>>> >>>>>> well on large (16 GB + 8 Cores) machines, but I'm sad to >>>>>>>>> report I've had >>>>>>>>> >>>>>> nothing but trouble trying to run on smaller machines, >>>>>>>>> perhaps I can learn >>>>>>>>> >>>>>> from other's experience? >>>>>>>>> >>>>>> >>>>>>>>> >>>>>> Full logs can be provided to anyone interested. >>>>>>>>> >>>>>> >>>>>>>>> >>>>>> Cheers >>>>>>>>> >>>>> >>>>>>>>> >>>>> >>>>>>>>> >>> >>>>>>>>> >> >>>>>>>>> > >>>>>>>>> >>>>>>>>> >>>>>>>>> >>>>>>>>> -- >>>>>>>>> Jon Haddad >>>>>>>>> http://www.rustyrazorblade.com >>>>>>>>> twitter: rustyrazorblade >>>>>>>>> >>>>>>>> >>>>>>>> >>>>>>> >>>>>> >>>>> >>>> >>> >> >