IT数码 购物 网址 头条 软件 日历 阅读 图书馆
TxT小说阅读器
↓语音阅读,小说下载,古典文学↓
图片批量下载器
↓批量下载图片,美女图库↓
图片自动播放器
↓图片自动播放器↓
一键清除垃圾
↓轻轻一点,清除系统垃圾↓
开发: C++知识库 Java知识库 JavaScript Python PHP知识库 人工智能 区块链 大数据 移动开发 嵌入式 开发工具 数据结构与算法 开发测试 游戏开发 网络协议 系统运维
教程: HTML教程 CSS教程 JavaScript教程 Go语言教程 JQuery教程 VUE教程 VUE3教程 Bootstrap教程 SQL数据库教程 C语言教程 C++教程 Java教程 Python教程 Python3教程 C#教程
数码: 电脑 笔记本 显卡 显示器 固态硬盘 硬盘 耳机 手机 iphone vivo oppo 小米 华为 单反 装机 图拉丁
 
   -> 大数据 -> Flink中基于State的有状态计算开发方法 -> 正文阅读

[大数据]Flink中基于State的有状态计算开发方法

文:王东阳

前言

状态在Flink中叫作State,用来保存中间计算结果或者缓存数据。根据是否需要保存中间结果,分为无状态计算和有状态计算。对于流计算而言,事件持续不断地产生,如果每次计算都是相互独立的,不依赖于上下游的事件,则是无状态计算。如果计算需要依赖于之前或者后续的事件,则是有状态计算。

在Flink中根据数据集是否根据Key进行分区,将状态分为Keyed StateOperator State(Non-keyed State)两种类型 ,本文主要介绍Keyed State,后续文章会介绍Operator State

Keyed State 表示和key相关的一种State,只能用于KeydStream类型数据集对应的Functions和Operators之上。Keyed State是Operator State的特例,区别在于Keyed State事先按照key对数据集进行了分区,每个Key State仅对应一个Operator和Key的组合。Keyed State可以通过Key Groups进行管理,主要用于当算子并行度发生变化时,自动重新分布 Keyed State数据。在系统运行过程中,一个Keyed算子实例可能运行一个或者多个Key Groups的keys。

按照数据结构的不同,Flink中定义了多种Keyed State,具体如下:

  • ValueState<T> 即类型为T的单值状态。这个状态与对应的Key绑定,是最简单的状态。
  • ListState<T>即Key上的状态值为一个列表。
  • MapState<UK,UV> 定义与Key对应键值对的状态,用于维护具有key-value结构类型的状态数据。
  • ReducingState<T>这种State通过用户传入的ReduceFucntion,每次调用add(T)方法添加元素时,会调用ReduceFucntion,最后合并到一个单一的状态值。
  • AggregatingState<IN,OUT> 聚合State和ReducingState<T>非常类似,不同的是,这里聚合的类型可以是不同的元素类型,使用add(IN)来加入元素,并使用AggregateFunction函数计算聚合结果。

State开发实战

本章节将通过实际的项目代码演示不同数据结构在不同计算场景下的开发方法。

本文中项目 pom文件

ValueState

ValueState<T> 即类型为T的单值状态。这个状态与对应的Key绑定,是最简单的状态。可以通过update(T)方法更新状态值,通过T value()方法获取状态值。

由于ValueState<T> 是与Key对应单个值的状态,应用场景可以是例如统计user_id对应的交易次数,每次用户交易都会在count状态值上进行更新。

接下来我们将通过一个具体的实例代码展示如何利用ValueState去统计各个用户的交易次数。

ValueStateCountUser实现

通过定义ValueState<Integer>类型的countState存储用户已经访问的次数,并在每次收到新的元素时,对countState中的数据加1更新,同时把当前用户名和已经访问的总次数返回。在本样例代码中将会通过 RichMapFunction 来对流元素进行处理。

  • 在其中的 open(Configuration parameters) 方法中对 countState进行初始化。Flink提供了RuntimeContext用于获取状态数据,同时RuntimeContext提供了常用的Managed Keyd State的获取方式,可以通过创建相应的StateDescriptor并调用RuntimeContext方法来获取状态数据。例如获取ValueState可以调用ValueState[T] getState(ValueStateDescriptor[T])方法。
  • Tuple2<String, Integer> map(Tuple2<String, String> s)中,通过 countState.value() 获取用户之前的访问次数,然后对countState中的数据加1更新,同时把当前用户名和已经访问的总次数返回。
import org.apache.flink.api.common.functions.RichMapFunction;
import org.apache.flink.api.common.state.ValueState;
import org.apache.flink.api.common.state.ValueStateDescriptor;
import org.apache.flink.api.java.tuple.Tuple2;
import org.apache.flink.configuration.Configuration;

public class ValueStateCountUser extends RichMapFunction<Tuple2<String, String>, Tuple2<String, Integer>> {
  private ValueState<Integer> countState;

  public ValueStateCountUser() {}

  @Override
  public void open(Configuration parameters) throws Exception {
    // super.open(parameters);
    countState = getRuntimeContext().getState(new ValueStateDescriptor<Integer>("count", Integer.class));
  }

  @Override
  public Tuple2<String, Integer> map(Tuple2<String, String> s) throws Exception {
    Integer count = countState.value();
    if (count == null) {
      countState.update(1);
      return Tuple2.of(s.f0, 1);
    }

    countState.update(count + 1);
    return Tuple2.of(s.f0, count + 1);
  }

  @Override
  public void close() throws Exception {
    // super.close();
    System.out.println(String.format("finally %d", countState.value()));
  }
}

代码地址:ValueStateCountUser.java

验证代码

编写验证代码如下

  private static void valueStateUserCount() throws Exception {
    StreamExecutionEnvironment env = StreamExecutionEnvironment.getExecutionEnvironment();
    env.setParallelism(1);
    DataStream<Tuple2<String, String>> inputStream = env.fromElements(
        new Tuple2<>("zhangsan","aa"),
        new Tuple2<>("lisi","aa"),
        new Tuple2<>("zhangsan","aa"),
        new Tuple2<>("lisi","aa"),
        new Tuple2<>("wangwu","aa")
    );

    inputStream.keyBy(new KeySelector<Tuple2<String, String>, String>() {
          @Override
          public String getKey(Tuple2<String, String> integerLongTuple2) throws Exception {
            return integerLongTuple2.f0;
          }
        })
        .map(new ValueStateCountUser())
        .print();

    env.execute("StateCompute");
  }

代码地址:valueStateUserCount

程序运行得到输出如下

(zhangsan,1)
(lisi,1)
(zhangsan,2)
(lisi,2)
(wangwu,1)
finally 1

可以看到对于流数据中的每一个元素,根据用户名进行统计,实时显示该用户已经访问了多少次。

ListState

ListState<T>即Key上的状态值为一个列表。可以通过add(T)方法或者addAll(List[T])往列表中附加值;也可以通过Iterable get()方法返回一个Iterable<T>来遍历状态值;使用update(List[T])来更新元素。

由于ListState保存的是与Key对应元素列表的状态,状态中存放元素的List列表,应用场景可以是例如定义ListState存储用户经常访问的IP地址。

接下来我们将通过一个具体的实例代码展示如何利用ListState去统计各个用户访问过的IP地址列表。为了更广泛的展示State的应用场景,本样例代码中将会基于KeyedProcessFunction来处理流元素。

ListStateUserIP 实现

  • 声明ListState<String> ipState 用于记录用户访问过的ip地址
  • open(Configuration parameters)中利用 getRuntimeContext().getListState 对 ipState 进行初始化,getListState需要传入ListStateDescriptor类型的参数。
  • processElement中,将当前记录中的IP地址追究到 ipState中,然后从ipState 获取已经访问过的所有IP地址,和用户名一起放入到Collector
import com.google.common.collect.Lists;
import java.util.List;
import org.apache.flink.api.common.state.ListState;
import org.apache.flink.api.common.state.ListStateDescriptor;
import org.apache.flink.api.java.tuple.Tuple2;
import org.apache.flink.configuration.Configuration;
import org.apache.flink.streaming.api.functions.KeyedProcessFunction;
import org.apache.flink.util.Collector;

public class ListStateUserIpKeyedProcessFunction
extends KeyedProcessFunction<String, Tuple2<String,String>, Tuple2<String, List<String>>>{
  private ListState<String> ipState;

  @Override
  public void open(Configuration parameters) throws Exception {
    // super.open(parameters);
    ipState = getRuntimeContext().getListState(new ListStateDescriptor<String>("ipstate", String.class));
  }

  @Override
  public void processElement(
      Tuple2<String, String> value,
      KeyedProcessFunction<String, Tuple2<String, String>, Tuple2<String, List<String>>>.Context context,
      Collector<Tuple2<String, List<String>>> collector) throws Exception {

    ipState.add(value.f1);
    List<String> ips = Lists.newArrayList(ipState.get());
    collector.collect(Tuple2.of(value.f0, ips));
  }
}

代码地址:ListStateUserIpKeyedProcessFunction.java

验证代码

编写验证程序如下

  private static void listStateUserIP() throws Exception {
    StreamExecutionEnvironment env = StreamExecutionEnvironment.getExecutionEnvironment();
    env.setParallelism(1);
    DataStream<Tuple2<String, String>> inputStream = env.fromElements(
        new Tuple2<>("zhangsan","192.168.0.1"),
        new Tuple2<>("lisi","192.168.0.1"),
        new Tuple2<>("zhangsan","192.168.0.2"),
        new Tuple2<>("lisi","192.168.0.3"),
        new Tuple2<>("wangwu","192.168.0.1")
    );

    inputStream.keyBy(new KeySelector<Tuple2<String, String>, String>() {
          @Override
          public String getKey(Tuple2<String, String> integerLongTuple2) throws Exception {
            return integerLongTuple2.f0;
          }
        })
        .process(new ListStateUserIpKeyedProcessFunction())
        .print();

    env.execute("listStateUserIP");
  }
}

代码地址:listStateUserIP

执行结果打印如下

(zhangsan,[192.168.0.1])
(lisi,[192.168.0.1])
(zhangsan,[192.168.0.1, 192.168.0.2])
(lisi,[192.168.0.1, 192.168.0.3])
(wangwu,[192.168.0.1])

达到预期效果

MapState

MapState<UK,UV> 定义与Key对应键值对的状态,用于维护具有key-value结构类型的状态数据。 MapState<UK,UV> 使 用 Map 存 储 Key-Value 对 , MapState的方法和Java的Map的方法极为相似,所以上手相对容易。常用的有如下:

  • get()方法获取值
  • put(),putAll()方法更新值
  • remove()删除某个key
  • contains()判断是否存在某个key
  • isEmpty() 判断是否为空

和HashMap接口相似,MapState也可以通过entries()、keys()、values()获取对应的keys或values的集合 。

接下来我们通过一个具体的实例介绍如何通过ReducingState<T> 计算每个人的当前最高和最低体温。

实现KeyedProcessFunction

  • 定义 MapState<String, Integer> minMaxState 用于记录每个用户的最低低温和最高体温。
  • open方法中通过getRuntimeContext().getMapState初始化minMaxStategetMapState 需要传入MapStateDescriptor作为参数,构建MapStateDescriptor的第二个参数标识Map中Key的类型,第三个参数标识Map中Value的类型
  • processElement方法中,将输入的流元素也就是用户当前的体温记录和minMaxState中的最低温和最高温进行对比,并更新到minMaxState中。
import org.apache.flink.api.common.state.MapState;
import org.apache.flink.api.common.state.MapStateDescriptor;
import org.apache.flink.api.java.tuple.Tuple2;
import org.apache.flink.api.java.tuple.Tuple3;
import org.apache.flink.configuration.Configuration;
import org.apache.flink.streaming.api.functions.KeyedProcessFunction;
import org.apache.flink.util.Collector;

public class MapStateMinMaxTempKeyedProcessFunction
    extends KeyedProcessFunction<String, Tuple2<String, Integer>, Tuple3<String, Integer, Integer>> {

  private MapState<String, Integer> minMaxState;

  private final String minKey = "min";
  private final String maxKey = "max";

  @Override
  public void open(Configuration parameters) throws Exception {
    // super.open(parameters);
    minMaxState = getRuntimeContext().getMapState(
        new MapStateDescriptor<String, Integer>("minMaxState",
            String.class,
            Integer.class));
  }

  @Override
  public void processElement(
      Tuple2<String, Integer> in,
      KeyedProcessFunction<String, Tuple2<String, Integer>, Tuple3<String, Integer, Integer>>.Context context,
      Collector<Tuple3<String, Integer, Integer>> collector) throws Exception {

    if (!minMaxState.contains(minKey)) {
      minMaxState.put(minKey, in.f1);
    }

    if (!minMaxState.contains(maxKey)) {
      minMaxState.put(maxKey, in.f1);
    }

    if (in.f1 > minMaxState.get(maxKey)) {
      minMaxState.put(maxKey, in.f1);
    }

    if (in.f1 < minMaxState.get(minKey)) {
      minMaxState.put(minKey, in.f1);
    }

    collector.collect(Tuple3.of(in.f0, minMaxState.get(minKey), minMaxState.get(maxKey)));

  }
}

代码地址:MapStateMinMaxTempKeyedProcessFunction.java

验证代码

编写验证代码如下:

  private static void mapStateMinMaxUserTemperature() throws Exception {
    StreamExecutionEnvironment env = StreamExecutionEnvironment.getExecutionEnvironment();
    env.setParallelism(1);
    DataStream<Tuple2<String, Integer>> inputStream = env.fromElements(
        new Tuple2<>("zhangsan",36),
        new Tuple2<>("lisi",37),
        new Tuple2<>("zhangsan",35),
        new Tuple2<>("lisi",38),
        new Tuple2<>("wangwu",37)
    );

    inputStream.keyBy(new KeySelector<Tuple2<String, Integer>, String>() {
          @Override
          public String getKey(Tuple2<String, Integer> integerLongTuple2) throws Exception {
            return integerLongTuple2.f0;
          }
        })
        .process(new MapStateMinMaxTempKeyedProcessFunction())
        .print();

    env.execute("mapStateMinMaxUserTemperature");
  }

代码地址:mapStateMinMaxUserTemperature

程序运行输出

(zhangsan,36,36)
(lisi,37,37)
(zhangsan,35,36)
(lisi,37,38)
(wangwu,37,37)

可以看到对于每一条用户记录,正确计算输出了当前的最低和最高低温,达到了预期的计算需求。

ReducingState

ReducingState<T>这种State通过用户传入的ReduceFucntion,每次调用add(T)方法添加元素时,会调用ReduceFucntion,最后合并到一个单一的状态值,因此,ReducingState需要指定ReduceFucntion完成状态数据的聚合。

ReducingState获取元素使用T get()方法。

接下来我们通过一个具体的实例介绍如何通过ReducingState<T> 计算每个人的当前最高体温。

定义ReduceFunction

实现 reduce方法,求当前最大值

        new ReduceFunction<Integer>() {
          @Override
          public Integer reduce(Integer integer, Integer t1) throws Exception {
            return integer > t1 ? integer : t1;
          }
        }

实现KeyedProcessFunction

  • 声明ReducingState<Integer> tempState 用于记录用户当前的最高体温;
  • open(Configuration parameters)中利用 getRuntimeContext().getReducingState 对 tempState进行初始化,getReducingState需要传入ReducingStateDescriptorReducingStateDescriptor构建的时候,第二个参数要参入ReduceFunction;
  • processElement中,将当前用户的当前体温传入到 tempState中(tempState在内部会调用ReduceFunction), 然后从tempState 获取用户目前为止的最高提问,和用户名一起放入到Collector
import org.apache.flink.api.common.functions.ReduceFunction;
import org.apache.flink.api.common.state.ReducingState;
import org.apache.flink.api.common.state.ReducingStateDescriptor;
import org.apache.flink.api.java.tuple.Tuple2;
import org.apache.flink.configuration.Configuration;
import org.apache.flink.streaming.api.functions.KeyedProcessFunction;
import org.apache.flink.util.Collector;

public class ReducingStateKeyedProcessFunction
extends KeyedProcessFunction<String, Tuple2<String, Integer> , Tuple2<String, Integer>> {

  private ReducingState<Integer> tempState;

  @Override
  public void open(Configuration parameters) throws Exception {
    // super.open(parameters);
    tempState = getRuntimeContext().getReducingState(new ReducingStateDescriptor<Integer>("reduce",
        new ReduceFunction<Integer>() {
          @Override
          public Integer reduce(Integer integer, Integer t1) throws Exception {
            return integer > t1 ? integer : t1;
          }
        }, Integer.class));
  }

  @Override
  public void processElement(
      Tuple2<String, Integer> stringIntegerTuple2,
      KeyedProcessFunction<String, Tuple2<String, Integer>, Tuple2<String, Integer>>.Context context,
      Collector<Tuple2<String, Integer>> collector) throws Exception {

    tempState.add(stringIntegerTuple2.f1);
    collector.collect(Tuple2.of(stringIntegerTuple2.f0, tempState.get()));

  }
}

代码地址:ReducingStateKeyedProcessFunction.java

验证代码

  private static void reducingStateUserTemperature() throws Exception {
    StreamExecutionEnvironment env = StreamExecutionEnvironment.getExecutionEnvironment();
    env.setParallelism(1);
    DataStream<Tuple2<String, Integer>> inputStream = env.fromElements(
        new Tuple2<>("zhangsan",36),
        new Tuple2<>("lisi",37),
        new Tuple2<>("zhangsan",35),
        new Tuple2<>("lisi",38),
        new Tuple2<>("wangwu",37)
    );

    inputStream.keyBy(new KeySelector<Tuple2<String, Integer>, String>() {
          @Override
          public String getKey(Tuple2<String, Integer> integerLongTuple2) throws Exception {
            return integerLongTuple2.f0;
          }
        })
        .process(new ReducingStateKeyedProcessFunction())
        .print();

    env.execute("reducingStateUserTemperature");
  }

代码地址:reducingStateUserTemperature

程序运行结果如下

(zhangsan,36)
(lisi,37)
(zhangsan,36)
(lisi,38)
(wangwu,37)

达到预期效果

AggregatingState

AggregatingState<IN,OUT> 聚合State和ReducingState<T>非常类似,不同的是,这里聚合的类型可以是不同的元素类型,使用add(IN)来加入元素,并使用AggregateFunction函数计算聚合结果。

AggregatingState<IN,OUT>这种State通过用户传入的AggregateFunction,每次调用add(IN)方法添加元素时,会调用AggregateFunction,最后合并到一个单一的状态值,因此,ReducingState需要指定AggregateFunction完成状态数据的聚合。

定义 AggregateFunction

  • AggregateFunction<IN, ACC, OUT> 中的三个类型参数,分别对应
    • IN: 流元素的类型;
    • ACC: AggregateFunction内部累加器的类型
    • OUT: 最终输出的聚合结果的类型
  • AggregateFunction<IN, ACC, OUT> 需要重写 createAccumulator, add, getResult, merge这四个函数
    • createAccumulator 用于创建累加器Accumulator
    • add 用于将流元素加入到累加器Accumulator
    • getResult 用于从累加器Accumulator中计算获取聚合结果
    • merge 目前暂时用不到

代码如下

import org.apache.flink.api.common.functions.AggregateFunction;
import org.apache.flink.api.java.tuple.Tuple2;

public class AvgTempAggregateFunction
    implements AggregateFunction<Tuple2<String, Integer>, Tuple2<Integer, Integer>, Double> {
  @Override
  public Tuple2<Integer, Integer> createAccumulator() {
    return Tuple2.of(0, 0);
  }

  @Override
  public Tuple2<Integer, Integer> add(
      Tuple2<String, Integer> in, Tuple2<Integer, Integer> acc) {
      // in 流元素, acc 内部累加器
    acc.f0 = acc.f0 + in.f1;
    acc.f1 = acc.f1 + 1;
    return acc;
  }

  @Override
  public Double getResult(Tuple2<Integer, Integer> acc) {
    return new Double(acc.f0)/ acc.f1;
  }

  @Override
  public Tuple2<Integer, Integer> merge(
      Tuple2<Integer, Integer> a, Tuple2<Integer, Integer> b) {
    return new Tuple2<>(a.f0 + b.f0, a.f1 + b.f1);
  }
}

代码地址:AvgTempAggregateFunction.java

实现 KeyedProcesssFunction

  • 声明AggregatingState<Tuple2<String, Integer>, Double> avgState 用于记录用户当前的平均体温。其中第一个类型参数标识流元素类型,第二个类型参数标识聚合结果的类型;
  • open(Configuration parameters)中利用 getRuntimeContext().getAggregatingState 对 avgState进行初始化,getAggregatingState需要传入AggregatingStateDescriptorAggregatingStateDescriptor构建的时候,第二个参数要参入AggregateFunction,第三个参数要传入AggregateFunction中累加器的TypeInformation
  • processElement中,将当前用户的当前体温传入到 avgState中(avgState内部会调用add),然后从avgState 获取用户体温的聚合结果(avgState内部调用getResult ),和用户名一起放入到Collector
import org.apache.flink.api.common.state.AggregatingState;
import org.apache.flink.api.common.state.AggregatingStateDescriptor;
import org.apache.flink.api.common.typeinfo.TypeHint;
import org.apache.flink.api.common.typeinfo.TypeInformation;
import org.apache.flink.api.java.tuple.Tuple2;
import org.apache.flink.configuration.Configuration;
import org.apache.flink.streaming.api.functions.KeyedProcessFunction;
import org.apache.flink.util.Collector;

public class AggregatingStateKeyedProcessFunction
extends KeyedProcessFunction<String, Tuple2<String, Integer>, Tuple2<String, Double>>  {
  private AggregatingState<Tuple2<String, Integer>, Double> avgState;

  @Override
  public void open(Configuration parameters) throws Exception {
    // super.open(parameters);
    avgState = getRuntimeContext().getAggregatingState(new AggregatingStateDescriptor<Tuple2<String, Integer>,
        Tuple2<Integer, Integer>, Double>("aggregating", new AvgTempAggregateFunction(),
        TypeInformation.of(new TypeHint<Tuple2<Integer, Integer>>() {})));
  }

  @Override
  public void processElement(
      Tuple2<String, Integer> in,
      KeyedProcessFunction<String, Tuple2<String, Integer>, Tuple2<String, Double>>.Context context,
      Collector<Tuple2<String, Double>> collector) throws Exception {

    avgState.add(in);
    collector.collect(Tuple2.of(in.f0, avgState.get().doubleValue()));

  }
}

代码地址:AggregatingStateKeyedProcessFunction.java

验证代码

  private static void aggregatingStateAvgUserTemperature() throws Exception {
    StreamExecutionEnvironment env = StreamExecutionEnvironment.getExecutionEnvironment();
    env.setParallelism(1);
    DataStream<Tuple2<String, Integer>> inputStream = env.fromElements(
        new Tuple2<>("zhangsan",36),
        new Tuple2<>("lisi",37),
        new Tuple2<>("zhangsan",35),
        new Tuple2<>("lisi",38),
        new Tuple2<>("wangwu",37)
    );

    inputStream.keyBy(new KeySelector<Tuple2<String, Integer>, String>() {
          @Override
          public String getKey(Tuple2<String, Integer> integerLongTuple2) throws Exception {
            return integerLongTuple2.f0;
          }
        })
        .process(new AggregatingStateKeyedProcessFunction())
        .print();

    env.execute("aggregatingStateAvgUserTemperature");
  }

代码地址:aggregatingStateAvgUserTemperature,程序运行结果如下

(zhangsan,36.0)
(lisi,37.0)
(zhangsan,35.5)
(lisi,37.5)
(wangwu,37.0)

可以看到对于每一条流元素,输出了用户以及当前的平均体温,完成了预期功能需求。

总结

本文通过典型的场景用例,展示了不同数据结构的State在不同计算场景下的使用方法,帮助Flink开发者熟悉State相关API以及State在相应的计算场景中如何存储、查询相关的中间计算信息从而完成计算目的。

参考资料

  • 《Flink原理、实战与性能优化》5.1 有状态计算
  • 《Flink内核原理与实现》第7章 状态原理
  • Flink教程(17) Keyed State状态管理之AggregatingState使用案例 求平均值 https://blog.csdn.net/winterking3/article/details/115133519
  大数据 最新文章
实现Kafka至少消费一次
亚马逊云科技:还在苦于ETL?Zero ETL的时代
初探MapReduce
【SpringBoot框架篇】32.基于注解+redis实现
Elasticsearch:如何减少 Elasticsearch 集
Go redis操作
Redis面试题
专题五 Redis高并发场景
基于GBase8s和Calcite的多数据源查询
Redis——底层数据结构原理
上一篇文章      下一篇文章      查看所有文章
加:2022-03-17 22:14:33  更:2022-03-17 22:16:32 
 
开发: C++知识库 Java知识库 JavaScript Python PHP知识库 人工智能 区块链 大数据 移动开发 嵌入式 开发工具 数据结构与算法 开发测试 游戏开发 网络协议 系统运维
教程: HTML教程 CSS教程 JavaScript教程 Go语言教程 JQuery教程 VUE教程 VUE3教程 Bootstrap教程 SQL数据库教程 C语言教程 C++教程 Java教程 Python教程 Python3教程 C#教程
数码: 电脑 笔记本 显卡 显示器 固态硬盘 硬盘 耳机 手机 iphone vivo oppo 小米 华为 单反 装机 图拉丁

360图书馆 购物 三丰科技 阅读网 日历 万年历 2025年1日历 -2025/1/16 17:36:31-

图片自动播放器
↓图片自动播放器↓
TxT小说阅读器
↓语音阅读,小说下载,古典文学↓
一键清除垃圾
↓轻轻一点,清除系统垃圾↓
图片批量下载器
↓批量下载图片,美女图库↓
  网站联系: qq:121756557 email:121756557@qq.com  IT数码