You are viewing a plain text version of this content. The canonical link for it is here.
Posted to user@spark.apache.org by "Ganelin, Ilya" <Il...@capitalone.com> on 2014/12/22 23:36:10 UTC

Long-running job cleanup

Hi all, I have a long running job iterating over a huge dataset. Parts of this operation are cached. Since the job runs for so long, eventually the overhead of spark shuffles starts to accumulate culminating in the driver starting to swap.

I am aware of the spark.cleanup.tll parameter that allows me to configure when cleanup happens but the issue with doing this is that it isn’t done safely, e.g. I can be in the middle of processing a stage when this cleanup happens and my cached RDDs get cleared. This ultimately causes a KeyNotFoundException when I try to reference the now cleared cached RDD. This behavior doesn’t make much sense to me, I would expect the cached RDD to either get regenerated or at the very least for there to be an option to execute this cleanup without deleting those RDDs.

Is there a programmatically safe way of doing this cleanup that doesn’t break everything?

If I instead tear down the spark context and bring up a new context for every iteration (assuming that each iteration is sufficiently long-lived), would memory get released appropriately?
________________________________________________________

The information contained in this e-mail is confidential and/or proprietary to Capital One and/or its affiliates. The information transmitted herewith is intended only for use by the individual or entity to which it is addressed.  If the reader of this message is not the intended recipient, you are hereby notified that any review, retransmission, dissemination, distribution, copying or other use of, or taking of any action in reliance upon this information is strictly prohibited. If you have received this communication in error, please contact the sender and delete the material from your computer.

Re: Long-running job cleanup

Posted by "Ganelin, Ilya" <Il...@capitalone.com>.
The previously submitted code doesn’t actually show the problem I was trying to show effectively since the issue becomes clear between subsequent steps. Within a single step it appears things are cleared up properly.  Memory usage becomes evident pretty quickly.

def showMemoryUsage(sc: SparkContext) = {
  val usersPerStep = 2500
  val count = 1000000
  val numSteps = count / usersPerStep

  val users = sc.parallelize(1 to count)
  val zippedUsers = users.zipWithIndex().cache()
  val userFeatures: RDD[(Int, Int)] = sc.parallelize(1 to count).map(s => (s, 2)).partitionBy(new HashPartitioner(200)).cache()
  val productFeatures: RDD[(Int, Int)] = sc.parallelize(1 to 1000000).map(s => (s, 4)).repartition(1).cache()

  for (i <- 1 to numSteps) {
    val usersFiltered = zippedUsers.filter(s => {
      ((i - 1) * usersPerStep <= s._2) && (s._2 < i * usersPerStep)
    }).map(_._1).collect()

    val results = usersFiltered.map(user => {
      val userScore = userFeatures.lookup(user).head
      val recPerUser = Array(1,2,userScore)
      recPerUser
    })

    val mapedResults: Array[Int] = results.flatMap(scores => scores).toArray
    log("State: Computed " + mapedResults.length + " predictions for stage " + i)

    sc.parallelize(mapedResults)
    // Write to disk (left out since problem is evident even without it)
  }
}

Example broadcast variable added:

14/12/30 19:25:19 INFO BlockManagerInfo: Added broadcast_0piece0 in memory on CLIENT_NODE:54640 (size: 794.0 B, free: 441.9 MB)

And then if I parse the entire log looking for “free : XXX.X MB” within a single step memory is cleared properly:

Free 441.1 MB
Free 439.8 MB
Free 439.8 MB
Free 441.1 MB
Free 441.1 MB
Free 439.8 MB

But between steps, the amount of available memory decreases (e.g. That range that things oscillate between shrinks) and over the course of many hours this eventually reduces to zero.

Free 440.7 MB
Free 438.7 MB
Free 438.7 MB
Free 440.7 MB

Free 435.4 MB
Free 425.0 MB
Free 425.0 MB
Free 435.4 MB
Free 425.0 MB
Free 425.0 MB
Free 435.4 MB

Free 426.7 MB
Free 402.5 MB
Free 402.5 MB
Free 426.7 MB
Free 426.7 MB
Free 402.5 MB
Free 402.5 MB
Free 426.7 MB
From: <Ganelin>, "Ganelin, Ilya" <il...@capitalone.com>>
Date: Tuesday, December 30, 2014 at 7:30 PM
To: Ilya Ganelin <il...@gmail.com>>, Patrick Wendell <pw...@gmail.com>>
Cc: "user@spark.apache.org<ma...@spark.apache.org>" <us...@spark.apache.org>>
Subject: Re: Long-running job cleanup

Hi Patrick, to follow up on the below discussion, I am including a short code snippet that produces the problem on 1.1. This is kind of stupid code since it’s a greatly simplified version of what I’m actually doing but it has a number of the key components in place. I’m also including some example log output. Thank you.


def showMemoryUsage(sc : SparkContext) = {

  val usersPerStep = 25000
  val count = 1000000
  val numSteps = count/usersPerStep

  val users = sc.parallelize(1 to count)
  val zippedUsers = users.zipWithIndex().cache()
  val userFeatures : RDD[(Int, Int)] = sc.parallelize(1 to count).map(s=>(s,2)).cache()
  val productFeatures : RDD[(Int, Int)] = sc.parallelize(1 to 50000000)
    .map(s => (s, 4)).cache()

  for (i <- 1 to numSteps) {
    val usersFiltered = zippedUsers.filter(s => {
      ((i - 1) * usersPerStep <= s._2) && (s._2 < i * usersPerStep)
    }).map(_._1).collect()

    usersFiltered.foreach(user => {
      val mult = productFeatures.map(s => s._2 * userFeatures.lookup(user).head)
      mult.takeOrdered(20)

      // Normally this would then be written to disk
      // For the sake of the example this is all we're doing
    })
  }
}

Example broadcast variable added:

14/12/30 19:25:19 INFO BlockManagerInfo: Added broadcast_0_piece0 in memory on innovationclientnode01.cof.ds.capitalone.com:54640 (size: 794.0 B, free: 441.9 MB)


And then if I parse the entire log looking for “free : XXX.X MB” I see the available memory slowly ticking away:

Free 441.8 MB

Free 441.8 MB

Free 441.8 MB

Free 441.8 MB

Free 441.8 MB

Free 441.8 MB

Free 441.8 MB

…

Free 441.7 MB

Free 441.7 MB

Free 441.7 MB

Free 441.7 MB

And so on.


Clearly the above code is not persisting the intermediate RDD (mult), yet memory is never being properly freed up.

From: Ilya Ganelin <il...@gmail.com>>
Date: Sunday, December 28, 2014 at 4:02 PM
To: Patrick Wendell <pw...@gmail.com>>, "Ganelin, Ilya" <il...@capitalone.com>>
Cc: "user@spark.apache.org<ma...@spark.apache.org>" <us...@spark.apache.org>>
Subject: Re: Long-running job cleanup

Hi Patrick - is that cleanup present in 1.1?

The overhead I am talking about is with regards to what I believe is shuffle related metadata. If I watch the execution log I see small broadcast variables created for every stage of execution, a few KB at a time, and a certain number of MB remaining of available memory on the driver. As I run, this available memory goes down, and these variables are never erased. The only RDDs that persist are those that are explicitly cached. The RDDs that are generated iteratively are not retained or referenced, so I would expect things to get cleaned up but they do not. The items consuming memory are not RDDs but what appears to be shuffle metadata.

I have a script that parses the logs to show memory consumption over time and the script shows memory very steadily being consumed over many hours without clearing one small bit at a time.

The specific computation I am doing is the generation of dot products between two RDDs of vectors. I need to generate this product for every combination of products between the two RDDs but both RDDs are too big to fit in memory. Consequently, I iteratively generate this product across one entry from the first RDD and all entries from the second and retain the pared-down result within an accumulator (by retaining the top N results it is possible to actually store the Cartesian which is otherwise too large to fit on disk). After a certain number of iterations these intermediate results are then written to disk. Each of these steps is tractable in itself but due to the accumulation of memory, the overall job becomes intractable.

I would appreciate any suggestions as to how to clean up these intermediate broadcast variables. Thank you.


On Sun, Dec 28, 2014 at 1:56 PM Patrick Wendell <pw...@gmail.com>> wrote:
What do you mean when you say "the overhead of spark shuffles start to
accumulate"? Could you elaborate more?

In newer versions of Spark shuffle data is cleaned up automatically
when an RDD goes out of scope. It is safe to remove shuffle data at
this point because the RDD can no longer be referenced. If you are
seeing a large build up of shuffle data, it's possible you are
retaining references to older RDDs inadvertently. Could you explain
what your job actually doing?

- Patrick

On Mon, Dec 22, 2014 at 2:36 PM, Ganelin, Ilya
<Il...@capitalone.com>> wrote:
> Hi all, I have a long running job iterating over a huge dataset. Parts of
> this operation are cached. Since the job runs for so long, eventually the
> overhead of spark shuffles starts to accumulate culminating in the driver
> starting to swap.
>
> I am aware of the spark.cleanup.tll parameter that allows me to configure
> when cleanup happens but the issue with doing this is that it isn't done
> safely, e.g. I can be in the middle of processing a stage when this cleanup
> happens and my cached RDDs get cleared. This ultimately causes a
> KeyNotFoundException when I try to reference the now cleared cached RDD.
> This behavior doesn't make much sense to me, I would expect the cached RDD
> to either get regenerated or at the very least for there to be an option to
> execute this cleanup without deleting those RDDs.
>
> Is there a programmatically safe way of doing this cleanup that doesn't
> break everything?
>
> If I instead tear down the spark context and bring up a new context for
> every iteration (assuming that each iteration is sufficiently long-lived),
> would memory get released appropriately?
>
> ________________________________
>
> The information contained in this e-mail is confidential and/or proprietary
> to Capital One and/or its affiliates. The information transmitted herewith
> is intended only for use by the individual or entity to which it is
> addressed.  If the reader of this message is not the intended recipient, you
> are hereby notified that any review, retransmission, dissemination,
> distribution, copying or other use of, or taking of any action in reliance
> upon this information is strictly prohibited. If you have received this
> communication in error, please contact the sender and delete the material
> from your computer.

---------------------------------------------------------------------
To unsubscribe, e-mail: user-unsubscribe@spark.apache.org<ma...@spark.apache.org>
For additional commands, e-mail: user-help@spark.apache.org<ma...@spark.apache.org>


________________________________

The information contained in this e-mail is confidential and/or proprietary to Capital One and/or its affiliates. The information transmitted herewith is intended only for use by the individual or entity to which it is addressed.  If the reader of this message is not the intended recipient, you are hereby notified that any review, retransmission, dissemination, distribution, copying or other use of, or taking of any action in reliance upon this information is strictly prohibited. If you have received this communication in error, please contact the sender and delete the material from your computer.
________________________________________________________

The information contained in this e-mail is confidential and/or proprietary to Capital One and/or its affiliates. The information transmitted herewith is intended only for use by the individual or entity to which it is addressed.  If the reader of this message is not the intended recipient, you are hereby notified that any review, retransmission, dissemination, distribution, copying or other use of, or taking of any action in reliance upon this information is strictly prohibited. If you have received this communication in error, please contact the sender and delete the material from your computer.

Re: Long-running job cleanup

Posted by "Ganelin, Ilya" <Il...@capitalone.com>.
Hi Patrick, to follow up on the below discussion, I am including a short code snippet that produces the problem on 1.1. This is kind of stupid code since it’s a greatly simplified version of what I’m actually doing but it has a number of the key components in place. I’m also including some example log output. Thank you.


def showMemoryUsage(sc : SparkContext) = {

  val usersPerStep = 25000
  val count = 1000000
  val numSteps = count/usersPerStep

  val users = sc.parallelize(1 to count)
  val zippedUsers = users.zipWithIndex().cache()
  val userFeatures : RDD[(Int, Int)] = sc.parallelize(1 to count).map(s=>(s,2)).cache()
  val productFeatures : RDD[(Int, Int)] = sc.parallelize(1 to 50000000)
    .map(s => (s, 4)).cache()

  for (i <- 1 to numSteps) {
    val usersFiltered = zippedUsers.filter(s => {
      ((i - 1) * usersPerStep <= s._2) && (s._2 < i * usersPerStep)
    }).map(_._1).collect()

    usersFiltered.foreach(user => {
      val mult = productFeatures.map(s => s._2 * userFeatures.lookup(user).head)
      mult.takeOrdered(20)

      // Normally this would then be written to disk
      // For the sake of the example this is all we're doing
    })
  }
}

Example broadcast variable added:

14/12/30 19:25:19 INFO BlockManagerInfo: Added broadcast_0_piece0 in memory on innovationclientnode01.cof.ds.capitalone.com:54640 (size: 794.0 B, free: 441.9 MB)


And then if I parse the entire log looking for “free : XXX.X MB” I see the available memory slowly ticking away:

Free 441.8 MB

Free 441.8 MB

Free 441.8 MB

Free 441.8 MB

Free 441.8 MB

Free 441.8 MB

Free 441.8 MB

…

Free 441.7 MB

Free 441.7 MB

Free 441.7 MB

Free 441.7 MB

And so on.


Clearly the above code is not persisting the intermediate RDD (mult), yet memory is never being properly freed up.

From: Ilya Ganelin <il...@gmail.com>>
Date: Sunday, December 28, 2014 at 4:02 PM
To: Patrick Wendell <pw...@gmail.com>>, "Ganelin, Ilya" <il...@capitalone.com>>
Cc: "user@spark.apache.org<ma...@spark.apache.org>" <us...@spark.apache.org>>
Subject: Re: Long-running job cleanup

Hi Patrick - is that cleanup present in 1.1?

The overhead I am talking about is with regards to what I believe is shuffle related metadata. If I watch the execution log I see small broadcast variables created for every stage of execution, a few KB at a time, and a certain number of MB remaining of available memory on the driver. As I run, this available memory goes down, and these variables are never erased. The only RDDs that persist are those that are explicitly cached. The RDDs that are generated iteratively are not retained or referenced, so I would expect things to get cleaned up but they do not. The items consuming memory are not RDDs but what appears to be shuffle metadata.

I have a script that parses the logs to show memory consumption over time and the script shows memory very steadily being consumed over many hours without clearing one small bit at a time.

The specific computation I am doing is the generation of dot products between two RDDs of vectors. I need to generate this product for every combination of products between the two RDDs but both RDDs are too big to fit in memory. Consequently, I iteratively generate this product across one entry from the first RDD and all entries from the second and retain the pared-down result within an accumulator (by retaining the top N results it is possible to actually store the Cartesian which is otherwise too large to fit on disk). After a certain number of iterations these intermediate results are then written to disk. Each of these steps is tractable in itself but due to the accumulation of memory, the overall job becomes intractable.

I would appreciate any suggestions as to how to clean up these intermediate broadcast variables. Thank you.


On Sun, Dec 28, 2014 at 1:56 PM Patrick Wendell <pw...@gmail.com>> wrote:
What do you mean when you say "the overhead of spark shuffles start to
accumulate"? Could you elaborate more?

In newer versions of Spark shuffle data is cleaned up automatically
when an RDD goes out of scope. It is safe to remove shuffle data at
this point because the RDD can no longer be referenced. If you are
seeing a large build up of shuffle data, it's possible you are
retaining references to older RDDs inadvertently. Could you explain
what your job actually doing?

- Patrick

On Mon, Dec 22, 2014 at 2:36 PM, Ganelin, Ilya
<Il...@capitalone.com>> wrote:
> Hi all, I have a long running job iterating over a huge dataset. Parts of
> this operation are cached. Since the job runs for so long, eventually the
> overhead of spark shuffles starts to accumulate culminating in the driver
> starting to swap.
>
> I am aware of the spark.cleanup.tll parameter that allows me to configure
> when cleanup happens but the issue with doing this is that it isn't done
> safely, e.g. I can be in the middle of processing a stage when this cleanup
> happens and my cached RDDs get cleared. This ultimately causes a
> KeyNotFoundException when I try to reference the now cleared cached RDD.
> This behavior doesn't make much sense to me, I would expect the cached RDD
> to either get regenerated or at the very least for there to be an option to
> execute this cleanup without deleting those RDDs.
>
> Is there a programmatically safe way of doing this cleanup that doesn't
> break everything?
>
> If I instead tear down the spark context and bring up a new context for
> every iteration (assuming that each iteration is sufficiently long-lived),
> would memory get released appropriately?
>
> ________________________________
>
> The information contained in this e-mail is confidential and/or proprietary
> to Capital One and/or its affiliates. The information transmitted herewith
> is intended only for use by the individual or entity to which it is
> addressed.  If the reader of this message is not the intended recipient, you
> are hereby notified that any review, retransmission, dissemination,
> distribution, copying or other use of, or taking of any action in reliance
> upon this information is strictly prohibited. If you have received this
> communication in error, please contact the sender and delete the material
> from your computer.

---------------------------------------------------------------------
To unsubscribe, e-mail: user-unsubscribe@spark.apache.org<ma...@spark.apache.org>
For additional commands, e-mail: user-help@spark.apache.org<ma...@spark.apache.org>

________________________________________________________

The information contained in this e-mail is confidential and/or proprietary to Capital One and/or its affiliates. The information transmitted herewith is intended only for use by the individual or entity to which it is addressed.  If the reader of this message is not the intended recipient, you are hereby notified that any review, retransmission, dissemination, distribution, copying or other use of, or taking of any action in reliance upon this information is strictly prohibited. If you have received this communication in error, please contact the sender and delete the material from your computer.

Re: Long-running job cleanup

Posted by Ilya Ganelin <il...@gmail.com>.
Hi Patrick - is that cleanup present in 1.1?

The overhead I am talking about is with regards to what I believe is
shuffle related metadata. If I watch the execution log I see small
broadcast variables created for every stage of execution, a few KB at a
time, and a certain number of MB remaining of available memory on the
driver. As I run, this available memory goes down, and these variables are
never erased. The only RDDs that persist are those that are explicitly
cached. The RDDs that are generated iteratively are not retained or
referenced, so I would expect things to get cleaned up but they do not. The
items consuming memory are not RDDs but what appears to be shuffle
metadata.

I have a script that parses the logs to show memory consumption over time
and the script shows memory very steadily being consumed over many hours
without clearing one small bit at a time.

The specific computation I am doing is the generation of dot products
between two RDDs of vectors. I need to generate this product for every
combination of products between the two RDDs but both RDDs are too big to
fit in memory. Consequently, I iteratively generate this product across one
entry from the first RDD and all entries from the second and retain the
pared-down result within an accumulator (by retaining the top N results it
is possible to actually store the Cartesian which is otherwise too large to
fit on disk). After a certain number of iterations these intermediate
results are then written to disk. Each of these steps is tractable in
itself but due to the accumulation of memory, the overall job becomes
intractable.

I would appreciate any suggestions as to how to clean up these intermediate
broadcast variables. Thank you.


On Sun, Dec 28, 2014 at 1:56 PM Patrick Wendell <pw...@gmail.com> wrote:

> What do you mean when you say "the overhead of spark shuffles start to
> accumulate"? Could you elaborate more?
>
> In newer versions of Spark shuffle data is cleaned up automatically
> when an RDD goes out of scope. It is safe to remove shuffle data at
> this point because the RDD can no longer be referenced. If you are
> seeing a large build up of shuffle data, it's possible you are
> retaining references to older RDDs inadvertently. Could you explain
> what your job actually doing?
>
> - Patrick
>
> On Mon, Dec 22, 2014 at 2:36 PM, Ganelin, Ilya
> <Il...@capitalone.com> wrote:
> > Hi all, I have a long running job iterating over a huge dataset. Parts of
> > this operation are cached. Since the job runs for so long, eventually the
> > overhead of spark shuffles starts to accumulate culminating in the driver
> > starting to swap.
> >
> > I am aware of the spark.cleanup.tll parameter that allows me to configure
> > when cleanup happens but the issue with doing this is that it isn't done
> > safely, e.g. I can be in the middle of processing a stage when this
> cleanup
> > happens and my cached RDDs get cleared. This ultimately causes a
> > KeyNotFoundException when I try to reference the now cleared cached RDD.
> > This behavior doesn't make much sense to me, I would expect the cached
> RDD
> > to either get regenerated or at the very least for there to be an option
> to
> > execute this cleanup without deleting those RDDs.
> >
> > Is there a programmatically safe way of doing this cleanup that doesn't
> > break everything?
> >
> > If I instead tear down the spark context and bring up a new context for
> > every iteration (assuming that each iteration is sufficiently
> long-lived),
> > would memory get released appropriately?
> >
> > ________________________________
> >
> > The information contained in this e-mail is confidential and/or
> proprietary
> > to Capital One and/or its affiliates. The information transmitted
> herewith
> > is intended only for use by the individual or entity to which it is
> > addressed.  If the reader of this message is not the intended recipient,
> you
> > are hereby notified that any review, retransmission, dissemination,
> > distribution, copying or other use of, or taking of any action in
> reliance
> > upon this information is strictly prohibited. If you have received this
> > communication in error, please contact the sender and delete the material
> > from your computer.
>
> ---------------------------------------------------------------------
> To unsubscribe, e-mail: user-unsubscribe@spark.apache.org
> For additional commands, e-mail: user-help@spark.apache.org
>
>

Re: Long-running job cleanup

Posted by Patrick Wendell <pw...@gmail.com>.
What do you mean when you say "the overhead of spark shuffles start to
accumulate"? Could you elaborate more?

In newer versions of Spark shuffle data is cleaned up automatically
when an RDD goes out of scope. It is safe to remove shuffle data at
this point because the RDD can no longer be referenced. If you are
seeing a large build up of shuffle data, it's possible you are
retaining references to older RDDs inadvertently. Could you explain
what your job actually doing?

- Patrick

On Mon, Dec 22, 2014 at 2:36 PM, Ganelin, Ilya
<Il...@capitalone.com> wrote:
> Hi all, I have a long running job iterating over a huge dataset. Parts of
> this operation are cached. Since the job runs for so long, eventually the
> overhead of spark shuffles starts to accumulate culminating in the driver
> starting to swap.
>
> I am aware of the spark.cleanup.tll parameter that allows me to configure
> when cleanup happens but the issue with doing this is that it isn't done
> safely, e.g. I can be in the middle of processing a stage when this cleanup
> happens and my cached RDDs get cleared. This ultimately causes a
> KeyNotFoundException when I try to reference the now cleared cached RDD.
> This behavior doesn't make much sense to me, I would expect the cached RDD
> to either get regenerated or at the very least for there to be an option to
> execute this cleanup without deleting those RDDs.
>
> Is there a programmatically safe way of doing this cleanup that doesn't
> break everything?
>
> If I instead tear down the spark context and bring up a new context for
> every iteration (assuming that each iteration is sufficiently long-lived),
> would memory get released appropriately?
>
> ________________________________
>
> The information contained in this e-mail is confidential and/or proprietary
> to Capital One and/or its affiliates. The information transmitted herewith
> is intended only for use by the individual or entity to which it is
> addressed.  If the reader of this message is not the intended recipient, you
> are hereby notified that any review, retransmission, dissemination,
> distribution, copying or other use of, or taking of any action in reliance
> upon this information is strictly prohibited. If you have received this
> communication in error, please contact the sender and delete the material
> from your computer.

---------------------------------------------------------------------
To unsubscribe, e-mail: user-unsubscribe@spark.apache.org
For additional commands, e-mail: user-help@spark.apache.org


Re: Long-running job cleanup

Posted by Ilya Ganelin <il...@gmail.com>.
Hello all - can anyone please offer any advice on this issue?

-Ilya Ganelin

On Mon, Dec 22, 2014 at 5:36 PM, Ganelin, Ilya <Il...@capitalone.com>
wrote:

> Hi all, I have a long running job iterating over a huge dataset. Parts of
> this operation are cached. Since the job runs for so long, eventually the
> overhead of spark shuffles starts to accumulate culminating in the driver
> starting to swap.
>
> I am aware of the spark.cleanup.tll parameter that allows me to configure
> when cleanup happens but the issue with doing this is that it isn’t done
> safely, e.g. I can be in the middle of processing a stage when this cleanup
> happens and my cached RDDs get cleared. This ultimately causes a
> KeyNotFoundException when I try to reference the now cleared cached RDD.
> This behavior doesn’t make much sense to me, I would expect the cached RDD
> to either get regenerated or at the very least for there to be an option to
> execute this cleanup without deleting those RDDs.
>
> Is there a programmatically safe way of doing this cleanup that doesn’t
> break everything?
>
> If I instead tear down the spark context and bring up a new context for
> every iteration (assuming that each iteration is sufficiently long-lived),
> would memory get released appropriately?
>
> ------------------------------
>
> The information contained in this e-mail is confidential and/or
> proprietary to Capital One and/or its affiliates. The information
> transmitted herewith is intended only for use by the individual or entity
> to which it is addressed.  If the reader of this message is not the
> intended recipient, you are hereby notified that any review,
> retransmission, dissemination, distribution, copying or other use of, or
> taking of any action in reliance upon this information is strictly
> prohibited. If you have received this communication in error, please
> contact the sender and delete the material from your computer.
>