Return-Path: X-Original-To: apmail-crunch-user-archive@www.apache.org Delivered-To: apmail-crunch-user-archive@www.apache.org Received: from mail.apache.org (hermes.apache.org [140.211.11.3]) by minotaur.apache.org (Postfix) with SMTP id 285F418B38 for ; Thu, 1 Oct 2015 22:28:50 +0000 (UTC) Received: (qmail 91561 invoked by uid 500); 1 Oct 2015 22:28:50 -0000 Delivered-To: apmail-crunch-user-archive@crunch.apache.org Received: (qmail 91522 invoked by uid 500); 1 Oct 2015 22:28:50 -0000 Mailing-List: contact user-help@crunch.apache.org; run by ezmlm Precedence: bulk List-Help: List-Unsubscribe: List-Post: List-Id: Reply-To: user@crunch.apache.org Delivered-To: mailing list user@crunch.apache.org Received: (qmail 91512 invoked by uid 99); 1 Oct 2015 22:28:49 -0000 Received: from Unknown (HELO spamd4-us-west.apache.org) (209.188.14.142) by apache.org (qpsmtpd/0.29) with ESMTP; Thu, 01 Oct 2015 22:28:49 +0000 Received: from localhost (localhost [127.0.0.1]) by spamd4-us-west.apache.org (ASF Mail Server at spamd4-us-west.apache.org) with ESMTP id 82FBDC0099 for ; Thu, 1 Oct 2015 22:28:49 +0000 (UTC) X-Virus-Scanned: Debian amavisd-new at spamd4-us-west.apache.org X-Spam-Flag: NO X-Spam-Score: 2.18 X-Spam-Level: ** X-Spam-Status: No, score=2.18 tagged_above=-999 required=6.31 tests=[DKIM_SIGNED=0.1, DKIM_VALID=-0.1, DKIM_VALID_AU=-0.1, HTML_MESSAGE=3, RCVD_IN_DNSWL_LOW=-0.7, RCVD_IN_MSPIKE_H3=-0.01, RCVD_IN_MSPIKE_WL=-0.01, SPF_PASS=-0.001, URIBL_BLOCKED=0.001] autolearn=disabled Authentication-Results: spamd4-us-west.apache.org (amavisd-new); dkim=pass (1024-bit key) header.d=nuna.com Received: from mx1-eu-west.apache.org ([10.40.0.8]) by localhost (spamd4-us-west.apache.org [10.40.0.11]) (amavisd-new, port 10024) with ESMTP id LiWAn2W2NKim for ; Thu, 1 Oct 2015 22:28:47 +0000 (UTC) Received: from mail-wi0-f176.google.com (mail-wi0-f176.google.com [209.85.212.176]) by mx1-eu-west.apache.org (ASF Mail Server at mx1-eu-west.apache.org) with ESMTPS id A8299201FB for ; Thu, 1 Oct 2015 22:28:46 +0000 (UTC) Received: by wiclk2 with SMTP id lk2so10515498wic.0 for ; Thu, 01 Oct 2015 15:28:46 -0700 (PDT) DKIM-Signature: v=1; a=rsa-sha256; c=relaxed/relaxed; d=nuna.com; s=nuna; h=mime-version:in-reply-to:references:date:message-id:subject:from:to :content-type; bh=ug8u25hczwQ++k9NzH/FyYAQHEJQOx1XZ+nc9B+yF5g=; b=kONkqiT/zyMF5vIbN149dGSwCvIpukpXEIxyydDfksr6h+ex/pkky50LWxBT5a1I+e EG4X0Xpgr9PpXFg8dBUreNJ8uk9+v5UtnfXt3RaTHApVFH+d/j9ZqVUR4LBu5QCVdIJd UFRBmgiuQO9mx+SlnigIly4cBNAzGCA2Im9lg= X-Google-DKIM-Signature: v=1; a=rsa-sha256; c=relaxed/relaxed; d=1e100.net; s=20130820; h=x-gm-message-state:mime-version:in-reply-to:references:date :message-id:subject:from:to:content-type; bh=ug8u25hczwQ++k9NzH/FyYAQHEJQOx1XZ+nc9B+yF5g=; b=FWLYpKvxz9FZZMyDjc/XEizgkzmIy7t5LoB/r8KJmyJbTsmKzuJ/ZPC1gWi3Mgxcrr SK4clJlZgy/tgWI97xVDlPiyeVXHqN4MVFTUAFYsVFRO2rHa/B+tSheKNtX749Bt4ryz KcshSNXRcLUmXEryAx3oWyPlSDEed0906V0VfmQMN76gxEz3paxV5Xe/GtahcNfT8K1O BKkzJGdZ7FNHIKKY3lgFIFNOmApktP9OyX1pgHSz2uI9mviTyccRh98PvxbKKWU6BXww kZOs/BunsBLYWWl3uMIdFx8HvpHNWmThlFA16UPRdLPK0jMP8Ie17J2ACqJFS8JLxHIm 5tPg== X-Gm-Message-State: ALoCoQlbwaEhFmyyEAwChr3XfuDwnqXP6Zv9HGXIN470MRxtk7Hru7WboOsIbLiLaYgk/eL1ZB6CcnPiMKqfp0HKYMH+FpjpvNCfFlykRwtqoW//hcUM5Y9xLF/AydheZZn1WNczr1uE MIME-Version: 1.0 X-Received: by 10.194.79.196 with SMTP id l4mr14062421wjx.93.1443738526282; Thu, 01 Oct 2015 15:28:46 -0700 (PDT) Received: by 10.194.18.200 with HTTP; Thu, 1 Oct 2015 15:28:46 -0700 (PDT) In-Reply-To: References: Date: Thu, 1 Oct 2015 15:28:46 -0700 Message-ID: Subject: Re: Preventing Cleanup of PCollections From: Everett Anderson To: user@crunch.apache.org Content-Type: multipart/alternative; boundary=047d7beb9f6e021307052112923c --047d7beb9f6e021307052112923c Content-Type: text/plain; charset=UTF-8 (Context: This is related to the 'LeaseExpiredExceptions and temp side effect files' thread.) In particular, the workaround would mean that we'd keep using the same PCollection/PTable references after a call to run()/cleanup(), which feels weird. Example: PTable liveTable = ... liveTable = liveTable.parallelDo(...) // Write the table somewhere we know won't get cleaned up, // which changes its internal Target. liveTable.write(To.sequenceFile(tempPath), Target.WriteMode.CHECKPOINT); // Call run() and cleanup() to flush old temporary data. pipeline.run(); pipeline.cleanup(false); // Keep using liveTable since we know it'll work under the // covers because its Target is a sequence file that wasn't // cleaned up. liveTable = liveTable.parallelDo(...) On Thu, Oct 1, 2015 at 10:54 AM, Jeff Quinn wrote: > Hello, > > Our crunch pipeline has suffered from ballooning HDFS usage which spikes > during the course of the job. Our solution has been to call Pipeline.run() > and Pipeline.cleanup() between the major operations, hoping to achieve > periodic "garbage collection" of the temporary outputs that are produced > during the course of the pipeline. > > The problem is some PCollections from one operation will need to be used > as input to subsequent operations, and cleanup() seems to blow away ALL > PCollections that have not been explicitly written to a target (from > reading the source, it seems to just blow away the pipeline temp directory). > > Our workaround has been to explicitly call .write on the PCollections we > know we will need across calls to run()/cleanup(). This seems to work as > far as I can tell, but it feels hacky. Is there a better or more supported > way to handle this, and is this approach likely to fail in future crunch > versions? > > Thanks! > > Jeff > > *DISCLAIMER:* The contents of this email, including any attachments, may > contain information that is confidential, proprietary in nature, protected > health information (PHI), or otherwise protected by law from disclosure, > and is solely for the use of the intended recipient(s). If you are not the > intended recipient, you are hereby notified that any use, disclosure or > copying of this email, including any attachments, is unauthorized and > strictly prohibited. If you have received this email in error, please > notify the sender of this email. Please delete this and all copies of this > email from your system. Any opinions either expressed or implied in this > email and all attachments, are those of its author only, and do not > necessarily reflect those of Nuna Health, Inc. -- *DISCLAIMER:* The contents of this email, including any attachments, may contain information that is confidential, proprietary in nature, protected health information (PHI), or otherwise protected by law from disclosure, and is solely for the use of the intended recipient(s). If you are not the intended recipient, you are hereby notified that any use, disclosure or copying of this email, including any attachments, is unauthorized and strictly prohibited. If you have received this email in error, please notify the sender of this email. Please delete this and all copies of this email from your system. Any opinions either expressed or implied in this email and all attachments, are those of its author only, and do not necessarily reflect those of Nuna Health, Inc. --047d7beb9f6e021307052112923c Content-Type: text/html; charset=UTF-8 Content-Transfer-Encoding: quoted-printable
(Context: This is related to the 'LeaseExpiredExceptio= ns and temp side effect files' thread.)

In particula= r, the workaround would mean that we'd keep using the same PCollection/= PTable references after a call to run()/cleanup(), which feels weird.
=

Example:

PTable liveTable =3D ...
liveTable =3D liveTable.parallelDo(...)
=
// Write the table somewhere we know won't get clean= ed up,
// which change= s its internal Target.
liveTable.write(To.sequenceFile(tempPath),=C2=A0
=C2=A0 =C2=A0 =C2=A0 =C2=A0 =C2=A0 =C2=A0 =C2= =A0 =C2=A0 Target.WriteMode.CHECKPOINT);

pipeline.run();
pipeline.cleanup(false);

// Keep using liveTable since we know it'll work under= the
// covers because= its Target is a sequence file that wasn't
// cleaned up.
liveTable =3D liveTable.parallelDo(...)








<= br>
On Thu, Oct 1, 2015 at 10:54 AM, Jeff Quinn <= span dir=3D"ltr"><jef= f@nuna.com> wrote:
Hello,

Our crunch pipeline has suffered from ba= llooning HDFS usage which spikes during the course of the job. Our solution= has been to call Pipeline.run() and Pipeline.cleanup() between the major o= perations, hoping to achieve periodic "garbage collection" of the= temporary outputs that are produced during the course of the pipeline.

The problem is some PCollections from one operation w= ill need to be used as input to subsequent operations, and cleanup() seems = to blow away ALL PCollections that have not been explicitly written to a ta= rget (from reading the source, it seems to just blow away the pipeline temp= directory).

Our workaround has been to explicitly= call .write on the PCollections we know we will need across calls to run()= /cleanup(). This seems to work as far as I can tell, but it feels hacky. Is= there a better or more supported way to handle this, and is this approach = likely to fail in future crunch versions?

Thanks!<= /div>

Jeff

DISCLAIMER:=C2=A0The conten= ts of this email, including any attachments, may contain information that i= s confidential, proprietary in nature, protected health information (PHI), = or otherwise protected by law from disclosure, and is solely for the use of= the intended recipient(s). If you are not the intended recipient, you are = hereby notified that any use, disclosure or copying of this email, includin= g any attachments, is unauthorized and strictly prohibited. If you have rec= eived this email in error, please notify the sender of this email. Please d= elete this and all copies of this email from your system. Any opinions eith= er expressed or implied in this email and all attachments, are those of its= author only, and do not necessarily reflect those of Nuna Health, Inc.


DISCLAIMER:=C2=A0The conten= ts of this email, including any attachments, may contain information that i= s confidential, proprietary in nature, protected health information (PHI), = or otherwise protected by law from disclosure, and is solely for the use of= the intended recipient(s). If you are not the intended recipient, you are = hereby notified that any use, disclosure or copying of this email, includin= g any attachments, is unauthorized and strictly prohibited. If you have rec= eived this email in error, please notify the sender of this email. Please d= elete this and all copies of this email from your system. Any opinions eith= er expressed or implied in this email and all attachments, are those of its= author only, and do not necessarily reflect those of Nuna Health, Inc. --047d7beb9f6e021307052112923c--