U may want to use FlinkMLTools.persist() methods which use TypeSerializerFormat and don't enforce IOReadableWritable.
On Tue, Mar 29, 2016 at 2:12 PM, Sourigna Phetsarath < gna.phetsar...@teamaol.com> wrote: > Till, > > Thank you for your reply. > > Having this issue though, WeightVector does not extend IOReadWriteable: > > *public* *class* SerializedOutputFormat<*T* *extends* IOReadableWritable> > > *case* *class* WeightVector(weights: Vector, intercept: Double) *extends* > Serializable {} > > > However, I will use the approach to write out the weights as text. > > > On Tue, Mar 29, 2016 at 5:01 AM, Till Rohrmann <trohrm...@apache.org> > wrote: > >> Hi Gna, >> >> there are no utilities yet to do that but you can do it manually. In the >> end, a model is simply a Flink DataSet which you can serialize to some >> file. Upon reading this DataSet you simply have to give it to your >> algorithm to be used as the model. The following code snippet illustrates >> this approach: >> >> mlr.fit(inputDS, parameters) >> >> // write model to disk using the SerializedOutputFormat >> mlr.weightsOption.get.write(new SerializedOutputFormat[WeightVector], "path") >> >> // read the serialized model from disk >> val model = env.readFile(new SerializedInputFormat[WeightVector], "path") >> >> // set the read model for the MLR algorithm >> mlr.weightsOption = model >> >> Cheers, >> Till >> >> >> On Tue, Mar 29, 2016 at 10:46 AM, Simone Robutti < >> simone.robu...@radicalbit.io> wrote: >> >>> To my knowledge there is nothing like that. PMML is not supported in any >>> form and there's no custom saving format yet. If you really need a quick >>> and dirty solution, it's not that hard to serialize the model into a file. >>> >>> 2016-03-28 17:59 GMT+02:00 Sourigna Phetsarath < >>> gna.phetsar...@teamaol.com>: >>> >>>> Flinksters, >>>> >>>> Is there an example of saving a Trained Model, loading a Trained Model >>>> and then scoring one or more feature vectors using Flink ML? >>>> >>>> All of the examples I've seen have shown only sequential fit and >>>> predict. >>>> >>>> Thank you. >>>> >>>> -Gna >>>> -- >>>> >>>> >>>> *Gna Phetsarath*System Architect // AOL Platforms // Data Services // >>>> Applied Research Chapter >>>> 770 Broadway, 5th Floor, New York, NY 10003 >>>> o: 212.402.4871 // m: 917.373.7363 >>>> vvmr: 8890237 aim: sphetsarath20 t: @sourigna >>>> >>>> * <http://www.aolplatforms.com>* >>>> >>> >>> >> > > > -- > > > *Gna Phetsarath*System Architect // AOL Platforms // Data Services // > Applied Research Chapter > 770 Broadway, 5th Floor, New York, NY 10003 > o: 212.402.4871 // m: 917.373.7363 > vvmr: 8890237 aim: sphetsarath20 t: @sourigna > > * <http://www.aolplatforms.com>* >