[GitHub] spark issue #19586: [SPARK-22367][WIP][CORE] Separate the serialization of c...

2017-10-30 Thread cloud-fan
Github user cloud-fan commented on the issue:

https://github.com/apache/spark/pull/19586
  
OK to test


---

-
To unsubscribe, e-mail: reviews-unsubscr...@spark.apache.org
For additional commands, e-mail: reviews-h...@spark.apache.org



[GitHub] spark issue #19586: [SPARK-22367][WIP][CORE] Separate the serialization of c...

2017-10-30 Thread ConeyLiu
Github user ConeyLiu commented on the issue:

https://github.com/apache/spark/pull/19586
  
Hi @cloud-fan, thanks for reviewing. There are some errors about 
`UnsafeShuffleWrite` need further fixed. I am not familiar with this code, so I 
need some time.


---

-
To unsubscribe, e-mail: reviews-unsubscr...@spark.apache.org
For additional commands, e-mail: reviews-h...@spark.apache.org



[GitHub] spark issue #19586: [SPARK-22367][WIP][CORE] Separate the serialization of c...

2017-10-31 Thread jerryshao
Github user jerryshao commented on the issue:

https://github.com/apache/spark/pull/19586
  
@ConeyLiu what about the below example, does your implementation support 
this?

```scala

trait Base { val name: String }
case class A(name: String) extends Base
case class B(name: String) extends Base

sc.parallelize(Seq(A("a"), B("b"))).map { i => (i, 1) }.reduceByKey(_ + 
_).collect()
```

Here not all the elements have same class type, does your PR here support 
such scenario?



---

-
To unsubscribe, e-mail: reviews-unsubscr...@spark.apache.org
For additional commands, e-mail: reviews-h...@spark.apache.org



[GitHub] spark issue #19586: [SPARK-22367][WIP][CORE] Separate the serialization of c...

2017-10-31 Thread ConeyLiu
Github user ConeyLiu commented on the issue:

https://github.com/apache/spark/pull/19586
  
Hi @jerryshao, Thanks for the reminder, it doesn't support it. I'm sorry I 
did not take that into account.  How about using configuration to determine 
whether we should use `SerializerInstance#serializeStreamForClass[T]`. For most 
case the data type should be same.

Can you give some advice? Also cc @cloud-fan @srowen 


---

-
To unsubscribe, e-mail: reviews-unsubscr...@spark.apache.org
For additional commands, e-mail: reviews-h...@spark.apache.org



[GitHub] spark issue #19586: [SPARK-22367][WIP][CORE] Separate the serialization of c...

2017-10-31 Thread jerryshao
Github user jerryshao commented on the issue:

https://github.com/apache/spark/pull/19586
  
Using configurations seems not so elegant, also configuration is 
application based, how would you turn off/on this feature in the runtime? Sorry 
I cannot give you a good advice, maybe kryo's solution is the best option for 
general case.  


---

-
To unsubscribe, e-mail: reviews-unsubscr...@spark.apache.org
For additional commands, e-mail: reviews-h...@spark.apache.org



[GitHub] spark issue #19586: [SPARK-22367][WIP][CORE] Separate the serialization of c...

2017-11-01 Thread ConeyLiu
Github user ConeyLiu commented on the issue:

https://github.com/apache/spark/pull/19586
  
Currently, I use it directly. Maybe this is suitable for some special case 
which has same type data, such as ml or else. 


---

-
To unsubscribe, e-mail: reviews-unsubscr...@spark.apache.org
For additional commands, e-mail: reviews-h...@spark.apache.org



[GitHub] spark issue #19586: [SPARK-22367][WIP][CORE] Separate the serialization of c...

2017-11-01 Thread cloud-fan
Github user cloud-fan commented on the issue:

https://github.com/apache/spark/pull/19586
  
For these cases, they can write their own serializer and set it via 
`spark.serializer`. I don't think Spark should have built-in support for them 
because it's not general.


---

-
To unsubscribe, e-mail: reviews-unsubscr...@spark.apache.org
For additional commands, e-mail: reviews-h...@spark.apache.org



[GitHub] spark issue #19586: [SPARK-22367][WIP][CORE] Separate the serialization of c...

2017-11-01 Thread ConeyLiu
Github user ConeyLiu commented on the issue:

https://github.com/apache/spark/pull/19586
  
Hi @cloud-fan, for most case the data type should be same. So I think this 
optimization is valuable, because it can save the space and cpu resource 
considerable. What about setting a flag for the RDD, which indicates whether 
the RDD only has the same types. If it'st not valid, could we putting it to the 
ml package for special serializer, then user could configure it. But for this 
case, there must be provided the exactly classtag of the RDD for serialization 
due to the relocation of unsafeshufflewrite.


---

-
To unsubscribe, e-mail: reviews-unsubscr...@spark.apache.org
For additional commands, e-mail: reviews-h...@spark.apache.org



[GitHub] spark issue #19586: [SPARK-22367][WIP][CORE] Separate the serialization of c...

2017-11-01 Thread jerryshao
Github user jerryshao commented on the issue:

https://github.com/apache/spark/pull/19586
  
I tend to agree with @cloud-fan , I think you can implement your own 
serializer out of Spark to be more specialized for your application, that will 
definitely be more efficient than the built-in one. But for the Spark's default 
solution, it should be general enough to cover all cases. Setting a flag or a 
configuration is not intuitive enough from my understanding.

And for ML, can you please provide an example about how this could be 
improved with your approach. From my understanding you approach is more useful 
when leverage custom class definition, like `Person` in your example. But for 
ML/SQL cases, all the types should be predefined or primitives, will that 
improved a lot?


---

-
To unsubscribe, e-mail: reviews-unsubscr...@spark.apache.org
For additional commands, e-mail: reviews-h...@spark.apache.org



[GitHub] spark issue #19586: [SPARK-22367][WIP][CORE] Separate the serialization of c...

2017-11-02 Thread ConeyLiu
Github user ConeyLiu commented on the issue:

https://github.com/apache/spark/pull/19586
  
OK, I can understand your concern. There is huge gc problem for K-means 
workload, it occupied about 10-20% percent. The source data is cached in 
memory, there is even worse performance when the source data can't be cached in 
memory. So we try the source data to off-heap. However, the training time even 
worse after using the off-heap memory.  Because the gc only occupied about 
10-20% with on-heap memory, while deserialization occupied about 30-40% with 
off-heap memory even if the gc problem solved.  
https://user-images.githubusercontent.com/12733256/32313752-5dbec220-bfdf-11e7-8b49-d5daa47cd50f.PNG";>

https://user-images.githubusercontent.com/12733256/32313788-824b8470-bfdf-11e7-9b59-aea26e9c6c0a.PNG";>

You can see the pic, the `readClass` occupied about 13% .  So I opened this 
pr.  With this path test result, the total time (loading data + training kmeans 
model) saved about 10% time.  The above picture is only about training phase, 
not include the loading source data phase,  so the improvement should be larger 
as we expected. And I plan to optimize the `readObjectOrNull` after this.

Also, I found the `Vector` is not registered, so I will test the 
performance with the registered vector. This maybe can reduce the cpu occupied, 
but can't save the serialized memory.
 


---

-
To unsubscribe, e-mail: reviews-unsubscr...@spark.apache.org
For additional commands, e-mail: reviews-h...@spark.apache.org



[GitHub] spark issue #19586: [SPARK-22367][WIP][CORE] Separate the serialization of c...

2017-11-02 Thread cloud-fan
Github user cloud-fan commented on the issue:

https://github.com/apache/spark/pull/19586
  
I think this problem will go away after mllib migrate to Spark SQL 
completely. For now I think we can make the serializer config job-wise and set 
this special serializer for ml jobs.


---

-
To unsubscribe, e-mail: reviews-unsubscr...@spark.apache.org
For additional commands, e-mail: reviews-h...@spark.apache.org



[GitHub] spark issue #19586: [SPARK-22367][WIP][CORE] Separate the serialization of c...

2017-11-03 Thread ConeyLiu
Github user ConeyLiu commented on the issue:

https://github.com/apache/spark/pull/19586
  
Hi @cloud-fan, @jerryshao. The problem of `writeClass` and `readClass` can 
be solved by register the class: Vector, DenseVector, SparseVector.  The follow 
is the test results:
```scala
val conf = new SparkConf().setAppName("Vector Register Test")
conf.registerKryoClasses(Array(classOf[Vector], classOf[DenseVector], 
classOf[SparseVector]))
val sc = new SparkContext(conf)

val sourceData = sc.sequenceFile[LongWritable, VectorWritable](args(0))
  .map { case (k, v) =>
val vector = v.get()
val tmpVector = new Array[Double](v.get().size())
for (i <- 0 until vector.size()) {
  tmpVector(i) = vector.get(i)
}
Vectors.dense(tmpVector)
  }

sourceData.persist(StorageLevel.OFF_HEAP)
var start = System.currentTimeMillis()
sourceData.count()
println("First: " + (System.currentTimeMillis() - start))
start = System.currentTimeMillis()
sourceData.count()
println("Second: " + (System.currentTimeMillis() - start))

sc.stop()
```


Results:
serialized size:  before 38.4GB after: 30.5GB
First time: before 93318msafter:  80708ms
Second time:  before: 5870msafter: 3382ms

Those classes are very common for ML,  and also `Matrix`, `DenseMatrix` and 
`SparseMatrix` too. I'm not sure whether we should register those classes in 
core directly,  because this could introduce extra jar dependency.  So could 
you give some advice? Or else we just remind in the ml doc?

The reason shoule be the problem of kryo, it  will write the full class 
name instead of the classID if the class is not registered.


---

-
To unsubscribe, e-mail: reviews-unsubscr...@spark.apache.org
For additional commands, e-mail: reviews-h...@spark.apache.org



[GitHub] spark issue #19586: [SPARK-22367][WIP][CORE] Separate the serialization of c...

2017-11-03 Thread cloud-fan
Github user cloud-fan commented on the issue:

https://github.com/apache/spark/pull/19586
  
You can call `SparkConf#registerKryoClasses` manually, maybe we can also 
register these ml classes automatically in `KryoSerializer.newKryo` via 
reflection.

cc @yanboliang @srowen 


---

-
To unsubscribe, e-mail: reviews-unsubscr...@spark.apache.org
For additional commands, e-mail: reviews-h...@spark.apache.org



[GitHub] spark issue #19586: [SPARK-22367][WIP][CORE] Separate the serialization of c...

2017-11-03 Thread jiangxb1987
Github user jiangxb1987 commented on the issue:

https://github.com/apache/spark/pull/19586
  
also cc @WeichenXu123 


---

-
To unsubscribe, e-mail: reviews-unsubscr...@spark.apache.org
For additional commands, e-mail: reviews-h...@spark.apache.org



[GitHub] spark issue #19586: [SPARK-22367][WIP][CORE] Separate the serialization of c...

2017-11-04 Thread WeichenXu123
Github user WeichenXu123 commented on the issue:

https://github.com/apache/spark/pull/19586
  
We can config the class to register by config 
`spark.kryo.classesToRegister`, does it need to add into spark code ?




---

-
To unsubscribe, e-mail: reviews-unsubscr...@spark.apache.org
For additional commands, e-mail: reviews-h...@spark.apache.org



[GitHub] spark issue #19586: [SPARK-22367][WIP][CORE] Separate the serialization of c...

2017-11-04 Thread WeichenXu123
Github user WeichenXu123 commented on the issue:

https://github.com/apache/spark/pull/19586
  
and in `ml`, if we want to register class before running algos, Some other 
classes like `LabeledPoint`, `Instance` also need registered. 
and there're some class temporary defined in some ml algos (when using RDD).
Maybe better way is to register classes automatically if possible. E.g. 
when using kyro, auto detecting which classes are used frequently, and register 
them?


---

-
To unsubscribe, e-mail: reviews-unsubscr...@spark.apache.org
For additional commands, e-mail: reviews-h...@spark.apache.org



[GitHub] spark issue #19586: [SPARK-22367][WIP][CORE] Separate the serialization of c...

2017-11-05 Thread ConeyLiu
Github user ConeyLiu commented on the issue:

https://github.com/apache/spark/pull/19586
  
Thanks for the suggestion, I re-raised a pr to solve this problem. Close it 
now.


---

-
To unsubscribe, e-mail: reviews-unsubscr...@spark.apache.org
For additional commands, e-mail: reviews-h...@spark.apache.org