Flink CDC Schema Evolution 详解

Flink CDC Schema Evolution 详解

github原文

glimpse

flink-cdc-3 glimpse

源码基于

~/project/flink_src/flink-cdc master !4 ❯ git remote -v
origin  https://github.com/apache/flink-cdc.git (fetch)
origin  https://github.com/apache/flink-cdc.git (push)
~/project/flink_src/flink-cdc master !4 ❯ git rev-parse HEAD
a5b666a3254b87b44b9a3843a4d001793e86552c
<revision>3.3-SNAPSHOT</revision>

flink-cdc 3.0 重要特性

  • 通过yaml文件定义pipeline
  • 能够感知schema变更

pipeline demo

我们使用一个特殊的sink类型“values”来观察各种事件的产生

values是专门为调试编写的一个sink,会将产生的事件打印在stdout

需要引入包flink-cdc-pipeline-connector-values-3.3-SNAPSHOT.jar,可以从flink-cdc工程中编译flink-cdc-pipeline-connector-values得到
在这里插入图片描述

pipeline yaml

################################################################################
# Description: Sync MySQL all tables to Doris
################################################################################
source:type: mysqlhostname: ${ip}port: ${port}username: ${username}password: ${password}tables: ${database}.${table}server-id: 5400-5404server-time-zone: UTC+8sink:type: valuesname: values Sinkpipeline:name: Sync Mysql Database to Valuesparallelism: 2

注意:parallelism > 1 时候一定要在flink中开启checkpoint

  1. 这是flink-cdc的已知bug,尚未解决 bug链接
  2. 如果想要使用flink-cdc源码调试,需要开启 --use-mini-cluster true
    请添加图片描述
  3. 并且修改FlinkPipelineComposer的ofMiniCluster方法,手动设置enableCheckpointing
    请添加图片描述

提交flink-cdc任务

./bin/flink-cdc.sh mysql-to-values.yaml
Pipeline has been submitted to cluster.
Job ID: a03966de35dc3141c890250daeac9699
Job Description: Sync Mysql Database to Values

在mysql中执行变更操作,观察flink taskmanager日志

mysql> insert into t1 values(13, 'm');
Query OK, 1 row affected (0.01 sec)mysql> alter table t1 add column c0 varchar(255);
Query OK, 0 rows affected (0.03 sec)
Records: 0  Duplicates: 0  Warnings: 0

flink日志

在这里插入图片描述

日志解析

注意看左侧的 “>”

由于yaml中设置的并发度是2,所有可以看到日志中有两个任务在打印

注意CreateTableEvent和AddColumnEvent这样的关于schema改变的事件会出现在两个并发中,而一个DataChangeEvent事件只会出现在单独一个并发中
在这里插入图片描述

flink-cdc 官方文档中描述: schema相关event与DataChangeEvent之间有如下的顺序保证

a CreateTableEvent must be emitted before any DataChangeEvent if a table is new to the framework, 
and SchemaChangeEvent must be emitted before any DataChangeEvent if the schema of a table is changed. 
This requirement makes sure that the framework has been aware of the schema before processing any data changes.

见understand-flink-cdc-api
在这里插入图片描述

schema evolution 实现原理

整体视角

在这里插入图片描述

SchemaRegistry运行在JobManager中,继承Coordinator与SchemaOperator交互,负责协调不同流水线中收到schema变更event后的同步

从yaml到pipeline的转化

  • 入口flink-cdc.sh
exec "$JAVA_RUN" -classpath "$CLASSPATH" "${LOG_SETTINGS[@]}" org.apache.flink.cdc.cli.CliFrontend "$@"
  • 入口类 CliFrontendCliFrontend.java
main 调用createExecutor 调用new CliExecutor 其中 pipelineDefPath 是yaml文件的路径
  • CliExecutor.java
1. 通过 YamlPipelineDefinitionParser 将 pipelineDefPath parse为pipelineDef
2. PipelineComposer 通过pipelineDef的定义调用flink的api构建流水线
  • FlinkPipelineComposer.java
// Build Source OperatorDataSourceTranslator sourceTranslator = new DataSourceTranslator();DataStream<Event> stream =sourceTranslator.translate(pipelineDef.getSource(), env, pipelineDef.getConfig(), parallelism);...// Schema operatorSchemaOperatorTranslator schemaOperatorTranslator =new SchemaOperatorTranslator(schemaChangeBehavior,pipelineDef.getConfig().get(PipelineOptions.PIPELINE_SCHEMA_OPERATOR_UID),pipelineDef.getConfig().get(PipelineOptions.PIPELINE_SCHEMA_OPERATOR_RPC_TIMEOUT));OperatorIDGenerator schemaOperatorIDGenerator =new OperatorIDGenerator(schemaOperatorTranslator.getSchemaOperatorUid());...// Build DataSink in advance as schema operator requires MetadataApplierDataSinkTranslator sinkTranslator = new DataSinkTranslator();DataSink dataSink =sinkTranslator.createDataSink(pipelineDef.getSink(), pipelineDef.getConfig(), env);stream =schemaOperatorTranslator.translate(stream,parallelism,dataSink.getMetadataApplier().setAcceptedSchemaEvolutionTypes(pipelineDef.getSink().getIncludedSchemaEvolutionTypes()),pipelineDef.getRoute());

这里可以看到从yaml的描述到stream的转化

stream 关联-> 当前 env 关联-> FlinkPipelineExecution
最终通过FlinkPipelineExecution.execute()调用用到env.executeAsync()

这里处理用户描述的source和sink节点,flink-cdc还自动插入了一个SchemaOperator节点
在这里插入图片描述

schema event的流动

SchemaOperator与sink绑定,这里绑定关系到之后的几个操作

  1. 定义一个sink的时候要提供MetadataApplier,运行在JobManager(上方),通过Rpc与SchemaOperator交互
        schemaOperatorTranslator.translate(...dataSink.getMetadataApplier()...);
  1. 所有的event都要经过SchemaOperator,SchemaOperator对于SchemaChangeEvent特殊处理 SchemaOperator.java
public void processElement(StreamRecord<Event> streamRecord)throws InterruptedException, TimeoutException, ExecutionException {Event event = streamRecord.getValue();if (event instanceof SchemaChangeEvent) {processSchemaChangeEvents((SchemaChangeEvent) event);} else if (event instanceof DataChangeEvent) {...}

最终调用到handleSchemaChangeEvent

private void handleSchemaChangeEvent(TableId tableId, SchemaChangeEvent schemaChangeEvent)throws InterruptedException, TimeoutException {...// The request will block if another schema change event is being handledSchemaChangeResponse response = requestSchemaChange(tableId, schemaChangeEvent);if (response.isAccepted()) {LOG.info("{}> Sending the FlushEvent for table {}.", subTaskId, tableId);output.collect(new StreamRecord<>(new FlushEvent(tableId)));...// The request will block until flushing finished in each sink writerSchemaChangeResultResponse schemaEvolveResponse = requestSchemaChangeResult();}

回想一下刚才在mysql中alter table add column的场景,每一个并发度都有一个AddColumnEvent,都会去调用
requestSchemaChange,向Coordinator发送SchemaChangeRequest

private SchemaChangeResponse requestSchemaChange(TableId tableId, SchemaChangeEvent schemaChangeEvent)throws InterruptedException, TimeoutException {...while (true) {SchemaChangeResponse response =sendRequestToCoordinator(new SchemaChangeRequest(tableId, schemaChangeEvent, subTaskId));...}}

SchemaRegistry.java响应请求

 public CompletableFuture<CoordinationResponse> handleCoordinationRequest(CoordinationRequest request) {...if (request instanceof SchemaChangeRequest) {SchemaChangeRequest schemaChangeRequest = (SchemaChangeRequest) request;requestHandler.handleSchemaChangeRequest(schemaChangeRequest, responseFuture);} else if (request instanceof SchemaChangeResultRequest) {requestHandler.getSchemaChangeResult(responseFuture);}...}

这时两个请求只有一个会被处理,另外一个会被认为是duplicate

处理的步骤如下

  • 发起schema变更请求requestSchemaChange

  • 如果被Coordinator Accept,执行output.collect(new StreamRecord<>(new FlushEvent(tableId)));

    • flushEvent在PrePartitionOperator.java被广播给下游所有的sink

      ```
      public void processElement(StreamRecord<Event> element) throws Exception {...if (event instanceof FlushEvent) {// Broadcast FlushEventbroadcastEvent(event);}...
      }
      ```
      
    • flushEvent在sink中会触发当前sink flush所有缓存的事件,之后通知Coordinator完成DataSinkFunctionOperator.java

      ```
      private void handleFlushEvent(FlushEvent event) throws Exception {userFunction.finish();schemaEvolutionClient.notifyFlushSuccess(getRuntimeContext().getIndexOfThisSubtask(), event.getTableId());
      }
      ```
      
  • hang在requestSchemaChangeResult,等待MetadataApplier变更下游数据库schema(比如Doris),天然hang住了上游消息

  • 如果不是第一个requestSchemaChange(相同请求已经在被处理),会hang在requestSchemaChange,也天然hang住上游消息,在Coordinator(SchemaRegistry/MetaAppier)处理好之后会走duplicate分支,只打印日志"{}> Schema change event {} has been handled in another subTask already."

  • 下游sink在处理完flush之后会触发notifyFlushSuccess,SchemaRegistry.java SchemaRegistry会调用handleEventFromOperator响应,最终调用到SchemaRegistryRequestHandler.java中的applySchemaChange, 调用对应sink的metadataApplier
    metadataApplier.applySchemaChange(changeEvent);

  • 上面步骤完成之后第一个hang住的requestSchemaChange会返回

MetadataApplier中干了什么

拿Doris举例, 直接去修改后端的列了,这时修改是安全的,因为上游的mysql修改schema之后产生的消息都被hang住,修改schema之前的消息都已经被各个sink flush消费完

DorisMetadataApplier.java

 public void applySchemaChange(SchemaChangeEvent event) {SchemaChangeEventVisitor.<Void, SchemaEvolveException>visit(event,addColumnEvent -> {applyAddColumnEvent(addColumnEvent);return null;},alterColumnTypeEvent -> {applyAlterColumnTypeEvent(alterColumnTypeEvent);return null;},createTableEvent -> {applyCreateTableEvent(createTableEvent);return null;},dropColumnEvent -> {applyDropColumnEvent(dropColumnEvent);return null;},...

glimpse 中没有说清楚的点

  1. schema变更消息会在每个并发度的源头都会产生吗?

回答:是的,只有这样SchemaOperator才有机会正确的hang住所有的并发度,并等待SchemaRegistry(MetadataApplier)的响应

更正回答:不是的,在values sink的stdout输出中我们可以看到对于schema变更消息,有两份输出,这时因为,在schemaOperator之后紧跟了一个PrePartition算子

其中的processElement 实现如下 在PrePartitionOperator.java

public void processElement(StreamRecord<Event> element) throws Exception {Event event = element.getValue();if (event instanceof SchemaChangeEvent) {// Update hash functionTableId tableId = ((SchemaChangeEvent) event).tableId();cachedHashFunctions.put(tableId, recreateHashFunction(tableId));// Broadcast SchemaChangeEventbroadcastEvent(event);} else if (event instanceof FlushEvent) {// Broadcast FlushEventbroadcastEvent(event);} else if (event instanceof DataChangeEvent) {// Partition DataChangeEvent by table ID and primary keyspartitionBy(((DataChangeEvent) event));}
}

可以看到对于SchemaChangeEvent和FlushEvent是向下游广播的,所以values sink中才会有多份打印

  1. 接上一个问题,如果多个source都有可能产生data change event,而一次schema变更只在一个source上产生,是如何保证正确性的呢?

简单想想一个有问题的场景,比如读取的binlog中消息1、2、3是datachange消息,s是schema change消息,发生的时间顺序是1 s 2 3,

但是由于并发度是2,这时有可能在s执行之前,2先执行了,这时就出现了逻辑问题,因为2是依赖变更后的schema, 2不可以先于s执行

┌─────────────┐                              ┌───────────────┐
│             │         ┌──┐    ┌──┐  ┌──┐   │               │
│   source1   ├─────────┼ 3┼────┼ s┼──┼ 1│──►│ schemaOprator │
│             │         └──┘    └──┘  └──┘   │               │
└─────────────┘                              └───────────────┘┌─────────────┐                              ┌───────────────┐
│             │               ┌───┐          │               │
│   source2   ├───────────────┼  2┼─────────►│ schemaOprator │
│             │               └───┘          │               │
└─────────────┘                              └───────────────┘

针对这个问题,flink-cdc的做法是,binlogsplit只有一个subTask在读取,可以看MySqlSourceEnumerator->MySqlSnapshotSplitAssigner->MySqlHybridSplitAssigner#getNext

@Overridepublic Optional<MySqlSplit> getNext() {if (AssignerStatus.isNewlyAddedAssigningSnapshotFinished(getAssignerStatus())) {// do not assign split until the adding table process finishedreturn Optional.empty();}if (snapshotSplitAssigner.noMoreSplits()) {// binlog split assigningif (isBinlogSplitAssigned) {// no more splits for the assignerreturn Optional.empty();} else if (AssignerStatus.isInitialAssigningFinished(snapshotSplitAssigner.getAssignerStatus())) {// we need to wait snapshot-assigner to be finished before// assigning the binlog split. Otherwise, records emitted from binlog split// might be out-of-order in terms of same primary key with snapshot splits.isBinlogSplitAssigned = true;return Optional.of(createBinlogSplit());} else if (AssignerStatus.isNewlyAddedAssigningFinished(snapshotSplitAssigner.getAssignerStatus())) {// do not need to create binlog, but send event to wake up the binlog readerisBinlogSplitAssigned = true;return Optional.empty();} else {// binlog split is not ready by nowreturn Optional.empty();}} else {// snapshot assigner still have remaining splits, assign split from itreturn snapshotSplitAssigner.getNext();}}

注意这里面isBinlogSplitAssigned如果被设置一次了,就不会再调用createBinlogSplit,所以在flink没有异常重启的情况下,createBinlogSplit只会被调用一次

也就是说,只有一个subTask能拿到binlogsplit

这很合理,因为在执行mysql语句show master status的时候,也只会返回一个活跃的binglog信息,作为消费者,只有单线程串行读才是正确行为

  1. 上面说的是通过binlog读取增量数据的逻辑,对于存量的大量数据,多source并发有意义吗?

回答:有意义。

首先需要明确source的实现架构 FLIP-27

Top level public interfaces这一小节描述了关键接口的含义,我们挑选出最重要的三个

Source - A factory style class that helps create SplitEnumerator and SourceReader at runtime.
...
SplitEnumerator - Discover the splits and assign them to the SourceReaders
...
SourceReader - Read the records from the splits assigned by the SplitEnumerator.

在这里插入图片描述

可以看到,SplitEnumerator运行在JobManager中,负责给reader分配split,这里面就包括snapShotSplit和binlogSplit

可以想象,在任务刚启动的时候,要拉取mysql的存量数据,这时是可以通过分隔主键区间,在多个任务上并行进行的

我们可以看MySqlSourceEnumerator的 splitAssigner,默认情况下是一个MySqlHybridSplitAssigner

一个assigner负责提供getNext方法,当reader需要split的时候会请求enumerator,enumerator会调用assigner的getNext方法枚举下一个split

上面看到在binlog的场景下只会枚举到1个split,第二个reader再来请求时就会返回空

这里hybrid的含义就是这个assigner工作在两种模式下,也就是两个阶段

  1. 存量数据读取阶段
  2. 增量数据读取阶段

在存量读取阶段,实际上是用了MySqlSnapshotSplitAssigner的能力

在MySqlSnapshotSplitAssigner.java的开头有一段描述

/*** A {@link MySqlSplitAssigner} that splits tables into small chunk splits based on primary key* range and chunk size.** @see MySqlSourceOptions#SCAN_INCREMENTAL_SNAPSHOT_CHUNK_SIZE*/

这个assigner会根据 primary key 切分出多个split,可以同时被多个reader执行

当存量数据读取完成,会进入到增量读取阶段,这时候只有一个source能够工作,其他的source实际上没有数据输入

阿里云上针对这个场景可以做到动态缩容关于MySQL CDC源表

全量阶段积累了大量历史数据,为了提高读取效率,通常采用并发的方式读取历史数据。而在Binlog增量阶段,因为Binlog数据量少且为了保证全局有序,通常只需要单并发读取。全量阶段和增量阶段对资源的不同需求,可以通过自动调优功能自动帮您实现性能和资源的平衡。自动调优会监控MySQL CDC Source的每个task的流量。当进入Binlog阶段,如果只有一个task在负责Binlog读取,其他task均空闲时,自动调优便会自动缩小Source的CU数和并发。开启自动调优只需要在作业运维页面,将自动调优的模式设置为Active模式。

另外可以开启多线程加速binlog解析,但是最终还是要存放到一个穿行队列中

MySQL连接器作为源表或数据摄入数据源使用时,在增量阶段会解析Binlog文件生成各种变更消息,Binlog文件使用二进制记录着所有表的变更,可以通过以下方式加速Binlog文件解析。开启并行解析和解析过滤配置使用配置项scan.only.deserialize.captured.tables.changelog.enabled:仅对指定表的变更事件进行解析。使用配置项scan.only.deserialize.captured.tables.changelog.enabled:采用多线程对Binlog文件进行解析、并按顺序投放到消费队列。优化Debezium参数debezium.max.queue.size: 162580
debezium.max.batch.size: 40960
debezium.poll.interval.ms: 50
debezium.max.queue.size:阻塞队列可以容纳的记录的最大数量。当Debezium从数据库读取事件流时,它会在将事件写入下游之前将它们放入阻塞队列。默认值为8192。debezium.max.batch.size:该连接器每次迭代处理的事件条数最大值。默认值为2048。debezium.poll.interval.ms:连接器应该在请求新的变更事件前等待多少毫秒。默认值为1000毫秒,即1秒。

另外在flink-cdc代码中也能看到,如果开启相关配置,会在进入binlog模式时关闭不需要的reader

在MySqlSourceEnumerator.java MySqlSourceEnumerator#assignSplits中

下面代码在BOUNDED模式并且开启scan.incremental.close-idle-reader.enabled才生效

if (shouldCloseIdleReader(nextAwaiting)) {// close idle readers when snapshot phase finished.context.signalNoMoreSplits(nextAwaiting);awaitingReader.remove();LOG.info("Close idle reader of subtask {}", nextAwaiting);continue;
}
同步原理

同步原理

MySQL binlog 数据同步的原理是,CDC source 会伪装成 MySQL 集群的一个 slave(使用指定的 server id 作为唯一 id),然后从 MySQL 拉取 binlog 数据。如果一个 MySQL 集群中有多个 slave 有同样的 id,就会导致拉取数据错乱的问题。

总结

flink-cdc 3.0 通过加入了SchemaOperator和MetadataApplier,监控链路上所有消息,当发生schema变更时,同步上下游

  1. hang住上游
  2. flush下游
  3. 修改下游schema
  4. 恢复运行

这样实现了自动schema变更

多并发会加速存量数据的同步,增量数据的读取还是只能通过一个并发

本文来自互联网用户投稿,该文观点仅代表作者本人,不代表本站立场。本站仅提供信息存储空间服务,不拥有所有权,不承担相关法律责任。如若转载,请注明出处:http://www.rhkb.cn/news/491257.html

如若内容造成侵权/违法违规/事实不符,请联系长河编程网进行投诉反馈email:809451989@qq.com,一经查实,立即删除!

相关文章

【Linux】结构化命令:if-then语句

结构化命令structured command&#xff1a;允许脚本根据条件跳过部分命令&#xff0c;改变执行流程。 1、if-then语句 格式1&#xff1a; if command then commands fi 格式2&#xff1a; if command; then commands fi 运行if之后的command命令&#xff0c;如果它的退出状态码…

23.DDD与微服务

学习视频来源&#xff1a;DDD独家秘籍视频合集 https://space.bilibili.com/24690212/channel/collectiondetail?sid1940048&ctype0 文章目录 DDD与微服务的关系1. DDD可以用微服务实现&#xff0c;也可以不用微服务实现2. DDD是微服务拆分的必须参考项之一3. 微服务架构…

IDEA 未启用lombok插件的Bug

项目中maven已引用了lombok依赖&#xff0c;之前运行没有问题的&#xff0c;但有时启动会提示&#xff1a; java: You arent using a compiler supported by lombok, so lombok will not work and has been disabled. Your processor is: com.sun.proxy.$Proxy8 Lombok support…

51c嵌入式~单片机~合集3

我自己的原文哦~ https://blog.51cto.com/whaosoft/12362395 一、STM32代码远程升级之IAP编程 IAP是什么 有时项目上需要远程升级单片机程序&#xff0c;此时需要接触到IAP编程。 IAP即为In Application Programming&#xff0c;解释为在应用中编程&#xff0c;用户自己的…

Jenkins容器使用宿主机Docker(五)

DevOps之安装和配置 Jenkins (一) DevOps 之 CI/CD入门操作 (二) Sonar Qube介绍和安装&#xff08;三&#xff09; Harbor镜像仓库介绍&安装 &#xff08;四&#xff09; Jenkins容器使用宿主机Docker&#xff08;五&#xff09; Jenkins流水线初体验&#xff08;六&#…

Python爬虫之Scrapy框架基础入门

Scrapy 是一个用于Python的开源网络爬虫框架&#xff0c;它为编写网络爬虫来抓取网站数据并提取结构化信息提供了一种高效的方法。Scrapy可以用于各种目的的数据抓取&#xff0c;如数据挖掘、监控和自动化测试等。 【1】安装 pip install scrapy安装成功如下所示&#xff1a;…

C++对象数组对象指针对象指针数组

一、对象数组 对象数组中的每一个元素都是同类的对象&#xff1b; 例1 对象数组成员的初始化 #include<iostream> using namespace std;class Student { public:Student( ){ };Student(int n,string nam,char s):num(n),name(nam),sex(s){};void display(){cout<&l…

git中的命令

目录 git的命令1.1 创建版本库1.2 添加文件和修改提交文件1.创建文件readme.txt, 此刻文件在工作区&#xff08;WorkSpace&#xff09;2.使用命令 git add readme.txt添加到暂存区里面去。3.提交文件到主分支4.修改文件&#xff0c;在文件中加入一行。查看git的状态 1.3 版本回…

《Keras3 minist 手写数字AI模型训练22秒精度达到:0.97》

《Keras3 minist 手写数字AI模型训练22秒精度达到&#xff1a;0.97》 一、修改源码加上如下两条代码二、源码修改如下三、Keras3 minist 训练22秒结束&#xff0c;训练过程截图四、Keras3 minist 源码截图 一、修改源码加上如下两条代码 import os os.environ["KERAS_BAC…

docker 容器相互访问

目前采用 network 方式 1. 创建自定义网络 docker network create network-group 如下 2. 相互访问的容器更改&#xff08;目前演示redis 以及netcore api 访问redis &#xff09; //redis 原有容器删除 跟之前区别就是加入 --network network-group docker run \ -p 6379:…

YOLOv8目标检测(三*)_最佳超参数训练

YOLOv8目标检测(一)_检测流程梳理&#xff1a;YOLOv8目标检测(一)_检测流程梳理_yolo检测流程-CSDN博客 YOLOv8目标检测(二)_准备数据集&#xff1a;YOLOv8目标检测(二)_准备数据集_yolov8 数据集准备-CSDN博客 YOLOv8目标检测(三)_训练模型&#xff1a;YOLOv8目标检测(三)_训…

uniapp使用百度地图配置了key,但是显示Map key not configured

搞了我两天的一个问题。 hbuilderx版本&#xff1a;4.36 问题介绍&#xff1a; 我的项目是公司的项目&#xff0c;需要在H5端使用百度地图&#xff0c;使用vue-cli创建的uniapp&#xff0c;就是uni代码在src里的目录结构。就是使用这种方式才会遇到这个问题。 问题原因&#xf…

Android通过okhttp下载文件(本文案例 下载mp4到本地,并更新到相册)

使用步骤分为两步 第一步导入 okhttp3 依赖 第二步调用本文提供的 utils 第一步这里不做说明了&#xff0c;直接提供第二步复制即用 DownloadUtil 中 download 为下载文件 参数说明 这里主要看你把 destFileName 下载文件名称定义为什么后缀&#xff0c;比如我定义为 .mp4 下…

【rust杂乱笔记】

code . 打开vscode fn main() {println!("hello world!") }loop{}循环; break跳出循环 // 引入三方库 use rand::Rng; // 引入标准库中的输入输出 use std::cmp::Ordering; use std::io;// main函数 先执行main函数 fn main() {// 打印的宏方法// 打印提示信息print…

【C++初阶】第9课—标准模板库STL(string_3)

文章目录 1. string类对象的查找操作1.1 c_str返回c格式字符串1.2 substr获取string类对象子串1.3 find和rfind查找字符串内容1.4 find_first_of与find_first_not_of1.5 find_last_of与find_last_not_of1.6 copy复制string类对象内的字符串内容1.7 compare对比string类字符串1.…

Android Studio、JDK、AGP、Gradle、kotlin-gradle-plugin 兼容性问题

文章目录 问题&#xff1a;解决办法&#xff1a;gradle与 java的版本兼容AGP与Gradle的版本兼容kotlin 与 jvm 的版本兼容KGP、Gradle、AGP兼容关系kotlin 与 java 的编译版本配置 问题&#xff1a; 你从githb上clone了一个项目&#xff0c;本地跑的时候&#xff0c;各种报错。…

Linux中 vim 常用命令大全详细讲解

文章目录 前言一、Vim 基本操作 &#x1f579;️1.1 打开或创建1.2 退出编辑1.3 模式切换 二、Vim 光标移动命令 ↕️2.1 基本移动2.2 行内移动2.3. 单词移动2.4. 页面移动2.5. 行跳转 三、Vim 文本编辑命令 &#x1f4cb;3.1 插入和删除3.2 复制、剪切与粘贴3.3 替换与修改 四…

ubuntu+ros新手笔记(二):古月·ROS2入门21讲学习笔记

系统ubuntu22.04 ros2 humble 按照如下视频教程学习的&#xff1a;【古月居】古月ROS2入门21讲 | 带你认识一个全新的机器人操作系统 此处仅记录我报错的地方&#xff0c;以及相应的解决方案&#xff0c;没有出错的略过&#xff01; 对应的古月居ROS2入门21讲源码下载地址&a…

.NET 技术 | 调用系统API创建Windows服务

01阅读须知 此文所提供的信息只为网络安全人员对自己所负责的网站、服务器等&#xff08;包括但不限于&#xff09;进行检测或维护参考&#xff0c;未经授权请勿利用文章中的技术资料对任何计算机系统进行入侵操作。利用此文所提供的信息而造成的直接或间接后果和损失&#xf…

win10配置子系统Ubuntu子系统(无需通过Windows应用市场)实际操作记录

win10配置子系统Ubuntu子系统&#xff08;无需通过Windows应用市场&#xff09;实际操作记录 参考教程 : win10配置子系统Ubuntu子系统&#xff08;无需通过Windows应用市场&#xff09; - 一佳一 - 博客园 开启虚拟机服务的 以管理员方式运行PowerShell运行命令。 &#xf…